From ac24f9108173aaee8ea21545b1147d343164e08c Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Sat, 5 Jan 2019 19:48:14 -0800 Subject: [PATCH 01/27] vreplication: playerPlan initial cut Signed-off-by: Sugu Sougoumarane --- .../{planbuilder.go => controller_plan.go} | 24 +- ...uilder_test.go => controller_plan_test.go} | 16 +- .../tabletmanager/vreplication/engine.go | 2 +- .../tabletmanager/vreplication/player_plan.go | 213 ++++++++++++++++++ .../tabletserver/vstreamer/planbuilder.go | 10 +- 5 files changed, 239 insertions(+), 26 deletions(-) rename go/vt/vttablet/tabletmanager/vreplication/{planbuilder.go => controller_plan.go} (88%) rename go/vt/vttablet/tabletmanager/vreplication/{planbuilder_test.go => controller_plan_test.go} (96%) create mode 100644 go/vt/vttablet/tabletmanager/vreplication/player_plan.go diff --git a/go/vt/vttablet/tabletmanager/vreplication/planbuilder.go b/go/vt/vttablet/tabletmanager/vreplication/controller_plan.go similarity index 88% rename from go/vt/vttablet/tabletmanager/vreplication/planbuilder.go rename to go/vt/vttablet/tabletmanager/vreplication/controller_plan.go index 22713df9478..1a964e7b02a 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/planbuilder.go +++ b/go/vt/vttablet/tabletmanager/vreplication/controller_plan.go @@ -23,8 +23,8 @@ import ( "vitess.io/vitess/go/vt/sqlparser" ) -// plan is the plan for vreplication control statements. -type plan struct { +// controllerPlan is the plan for vreplication control statements. +type controllerPlan struct { opcode int query string id int @@ -37,8 +37,8 @@ const ( selectQuery ) -// getPlan parses the input query and returns an appropriate plan. -func getPlan(query string) (*plan, error) { +// buildControllerPlan parses the input query and returns an appropriate plan. +func buildControllerPlan(query string) (*controllerPlan, error) { stmt, err := sqlparser.Parse(query) if err != nil { return nil, err @@ -57,7 +57,7 @@ func getPlan(query string) (*plan, error) { } } -func buildInsertPlan(ins *sqlparser.Insert) (*plan, error) { +func buildInsertPlan(ins *sqlparser.Insert) (*controllerPlan, error) { if ins.Action != sqlparser.InsertStr { return nil, fmt.Errorf("unsupported construct: %v", sqlparser.String(ins)) } @@ -99,13 +99,13 @@ func buildInsertPlan(ins *sqlparser.Insert) (*plan, error) { return nil, fmt.Errorf("id should not have a value: %v", sqlparser.String(ins)) } } - return &plan{ + return &controllerPlan{ opcode: insertQuery, query: sqlparser.String(ins), }, nil } -func buildUpdatePlan(upd *sqlparser.Update) (*plan, error) { +func buildUpdatePlan(upd *sqlparser.Update) (*controllerPlan, error) { if sqlparser.String(upd.TableExprs) != "_vt.vreplication" { return nil, fmt.Errorf("invalid table name: %v", sqlparser.String(upd.TableExprs)) } @@ -123,14 +123,14 @@ func buildUpdatePlan(upd *sqlparser.Update) (*plan, error) { return nil, err } - return &plan{ + return &controllerPlan{ opcode: updateQuery, query: sqlparser.String(upd), id: id, }, nil } -func buildDeletePlan(del *sqlparser.Delete) (*plan, error) { +func buildDeletePlan(del *sqlparser.Delete) (*controllerPlan, error) { if del.Targets != nil { return nil, fmt.Errorf("unsupported construct: %v", sqlparser.String(del)) } @@ -149,18 +149,18 @@ func buildDeletePlan(del *sqlparser.Delete) (*plan, error) { return nil, err } - return &plan{ + return &controllerPlan{ opcode: deleteQuery, query: sqlparser.String(del), id: id, }, nil } -func buildSelectPlan(sel *sqlparser.Select) (*plan, error) { +func buildSelectPlan(sel *sqlparser.Select) (*controllerPlan, error) { if sqlparser.String(sel.From) != "_vt.vreplication" { return nil, fmt.Errorf("invalid table name: %v", sqlparser.String(sel.From)) } - return &plan{ + return &controllerPlan{ opcode: selectQuery, query: sqlparser.String(sel), }, nil diff --git a/go/vt/vttablet/tabletmanager/vreplication/planbuilder_test.go b/go/vt/vttablet/tabletmanager/vreplication/controller_plan_test.go similarity index 96% rename from go/vt/vttablet/tabletmanager/vreplication/planbuilder_test.go rename to go/vt/vttablet/tabletmanager/vreplication/controller_plan_test.go index af89c949c65..6d9a7c54caf 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/planbuilder_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/controller_plan_test.go @@ -24,24 +24,24 @@ import ( func TestPlanBuilder(t *testing.T) { tcases := []struct { in string - plan *plan + plan *controllerPlan err string }{{ // Insert in: "insert into _vt.vreplication values(null)", - plan: &plan{ + plan: &controllerPlan{ opcode: insertQuery, query: "insert into _vt.vreplication values (null)", }, }, { in: "insert into _vt.vreplication(id) values(null)", - plan: &plan{ + plan: &controllerPlan{ opcode: insertQuery, query: "insert into _vt.vreplication(id) values (null)", }, }, { in: "insert into _vt.vreplication(workflow, id) values('', null)", - plan: &plan{ + plan: &controllerPlan{ opcode: insertQuery, query: "insert into _vt.vreplication(workflow, id) values ('', null)", }, @@ -79,7 +79,7 @@ func TestPlanBuilder(t *testing.T) { // Update }, { in: "update _vt.vreplication set state='Running' where id = 1", - plan: &plan{ + plan: &controllerPlan{ opcode: updateQuery, query: "update _vt.vreplication set state = 'Running' where id = 1", id: 1, @@ -115,7 +115,7 @@ func TestPlanBuilder(t *testing.T) { // Delete }, { in: "delete from _vt.vreplication where id = 1", - plan: &plan{ + plan: &controllerPlan{ opcode: deleteQuery, query: "delete from _vt.vreplication where id = 1", id: 1, @@ -154,7 +154,7 @@ func TestPlanBuilder(t *testing.T) { // Select }, { in: "select * from _vt.vreplication where id = 1", - plan: &plan{ + plan: &controllerPlan{ opcode: selectQuery, query: "select * from _vt.vreplication where id = 1", }, @@ -171,7 +171,7 @@ func TestPlanBuilder(t *testing.T) { err: "unsupported construct: set a = 1", }} for _, tcase := range tcases { - pl, err := getPlan(tcase.in) + pl, err := buildControllerPlan(tcase.in) if err != nil { if err.Error() != tcase.err { t.Errorf("getPlan(%v) error:\n%v, want\n%v", tcase.in, err, tcase.err) diff --git a/go/vt/vttablet/tabletmanager/vreplication/engine.go b/go/vt/vttablet/tabletmanager/vreplication/engine.go index c499225cf4c..f214876de88 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/engine.go +++ b/go/vt/vttablet/tabletmanager/vreplication/engine.go @@ -200,7 +200,7 @@ func (vre *Engine) Exec(query string) (*sqltypes.Result, error) { } defer vre.updateStats() - plan, err := getPlan(query) + plan, err := buildControllerPlan(query) if err != nil { return nil, err } diff --git a/go/vt/vttablet/tabletmanager/vreplication/player_plan.go b/go/vt/vttablet/tabletmanager/vreplication/player_plan.go new file mode 100644 index 00000000000..f08c59f7e8e --- /dev/null +++ b/go/vt/vttablet/tabletmanager/vreplication/player_plan.go @@ -0,0 +1,213 @@ +/* +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 vreplication + +import ( + "fmt" + "strings" + + "vitess.io/vitess/go/vt/sqlparser" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" +) + +type playerPlan struct { + vstreamFilter *binlogdatapb.Filter + tablePlans map[string]*tablePlan +} + +type tablePlan struct { + colExprs []*colExpr + onInsert insertType + updateCols []int +} + +func (tp *tablePlan) findCol(name sqlparser.ColIdent) *colExpr { + for _, cExpr := range tp.colExprs { + if cExpr.colname.Equal(name) { + return cExpr + } + } + return nil +} + +type colExpr struct { + colname sqlparser.ColIdent + colnum int + op operation +} + +type operation int + +const ( + opNone = operation(iota) + opCount + opSum + opExclude +) + +type insertType int + +const ( + insertNormal = insertType(iota) + insertOndup + insertIgnore +) + +func buildPlayerPlan(filter *binlogdatapb.Filter) (*playerPlan, error) { + plan := &playerPlan{ + vstreamFilter: &binlogdatapb.Filter{ + Rules: make([]*binlogdatapb.Rule, len(filter.Rules)), + }, + tablePlans: make(map[string]*tablePlan), + } + for i, rule := range filter.Rules { + if strings.HasPrefix(rule.Match, "/") { + plan.vstreamFilter.Rules[i] = rule + continue + } + sendRule, tplan, err := buildTablePlan(rule) + if err != nil { + return nil, err + } + if tplan == nil { + continue + } + plan.vstreamFilter.Rules[i] = sendRule + plan.tablePlans[rule.Match] = tplan + } + return plan, nil +} + +func buildTablePlan(rule *binlogdatapb.Rule) (*binlogdatapb.Rule, *tablePlan, error) { + statement, err := sqlparser.Parse(rule.Filter) + if err != nil { + return nil, nil, err + } + sel, ok := statement.(*sqlparser.Select) + if !ok { + return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(statement)) + } + if len(sel.From) > 1 { + return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(sel)) + } + node, ok := sel.From[0].(*sqlparser.AliasedTableExpr) + if !ok { + return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(sel)) + } + fromTable := sqlparser.GetTableName(node.Expr) + if fromTable.IsEmpty() { + return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(sel)) + } + if fromTable.String() != rule.Match { + return nil, nil, fmt.Errorf("unexpected: select expression table %v does not match the table entry name %s", sqlparser.String(fromTable), rule.Match) + } + + if _, ok := sel.SelectExprs[0].(*sqlparser.StarExpr); ok { + if len(sel.SelectExprs) != 1 { + return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(sel)) + } + return rule, nil, nil + } + + tplan := &tablePlan{} + sendSelect := &sqlparser.Select{ + From: sel.From, + Where: sel.Where, + } + for _, expr := range sel.SelectExprs { + selExpr, cExpr, err := analyzeExpr(expr) + if err != nil { + return nil, nil, err + } + if selExpr != nil { + sendSelect.SelectExprs = append(sendSelect.SelectExprs, selExpr) + cExpr.colnum = len(sendSelect.SelectExprs) - 1 + } + tplan.colExprs = append(tplan.colExprs, cExpr) + } + + if sel.GroupBy != nil { + if err := analyzeGroupBy(sel.GroupBy, tplan); err != nil { + return nil, nil, err + } + } + sendRule := &binlogdatapb.Rule{ + Match: rule.Match, + Filter: sqlparser.String(sendSelect), + } + return sendRule, tplan, nil +} + +func analyzeExpr(selExpr sqlparser.SelectExpr) (sqlparser.SelectExpr, *colExpr, error) { + aliased, ok := selExpr.(*sqlparser.AliasedExpr) + if !ok { + return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(selExpr)) + } + switch expr := aliased.Expr.(type) { + case *sqlparser.ColName: + return selExpr, &colExpr{colname: expr.Name}, nil + case *sqlparser.FuncExpr: + if expr.Distinct || len(expr.Exprs) != 1 { + return nil, nil, fmt.Errorf("unsupported: %v", sqlparser.String(expr)) + } + if aliased.As.IsEmpty() { + return nil, nil, fmt.Errorf("expression needs an alias: %v", sqlparser.String(expr)) + } + switch fname := expr.Name.Lowered(); fname { + case "month", "day", "hour": + return selExpr, &colExpr{colname: aliased.As}, nil + case "count": + if _, ok := expr.Exprs[0].(*sqlparser.StarExpr); !ok { + return nil, nil, fmt.Errorf("only count(*) is supported: %v", sqlparser.String(expr)) + } + return nil, &colExpr{colname: aliased.As, op: opCount}, nil + case "sum": + aInner, ok := expr.Exprs[0].(*sqlparser.AliasedExpr) + if !ok { + return nil, nil, fmt.Errorf("unsupported: %v", sqlparser.String(expr)) + } + innerCol, ok := aInner.Expr.(*sqlparser.ColName) + if !ok { + return nil, nil, fmt.Errorf("unsupported: %v", sqlparser.String(expr)) + } + return &sqlparser.AliasedExpr{Expr: innerCol}, &colExpr{colname: aliased.As, op: opSum}, nil + default: + return nil, nil, fmt.Errorf("unsupported: %v", sqlparser.String(expr)) + } + default: + return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(expr)) + } +} + +func analyzeGroupBy(groupBy sqlparser.GroupBy, tplan *tablePlan) error { + for _, expr := range groupBy { + colname, ok := expr.(*sqlparser.ColName) + if !ok { + return fmt.Errorf("unsupported: %v", sqlparser.String(expr)) + } + cExpr := tplan.findCol(colname.Name) + if cExpr == nil { + return fmt.Errorf("group by expression does not reference an alias in the select list: %v", sqlparser.String(expr)) + } + if cExpr.op != opNone { + return fmt.Errorf("group by expression is not allowed to reference an aggregate expression: %v", sqlparser.String(expr)) + } + cExpr.op = opExclude + } + return nil +} diff --git a/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go b/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go index d3709ca7361..432251f8cba 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go +++ b/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go @@ -293,12 +293,12 @@ func buildTablePlan(ti *Table, kschema *vindexes.KeyspaceSchema, query string) ( return plan, nil } -func analyzeExpr(ti *Table, expr sqlparser.SelectExpr) (cExpr ColExpr, err error) { - aexpr, ok := expr.(*sqlparser.AliasedExpr) +func analyzeExpr(ti *Table, selExpr sqlparser.SelectExpr) (cExpr ColExpr, err error) { + aliased, ok := selExpr.(*sqlparser.AliasedExpr) if !ok { - return ColExpr{}, fmt.Errorf("unexpected: %v", sqlparser.String(expr)) + return ColExpr{}, fmt.Errorf("unexpected: %v", sqlparser.String(selExpr)) } - switch expr := aexpr.Expr.(type) { + switch expr := aliased.Expr.(type) { case *sqlparser.ColName: colnum, err := findColumn(ti, expr.Name) if err != nil { @@ -319,7 +319,7 @@ func analyzeExpr(ti *Table, expr sqlparser.SelectExpr) (cExpr ColExpr, err error if !ok { return ColExpr{}, fmt.Errorf("unsupported: %v", sqlparser.String(expr)) } - as := aexpr.As + as := aliased.As if as.IsEmpty() { as = sqlparser.NewColIdent(sqlparser.String(expr)) } From 1f0f55e783a21f3befd4459db91d3c337557481e Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Sun, 6 Jan 2019 19:53:04 -0800 Subject: [PATCH 02/27] vreplication: vplayer WIP Signed-off-by: Sugu Sougoumarane --- go/vt/binlog/binlogplayer/binlog_player.go | 26 +- .../binlog/binlogplayer/binlog_player_test.go | 4 +- go/vt/proto/binlogdata/binlogdata.pb.go | 236 ++++++++++-------- go/vt/proto/queryservice/queryservice.pb.go | 134 +++++++--- go/vt/vtcombo/tablet_map.go | 7 + go/vt/vttablet/grpcqueryservice/server.go | 18 ++ go/vt/vttablet/grpctabletconn/conn.go | 45 ++++ go/vt/vttablet/queryservice/queryservice.go | 4 + go/vt/vttablet/queryservice/wrapped.go | 8 + go/vt/vttablet/sandboxconn/sandboxconn.go | 7 +- .../tabletconntest/fakequeryservice.go | 6 + .../tabletmanager/vreplication/controller.go | 2 +- .../tabletmanager/vreplication/player_plan.go | 26 +- .../tabletmanager/vreplication/vplayer.go | 236 ++++++++++++++++++ go/vt/vttablet/tabletserver/tabletserver.go | 2 +- .../vttablet/tabletserver/vstreamer/engine.go | 2 +- .../tabletserver/vstreamer/planbuilder.go | 10 +- .../tabletserver/vstreamer/vstreamer.go | 14 +- .../tabletserver/vstreamer/vstreamer_test.go | 46 ++-- proto/binlogdata.proto | 20 +- proto/queryservice.proto | 4 + py/vtproto/binlogdata_pb2.py | 136 ++++++---- py/vtproto/queryservice_pb2.py | 67 ++--- py/vtproto/queryservice_pb2_grpc.py | 18 ++ 24 files changed, 798 insertions(+), 280 deletions(-) create mode 100644 go/vt/vttablet/tabletmanager/vreplication/vplayer.go diff --git a/go/vt/binlog/binlogplayer/binlog_player.go b/go/vt/binlog/binlogplayer/binlog_player.go index b76fa52f42c..da81c6f1053 100644 --- a/go/vt/binlog/binlogplayer/binlog_player.go +++ b/go/vt/binlog/binlogplayer/binlog_player.go @@ -170,7 +170,7 @@ func NewBinlogPlayerTables(dbClient DBClient, tablet *topodatapb.Tablet, tables // If an error is encountered, it updates the vreplication state to "Error". // If a stop position was specifed, and reached, the state is updated to "Stopped". func (blp *BinlogPlayer) ApplyBinlogEvents(ctx context.Context) error { - if err := setVReplicationState(blp.dbClient, blp.uid, BlpRunning, ""); err != nil { + if err := SetVReplicationState(blp.dbClient, blp.uid, BlpRunning, ""); err != nil { log.Errorf("Error writing Running state: %v", err) } @@ -180,7 +180,7 @@ func (blp *BinlogPlayer) ApplyBinlogEvents(ctx context.Context) error { Time: time.Now(), Message: msg, }) - if err := setVReplicationState(blp.dbClient, blp.uid, BlpError, msg); err != nil { + if err := SetVReplicationState(blp.dbClient, blp.uid, BlpError, msg); err != nil { log.Errorf("Error writing stop state: %v", err) } return err @@ -191,7 +191,7 @@ func (blp *BinlogPlayer) ApplyBinlogEvents(ctx context.Context) error { // applyEvents returns a recordable status message on termination or an error otherwise. func (blp *BinlogPlayer) applyEvents(ctx context.Context) error { // Read starting values for vreplication. - pos, stopPos, maxTPS, maxReplicationLag, err := readVRSettings(blp.dbClient, blp.uid) + pos, stopPos, maxTPS, maxReplicationLag, err := ReadVRSettings(blp.dbClient, blp.uid) if err != nil { log.Error(err) return err @@ -244,14 +244,14 @@ func (blp *BinlogPlayer) applyEvents(ctx context.Context) error { case blp.position.Equal(blp.stopPosition): msg := fmt.Sprintf("not starting BinlogPlayer, we're already at the desired position %v", blp.stopPosition) log.Info(msg) - if err := setVReplicationState(blp.dbClient, blp.uid, BlpStopped, msg); err != nil { + if err := SetVReplicationState(blp.dbClient, blp.uid, BlpStopped, msg); err != nil { log.Errorf("Error writing stop state: %v", err) } return nil case blp.position.AtLeast(blp.stopPosition): msg := fmt.Sprintf("starting point %v greater than stopping point %v", blp.position, blp.stopPosition) log.Error(msg) - if err := setVReplicationState(blp.dbClient, blp.uid, BlpStopped, msg); err != nil { + if err := SetVReplicationState(blp.dbClient, blp.uid, BlpStopped, msg); err != nil { log.Errorf("Error writing stop state: %v", err) } // Don't return an error. Otherwise, it will keep retrying. @@ -351,7 +351,7 @@ func (blp *BinlogPlayer) applyEvents(ctx context.Context) error { if blp.position.AtLeast(blp.stopPosition) { msg := "Reached stopping position, done playing logs" log.Info(msg) - if err := setVReplicationState(blp.dbClient, blp.uid, BlpStopped, msg); err != nil { + if err := SetVReplicationState(blp.dbClient, blp.uid, BlpStopped, msg); err != nil { log.Errorf("Error writing stop state: %v", err) } return nil @@ -447,7 +447,7 @@ func (blp *BinlogPlayer) writeRecoveryPosition(tx *binlogdatapb.BinlogTransactio } now := time.Now().Unix() - updateRecovery := updateVReplicationPos(blp.uid, position, now, tx.EventToken.Timestamp) + updateRecovery := GenerateUpdatePos(blp.uid, position, now, tx.EventToken.Timestamp) qr, err := blp.exec(updateRecovery) if err != nil { @@ -503,8 +503,8 @@ func CreateVReplicationTable() []string { ) ENGINE=InnoDB`} } -// setVReplicationState updates the state in the _vt.vreplication table. -func setVReplicationState(dbClient DBClient, uid uint32, state, message string) error { +// SetVReplicationState updates the state in the _vt.vreplication table. +func SetVReplicationState(dbClient DBClient, uid uint32, state, message string) error { query := fmt.Sprintf("update _vt.vreplication set state='%v', message=%v where id=%v", state, encodeString(message), uid) if _, err := dbClient.ExecuteFetch(query, 1); err != nil { return fmt.Errorf("could not set state: %v: %v", query, err) @@ -512,9 +512,9 @@ func setVReplicationState(dbClient DBClient, uid uint32, state, message string) return nil } -// readVRSettings retrieves the throttler settings for +// ReadVRSettings retrieves the throttler settings for // vreplication from the checkpoint table. -func readVRSettings(dbClient DBClient, uid uint32) (pos, stopPos string, maxTPS, maxReplicationLag int64, err error) { +func ReadVRSettings(dbClient DBClient, uid uint32) (pos, stopPos string, maxTPS, maxReplicationLag int64, err error) { query := fmt.Sprintf("select pos, stop_pos, max_tps, max_replication_lag from _vt.vreplication where id=%v", uid) qr, err := dbClient.ExecuteFetch(query, 1) if err != nil { @@ -554,9 +554,9 @@ func CreateVReplicationStopped(workflow string, source *binlogdatapb.BinlogSourc encodeString(workflow), encodeString(source.String()), encodeString(position), throttler.MaxRateModuleDisabled, throttler.ReplicationLagModuleDisabled, time.Now().Unix(), BlpStopped) } -// updateVReplicationPos returns a statement to update a value in the +// GenerateUpdatePos returns a statement to update a value in the // _vt.vreplication table. -func updateVReplicationPos(uid uint32, pos mysql.Position, timeUpdated int64, txTimestamp int64) string { +func GenerateUpdatePos(uid uint32, pos mysql.Position, timeUpdated int64, txTimestamp int64) string { if txTimestamp != 0 { return fmt.Sprintf( "update _vt.vreplication set pos=%v, time_updated=%v, transaction_timestamp=%v where id=%v", diff --git a/go/vt/binlog/binlogplayer/binlog_player_test.go b/go/vt/binlog/binlogplayer/binlog_player_test.go index 8f4376f8212..8db24607dab 100644 --- a/go/vt/binlog/binlogplayer/binlog_player_test.go +++ b/go/vt/binlog/binlogplayer/binlog_player_test.go @@ -355,7 +355,7 @@ func TestUpdateVReplicationPos(t *testing.T) { "set pos='MariaDB/0-1-8283', time_updated=88822 " + "where id=78522" - got := updateVReplicationPos(78522, mysql.Position{GTIDSet: gtid.GTIDSet()}, 88822, 0) + got := GenerateUpdatePos(78522, mysql.Position{GTIDSet: gtid.GTIDSet()}, 88822, 0) if got != want { t.Errorf("updateVReplicationPos() = %#v, want %#v", got, want) } @@ -367,7 +367,7 @@ func TestUpdateVReplicationTimestamp(t *testing.T) { "set pos='MariaDB/0-2-582', time_updated=88822, transaction_timestamp=481828 " + "where id=78522" - got := updateVReplicationPos(78522, mysql.Position{GTIDSet: gtid.GTIDSet()}, 88822, 481828) + got := GenerateUpdatePos(78522, mysql.Position{GTIDSet: gtid.GTIDSet()}, 88822, 481828) if got != want { t.Errorf("updateVReplicationPos() = %#v, want %#v", got, want) } diff --git a/go/vt/proto/binlogdata/binlogdata.pb.go b/go/vt/proto/binlogdata/binlogdata.pb.go index 1a3510d1bc5..0cddb2d2342 100644 --- a/go/vt/proto/binlogdata/binlogdata.pb.go +++ b/go/vt/proto/binlogdata/binlogdata.pb.go @@ -8,6 +8,7 @@ import fmt "fmt" import math "math" import query "vitess.io/vitess/go/vt/proto/query" import topodata "vitess.io/vitess/go/vt/proto/topodata" +import vtrpc "vitess.io/vitess/go/vt/proto/vtrpc" // Reference imports to suppress errors if they are not otherwise used. var _ = proto.Marshal @@ -79,7 +80,7 @@ func (x VEventType) String() string { return proto.EnumName(VEventType_name, int32(x)) } func (VEventType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_e1edbb575eea20d0, []int{0} + return fileDescriptor_binlogdata_602eda7964a9642b, []int{0} } type BinlogTransaction_Statement_Category int32 @@ -127,7 +128,7 @@ func (x BinlogTransaction_Statement_Category) String() string { return proto.EnumName(BinlogTransaction_Statement_Category_name, int32(x)) } func (BinlogTransaction_Statement_Category) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_e1edbb575eea20d0, []int{1, 0, 0} + return fileDescriptor_binlogdata_602eda7964a9642b, []int{1, 0, 0} } // Charset is the per-statement charset info from a QUERY_EVENT binlog entry. @@ -147,7 +148,7 @@ func (m *Charset) Reset() { *m = Charset{} } func (m *Charset) String() string { return proto.CompactTextString(m) } func (*Charset) ProtoMessage() {} func (*Charset) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_e1edbb575eea20d0, []int{0} + return fileDescriptor_binlogdata_602eda7964a9642b, []int{0} } func (m *Charset) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Charset.Unmarshal(m, b) @@ -204,7 +205,7 @@ func (m *BinlogTransaction) Reset() { *m = BinlogTransaction{} } func (m *BinlogTransaction) String() string { return proto.CompactTextString(m) } func (*BinlogTransaction) ProtoMessage() {} func (*BinlogTransaction) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_e1edbb575eea20d0, []int{1} + return fileDescriptor_binlogdata_602eda7964a9642b, []int{1} } func (m *BinlogTransaction) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_BinlogTransaction.Unmarshal(m, b) @@ -254,7 +255,7 @@ func (m *BinlogTransaction_Statement) Reset() { *m = BinlogTransaction_S func (m *BinlogTransaction_Statement) String() string { return proto.CompactTextString(m) } func (*BinlogTransaction_Statement) ProtoMessage() {} func (*BinlogTransaction_Statement) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_e1edbb575eea20d0, []int{1, 0} + return fileDescriptor_binlogdata_602eda7964a9642b, []int{1, 0} } func (m *BinlogTransaction_Statement) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_BinlogTransaction_Statement.Unmarshal(m, b) @@ -312,7 +313,7 @@ func (m *StreamKeyRangeRequest) Reset() { *m = StreamKeyRangeRequest{} } func (m *StreamKeyRangeRequest) String() string { return proto.CompactTextString(m) } func (*StreamKeyRangeRequest) ProtoMessage() {} func (*StreamKeyRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_e1edbb575eea20d0, []int{2} + return fileDescriptor_binlogdata_602eda7964a9642b, []int{2} } func (m *StreamKeyRangeRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_StreamKeyRangeRequest.Unmarshal(m, b) @@ -365,7 +366,7 @@ func (m *StreamKeyRangeResponse) Reset() { *m = StreamKeyRangeResponse{} func (m *StreamKeyRangeResponse) String() string { return proto.CompactTextString(m) } func (*StreamKeyRangeResponse) ProtoMessage() {} func (*StreamKeyRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_e1edbb575eea20d0, []int{3} + return fileDescriptor_binlogdata_602eda7964a9642b, []int{3} } func (m *StreamKeyRangeResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_StreamKeyRangeResponse.Unmarshal(m, b) @@ -409,7 +410,7 @@ func (m *StreamTablesRequest) Reset() { *m = StreamTablesRequest{} } func (m *StreamTablesRequest) String() string { return proto.CompactTextString(m) } func (*StreamTablesRequest) ProtoMessage() {} func (*StreamTablesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_e1edbb575eea20d0, []int{4} + return fileDescriptor_binlogdata_602eda7964a9642b, []int{4} } func (m *StreamTablesRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_StreamTablesRequest.Unmarshal(m, b) @@ -462,7 +463,7 @@ func (m *StreamTablesResponse) Reset() { *m = StreamTablesResponse{} } func (m *StreamTablesResponse) String() string { return proto.CompactTextString(m) } func (*StreamTablesResponse) ProtoMessage() {} func (*StreamTablesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_e1edbb575eea20d0, []int{5} + return fileDescriptor_binlogdata_602eda7964a9642b, []int{5} } func (m *StreamTablesResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_StreamTablesResponse.Unmarshal(m, b) @@ -507,7 +508,7 @@ func (m *Rule) Reset() { *m = Rule{} } func (m *Rule) String() string { return proto.CompactTextString(m) } func (*Rule) ProtoMessage() {} func (*Rule) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_e1edbb575eea20d0, []int{6} + return fileDescriptor_binlogdata_602eda7964a9642b, []int{6} } func (m *Rule) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Rule.Unmarshal(m, b) @@ -554,7 +555,7 @@ func (m *Filter) Reset() { *m = Filter{} } func (m *Filter) String() string { return proto.CompactTextString(m) } func (*Filter) ProtoMessage() {} func (*Filter) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_e1edbb575eea20d0, []int{7} + return fileDescriptor_binlogdata_602eda7964a9642b, []int{7} } func (m *Filter) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Filter.Unmarshal(m, b) @@ -607,7 +608,7 @@ func (m *BinlogSource) Reset() { *m = BinlogSource{} } func (m *BinlogSource) String() string { return proto.CompactTextString(m) } func (*BinlogSource) ProtoMessage() {} func (*BinlogSource) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_e1edbb575eea20d0, []int{8} + return fileDescriptor_binlogdata_602eda7964a9642b, []int{8} } func (m *BinlogSource) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_BinlogSource.Unmarshal(m, b) @@ -682,7 +683,7 @@ func (m *RowChange) Reset() { *m = RowChange{} } func (m *RowChange) String() string { return proto.CompactTextString(m) } func (*RowChange) ProtoMessage() {} func (*RowChange) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_e1edbb575eea20d0, []int{9} + return fileDescriptor_binlogdata_602eda7964a9642b, []int{9} } func (m *RowChange) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_RowChange.Unmarshal(m, b) @@ -729,7 +730,7 @@ func (m *RowEvent) Reset() { *m = RowEvent{} } func (m *RowEvent) String() string { return proto.CompactTextString(m) } func (*RowEvent) ProtoMessage() {} func (*RowEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_e1edbb575eea20d0, []int{10} + return fileDescriptor_binlogdata_602eda7964a9642b, []int{10} } func (m *RowEvent) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_RowEvent.Unmarshal(m, b) @@ -775,7 +776,7 @@ func (m *FieldEvent) Reset() { *m = FieldEvent{} } func (m *FieldEvent) String() string { return proto.CompactTextString(m) } func (*FieldEvent) ProtoMessage() {} func (*FieldEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_e1edbb575eea20d0, []int{11} + return fileDescriptor_binlogdata_602eda7964a9642b, []int{11} } func (m *FieldEvent) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_FieldEvent.Unmarshal(m, b) @@ -812,10 +813,11 @@ func (m *FieldEvent) GetFields() []*query.Field { // VEvent represents a vstream event type VEvent struct { Type VEventType `protobuf:"varint,1,opt,name=type,proto3,enum=binlogdata.VEventType" json:"type,omitempty"` - Gtid string `protobuf:"bytes,2,opt,name=gtid,proto3" json:"gtid,omitempty"` - Ddl string `protobuf:"bytes,3,opt,name=ddl,proto3" json:"ddl,omitempty"` - RowEvent *RowEvent `protobuf:"bytes,4,opt,name=row_event,json=rowEvent,proto3" json:"row_event,omitempty"` - FieldEvent *FieldEvent `protobuf:"bytes,5,opt,name=field_event,json=fieldEvent,proto3" json:"field_event,omitempty"` + Timestamp int64 `protobuf:"varint,2,opt,name=timestamp,proto3" json:"timestamp,omitempty"` + Gtid string `protobuf:"bytes,3,opt,name=gtid,proto3" json:"gtid,omitempty"` + Ddl string `protobuf:"bytes,4,opt,name=ddl,proto3" json:"ddl,omitempty"` + RowEvent *RowEvent `protobuf:"bytes,5,opt,name=row_event,json=rowEvent,proto3" json:"row_event,omitempty"` + FieldEvent *FieldEvent `protobuf:"bytes,6,opt,name=field_event,json=fieldEvent,proto3" json:"field_event,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -825,7 +827,7 @@ func (m *VEvent) Reset() { *m = VEvent{} } func (m *VEvent) String() string { return proto.CompactTextString(m) } func (*VEvent) ProtoMessage() {} func (*VEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_e1edbb575eea20d0, []int{12} + return fileDescriptor_binlogdata_602eda7964a9642b, []int{12} } func (m *VEvent) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_VEvent.Unmarshal(m, b) @@ -852,6 +854,13 @@ func (m *VEvent) GetType() VEventType { return VEventType_UNKNOWN } +func (m *VEvent) GetTimestamp() int64 { + if m != nil { + return m.Timestamp + } + return 0 +} + func (m *VEvent) GetGtid() string { if m != nil { return m.Gtid @@ -882,18 +891,21 @@ func (m *VEvent) GetFieldEvent() *FieldEvent { // VStreamRequest is the payload for VStream type VStreamRequest struct { - Position string `protobuf:"bytes,1,opt,name=position,proto3" json:"position,omitempty"` - Filter *Filter `protobuf:"bytes,2,opt,name=filter,proto3" json:"filter,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + EffectiveCallerId *vtrpc.CallerID `protobuf:"bytes,1,opt,name=effective_caller_id,json=effectiveCallerId,proto3" json:"effective_caller_id,omitempty"` + ImmediateCallerId *query.VTGateCallerID `protobuf:"bytes,2,opt,name=immediate_caller_id,json=immediateCallerId,proto3" json:"immediate_caller_id,omitempty"` + Target *query.Target `protobuf:"bytes,3,opt,name=target,proto3" json:"target,omitempty"` + Position string `protobuf:"bytes,4,opt,name=position,proto3" json:"position,omitempty"` + Filter *Filter `protobuf:"bytes,5,opt,name=filter,proto3" json:"filter,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *VStreamRequest) Reset() { *m = VStreamRequest{} } func (m *VStreamRequest) String() string { return proto.CompactTextString(m) } func (*VStreamRequest) ProtoMessage() {} func (*VStreamRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_e1edbb575eea20d0, []int{13} + return fileDescriptor_binlogdata_602eda7964a9642b, []int{13} } func (m *VStreamRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_VStreamRequest.Unmarshal(m, b) @@ -913,6 +925,27 @@ func (m *VStreamRequest) XXX_DiscardUnknown() { var xxx_messageInfo_VStreamRequest proto.InternalMessageInfo +func (m *VStreamRequest) GetEffectiveCallerId() *vtrpc.CallerID { + if m != nil { + return m.EffectiveCallerId + } + return nil +} + +func (m *VStreamRequest) GetImmediateCallerId() *query.VTGateCallerID { + if m != nil { + return m.ImmediateCallerId + } + return nil +} + +func (m *VStreamRequest) GetTarget() *query.Target { + if m != nil { + return m.Target + } + return nil +} + func (m *VStreamRequest) GetPosition() string { if m != nil { return m.Position @@ -929,7 +962,7 @@ func (m *VStreamRequest) GetFilter() *Filter { // VStreamResponse is the response from VStream type VStreamResponse struct { - Event []*VEvent `protobuf:"bytes,1,rep,name=event,proto3" json:"event,omitempty"` + Events []*VEvent `protobuf:"bytes,1,rep,name=events,proto3" json:"events,omitempty"` XXX_NoUnkeyedLiteral struct{} `json:"-"` XXX_unrecognized []byte `json:"-"` XXX_sizecache int32 `json:"-"` @@ -939,7 +972,7 @@ func (m *VStreamResponse) Reset() { *m = VStreamResponse{} } func (m *VStreamResponse) String() string { return proto.CompactTextString(m) } func (*VStreamResponse) ProtoMessage() {} func (*VStreamResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_e1edbb575eea20d0, []int{14} + return fileDescriptor_binlogdata_602eda7964a9642b, []int{14} } func (m *VStreamResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_VStreamResponse.Unmarshal(m, b) @@ -959,9 +992,9 @@ func (m *VStreamResponse) XXX_DiscardUnknown() { var xxx_messageInfo_VStreamResponse proto.InternalMessageInfo -func (m *VStreamResponse) GetEvent() []*VEvent { +func (m *VStreamResponse) GetEvents() []*VEvent { if m != nil { - return m.Event + return m.Events } return nil } @@ -987,72 +1020,79 @@ func init() { proto.RegisterEnum("binlogdata.BinlogTransaction_Statement_Category", BinlogTransaction_Statement_Category_name, BinlogTransaction_Statement_Category_value) } -func init() { proto.RegisterFile("binlogdata.proto", fileDescriptor_binlogdata_e1edbb575eea20d0) } - -var fileDescriptor_binlogdata_e1edbb575eea20d0 = []byte{ - // 1017 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x56, 0xdb, 0x6e, 0xdb, 0x46, - 0x13, 0x0e, 0x45, 0x8a, 0x22, 0x87, 0x8e, 0xbd, 0x5e, 0x1f, 0x20, 0x18, 0x08, 0x60, 0x10, 0x3f, - 0xfe, 0xb8, 0x06, 0x2a, 0xa5, 0xea, 0xe9, 0xa2, 0x57, 0x96, 0x44, 0xbb, 0x8a, 0x69, 0xc9, 0x59, - 0xd3, 0x49, 0x91, 0x1b, 0x82, 0x96, 0xd6, 0xb2, 0x60, 0x89, 0x94, 0xc9, 0x95, 0x5d, 0x3d, 0x47, - 0x9f, 0xa2, 0x7d, 0x90, 0xbc, 0x49, 0xef, 0xfa, 0x10, 0xc5, 0x1e, 0x48, 0x49, 0x0e, 0x90, 0xaa, - 0x17, 0xbd, 0x9b, 0xd3, 0x7e, 0x3b, 0xf3, 0xcd, 0x70, 0x87, 0x80, 0x6e, 0x46, 0xf1, 0x38, 0x19, - 0x0e, 0x22, 0x16, 0xd5, 0xa6, 0x69, 0xc2, 0x12, 0x0c, 0x0b, 0xcb, 0x81, 0xf3, 0x30, 0xa3, 0xe9, - 0x5c, 0x3a, 0x0e, 0x36, 0x59, 0x32, 0x4d, 0x16, 0x81, 0xee, 0x05, 0x54, 0x5a, 0x77, 0x51, 0x9a, - 0x51, 0x86, 0xf7, 0xc1, 0xec, 0x8f, 0x47, 0x34, 0x66, 0x55, 0xed, 0x50, 0x3b, 0x2a, 0x13, 0xa5, - 0x61, 0x0c, 0x46, 0x3f, 0x89, 0xe3, 0x6a, 0x49, 0x58, 0x85, 0xcc, 0x63, 0x33, 0x9a, 0x3e, 0xd2, - 0xb4, 0xaa, 0xcb, 0x58, 0xa9, 0xb9, 0x7f, 0xea, 0xb0, 0xdd, 0x14, 0x57, 0x07, 0x69, 0x14, 0x67, - 0x51, 0x9f, 0x8d, 0x92, 0x18, 0x9f, 0x01, 0x64, 0x2c, 0x62, 0x74, 0x42, 0x63, 0x96, 0x55, 0xb5, - 0x43, 0xfd, 0xc8, 0x69, 0xbc, 0xae, 0x2d, 0x25, 0xfd, 0xd9, 0x91, 0xda, 0x55, 0x1e, 0x4f, 0x96, - 0x8e, 0xe2, 0x06, 0x38, 0xf4, 0x91, 0xc6, 0x2c, 0x64, 0xc9, 0x3d, 0x8d, 0xab, 0xc6, 0xa1, 0x76, - 0xe4, 0x34, 0xb6, 0x6b, 0xb2, 0x40, 0x8f, 0x7b, 0x02, 0xee, 0x20, 0x40, 0x0b, 0xf9, 0xe0, 0x53, - 0x09, 0xec, 0x02, 0x0d, 0xfb, 0x60, 0xf5, 0x23, 0x46, 0x87, 0x49, 0x3a, 0x17, 0x65, 0x6e, 0x36, - 0xde, 0xac, 0x99, 0x48, 0xad, 0xa5, 0xce, 0x91, 0x02, 0x01, 0x7f, 0x0d, 0x95, 0xbe, 0x64, 0x4f, - 0xb0, 0xe3, 0x34, 0x76, 0x96, 0xc1, 0x14, 0xb1, 0x24, 0x8f, 0xc1, 0x08, 0xf4, 0xec, 0x61, 0x2c, - 0x28, 0xdb, 0x20, 0x5c, 0x74, 0x7f, 0xd7, 0xc0, 0xca, 0x71, 0xf1, 0x0e, 0x6c, 0x35, 0xfd, 0xf0, - 0xba, 0x4b, 0xbc, 0x56, 0xef, 0xac, 0xdb, 0xf9, 0xe8, 0xb5, 0xd1, 0x0b, 0xbc, 0x01, 0x56, 0xd3, - 0x0f, 0x9b, 0xde, 0x59, 0xa7, 0x8b, 0x34, 0xfc, 0x12, 0xec, 0xa6, 0x1f, 0xb6, 0x7a, 0x17, 0x17, - 0x9d, 0x00, 0x95, 0xf0, 0x16, 0x38, 0x4d, 0x3f, 0x24, 0x3d, 0xdf, 0x6f, 0x9e, 0xb4, 0xce, 0x91, - 0x8e, 0xf7, 0x60, 0xbb, 0xe9, 0x87, 0xed, 0x0b, 0x3f, 0x6c, 0x7b, 0x97, 0xc4, 0x6b, 0x9d, 0x04, - 0x5e, 0x1b, 0x19, 0x18, 0xc0, 0xe4, 0xe6, 0xb6, 0x8f, 0xca, 0x4a, 0xbe, 0xf2, 0x02, 0x64, 0x2a, - 0xb8, 0x4e, 0xf7, 0xca, 0x23, 0x01, 0xaa, 0x28, 0xf5, 0xfa, 0xb2, 0x7d, 0x12, 0x78, 0xc8, 0x52, - 0x6a, 0xdb, 0xf3, 0xbd, 0xc0, 0x43, 0xf6, 0x5b, 0xc3, 0x2a, 0x21, 0xfd, 0xad, 0x61, 0xe9, 0xc8, - 0x70, 0x7f, 0xd3, 0x60, 0xef, 0x8a, 0xa5, 0x34, 0x9a, 0x9c, 0xd3, 0x39, 0x89, 0xe2, 0x21, 0x25, - 0xf4, 0x61, 0x46, 0x33, 0x86, 0x0f, 0xc0, 0x9a, 0x26, 0xd9, 0x88, 0x73, 0x27, 0x08, 0xb6, 0x49, - 0xa1, 0xe3, 0x3a, 0xd8, 0xf7, 0x74, 0x1e, 0xa6, 0x3c, 0x5e, 0x11, 0x86, 0x6b, 0xc5, 0x40, 0x16, - 0x48, 0xd6, 0xbd, 0x92, 0x96, 0xf9, 0xd5, 0xff, 0x99, 0x5f, 0xf7, 0x16, 0xf6, 0x9f, 0x27, 0x95, - 0x4d, 0x93, 0x38, 0xa3, 0xd8, 0x07, 0x2c, 0x0f, 0x86, 0x6c, 0xd1, 0x5b, 0x91, 0x9f, 0xd3, 0x78, - 0xf5, 0xc5, 0x01, 0x20, 0xdb, 0x37, 0xcf, 0x4d, 0xee, 0xaf, 0xb0, 0x23, 0xef, 0x09, 0xa2, 0x9b, - 0x31, 0xcd, 0xd6, 0x29, 0x7d, 0x1f, 0x4c, 0x26, 0x82, 0xab, 0xa5, 0x43, 0xfd, 0xc8, 0x26, 0x4a, - 0xfb, 0xb7, 0x15, 0x0e, 0x60, 0x77, 0xf5, 0xe6, 0xff, 0xa4, 0xbe, 0xef, 0xc0, 0x20, 0xb3, 0x31, - 0xc5, 0xbb, 0x50, 0x9e, 0x44, 0xac, 0x7f, 0xa7, 0xaa, 0x91, 0x0a, 0x2f, 0xe5, 0x76, 0x34, 0x66, - 0x34, 0x15, 0x2d, 0xb4, 0x89, 0xd2, 0xdc, 0x37, 0x60, 0x9e, 0x0a, 0x09, 0xff, 0x1f, 0xca, 0xe9, - 0x8c, 0xd7, 0x2a, 0x3f, 0x75, 0xb4, 0x9c, 0x00, 0x07, 0x26, 0xd2, 0xed, 0xfe, 0xa5, 0xc1, 0x86, - 0x4c, 0xe8, 0x2a, 0x99, 0xa5, 0x7d, 0xca, 0x19, 0xbc, 0xa7, 0xf3, 0x6c, 0x1a, 0xf5, 0x69, 0xce, - 0x60, 0xae, 0xf3, 0x64, 0xb2, 0xbb, 0x28, 0x1d, 0xa8, 0x5b, 0xa5, 0x82, 0xbf, 0x07, 0x47, 0x30, - 0xc9, 0x42, 0x36, 0x9f, 0x52, 0xc1, 0xe1, 0x66, 0x63, 0x77, 0x31, 0x54, 0x82, 0x27, 0x16, 0xcc, - 0xa7, 0x94, 0x00, 0x2b, 0xe4, 0xd5, 0x49, 0x34, 0xd6, 0x98, 0xc4, 0x45, 0xff, 0xca, 0x2b, 0xfd, - 0x3b, 0x2e, 0xc8, 0x30, 0x15, 0xca, 0x52, 0xad, 0x92, 0x8e, 0x82, 0xa0, 0x77, 0x60, 0x93, 0xe4, - 0xa9, 0x75, 0x27, 0x00, 0x5d, 0x30, 0x6f, 0xe8, 0x6d, 0x92, 0x52, 0xd5, 0x25, 0x50, 0xaf, 0x18, - 0x49, 0x9e, 0x88, 0xf2, 0xe0, 0x43, 0x28, 0x47, 0xb7, 0x39, 0xd1, 0xab, 0x21, 0xd2, 0xe1, 0x46, - 0x60, 0x91, 0xe4, 0x49, 0xbc, 0x7c, 0xf8, 0x15, 0xc8, 0x0a, 0xc3, 0x38, 0x9a, 0xe4, 0xf4, 0xd9, - 0xc2, 0xd2, 0x8d, 0x26, 0x14, 0xff, 0x00, 0x4e, 0x9a, 0x3c, 0x85, 0x7d, 0x71, 0xbd, 0x1c, 0x43, - 0xa7, 0xb1, 0xb7, 0xd2, 0x9a, 0x3c, 0x39, 0x02, 0x69, 0x2e, 0x66, 0xee, 0x3b, 0x80, 0xd3, 0x11, - 0x1d, 0x0f, 0xd6, 0xba, 0xe4, 0x7f, 0x9c, 0x0e, 0x3a, 0x1e, 0xe4, 0xf8, 0x1b, 0x2a, 0x65, 0x81, - 0x40, 0x94, 0xcf, 0xfd, 0xa4, 0x81, 0xf9, 0x5e, 0xe2, 0x1d, 0x83, 0x21, 0x1a, 0x27, 0xdf, 0xe2, - 0xfd, 0xe5, 0x74, 0x64, 0x84, 0x68, 0x9d, 0x88, 0xe1, 0x8b, 0x68, 0xc8, 0x46, 0xf9, 0x00, 0x08, - 0x99, 0x3f, 0xa9, 0x83, 0x81, 0x7c, 0x52, 0x6d, 0xc2, 0x45, 0xfc, 0x0d, 0xd8, 0xbc, 0x4e, 0xb1, - 0x01, 0x54, 0x6b, 0x77, 0x9f, 0x55, 0x29, 0x80, 0x89, 0x95, 0xe6, 0xcc, 0xfd, 0x08, 0x8e, 0xc8, - 0x4c, 0x1d, 0x2a, 0x8b, 0x43, 0xfb, 0xab, 0x9d, 0xcc, 0x19, 0x20, 0x70, 0x5b, 0xc8, 0xee, 0x2f, - 0xb0, 0xf9, 0x5e, 0x7e, 0x8f, 0xeb, 0xbc, 0x01, 0xc7, 0x2b, 0x1f, 0xce, 0x97, 0x67, 0xe5, 0x27, - 0xd8, 0x2a, 0x90, 0xd5, 0x37, 0x7e, 0x04, 0x65, 0x99, 0x9f, 0xfc, 0xaa, 0xf0, 0xe7, 0x5c, 0x11, - 0x19, 0x70, 0xfc, 0x87, 0x06, 0xb0, 0x60, 0x0f, 0x3b, 0x50, 0xb9, 0xee, 0x9e, 0x77, 0x7b, 0x1f, - 0xba, 0xe8, 0x05, 0xb6, 0xc0, 0x38, 0x0b, 0x3a, 0x6d, 0xa4, 0x61, 0x1b, 0xca, 0x72, 0xad, 0x94, - 0xf8, 0x4e, 0x50, 0x3b, 0x45, 0xe7, 0x0b, 0xa7, 0x58, 0x28, 0x06, 0xae, 0x80, 0x5e, 0xac, 0x0d, - 0xb5, 0x27, 0x4c, 0x0e, 0x48, 0xbc, 0x4b, 0xff, 0xa4, 0xe5, 0xa1, 0x0a, 0x77, 0x14, 0x1b, 0x03, - 0xc0, 0xcc, 0xd7, 0x05, 0x3f, 0xc9, 0x97, 0x0c, 0xf0, 0x7b, 0x7a, 0xc1, 0xcf, 0x1e, 0x41, 0x0e, - 0xb7, 0x91, 0xde, 0x07, 0xb4, 0xc1, 0x6d, 0xa7, 0x1d, 0xcf, 0x6f, 0xa3, 0x97, 0xcd, 0xaf, 0x3e, - 0xbe, 0x7e, 0x1c, 0x31, 0x9a, 0x65, 0xb5, 0x51, 0x52, 0x97, 0x52, 0x7d, 0x98, 0xd4, 0x1f, 0x59, - 0x5d, 0xfc, 0xa1, 0xd4, 0x17, 0x55, 0xde, 0x98, 0xc2, 0xf2, 0xed, 0xdf, 0x01, 0x00, 0x00, 0xff, - 0xff, 0xda, 0xbf, 0xad, 0x34, 0xf0, 0x08, 0x00, 0x00, +func init() { proto.RegisterFile("binlogdata.proto", fileDescriptor_binlogdata_602eda7964a9642b) } + +var fileDescriptor_binlogdata_602eda7964a9642b = []byte{ + // 1123 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x56, 0xdd, 0x6e, 0xe2, 0x56, + 0x10, 0x5e, 0xb0, 0x31, 0x78, 0x9c, 0x1f, 0xe7, 0xe4, 0x47, 0x28, 0xea, 0x4a, 0x91, 0xd5, 0x76, + 0xd3, 0x48, 0x85, 0x2d, 0xfd, 0xbb, 0xaa, 0xaa, 0x00, 0x4e, 0xca, 0xc6, 0x81, 0xec, 0x89, 0x93, + 0x95, 0xf6, 0xc6, 0x72, 0xcc, 0x81, 0x58, 0x31, 0x36, 0xb1, 0x4f, 0x48, 0x79, 0x82, 0x3e, 0x40, + 0x9f, 0xa2, 0x7d, 0x90, 0x3e, 0x49, 0x7b, 0xd7, 0x87, 0xa8, 0xce, 0x8f, 0x0d, 0x64, 0xa5, 0x6e, + 0x7a, 0xd1, 0xbb, 0xf9, 0x3f, 0x33, 0xdf, 0x8c, 0x67, 0x0c, 0xe6, 0x4d, 0x18, 0x47, 0xc9, 0x78, + 0xe8, 0x53, 0xbf, 0x31, 0x4d, 0x13, 0x9a, 0x20, 0x58, 0x48, 0xf6, 0x8d, 0x19, 0x4d, 0xa7, 0x81, + 0x50, 0xec, 0x1b, 0xf7, 0x0f, 0x24, 0x9d, 0x4b, 0x66, 0x83, 0x26, 0xd3, 0x64, 0xe1, 0x65, 0x9d, + 0x43, 0xb5, 0x73, 0xeb, 0xa7, 0x19, 0xa1, 0x68, 0x0f, 0xb4, 0x20, 0x0a, 0x49, 0x4c, 0xeb, 0xa5, + 0x83, 0xd2, 0x61, 0x05, 0x4b, 0x0e, 0x21, 0x50, 0x83, 0x24, 0x8e, 0xeb, 0x65, 0x2e, 0xe5, 0x34, + 0xb3, 0xcd, 0x48, 0x3a, 0x23, 0x69, 0x5d, 0x11, 0xb6, 0x82, 0xb3, 0xfe, 0x52, 0x60, 0xab, 0xcd, + 0xf3, 0x70, 0x53, 0x3f, 0xce, 0xfc, 0x80, 0x86, 0x49, 0x8c, 0x4e, 0x01, 0x32, 0xea, 0x53, 0x32, + 0x21, 0x31, 0xcd, 0xea, 0xa5, 0x03, 0xe5, 0xd0, 0x68, 0xbd, 0x6a, 0x2c, 0x55, 0xf0, 0x81, 0x4b, + 0xe3, 0x32, 0xb7, 0xc7, 0x4b, 0xae, 0xa8, 0x05, 0x06, 0x99, 0x91, 0x98, 0x7a, 0x34, 0xb9, 0x23, + 0x71, 0x5d, 0x3d, 0x28, 0x1d, 0x1a, 0xad, 0xad, 0x86, 0x28, 0xd0, 0x66, 0x1a, 0x97, 0x29, 0x30, + 0x90, 0x82, 0xde, 0xff, 0xa3, 0x0c, 0x7a, 0x11, 0x0d, 0x39, 0x50, 0x0b, 0x7c, 0x4a, 0xc6, 0x49, + 0x3a, 0xe7, 0x65, 0x6e, 0xb4, 0x5e, 0x3f, 0x33, 0x91, 0x46, 0x47, 0xfa, 0xe1, 0x22, 0x02, 0xfa, + 0x12, 0xaa, 0x81, 0x40, 0x8f, 0xa3, 0x63, 0xb4, 0xb6, 0x97, 0x83, 0x49, 0x60, 0x71, 0x6e, 0x83, + 0x4c, 0x50, 0xb2, 0xfb, 0x88, 0x43, 0xb6, 0x86, 0x19, 0x69, 0xfd, 0x56, 0x82, 0x5a, 0x1e, 0x17, + 0x6d, 0xc3, 0x66, 0xdb, 0xf1, 0xae, 0xfa, 0xd8, 0xee, 0x0c, 0x4e, 0xfb, 0xbd, 0xf7, 0x76, 0xd7, + 0x7c, 0x81, 0xd6, 0xa0, 0xd6, 0x76, 0xbc, 0xb6, 0x7d, 0xda, 0xeb, 0x9b, 0x25, 0xb4, 0x0e, 0x7a, + 0xdb, 0xf1, 0x3a, 0x83, 0xf3, 0xf3, 0x9e, 0x6b, 0x96, 0xd1, 0x26, 0x18, 0x6d, 0xc7, 0xc3, 0x03, + 0xc7, 0x69, 0x1f, 0x77, 0xce, 0x4c, 0x05, 0xed, 0xc2, 0x56, 0xdb, 0xf1, 0xba, 0xe7, 0x8e, 0xd7, + 0xb5, 0x2f, 0xb0, 0xdd, 0x39, 0x76, 0xed, 0xae, 0xa9, 0x22, 0x00, 0x8d, 0x89, 0xbb, 0x8e, 0x59, + 0x91, 0xf4, 0xa5, 0xed, 0x9a, 0x9a, 0x0c, 0xd7, 0xeb, 0x5f, 0xda, 0xd8, 0x35, 0xab, 0x92, 0xbd, + 0xba, 0xe8, 0x1e, 0xbb, 0xb6, 0x59, 0x93, 0x6c, 0xd7, 0x76, 0x6c, 0xd7, 0x36, 0xf5, 0x37, 0x6a, + 0xad, 0x6c, 0x2a, 0x6f, 0xd4, 0x9a, 0x62, 0xaa, 0xd6, 0xaf, 0x25, 0xd8, 0xbd, 0xa4, 0x29, 0xf1, + 0x27, 0x67, 0x64, 0x8e, 0xfd, 0x78, 0x4c, 0x30, 0xb9, 0x7f, 0x20, 0x19, 0x45, 0xfb, 0x50, 0x9b, + 0x26, 0x59, 0xc8, 0xb0, 0xe3, 0x00, 0xeb, 0xb8, 0xe0, 0x51, 0x13, 0xf4, 0x3b, 0x32, 0xf7, 0x52, + 0x66, 0x2f, 0x01, 0x43, 0x8d, 0x62, 0x20, 0x8b, 0x48, 0xb5, 0x3b, 0x49, 0x2d, 0xe3, 0xab, 0x7c, + 0x1c, 0x5f, 0x6b, 0x04, 0x7b, 0x4f, 0x93, 0xca, 0xa6, 0x49, 0x9c, 0x11, 0xe4, 0x00, 0x12, 0x8e, + 0x1e, 0x5d, 0xf4, 0x96, 0xe7, 0x67, 0xb4, 0x5e, 0xfe, 0xeb, 0x00, 0xe0, 0xad, 0x9b, 0xa7, 0x22, + 0xeb, 0x67, 0xd8, 0x16, 0xef, 0xb8, 0xfe, 0x4d, 0x44, 0xb2, 0xe7, 0x94, 0xbe, 0x07, 0x1a, 0xe5, + 0xc6, 0xf5, 0xf2, 0x81, 0x72, 0xa8, 0x63, 0xc9, 0xfd, 0xd7, 0x0a, 0x87, 0xb0, 0xb3, 0xfa, 0xf2, + 0xff, 0x52, 0xdf, 0x37, 0xa0, 0xe2, 0x87, 0x88, 0xa0, 0x1d, 0xa8, 0x4c, 0x7c, 0x1a, 0xdc, 0xca, + 0x6a, 0x04, 0xc3, 0x4a, 0x19, 0x85, 0x11, 0x25, 0x29, 0x6f, 0xa1, 0x8e, 0x25, 0x67, 0xbd, 0x06, + 0xed, 0x84, 0x53, 0xe8, 0x73, 0xa8, 0xa4, 0x0f, 0xac, 0x56, 0xf1, 0xa9, 0x9b, 0xcb, 0x09, 0xb0, + 0xc0, 0x58, 0xa8, 0xad, 0xbf, 0x4b, 0xb0, 0x26, 0x12, 0xba, 0x4c, 0x1e, 0xd2, 0x80, 0x30, 0x04, + 0xef, 0xc8, 0x3c, 0x9b, 0xfa, 0x01, 0xc9, 0x11, 0xcc, 0x79, 0x96, 0x4c, 0x76, 0xeb, 0xa7, 0x43, + 0xf9, 0xaa, 0x60, 0xd0, 0xb7, 0x60, 0x70, 0x24, 0xa9, 0x47, 0xe7, 0x53, 0xc2, 0x31, 0xdc, 0x68, + 0xed, 0x2c, 0x86, 0x8a, 0xe3, 0x44, 0xdd, 0xf9, 0x94, 0x60, 0xa0, 0x05, 0xbd, 0x3a, 0x89, 0xea, + 0x33, 0x26, 0x71, 0xd1, 0xbf, 0xca, 0x4a, 0xff, 0x8e, 0x0a, 0x30, 0x34, 0x19, 0x65, 0xa9, 0x56, + 0x01, 0x47, 0x01, 0xd0, 0x5b, 0xd0, 0x71, 0xf2, 0xd8, 0xb9, 0xe5, 0x01, 0x2d, 0xd0, 0x6e, 0xc8, + 0x28, 0x49, 0x89, 0xec, 0x12, 0xc8, 0x2d, 0x86, 0x93, 0x47, 0x2c, 0x35, 0xe8, 0x00, 0x2a, 0xfe, + 0x28, 0x07, 0x7a, 0xd5, 0x44, 0x28, 0x2c, 0x1f, 0x6a, 0x38, 0x79, 0xe4, 0x9b, 0x0f, 0xbd, 0x04, + 0x51, 0xa1, 0x17, 0xfb, 0x93, 0x1c, 0x3e, 0x9d, 0x4b, 0xfa, 0xfe, 0x84, 0xa0, 0xef, 0xc0, 0x48, + 0x93, 0x47, 0x2f, 0xe0, 0xcf, 0x8b, 0x31, 0x34, 0x5a, 0xbb, 0x2b, 0xad, 0xc9, 0x93, 0xc3, 0x90, + 0xe6, 0x64, 0x66, 0xbd, 0x05, 0x38, 0x09, 0x49, 0x34, 0x7c, 0xd6, 0x23, 0x9f, 0x32, 0x38, 0x48, + 0x34, 0xcc, 0xe3, 0xaf, 0xc9, 0x94, 0x79, 0x04, 0x2c, 0x75, 0xd6, 0x9f, 0x25, 0xd0, 0xae, 0x45, + 0xbc, 0x23, 0x50, 0x79, 0xe3, 0xc4, 0x2e, 0xde, 0x5b, 0x4e, 0x47, 0x58, 0xf0, 0xd6, 0x71, 0x1b, + 0xf4, 0x09, 0xe8, 0x34, 0x9c, 0x90, 0x8c, 0xfa, 0x93, 0x29, 0x87, 0x44, 0xc1, 0x0b, 0x01, 0x3b, + 0x53, 0x63, 0x1a, 0x0e, 0xf9, 0x08, 0xe8, 0x98, 0xd3, 0x6c, 0xe1, 0x0e, 0x87, 0x11, 0x6f, 0xb0, + 0x8e, 0x19, 0x89, 0xbe, 0x02, 0x9d, 0xa1, 0xc0, 0xef, 0x43, 0xbd, 0xc2, 0x61, 0xdd, 0x79, 0x82, + 0x01, 0x7f, 0x16, 0xd7, 0xd2, 0x1c, 0xd7, 0xef, 0xc1, 0xe0, 0x79, 0x4b, 0x27, 0xd1, 0xe7, 0xbd, + 0xd5, 0x3e, 0xe7, 0xf8, 0x60, 0x18, 0x15, 0xb4, 0xf5, 0x4b, 0x19, 0x36, 0xae, 0xc5, 0xe7, 0x9a, + 0xaf, 0x88, 0x1f, 0x61, 0x9b, 0x8c, 0x46, 0x24, 0xa0, 0xe1, 0x8c, 0x78, 0x81, 0x1f, 0x45, 0x24, + 0xf5, 0xc2, 0xa1, 0x1c, 0x81, 0xcd, 0x86, 0x38, 0xdb, 0x1d, 0x2e, 0xef, 0x75, 0xf1, 0x56, 0x61, + 0x2b, 0x45, 0x43, 0x64, 0xc3, 0x76, 0x38, 0x99, 0x90, 0x61, 0xe8, 0xd3, 0xe5, 0x00, 0x62, 0x40, + 0x76, 0x25, 0xda, 0xd7, 0xee, 0xa9, 0x4f, 0xc9, 0x22, 0x4c, 0xe1, 0x51, 0x84, 0xf9, 0x8c, 0x8d, + 0x73, 0x3a, 0x2e, 0xb6, 0xce, 0xba, 0xf4, 0x74, 0xb9, 0x10, 0x4b, 0xe5, 0xca, 0x46, 0x53, 0x9f, + 0x6c, 0xb4, 0xc5, 0xe4, 0x57, 0x3e, 0x3a, 0xf9, 0x3f, 0xc0, 0x66, 0x01, 0x84, 0xdc, 0x58, 0x47, + 0xa0, 0x71, 0x3c, 0xf3, 0x25, 0x81, 0x3e, 0x6c, 0x3d, 0x96, 0x16, 0x47, 0xbf, 0x97, 0x00, 0x16, + 0xd3, 0x80, 0x0c, 0xa8, 0x5e, 0xf5, 0xcf, 0xfa, 0x83, 0x77, 0x7d, 0xf3, 0x05, 0xaa, 0x81, 0x7a, + 0xea, 0xf6, 0xba, 0x66, 0x09, 0xe9, 0x50, 0x11, 0x67, 0xb2, 0xcc, 0x6e, 0x9c, 0xbc, 0x91, 0x0a, + 0x3b, 0xa0, 0xc5, 0x81, 0x54, 0x51, 0x15, 0x94, 0xe2, 0x0c, 0xca, 0xbb, 0xa7, 0xb1, 0x80, 0xd8, + 0xbe, 0x70, 0x8e, 0x3b, 0xb6, 0x59, 0x65, 0x8a, 0xe2, 0x02, 0x02, 0x68, 0xf9, 0xf9, 0x63, 0x9e, + 0xec, 0x68, 0x02, 0x7b, 0x67, 0xe0, 0xfe, 0x64, 0x63, 0xd3, 0x60, 0x32, 0x3c, 0x78, 0x67, 0xae, + 0x31, 0xd9, 0x49, 0xcf, 0x76, 0xba, 0xe6, 0x7a, 0xfb, 0x8b, 0xf7, 0xaf, 0x66, 0x21, 0x25, 0x59, + 0xd6, 0x08, 0x93, 0xa6, 0xa0, 0x9a, 0xe3, 0xa4, 0x39, 0xa3, 0x4d, 0xfe, 0xc7, 0xd5, 0x5c, 0x94, + 0x79, 0xa3, 0x71, 0xc9, 0xd7, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, 0xcb, 0x74, 0x80, 0x55, 0xcd, + 0x09, 0x00, 0x00, } diff --git a/go/vt/proto/queryservice/queryservice.pb.go b/go/vt/proto/queryservice/queryservice.pb.go index dc4bf2f2ba8..7a0bce4f401 100644 --- a/go/vt/proto/queryservice/queryservice.pb.go +++ b/go/vt/proto/queryservice/queryservice.pb.go @@ -6,6 +6,7 @@ package queryservice // import "vitess.io/vitess/go/vt/proto/queryservice" import proto "github.com/golang/protobuf/proto" import fmt "fmt" import math "math" +import binlogdata "vitess.io/vitess/go/vt/proto/binlogdata" import query "vitess.io/vitess/go/vt/proto/query" import ( @@ -85,6 +86,8 @@ type QueryClient interface { StreamHealth(ctx context.Context, in *query.StreamHealthRequest, opts ...grpc.CallOption) (Query_StreamHealthClient, error) // UpdateStream asks the server to return a stream of the updates that have been applied to its database. UpdateStream(ctx context.Context, in *query.UpdateStreamRequest, opts ...grpc.CallOption) (Query_UpdateStreamClient, error) + // VStream streams vreplication events. + VStream(ctx context.Context, in *binlogdata.VStreamRequest, opts ...grpc.CallOption) (Query_VStreamClient, error) } type queryClient struct { @@ -376,6 +379,38 @@ func (x *queryUpdateStreamClient) Recv() (*query.UpdateStreamResponse, error) { return m, nil } +func (c *queryClient) VStream(ctx context.Context, in *binlogdata.VStreamRequest, opts ...grpc.CallOption) (Query_VStreamClient, error) { + stream, err := c.cc.NewStream(ctx, &_Query_serviceDesc.Streams[4], "/queryservice.Query/VStream", opts...) + if err != nil { + return nil, err + } + x := &queryVStreamClient{stream} + if err := x.ClientStream.SendMsg(in); err != nil { + return nil, err + } + if err := x.ClientStream.CloseSend(); err != nil { + return nil, err + } + return x, nil +} + +type Query_VStreamClient interface { + Recv() (*binlogdata.VStreamResponse, error) + grpc.ClientStream +} + +type queryVStreamClient struct { + grpc.ClientStream +} + +func (x *queryVStreamClient) Recv() (*binlogdata.VStreamResponse, error) { + m := new(binlogdata.VStreamResponse) + if err := x.ClientStream.RecvMsg(m); err != nil { + return nil, err + } + return m, nil +} + // QueryServer is the server API for Query service. type QueryServer interface { // Execute executes the specified SQL query (might be in a @@ -427,6 +462,8 @@ type QueryServer interface { StreamHealth(*query.StreamHealthRequest, Query_StreamHealthServer) error // UpdateStream asks the server to return a stream of the updates that have been applied to its database. UpdateStream(*query.UpdateStreamRequest, Query_UpdateStreamServer) error + // VStream streams vreplication events. + VStream(*binlogdata.VStreamRequest, Query_VStreamServer) error } func RegisterQueryServer(s *grpc.Server, srv QueryServer) { @@ -823,6 +860,27 @@ func (x *queryUpdateStreamServer) Send(m *query.UpdateStreamResponse) error { return x.ServerStream.SendMsg(m) } +func _Query_VStream_Handler(srv interface{}, stream grpc.ServerStream) error { + m := new(binlogdata.VStreamRequest) + if err := stream.RecvMsg(m); err != nil { + return err + } + return srv.(QueryServer).VStream(m, &queryVStreamServer{stream}) +} + +type Query_VStreamServer interface { + Send(*binlogdata.VStreamResponse) error + grpc.ServerStream +} + +type queryVStreamServer struct { + grpc.ServerStream +} + +func (x *queryVStreamServer) Send(m *binlogdata.VStreamResponse) error { + return x.ServerStream.SendMsg(m) +} + var _Query_serviceDesc = grpc.ServiceDesc{ ServiceName: "queryservice.Query", HandlerType: (*QueryServer)(nil), @@ -917,45 +975,51 @@ var _Query_serviceDesc = grpc.ServiceDesc{ Handler: _Query_UpdateStream_Handler, ServerStreams: true, }, + { + StreamName: "VStream", + Handler: _Query_VStream_Handler, + ServerStreams: true, + }, }, Metadata: "queryservice.proto", } -func init() { proto.RegisterFile("queryservice.proto", fileDescriptor_queryservice_bb9da33c43234b51) } +func init() { proto.RegisterFile("queryservice.proto", fileDescriptor_queryservice_17509881eb07629d) } -var fileDescriptor_queryservice_bb9da33c43234b51 = []byte{ - // 519 bytes of a gzipped FileDescriptorProto +var fileDescriptor_queryservice_17509881eb07629d = []byte{ + // 544 bytes of a gzipped FileDescriptorProto 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x7c, 0x95, 0xdf, 0x6b, 0xd4, 0x40, - 0x10, 0xc7, 0xf5, 0xa1, 0xad, 0x4c, 0xe3, 0xaf, 0xad, 0x55, 0x9b, 0xd6, 0xb6, 0xf6, 0x4d, 0x84, - 0x46, 0x54, 0x10, 0x0a, 0x3e, 0xf4, 0x82, 0xa2, 0x14, 0x7f, 0xdd, 0x59, 0x10, 0x1f, 0x84, 0x6d, - 0x6e, 0x38, 0x43, 0x73, 0x49, 0xba, 0xbb, 0x77, 0xe8, 0x5f, 0xe4, 0xbf, 0x29, 0x66, 0x33, 0x93, - 0xdd, 0xbd, 0xc4, 0xb7, 0xce, 0xf7, 0x3b, 0xf3, 0x61, 0x6e, 0xa7, 0x33, 0x01, 0x71, 0xb5, 0x40, - 0xf5, 0x5b, 0xa3, 0x5a, 0xe6, 0x19, 0x1e, 0xd7, 0xaa, 0x32, 0x95, 0x88, 0x5c, 0x2d, 0xde, 0x6c, - 0x22, 0x6b, 0x3d, 0xff, 0x13, 0xc1, 0xda, 0x97, 0x7f, 0xb1, 0x38, 0x81, 0x8d, 0x37, 0xbf, 0x30, - 0x5b, 0x18, 0x14, 0xdb, 0xc7, 0x36, 0xa5, 0x8d, 0xc7, 0x78, 0xb5, 0x40, 0x6d, 0xe2, 0xfb, 0xa1, - 0xac, 0xeb, 0xaa, 0xd4, 0x78, 0x74, 0x4d, 0xbc, 0x87, 0xa8, 0x15, 0x47, 0xd2, 0x64, 0x3f, 0x45, - 0xec, 0x67, 0x36, 0x22, 0x51, 0x76, 0x7b, 0x3d, 0x46, 0x7d, 0x84, 0x9b, 0x13, 0xa3, 0x50, 0xce, - 0xa9, 0x19, 0xca, 0xf7, 0x54, 0x82, 0xed, 0xf5, 0x9b, 0x44, 0x7b, 0x76, 0x5d, 0xbc, 0x84, 0xb5, - 0x11, 0xce, 0xf2, 0x52, 0x6c, 0xb5, 0xa9, 0x4d, 0x44, 0xf5, 0xf7, 0x7c, 0x91, 0xbb, 0x78, 0x05, - 0xeb, 0x69, 0x35, 0x9f, 0xe7, 0x46, 0x50, 0x86, 0x0d, 0xa9, 0x6e, 0x3b, 0x50, 0xb9, 0xf0, 0x35, - 0xdc, 0x18, 0x57, 0x45, 0x71, 0x21, 0xb3, 0x4b, 0x41, 0xef, 0x45, 0x02, 0x15, 0x3f, 0x58, 0xd1, - 0xb9, 0xfc, 0x04, 0x36, 0x3e, 0x2b, 0xac, 0xa5, 0xea, 0x86, 0xd0, 0xc6, 0xe1, 0x10, 0x58, 0xe6, - 0xda, 0x4f, 0x70, 0xcb, 0xb6, 0xd3, 0x5a, 0x53, 0xb1, 0xe7, 0x75, 0x49, 0x32, 0x91, 0x1e, 0x0d, - 0xb8, 0x0c, 0x3c, 0x87, 0x3b, 0xd4, 0x22, 0x23, 0xf7, 0x83, 0xde, 0x43, 0xe8, 0xc1, 0xa0, 0xcf, - 0xd8, 0x6f, 0x70, 0x37, 0x55, 0x28, 0x0d, 0x7e, 0x55, 0xb2, 0xd4, 0x32, 0x33, 0x79, 0x55, 0x0a, - 0xaa, 0x5b, 0x71, 0x08, 0x7c, 0x38, 0x9c, 0xc0, 0xe4, 0xb7, 0xb0, 0x39, 0x31, 0x52, 0x99, 0x76, - 0x74, 0x3b, 0xfc, 0xcf, 0xc1, 0x1a, 0xd1, 0xe2, 0x3e, 0xcb, 0xe3, 0xa0, 0xe1, 0x39, 0x32, 0xa7, - 0xd3, 0x56, 0x38, 0xae, 0xc5, 0x9c, 0x1f, 0xb0, 0x95, 0x56, 0x65, 0x56, 0x2c, 0xa6, 0xde, 0x6f, - 0x7d, 0xcc, 0x0f, 0xbf, 0xe2, 0x11, 0xf7, 0xe8, 0x7f, 0x29, 0xcc, 0x1f, 0xc3, 0xed, 0x31, 0xca, - 0xa9, 0xcb, 0xa6, 0xa1, 0x06, 0x3a, 0x71, 0xf7, 0x87, 0x6c, 0x77, 0x95, 0x9b, 0x65, 0xa0, 0xf5, - 0x8b, 0xdd, 0x0d, 0x09, 0xb6, 0x6f, 0xb7, 0xd7, 0x73, 0x07, 0xed, 0x3a, 0xf6, 0x34, 0x1c, 0xf4, - 0xd4, 0x78, 0xf7, 0xe1, 0x70, 0x38, 0xc1, 0x3d, 0x12, 0x1f, 0x50, 0x6b, 0x39, 0x43, 0xbb, 0xf8, - 0x7c, 0x24, 0x3c, 0x35, 0x3c, 0x12, 0x81, 0xe9, 0x1c, 0x89, 0x14, 0xa0, 0x35, 0x4f, 0xb3, 0x4b, - 0xf1, 0xd0, 0xcf, 0x3f, 0xed, 0xc6, 0xbd, 0xd3, 0xe3, 0x70, 0x53, 0x29, 0xc0, 0xa4, 0x2e, 0x72, - 0x63, 0xcf, 0x29, 0x41, 0x3a, 0x29, 0x84, 0xb8, 0x0e, 0x43, 0xce, 0x20, 0xb2, 0xfd, 0xbd, 0x43, - 0x59, 0x98, 0xee, 0x92, 0xba, 0x62, 0xf8, 0xfc, 0xbe, 0xe7, 0xfc, 0xac, 0x33, 0x88, 0xce, 0xeb, - 0xa9, 0x34, 0xf4, 0x4a, 0x04, 0x73, 0xc5, 0x10, 0xe6, 0x7b, 0x1d, 0x6c, 0xf4, 0xf4, 0xfb, 0x93, - 0x65, 0x6e, 0x50, 0xeb, 0xe3, 0xbc, 0x4a, 0xec, 0x5f, 0xc9, 0xac, 0x4a, 0x96, 0x26, 0x69, 0xbe, - 0x24, 0x89, 0xfb, 0x8d, 0xb9, 0x58, 0x6f, 0xb4, 0x17, 0x7f, 0x03, 0x00, 0x00, 0xff, 0xff, 0x51, - 0x5a, 0xbc, 0xc0, 0x8e, 0x06, 0x00, 0x00, + 0x10, 0xc7, 0xf5, 0xa1, 0xad, 0x4c, 0x4f, 0xad, 0x5b, 0xab, 0x36, 0xad, 0x6d, 0xed, 0x9b, 0x08, + 0x17, 0x51, 0x41, 0x28, 0xf8, 0xd0, 0x0b, 0x16, 0xa5, 0xf8, 0xeb, 0xce, 0x16, 0xf1, 0x41, 0xd8, + 0x4b, 0x86, 0x33, 0x34, 0x97, 0x4d, 0x93, 0xbd, 0x43, 0xff, 0x6a, 0xff, 0x05, 0x31, 0x9b, 0x99, + 0xec, 0xee, 0x25, 0xbe, 0xdd, 0x7e, 0xbf, 0x33, 0x1f, 0x26, 0x3b, 0x37, 0xb3, 0x20, 0xae, 0x17, + 0x58, 0xfe, 0xae, 0xb0, 0x5c, 0xa6, 0x31, 0x0e, 0x8b, 0x52, 0x69, 0x25, 0x06, 0xb6, 0x16, 0x6c, + 0xd6, 0x27, 0x63, 0x05, 0x5b, 0xd3, 0x34, 0xcf, 0xd4, 0x2c, 0x91, 0x5a, 0x1a, 0xe5, 0xc5, 0x9f, + 0x01, 0xac, 0x7d, 0xf9, 0x17, 0x21, 0x4e, 0x60, 0xe3, 0xed, 0x2f, 0x8c, 0x17, 0x1a, 0xc5, 0xce, + 0xd0, 0x24, 0x35, 0xe7, 0x31, 0x5e, 0x2f, 0xb0, 0xd2, 0xc1, 0x03, 0x5f, 0xae, 0x0a, 0x95, 0x57, + 0x78, 0x7c, 0x43, 0xbc, 0x87, 0x41, 0x23, 0x8e, 0xa4, 0x8e, 0x7f, 0x8a, 0xc0, 0x8d, 0xac, 0x45, + 0xa2, 0xec, 0x75, 0x7a, 0x8c, 0xfa, 0x08, 0xb7, 0x27, 0xba, 0x44, 0x39, 0xa7, 0x62, 0x28, 0xde, + 0x51, 0x09, 0xb6, 0xdf, 0x6d, 0x12, 0xed, 0xf9, 0x4d, 0xf1, 0x0a, 0xd6, 0x46, 0x38, 0x4b, 0x73, + 0xb1, 0xdd, 0x84, 0xd6, 0x27, 0xca, 0xbf, 0xef, 0x8a, 0x5c, 0xc5, 0x6b, 0x58, 0x8f, 0xd4, 0x7c, + 0x9e, 0x6a, 0x41, 0x11, 0xe6, 0x48, 0x79, 0x3b, 0x9e, 0xca, 0x89, 0x6f, 0xe0, 0xd6, 0x58, 0x65, + 0xd9, 0x54, 0xc6, 0x57, 0x82, 0xee, 0x8b, 0x04, 0x4a, 0x7e, 0xb8, 0xa2, 0x73, 0xfa, 0x09, 0x6c, + 0x7c, 0x2e, 0xb1, 0x90, 0x65, 0xdb, 0x84, 0xe6, 0xec, 0x37, 0x81, 0x65, 0xce, 0xfd, 0x04, 0x77, + 0x4c, 0x39, 0x8d, 0x95, 0x88, 0x7d, 0xa7, 0x4a, 0x92, 0x89, 0xf4, 0xb8, 0xc7, 0x65, 0xe0, 0x05, + 0x6c, 0x51, 0x89, 0x8c, 0x3c, 0xf0, 0x6a, 0xf7, 0xa1, 0x87, 0xbd, 0x3e, 0x63, 0xbf, 0xc1, 0xbd, + 0xa8, 0x44, 0xa9, 0xf1, 0x6b, 0x29, 0xf3, 0x4a, 0xc6, 0x3a, 0x55, 0xb9, 0xa0, 0xbc, 0x15, 0x87, + 0xc0, 0x47, 0xfd, 0x01, 0x4c, 0x3e, 0x83, 0xcd, 0x89, 0x96, 0xa5, 0x6e, 0x5a, 0xb7, 0xcb, 0x7f, + 0x0e, 0xd6, 0x88, 0x16, 0x74, 0x59, 0x0e, 0x07, 0x35, 0xf7, 0x91, 0x39, 0xad, 0xb6, 0xc2, 0xb1, + 0x2d, 0xe6, 0xfc, 0x80, 0xed, 0x48, 0xe5, 0x71, 0xb6, 0x48, 0x9c, 0x6f, 0x7d, 0xc2, 0x17, 0xbf, + 0xe2, 0x11, 0xf7, 0xf8, 0x7f, 0x21, 0xcc, 0x1f, 0xc3, 0xdd, 0x31, 0xca, 0xc4, 0x66, 0x53, 0x53, + 0x3d, 0x9d, 0xb8, 0x07, 0x7d, 0xb6, 0x3d, 0xca, 0xf5, 0x30, 0xd0, 0xf8, 0x05, 0xf6, 0x84, 0x78, + 0xd3, 0xb7, 0xd7, 0xe9, 0xd9, 0x8d, 0xb6, 0x1d, 0xb3, 0x1a, 0x0e, 0x3b, 0x72, 0x9c, 0xfd, 0x70, + 0xd4, 0x1f, 0x60, 0x2f, 0x89, 0x0f, 0x58, 0x55, 0x72, 0x86, 0x66, 0xf0, 0x79, 0x49, 0x38, 0xaa, + 0xbf, 0x24, 0x3c, 0xd3, 0x5a, 0x12, 0x11, 0x40, 0x63, 0x9e, 0xc6, 0x57, 0xe2, 0x91, 0x1b, 0x7f, + 0xda, 0xb6, 0x7b, 0xb7, 0xc3, 0xe1, 0xa2, 0x22, 0x80, 0x49, 0x91, 0xa5, 0xda, 0xac, 0x53, 0x82, + 0xb4, 0x92, 0x0f, 0xb1, 0x1d, 0x86, 0x9c, 0xc3, 0xc0, 0xd4, 0xf7, 0x0e, 0x65, 0xa6, 0xdb, 0x4d, + 0x6a, 0x8b, 0xfe, 0xf5, 0xbb, 0x9e, 0xf5, 0x59, 0xe7, 0x30, 0xb8, 0x28, 0x12, 0xa9, 0xe9, 0x96, + 0x08, 0x66, 0x8b, 0x3e, 0xcc, 0xf5, 0x2c, 0xd8, 0x19, 0x6c, 0x5c, 0x32, 0xc7, 0x7a, 0x47, 0x2e, + 0x7d, 0x4e, 0x97, 0xd7, 0x72, 0x46, 0xcf, 0xbe, 0x3f, 0x5d, 0xa6, 0x1a, 0xab, 0x6a, 0x98, 0xaa, + 0xd0, 0xfc, 0x0a, 0x67, 0x2a, 0x5c, 0xea, 0xb0, 0x7e, 0x91, 0x42, 0xfb, 0xf5, 0x9a, 0xae, 0xd7, + 0xda, 0xcb, 0xbf, 0x01, 0x00, 0x00, 0xff, 0xff, 0xbd, 0x47, 0x8e, 0x80, 0xe8, 0x06, 0x00, 0x00, } diff --git a/go/vt/vtcombo/tablet_map.go b/go/vt/vtcombo/tablet_map.go index 0e6c79d254a..849fbb979af 100644 --- a/go/vt/vtcombo/tablet_map.go +++ b/go/vt/vtcombo/tablet_map.go @@ -47,6 +47,7 @@ import ( "vitess.io/vitess/go/vt/vttablet/tmclient" "vitess.io/vitess/go/vt/wrangler" + 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" @@ -472,6 +473,12 @@ func (itc *internalTabletConn) UpdateStream(ctx context.Context, target *querypb return tabletconn.ErrorFromGRPC(vterrors.ToGRPC(err)) } +// VStream is part of queryservice.QueryService. +func (itc *internalTabletConn) VStream(ctx context.Context, target *querypb.Target, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { + err := itc.tablet.qsc.QueryService().VStream(ctx, target, startPos, filter, send) + return tabletconn.ErrorFromGRPC(vterrors.ToGRPC(err)) +} + // // TabletManagerClient implementation // diff --git a/go/vt/vttablet/grpcqueryservice/server.go b/go/vt/vttablet/grpcqueryservice/server.go index c3ada0faecf..76f90995f74 100644 --- a/go/vt/vttablet/grpcqueryservice/server.go +++ b/go/vt/vttablet/grpcqueryservice/server.go @@ -26,6 +26,7 @@ import ( "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vttablet/queryservice" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" querypb "vitess.io/vitess/go/vt/proto/query" queryservicepb "vitess.io/vitess/go/vt/proto/queryservice" ) @@ -375,6 +376,23 @@ func (q *query) UpdateStream(request *querypb.UpdateStreamRequest, stream querys return nil } +// VStream is part of the queryservice.QueryServer interface +func (q *query) VStream(request *binlogdatapb.VStreamRequest, stream queryservicepb.Query_VStreamServer) (err error) { + defer q.server.HandlePanic(&err) + ctx := callerid.NewContext(callinfo.GRPCCallInfo(stream.Context()), + request.EffectiveCallerId, + request.ImmediateCallerId, + ) + if err := q.server.VStream(ctx, request.Target, request.Position, request.Filter, func(events []*binlogdatapb.VEvent) error { + return stream.Send(&binlogdatapb.VStreamResponse{ + Events: events, + }) + }); err != nil { + return vterrors.ToGRPC(err) + } + return nil +} + // Register registers the implementation on the provide gRPC Server. func Register(s *grpc.Server, server queryservice.QueryService) { queryservicepb.RegisterQueryServer(s, &query{server}) diff --git a/go/vt/vttablet/grpctabletconn/conn.go b/go/vt/vttablet/grpctabletconn/conn.go index f4871b5a860..7410c56e5ac 100644 --- a/go/vt/vttablet/grpctabletconn/conn.go +++ b/go/vt/vttablet/grpctabletconn/conn.go @@ -30,6 +30,7 @@ import ( "vitess.io/vitess/go/vt/vttablet/queryservice" "vitess.io/vitess/go/vt/vttablet/tabletconn" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" querypb "vitess.io/vitess/go/vt/proto/query" queryservicepb "vitess.io/vitess/go/vt/proto/queryservice" topodatapb "vitess.io/vitess/go/vt/proto/topodata" @@ -668,6 +669,50 @@ func (conn *gRPCQueryClient) UpdateStream(ctx context.Context, target *querypb.T } } +// VStream starts a VReplication stream. +func (conn *gRPCQueryClient) VStream(ctx context.Context, target *querypb.Target, position string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { + stream, err := func() (queryservicepb.Query_VStreamClient, error) { + conn.mu.RLock() + defer conn.mu.RUnlock() + if conn.cc == nil { + return nil, tabletconn.ConnClosed + } + + req := &binlogdatapb.VStreamRequest{ + Target: target, + EffectiveCallerId: callerid.EffectiveCallerIDFromContext(ctx), + ImmediateCallerId: callerid.ImmediateCallerIDFromContext(ctx), + Position: position, + Filter: filter, + } + stream, err := conn.c.VStream(ctx, req) + if err != nil { + return nil, tabletconn.ErrorFromGRPC(err) + } + return stream, nil + }() + if err != nil { + return err + } + for { + r, err := stream.Recv() + if err != nil { + return tabletconn.ErrorFromGRPC(err) + } + select { + case <-ctx.Done(): + return nil + default: + } + if err := send(r.Events); err != nil { + if err == io.EOF { + return nil + } + return err + } + } +} + // HandlePanic is a no-op. func (conn *gRPCQueryClient) HandlePanic(err *error) { } diff --git a/go/vt/vttablet/queryservice/queryservice.go b/go/vt/vttablet/queryservice/queryservice.go index 225912ab26f..4f3204fc800 100644 --- a/go/vt/vttablet/queryservice/queryservice.go +++ b/go/vt/vttablet/queryservice/queryservice.go @@ -25,6 +25,7 @@ import ( "vitess.io/vitess/go/sqltypes" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" querypb "vitess.io/vitess/go/vt/proto/query" ) @@ -99,6 +100,9 @@ type QueryService interface { // UpdateStream streams updates from the provided position or timestamp. UpdateStream(ctx context.Context, target *querypb.Target, position string, timestamp int64, callback func(*querypb.StreamEvent) error) error + // VStream streams VReplication events based on the specified filter. + VStream(ctx context.Context, target *querypb.Target, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error + // StreamHealth streams health status. StreamHealth(ctx context.Context, callback func(*querypb.StreamHealthResponse) error) error diff --git a/go/vt/vttablet/queryservice/wrapped.go b/go/vt/vttablet/queryservice/wrapped.go index 0ba474dff63..611c0630711 100644 --- a/go/vt/vttablet/queryservice/wrapped.go +++ b/go/vt/vttablet/queryservice/wrapped.go @@ -22,6 +22,7 @@ import ( "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/vterrors" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" querypb "vitess.io/vitess/go/vt/proto/query" vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" ) @@ -250,6 +251,13 @@ func (ws *wrappedService) UpdateStream(ctx context.Context, target *querypb.Targ }) } +func (ws *wrappedService) VStream(ctx context.Context, target *querypb.Target, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { + return ws.wrapper(ctx, target, ws.impl, "UpdateStream", false, func(ctx context.Context, target *querypb.Target, conn QueryService) (error, bool) { + innerErr := conn.VStream(ctx, target, startPos, filter, send) + return innerErr, canRetry(ctx, innerErr) + }) +} + func (ws *wrappedService) StreamHealth(ctx context.Context, callback func(*querypb.StreamHealthResponse) error) error { return ws.wrapper(ctx, nil, ws.impl, "StreamHealth", false, func(ctx context.Context, target *querypb.Target, conn QueryService) (error, bool) { innerErr := conn.StreamHealth(ctx, callback) diff --git a/go/vt/vttablet/sandboxconn/sandboxconn.go b/go/vt/vttablet/sandboxconn/sandboxconn.go index dc614c7d2be..a856555098f 100644 --- a/go/vt/vttablet/sandboxconn/sandboxconn.go +++ b/go/vt/vttablet/sandboxconn/sandboxconn.go @@ -27,6 +27,7 @@ import ( "vitess.io/vitess/go/vt/vterrors" "vitess.io/vitess/go/vt/vttablet/queryservice" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" querypb "vitess.io/vitess/go/vt/proto/query" topodatapb "vitess.io/vitess/go/vt/proto/topodata" vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" @@ -354,7 +355,11 @@ func (sbc *SandboxConn) StreamHealth(ctx context.Context, callback func(*querypb // UpdateStream is part of the QueryService interface. func (sbc *SandboxConn) UpdateStream(ctx context.Context, target *querypb.Target, position string, timestamp int64, callback func(*querypb.StreamEvent) error) error { - // FIXME(alainjobart) implement, use in vtgate tests. + return fmt.Errorf("Not implemented in test") +} + +// VStream is part of the QueryService interface. +func (sbc *SandboxConn) VStream(ctx context.Context, target *querypb.Target, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { return fmt.Errorf("Not implemented in test") } diff --git a/go/vt/vttablet/tabletconntest/fakequeryservice.go b/go/vt/vttablet/tabletconntest/fakequeryservice.go index 813a48efd3a..f6ebc1e5bdd 100644 --- a/go/vt/vttablet/tabletconntest/fakequeryservice.go +++ b/go/vt/vttablet/tabletconntest/fakequeryservice.go @@ -28,6 +28,7 @@ import ( "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/callerid" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" querypb "vitess.io/vitess/go/vt/proto/query" topodatapb "vitess.io/vitess/go/vt/proto/topodata" vtrpcpb "vitess.io/vitess/go/vt/proto/vtrpc" @@ -850,6 +851,11 @@ func (f *FakeQueryService) UpdateStream(ctx context.Context, target *querypb.Tar return nil } +// VStream is part of the queryservice.QueryService interface +func (f *FakeQueryService) VStream(ctx context.Context, target *querypb.Target, position string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { + panic("not implemented") +} + // CreateFakeServer returns the fake server for the tests func CreateFakeServer(t *testing.T) *FakeQueryService { return &FakeQueryService{ diff --git a/go/vt/vttablet/tabletmanager/vreplication/controller.go b/go/vt/vttablet/tabletmanager/vreplication/controller.go index 784d104c1f8..3379ba39743 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/controller.go +++ b/go/vt/vttablet/tabletmanager/vreplication/controller.go @@ -131,7 +131,7 @@ func (ct *controller) run(ctx context.Context) { return default: } - log.Warningf("stream %v: %v, retrying after %v", ct.id, err, *retryDelay) + log.Errorf("stream %v: %v, retrying after %v", ct.id, err, *retryDelay) time.Sleep(*retryDelay) } } diff --git a/go/vt/vttablet/tabletmanager/vreplication/player_plan.go b/go/vt/vttablet/tabletmanager/vreplication/player_plan.go index f08c59f7e8e..d848b02dcb8 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/player_plan.go +++ b/go/vt/vttablet/tabletmanager/vreplication/player_plan.go @@ -23,6 +23,7 @@ import ( "vitess.io/vitess/go/vt/sqlparser" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + querypb "vitess.io/vitess/go/vt/proto/query" ) type playerPlan struct { @@ -31,9 +32,13 @@ type playerPlan struct { } type tablePlan struct { + name string colExprs []*colExpr onInsert insertType updateCols []int + + fields []*querypb.Field + pkCols []*colExpr } func (tp *tablePlan) findCol(name sqlparser.ColIdent) *colExpr { @@ -113,15 +118,16 @@ func buildTablePlan(rule *binlogdatapb.Rule) (*binlogdatapb.Rule, *tablePlan, er if fromTable.IsEmpty() { return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(sel)) } - if fromTable.String() != rule.Match { - return nil, nil, fmt.Errorf("unexpected: select expression table %v does not match the table entry name %s", sqlparser.String(fromTable), rule.Match) - } if _, ok := sel.SelectExprs[0].(*sqlparser.StarExpr); ok { if len(sel.SelectExprs) != 1 { return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(sel)) } - return rule, nil, nil + sendRule := &binlogdatapb.Rule{ + Match: fromTable.String(), + Filter: rule.Filter, + } + return sendRule, &tablePlan{name: rule.Match}, nil } tplan := &tablePlan{} @@ -158,9 +164,13 @@ func analyzeExpr(selExpr sqlparser.SelectExpr) (sqlparser.SelectExpr, *colExpr, if !ok { return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(selExpr)) } + as := aliased.As + if as.IsEmpty() { + as = sqlparser.NewColIdent(sqlparser.String(aliased.Expr)) + } switch expr := aliased.Expr.(type) { case *sqlparser.ColName: - return selExpr, &colExpr{colname: expr.Name}, nil + return selExpr, &colExpr{colname: as}, nil case *sqlparser.FuncExpr: if expr.Distinct || len(expr.Exprs) != 1 { return nil, nil, fmt.Errorf("unsupported: %v", sqlparser.String(expr)) @@ -170,12 +180,12 @@ func analyzeExpr(selExpr sqlparser.SelectExpr) (sqlparser.SelectExpr, *colExpr, } switch fname := expr.Name.Lowered(); fname { case "month", "day", "hour": - return selExpr, &colExpr{colname: aliased.As}, nil + return selExpr, &colExpr{colname: as}, nil case "count": if _, ok := expr.Exprs[0].(*sqlparser.StarExpr); !ok { return nil, nil, fmt.Errorf("only count(*) is supported: %v", sqlparser.String(expr)) } - return nil, &colExpr{colname: aliased.As, op: opCount}, nil + return nil, &colExpr{colname: as, op: opCount}, nil case "sum": aInner, ok := expr.Exprs[0].(*sqlparser.AliasedExpr) if !ok { @@ -185,7 +195,7 @@ func analyzeExpr(selExpr sqlparser.SelectExpr) (sqlparser.SelectExpr, *colExpr, if !ok { return nil, nil, fmt.Errorf("unsupported: %v", sqlparser.String(expr)) } - return &sqlparser.AliasedExpr{Expr: innerCol}, &colExpr{colname: aliased.As, op: opSum}, nil + return &sqlparser.AliasedExpr{Expr: innerCol}, &colExpr{colname: as, op: opSum}, nil default: return nil, nil, fmt.Errorf("unsupported: %v", sqlparser.String(expr)) } diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go new file mode 100644 index 00000000000..dff3cfa7b77 --- /dev/null +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -0,0 +1,236 @@ +/* +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 vreplication + +import ( + "fmt" + "io" + "time" + + "golang.org/x/net/context" + + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/vt/binlog/binlogplayer" + "vitess.io/vitess/go/vt/grpcclient" + "vitess.io/vitess/go/vt/log" + "vitess.io/vitess/go/vt/mysqlctl" + "vitess.io/vitess/go/vt/sqlparser" + "vitess.io/vitess/go/vt/vttablet/tabletconn" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + querypb "vitess.io/vitess/go/vt/proto/query" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" +) + +type vplayer struct { + id uint32 + filter *binlogdatapb.Filter + sourceTablet *topodatapb.Tablet + stats *binlogplayer.Stats + dbClient binlogplayer.DBClient + mysqld *mysqlctl.Mysqld + + pos mysql.Position + stopPos mysql.Position + pplan *playerPlan + tplans map[string]*tablePlan + + retryDelay time.Duration +} + +func newVStreamer(id uint32, filter *binlogdatapb.Filter, sourceTablet *topodatapb.Tablet, stats *binlogplayer.Stats, dbClient binlogplayer.DBClient, mysqld *mysqlctl.Mysqld) *vplayer { + return &vplayer{ + id: id, + filter: filter, + sourceTablet: sourceTablet, + stats: stats, + dbClient: dbClient, + mysqld: mysqld, + retryDelay: 1 * time.Second, + } +} + +func (vp *vplayer) Play(ctx context.Context) error { + vp.setState(binlogplayer.BlpRunning, "") + if err := vp.play(ctx); err != nil { + msg := err.Error() + vp.stats.History.Add(&binlogplayer.StatsHistoryRecord{ + Time: time.Now(), + Message: msg, + }) + vp.setState(binlogplayer.BlpError, msg) + return err + } + return nil +} + +func (vp *vplayer) play(ctx context.Context) error { + startPos, stopPos, _, _, err := binlogplayer.ReadVRSettings(vp.dbClient, vp.id) + if err != nil { + return fmt.Errorf("error reading VReplication settings: %v", err) + } + vp.pos, err = mysql.DecodePosition(startPos) + if err != nil { + return fmt.Errorf("error decoding start position %v: %v", startPos, err) + } + if stopPos != "" { + vp.stopPos, err = mysql.DecodePosition(stopPos) + if err != nil { + return fmt.Errorf("error decoding stop position %v: %v", stopPos, err) + } + } + if !vp.stopPos.IsZero() { + if vp.pos.AtLeast(vp.stopPos) { + vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("Stop position %v already reached: %v", vp.pos, vp.stopPos)) + return nil + } + } + log.Infof("Starting VReplication player id: %v, startPos: %v, stop: %v, source: %v", vp.id, startPos, vp.stopPos, vp.sourceTablet) + + plan, err := buildPlayerPlan(vp.filter) + if err != nil { + return err + } + + vsClient, err := tabletconn.GetDialer()(vp.sourceTablet, grpcclient.FailFast(false)) + if err != nil { + return fmt.Errorf("error dialing tablet: %v", err) + } + target := &querypb.Target{ + Keyspace: vp.sourceTablet.Keyspace, + Shard: vp.sourceTablet.Shard, + TabletType: vp.sourceTablet.Type, + } + return vsClient.VStream(ctx, target, startPos, plan.vstreamFilter, func(events []*binlogdatapb.VEvent) error { + for _, event := range events { + select { + case <-ctx.Done(): + return io.EOF + default: + } + if err := vp.applyEvent(event); err != nil { + return err + } + } + return nil + }) +} + +func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { + switch event.Type { + case binlogdatapb.VEventType_GTID: + pos, err := mysql.DecodePosition(event.Gtid) + if err != nil { + return err + } + vp.pos = pos + if vp.stopPos.IsZero() { + return nil + } + if !vp.pos.Equal(vp.stopPos) && vp.pos.AtLeast(vp.stopPos) { + return fmt.Errorf("next event position %v exceeds stop pos %v, exiting without applying", vp.pos, vp.stopPos) + } + case binlogdatapb.VEventType_BEGIN: + if err := vp.dbClient.Begin(); err != nil { + return err + } + case binlogdatapb.VEventType_COMMIT: + updatePos := binlogplayer.GenerateUpdatePos(vp.id, vp.pos, time.Now().Unix(), event.Timestamp) + if _, err := vp.dbClient.ExecuteFetch(updatePos, 0); err != nil { + _ = vp.dbClient.Rollback() + return fmt.Errorf("error %v updating position", err) + } + if err := vp.dbClient.Commit(); err != nil { + return err + } + case binlogdatapb.VEventType_ROLLBACK: + // This code is unreachable. It's just here as failsafe. + _ = vp.dbClient.Rollback() + case binlogdatapb.VEventType_FIELD: + if err := vp.updatePlans(event.FieldEvent); err != nil { + return err + } + } + return nil +} + +func (vp *vplayer) setState(state, message string) { + if err := binlogplayer.SetVReplicationState(vp.dbClient, vp.id, state, message); err != nil { + log.Errorf("Error writing state: %s, msg: %s, err: %v", state, message, err) + } +} + +func (vp *vplayer) updatePlans(fieldEvent *binlogdatapb.FieldEvent) error { + prelim := vp.pplan.tablePlans[fieldEvent.TableName] + tplan := &tablePlan{ + name: fieldEvent.TableName, + } + if prelim != nil { + *tplan = *prelim + } + tplan.fields = fieldEvent.Fields + + if tplan.colExprs == nil { + tplan.colExprs = make([]*colExpr, len(tplan.fields)) + for i, field := range tplan.fields { + tplan.colExprs[i] = &colExpr{ + colname: sqlparser.NewColIdent(field.Name), + colnum: i, + } + } + } else { + if len(tplan.fields) != len(tplan.colExprs) { + return fmt.Errorf("columns received from vreplication: %v, do not match expected: %v", tplan.fields, tplan.colExprs) + } + for i, field := range tplan.fields { + if tplan.colExprs[i].colname.EqualString(field.Name) { + return fmt.Errorf("column name from vreplication field %d: %s, does not match expected: %s", i, field.Name, tplan.colExprs[i].colname) + } + } + } + + qr, err := vp.dbClient.ExecuteFetch("select database()", 1) + if err != nil { + return err + } + if len(qr.Rows) == 0 || len(qr.Rows[0]) == 0 { + return fmt.Errorf("unexpected result from 'select database()': %v", qr) + } + dbname := qr.Rows[0][0].ToString() + pkcols, err := vp.mysqld.GetPrimaryKeyColumns(dbname, tplan.name) + if err != nil { + return fmt.Errorf("error fetching pk columns for %s: %v", tplan.name, err) + } + for _, pkcol := range pkcols { + found := false + for i, cExpr := range tplan.colExprs { + if cExpr.colname.EqualString(pkcol) { + found = true + tplan.pkCols = append(tplan.pkCols, &colExpr{ + colname: cExpr.colname, + colnum: i, + }) + break + } + } + if !found { + return fmt.Errorf("primary key column %s missing from select list for table %s", pkcol, tplan.name) + } + } + vp.tplans[fieldEvent.TableName] = tplan + return nil +} diff --git a/go/vt/vttablet/tabletserver/tabletserver.go b/go/vt/vttablet/tabletserver/tabletserver.go index 51b7c0bf907..eb707aa1031 100644 --- a/go/vt/vttablet/tabletserver/tabletserver.go +++ b/go/vt/vttablet/tabletserver/tabletserver.go @@ -1274,7 +1274,7 @@ func (tsv *TabletServer) execDML(ctx context.Context, target *querypb.Target, qu } // VStream streams VReplication events. -func (tsv *TabletServer) VStream(ctx context.Context, target *querypb.Target, startPos mysql.Position, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { +func (tsv *TabletServer) VStream(ctx context.Context, target *querypb.Target, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { // This code is partially duplicated from startRequest. This is because // is allowed even if the tablet is in non-serving state. err := func() error { diff --git a/go/vt/vttablet/tabletserver/vstreamer/engine.go b/go/vt/vttablet/tabletserver/vstreamer/engine.go index 61cb603976c..9e0e014bd95 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/engine.go +++ b/go/vt/vttablet/tabletserver/vstreamer/engine.go @@ -136,7 +136,7 @@ func (vse *Engine) vschema() *vindexes.KeyspaceSchema { } // Stream starts a new stream. -func (vse *Engine) Stream(ctx context.Context, startPos mysql.Position, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { +func (vse *Engine) Stream(ctx context.Context, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { // Ensure kschema 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. diff --git a/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go b/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go index 432251f8cba..04e430bcb2b 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go +++ b/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go @@ -298,13 +298,17 @@ func analyzeExpr(ti *Table, selExpr sqlparser.SelectExpr) (cExpr ColExpr, err er if !ok { return ColExpr{}, fmt.Errorf("unexpected: %v", sqlparser.String(selExpr)) } + as := aliased.As + if as.IsEmpty() { + as = sqlparser.NewColIdent(sqlparser.String(aliased.Expr)) + } switch expr := aliased.Expr.(type) { case *sqlparser.ColName: colnum, err := findColumn(ti, expr.Name) if err != nil { return ColExpr{}, err } - return ColExpr{ColNum: colnum, Alias: expr.Name, Type: ti.Columns[colnum].Type}, nil + return ColExpr{ColNum: colnum, Alias: as, Type: ti.Columns[colnum].Type}, nil case *sqlparser.FuncExpr: if expr.Distinct || len(expr.Exprs) != 1 { return ColExpr{}, fmt.Errorf("unsupported: %v", sqlparser.String(expr)) @@ -319,10 +323,6 @@ func analyzeExpr(ti *Table, selExpr sqlparser.SelectExpr) (cExpr ColExpr, err er if !ok { return ColExpr{}, fmt.Errorf("unsupported: %v", sqlparser.String(expr)) } - as := aliased.As - if as.IsEmpty() { - as = sqlparser.NewColIdent(sqlparser.String(expr)) - } colnum, err := findColumn(ti, innerCol.Name) if err != nil { return ColExpr{}, err diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go index e2b4f74c213..ade2ced8c32 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go +++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go @@ -41,7 +41,7 @@ type vstreamer struct { cp *mysql.ConnParams se *schema.Engine - startPos mysql.Position + startPos string filter *binlogdatapb.Filter send func([]*binlogdatapb.VEvent) error @@ -55,7 +55,7 @@ type vstreamer struct { pos mysql.Position } -func newVStreamer(ctx context.Context, cp *mysql.ConnParams, se *schema.Engine, startPos mysql.Position, filter *binlogdatapb.Filter, kschema *vindexes.KeyspaceSchema, send func([]*binlogdatapb.VEvent) error) *vstreamer { +func newVStreamer(ctx context.Context, cp *mysql.ConnParams, se *schema.Engine, startPos string, filter *binlogdatapb.Filter, kschema *vindexes.KeyspaceSchema, send func([]*binlogdatapb.VEvent) error) *vstreamer { ctx, cancel := context.WithCancel(ctx) return &vstreamer{ ctx: ctx, @@ -88,7 +88,12 @@ func (vs *vstreamer) Cancel() { // Stream runs a single-threaded loop. func (vs *vstreamer) Stream() error { defer vs.cancel() - vs.pos = vs.startPos + + pos, err := mysql.DecodePosition(vs.startPos) + if err != nil { + return err + } + vs.pos = pos // Ensure se is Open. If vttablet came up in a non_serving role, // the schema engine may not have been initialized. @@ -375,6 +380,9 @@ func (vs *vstreamer) parseEvent(ev mysql.BinlogEvent) ([]*binlogdatapb.VEvent, e }) } } + for _, vevent := range vevents { + vevent.Timestamp = int64(ev.Timestamp()) + } return vevents, nil } diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go index 9a9ff697b07..55e68399286 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go @@ -374,10 +374,7 @@ func TestDDLAddColumn(t *testing.T) { }) // Record position before the next few statements. - pos, err := mysqld.MasterPosition() - if err != nil { - t.Fatal(err) - } + pos := masterPosition(t) execStatements(t, []string{ "begin", "insert into ddl_test1 values(1, 'aaa')", @@ -450,10 +447,7 @@ func TestDDLDropColumn(t *testing.T) { defer execStatement(t, "drop table ddl_test2") // Record position before the next few statements. - pos, err := mysqld.MasterPosition() - if err != nil { - t.Fatal(err) - } + pos := masterPosition(t) execStatements(t, []string{ "insert into ddl_test2 values(1, 'aaa', 'ccc')", // Adding columns is allowed. @@ -471,7 +465,7 @@ func TestDDLDropColumn(t *testing.T) { } }() defer close(ch) - err = vstream(ctx, t, pos, nil, ch) + err := vstream(ctx, t, pos, nil, ch) want := "cannot determine table columns" if err == nil || !strings.Contains(err.Error(), want) { t.Errorf("err: %v, must contain %s", err, want) @@ -795,10 +789,7 @@ func TestMinimalMode(t *testing.T) { engine.se.Reload(context.Background()) // Record position before the next few statements. - pos, err := mysqld.MasterPosition() - if err != nil { - t.Fatal(err) - } + pos := masterPosition(t) execStatements(t, []string{ "set @@session.binlog_row_image='minimal'", "update t1 set val1='bbb' where id=1", @@ -815,7 +806,7 @@ func TestMinimalMode(t *testing.T) { } }() defer close(ch) - err = vstream(ctx, t, pos, nil, ch) + err := vstream(ctx, t, pos, nil, ch) want := "partial row image encountered" if err == nil || !strings.Contains(err.Error(), want) { t.Errorf("err: %v, must contain '%s'", err, want) @@ -837,10 +828,7 @@ func TestStatementMode(t *testing.T) { engine.se.Reload(context.Background()) // Record position before the next few statements. - pos, err := mysqld.MasterPosition() - if err != nil { - t.Fatal(err) - } + pos := masterPosition(t) execStatements(t, []string{ "set @@session.binlog_format='statement'", "update t1 set val1='bbb' where id=1", @@ -857,7 +845,7 @@ func TestStatementMode(t *testing.T) { } }() defer close(ch) - err = vstream(ctx, t, pos, nil, ch) + err := vstream(ctx, t, pos, nil, ch) want := "unexpected statement type" if err == nil || !strings.Contains(err.Error(), want) { t.Errorf("err: %v, must contain '%s'", err, want) @@ -920,6 +908,10 @@ func expectLog(ctx context.Context, t *testing.T, input interface{}, ch <-chan [ t.Fatalf("%v (%d): event: %v, want commit", input, i, evs[i]) } default: + if evs[i].Timestamp == 0 { + t.Fatalf("evs[%d].Timestamp: 0, want non-zero", i) + } + evs[i].Timestamp = 0 if got := fmt.Sprintf("%v", evs[i]); got != want { t.Fatalf("%v (%d): event:\n%q, want\n%q", input, i, got, want) } @@ -929,10 +921,7 @@ func expectLog(ctx context.Context, t *testing.T, input interface{}, ch <-chan [ } func startStream(ctx context.Context, t *testing.T, filter *binlogdatapb.Filter) <-chan []*binlogdatapb.VEvent { - pos, err := mysqld.MasterPosition() - if err != nil { - t.Fatal(err) - } + pos := masterPosition(t) ch := make(chan []*binlogdatapb.VEvent) go func() { @@ -944,7 +933,7 @@ func startStream(ctx context.Context, t *testing.T, filter *binlogdatapb.Filter) return ch } -func vstream(ctx context.Context, t *testing.T, pos mysql.Position, filter *binlogdatapb.Filter, ch chan []*binlogdatapb.VEvent) error { +func vstream(ctx context.Context, t *testing.T, pos string, filter *binlogdatapb.Filter, ch chan []*binlogdatapb.VEvent) error { if filter == nil { filter = &binlogdatapb.Filter{ Rules: []*binlogdatapb.Rule{{ @@ -976,3 +965,12 @@ func execStatements(t *testing.T, queries []string) { t.Fatal(err) } } + +func masterPosition(t *testing.T) string { + t.Helper() + pos, err := mysqld.MasterPosition() + if err != nil { + t.Fatal(err) + } + return mysql.EncodePosition(pos) +} diff --git a/proto/binlogdata.proto b/proto/binlogdata.proto index 8c758d9895f..83f5abfca49 100644 --- a/proto/binlogdata.proto +++ b/proto/binlogdata.proto @@ -23,6 +23,7 @@ option go_package = "vitess.io/vitess/go/vt/proto/binlogdata"; package binlogdata; +import "vtrpc.proto"; import "query.proto"; import "topodata.proto"; @@ -195,19 +196,24 @@ message FieldEvent { // VEvent represents a vstream event message VEvent { VEventType type = 1; - string gtid = 2; - string ddl = 3; - RowEvent row_event = 4; - FieldEvent field_event = 5; + int64 timestamp = 2; + string gtid = 3; + string ddl = 4; + RowEvent row_event = 5; + FieldEvent field_event = 6; } // VStreamRequest is the payload for VStream message VStreamRequest { - string position = 1; - Filter filter = 2; + vtrpc.CallerID effective_caller_id = 1; + query.VTGateCallerID immediate_caller_id = 2; + query.Target target = 3; + + string position = 4; + Filter filter = 5; } // VStreamResponse is the response from VStream message VStreamResponse { - repeated VEvent event = 1; + repeated VEvent events = 1; } diff --git a/proto/queryservice.proto b/proto/queryservice.proto index a6a7074c9dc..897cbf3f034 100644 --- a/proto/queryservice.proto +++ b/proto/queryservice.proto @@ -22,6 +22,7 @@ package queryservice; option go_package = "vitess.io/vitess/go/vt/proto/queryservice"; import "query.proto"; +import "binlogdata.proto"; // Query defines the tablet query service, implemented by vttablet. service Query { @@ -94,4 +95,7 @@ service Query { // UpdateStream asks the server to return a stream of the updates that have been applied to its database. rpc UpdateStream(query.UpdateStreamRequest) returns (stream query.UpdateStreamResponse) {}; + + // VStream streams vreplication events. + rpc VStream(binlogdata.VStreamRequest) returns (stream binlogdata.VStreamResponse) {}; } diff --git a/py/vtproto/binlogdata_pb2.py b/py/vtproto/binlogdata_pb2.py index d9d8e896d92..0278b0f83f0 100644 --- a/py/vtproto/binlogdata_pb2.py +++ b/py/vtproto/binlogdata_pb2.py @@ -13,6 +13,7 @@ _sym_db = _symbol_database.Default() +import vtrpc_pb2 as vtrpc__pb2 import query_pb2 as query__pb2 import topodata_pb2 as topodata__pb2 @@ -22,9 +23,9 @@ package='binlogdata', syntax='proto3', serialized_options=_b('Z\'vitess.io/vitess/go/vt/proto/binlogdata'), - serialized_pb=_b('\n\x10\x62inlogdata.proto\x12\nbinlogdata\x1a\x0bquery.proto\x1a\x0etopodata.proto\"7\n\x07\x43harset\x12\x0e\n\x06\x63lient\x18\x01 \x01(\x05\x12\x0c\n\x04\x63onn\x18\x02 \x01(\x05\x12\x0e\n\x06server\x18\x03 \x01(\x05\"\xb5\x03\n\x11\x42inlogTransaction\x12;\n\nstatements\x18\x01 \x03(\x0b\x32\'.binlogdata.BinlogTransaction.Statement\x12&\n\x0b\x65vent_token\x18\x04 \x01(\x0b\x32\x11.query.EventToken\x1a\xae\x02\n\tStatement\x12\x42\n\x08\x63\x61tegory\x18\x01 \x01(\x0e\x32\x30.binlogdata.BinlogTransaction.Statement.Category\x12$\n\x07\x63harset\x18\x02 \x01(\x0b\x32\x13.binlogdata.Charset\x12\x0b\n\x03sql\x18\x03 \x01(\x0c\"\xa9\x01\n\x08\x43\x61tegory\x12\x13\n\x0f\x42L_UNRECOGNIZED\x10\x00\x12\x0c\n\x08\x42L_BEGIN\x10\x01\x12\r\n\tBL_COMMIT\x10\x02\x12\x0f\n\x0b\x42L_ROLLBACK\x10\x03\x12\x15\n\x11\x42L_DML_DEPRECATED\x10\x04\x12\n\n\x06\x42L_DDL\x10\x05\x12\n\n\x06\x42L_SET\x10\x06\x12\r\n\tBL_INSERT\x10\x07\x12\r\n\tBL_UPDATE\x10\x08\x12\r\n\tBL_DELETE\x10\tJ\x04\x08\x02\x10\x03J\x04\x08\x03\x10\x04\"v\n\x15StreamKeyRangeRequest\x12\x10\n\x08position\x18\x01 \x01(\t\x12%\n\tkey_range\x18\x02 \x01(\x0b\x32\x12.topodata.KeyRange\x12$\n\x07\x63harset\x18\x03 \x01(\x0b\x32\x13.binlogdata.Charset\"S\n\x16StreamKeyRangeResponse\x12\x39\n\x12\x62inlog_transaction\x18\x01 \x01(\x0b\x32\x1d.binlogdata.BinlogTransaction\"]\n\x13StreamTablesRequest\x12\x10\n\x08position\x18\x01 \x01(\t\x12\x0e\n\x06tables\x18\x02 \x03(\t\x12$\n\x07\x63harset\x18\x03 \x01(\x0b\x32\x13.binlogdata.Charset\"Q\n\x14StreamTablesResponse\x12\x39\n\x12\x62inlog_transaction\x18\x01 \x01(\x0b\x32\x1d.binlogdata.BinlogTransaction\"%\n\x04Rule\x12\r\n\x05match\x18\x01 \x01(\t\x12\x0e\n\x06\x66ilter\x18\x02 \x01(\t\")\n\x06\x46ilter\x12\x1f\n\x05rules\x18\x01 \x03(\x0b\x32\x10.binlogdata.Rule\"\xb5\x01\n\x0c\x42inlogSource\x12\x10\n\x08keyspace\x18\x01 \x01(\t\x12\r\n\x05shard\x18\x02 \x01(\t\x12)\n\x0btablet_type\x18\x03 \x01(\x0e\x32\x14.topodata.TabletType\x12%\n\tkey_range\x18\x04 \x01(\x0b\x32\x12.topodata.KeyRange\x12\x0e\n\x06tables\x18\x05 \x03(\t\x12\"\n\x06\x66ilter\x18\x06 \x01(\x0b\x32\x12.binlogdata.Filter\"B\n\tRowChange\x12\x1a\n\x06\x62\x65\x66ore\x18\x01 \x01(\x0b\x32\n.query.Row\x12\x19\n\x05\x61\x66ter\x18\x02 \x01(\x0b\x32\n.query.Row\"J\n\x08RowEvent\x12\x12\n\ntable_name\x18\x01 \x01(\t\x12*\n\x0brow_changes\x18\x02 \x03(\x0b\x32\x15.binlogdata.RowChange\">\n\nFieldEvent\x12\x12\n\ntable_name\x18\x01 \x01(\t\x12\x1c\n\x06\x66ields\x18\x02 \x03(\x0b\x32\x0c.query.Field\"\x9f\x01\n\x06VEvent\x12$\n\x04type\x18\x01 \x01(\x0e\x32\x16.binlogdata.VEventType\x12\x0c\n\x04gtid\x18\x02 \x01(\t\x12\x0b\n\x03\x64\x64l\x18\x03 \x01(\t\x12\'\n\trow_event\x18\x04 \x01(\x0b\x32\x14.binlogdata.RowEvent\x12+\n\x0b\x66ield_event\x18\x05 \x01(\x0b\x32\x16.binlogdata.FieldEvent\"F\n\x0eVStreamRequest\x12\x10\n\x08position\x18\x01 \x01(\t\x12\"\n\x06\x66ilter\x18\x02 \x01(\x0b\x32\x12.binlogdata.Filter\"4\n\x0fVStreamResponse\x12!\n\x05\x65vent\x18\x01 \x03(\x0b\x32\x12.binlogdata.VEvent*\xaa\x01\n\nVEventType\x12\x0b\n\x07UNKNOWN\x10\x00\x12\x08\n\x04GTID\x10\x01\x12\t\n\x05\x42\x45GIN\x10\x02\x12\n\n\x06\x43OMMIT\x10\x03\x12\x0c\n\x08ROLLBACK\x10\x04\x12\x07\n\x03\x44\x44L\x10\x05\x12\n\n\x06INSERT\x10\x06\x12\x0b\n\x07REPLACE\x10\x07\x12\n\n\x06UPDATE\x10\x08\x12\n\n\x06\x44\x45LETE\x10\t\x12\x07\n\x03SET\x10\n\x12\t\n\x05OTHER\x10\x0b\x12\x07\n\x03ROW\x10\x0c\x12\t\n\x05\x46IELD\x10\rB)Z\'vitess.io/vitess/go/vt/proto/binlogdatab\x06proto3') + serialized_pb=_b('\n\x10\x62inlogdata.proto\x12\nbinlogdata\x1a\x0bvtrpc.proto\x1a\x0bquery.proto\x1a\x0etopodata.proto\"7\n\x07\x43harset\x12\x0e\n\x06\x63lient\x18\x01 \x01(\x05\x12\x0c\n\x04\x63onn\x18\x02 \x01(\x05\x12\x0e\n\x06server\x18\x03 \x01(\x05\"\xb5\x03\n\x11\x42inlogTransaction\x12;\n\nstatements\x18\x01 \x03(\x0b\x32\'.binlogdata.BinlogTransaction.Statement\x12&\n\x0b\x65vent_token\x18\x04 \x01(\x0b\x32\x11.query.EventToken\x1a\xae\x02\n\tStatement\x12\x42\n\x08\x63\x61tegory\x18\x01 \x01(\x0e\x32\x30.binlogdata.BinlogTransaction.Statement.Category\x12$\n\x07\x63harset\x18\x02 \x01(\x0b\x32\x13.binlogdata.Charset\x12\x0b\n\x03sql\x18\x03 \x01(\x0c\"\xa9\x01\n\x08\x43\x61tegory\x12\x13\n\x0f\x42L_UNRECOGNIZED\x10\x00\x12\x0c\n\x08\x42L_BEGIN\x10\x01\x12\r\n\tBL_COMMIT\x10\x02\x12\x0f\n\x0b\x42L_ROLLBACK\x10\x03\x12\x15\n\x11\x42L_DML_DEPRECATED\x10\x04\x12\n\n\x06\x42L_DDL\x10\x05\x12\n\n\x06\x42L_SET\x10\x06\x12\r\n\tBL_INSERT\x10\x07\x12\r\n\tBL_UPDATE\x10\x08\x12\r\n\tBL_DELETE\x10\tJ\x04\x08\x02\x10\x03J\x04\x08\x03\x10\x04\"v\n\x15StreamKeyRangeRequest\x12\x10\n\x08position\x18\x01 \x01(\t\x12%\n\tkey_range\x18\x02 \x01(\x0b\x32\x12.topodata.KeyRange\x12$\n\x07\x63harset\x18\x03 \x01(\x0b\x32\x13.binlogdata.Charset\"S\n\x16StreamKeyRangeResponse\x12\x39\n\x12\x62inlog_transaction\x18\x01 \x01(\x0b\x32\x1d.binlogdata.BinlogTransaction\"]\n\x13StreamTablesRequest\x12\x10\n\x08position\x18\x01 \x01(\t\x12\x0e\n\x06tables\x18\x02 \x03(\t\x12$\n\x07\x63harset\x18\x03 \x01(\x0b\x32\x13.binlogdata.Charset\"Q\n\x14StreamTablesResponse\x12\x39\n\x12\x62inlog_transaction\x18\x01 \x01(\x0b\x32\x1d.binlogdata.BinlogTransaction\"%\n\x04Rule\x12\r\n\x05match\x18\x01 \x01(\t\x12\x0e\n\x06\x66ilter\x18\x02 \x01(\t\")\n\x06\x46ilter\x12\x1f\n\x05rules\x18\x01 \x03(\x0b\x32\x10.binlogdata.Rule\"\xb5\x01\n\x0c\x42inlogSource\x12\x10\n\x08keyspace\x18\x01 \x01(\t\x12\r\n\x05shard\x18\x02 \x01(\t\x12)\n\x0btablet_type\x18\x03 \x01(\x0e\x32\x14.topodata.TabletType\x12%\n\tkey_range\x18\x04 \x01(\x0b\x32\x12.topodata.KeyRange\x12\x0e\n\x06tables\x18\x05 \x03(\t\x12\"\n\x06\x66ilter\x18\x06 \x01(\x0b\x32\x12.binlogdata.Filter\"B\n\tRowChange\x12\x1a\n\x06\x62\x65\x66ore\x18\x01 \x01(\x0b\x32\n.query.Row\x12\x19\n\x05\x61\x66ter\x18\x02 \x01(\x0b\x32\n.query.Row\"J\n\x08RowEvent\x12\x12\n\ntable_name\x18\x01 \x01(\t\x12*\n\x0brow_changes\x18\x02 \x03(\x0b\x32\x15.binlogdata.RowChange\">\n\nFieldEvent\x12\x12\n\ntable_name\x18\x01 \x01(\t\x12\x1c\n\x06\x66ields\x18\x02 \x03(\x0b\x32\x0c.query.Field\"\xb2\x01\n\x06VEvent\x12$\n\x04type\x18\x01 \x01(\x0e\x32\x16.binlogdata.VEventType\x12\x11\n\ttimestamp\x18\x02 \x01(\x03\x12\x0c\n\x04gtid\x18\x03 \x01(\t\x12\x0b\n\x03\x64\x64l\x18\x04 \x01(\t\x12\'\n\trow_event\x18\x05 \x01(\x0b\x32\x14.binlogdata.RowEvent\x12+\n\x0b\x66ield_event\x18\x06 \x01(\x0b\x32\x16.binlogdata.FieldEvent\"\xc7\x01\n\x0eVStreamRequest\x12,\n\x13\x65\x66\x66\x65\x63tive_caller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12\x32\n\x13immediate_caller_id\x18\x02 \x01(\x0b\x32\x15.query.VTGateCallerID\x12\x1d\n\x06target\x18\x03 \x01(\x0b\x32\r.query.Target\x12\x10\n\x08position\x18\x04 \x01(\t\x12\"\n\x06\x66ilter\x18\x05 \x01(\x0b\x32\x12.binlogdata.Filter\"5\n\x0fVStreamResponse\x12\"\n\x06\x65vents\x18\x01 \x03(\x0b\x32\x12.binlogdata.VEvent*\xaa\x01\n\nVEventType\x12\x0b\n\x07UNKNOWN\x10\x00\x12\x08\n\x04GTID\x10\x01\x12\t\n\x05\x42\x45GIN\x10\x02\x12\n\n\x06\x43OMMIT\x10\x03\x12\x0c\n\x08ROLLBACK\x10\x04\x12\x07\n\x03\x44\x44L\x10\x05\x12\n\n\x06INSERT\x10\x06\x12\x0b\n\x07REPLACE\x10\x07\x12\n\n\x06UPDATE\x10\x08\x12\n\n\x06\x44\x45LETE\x10\t\x12\x07\n\x03SET\x10\n\x12\t\n\x05OTHER\x10\x0b\x12\x07\n\x03ROW\x10\x0c\x12\t\n\x05\x46IELD\x10\rB)Z\'vitess.io/vitess/go/vt/proto/binlogdatab\x06proto3') , - dependencies=[query__pb2.DESCRIPTOR,topodata__pb2.DESCRIPTOR,]) + dependencies=[vtrpc__pb2.DESCRIPTOR,query__pb2.DESCRIPTOR,topodata__pb2.DESCRIPTOR,]) _VEVENTTYPE = _descriptor.EnumDescriptor( name='VEventType', @@ -91,8 +92,8 @@ ], containing_type=None, serialized_options=None, - serialized_start=1704, - serialized_end=1874, + serialized_start=1867, + serialized_end=2037, ) _sym_db.RegisterEnumDescriptor(_VEVENTTYPE) @@ -162,8 +163,8 @@ ], containing_type=None, serialized_options=None, - serialized_start=375, - serialized_end=544, + serialized_start=388, + serialized_end=557, ) _sym_db.RegisterEnumDescriptor(_BINLOGTRANSACTION_STATEMENT_CATEGORY) @@ -208,8 +209,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=61, - serialized_end=116, + serialized_start=74, + serialized_end=129, ) @@ -254,8 +255,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=242, - serialized_end=544, + serialized_start=255, + serialized_end=557, ) _BINLOGTRANSACTION = _descriptor.Descriptor( @@ -291,8 +292,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=119, - serialized_end=556, + serialized_start=132, + serialized_end=569, ) @@ -336,8 +337,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=558, - serialized_end=676, + serialized_start=571, + serialized_end=689, ) @@ -367,8 +368,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=678, - serialized_end=761, + serialized_start=691, + serialized_end=774, ) @@ -412,8 +413,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=763, - serialized_end=856, + serialized_start=776, + serialized_end=869, ) @@ -443,8 +444,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=858, - serialized_end=939, + serialized_start=871, + serialized_end=952, ) @@ -481,8 +482,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=941, - serialized_end=978, + serialized_start=954, + serialized_end=991, ) @@ -512,8 +513,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=980, - serialized_end=1021, + serialized_start=993, + serialized_end=1034, ) @@ -578,8 +579,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1024, - serialized_end=1205, + serialized_start=1037, + serialized_end=1218, ) @@ -616,8 +617,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1207, - serialized_end=1273, + serialized_start=1220, + serialized_end=1286, ) @@ -654,8 +655,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1275, - serialized_end=1349, + serialized_start=1288, + serialized_end=1362, ) @@ -692,8 +693,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1351, - serialized_end=1413, + serialized_start=1364, + serialized_end=1426, ) @@ -712,33 +713,40 @@ is_extension=False, extension_scope=None, serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='gtid', full_name='binlogdata.VEvent.gtid', index=1, - number=2, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), + name='timestamp', full_name='binlogdata.VEvent.timestamp', index=1, + number=2, type=3, cpp_type=2, label=1, + has_default_value=False, default_value=0, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='ddl', full_name='binlogdata.VEvent.ddl', index=2, + name='gtid', full_name='binlogdata.VEvent.gtid', index=2, number=3, type=9, cpp_type=9, label=1, has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='row_event', full_name='binlogdata.VEvent.row_event', index=3, - number=4, type=11, cpp_type=10, label=1, - has_default_value=False, default_value=None, + name='ddl', full_name='binlogdata.VEvent.ddl', index=3, + number=4, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='field_event', full_name='binlogdata.VEvent.field_event', index=4, + name='row_event', full_name='binlogdata.VEvent.row_event', index=4, number=5, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='field_event', full_name='binlogdata.VEvent.field_event', index=5, + number=6, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), ], extensions=[ ], @@ -751,8 +759,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1416, - serialized_end=1575, + serialized_start=1429, + serialized_end=1607, ) @@ -764,19 +772,40 @@ containing_type=None, fields=[ _descriptor.FieldDescriptor( - name='position', full_name='binlogdata.VStreamRequest.position', index=0, - number=1, type=9, cpp_type=9, label=1, - has_default_value=False, default_value=_b("").decode('utf-8'), + name='effective_caller_id', full_name='binlogdata.VStreamRequest.effective_caller_id', index=0, + number=1, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, serialized_options=None, file=DESCRIPTOR), _descriptor.FieldDescriptor( - name='filter', full_name='binlogdata.VStreamRequest.filter', index=1, + name='immediate_caller_id', full_name='binlogdata.VStreamRequest.immediate_caller_id', index=1, number=2, type=11, cpp_type=10, label=1, has_default_value=False, default_value=None, message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='target', full_name='binlogdata.VStreamRequest.target', index=2, + number=3, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='position', full_name='binlogdata.VStreamRequest.position', index=3, + number=4, type=9, cpp_type=9, label=1, + has_default_value=False, default_value=_b("").decode('utf-8'), + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='filter', full_name='binlogdata.VStreamRequest.filter', index=4, + number=5, type=11, cpp_type=10, label=1, + has_default_value=False, default_value=None, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), ], extensions=[ ], @@ -789,8 +818,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1577, - serialized_end=1647, + serialized_start=1610, + serialized_end=1809, ) @@ -802,7 +831,7 @@ containing_type=None, fields=[ _descriptor.FieldDescriptor( - name='event', full_name='binlogdata.VStreamResponse.event', index=0, + name='events', full_name='binlogdata.VStreamResponse.events', index=0, number=1, type=11, cpp_type=10, label=3, has_default_value=False, default_value=[], message_type=None, enum_type=None, containing_type=None, @@ -820,8 +849,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1649, - serialized_end=1701, + serialized_start=1811, + serialized_end=1864, ) _BINLOGTRANSACTION_STATEMENT.fields_by_name['category'].enum_type = _BINLOGTRANSACTION_STATEMENT_CATEGORY @@ -846,8 +875,11 @@ _VEVENT.fields_by_name['type'].enum_type = _VEVENTTYPE _VEVENT.fields_by_name['row_event'].message_type = _ROWEVENT _VEVENT.fields_by_name['field_event'].message_type = _FIELDEVENT +_VSTREAMREQUEST.fields_by_name['effective_caller_id'].message_type = vtrpc__pb2._CALLERID +_VSTREAMREQUEST.fields_by_name['immediate_caller_id'].message_type = query__pb2._VTGATECALLERID +_VSTREAMREQUEST.fields_by_name['target'].message_type = query__pb2._TARGET _VSTREAMREQUEST.fields_by_name['filter'].message_type = _FILTER -_VSTREAMRESPONSE.fields_by_name['event'].message_type = _VEVENT +_VSTREAMRESPONSE.fields_by_name['events'].message_type = _VEVENT DESCRIPTOR.message_types_by_name['Charset'] = _CHARSET DESCRIPTOR.message_types_by_name['BinlogTransaction'] = _BINLOGTRANSACTION DESCRIPTOR.message_types_by_name['StreamKeyRangeRequest'] = _STREAMKEYRANGEREQUEST diff --git a/py/vtproto/queryservice_pb2.py b/py/vtproto/queryservice_pb2.py index 467e8b4dd44..3c443df129b 100644 --- a/py/vtproto/queryservice_pb2.py +++ b/py/vtproto/queryservice_pb2.py @@ -7,39 +7,39 @@ from google.protobuf import message as _message from google.protobuf import reflection as _reflection from google.protobuf import symbol_database as _symbol_database -from google.protobuf import descriptor_pb2 # @@protoc_insertion_point(imports) _sym_db = _symbol_database.Default() import query_pb2 as query__pb2 +import binlogdata_pb2 as binlogdata__pb2 DESCRIPTOR = _descriptor.FileDescriptor( name='queryservice.proto', package='queryservice', syntax='proto3', - serialized_pb=_b('\n\x12queryservice.proto\x12\x0cqueryservice\x1a\x0bquery.proto2\xa7\x0c\n\x05Query\x12:\n\x07\x45xecute\x12\x15.query.ExecuteRequest\x1a\x16.query.ExecuteResponse\"\x00\x12I\n\x0c\x45xecuteBatch\x12\x1a.query.ExecuteBatchRequest\x1a\x1b.query.ExecuteBatchResponse\"\x00\x12N\n\rStreamExecute\x12\x1b.query.StreamExecuteRequest\x1a\x1c.query.StreamExecuteResponse\"\x00\x30\x01\x12\x34\n\x05\x42\x65gin\x12\x13.query.BeginRequest\x1a\x14.query.BeginResponse\"\x00\x12\x37\n\x06\x43ommit\x12\x14.query.CommitRequest\x1a\x15.query.CommitResponse\"\x00\x12=\n\x08Rollback\x12\x16.query.RollbackRequest\x1a\x17.query.RollbackResponse\"\x00\x12:\n\x07Prepare\x12\x15.query.PrepareRequest\x1a\x16.query.PrepareResponse\"\x00\x12O\n\x0e\x43ommitPrepared\x12\x1c.query.CommitPreparedRequest\x1a\x1d.query.CommitPreparedResponse\"\x00\x12U\n\x10RollbackPrepared\x12\x1e.query.RollbackPreparedRequest\x1a\x1f.query.RollbackPreparedResponse\"\x00\x12X\n\x11\x43reateTransaction\x12\x1f.query.CreateTransactionRequest\x1a .query.CreateTransactionResponse\"\x00\x12\x46\n\x0bStartCommit\x12\x19.query.StartCommitRequest\x1a\x1a.query.StartCommitResponse\"\x00\x12\x46\n\x0bSetRollback\x12\x19.query.SetRollbackRequest\x1a\x1a.query.SetRollbackResponse\"\x00\x12^\n\x13\x43oncludeTransaction\x12!.query.ConcludeTransactionRequest\x1a\".query.ConcludeTransactionResponse\"\x00\x12R\n\x0fReadTransaction\x12\x1d.query.ReadTransactionRequest\x1a\x1e.query.ReadTransactionResponse\"\x00\x12I\n\x0c\x42\x65ginExecute\x12\x1a.query.BeginExecuteRequest\x1a\x1b.query.BeginExecuteResponse\"\x00\x12X\n\x11\x42\x65ginExecuteBatch\x12\x1f.query.BeginExecuteBatchRequest\x1a .query.BeginExecuteBatchResponse\"\x00\x12N\n\rMessageStream\x12\x1b.query.MessageStreamRequest\x1a\x1c.query.MessageStreamResponse\"\x00\x30\x01\x12\x43\n\nMessageAck\x12\x18.query.MessageAckRequest\x1a\x19.query.MessageAckResponse\"\x00\x12\x43\n\nSplitQuery\x12\x18.query.SplitQueryRequest\x1a\x19.query.SplitQueryResponse\"\x00\x12K\n\x0cStreamHealth\x12\x1a.query.StreamHealthRequest\x1a\x1b.query.StreamHealthResponse\"\x00\x30\x01\x12K\n\x0cUpdateStream\x12\x1a.query.UpdateStreamRequest\x1a\x1b.query.UpdateStreamResponse\"\x00\x30\x01\x42+Z)vitess.io/vitess/go/vt/proto/queryserviceb\x06proto3') + serialized_options=_b('Z)vitess.io/vitess/go/vt/proto/queryservice'), + serialized_pb=_b('\n\x12queryservice.proto\x12\x0cqueryservice\x1a\x0bquery.proto\x1a\x10\x62inlogdata.proto2\xef\x0c\n\x05Query\x12:\n\x07\x45xecute\x12\x15.query.ExecuteRequest\x1a\x16.query.ExecuteResponse\"\x00\x12I\n\x0c\x45xecuteBatch\x12\x1a.query.ExecuteBatchRequest\x1a\x1b.query.ExecuteBatchResponse\"\x00\x12N\n\rStreamExecute\x12\x1b.query.StreamExecuteRequest\x1a\x1c.query.StreamExecuteResponse\"\x00\x30\x01\x12\x34\n\x05\x42\x65gin\x12\x13.query.BeginRequest\x1a\x14.query.BeginResponse\"\x00\x12\x37\n\x06\x43ommit\x12\x14.query.CommitRequest\x1a\x15.query.CommitResponse\"\x00\x12=\n\x08Rollback\x12\x16.query.RollbackRequest\x1a\x17.query.RollbackResponse\"\x00\x12:\n\x07Prepare\x12\x15.query.PrepareRequest\x1a\x16.query.PrepareResponse\"\x00\x12O\n\x0e\x43ommitPrepared\x12\x1c.query.CommitPreparedRequest\x1a\x1d.query.CommitPreparedResponse\"\x00\x12U\n\x10RollbackPrepared\x12\x1e.query.RollbackPreparedRequest\x1a\x1f.query.RollbackPreparedResponse\"\x00\x12X\n\x11\x43reateTransaction\x12\x1f.query.CreateTransactionRequest\x1a .query.CreateTransactionResponse\"\x00\x12\x46\n\x0bStartCommit\x12\x19.query.StartCommitRequest\x1a\x1a.query.StartCommitResponse\"\x00\x12\x46\n\x0bSetRollback\x12\x19.query.SetRollbackRequest\x1a\x1a.query.SetRollbackResponse\"\x00\x12^\n\x13\x43oncludeTransaction\x12!.query.ConcludeTransactionRequest\x1a\".query.ConcludeTransactionResponse\"\x00\x12R\n\x0fReadTransaction\x12\x1d.query.ReadTransactionRequest\x1a\x1e.query.ReadTransactionResponse\"\x00\x12I\n\x0c\x42\x65ginExecute\x12\x1a.query.BeginExecuteRequest\x1a\x1b.query.BeginExecuteResponse\"\x00\x12X\n\x11\x42\x65ginExecuteBatch\x12\x1f.query.BeginExecuteBatchRequest\x1a .query.BeginExecuteBatchResponse\"\x00\x12N\n\rMessageStream\x12\x1b.query.MessageStreamRequest\x1a\x1c.query.MessageStreamResponse\"\x00\x30\x01\x12\x43\n\nMessageAck\x12\x18.query.MessageAckRequest\x1a\x19.query.MessageAckResponse\"\x00\x12\x43\n\nSplitQuery\x12\x18.query.SplitQueryRequest\x1a\x19.query.SplitQueryResponse\"\x00\x12K\n\x0cStreamHealth\x12\x1a.query.StreamHealthRequest\x1a\x1b.query.StreamHealthResponse\"\x00\x30\x01\x12K\n\x0cUpdateStream\x12\x1a.query.UpdateStreamRequest\x1a\x1b.query.UpdateStreamResponse\"\x00\x30\x01\x12\x46\n\x07VStream\x12\x1a.binlogdata.VStreamRequest\x1a\x1b.binlogdata.VStreamResponse\"\x00\x30\x01\x42+Z)vitess.io/vitess/go/vt/proto/queryserviceb\x06proto3') , - dependencies=[query__pb2.DESCRIPTOR,]) + dependencies=[query__pb2.DESCRIPTOR,binlogdata__pb2.DESCRIPTOR,]) _sym_db.RegisterFileDescriptor(DESCRIPTOR) -DESCRIPTOR.has_options = True -DESCRIPTOR._options = _descriptor._ParseOptions(descriptor_pb2.FileOptions(), _b('Z)vitess.io/vitess/go/vt/proto/queryservice')) +DESCRIPTOR._options = None _QUERY = _descriptor.ServiceDescriptor( name='Query', full_name='queryservice.Query', file=DESCRIPTOR, index=0, - options=None, - serialized_start=50, - serialized_end=1625, + serialized_options=None, + serialized_start=68, + serialized_end=1715, methods=[ _descriptor.MethodDescriptor( name='Execute', @@ -48,7 +48,7 @@ containing_service=None, input_type=query__pb2._EXECUTEREQUEST, output_type=query__pb2._EXECUTERESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='ExecuteBatch', @@ -57,7 +57,7 @@ containing_service=None, input_type=query__pb2._EXECUTEBATCHREQUEST, output_type=query__pb2._EXECUTEBATCHRESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='StreamExecute', @@ -66,7 +66,7 @@ containing_service=None, input_type=query__pb2._STREAMEXECUTEREQUEST, output_type=query__pb2._STREAMEXECUTERESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='Begin', @@ -75,7 +75,7 @@ containing_service=None, input_type=query__pb2._BEGINREQUEST, output_type=query__pb2._BEGINRESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='Commit', @@ -84,7 +84,7 @@ containing_service=None, input_type=query__pb2._COMMITREQUEST, output_type=query__pb2._COMMITRESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='Rollback', @@ -93,7 +93,7 @@ containing_service=None, input_type=query__pb2._ROLLBACKREQUEST, output_type=query__pb2._ROLLBACKRESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='Prepare', @@ -102,7 +102,7 @@ containing_service=None, input_type=query__pb2._PREPAREREQUEST, output_type=query__pb2._PREPARERESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='CommitPrepared', @@ -111,7 +111,7 @@ containing_service=None, input_type=query__pb2._COMMITPREPAREDREQUEST, output_type=query__pb2._COMMITPREPAREDRESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='RollbackPrepared', @@ -120,7 +120,7 @@ containing_service=None, input_type=query__pb2._ROLLBACKPREPAREDREQUEST, output_type=query__pb2._ROLLBACKPREPAREDRESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='CreateTransaction', @@ -129,7 +129,7 @@ containing_service=None, input_type=query__pb2._CREATETRANSACTIONREQUEST, output_type=query__pb2._CREATETRANSACTIONRESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='StartCommit', @@ -138,7 +138,7 @@ containing_service=None, input_type=query__pb2._STARTCOMMITREQUEST, output_type=query__pb2._STARTCOMMITRESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='SetRollback', @@ -147,7 +147,7 @@ containing_service=None, input_type=query__pb2._SETROLLBACKREQUEST, output_type=query__pb2._SETROLLBACKRESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='ConcludeTransaction', @@ -156,7 +156,7 @@ containing_service=None, input_type=query__pb2._CONCLUDETRANSACTIONREQUEST, output_type=query__pb2._CONCLUDETRANSACTIONRESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='ReadTransaction', @@ -165,7 +165,7 @@ containing_service=None, input_type=query__pb2._READTRANSACTIONREQUEST, output_type=query__pb2._READTRANSACTIONRESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='BeginExecute', @@ -174,7 +174,7 @@ containing_service=None, input_type=query__pb2._BEGINEXECUTEREQUEST, output_type=query__pb2._BEGINEXECUTERESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='BeginExecuteBatch', @@ -183,7 +183,7 @@ containing_service=None, input_type=query__pb2._BEGINEXECUTEBATCHREQUEST, output_type=query__pb2._BEGINEXECUTEBATCHRESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='MessageStream', @@ -192,7 +192,7 @@ containing_service=None, input_type=query__pb2._MESSAGESTREAMREQUEST, output_type=query__pb2._MESSAGESTREAMRESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='MessageAck', @@ -201,7 +201,7 @@ containing_service=None, input_type=query__pb2._MESSAGEACKREQUEST, output_type=query__pb2._MESSAGEACKRESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='SplitQuery', @@ -210,7 +210,7 @@ containing_service=None, input_type=query__pb2._SPLITQUERYREQUEST, output_type=query__pb2._SPLITQUERYRESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='StreamHealth', @@ -219,7 +219,7 @@ containing_service=None, input_type=query__pb2._STREAMHEALTHREQUEST, output_type=query__pb2._STREAMHEALTHRESPONSE, - options=None, + serialized_options=None, ), _descriptor.MethodDescriptor( name='UpdateStream', @@ -228,7 +228,16 @@ containing_service=None, input_type=query__pb2._UPDATESTREAMREQUEST, output_type=query__pb2._UPDATESTREAMRESPONSE, - options=None, + serialized_options=None, + ), + _descriptor.MethodDescriptor( + name='VStream', + full_name='queryservice.Query.VStream', + index=21, + containing_service=None, + input_type=binlogdata__pb2._VSTREAMREQUEST, + output_type=binlogdata__pb2._VSTREAMRESPONSE, + serialized_options=None, ), ]) _sym_db.RegisterServiceDescriptor(_QUERY) diff --git a/py/vtproto/queryservice_pb2_grpc.py b/py/vtproto/queryservice_pb2_grpc.py index 3fc203d463e..61adf9b42bd 100644 --- a/py/vtproto/queryservice_pb2_grpc.py +++ b/py/vtproto/queryservice_pb2_grpc.py @@ -1,6 +1,7 @@ # Generated by the gRPC Python protocol compiler plugin. DO NOT EDIT! import grpc +import binlogdata_pb2 as binlogdata__pb2 import query_pb2 as query__pb2 @@ -119,6 +120,11 @@ def __init__(self, channel): request_serializer=query__pb2.UpdateStreamRequest.SerializeToString, response_deserializer=query__pb2.UpdateStreamResponse.FromString, ) + self.VStream = channel.unary_stream( + '/queryservice.Query/VStream', + request_serializer=binlogdata__pb2.VStreamRequest.SerializeToString, + response_deserializer=binlogdata__pb2.VStreamResponse.FromString, + ) class QueryServicer(object): @@ -279,6 +285,13 @@ def UpdateStream(self, request, context): context.set_details('Method not implemented!') raise NotImplementedError('Method not implemented!') + def VStream(self, request, context): + """VStream streams vreplication events. + """ + context.set_code(grpc.StatusCode.UNIMPLEMENTED) + context.set_details('Method not implemented!') + raise NotImplementedError('Method not implemented!') + def add_QueryServicer_to_server(servicer, server): rpc_method_handlers = { @@ -387,6 +400,11 @@ def add_QueryServicer_to_server(servicer, server): request_deserializer=query__pb2.UpdateStreamRequest.FromString, response_serializer=query__pb2.UpdateStreamResponse.SerializeToString, ), + 'VStream': grpc.unary_stream_rpc_method_handler( + servicer.VStream, + request_deserializer=binlogdata__pb2.VStreamRequest.FromString, + response_serializer=binlogdata__pb2.VStreamResponse.SerializeToString, + ), } generic_handler = grpc.method_handlers_generic_handler( 'queryservice.Query', rpc_method_handlers) From 39e974899d614c2c398e5d73c1b4347d5a4ed4cc Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Tue, 8 Jan 2019 21:58:57 -0800 Subject: [PATCH 03/27] vreplication: vplayer basic functionality done Signed-off-by: Sugu Sougoumarane --- go/vt/proto/binlogdata/binlogdata.pb.go | 237 +++++++++++------- .../tabletmanager/vreplication/player_plan.go | 24 +- .../tabletmanager/vreplication/vplayer.go | 232 ++++++++++++++++- proto/binlogdata.proto | 11 + py/vtproto/binlogdata_pb2.py | 76 ++++-- 5 files changed, 448 insertions(+), 132 deletions(-) diff --git a/go/vt/proto/binlogdata/binlogdata.pb.go b/go/vt/proto/binlogdata/binlogdata.pb.go index 0cddb2d2342..2683321fd16 100644 --- a/go/vt/proto/binlogdata/binlogdata.pb.go +++ b/go/vt/proto/binlogdata/binlogdata.pb.go @@ -21,6 +21,36 @@ var _ = math.Inf // proto package needs to be updated. const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package +// OnDDLAction lists the possible actions for DDLs. +type OnDDLAction int32 + +const ( + OnDDLAction_IGNORE OnDDLAction = 0 + OnDDLAction_STOP OnDDLAction = 1 + OnDDLAction_EXEC OnDDLAction = 2 + OnDDLAction_EXEC_IGNORE OnDDLAction = 3 +) + +var OnDDLAction_name = map[int32]string{ + 0: "IGNORE", + 1: "STOP", + 2: "EXEC", + 3: "EXEC_IGNORE", +} +var OnDDLAction_value = map[string]int32{ + "IGNORE": 0, + "STOP": 1, + "EXEC": 2, + "EXEC_IGNORE": 3, +} + +func (x OnDDLAction) String() string { + return proto.EnumName(OnDDLAction_name, int32(x)) +} +func (OnDDLAction) EnumDescriptor() ([]byte, []int) { + return fileDescriptor_binlogdata_6d214635eb8c538c, []int{0} +} + // VEventType enumerates the event types. // This list is comprehensive. Many of these types // will not be encountered in RBR mode. @@ -80,7 +110,7 @@ func (x VEventType) String() string { return proto.EnumName(VEventType_name, int32(x)) } func (VEventType) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_602eda7964a9642b, []int{0} + return fileDescriptor_binlogdata_6d214635eb8c538c, []int{1} } type BinlogTransaction_Statement_Category int32 @@ -128,7 +158,7 @@ func (x BinlogTransaction_Statement_Category) String() string { return proto.EnumName(BinlogTransaction_Statement_Category_name, int32(x)) } func (BinlogTransaction_Statement_Category) EnumDescriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_602eda7964a9642b, []int{1, 0, 0} + return fileDescriptor_binlogdata_6d214635eb8c538c, []int{1, 0, 0} } // Charset is the per-statement charset info from a QUERY_EVENT binlog entry. @@ -148,7 +178,7 @@ func (m *Charset) Reset() { *m = Charset{} } func (m *Charset) String() string { return proto.CompactTextString(m) } func (*Charset) ProtoMessage() {} func (*Charset) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_602eda7964a9642b, []int{0} + return fileDescriptor_binlogdata_6d214635eb8c538c, []int{0} } func (m *Charset) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Charset.Unmarshal(m, b) @@ -205,7 +235,7 @@ func (m *BinlogTransaction) Reset() { *m = BinlogTransaction{} } func (m *BinlogTransaction) String() string { return proto.CompactTextString(m) } func (*BinlogTransaction) ProtoMessage() {} func (*BinlogTransaction) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_602eda7964a9642b, []int{1} + return fileDescriptor_binlogdata_6d214635eb8c538c, []int{1} } func (m *BinlogTransaction) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_BinlogTransaction.Unmarshal(m, b) @@ -255,7 +285,7 @@ func (m *BinlogTransaction_Statement) Reset() { *m = BinlogTransaction_S func (m *BinlogTransaction_Statement) String() string { return proto.CompactTextString(m) } func (*BinlogTransaction_Statement) ProtoMessage() {} func (*BinlogTransaction_Statement) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_602eda7964a9642b, []int{1, 0} + return fileDescriptor_binlogdata_6d214635eb8c538c, []int{1, 0} } func (m *BinlogTransaction_Statement) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_BinlogTransaction_Statement.Unmarshal(m, b) @@ -313,7 +343,7 @@ func (m *StreamKeyRangeRequest) Reset() { *m = StreamKeyRangeRequest{} } func (m *StreamKeyRangeRequest) String() string { return proto.CompactTextString(m) } func (*StreamKeyRangeRequest) ProtoMessage() {} func (*StreamKeyRangeRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_602eda7964a9642b, []int{2} + return fileDescriptor_binlogdata_6d214635eb8c538c, []int{2} } func (m *StreamKeyRangeRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_StreamKeyRangeRequest.Unmarshal(m, b) @@ -366,7 +396,7 @@ func (m *StreamKeyRangeResponse) Reset() { *m = StreamKeyRangeResponse{} func (m *StreamKeyRangeResponse) String() string { return proto.CompactTextString(m) } func (*StreamKeyRangeResponse) ProtoMessage() {} func (*StreamKeyRangeResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_602eda7964a9642b, []int{3} + return fileDescriptor_binlogdata_6d214635eb8c538c, []int{3} } func (m *StreamKeyRangeResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_StreamKeyRangeResponse.Unmarshal(m, b) @@ -410,7 +440,7 @@ func (m *StreamTablesRequest) Reset() { *m = StreamTablesRequest{} } func (m *StreamTablesRequest) String() string { return proto.CompactTextString(m) } func (*StreamTablesRequest) ProtoMessage() {} func (*StreamTablesRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_602eda7964a9642b, []int{4} + return fileDescriptor_binlogdata_6d214635eb8c538c, []int{4} } func (m *StreamTablesRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_StreamTablesRequest.Unmarshal(m, b) @@ -463,7 +493,7 @@ func (m *StreamTablesResponse) Reset() { *m = StreamTablesResponse{} } func (m *StreamTablesResponse) String() string { return proto.CompactTextString(m) } func (*StreamTablesResponse) ProtoMessage() {} func (*StreamTablesResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_602eda7964a9642b, []int{5} + return fileDescriptor_binlogdata_6d214635eb8c538c, []int{5} } func (m *StreamTablesResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_StreamTablesResponse.Unmarshal(m, b) @@ -508,7 +538,7 @@ func (m *Rule) Reset() { *m = Rule{} } func (m *Rule) String() string { return proto.CompactTextString(m) } func (*Rule) ProtoMessage() {} func (*Rule) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_602eda7964a9642b, []int{6} + return fileDescriptor_binlogdata_6d214635eb8c538c, []int{6} } func (m *Rule) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Rule.Unmarshal(m, b) @@ -555,7 +585,7 @@ func (m *Filter) Reset() { *m = Filter{} } func (m *Filter) String() string { return proto.CompactTextString(m) } func (*Filter) ProtoMessage() {} func (*Filter) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_602eda7964a9642b, []int{7} + return fileDescriptor_binlogdata_6d214635eb8c538c, []int{7} } func (m *Filter) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_Filter.Unmarshal(m, b) @@ -598,17 +628,19 @@ type BinlogSource struct { Tables []string `protobuf:"bytes,5,rep,name=tables,proto3" json:"tables,omitempty"` // filter is set if we're using the generalized representation // for the filter. - Filter *Filter `protobuf:"bytes,6,opt,name=filter,proto3" json:"filter,omitempty"` - XXX_NoUnkeyedLiteral struct{} `json:"-"` - XXX_unrecognized []byte `json:"-"` - XXX_sizecache int32 `json:"-"` + Filter *Filter `protobuf:"bytes,6,opt,name=filter,proto3" json:"filter,omitempty"` + // on_ddl specifies the action to be taken when a DDL is encountered. + OnDdl OnDDLAction `protobuf:"varint,7,opt,name=on_ddl,json=onDdl,proto3,enum=binlogdata.OnDDLAction" json:"on_ddl,omitempty"` + XXX_NoUnkeyedLiteral struct{} `json:"-"` + XXX_unrecognized []byte `json:"-"` + XXX_sizecache int32 `json:"-"` } func (m *BinlogSource) Reset() { *m = BinlogSource{} } func (m *BinlogSource) String() string { return proto.CompactTextString(m) } func (*BinlogSource) ProtoMessage() {} func (*BinlogSource) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_602eda7964a9642b, []int{8} + return fileDescriptor_binlogdata_6d214635eb8c538c, []int{8} } func (m *BinlogSource) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_BinlogSource.Unmarshal(m, b) @@ -670,6 +702,13 @@ func (m *BinlogSource) GetFilter() *Filter { return nil } +func (m *BinlogSource) GetOnDdl() OnDDLAction { + if m != nil { + return m.OnDdl + } + return OnDDLAction_IGNORE +} + // RowChange represents one row change type RowChange struct { Before *query.Row `protobuf:"bytes,1,opt,name=before,proto3" json:"before,omitempty"` @@ -683,7 +722,7 @@ func (m *RowChange) Reset() { *m = RowChange{} } func (m *RowChange) String() string { return proto.CompactTextString(m) } func (*RowChange) ProtoMessage() {} func (*RowChange) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_602eda7964a9642b, []int{9} + return fileDescriptor_binlogdata_6d214635eb8c538c, []int{9} } func (m *RowChange) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_RowChange.Unmarshal(m, b) @@ -730,7 +769,7 @@ func (m *RowEvent) Reset() { *m = RowEvent{} } func (m *RowEvent) String() string { return proto.CompactTextString(m) } func (*RowEvent) ProtoMessage() {} func (*RowEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_602eda7964a9642b, []int{10} + return fileDescriptor_binlogdata_6d214635eb8c538c, []int{10} } func (m *RowEvent) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_RowEvent.Unmarshal(m, b) @@ -776,7 +815,7 @@ func (m *FieldEvent) Reset() { *m = FieldEvent{} } func (m *FieldEvent) String() string { return proto.CompactTextString(m) } func (*FieldEvent) ProtoMessage() {} func (*FieldEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_602eda7964a9642b, []int{11} + return fileDescriptor_binlogdata_6d214635eb8c538c, []int{11} } func (m *FieldEvent) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_FieldEvent.Unmarshal(m, b) @@ -827,7 +866,7 @@ func (m *VEvent) Reset() { *m = VEvent{} } func (m *VEvent) String() string { return proto.CompactTextString(m) } func (*VEvent) ProtoMessage() {} func (*VEvent) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_602eda7964a9642b, []int{12} + return fileDescriptor_binlogdata_6d214635eb8c538c, []int{12} } func (m *VEvent) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_VEvent.Unmarshal(m, b) @@ -905,7 +944,7 @@ func (m *VStreamRequest) Reset() { *m = VStreamRequest{} } func (m *VStreamRequest) String() string { return proto.CompactTextString(m) } func (*VStreamRequest) ProtoMessage() {} func (*VStreamRequest) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_602eda7964a9642b, []int{13} + return fileDescriptor_binlogdata_6d214635eb8c538c, []int{13} } func (m *VStreamRequest) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_VStreamRequest.Unmarshal(m, b) @@ -972,7 +1011,7 @@ func (m *VStreamResponse) Reset() { *m = VStreamResponse{} } func (m *VStreamResponse) String() string { return proto.CompactTextString(m) } func (*VStreamResponse) ProtoMessage() {} func (*VStreamResponse) Descriptor() ([]byte, []int) { - return fileDescriptor_binlogdata_602eda7964a9642b, []int{14} + return fileDescriptor_binlogdata_6d214635eb8c538c, []int{14} } func (m *VStreamResponse) XXX_Unmarshal(b []byte) error { return xxx_messageInfo_VStreamResponse.Unmarshal(m, b) @@ -1016,83 +1055,87 @@ func init() { proto.RegisterType((*VEvent)(nil), "binlogdata.VEvent") proto.RegisterType((*VStreamRequest)(nil), "binlogdata.VStreamRequest") proto.RegisterType((*VStreamResponse)(nil), "binlogdata.VStreamResponse") + proto.RegisterEnum("binlogdata.OnDDLAction", OnDDLAction_name, OnDDLAction_value) proto.RegisterEnum("binlogdata.VEventType", VEventType_name, VEventType_value) proto.RegisterEnum("binlogdata.BinlogTransaction_Statement_Category", BinlogTransaction_Statement_Category_name, BinlogTransaction_Statement_Category_value) } -func init() { proto.RegisterFile("binlogdata.proto", fileDescriptor_binlogdata_602eda7964a9642b) } - -var fileDescriptor_binlogdata_602eda7964a9642b = []byte{ - // 1123 bytes of a gzipped FileDescriptorProto - 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x56, 0xdd, 0x6e, 0xe2, 0x56, - 0x10, 0x5e, 0xb0, 0x31, 0x78, 0x9c, 0x1f, 0xe7, 0xe4, 0x47, 0x28, 0xea, 0x4a, 0x91, 0xd5, 0x76, - 0xd3, 0x48, 0x85, 0x2d, 0xfd, 0xbb, 0xaa, 0xaa, 0x00, 0x4e, 0xca, 0xc6, 0x81, 0xec, 0x89, 0x93, - 0x95, 0xf6, 0xc6, 0x72, 0xcc, 0x81, 0x58, 0x31, 0x36, 0xb1, 0x4f, 0x48, 0x79, 0x82, 0x3e, 0x40, - 0x9f, 0xa2, 0x7d, 0x90, 0x3e, 0x49, 0x7b, 0xd7, 0x87, 0xa8, 0xce, 0x8f, 0x0d, 0x64, 0xa5, 0x6e, - 0x7a, 0xd1, 0xbb, 0xf9, 0x3f, 0x33, 0xdf, 0x8c, 0x67, 0x0c, 0xe6, 0x4d, 0x18, 0x47, 0xc9, 0x78, - 0xe8, 0x53, 0xbf, 0x31, 0x4d, 0x13, 0x9a, 0x20, 0x58, 0x48, 0xf6, 0x8d, 0x19, 0x4d, 0xa7, 0x81, - 0x50, 0xec, 0x1b, 0xf7, 0x0f, 0x24, 0x9d, 0x4b, 0x66, 0x83, 0x26, 0xd3, 0x64, 0xe1, 0x65, 0x9d, - 0x43, 0xb5, 0x73, 0xeb, 0xa7, 0x19, 0xa1, 0x68, 0x0f, 0xb4, 0x20, 0x0a, 0x49, 0x4c, 0xeb, 0xa5, - 0x83, 0xd2, 0x61, 0x05, 0x4b, 0x0e, 0x21, 0x50, 0x83, 0x24, 0x8e, 0xeb, 0x65, 0x2e, 0xe5, 0x34, - 0xb3, 0xcd, 0x48, 0x3a, 0x23, 0x69, 0x5d, 0x11, 0xb6, 0x82, 0xb3, 0xfe, 0x52, 0x60, 0xab, 0xcd, - 0xf3, 0x70, 0x53, 0x3f, 0xce, 0xfc, 0x80, 0x86, 0x49, 0x8c, 0x4e, 0x01, 0x32, 0xea, 0x53, 0x32, - 0x21, 0x31, 0xcd, 0xea, 0xa5, 0x03, 0xe5, 0xd0, 0x68, 0xbd, 0x6a, 0x2c, 0x55, 0xf0, 0x81, 0x4b, - 0xe3, 0x32, 0xb7, 0xc7, 0x4b, 0xae, 0xa8, 0x05, 0x06, 0x99, 0x91, 0x98, 0x7a, 0x34, 0xb9, 0x23, - 0x71, 0x5d, 0x3d, 0x28, 0x1d, 0x1a, 0xad, 0xad, 0x86, 0x28, 0xd0, 0x66, 0x1a, 0x97, 0x29, 0x30, - 0x90, 0x82, 0xde, 0xff, 0xa3, 0x0c, 0x7a, 0x11, 0x0d, 0x39, 0x50, 0x0b, 0x7c, 0x4a, 0xc6, 0x49, - 0x3a, 0xe7, 0x65, 0x6e, 0xb4, 0x5e, 0x3f, 0x33, 0x91, 0x46, 0x47, 0xfa, 0xe1, 0x22, 0x02, 0xfa, - 0x12, 0xaa, 0x81, 0x40, 0x8f, 0xa3, 0x63, 0xb4, 0xb6, 0x97, 0x83, 0x49, 0x60, 0x71, 0x6e, 0x83, - 0x4c, 0x50, 0xb2, 0xfb, 0x88, 0x43, 0xb6, 0x86, 0x19, 0x69, 0xfd, 0x56, 0x82, 0x5a, 0x1e, 0x17, - 0x6d, 0xc3, 0x66, 0xdb, 0xf1, 0xae, 0xfa, 0xd8, 0xee, 0x0c, 0x4e, 0xfb, 0xbd, 0xf7, 0x76, 0xd7, - 0x7c, 0x81, 0xd6, 0xa0, 0xd6, 0x76, 0xbc, 0xb6, 0x7d, 0xda, 0xeb, 0x9b, 0x25, 0xb4, 0x0e, 0x7a, - 0xdb, 0xf1, 0x3a, 0x83, 0xf3, 0xf3, 0x9e, 0x6b, 0x96, 0xd1, 0x26, 0x18, 0x6d, 0xc7, 0xc3, 0x03, - 0xc7, 0x69, 0x1f, 0x77, 0xce, 0x4c, 0x05, 0xed, 0xc2, 0x56, 0xdb, 0xf1, 0xba, 0xe7, 0x8e, 0xd7, - 0xb5, 0x2f, 0xb0, 0xdd, 0x39, 0x76, 0xed, 0xae, 0xa9, 0x22, 0x00, 0x8d, 0x89, 0xbb, 0x8e, 0x59, - 0x91, 0xf4, 0xa5, 0xed, 0x9a, 0x9a, 0x0c, 0xd7, 0xeb, 0x5f, 0xda, 0xd8, 0x35, 0xab, 0x92, 0xbd, - 0xba, 0xe8, 0x1e, 0xbb, 0xb6, 0x59, 0x93, 0x6c, 0xd7, 0x76, 0x6c, 0xd7, 0x36, 0xf5, 0x37, 0x6a, - 0xad, 0x6c, 0x2a, 0x6f, 0xd4, 0x9a, 0x62, 0xaa, 0xd6, 0xaf, 0x25, 0xd8, 0xbd, 0xa4, 0x29, 0xf1, - 0x27, 0x67, 0x64, 0x8e, 0xfd, 0x78, 0x4c, 0x30, 0xb9, 0x7f, 0x20, 0x19, 0x45, 0xfb, 0x50, 0x9b, - 0x26, 0x59, 0xc8, 0xb0, 0xe3, 0x00, 0xeb, 0xb8, 0xe0, 0x51, 0x13, 0xf4, 0x3b, 0x32, 0xf7, 0x52, - 0x66, 0x2f, 0x01, 0x43, 0x8d, 0x62, 0x20, 0x8b, 0x48, 0xb5, 0x3b, 0x49, 0x2d, 0xe3, 0xab, 0x7c, - 0x1c, 0x5f, 0x6b, 0x04, 0x7b, 0x4f, 0x93, 0xca, 0xa6, 0x49, 0x9c, 0x11, 0xe4, 0x00, 0x12, 0x8e, - 0x1e, 0x5d, 0xf4, 0x96, 0xe7, 0x67, 0xb4, 0x5e, 0xfe, 0xeb, 0x00, 0xe0, 0xad, 0x9b, 0xa7, 0x22, - 0xeb, 0x67, 0xd8, 0x16, 0xef, 0xb8, 0xfe, 0x4d, 0x44, 0xb2, 0xe7, 0x94, 0xbe, 0x07, 0x1a, 0xe5, - 0xc6, 0xf5, 0xf2, 0x81, 0x72, 0xa8, 0x63, 0xc9, 0xfd, 0xd7, 0x0a, 0x87, 0xb0, 0xb3, 0xfa, 0xf2, - 0xff, 0x52, 0xdf, 0x37, 0xa0, 0xe2, 0x87, 0x88, 0xa0, 0x1d, 0xa8, 0x4c, 0x7c, 0x1a, 0xdc, 0xca, - 0x6a, 0x04, 0xc3, 0x4a, 0x19, 0x85, 0x11, 0x25, 0x29, 0x6f, 0xa1, 0x8e, 0x25, 0x67, 0xbd, 0x06, - 0xed, 0x84, 0x53, 0xe8, 0x73, 0xa8, 0xa4, 0x0f, 0xac, 0x56, 0xf1, 0xa9, 0x9b, 0xcb, 0x09, 0xb0, - 0xc0, 0x58, 0xa8, 0xad, 0xbf, 0x4b, 0xb0, 0x26, 0x12, 0xba, 0x4c, 0x1e, 0xd2, 0x80, 0x30, 0x04, - 0xef, 0xc8, 0x3c, 0x9b, 0xfa, 0x01, 0xc9, 0x11, 0xcc, 0x79, 0x96, 0x4c, 0x76, 0xeb, 0xa7, 0x43, - 0xf9, 0xaa, 0x60, 0xd0, 0xb7, 0x60, 0x70, 0x24, 0xa9, 0x47, 0xe7, 0x53, 0xc2, 0x31, 0xdc, 0x68, - 0xed, 0x2c, 0x86, 0x8a, 0xe3, 0x44, 0xdd, 0xf9, 0x94, 0x60, 0xa0, 0x05, 0xbd, 0x3a, 0x89, 0xea, - 0x33, 0x26, 0x71, 0xd1, 0xbf, 0xca, 0x4a, 0xff, 0x8e, 0x0a, 0x30, 0x34, 0x19, 0x65, 0xa9, 0x56, - 0x01, 0x47, 0x01, 0xd0, 0x5b, 0xd0, 0x71, 0xf2, 0xd8, 0xb9, 0xe5, 0x01, 0x2d, 0xd0, 0x6e, 0xc8, - 0x28, 0x49, 0x89, 0xec, 0x12, 0xc8, 0x2d, 0x86, 0x93, 0x47, 0x2c, 0x35, 0xe8, 0x00, 0x2a, 0xfe, - 0x28, 0x07, 0x7a, 0xd5, 0x44, 0x28, 0x2c, 0x1f, 0x6a, 0x38, 0x79, 0xe4, 0x9b, 0x0f, 0xbd, 0x04, - 0x51, 0xa1, 0x17, 0xfb, 0x93, 0x1c, 0x3e, 0x9d, 0x4b, 0xfa, 0xfe, 0x84, 0xa0, 0xef, 0xc0, 0x48, - 0x93, 0x47, 0x2f, 0xe0, 0xcf, 0x8b, 0x31, 0x34, 0x5a, 0xbb, 0x2b, 0xad, 0xc9, 0x93, 0xc3, 0x90, - 0xe6, 0x64, 0x66, 0xbd, 0x05, 0x38, 0x09, 0x49, 0x34, 0x7c, 0xd6, 0x23, 0x9f, 0x32, 0x38, 0x48, - 0x34, 0xcc, 0xe3, 0xaf, 0xc9, 0x94, 0x79, 0x04, 0x2c, 0x75, 0xd6, 0x9f, 0x25, 0xd0, 0xae, 0x45, - 0xbc, 0x23, 0x50, 0x79, 0xe3, 0xc4, 0x2e, 0xde, 0x5b, 0x4e, 0x47, 0x58, 0xf0, 0xd6, 0x71, 0x1b, - 0xf4, 0x09, 0xe8, 0x34, 0x9c, 0x90, 0x8c, 0xfa, 0x93, 0x29, 0x87, 0x44, 0xc1, 0x0b, 0x01, 0x3b, - 0x53, 0x63, 0x1a, 0x0e, 0xf9, 0x08, 0xe8, 0x98, 0xd3, 0x6c, 0xe1, 0x0e, 0x87, 0x11, 0x6f, 0xb0, - 0x8e, 0x19, 0x89, 0xbe, 0x02, 0x9d, 0xa1, 0xc0, 0xef, 0x43, 0xbd, 0xc2, 0x61, 0xdd, 0x79, 0x82, - 0x01, 0x7f, 0x16, 0xd7, 0xd2, 0x1c, 0xd7, 0xef, 0xc1, 0xe0, 0x79, 0x4b, 0x27, 0xd1, 0xe7, 0xbd, - 0xd5, 0x3e, 0xe7, 0xf8, 0x60, 0x18, 0x15, 0xb4, 0xf5, 0x4b, 0x19, 0x36, 0xae, 0xc5, 0xe7, 0x9a, - 0xaf, 0x88, 0x1f, 0x61, 0x9b, 0x8c, 0x46, 0x24, 0xa0, 0xe1, 0x8c, 0x78, 0x81, 0x1f, 0x45, 0x24, - 0xf5, 0xc2, 0xa1, 0x1c, 0x81, 0xcd, 0x86, 0x38, 0xdb, 0x1d, 0x2e, 0xef, 0x75, 0xf1, 0x56, 0x61, - 0x2b, 0x45, 0x43, 0x64, 0xc3, 0x76, 0x38, 0x99, 0x90, 0x61, 0xe8, 0xd3, 0xe5, 0x00, 0x62, 0x40, - 0x76, 0x25, 0xda, 0xd7, 0xee, 0xa9, 0x4f, 0xc9, 0x22, 0x4c, 0xe1, 0x51, 0x84, 0xf9, 0x8c, 0x8d, - 0x73, 0x3a, 0x2e, 0xb6, 0xce, 0xba, 0xf4, 0x74, 0xb9, 0x10, 0x4b, 0xe5, 0xca, 0x46, 0x53, 0x9f, - 0x6c, 0xb4, 0xc5, 0xe4, 0x57, 0x3e, 0x3a, 0xf9, 0x3f, 0xc0, 0x66, 0x01, 0x84, 0xdc, 0x58, 0x47, - 0xa0, 0x71, 0x3c, 0xf3, 0x25, 0x81, 0x3e, 0x6c, 0x3d, 0x96, 0x16, 0x47, 0xbf, 0x97, 0x00, 0x16, - 0xd3, 0x80, 0x0c, 0xa8, 0x5e, 0xf5, 0xcf, 0xfa, 0x83, 0x77, 0x7d, 0xf3, 0x05, 0xaa, 0x81, 0x7a, - 0xea, 0xf6, 0xba, 0x66, 0x09, 0xe9, 0x50, 0x11, 0x67, 0xb2, 0xcc, 0x6e, 0x9c, 0xbc, 0x91, 0x0a, - 0x3b, 0xa0, 0xc5, 0x81, 0x54, 0x51, 0x15, 0x94, 0xe2, 0x0c, 0xca, 0xbb, 0xa7, 0xb1, 0x80, 0xd8, - 0xbe, 0x70, 0x8e, 0x3b, 0xb6, 0x59, 0x65, 0x8a, 0xe2, 0x02, 0x02, 0x68, 0xf9, 0xf9, 0x63, 0x9e, - 0xec, 0x68, 0x02, 0x7b, 0x67, 0xe0, 0xfe, 0x64, 0x63, 0xd3, 0x60, 0x32, 0x3c, 0x78, 0x67, 0xae, - 0x31, 0xd9, 0x49, 0xcf, 0x76, 0xba, 0xe6, 0x7a, 0xfb, 0x8b, 0xf7, 0xaf, 0x66, 0x21, 0x25, 0x59, - 0xd6, 0x08, 0x93, 0xa6, 0xa0, 0x9a, 0xe3, 0xa4, 0x39, 0xa3, 0x4d, 0xfe, 0xc7, 0xd5, 0x5c, 0x94, - 0x79, 0xa3, 0x71, 0xc9, 0xd7, 0xff, 0x04, 0x00, 0x00, 0xff, 0xff, 0xcb, 0x74, 0x80, 0x55, 0xcd, - 0x09, 0x00, 0x00, +func init() { proto.RegisterFile("binlogdata.proto", fileDescriptor_binlogdata_6d214635eb8c538c) } + +var fileDescriptor_binlogdata_6d214635eb8c538c = []byte{ + // 1184 bytes of a gzipped FileDescriptorProto + 0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0xb4, 0x56, 0x5b, 0x6e, 0xdb, 0x56, + 0x13, 0x8e, 0x44, 0x8a, 0x12, 0x87, 0x8e, 0x4d, 0x1f, 0x5f, 0x7e, 0xc1, 0xf8, 0x03, 0x18, 0x44, + 0xdb, 0xb8, 0x06, 0x2a, 0xa7, 0xea, 0xed, 0xa9, 0x2d, 0x2c, 0x91, 0x71, 0x95, 0xd0, 0x92, 0x73, + 0xcc, 0x24, 0x45, 0x5e, 0x08, 0x9a, 0x3c, 0xb2, 0x09, 0x53, 0xa4, 0x4c, 0x1e, 0xcb, 0xd5, 0x0a, + 0xba, 0x80, 0xbe, 0x76, 0x03, 0xed, 0x42, 0xba, 0x92, 0x76, 0x1f, 0xc5, 0xb9, 0x90, 0x92, 0x1c, + 0xa0, 0x71, 0x1f, 0xfa, 0x36, 0xf7, 0x33, 0xf3, 0xcd, 0x70, 0x86, 0x60, 0x5e, 0xc4, 0x69, 0x92, + 0x5d, 0x46, 0x01, 0x0d, 0x3a, 0xd3, 0x3c, 0xa3, 0x19, 0x82, 0x85, 0x64, 0xcf, 0x98, 0xd1, 0x7c, + 0x1a, 0x0a, 0xc5, 0x9e, 0x71, 0x73, 0x4b, 0xf2, 0xb9, 0x64, 0xd6, 0x69, 0x36, 0xcd, 0x16, 0x5e, + 0xd6, 0x29, 0x34, 0xfb, 0x57, 0x41, 0x5e, 0x10, 0x8a, 0x76, 0x41, 0x0b, 0x93, 0x98, 0xa4, 0xb4, + 0x5d, 0xdb, 0xaf, 0x1d, 0x34, 0xb0, 0xe4, 0x10, 0x02, 0x35, 0xcc, 0xd2, 0xb4, 0x5d, 0xe7, 0x52, + 0x4e, 0x33, 0xdb, 0x82, 0xe4, 0x33, 0x92, 0xb7, 0x15, 0x61, 0x2b, 0x38, 0xeb, 0x2f, 0x05, 0x36, + 0x7b, 0x3c, 0x0f, 0x2f, 0x0f, 0xd2, 0x22, 0x08, 0x69, 0x9c, 0xa5, 0xe8, 0x04, 0xa0, 0xa0, 0x01, + 0x25, 0x13, 0x92, 0xd2, 0xa2, 0x5d, 0xdb, 0x57, 0x0e, 0x8c, 0xee, 0xd3, 0xce, 0x52, 0x05, 0xef, + 0xb9, 0x74, 0xce, 0x4b, 0x7b, 0xbc, 0xe4, 0x8a, 0xba, 0x60, 0x90, 0x19, 0x49, 0xa9, 0x4f, 0xb3, + 0x6b, 0x92, 0xb6, 0xd5, 0xfd, 0xda, 0x81, 0xd1, 0xdd, 0xec, 0x88, 0x02, 0x1d, 0xa6, 0xf1, 0x98, + 0x02, 0x03, 0xa9, 0xe8, 0xbd, 0x3f, 0xea, 0xa0, 0x57, 0xd1, 0x90, 0x0b, 0xad, 0x30, 0xa0, 0xe4, + 0x32, 0xcb, 0xe7, 0xbc, 0xcc, 0xf5, 0xee, 0xb3, 0x07, 0x26, 0xd2, 0xe9, 0x4b, 0x3f, 0x5c, 0x45, + 0x40, 0x9f, 0x41, 0x33, 0x14, 0xe8, 0x71, 0x74, 0x8c, 0xee, 0xd6, 0x72, 0x30, 0x09, 0x2c, 0x2e, + 0x6d, 0x90, 0x09, 0x4a, 0x71, 0x93, 0x70, 0xc8, 0xd6, 0x30, 0x23, 0xad, 0xdf, 0x6a, 0xd0, 0x2a, + 0xe3, 0xa2, 0x2d, 0xd8, 0xe8, 0xb9, 0xfe, 0xeb, 0x21, 0x76, 0xfa, 0xa3, 0x93, 0xe1, 0xe0, 0x9d, + 0x63, 0x9b, 0x8f, 0xd0, 0x1a, 0xb4, 0x7a, 0xae, 0xdf, 0x73, 0x4e, 0x06, 0x43, 0xb3, 0x86, 0x1e, + 0x83, 0xde, 0x73, 0xfd, 0xfe, 0xe8, 0xf4, 0x74, 0xe0, 0x99, 0x75, 0xb4, 0x01, 0x46, 0xcf, 0xf5, + 0xf1, 0xc8, 0x75, 0x7b, 0xc7, 0xfd, 0x97, 0xa6, 0x82, 0x76, 0x60, 0xb3, 0xe7, 0xfa, 0xf6, 0xa9, + 0xeb, 0xdb, 0xce, 0x19, 0x76, 0xfa, 0xc7, 0x9e, 0x63, 0x9b, 0x2a, 0x02, 0xd0, 0x98, 0xd8, 0x76, + 0xcd, 0x86, 0xa4, 0xcf, 0x1d, 0xcf, 0xd4, 0x64, 0xb8, 0xc1, 0xf0, 0xdc, 0xc1, 0x9e, 0xd9, 0x94, + 0xec, 0xeb, 0x33, 0xfb, 0xd8, 0x73, 0xcc, 0x96, 0x64, 0x6d, 0xc7, 0x75, 0x3c, 0xc7, 0xd4, 0x5f, + 0xa8, 0xad, 0xba, 0xa9, 0xbc, 0x50, 0x5b, 0x8a, 0xa9, 0x5a, 0xbf, 0xd4, 0x60, 0xe7, 0x9c, 0xe6, + 0x24, 0x98, 0xbc, 0x24, 0x73, 0x1c, 0xa4, 0x97, 0x04, 0x93, 0x9b, 0x5b, 0x52, 0x50, 0xb4, 0x07, + 0xad, 0x69, 0x56, 0xc4, 0x0c, 0x3b, 0x0e, 0xb0, 0x8e, 0x2b, 0x1e, 0x1d, 0x81, 0x7e, 0x4d, 0xe6, + 0x7e, 0xce, 0xec, 0x25, 0x60, 0xa8, 0x53, 0x0d, 0x64, 0x15, 0xa9, 0x75, 0x2d, 0xa9, 0x65, 0x7c, + 0x95, 0x0f, 0xe3, 0x6b, 0x8d, 0x61, 0xf7, 0x7e, 0x52, 0xc5, 0x34, 0x4b, 0x0b, 0x82, 0x5c, 0x40, + 0xc2, 0xd1, 0xa7, 0x8b, 0xde, 0xf2, 0xfc, 0x8c, 0xee, 0x93, 0x7f, 0x1c, 0x00, 0xbc, 0x79, 0x71, + 0x5f, 0x64, 0xfd, 0x04, 0x5b, 0xe2, 0x1d, 0x2f, 0xb8, 0x48, 0x48, 0xf1, 0x90, 0xd2, 0x77, 0x41, + 0xa3, 0xdc, 0xb8, 0x5d, 0xdf, 0x57, 0x0e, 0x74, 0x2c, 0xb9, 0x7f, 0x5b, 0x61, 0x04, 0xdb, 0xab, + 0x2f, 0xff, 0x27, 0xf5, 0x7d, 0x09, 0x2a, 0xbe, 0x4d, 0x08, 0xda, 0x86, 0xc6, 0x24, 0xa0, 0xe1, + 0x95, 0xac, 0x46, 0x30, 0xac, 0x94, 0x71, 0x9c, 0x50, 0x92, 0xf3, 0x16, 0xea, 0x58, 0x72, 0xd6, + 0x33, 0xd0, 0x9e, 0x73, 0x0a, 0x7d, 0x02, 0x8d, 0xfc, 0x96, 0xd5, 0x2a, 0x3e, 0x75, 0x73, 0x39, + 0x01, 0x16, 0x18, 0x0b, 0xb5, 0xf5, 0x6b, 0x1d, 0xd6, 0x44, 0x42, 0xe7, 0xd9, 0x6d, 0x1e, 0x12, + 0x86, 0xe0, 0x35, 0x99, 0x17, 0xd3, 0x20, 0x24, 0x25, 0x82, 0x25, 0xcf, 0x92, 0x29, 0xae, 0x82, + 0x3c, 0x92, 0xaf, 0x0a, 0x06, 0x7d, 0x05, 0x06, 0x47, 0x92, 0xfa, 0x74, 0x3e, 0x25, 0x1c, 0xc3, + 0xf5, 0xee, 0xf6, 0x62, 0xa8, 0x38, 0x4e, 0xd4, 0x9b, 0x4f, 0x09, 0x06, 0x5a, 0xd1, 0xab, 0x93, + 0xa8, 0x3e, 0x60, 0x12, 0x17, 0xfd, 0x6b, 0xac, 0xf4, 0xef, 0xb0, 0x02, 0x43, 0x93, 0x51, 0x96, + 0x6a, 0x15, 0x70, 0x94, 0x00, 0xa1, 0x0e, 0x68, 0x59, 0xea, 0x47, 0x51, 0xd2, 0x6e, 0xf2, 0x34, + 0xff, 0xb7, 0x6c, 0x3b, 0x4a, 0x6d, 0xdb, 0x3d, 0x16, 0x2d, 0x69, 0x64, 0xa9, 0x1d, 0x25, 0xd6, + 0x2b, 0xd0, 0x71, 0x76, 0xd7, 0xbf, 0xe2, 0x09, 0x58, 0xa0, 0x5d, 0x90, 0x71, 0x96, 0x13, 0xd9, + 0x55, 0x90, 0x5b, 0x0f, 0x67, 0x77, 0x58, 0x6a, 0xd0, 0x3e, 0x34, 0x82, 0x71, 0xd9, 0x98, 0x55, + 0x13, 0xa1, 0xb0, 0x02, 0x68, 0xe1, 0xec, 0x8e, 0x6f, 0x4a, 0xf4, 0x04, 0x04, 0x22, 0x7e, 0x1a, + 0x4c, 0x4a, 0xb8, 0x75, 0x2e, 0x19, 0x06, 0x13, 0x82, 0xbe, 0x06, 0x23, 0xcf, 0xee, 0xfc, 0x90, + 0x3f, 0x2f, 0xc6, 0xd6, 0xe8, 0xee, 0xac, 0xb4, 0xb2, 0x4c, 0x0e, 0x43, 0x5e, 0x92, 0x85, 0xf5, + 0x0a, 0xe0, 0x79, 0x4c, 0x92, 0xe8, 0x41, 0x8f, 0x7c, 0xc4, 0xe0, 0x23, 0x49, 0x54, 0xc6, 0x5f, + 0x93, 0x29, 0xf3, 0x08, 0x58, 0xea, 0xac, 0x3f, 0x6b, 0xa0, 0xbd, 0x11, 0xf1, 0x0e, 0x41, 0xe5, + 0x8d, 0x16, 0xbb, 0x7b, 0x77, 0x39, 0x1d, 0x61, 0xc1, 0x5b, 0xcd, 0x6d, 0xd0, 0xff, 0x41, 0xa7, + 0xf1, 0x84, 0x14, 0x34, 0x98, 0x4c, 0x39, 0x24, 0x0a, 0x5e, 0x08, 0xd8, 0x59, 0xbb, 0xa4, 0x71, + 0xc4, 0x47, 0x46, 0xc7, 0x9c, 0x66, 0x0b, 0x9a, 0xb5, 0x47, 0xe5, 0x22, 0x46, 0xa2, 0xcf, 0x41, + 0x67, 0x28, 0xf0, 0x7b, 0xd2, 0x6e, 0x70, 0x58, 0xb7, 0xef, 0x61, 0xc0, 0x9f, 0xc5, 0xad, 0xbc, + 0xc4, 0xf5, 0x1b, 0x30, 0x78, 0xde, 0xd2, 0x49, 0xcc, 0xc5, 0xee, 0xea, 0x5c, 0x94, 0xf8, 0x60, + 0x18, 0x57, 0xb4, 0xf5, 0x73, 0x1d, 0xd6, 0xdf, 0x88, 0xcf, 0xbb, 0x5c, 0x29, 0xdf, 0xc3, 0x16, + 0x19, 0x8f, 0x49, 0x48, 0xe3, 0x19, 0xf1, 0xc3, 0x20, 0x49, 0x48, 0xee, 0xc7, 0x91, 0x1c, 0x81, + 0x8d, 0x8e, 0x38, 0xf3, 0x7d, 0x2e, 0x1f, 0xd8, 0x78, 0xb3, 0xb2, 0x95, 0xa2, 0x08, 0x39, 0xb0, + 0x15, 0x4f, 0x26, 0x24, 0x8a, 0x03, 0xba, 0x1c, 0x40, 0x0c, 0xc8, 0x8e, 0x44, 0xfb, 0x8d, 0x77, + 0x12, 0x50, 0xb2, 0x08, 0x53, 0x79, 0x54, 0x61, 0x3e, 0x66, 0xe3, 0x9f, 0x5f, 0x56, 0x5b, 0xea, + 0xb1, 0xf4, 0xf4, 0xb8, 0x10, 0x4b, 0xe5, 0xca, 0x06, 0x54, 0xef, 0x6d, 0xc0, 0xc5, 0x97, 0xd2, + 0xf8, 0xd0, 0x97, 0x62, 0x7d, 0x0b, 0x1b, 0x15, 0x10, 0x72, 0xc3, 0x1d, 0x82, 0xc6, 0xf1, 0x2c, + 0x97, 0x0a, 0x7a, 0xbf, 0xf5, 0x58, 0x5a, 0x1c, 0x7e, 0x07, 0xc6, 0xd2, 0xe7, 0xc4, 0x2e, 0xde, + 0xe0, 0x64, 0x38, 0xc2, 0x8e, 0xf9, 0x08, 0xb5, 0x40, 0x3d, 0xf7, 0x46, 0x67, 0x66, 0x8d, 0x51, + 0xce, 0x8f, 0x4e, 0x5f, 0x5c, 0x51, 0x46, 0xf9, 0xd2, 0x48, 0x39, 0xfc, 0xbd, 0x06, 0xb0, 0x98, + 0x26, 0x64, 0x40, 0xf3, 0xf5, 0xf0, 0xe5, 0x70, 0xf4, 0x76, 0x28, 0x02, 0x9c, 0x78, 0x03, 0xdb, + 0xac, 0x21, 0x1d, 0x1a, 0xe2, 0x2c, 0xd7, 0xd9, 0x0b, 0xf2, 0x26, 0x2b, 0xec, 0x60, 0x57, 0x07, + 0x59, 0x45, 0x4d, 0x50, 0xaa, 0xb3, 0x2b, 0xef, 0xac, 0xc6, 0x02, 0x62, 0xe7, 0xcc, 0x3d, 0xee, + 0x3b, 0x66, 0x93, 0x29, 0xaa, 0x8b, 0x0b, 0xa0, 0x95, 0xe7, 0x96, 0x79, 0xb2, 0x23, 0x0d, 0xec, + 0x9d, 0x91, 0xf7, 0x83, 0x83, 0x4d, 0x83, 0xc9, 0xf0, 0xe8, 0xad, 0xb9, 0xc6, 0x64, 0xcf, 0x07, + 0x8e, 0x6b, 0x9b, 0x8f, 0x7b, 0x9f, 0xbe, 0x7b, 0x3a, 0x8b, 0x29, 0x29, 0x8a, 0x4e, 0x9c, 0x1d, + 0x09, 0xea, 0xe8, 0x32, 0x3b, 0x9a, 0xd1, 0x23, 0xfe, 0x87, 0x77, 0xb4, 0x80, 0xe9, 0x42, 0xe3, + 0x92, 0x2f, 0xfe, 0x0e, 0x00, 0x00, 0xff, 0xff, 0x68, 0xbd, 0x20, 0x05, 0x3d, 0x0a, 0x00, 0x00, } diff --git a/go/vt/vttablet/tabletmanager/vreplication/player_plan.go b/go/vt/vttablet/tabletmanager/vreplication/player_plan.go index d848b02dcb8..67b7c566972 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/player_plan.go +++ b/go/vt/vttablet/tabletmanager/vreplication/player_plan.go @@ -32,10 +32,9 @@ type playerPlan struct { } type tablePlan struct { - name string - colExprs []*colExpr - onInsert insertType - updateCols []int + name string + colExprs []*colExpr + onInsert insertType fields []*querypb.Field pkCols []*colExpr @@ -51,9 +50,10 @@ func (tp *tablePlan) findCol(name sqlparser.ColIdent) *colExpr { } type colExpr struct { - colname sqlparser.ColIdent - colnum int - op operation + colname sqlparser.ColIdent + colnum int + op operation + isGrouped bool } type operation int @@ -62,7 +62,6 @@ const ( opNone = operation(iota) opCount opSum - opExclude ) type insertType int @@ -151,6 +150,13 @@ func buildTablePlan(rule *binlogdatapb.Rule) (*binlogdatapb.Rule, *tablePlan, er if err := analyzeGroupBy(sel.GroupBy, tplan); err != nil { return nil, nil, err } + tplan.onInsert = insertIgnore + for _, cExpr := range tplan.colExprs { + if !cExpr.isGrouped { + tplan.onInsert = insertOndup + break + } + } } sendRule := &binlogdatapb.Rule{ Match: rule.Match, @@ -217,7 +223,7 @@ func analyzeGroupBy(groupBy sqlparser.GroupBy, tplan *tablePlan) error { if cExpr.op != opNone { return fmt.Errorf("group by expression is not allowed to reference an aggregate expression: %v", sqlparser.String(expr)) } - cExpr.op = opExclude + cExpr.isGrouped = true } return nil } diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index dff3cfa7b77..26bc1479acd 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -17,6 +17,7 @@ limitations under the License. package vreplication import ( + "bytes" "fmt" "io" "time" @@ -24,6 +25,7 @@ import ( "golang.org/x/net/context" "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/grpcclient" "vitess.io/vitess/go/vt/log" @@ -38,7 +40,7 @@ import ( type vplayer struct { id uint32 - filter *binlogdatapb.Filter + source *binlogdatapb.BinlogSource sourceTablet *topodatapb.Tablet stats *binlogplayer.Stats dbClient binlogplayer.DBClient @@ -52,10 +54,10 @@ type vplayer struct { retryDelay time.Duration } -func newVStreamer(id uint32, filter *binlogdatapb.Filter, sourceTablet *topodatapb.Tablet, stats *binlogplayer.Stats, dbClient binlogplayer.DBClient, mysqld *mysqlctl.Mysqld) *vplayer { +func newVPlayer(id uint32, source *binlogdatapb.BinlogSource, sourceTablet *topodatapb.Tablet, stats *binlogplayer.Stats, dbClient binlogplayer.DBClient, mysqld *mysqlctl.Mysqld) *vplayer { return &vplayer{ id: id, - filter: filter, + source: source, sourceTablet: sourceTablet, stats: stats, dbClient: dbClient, @@ -101,7 +103,7 @@ func (vp *vplayer) play(ctx context.Context) error { } log.Infof("Starting VReplication player id: %v, startPos: %v, stop: %v, source: %v", vp.id, startPos, vp.stopPos, vp.sourceTablet) - plan, err := buildPlayerPlan(vp.filter) + plan, err := buildPlayerPlan(vp.source.Filter) if err != nil { return err } @@ -149,10 +151,8 @@ func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { return err } case binlogdatapb.VEventType_COMMIT: - updatePos := binlogplayer.GenerateUpdatePos(vp.id, vp.pos, time.Now().Unix(), event.Timestamp) - if _, err := vp.dbClient.ExecuteFetch(updatePos, 0); err != nil { - _ = vp.dbClient.Rollback() - return fmt.Errorf("error %v updating position", err) + if err := vp.updatePos(event.Timestamp); err != nil { + return err } if err := vp.dbClient.Commit(); err != nil { return err @@ -161,9 +161,51 @@ func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { // This code is unreachable. It's just here as failsafe. _ = vp.dbClient.Rollback() case binlogdatapb.VEventType_FIELD: - if err := vp.updatePlans(event.FieldEvent); err != nil { + if err := vp.updatePlan(event.FieldEvent); err != nil { + return err + } + case binlogdatapb.VEventType_ROW: + if err := vp.applyRowEvent(event.RowEvent); err != nil { return err } + case binlogdatapb.VEventType_DDL: + switch vp.source.OnDdl { + case binlogdatapb.OnDDLAction_IGNORE: + if err := vp.updatePos(event.Timestamp); err != nil { + return err + } + if err := vp.dbClient.Commit(); err != nil { + return err + } + case binlogdatapb.OnDDLAction_STOP: + if err := vp.updatePos(event.Timestamp); err != nil { + return err + } + vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("stopped at DDL %s", event.Ddl)) + if err := vp.dbClient.Commit(); err != nil { + return err + } + case binlogdatapb.OnDDLAction_EXEC: + if err := vp.updatePos(event.Timestamp); err != nil { + return err + } + if err := vp.exec(event.Ddl); err != nil { + return err + } + if err := vp.dbClient.Commit(); err != nil { + return err + } + case binlogdatapb.OnDDLAction_EXEC_IGNORE: + if err := vp.updatePos(event.Timestamp); err != nil { + return err + } + if err := vp.exec(event.Ddl); err != nil { + log.Infof("Ignoring error: %v for DDL: %s", err, event.Ddl) + } + if err := vp.dbClient.Commit(); err != nil { + return err + } + } } return nil } @@ -174,7 +216,7 @@ func (vp *vplayer) setState(state, message string) { } } -func (vp *vplayer) updatePlans(fieldEvent *binlogdatapb.FieldEvent) error { +func (vp *vplayer) updatePlan(fieldEvent *binlogdatapb.FieldEvent) error { prelim := vp.pplan.tablePlans[fieldEvent.TableName] tplan := &tablePlan{ name: fieldEvent.TableName, @@ -234,3 +276,173 @@ func (vp *vplayer) updatePlans(fieldEvent *binlogdatapb.FieldEvent) error { vp.tplans[fieldEvent.TableName] = tplan return nil } + +func (vp *vplayer) applyRowEvent(rowEvent *binlogdatapb.RowEvent) error { + tplan := vp.tplans[rowEvent.TableName] + if tplan != nil { + return fmt.Errorf("unexpected event on table %s", rowEvent.TableName) + } + for _, change := range rowEvent.RowChanges { + if err := vp.applyRowChange(tplan, change); err != nil { + return err + } + } + return nil +} + +func (vp *vplayer) applyRowChange(tplan *tablePlan, rowChange *binlogdatapb.RowChange) error { + // MakeRowTrusted is needed here because because Proto3ToResult is not convenient. + var before, after []sqltypes.Value + if rowChange.Before != nil { + before = sqltypes.MakeRowTrusted(tplan.fields, rowChange.Before) + } + if rowChange.After != nil { + after = sqltypes.MakeRowTrusted(tplan.fields, rowChange.After) + } + var query string + switch { + case before == nil && after != nil: + query = vp.generateInsert(tplan, after) + case before != nil && after != nil: + query = vp.generateUpdate(tplan, before, after) + case before != nil && after == nil: + query = vp.generateDelete(tplan, before) + case before == nil && after == nil: + // unreachable + } + return vp.exec(query) +} + +func (vp *vplayer) generateInsert(tplan *tablePlan, after []sqltypes.Value) string { + sql := sqlparser.NewTrackedBuffer(nil) + if tplan.onInsert == insertIgnore { + sql.Myprintf("insert ignore into %s set ", sqlparser.NewTableIdent(tplan.name)) + } else { + sql.Myprintf("insert into %s set ", sqlparser.NewTableIdent(tplan.name)) + } + vp.writeInsertValues(sql, tplan, after) + if tplan.onInsert == insertOndup { + sql.Myprintf(" on duplicate key update ") + vp.writeUpdateValues(sql, tplan, nil, after) + } + return sql.String() +} + +func (vp *vplayer) generateUpdate(tplan *tablePlan, before, after []sqltypes.Value) string { + if tplan.onInsert == insertIgnore { + return "" + } + sql := sqlparser.NewTrackedBuffer(nil) + sql.Myprintf("update %v set ", sqlparser.NewTableIdent(tplan.name)) + vp.writeUpdateValues(sql, tplan, before, after) + sql.Myprintf(" where ") + vp.writeWhereValues(sql, tplan, before) + return sql.String() +} + +func (vp *vplayer) generateDelete(tplan *tablePlan, before []sqltypes.Value) string { + sql := sqlparser.NewTrackedBuffer(nil) + if tplan.onInsert == insertNormal { + sql.Myprintf("delete from %v where ", sqlparser.NewTableIdent(tplan.name)) + vp.writeWhereValues(sql, tplan, before) + return sql.String() + } + sql.Myprintf("update %v set ", sqlparser.NewTableIdent(tplan.name)) + vp.writeUpdateValues(sql, tplan, before, nil) + sql.Myprintf(" where ") + vp.writeWhereValues(sql, tplan, before) + return sql.String() +} + +func (vp *vplayer) writeInsertValues(sql *sqlparser.TrackedBuffer, tplan *tablePlan, after []sqltypes.Value) { + separator := "" + for _, cExpr := range tplan.colExprs { + sql.Myprintf("%s%s=", separator, cExpr.colname) + if separator == "" { + separator = ", " + } + if cExpr.op == opCount { + sql.WriteString("1") + } else { + encodeValue(sql, after[cExpr.colnum]) + } + } +} + +func (vp *vplayer) writeUpdateValues(sql *sqlparser.TrackedBuffer, tplan *tablePlan, before, after []sqltypes.Value) { + separator := "" + for _, cExpr := range tplan.colExprs { + if cExpr.isGrouped { + continue + } + sql.Myprintf("%s%s=", separator, cExpr.colname) + if separator == "" { + separator = ", " + } + if cExpr.op == opCount || cExpr.op == opSum { + sql.Myprintf("%s", cExpr.colname) + } + if len(before) != 0 { + switch cExpr.op { + case opNone: + if len(after) == 0 { + sql.WriteString("NULL") + } + case opCount: + sql.WriteString("-1") + case opSum: + sql.WriteString("-") + encodeValue(sql, before[cExpr.colnum]) + } + } + if len(after) != 0 { + switch cExpr.op { + case opNone: + encodeValue(sql, after[cExpr.colnum]) + case opCount: + sql.WriteString("+1") + case opSum: + sql.WriteString("+") + encodeValue(sql, after[cExpr.colnum]) + } + } + } +} + +func (vp *vplayer) writeWhereValues(sql *sqlparser.TrackedBuffer, tplan *tablePlan, before []sqltypes.Value) { + separator := "" + for _, cExpr := range tplan.pkCols { + sql.Myprintf("%s%s=", separator, cExpr.colname) + if separator == "" { + separator = " AND " + } + encodeValue(sql, before[cExpr.colnum]) + } +} +func (vp *vplayer) updatePos(ts int64) error { + updatePos := binlogplayer.GenerateUpdatePos(vp.id, vp.pos, time.Now().Unix(), ts) + if _, err := vp.dbClient.ExecuteFetch(updatePos, 0); err != nil { + _ = vp.dbClient.Rollback() + return fmt.Errorf("error %v updating position", err) + } + return nil +} + +func (vp *vplayer) exec(sql string) error { + vp.stats.Timings.Record("query", time.Now()) + _, err := vp.dbClient.ExecuteFetch(sql, 0) + return err +} + +func encodeValue(sql *sqlparser.TrackedBuffer, value sqltypes.Value) { + if value.Type() == querypb.Type_TIMESTAMP && !bytes.HasPrefix(value.ToBytes(), mysql.ZeroTimestamp) { + // Values in the binary log are UTC. Let's convert them + // to whatever timezone the connection is using, + // so MySQL properly converts them back to UTC. + sql.WriteString("convert_tz(") + value.EncodeSQL(sql) + sql.WriteString(", '+00:00', @@session.time_zone)") + } else { + value.EncodeSQL(sql) + } +} diff --git a/proto/binlogdata.proto b/proto/binlogdata.proto index 83f5abfca49..a1471e44f5c 100644 --- a/proto/binlogdata.proto +++ b/proto/binlogdata.proto @@ -132,6 +132,14 @@ message Filter { repeated Rule rules = 1; } +// OnDDLAction lists the possible actions for DDLs. +enum OnDDLAction { + IGNORE = 0; + STOP = 1; + EXEC = 2; + EXEC_IGNORE = 3; +} + // BinlogSource specifies the source and filter parameters for // Filtered Replication. It currently supports a keyrange // or a list of tables. @@ -154,6 +162,9 @@ message BinlogSource { // filter is set if we're using the generalized representation // for the filter. Filter filter = 6; + + // on_ddl specifies the action to be taken when a DDL is encountered. + OnDDLAction on_ddl = 7; } // VEventType enumerates the event types. diff --git a/py/vtproto/binlogdata_pb2.py b/py/vtproto/binlogdata_pb2.py index 0278b0f83f0..e586658b98d 100644 --- a/py/vtproto/binlogdata_pb2.py +++ b/py/vtproto/binlogdata_pb2.py @@ -23,10 +23,41 @@ package='binlogdata', syntax='proto3', serialized_options=_b('Z\'vitess.io/vitess/go/vt/proto/binlogdata'), - serialized_pb=_b('\n\x10\x62inlogdata.proto\x12\nbinlogdata\x1a\x0bvtrpc.proto\x1a\x0bquery.proto\x1a\x0etopodata.proto\"7\n\x07\x43harset\x12\x0e\n\x06\x63lient\x18\x01 \x01(\x05\x12\x0c\n\x04\x63onn\x18\x02 \x01(\x05\x12\x0e\n\x06server\x18\x03 \x01(\x05\"\xb5\x03\n\x11\x42inlogTransaction\x12;\n\nstatements\x18\x01 \x03(\x0b\x32\'.binlogdata.BinlogTransaction.Statement\x12&\n\x0b\x65vent_token\x18\x04 \x01(\x0b\x32\x11.query.EventToken\x1a\xae\x02\n\tStatement\x12\x42\n\x08\x63\x61tegory\x18\x01 \x01(\x0e\x32\x30.binlogdata.BinlogTransaction.Statement.Category\x12$\n\x07\x63harset\x18\x02 \x01(\x0b\x32\x13.binlogdata.Charset\x12\x0b\n\x03sql\x18\x03 \x01(\x0c\"\xa9\x01\n\x08\x43\x61tegory\x12\x13\n\x0f\x42L_UNRECOGNIZED\x10\x00\x12\x0c\n\x08\x42L_BEGIN\x10\x01\x12\r\n\tBL_COMMIT\x10\x02\x12\x0f\n\x0b\x42L_ROLLBACK\x10\x03\x12\x15\n\x11\x42L_DML_DEPRECATED\x10\x04\x12\n\n\x06\x42L_DDL\x10\x05\x12\n\n\x06\x42L_SET\x10\x06\x12\r\n\tBL_INSERT\x10\x07\x12\r\n\tBL_UPDATE\x10\x08\x12\r\n\tBL_DELETE\x10\tJ\x04\x08\x02\x10\x03J\x04\x08\x03\x10\x04\"v\n\x15StreamKeyRangeRequest\x12\x10\n\x08position\x18\x01 \x01(\t\x12%\n\tkey_range\x18\x02 \x01(\x0b\x32\x12.topodata.KeyRange\x12$\n\x07\x63harset\x18\x03 \x01(\x0b\x32\x13.binlogdata.Charset\"S\n\x16StreamKeyRangeResponse\x12\x39\n\x12\x62inlog_transaction\x18\x01 \x01(\x0b\x32\x1d.binlogdata.BinlogTransaction\"]\n\x13StreamTablesRequest\x12\x10\n\x08position\x18\x01 \x01(\t\x12\x0e\n\x06tables\x18\x02 \x03(\t\x12$\n\x07\x63harset\x18\x03 \x01(\x0b\x32\x13.binlogdata.Charset\"Q\n\x14StreamTablesResponse\x12\x39\n\x12\x62inlog_transaction\x18\x01 \x01(\x0b\x32\x1d.binlogdata.BinlogTransaction\"%\n\x04Rule\x12\r\n\x05match\x18\x01 \x01(\t\x12\x0e\n\x06\x66ilter\x18\x02 \x01(\t\")\n\x06\x46ilter\x12\x1f\n\x05rules\x18\x01 \x03(\x0b\x32\x10.binlogdata.Rule\"\xb5\x01\n\x0c\x42inlogSource\x12\x10\n\x08keyspace\x18\x01 \x01(\t\x12\r\n\x05shard\x18\x02 \x01(\t\x12)\n\x0btablet_type\x18\x03 \x01(\x0e\x32\x14.topodata.TabletType\x12%\n\tkey_range\x18\x04 \x01(\x0b\x32\x12.topodata.KeyRange\x12\x0e\n\x06tables\x18\x05 \x03(\t\x12\"\n\x06\x66ilter\x18\x06 \x01(\x0b\x32\x12.binlogdata.Filter\"B\n\tRowChange\x12\x1a\n\x06\x62\x65\x66ore\x18\x01 \x01(\x0b\x32\n.query.Row\x12\x19\n\x05\x61\x66ter\x18\x02 \x01(\x0b\x32\n.query.Row\"J\n\x08RowEvent\x12\x12\n\ntable_name\x18\x01 \x01(\t\x12*\n\x0brow_changes\x18\x02 \x03(\x0b\x32\x15.binlogdata.RowChange\">\n\nFieldEvent\x12\x12\n\ntable_name\x18\x01 \x01(\t\x12\x1c\n\x06\x66ields\x18\x02 \x03(\x0b\x32\x0c.query.Field\"\xb2\x01\n\x06VEvent\x12$\n\x04type\x18\x01 \x01(\x0e\x32\x16.binlogdata.VEventType\x12\x11\n\ttimestamp\x18\x02 \x01(\x03\x12\x0c\n\x04gtid\x18\x03 \x01(\t\x12\x0b\n\x03\x64\x64l\x18\x04 \x01(\t\x12\'\n\trow_event\x18\x05 \x01(\x0b\x32\x14.binlogdata.RowEvent\x12+\n\x0b\x66ield_event\x18\x06 \x01(\x0b\x32\x16.binlogdata.FieldEvent\"\xc7\x01\n\x0eVStreamRequest\x12,\n\x13\x65\x66\x66\x65\x63tive_caller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12\x32\n\x13immediate_caller_id\x18\x02 \x01(\x0b\x32\x15.query.VTGateCallerID\x12\x1d\n\x06target\x18\x03 \x01(\x0b\x32\r.query.Target\x12\x10\n\x08position\x18\x04 \x01(\t\x12\"\n\x06\x66ilter\x18\x05 \x01(\x0b\x32\x12.binlogdata.Filter\"5\n\x0fVStreamResponse\x12\"\n\x06\x65vents\x18\x01 \x03(\x0b\x32\x12.binlogdata.VEvent*\xaa\x01\n\nVEventType\x12\x0b\n\x07UNKNOWN\x10\x00\x12\x08\n\x04GTID\x10\x01\x12\t\n\x05\x42\x45GIN\x10\x02\x12\n\n\x06\x43OMMIT\x10\x03\x12\x0c\n\x08ROLLBACK\x10\x04\x12\x07\n\x03\x44\x44L\x10\x05\x12\n\n\x06INSERT\x10\x06\x12\x0b\n\x07REPLACE\x10\x07\x12\n\n\x06UPDATE\x10\x08\x12\n\n\x06\x44\x45LETE\x10\t\x12\x07\n\x03SET\x10\n\x12\t\n\x05OTHER\x10\x0b\x12\x07\n\x03ROW\x10\x0c\x12\t\n\x05\x46IELD\x10\rB)Z\'vitess.io/vitess/go/vt/proto/binlogdatab\x06proto3') + serialized_pb=_b('\n\x10\x62inlogdata.proto\x12\nbinlogdata\x1a\x0bvtrpc.proto\x1a\x0bquery.proto\x1a\x0etopodata.proto\"7\n\x07\x43harset\x12\x0e\n\x06\x63lient\x18\x01 \x01(\x05\x12\x0c\n\x04\x63onn\x18\x02 \x01(\x05\x12\x0e\n\x06server\x18\x03 \x01(\x05\"\xb5\x03\n\x11\x42inlogTransaction\x12;\n\nstatements\x18\x01 \x03(\x0b\x32\'.binlogdata.BinlogTransaction.Statement\x12&\n\x0b\x65vent_token\x18\x04 \x01(\x0b\x32\x11.query.EventToken\x1a\xae\x02\n\tStatement\x12\x42\n\x08\x63\x61tegory\x18\x01 \x01(\x0e\x32\x30.binlogdata.BinlogTransaction.Statement.Category\x12$\n\x07\x63harset\x18\x02 \x01(\x0b\x32\x13.binlogdata.Charset\x12\x0b\n\x03sql\x18\x03 \x01(\x0c\"\xa9\x01\n\x08\x43\x61tegory\x12\x13\n\x0f\x42L_UNRECOGNIZED\x10\x00\x12\x0c\n\x08\x42L_BEGIN\x10\x01\x12\r\n\tBL_COMMIT\x10\x02\x12\x0f\n\x0b\x42L_ROLLBACK\x10\x03\x12\x15\n\x11\x42L_DML_DEPRECATED\x10\x04\x12\n\n\x06\x42L_DDL\x10\x05\x12\n\n\x06\x42L_SET\x10\x06\x12\r\n\tBL_INSERT\x10\x07\x12\r\n\tBL_UPDATE\x10\x08\x12\r\n\tBL_DELETE\x10\tJ\x04\x08\x02\x10\x03J\x04\x08\x03\x10\x04\"v\n\x15StreamKeyRangeRequest\x12\x10\n\x08position\x18\x01 \x01(\t\x12%\n\tkey_range\x18\x02 \x01(\x0b\x32\x12.topodata.KeyRange\x12$\n\x07\x63harset\x18\x03 \x01(\x0b\x32\x13.binlogdata.Charset\"S\n\x16StreamKeyRangeResponse\x12\x39\n\x12\x62inlog_transaction\x18\x01 \x01(\x0b\x32\x1d.binlogdata.BinlogTransaction\"]\n\x13StreamTablesRequest\x12\x10\n\x08position\x18\x01 \x01(\t\x12\x0e\n\x06tables\x18\x02 \x03(\t\x12$\n\x07\x63harset\x18\x03 \x01(\x0b\x32\x13.binlogdata.Charset\"Q\n\x14StreamTablesResponse\x12\x39\n\x12\x62inlog_transaction\x18\x01 \x01(\x0b\x32\x1d.binlogdata.BinlogTransaction\"%\n\x04Rule\x12\r\n\x05match\x18\x01 \x01(\t\x12\x0e\n\x06\x66ilter\x18\x02 \x01(\t\")\n\x06\x46ilter\x12\x1f\n\x05rules\x18\x01 \x03(\x0b\x32\x10.binlogdata.Rule\"\xde\x01\n\x0c\x42inlogSource\x12\x10\n\x08keyspace\x18\x01 \x01(\t\x12\r\n\x05shard\x18\x02 \x01(\t\x12)\n\x0btablet_type\x18\x03 \x01(\x0e\x32\x14.topodata.TabletType\x12%\n\tkey_range\x18\x04 \x01(\x0b\x32\x12.topodata.KeyRange\x12\x0e\n\x06tables\x18\x05 \x03(\t\x12\"\n\x06\x66ilter\x18\x06 \x01(\x0b\x32\x12.binlogdata.Filter\x12\'\n\x06on_ddl\x18\x07 \x01(\x0e\x32\x17.binlogdata.OnDDLAction\"B\n\tRowChange\x12\x1a\n\x06\x62\x65\x66ore\x18\x01 \x01(\x0b\x32\n.query.Row\x12\x19\n\x05\x61\x66ter\x18\x02 \x01(\x0b\x32\n.query.Row\"J\n\x08RowEvent\x12\x12\n\ntable_name\x18\x01 \x01(\t\x12*\n\x0brow_changes\x18\x02 \x03(\x0b\x32\x15.binlogdata.RowChange\">\n\nFieldEvent\x12\x12\n\ntable_name\x18\x01 \x01(\t\x12\x1c\n\x06\x66ields\x18\x02 \x03(\x0b\x32\x0c.query.Field\"\xb2\x01\n\x06VEvent\x12$\n\x04type\x18\x01 \x01(\x0e\x32\x16.binlogdata.VEventType\x12\x11\n\ttimestamp\x18\x02 \x01(\x03\x12\x0c\n\x04gtid\x18\x03 \x01(\t\x12\x0b\n\x03\x64\x64l\x18\x04 \x01(\t\x12\'\n\trow_event\x18\x05 \x01(\x0b\x32\x14.binlogdata.RowEvent\x12+\n\x0b\x66ield_event\x18\x06 \x01(\x0b\x32\x16.binlogdata.FieldEvent\"\xc7\x01\n\x0eVStreamRequest\x12,\n\x13\x65\x66\x66\x65\x63tive_caller_id\x18\x01 \x01(\x0b\x32\x0f.vtrpc.CallerID\x12\x32\n\x13immediate_caller_id\x18\x02 \x01(\x0b\x32\x15.query.VTGateCallerID\x12\x1d\n\x06target\x18\x03 \x01(\x0b\x32\r.query.Target\x12\x10\n\x08position\x18\x04 \x01(\t\x12\"\n\x06\x66ilter\x18\x05 \x01(\x0b\x32\x12.binlogdata.Filter\"5\n\x0fVStreamResponse\x12\"\n\x06\x65vents\x18\x01 \x03(\x0b\x32\x12.binlogdata.VEvent*>\n\x0bOnDDLAction\x12\n\n\x06IGNORE\x10\x00\x12\x08\n\x04STOP\x10\x01\x12\x08\n\x04\x45XEC\x10\x02\x12\x0f\n\x0b\x45XEC_IGNORE\x10\x03*\xaa\x01\n\nVEventType\x12\x0b\n\x07UNKNOWN\x10\x00\x12\x08\n\x04GTID\x10\x01\x12\t\n\x05\x42\x45GIN\x10\x02\x12\n\n\x06\x43OMMIT\x10\x03\x12\x0c\n\x08ROLLBACK\x10\x04\x12\x07\n\x03\x44\x44L\x10\x05\x12\n\n\x06INSERT\x10\x06\x12\x0b\n\x07REPLACE\x10\x07\x12\n\n\x06UPDATE\x10\x08\x12\n\n\x06\x44\x45LETE\x10\t\x12\x07\n\x03SET\x10\n\x12\t\n\x05OTHER\x10\x0b\x12\x07\n\x03ROW\x10\x0c\x12\t\n\x05\x46IELD\x10\rB)Z\'vitess.io/vitess/go/vt/proto/binlogdatab\x06proto3') , dependencies=[vtrpc__pb2.DESCRIPTOR,query__pb2.DESCRIPTOR,topodata__pb2.DESCRIPTOR,]) +_ONDDLACTION = _descriptor.EnumDescriptor( + name='OnDDLAction', + full_name='binlogdata.OnDDLAction', + filename=None, + file=DESCRIPTOR, + values=[ + _descriptor.EnumValueDescriptor( + name='IGNORE', index=0, number=0, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='STOP', index=1, number=1, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='EXEC', index=2, number=2, + serialized_options=None, + type=None), + _descriptor.EnumValueDescriptor( + name='EXEC_IGNORE', index=3, number=3, + serialized_options=None, + type=None), + ], + containing_type=None, + serialized_options=None, + serialized_start=1907, + serialized_end=1969, +) +_sym_db.RegisterEnumDescriptor(_ONDDLACTION) + +OnDDLAction = enum_type_wrapper.EnumTypeWrapper(_ONDDLACTION) _VEVENTTYPE = _descriptor.EnumDescriptor( name='VEventType', full_name='binlogdata.VEventType', @@ -92,12 +123,16 @@ ], containing_type=None, serialized_options=None, - serialized_start=1867, - serialized_end=2037, + serialized_start=1972, + serialized_end=2142, ) _sym_db.RegisterEnumDescriptor(_VEVENTTYPE) VEventType = enum_type_wrapper.EnumTypeWrapper(_VEVENTTYPE) +IGNORE = 0 +STOP = 1 +EXEC = 2 +EXEC_IGNORE = 3 UNKNOWN = 0 GTID = 1 BEGIN = 2 @@ -567,6 +602,13 @@ message_type=None, enum_type=None, containing_type=None, is_extension=False, extension_scope=None, serialized_options=None, file=DESCRIPTOR), + _descriptor.FieldDescriptor( + name='on_ddl', full_name='binlogdata.BinlogSource.on_ddl', index=6, + number=7, type=14, cpp_type=8, label=1, + has_default_value=False, default_value=0, + message_type=None, enum_type=None, containing_type=None, + is_extension=False, extension_scope=None, + serialized_options=None, file=DESCRIPTOR), ], extensions=[ ], @@ -580,7 +622,7 @@ oneofs=[ ], serialized_start=1037, - serialized_end=1218, + serialized_end=1259, ) @@ -617,8 +659,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1220, - serialized_end=1286, + serialized_start=1261, + serialized_end=1327, ) @@ -655,8 +697,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1288, - serialized_end=1362, + serialized_start=1329, + serialized_end=1403, ) @@ -693,8 +735,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1364, - serialized_end=1426, + serialized_start=1405, + serialized_end=1467, ) @@ -759,8 +801,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1429, - serialized_end=1607, + serialized_start=1470, + serialized_end=1648, ) @@ -818,8 +860,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1610, - serialized_end=1809, + serialized_start=1651, + serialized_end=1850, ) @@ -849,8 +891,8 @@ extension_ranges=[], oneofs=[ ], - serialized_start=1811, - serialized_end=1864, + serialized_start=1852, + serialized_end=1905, ) _BINLOGTRANSACTION_STATEMENT.fields_by_name['category'].enum_type = _BINLOGTRANSACTION_STATEMENT_CATEGORY @@ -868,6 +910,7 @@ _BINLOGSOURCE.fields_by_name['tablet_type'].enum_type = topodata__pb2._TABLETTYPE _BINLOGSOURCE.fields_by_name['key_range'].message_type = topodata__pb2._KEYRANGE _BINLOGSOURCE.fields_by_name['filter'].message_type = _FILTER +_BINLOGSOURCE.fields_by_name['on_ddl'].enum_type = _ONDDLACTION _ROWCHANGE.fields_by_name['before'].message_type = query__pb2._ROW _ROWCHANGE.fields_by_name['after'].message_type = query__pb2._ROW _ROWEVENT.fields_by_name['row_changes'].message_type = _ROWCHANGE @@ -895,6 +938,7 @@ DESCRIPTOR.message_types_by_name['VEvent'] = _VEVENT DESCRIPTOR.message_types_by_name['VStreamRequest'] = _VSTREAMREQUEST DESCRIPTOR.message_types_by_name['VStreamResponse'] = _VSTREAMRESPONSE +DESCRIPTOR.enum_types_by_name['OnDDLAction'] = _ONDDLACTION DESCRIPTOR.enum_types_by_name['VEventType'] = _VEVENTTYPE _sym_db.RegisterFileDescriptor(DESCRIPTOR) From 0fe23126ac2f1028a660a8970ddb0800e8601b85 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Wed, 9 Jan 2019 00:03:01 -0800 Subject: [PATCH 04/27] vreplication: refactored vstreamer test for reuse Signed-off-by: Sugu Sougoumarane --- .../tabletserver/vstreamer/engine_test.go | 4 +- .../tabletserver/vstreamer/main_test.go | 123 ++-------------- .../vstreamer/planbuilder_test.go | 2 +- .../tabletserver/vstreamer/testenv/testenv.go | 138 ++++++++++++++++++ .../tabletserver/vstreamer/vstreamer_test.go | 14 +- 5 files changed, 161 insertions(+), 120 deletions(-) create mode 100644 go/vt/vttablet/tabletserver/vstreamer/testenv/testenv.go diff --git a/go/vt/vttablet/tabletserver/vstreamer/engine_test.go b/go/vt/vttablet/tabletserver/vstreamer/engine_test.go index 1e209663b77..c88a9aeef40 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/engine_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/engine_test.go @@ -50,7 +50,7 @@ func TestUpdateVSchema(t *testing.T) { t.Skip() } - defer setVSchema("{}") + defer env.SetVSchema("{}") // We have to start at least one stream to start the vschema watcher. ctx, cancel := context.WithCancel(context.Background()) @@ -67,7 +67,7 @@ func TestUpdateVSchema(t *testing.T) { startCount := expectUpdateCount(t, 1) - if err := setVSchema(shardedVSchema); err != nil { + if err := env.SetVSchema(shardedVSchema); err != nil { t.Fatal(err) } expectUpdateCount(t, startCount+1) diff --git a/go/vt/vttablet/tabletserver/vstreamer/main_test.go b/go/vt/vttablet/tabletserver/vstreamer/main_test.go index d308e431690..224091078f0 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/main_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/main_test.go @@ -20,45 +20,16 @@ import ( "flag" "fmt" "os" - "path" "testing" - "golang.org/x/net/context" - "vitess.io/vitess/go/json2" - "vitess.io/vitess/go/mysql" - "vitess.io/vitess/go/vt/dbconfigs" - "vitess.io/vitess/go/vt/logutil" - "vitess.io/vitess/go/vt/mysqlctl" - "vitess.io/vitess/go/vt/srvtopo" - "vitess.io/vitess/go/vt/topo" - "vitess.io/vitess/go/vt/topo/memorytopo" - "vitess.io/vitess/go/vt/topotools" - "vitess.io/vitess/go/vt/vttablet/tabletserver/connpool" - "vitess.io/vitess/go/vt/vttablet/tabletserver/schema" - "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv" - "vitess.io/vitess/go/vt/vttest" - - topodatapb "vitess.io/vitess/go/vt/proto/topodata" - vschemapb "vitess.io/vitess/go/vt/proto/vschema" - vttestpb "vitess.io/vitess/go/vt/proto/vttest" + "vitess.io/vitess/go/vt/vttablet/tabletserver/vstreamer/testenv" ) var ( - engine *Engine - mysqld *mysqlctl.Mysqld - connParams mysql.ConnParams - connAppDebugParams mysql.ConnParams - topoServ *topo.Server - keyspaceName = "vttest" - cells = []string{"cell1"} + engine *Engine + env *testenv.Env ) -type checker struct{} - -var _ = connpool.MySQLChecker(checker{}) - -func (checker) CheckMySQL() {} - func TestMain(m *testing.M) { flag.Parse() // Do not remove this comment, import into google3 depends on it @@ -67,90 +38,22 @@ func TestMain(m *testing.M) { } 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: keyspaceName, - Shards: []*vttestpb.Shard{ - { - Name: "0", - DbNameOverride: "vttest", - }, - }, - }, - }, - }, - ExtraMyCnf: []string{path.Join(os.Getenv("VTTOP"), "config/mycnf/rbr.cnf")}, - OnlyMySQL: true, - } - 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() - - // initTopo initializes topoServ. - if err := initEngine(&cluster); err != nil { + var err error + env, err = testenv.Init() + if err != nil { fmt.Fprintf(os.Stderr, "%v", err) return 1 } + defer env.Close() + + // engine cannot be initialized in testenv because it introduces + // circular dependencies. + engine = NewEngine(env.SrvTopo, env.SchemaEngine) + engine.InitDBConfig(env.Dbcfgs) + engine.Open(env.KeyspaceName, env.Cells[0]) defer engine.Close() return m.Run() }() os.Exit(exitCode) } - -func initEngine(cluster *vttest.LocalCluster) error { - if err := initTopo(); err != nil { - return err - } - - se := schema.NewEngine(checker{}, tabletenv.DefaultQsConfig) - srvTopoServer := srvtopo.NewResilientServer(topoServ, "TestTopo") - engine = NewEngine(srvTopoServer, se) - - dbcfgs := dbconfigs.NewTestDBConfigs(cluster.MySQLConnParams(), cluster.MySQLAppDebugConnParams(), cluster.DbName()) - mysqld = mysqlctl.NewMysqld(dbcfgs) - se.InitDBConfig(dbcfgs) - engine.InitDBConfig(dbcfgs) - - engine.Open(keyspaceName, cells[0]) - return nil -} - -func initTopo() error { - ctx := context.Background() - - topoServ = memorytopo.NewServer(cells...) - if err := topoServ.CreateKeyspace(ctx, keyspaceName, &topodatapb.Keyspace{}); err != nil { - return err - } - // The first vschema should not be empty. Leads to Node not found error. - // TODO(sougou): need to fix the bug. - return setVSchema(`{"sharded": true}`) -} - -func setVSchema(vs string) error { - ctx := context.Background() - logger := logutil.NewConsoleLogger() - var kspb vschemapb.Keyspace - if err := json2.Unmarshal([]byte(vs), &kspb); err != nil { - return fmt.Errorf("Unmarshal failed: %v", err) - } - if err := topoServ.SaveVSchema(ctx, keyspaceName, &kspb); err != nil { - return fmt.Errorf("SaveVSchema failed: %v", err) - } - if err := topotools.RebuildVSchema(ctx, logger, topoServ, cells); err != nil { - return fmt.Errorf("RebuildVSchema failed: %v", err) - } - return nil -} diff --git a/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go b/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go index 1dda4116101..f3bf8d4aaa4 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go @@ -60,7 +60,7 @@ func init() { if err := json2.Unmarshal([]byte(input), &kspb); err != nil { panic(fmt.Errorf("Unmarshal failed: %v", err)) } - kschema, err := vindexes.BuildKeyspaceSchema(&kspb, keyspaceName) + kschema, err := vindexes.BuildKeyspaceSchema(&kspb, "ks") if err != nil { panic(err) } diff --git a/go/vt/vttablet/tabletserver/vstreamer/testenv/testenv.go b/go/vt/vttablet/tabletserver/vstreamer/testenv/testenv.go new file mode 100644 index 00000000000..036bcd2e77f --- /dev/null +++ b/go/vt/vttablet/tabletserver/vstreamer/testenv/testenv.go @@ -0,0 +1,138 @@ +/* +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 testenv supplies test functions for testing vstreamer. +package testenv + +import ( + "context" + "fmt" + "os" + "path" + + "vitess.io/vitess/go/json2" + "vitess.io/vitess/go/vt/dbconfigs" + "vitess.io/vitess/go/vt/logutil" + "vitess.io/vitess/go/vt/mysqlctl" + "vitess.io/vitess/go/vt/srvtopo" + "vitess.io/vitess/go/vt/topo" + "vitess.io/vitess/go/vt/topo/memorytopo" + "vitess.io/vitess/go/vt/topotools" + "vitess.io/vitess/go/vt/vttablet/tabletserver/connpool" + "vitess.io/vitess/go/vt/vttablet/tabletserver/schema" + "vitess.io/vitess/go/vt/vttablet/tabletserver/tabletenv" + "vitess.io/vitess/go/vt/vttest" + + topodatapb "vitess.io/vitess/go/vt/proto/topodata" + vschemapb "vitess.io/vitess/go/vt/proto/vschema" + vttestpb "vitess.io/vitess/go/vt/proto/vttest" +) + +// Env contains all the env vars for a test against a mysql instance. +type Env struct { + cluster *vttest.LocalCluster + + KeyspaceName string + Cells []string + + TopoServ *topo.Server + SrvTopo srvtopo.Server + Dbcfgs *dbconfigs.DBConfigs + Mysqld *mysqlctl.Mysqld + SchemaEngine *schema.Engine +} + +type checker struct{} + +var _ = connpool.MySQLChecker(checker{}) + +func (checker) CheckMySQL() {} + +// Init initializes an Env. +func Init() (*Env, error) { + te := &Env{ + KeyspaceName: "vttest", + Cells: []string{"cell1"}, + } + + ctx := context.Background() + te.TopoServ = memorytopo.NewServer(te.Cells...) + if err := te.TopoServ.CreateKeyspace(ctx, te.KeyspaceName, &topodatapb.Keyspace{}); err != nil { + return nil, err + } + te.SrvTopo = srvtopo.NewResilientServer(te.TopoServ, "TestTopo") + + cfg := vttest.Config{ + Topology: &vttestpb.VTTestTopology{ + Keyspaces: []*vttestpb.Keyspace{ + { + Name: te.KeyspaceName, + Shards: []*vttestpb.Shard{ + { + Name: "0", + DbNameOverride: "vttest", + }, + }, + }, + }, + }, + ExtraMyCnf: []string{path.Join(os.Getenv("VTTOP"), "config/mycnf/rbr.cnf")}, + OnlyMySQL: true, + } + te.cluster = &vttest.LocalCluster{ + Config: cfg, + } + if err := te.cluster.Setup(); err != nil { + os.RemoveAll(te.cluster.Config.SchemaDir) + return nil, fmt.Errorf("could not launch mysql: %v", err) + } + + te.Dbcfgs = dbconfigs.NewTestDBConfigs(te.cluster.MySQLConnParams(), te.cluster.MySQLAppDebugConnParams(), te.cluster.DbName()) + te.Mysqld = mysqlctl.NewMysqld(te.Dbcfgs) + te.SchemaEngine = schema.NewEngine(checker{}, tabletenv.DefaultQsConfig) + te.SchemaEngine.InitDBConfig(te.Dbcfgs) + + // The first vschema should not be empty. Leads to Node not found error. + // TODO(sougou): need to fix the bug. + if err := te.SetVSchema(`{"sharded": true}`); err != nil { + te.Close() + return nil, err + } + + return te, nil +} + +// Close tears down TestEnv. +func (te *Env) Close() { + te.SchemaEngine.Close() + te.Mysqld.Close() + te.cluster.TearDown() + os.RemoveAll(te.cluster.Config.SchemaDir) +} + +// SetVSchema sets the vschema for the test keyspace. +func (te *Env) SetVSchema(vs string) error { + ctx := context.Background() + logger := logutil.NewConsoleLogger() + var kspb vschemapb.Keyspace + if err := json2.Unmarshal([]byte(vs), &kspb); err != nil { + return err + } + if err := te.TopoServ.SaveVSchema(ctx, te.KeyspaceName, &kspb); err != nil { + return err + } + return topotools.RebuildVSchema(ctx, logger, te.TopoServ, te.Cells) +} diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go index 55e68399286..03dfe5470ba 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go @@ -199,10 +199,10 @@ func TestREKeyrange(t *testing.T) { } ch := startStream(ctx, t, filter) - if err := setVSchema(shardedVSchema); err != nil { + if err := env.SetVSchema(shardedVSchema); err != nil { t.Fatal(err) } - defer setVSchema("{}") + defer env.SetVSchema("{}") // 1, 2, 3 and 5 are in shard -80. // 4 and 6 are in shard 80-. @@ -249,7 +249,7 @@ func TestREKeyrange(t *testing.T) { } } }` - if err := setVSchema(altVSchema); err != nil { + if err := env.SetVSchema(altVSchema); err != nil { t.Fatal(err) } @@ -719,7 +719,7 @@ func TestJSON(t *testing.T) { } // JSON is supported only after mysql57. - if err := mysqld.ExecuteSuperQuery(context.Background(), "create table vitess_json(id int default 1, val json, primary key(id))"); err != nil { + if err := env.Mysqld.ExecuteSuperQuery(context.Background(), "create table vitess_json(id int default 1, val json, primary key(id))"); err != nil { // If it's a syntax error, MySQL is an older version. Skip this test. if strings.Contains(err.Error(), "syntax") { return @@ -954,21 +954,21 @@ func vstream(ctx context.Context, t *testing.T, pos string, filter *binlogdatapb func execStatement(t *testing.T, query string) { t.Helper() - if err := mysqld.ExecuteSuperQuery(context.Background(), query); err != nil { + if err := env.Mysqld.ExecuteSuperQuery(context.Background(), query); err != nil { t.Fatal(err) } } func execStatements(t *testing.T, queries []string) { t.Helper() - if err := mysqld.ExecuteSuperQueryList(context.Background(), queries); err != nil { + if err := env.Mysqld.ExecuteSuperQueryList(context.Background(), queries); err != nil { t.Fatal(err) } } func masterPosition(t *testing.T) string { t.Helper() - pos, err := mysqld.MasterPosition() + pos, err := env.Mysqld.MasterPosition() if err != nil { t.Fatal(err) } From 226aff463b514e48269bd3937986c997b12d61c4 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Thu, 10 Jan 2019 10:53:13 -0800 Subject: [PATCH 05/27] vreplication: preliminary vplayer tests Signed-off-by: Sugu Sougoumarane --- .../fakemysqldaemon/fakemysqldaemon.go | 5 + go/vt/mysqlctl/mysql_daemon.go | 1 + .../tabletmanager/vreplication/controller.go | 12 +- .../vreplication/controller_test.go | 69 +++++---- .../tabletmanager/vreplication/engine_test.go | 52 ++++--- .../vreplication/framework_test.go | 131 ++++++++++-------- .../tabletmanager/vreplication/player_plan.go | 8 +- .../vreplication/tablet_picker_test.go | 39 +++--- .../tabletmanager/vreplication/vplayer.go | 19 +-- .../vreplication/vplayer_test.go | 125 +++++++++++++++++ .../tabletserver/vstreamer/testenv/testenv.go | 5 + .../tabletserver/vstreamer/vstreamer.go | 3 + 12 files changed, 314 insertions(+), 155 deletions(-) create mode 100644 go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go diff --git a/go/vt/mysqlctl/fakemysqldaemon/fakemysqldaemon.go b/go/vt/mysqlctl/fakemysqldaemon/fakemysqldaemon.go index a7d945f06b6..e2dda296fb0 100644 --- a/go/vt/mysqlctl/fakemysqldaemon/fakemysqldaemon.go +++ b/go/vt/mysqlctl/fakemysqldaemon/fakemysqldaemon.go @@ -383,6 +383,11 @@ func (fmd *FakeMysqlDaemon) GetSchema(dbName string, tables, excludeTables []str return tmutils.FilterTables(fmd.Schema, tables, excludeTables, includeViews) } +// GetPrimaryKeyColumns is part of the MysqlDaemon interface +func (fmd *FakeMysqlDaemon) GetPrimaryKeyColumns(dbName, table string) ([]string, error) { + return []string{}, nil +} + // PreflightSchemaChange is part of the MysqlDaemon interface func (fmd *FakeMysqlDaemon) PreflightSchemaChange(dbName string, changes []string) ([]*tabletmanagerdatapb.SchemaChangeResult, error) { if fmd.PreflightSchemaChangeResult == nil { diff --git a/go/vt/mysqlctl/mysql_daemon.go b/go/vt/mysqlctl/mysql_daemon.go index c8426e8915b..49b30dbf65e 100644 --- a/go/vt/mysqlctl/mysql_daemon.go +++ b/go/vt/mysqlctl/mysql_daemon.go @@ -69,6 +69,7 @@ type MysqlDaemon interface { // Schema related methods GetSchema(dbName string, tables, excludeTables []string, includeViews bool) (*tabletmanagerdatapb.SchemaDefinition, error) + GetPrimaryKeyColumns(dbName, table string) ([]string, error) PreflightSchemaChange(dbName string, changes []string) ([]*tabletmanagerdatapb.SchemaChangeResult, error) ApplySchemaChange(dbName string, change *tmutils.SchemaChange) (*tabletmanagerdatapb.SchemaChangeResult, error) diff --git a/go/vt/vttablet/tabletmanager/vreplication/controller.go b/go/vt/vttablet/tabletmanager/vreplication/controller.go index 3379ba39743..200492e0cdf 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/controller.go +++ b/go/vt/vttablet/tabletmanager/vreplication/controller.go @@ -169,7 +169,8 @@ func (ct *controller) runBlp(ctx context.Context) (err error) { } ct.sourceTablet.Set(tablet.Alias.String()) - if len(ct.source.Tables) > 0 { + switch { + case len(ct.source.Tables) > 0: // Table names can have search patterns. Resolve them against the schema. tables, err := mysqlctl.ResolveTables(ct.mysqld, dbClient.DBName(), ct.source.Tables) if err != nil { @@ -178,9 +179,14 @@ func (ct *controller) runBlp(ctx context.Context) (err error) { player := binlogplayer.NewBinlogPlayerTables(dbClient, tablet, tables, ct.id, ct.blpStats) return player.ApplyBinlogEvents(ctx) + case ct.source.KeyRange != nil: + player := binlogplayer.NewBinlogPlayerKeyRange(dbClient, tablet, ct.source.KeyRange, ct.id, ct.blpStats) + return player.ApplyBinlogEvents(ctx) + case ct.source.Filter != nil: + player := newVPlayer(ct.id, &ct.source, tablet, ct.blpStats, dbClient, ct.mysqld) + return player.Play(ctx) } - player := binlogplayer.NewBinlogPlayerKeyRange(dbClient, tablet, ct.source.KeyRange, ct.id, ct.blpStats) - return player.ApplyBinlogEvents(ctx) + return fmt.Errorf("missing source") } func (ct *controller) Stop() { diff --git a/go/vt/vttablet/tabletmanager/vreplication/controller_test.go b/go/vt/vttablet/tabletmanager/vreplication/controller_test.go index 52c2a0f0b69..e261444ee6c 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/controller_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/controller_test.go @@ -18,6 +18,7 @@ package vreplication import ( "errors" + "fmt" "testing" "time" @@ -51,14 +52,14 @@ var ( ) func TestControllerKeyRange(t *testing.T) { - ts := createTopo() - fbc := newFakeBinlogClient() - wantTablet := addTablet(ts, 100, "0", topodatapb.TabletType_REPLICA, true, true) + resetBinlogClient() + wantTablet := addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true) + defer deleteTablet(wantTablet) params := map[string]string{ "id": "1", "state": binlogplayer.BlpRunning, - "source": `keyspace:"ks" shard:"0" key_range: `, + "source": fmt.Sprintf(`keyspace:"%s" shard:"0" key_range: `, env.KeyspaceName), } dbClient := binlogplayer.NewMockDBClient(t) @@ -72,7 +73,7 @@ func TestControllerKeyRange(t *testing.T) { dbClientFactory := func() binlogplayer.DBClient { return dbClient } mysqld := &fakemysqldaemon.FakeMysqlDaemon{MysqlPort: 3306} - ct, err := newController(context.Background(), params, dbClientFactory, mysqld, ts, testCell, "replica", nil) + ct, err := newController(context.Background(), params, dbClientFactory, mysqld, env.TopoServ, env.Cells[0], "replica", nil) if err != nil { t.Fatal(err) } @@ -82,18 +83,18 @@ func TestControllerKeyRange(t *testing.T) { }() dbClient.Wait() - expectFBCRequest(t, fbc, wantTablet, testPos, nil, &topodatapb.KeyRange{End: []byte{0x80}}) + expectFBCRequest(t, wantTablet, testPos, nil, &topodatapb.KeyRange{End: []byte{0x80}}) } func TestControllerTables(t *testing.T) { - ts := createTopo() - wantTablet := addTablet(ts, 100, "0", topodatapb.TabletType_REPLICA, true, true) - fbc := newFakeBinlogClient() + wantTablet := addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true) + defer deleteTablet(wantTablet) + resetBinlogClient() params := map[string]string{ "id": "1", "state": binlogplayer.BlpRunning, - "source": `keyspace:"ks" shard:"0" tables:"table1" tables:"/funtables_/" `, + "source": fmt.Sprintf(`keyspace:"%s" shard:"0" tables:"table1" tables:"/funtables_/" `, env.KeyspaceName), } dbClient := binlogplayer.NewMockDBClient(t) @@ -132,7 +133,7 @@ func TestControllerTables(t *testing.T) { }, } - ct, err := newController(context.Background(), params, dbClientFactory, mysqld, ts, testCell, "replica", nil) + ct, err := newController(context.Background(), params, dbClientFactory, mysqld, env.TopoServ, env.Cells[0], "replica", nil) if err != nil { t.Fatal(err) } @@ -142,7 +143,7 @@ func TestControllerTables(t *testing.T) { }() dbClient.Wait() - expectFBCRequest(t, fbc, wantTablet, testPos, []string{"table1", "funtables_one"}, nil) + expectFBCRequest(t, wantTablet, testPos, []string{"table1", "funtables_one"}, nil) } func TestControllerBadID(t *testing.T) { @@ -176,15 +177,15 @@ func TestControllerStopped(t *testing.T) { } func TestControllerOverrides(t *testing.T) { - ts := createTopo() - fbc := newFakeBinlogClient() - wantTablet := addTablet(ts, 100, "0", topodatapb.TabletType_REPLICA, true, true) + resetBinlogClient() + wantTablet := addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true) + defer deleteTablet(wantTablet) params := map[string]string{ "id": "1", "state": binlogplayer.BlpRunning, - "source": `keyspace:"ks" shard:"0" key_range: `, - "cell": testCell, + "source": fmt.Sprintf(`keyspace:"%s" shard:"0" key_range: `, env.KeyspaceName), + "cell": env.Cells[0], "tablet_types": "replica", } @@ -199,7 +200,7 @@ func TestControllerOverrides(t *testing.T) { dbClientFactory := func() binlogplayer.DBClient { return dbClient } mysqld := &fakemysqldaemon.FakeMysqlDaemon{MysqlPort: 3306} - ct, err := newController(context.Background(), params, dbClientFactory, mysqld, ts, testCell, "rdonly", nil) + ct, err := newController(context.Background(), params, dbClientFactory, mysqld, env.TopoServ, env.Cells[0], "rdonly", nil) if err != nil { t.Fatal(err) } @@ -209,22 +210,21 @@ func TestControllerOverrides(t *testing.T) { }() dbClient.Wait() - expectFBCRequest(t, fbc, wantTablet, testPos, nil, &topodatapb.KeyRange{End: []byte{0x80}}) + expectFBCRequest(t, wantTablet, testPos, nil, &topodatapb.KeyRange{End: []byte{0x80}}) } func TestControllerCanceledContext(t *testing.T) { - ts := createTopo() - _ = addTablet(ts, 100, "0", topodatapb.TabletType_REPLICA, true, true) + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) params := map[string]string{ "id": "1", "state": binlogplayer.BlpRunning, - "source": `keyspace:"ks" shard:"0" key_range: `, + "source": fmt.Sprintf(`keyspace:"%s" shard:"0" key_range: `, env.KeyspaceName), } ctx, cancel := context.WithCancel(context.Background()) cancel() - ct, err := newController(ctx, params, nil, nil, ts, testCell, "rdonly", nil) + ct, err := newController(ctx, params, nil, nil, env.TopoServ, env.Cells[0], "rdonly", nil) if err != nil { t.Fatal(err) } @@ -242,15 +242,14 @@ func TestControllerRetry(t *testing.T) { defer func() { *retryDelay = savedDelay }() *retryDelay = 10 * time.Millisecond - ts := createTopo() - _ = newFakeBinlogClient() - _ = addTablet(ts, 100, "0", topodatapb.TabletType_REPLICA, true, true) + resetBinlogClient() + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) params := map[string]string{ "id": "1", "state": binlogplayer.BlpRunning, - "source": `keyspace:"ks" shard:"0" key_range: `, - "cell": testCell, + "source": fmt.Sprintf(`keyspace:"%s" shard:"0" key_range: `, env.KeyspaceName), + "cell": env.Cells[0], "tablet_types": "replica", } @@ -267,7 +266,7 @@ func TestControllerRetry(t *testing.T) { dbClientFactory := func() binlogplayer.DBClient { return dbClient } mysqld := &fakemysqldaemon.FakeMysqlDaemon{MysqlPort: 3306} - ct, err := newController(context.Background(), params, dbClientFactory, mysqld, ts, testCell, "rdonly", nil) + ct, err := newController(context.Background(), params, dbClientFactory, mysqld, env.TopoServ, env.Cells[0], "rdonly", nil) if err != nil { t.Fatal(err) } @@ -277,14 +276,14 @@ func TestControllerRetry(t *testing.T) { } func TestControllerStopPosition(t *testing.T) { - ts := createTopo() - fbc := newFakeBinlogClient() - wantTablet := addTablet(ts, 100, "0", topodatapb.TabletType_REPLICA, true, true) + resetBinlogClient() + wantTablet := addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true) + defer deleteTablet(wantTablet) params := map[string]string{ "id": "1", "state": binlogplayer.BlpRunning, - "source": `keyspace:"ks" shard:"0" key_range: `, + "source": fmt.Sprintf(`keyspace:"%s" shard:"0" key_range: `, env.KeyspaceName), } dbClient := binlogplayer.NewMockDBClient(t) @@ -312,7 +311,7 @@ func TestControllerStopPosition(t *testing.T) { dbClientFactory := func() binlogplayer.DBClient { return dbClient } mysqld := &fakemysqldaemon.FakeMysqlDaemon{MysqlPort: 3306} - ct, err := newController(context.Background(), params, dbClientFactory, mysqld, ts, testCell, "replica", nil) + ct, err := newController(context.Background(), params, dbClientFactory, mysqld, env.TopoServ, env.Cells[0], "replica", nil) if err != nil { t.Fatal(err) } @@ -329,5 +328,5 @@ func TestControllerStopPosition(t *testing.T) { } dbClient.Wait() - expectFBCRequest(t, fbc, wantTablet, testPos, nil, &topodatapb.KeyRange{End: []byte{0x80}}) + expectFBCRequest(t, wantTablet, testPos, nil, &topodatapb.KeyRange{End: []byte{0x80}}) } diff --git a/go/vt/vttablet/tabletmanager/vreplication/engine_test.go b/go/vt/vttablet/tabletmanager/vreplication/engine_test.go index f93ec5559e5..5b335a05a9a 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/engine_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/engine_test.go @@ -17,6 +17,7 @@ limitations under the License. package vreplication import ( + "fmt" "reflect" "testing" "time" @@ -32,16 +33,15 @@ import ( func TestEngineOpen(t *testing.T) { defer func() { globalStats = &vrStats{} }() - ts := createTopo() - _ = addTablet(ts, 100, "0", topodatapb.TabletType_REPLICA, true, true) - _ = newFakeBinlogClient() + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + resetBinlogClient() dbClient := binlogplayer.NewMockDBClient(t) dbClientFactory := func() binlogplayer.DBClient { return dbClient } mysqld := &fakemysqldaemon.FakeMysqlDaemon{MysqlPort: 3306} // Test Insert - vre := NewEngine(ts, testCell, mysqld, dbClientFactory) + vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory) if vre.IsOpen() { t.Errorf("IsOpen: %v, want false", vre.IsOpen()) } @@ -51,7 +51,7 @@ func TestEngineOpen(t *testing.T) { "id|state|source", "int64|varchar|varchar", ), - `1|Running|keyspace:"ks" shard:"0" key_range: `, + fmt.Sprintf(`1|Running|keyspace:"%s" shard:"0" key_range: `, env.KeyspaceName), ), 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 from _vt.vreplication where id=1", testSettingsResponse, nil) @@ -81,16 +81,15 @@ func TestEngineOpen(t *testing.T) { func TestEngineExec(t *testing.T) { defer func() { globalStats = &vrStats{} }() - ts := createTopo() - _ = addTablet(ts, 100, "0", topodatapb.TabletType_REPLICA, true, true) - _ = newFakeBinlogClient() + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + resetBinlogClient() dbClient := binlogplayer.NewMockDBClient(t) dbClientFactory := func() binlogplayer.DBClient { return dbClient } mysqld := &fakemysqldaemon.FakeMysqlDaemon{MysqlPort: 3306} // Test Insert - vre := NewEngine(ts, testCell, mysqld, dbClientFactory) + vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory) dbClient.ExpectRequest("select * from _vt.vreplication", &sqltypes.Result{}, nil) if err := vre.Open(context.Background()); err != nil { @@ -105,7 +104,7 @@ func TestEngineExec(t *testing.T) { "id|state|source", "int64|varchar|varchar", ), - `1|Running|keyspace:"ks" shard:"0" key_range: `, + fmt.Sprintf(`1|Running|keyspace:"%s" shard:"0" key_range: `, env.KeyspaceName), ), 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 from _vt.vreplication where id=1", testSettingsResponse, nil) @@ -145,7 +144,7 @@ func TestEngineExec(t *testing.T) { "id|state|source", "int64|varchar|varchar", ), - `1|Running|keyspace:"ks" shard:"0" key_range: `, + fmt.Sprintf(`1|Running|keyspace:"%s" shard:"0" key_range: `, env.KeyspaceName), ), 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 from _vt.vreplication where id=1", testSettingsResponse, nil) @@ -206,15 +205,14 @@ func TestEngineExec(t *testing.T) { func TestEngineBadInsert(t *testing.T) { defer func() { globalStats = &vrStats{} }() - ts := createTopo() - _ = addTablet(ts, 100, "0", topodatapb.TabletType_REPLICA, true, true) - _ = newFakeBinlogClient() + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + resetBinlogClient() dbClient := binlogplayer.NewMockDBClient(t) dbClientFactory := func() binlogplayer.DBClient { return dbClient } mysqld := &fakemysqldaemon.FakeMysqlDaemon{MysqlPort: 3306} - vre := NewEngine(ts, testCell, mysqld, dbClientFactory) + vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory) dbClient.ExpectRequest("select * from _vt.vreplication", &sqltypes.Result{}, nil) if err := vre.Open(context.Background()); err != nil { @@ -237,15 +235,14 @@ func TestEngineBadInsert(t *testing.T) { } func TestEngineSelect(t *testing.T) { - ts := createTopo() - _ = addTablet(ts, 100, "0", topodatapb.TabletType_REPLICA, true, true) - _ = newFakeBinlogClient() + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + resetBinlogClient() dbClient := binlogplayer.NewMockDBClient(t) dbClientFactory := func() binlogplayer.DBClient { return dbClient } mysqld := &fakemysqldaemon.FakeMysqlDaemon{MysqlPort: 3306} - vre := NewEngine(ts, testCell, mysqld, dbClientFactory) + vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory) dbClient.ExpectRequest("select * from _vt.vreplication", &sqltypes.Result{}, nil) if err := vre.Open(context.Background()); err != nil { @@ -260,7 +257,7 @@ func TestEngineSelect(t *testing.T) { "id|state|source|pos", "int64|varchar|varchar|varchar", ), - `1|Running|keyspace:"ks" shard:"0" key_range: |MariaDB/0-1-1083`, + fmt.Sprintf(`1|Running|keyspace:"%s" shard:"0" key_range: |MariaDB/0-1-1083`, env.KeyspaceName), ) dbClient.ExpectRequest(wantQuery, wantResult, nil) qr, err := vre.Exec(wantQuery) @@ -280,7 +277,7 @@ func TestWaitForPos(t *testing.T) { dbClient := binlogplayer.NewMockDBClient(t) mysqld := &fakemysqldaemon.FakeMysqlDaemon{MysqlPort: 3306} dbClientFactory := func() binlogplayer.DBClient { return dbClient } - vre := NewEngine(createTopo(), testCell, mysqld, dbClientFactory) + vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory) dbClient.ExpectRequest("select * from _vt.vreplication", &sqltypes.Result{}, nil) if err := vre.Open(context.Background()); err != nil { @@ -306,7 +303,7 @@ func TestWaitForPosError(t *testing.T) { dbClient := binlogplayer.NewMockDBClient(t) mysqld := &fakemysqldaemon.FakeMysqlDaemon{MysqlPort: 3306} dbClientFactory := func() binlogplayer.DBClient { return dbClient } - vre := NewEngine(createTopo(), testCell, mysqld, dbClientFactory) + vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory) err := vre.WaitForPos(context.Background(), 1, "MariaDB/0-1-1084") want := `vreplication engine is closed` @@ -348,7 +345,7 @@ func TestWaitForPosCancel(t *testing.T) { dbClient := binlogplayer.NewMockDBClient(t) mysqld := &fakemysqldaemon.FakeMysqlDaemon{MysqlPort: 3306} dbClientFactory := func() binlogplayer.DBClient { return dbClient } - vre := NewEngine(createTopo(), testCell, mysqld, dbClientFactory) + vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory) dbClient.ExpectRequest("select * from _vt.vreplication", &sqltypes.Result{}, nil) if err := vre.Open(context.Background()); err != nil { @@ -383,16 +380,15 @@ func TestWaitForPosCancel(t *testing.T) { func TestCreateDBAndTable(t *testing.T) { defer func() { globalStats = &vrStats{} }() - ts := createTopo() - _ = addTablet(ts, 100, "0", topodatapb.TabletType_REPLICA, true, true) - _ = newFakeBinlogClient() + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + resetBinlogClient() dbClient := binlogplayer.NewMockDBClient(t) dbClientFactory := func() binlogplayer.DBClient { return dbClient } mysqld := &fakemysqldaemon.FakeMysqlDaemon{MysqlPort: 3306} // Test Insert - vre := NewEngine(ts, testCell, mysqld, dbClientFactory) + vre := NewEngine(env.TopoServ, env.Cells[0], mysqld, dbClientFactory) tableNotFound := mysql.SQLError{Num: 1146, Message: "table not found"} dbClient.ExpectRequest("select * from _vt.vreplication", nil, &tableNotFound) @@ -432,7 +428,7 @@ func TestCreateDBAndTable(t *testing.T) { "id|state|source", "int64|varchar|varchar", ), - `1|Running|keyspace:"ks" shard:"0" key_range: `, + fmt.Sprintf(`1|Running|keyspace:"%s" shard:"0" key_range: `, env.KeyspaceName), ), 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 from _vt.vreplication where id=1", testSettingsResponse, nil) diff --git a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go index 559f285e1ff..c8ef077e56f 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go @@ -18,6 +18,8 @@ package vreplication import ( "flag" + "fmt" + "os" "reflect" "testing" @@ -27,49 +29,83 @@ import ( "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/grpcclient" "vitess.io/vitess/go/vt/topo" - "vitess.io/vitess/go/vt/topo/memorytopo" "vitess.io/vitess/go/vt/vttablet/queryservice" "vitess.io/vitess/go/vt/vttablet/queryservice/fakes" "vitess.io/vitess/go/vt/vttablet/tabletconn" + "vitess.io/vitess/go/vt/vttablet/tabletserver/vstreamer" + "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" topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) -const ( - testCell = "cell" - testKeyspace = "ks" - testShard = "0" +var ( + engine *vstreamer.Engine + env *testenv.Env + globalFBC = &fakeBinlogClient{} ) -// This file provides support functions for tests. -// It's capable of creating a single unsharded keyspace -// and allows you to add various tablet types. +func init() { + tabletconn.RegisterDialer("test", func(tablet *topodatapb.Tablet, failFast grpcclient.FailFast) (queryservice.QueryService, error) { + return &fakeTabletConn{ + QueryService: fakes.ErrorQueryService, + tablet: tablet, + }, nil + }) + flag.Set("tablet_protocol", "test") -//-------------------------------------- -// Topos and tablets + binlogplayer.RegisterClientFactory("test", func() binlogplayer.Client { return globalFBC }) + flag.Set("binlog_player_protocol", "test") +} -func createTopo() *topo.Server { - ts := memorytopo.NewServer(testCell) - ctx := context.Background() - if err := ts.CreateKeyspace(ctx, testKeyspace, &topodatapb.Keyspace{}); err != nil { - panic(err) - } - if err := ts.CreateShard(ctx, testKeyspace, testShard); err != nil { - panic(err) +func TestMain(m *testing.M) { + flag.Parse() // Do not remove this comment, import into google3 depends on it + + if testing.Short() { + os.Exit(m.Run()) } - return ts + + exitCode := func() int { + var err error + env, err = testenv.Init() + if err != nil { + fmt.Fprintf(os.Stderr, "%v", err) + return 1 + } + defer env.Close() + + // engine cannot be initialized in testenv because it introduces + // circular dependencies. + engine = vstreamer.NewEngine(env.SrvTopo, env.SchemaEngine) + engine.InitDBConfig(env.Dbcfgs) + engine.Open(env.KeyspaceName, env.Cells[0]) + defer engine.Close() + + return m.Run() + }() + os.Exit(exitCode) +} + +func resetBinlogClient() { + globalFBC = &fakeBinlogClient{} } -func addTablet(ts *topo.Server, id int, shard string, tabletType topodatapb.TabletType, serving, healthy bool) *topodatapb.Tablet { +//-------------------------------------- +// Topos and tablets + +func addTablet(id int, shard string, tabletType topodatapb.TabletType, serving, healthy bool) *topodatapb.Tablet { t := newTablet(id, shard, tabletType, serving, healthy) - if err := ts.CreateTablet(context.Background(), t); err != nil { + if err := env.TopoServ.CreateTablet(context.Background(), t); err != nil { panic(err) } return t } +func deleteTablet(t *topodatapb.Tablet) { + env.TopoServ.DeleteTablet(context.Background(), t.Alias) +} + func newTablet(id int, shard string, tabletType topodatapb.TabletType, serving, healthy bool) *topodatapb.Tablet { stag := "not_serving" if serving { @@ -85,11 +121,11 @@ func newTablet(id int, shard string, tabletType topodatapb.TabletType, serving, } return &topodatapb.Tablet{ Alias: &topodatapb.TabletAlias{ - Cell: testCell, + Cell: env.Cells[0], Uid: uint32(id), }, - Keyspace: testKeyspace, - Shard: testShard, + Keyspace: env.KeyspaceName, + Shard: env.ShardName, KeyRange: kr, Type: tabletType, Tags: map[string]string{ @@ -132,6 +168,11 @@ func (ftc *fakeTabletConn) StreamHealth(ctx context.Context, callback func(*quer return nil } +// VStream directly calls into the pre-initialized engine. +func (ftc *fakeTabletConn) VStream(ctx context.Context, target *querypb.Target, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { + return engine.Stream(ctx, startPos, filter, send) +} + //-------------------------------------- // Binlog Client to TabletManager @@ -145,11 +186,6 @@ type fakeBinlogClient struct { lastCharset *binlogdatapb.Charset } -func newFakeBinlogClient() *fakeBinlogClient { - globalFBC = &fakeBinlogClient{} - return globalFBC -} - func (fbc *fakeBinlogClient) Dial(tablet *topodatapb.Tablet) error { fbc.lastTablet = tablet return nil @@ -198,37 +234,18 @@ func (t *btStream) Recv() (*binlogdatapb.BinlogTransaction, error) { return nil, t.ctx.Err() } -func expectFBCRequest(t *testing.T, fbc *fakeBinlogClient, tablet *topodatapb.Tablet, pos string, tables []string, kr *topodatapb.KeyRange) { +func expectFBCRequest(t *testing.T, tablet *topodatapb.Tablet, pos string, tables []string, kr *topodatapb.KeyRange) { t.Helper() - if !proto.Equal(tablet, fbc.lastTablet) { - t.Errorf("Request tablet: %v, want %v", fbc.lastTablet, tablet) + if !proto.Equal(tablet, globalFBC.lastTablet) { + t.Errorf("Request tablet: %v, want %v", globalFBC.lastTablet, tablet) } - if pos != fbc.lastPos { - t.Errorf("Request pos: %v, want %v", fbc.lastPos, pos) + if pos != globalFBC.lastPos { + t.Errorf("Request pos: %v, want %v", globalFBC.lastPos, pos) } - if !reflect.DeepEqual(tables, fbc.lastTables) { - t.Errorf("Request tables: %v, want %v", fbc.lastTables, tables) + if !reflect.DeepEqual(tables, globalFBC.lastTables) { + t.Errorf("Request tables: %v, want %v", globalFBC.lastTables, tables) } - if !proto.Equal(kr, fbc.lastKeyRange) { - t.Errorf("Request KeyRange: %v, want %v", fbc.lastKeyRange, kr) + if !proto.Equal(kr, globalFBC.lastKeyRange) { + t.Errorf("Request KeyRange: %v, want %v", globalFBC.lastKeyRange, kr) } } - -//-------------------------------------- -// init - -// globalFBC is set by newFakeBinlogClient, which is then returned by the client factory below. -var globalFBC *fakeBinlogClient - -func init() { - tabletconn.RegisterDialer("test", func(tablet *topodatapb.Tablet, failFast grpcclient.FailFast) (queryservice.QueryService, error) { - return &fakeTabletConn{ - QueryService: fakes.ErrorQueryService, - tablet: tablet, - }, nil - }) - flag.Set("tablet_protocol", "test") - - binlogplayer.RegisterClientFactory("test", func() binlogplayer.Client { return globalFBC }) - flag.Set("binlog_player_protocol", "test") -} diff --git a/go/vt/vttablet/tabletmanager/vreplication/player_plan.go b/go/vt/vttablet/tabletmanager/vreplication/player_plan.go index 67b7c566972..25100ae3773 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/player_plan.go +++ b/go/vt/vttablet/tabletmanager/vreplication/player_plan.go @@ -92,7 +92,7 @@ func buildPlayerPlan(filter *binlogdatapb.Filter) (*playerPlan, error) { continue } plan.vstreamFilter.Rules[i] = sendRule - plan.tablePlans[rule.Match] = tplan + plan.tablePlans[sendRule.Match] = tplan } return plan, nil } @@ -129,7 +129,9 @@ func buildTablePlan(rule *binlogdatapb.Rule) (*binlogdatapb.Rule, *tablePlan, er return sendRule, &tablePlan{name: rule.Match}, nil } - tplan := &tablePlan{} + tplan := &tablePlan{ + name: rule.Match, + } sendSelect := &sqlparser.Select{ From: sel.From, Where: sel.Where, @@ -159,7 +161,7 @@ func buildTablePlan(rule *binlogdatapb.Rule) (*binlogdatapb.Rule, *tablePlan, er } } sendRule := &binlogdatapb.Rule{ - Match: rule.Match, + Match: fromTable.String(), Filter: sqlparser.String(sendSelect), } return sendRule, tplan, nil diff --git a/go/vt/vttablet/tabletmanager/vreplication/tablet_picker_test.go b/go/vt/vttablet/tabletmanager/vreplication/tablet_picker_test.go index 3bbec75cb5e..e3e4daf5e7b 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/tablet_picker_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/tablet_picker_test.go @@ -17,6 +17,7 @@ limitations under the License. package vreplication import ( + "fmt" "testing" "github.com/golang/protobuf/proto" @@ -26,11 +27,10 @@ import ( ) func TestPickSimple(t *testing.T) { - ts := createTopo() - defer ts.Close() - want := addTablet(ts, 100, "0", topodatapb.TabletType_REPLICA, true, true) + want := addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true) + defer deleteTablet(want) - tp, err := newTabletPicker(ts, testCell, testKeyspace, testShard, "replica") + tp, err := newTabletPicker(env.TopoServ, env.Cells[0], env.KeyspaceName, env.ShardName, "replica") if err != nil { t.Fatal(err) } @@ -46,12 +46,12 @@ func TestPickSimple(t *testing.T) { } func TestPickFromTwoHealthy(t *testing.T) { - ts := createTopo() - defer ts.Close() - want1 := addTablet(ts, 100, "0", topodatapb.TabletType_REPLICA, true, true) - want2 := addTablet(ts, 101, "0", topodatapb.TabletType_RDONLY, true, true) + want1 := addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true) + defer deleteTablet(want1) + want2 := addTablet(101, "0", topodatapb.TabletType_RDONLY, true, true) + defer deleteTablet(want2) - tp, err := newTabletPicker(ts, testCell, testKeyspace, testShard, "replica,rdonly") + tp, err := newTabletPicker(env.TopoServ, env.Cells[0], env.KeyspaceName, env.ShardName, "replica,rdonly") if err != nil { t.Fatal(err) } @@ -65,7 +65,7 @@ func TestPickFromTwoHealthy(t *testing.T) { t.Errorf("Pick:\n%v, want\n%v", tablet, want1) } - tp, err = newTabletPicker(ts, testCell, testKeyspace, testShard, "rdonly,replica") + tp, err = newTabletPicker(env.TopoServ, env.Cells[0], env.KeyspaceName, env.ShardName, "rdonly,replica") if err != nil { t.Fatal(err) } @@ -81,12 +81,11 @@ func TestPickFromTwoHealthy(t *testing.T) { } func TestPickFromSomeUnhealthy(t *testing.T) { - ts := createTopo() - defer ts.Close() - _ = addTablet(ts, 100, "0", topodatapb.TabletType_REPLICA, false, false) - want := addTablet(ts, 101, "0", topodatapb.TabletType_RDONLY, false, true) + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, false, false)) + want := addTablet(101, "0", topodatapb.TabletType_RDONLY, false, true) + defer deleteTablet(want) - tp, err := newTabletPicker(ts, testCell, testKeyspace, testShard, "replica,rdonly") + tp, err := newTabletPicker(env.TopoServ, env.Cells[0], env.KeyspaceName, env.ShardName, "replica,rdonly") if err != nil { t.Fatal(err) } @@ -102,24 +101,22 @@ func TestPickFromSomeUnhealthy(t *testing.T) { } func TestPickError(t *testing.T) { - ts := createTopo() - defer ts.Close() - _ = addTablet(ts, 100, "0", topodatapb.TabletType_REPLICA, false, false) + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, false, false)) - _, err := newTabletPicker(ts, testCell, testKeyspace, testShard, "badtype") + _, err := newTabletPicker(env.TopoServ, env.Cells[0], env.KeyspaceName, env.ShardName, "badtype") want := "failed to parse list of tablet types: badtype" if err == nil || err.Error() != want { t.Errorf("newTabletPicker err: %v, want %v", err, want) } - tp, err := newTabletPicker(ts, testCell, testKeyspace, testShard, "replica,rdonly") + tp, err := newTabletPicker(env.TopoServ, env.Cells[0], env.KeyspaceName, env.ShardName, "replica,rdonly") if err != nil { t.Fatal(err) } defer tp.Close() _, err = tp.Pick(context.Background()) - want = "can't find any healthy source tablet for ks 0 [REPLICA RDONLY]" + want = fmt.Sprintf("can't find any healthy source tablet for %s 0 [REPLICA RDONLY]", env.KeyspaceName) if err == nil || err.Error() != want { t.Errorf("Pick err: %v, want %v", err, want) } diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index 26bc1479acd..19fd8e08967 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -44,7 +44,7 @@ type vplayer struct { sourceTablet *topodatapb.Tablet stats *binlogplayer.Stats dbClient binlogplayer.DBClient - mysqld *mysqlctl.Mysqld + mysqld mysqlctl.MysqlDaemon pos mysql.Position stopPos mysql.Position @@ -54,7 +54,7 @@ type vplayer struct { retryDelay time.Duration } -func newVPlayer(id uint32, source *binlogdatapb.BinlogSource, sourceTablet *topodatapb.Tablet, stats *binlogplayer.Stats, dbClient binlogplayer.DBClient, mysqld *mysqlctl.Mysqld) *vplayer { +func newVPlayer(id uint32, source *binlogdatapb.BinlogSource, sourceTablet *topodatapb.Tablet, stats *binlogplayer.Stats, dbClient binlogplayer.DBClient, mysqld mysqlctl.MysqlDaemon) *vplayer { return &vplayer{ id: id, source: source, @@ -63,6 +63,7 @@ func newVPlayer(id uint32, source *binlogdatapb.BinlogSource, sourceTablet *topo dbClient: dbClient, mysqld: mysqld, retryDelay: 1 * time.Second, + tplans: make(map[string]*tablePlan), } } @@ -107,6 +108,7 @@ func (vp *vplayer) play(ctx context.Context) error { if err != nil { return err } + vp.pplan = plan vsClient, err := tabletconn.GetDialer()(vp.sourceTablet, grpcclient.FailFast(false)) if err != nil { @@ -133,6 +135,7 @@ func (vp *vplayer) play(ctx context.Context) error { } func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { + fmt.Printf("applying: %v\n", event) switch event.Type { case binlogdatapb.VEventType_GTID: pos, err := mysql.DecodePosition(event.Gtid) @@ -279,7 +282,7 @@ func (vp *vplayer) updatePlan(fieldEvent *binlogdatapb.FieldEvent) error { func (vp *vplayer) applyRowEvent(rowEvent *binlogdatapb.RowEvent) error { tplan := vp.tplans[rowEvent.TableName] - if tplan != nil { + if tplan == nil { return fmt.Errorf("unexpected event on table %s", rowEvent.TableName) } for _, change := range rowEvent.RowChanges { @@ -316,9 +319,9 @@ func (vp *vplayer) applyRowChange(tplan *tablePlan, rowChange *binlogdatapb.RowC func (vp *vplayer) generateInsert(tplan *tablePlan, after []sqltypes.Value) string { sql := sqlparser.NewTrackedBuffer(nil) if tplan.onInsert == insertIgnore { - sql.Myprintf("insert ignore into %s set ", sqlparser.NewTableIdent(tplan.name)) + sql.Myprintf("insert ignore into %v set ", sqlparser.NewTableIdent(tplan.name)) } else { - sql.Myprintf("insert into %s set ", sqlparser.NewTableIdent(tplan.name)) + sql.Myprintf("insert into %v set ", sqlparser.NewTableIdent(tplan.name)) } vp.writeInsertValues(sql, tplan, after) if tplan.onInsert == insertOndup { @@ -357,7 +360,7 @@ func (vp *vplayer) generateDelete(tplan *tablePlan, before []sqltypes.Value) str func (vp *vplayer) writeInsertValues(sql *sqlparser.TrackedBuffer, tplan *tablePlan, after []sqltypes.Value) { separator := "" for _, cExpr := range tplan.colExprs { - sql.Myprintf("%s%s=", separator, cExpr.colname) + sql.Myprintf("%s%v=", separator, cExpr.colname) if separator == "" { separator = ", " } @@ -375,7 +378,7 @@ func (vp *vplayer) writeUpdateValues(sql *sqlparser.TrackedBuffer, tplan *tableP if cExpr.isGrouped { continue } - sql.Myprintf("%s%s=", separator, cExpr.colname) + sql.Myprintf("%s%v=", separator, cExpr.colname) if separator == "" { separator = ", " } @@ -412,7 +415,7 @@ func (vp *vplayer) writeUpdateValues(sql *sqlparser.TrackedBuffer, tplan *tableP func (vp *vplayer) writeWhereValues(sql *sqlparser.TrackedBuffer, tplan *tablePlan, before []sqltypes.Value) { separator := "" for _, cExpr := range tplan.pkCols { - sql.Myprintf("%s%s=", separator, cExpr.colname) + sql.Myprintf("%s%v=", separator, cExpr.colname) if separator == "" { separator = " AND " } diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go new file mode 100644 index 00000000000..04527d179b6 --- /dev/null +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go @@ -0,0 +1,125 @@ +/* +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 vreplication + +import ( + "fmt" + "strings" + "testing" + "time" + + "golang.org/x/net/context" + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/sqltypes" + + "vitess.io/vitess/go/vt/binlog/binlogplayer" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + topodatapb "vitess.io/vitess/go/vt/proto/topodata" +) + +func TestSimple(t *testing.T) { + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + + execStatements(t, []string{ + "create table t1(id int, val varbinary(128), primary key(id))", + "create table t2(id int, val varbinary(128), primary key(id))", + }) + defer execStatements(t, []string{ + "drop table t1", + "drop table t2", + }) + env.SchemaEngine.Reload(context.Background()) + vre := NewEngine(env.TopoServ, env.Cells[0], env.Mysqld, realDBClientFactory) + vre.Open(context.Background()) + defer vre.Close() + + bls := &binlogdatapb.BinlogSource{ + Keyspace: env.KeyspaceName, + Shard: env.ShardName, + Filter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t2", + Filter: "select * from t1", + }}, + }, + } + pos := masterPosition(t) + query := fmt.Sprintf(`insert into _vt.vreplication`+ + `(workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state)`+ + `values('test', '%v', '%s', 9223372036854775807, 9223372036854775807, 481823, 0, 'Running')`, + bls, pos, + ) + if _, err := vre.Exec(query); err != nil { + t.Fatal(err) + } + execStatements(t, []string{"insert into t1 values(1, 'aaa')"}) + time.Sleep(1 * time.Second) +} + +func execStatements(t *testing.T, queries []string) { + t.Helper() + if err := env.Mysqld.ExecuteSuperQueryList(context.Background(), queries); err != nil { + t.Fatal(err) + } +} + +func masterPosition(t *testing.T) string { + t.Helper() + pos, err := env.Mysqld.MasterPosition() + if err != nil { + t.Fatal(err) + } + return mysql.EncodePosition(pos) +} + +func realDBClientFactory() binlogplayer.DBClient { + return realDBClient{} +} + +type realDBClient struct{} + +func (dbc realDBClient) DBName() string { + return env.KeyspaceName +} + +func (dbc realDBClient) Connect() error { + return nil +} + +func (dbc realDBClient) Begin() error { + return env.Mysqld.ExecuteSuperQueryList(context.Background(), []string{"begin"}) +} + +func (dbc realDBClient) Commit() error { + return env.Mysqld.ExecuteSuperQueryList(context.Background(), []string{"commit"}) +} + +func (dbc realDBClient) Rollback() error { + return env.Mysqld.ExecuteSuperQueryList(context.Background(), []string{"rollback"}) +} + +func (dbc realDBClient) Close() { +} + +func (dbc realDBClient) ExecuteFetch(query string, maxrows int) (qr *sqltypes.Result, err error) { + fmt.Printf("executing: %v\n", query) + if strings.HasPrefix(query, "use") { + return nil, nil + } + return env.Mysqld.FetchSuperQuery(context.Background(), query) +} diff --git a/go/vt/vttablet/tabletserver/vstreamer/testenv/testenv.go b/go/vt/vttablet/tabletserver/vstreamer/testenv/testenv.go index 036bcd2e77f..557fa40a60a 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/testenv/testenv.go +++ b/go/vt/vttablet/tabletserver/vstreamer/testenv/testenv.go @@ -46,6 +46,7 @@ type Env struct { cluster *vttest.LocalCluster KeyspaceName string + ShardName string Cells []string TopoServ *topo.Server @@ -65,6 +66,7 @@ func (checker) CheckMySQL() {} func Init() (*Env, error) { te := &Env{ KeyspaceName: "vttest", + ShardName: "0", Cells: []string{"cell1"}, } @@ -73,6 +75,9 @@ func Init() (*Env, error) { if err := te.TopoServ.CreateKeyspace(ctx, te.KeyspaceName, &topodatapb.Keyspace{}); err != nil { return nil, err } + if err := te.TopoServ.CreateShard(ctx, te.KeyspaceName, te.ShardName); err != nil { + panic(err) + } te.SrvTopo = srvtopo.NewResilientServer(te.TopoServ, "TestTopo") cfg := vttest.Config{ diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go index ade2ced8c32..df3947f4b45 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go +++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go @@ -138,6 +138,9 @@ func (vs *vstreamer) parseEvents(ctx context.Context, events <-chan mysql.Binlog vevents := bufferedEvents bufferedEvents = nil curSize = 0 + if len(vevents) == 3 && vevents[0].Type == binlogdatapb.VEventType_GTID && vevents[1].Type == binlogdatapb.VEventType_BEGIN && vevents[2].Type == binlogdatapb.VEventType_COMMIT { + vevents = vevents[:1] + } return vs.send(vevents) case binlogdatapb.VEventType_ROW: // ROW events happen inside transactions. So, we can chunk them. From afa7baf9b1e88fb7b598518b5e05929e71bf8284 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Sun, 13 Jan 2019 17:06:00 -0800 Subject: [PATCH 06/27] vreplication: more tweaks Signed-off-by: Sugu Sougoumarane --- .../vreplication/framework_test.go | 103 +++++++++++++++-- .../tabletmanager/vreplication/vplayer.go | 33 +++--- .../vreplication/vplayer_test.go | 105 ++++++++---------- .../tabletserver/vstreamer/planbuilder.go | 55 +++++++++ .../vstreamer/planbuilder_test.go | 81 ++++++++++++++ .../tabletserver/vstreamer/vstreamer.go | 23 ++-- .../tabletserver/vstreamer/vstreamer_test.go | 27 +++++ 7 files changed, 335 insertions(+), 92 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go index c8ef077e56f..275f1763306 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go @@ -21,11 +21,15 @@ import ( "fmt" "os" "reflect" + "regexp" + "strings" "testing" + "time" "github.com/golang/protobuf/proto" "golang.org/x/net/context" + "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/grpcclient" "vitess.io/vitess/go/vt/topo" @@ -41,9 +45,11 @@ import ( ) var ( - engine *vstreamer.Engine - env *testenv.Env - globalFBC = &fakeBinlogClient{} + playerEngine *Engine + streamerEngine *vstreamer.Engine + env *testenv.Env + globalFBC = &fakeBinlogClient{} + globalDBClient = &realDBClient{} ) func init() { @@ -75,12 +81,16 @@ func TestMain(m *testing.M) { } defer env.Close() - // engine cannot be initialized in testenv because it introduces + // engines cannot be initialized in testenv because it introduces // circular dependencies. - engine = vstreamer.NewEngine(env.SrvTopo, env.SchemaEngine) - engine.InitDBConfig(env.Dbcfgs) - engine.Open(env.KeyspaceName, env.Cells[0]) - defer engine.Close() + streamerEngine = vstreamer.NewEngine(env.SrvTopo, env.SchemaEngine) + streamerEngine.InitDBConfig(env.Dbcfgs) + streamerEngine.Open(env.KeyspaceName, env.Cells[0]) + defer streamerEngine.Close() + + playerEngine = NewEngine(env.TopoServ, env.Cells[0], env.Mysqld, realDBClientFactory) + playerEngine.Open(context.Background()) + defer playerEngine.Close() return m.Run() }() @@ -170,7 +180,7 @@ func (ftc *fakeTabletConn) StreamHealth(ctx context.Context, callback func(*quer // VStream directly calls into the pre-initialized engine. func (ftc *fakeTabletConn) VStream(ctx context.Context, target *querypb.Target, startPos string, filter *binlogdatapb.Filter, send func([]*binlogdatapb.VEvent) error) error { - return engine.Stream(ctx, startPos, filter, send) + return streamerEngine.Stream(ctx, startPos, filter, send) } //-------------------------------------- @@ -249,3 +259,78 @@ func expectFBCRequest(t *testing.T, tablet *topodatapb.Tablet, pos string, table t.Errorf("Request KeyRange: %v, want %v", globalFBC.lastKeyRange, kr) } } + +//-------------------------------------- +// DBCLient wrapper + +func resetDBClient() { + globalDBClient.queries = make(chan string, 1000) +} + +func realDBClientFactory() binlogplayer.DBClient { + resetDBClient() + return globalDBClient +} + +type realDBClient struct { + queries chan string +} + +func (dbc *realDBClient) DBName() string { + return env.KeyspaceName +} + +func (dbc *realDBClient) Connect() error { + return nil +} + +func (dbc *realDBClient) Begin() error { + dbc.queries <- "begin" + return env.Mysqld.ExecuteSuperQueryList(context.Background(), []string{"begin"}) +} + +func (dbc *realDBClient) Commit() error { + dbc.queries <- "commit" + return env.Mysqld.ExecuteSuperQueryList(context.Background(), []string{"commit"}) +} + +func (dbc *realDBClient) Rollback() error { + panic("rollback should never be called") +} + +func (dbc *realDBClient) Close() { +} + +func (dbc *realDBClient) ExecuteFetch(query string, maxrows int) (qr *sqltypes.Result, err error) { + if !strings.HasPrefix(query, "select") { + dbc.queries <- query + } + if strings.HasPrefix(query, "use") { + return nil, nil + } + return env.Mysqld.FetchSuperQuery(context.Background(), query) +} + +func expectDBClientQueries(t *testing.T, queries []string) { + t.Helper() + for i, query := range queries { + var got string + select { + case got = <-globalDBClient.queries: + match, err := regexp.MatchString(query, got) + if err != nil { + panic(err) + } + if !match { + t.Fatalf("query:\n%s, does not match query %d:\n%s", got, i, query) + } + case <-time.After(5 * time.Second): + t.Fatalf("no query received, expecting %s", query) + } + } + select { + case got := <-globalDBClient.queries: + t.Fatalf("unexpected query: %s", got) + default: + } +} diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index 19fd8e08967..02938c7479e 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -135,7 +135,6 @@ func (vp *vplayer) play(ctx context.Context) error { } func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { - fmt.Printf("applying: %v\n", event) switch event.Type { case binlogdatapb.VEventType_GTID: pos, err := mysql.DecodePosition(event.Gtid) @@ -174,13 +173,11 @@ func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { case binlogdatapb.VEventType_DDL: switch vp.source.OnDdl { case binlogdatapb.OnDDLAction_IGNORE: - if err := vp.updatePos(event.Timestamp); err != nil { - return err - } - if err := vp.dbClient.Commit(); err != nil { + // no-op + case binlogdatapb.OnDDLAction_STOP: + if err := vp.dbClient.Begin(); err != nil { return err } - case binlogdatapb.OnDDLAction_STOP: if err := vp.updatePos(event.Timestamp); err != nil { return err } @@ -189,23 +186,17 @@ func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { return err } case binlogdatapb.OnDDLAction_EXEC: - if err := vp.updatePos(event.Timestamp); err != nil { - return err - } if err := vp.exec(event.Ddl); err != nil { return err } - if err := vp.dbClient.Commit(); err != nil { + if err := vp.savePos(event.Timestamp); err != nil { return err } case binlogdatapb.OnDDLAction_EXEC_IGNORE: - if err := vp.updatePos(event.Timestamp); err != nil { - return err - } if err := vp.exec(event.Ddl); err != nil { log.Infof("Ignoring error: %v for DDL: %s", err, event.Ddl) } - if err := vp.dbClient.Commit(); err != nil { + if err := vp.savePos(event.Timestamp); err != nil { return err } } @@ -214,6 +205,8 @@ func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { } func (vp *vplayer) setState(state, message string) { + if state == binlogplayer.BlpStopped { + } if err := binlogplayer.SetVReplicationState(vp.dbClient, vp.id, state, message); err != nil { log.Errorf("Error writing state: %s, msg: %s, err: %v", state, message, err) } @@ -422,6 +415,18 @@ func (vp *vplayer) writeWhereValues(sql *sqlparser.TrackedBuffer, tplan *tablePl encodeValue(sql, before[cExpr.colnum]) } } + +// savePos performs an updatePos in its own transaction. +func (vp *vplayer) savePos(ts int64) error { + if err := vp.dbClient.Begin(); err != nil { + return err + } + if err := vp.updatePos(ts); err != nil { + return err + } + return vp.dbClient.Commit() +} + func (vp *vplayer) updatePos(ts int64) error { updatePos := binlogplayer.GenerateUpdatePos(vp.id, vp.pos, time.Now().Unix(), ts) if _, err := vp.dbClient.ExecuteFetch(updatePos, 0); err != nil { diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go index 04527d179b6..11e2d8bc5da 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go @@ -18,15 +18,10 @@ package vreplication import ( "fmt" - "strings" "testing" - "time" "golang.org/x/net/context" "vitess.io/vitess/go/mysql" - "vitess.io/vitess/go/sqltypes" - - "vitess.io/vitess/go/vt/binlog/binlogplayer" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" topodatapb "vitess.io/vitess/go/vt/proto/topodata" @@ -44,37 +39,62 @@ func TestSimple(t *testing.T) { "drop table t2", }) env.SchemaEngine.Reload(context.Background()) - vre := NewEngine(env.TopoServ, env.Cells[0], env.Mysqld, realDBClientFactory) - vre.Open(context.Background()) - defer vre.Close() + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t2", + Filter: "select * from t1", + }}, + } + cancel := startVReplication(t, playerEngine, filter, "") + defer cancel() + + execStatements(t, []string{"insert into t1 values(1, 'aaa')"}) + expectDBClientQueries(t, []string{ + "update _vt.vreplication set state='Running'.*", + "begin", + "update _vt.vreplication set pos=.*", + "commit", + "begin", + "insert into t2 set id=1, val='aaa'", + "update _vt.vreplication set pos=.*", + "commit", + }) +} + +func execStatements(t *testing.T, queries []string) { + t.Helper() + if err := env.Mysqld.ExecuteSuperQueryList(context.Background(), queries); err != nil { + t.Fatal(err) + } +} + +func startVReplication(t *testing.T, pe *Engine, filter *binlogdatapb.Filter, pos string) (cancelFunc func()) { + t.Helper() bls := &binlogdatapb.BinlogSource{ Keyspace: env.KeyspaceName, Shard: env.ShardName, - Filter: &binlogdatapb.Filter{ - Rules: []*binlogdatapb.Rule{{ - Match: "t2", - Filter: "select * from t1", - }}, - }, + Filter: filter, + } + if pos == "" { + pos = masterPosition(t) } - pos := masterPosition(t) query := fmt.Sprintf(`insert into _vt.vreplication`+ `(workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state)`+ `values('test', '%v', '%s', 9223372036854775807, 9223372036854775807, 481823, 0, 'Running')`, bls, pos, ) - if _, err := vre.Exec(query); err != nil { + qr, err := pe.Exec(query) + if err != nil { t.Fatal(err) } - execStatements(t, []string{"insert into t1 values(1, 'aaa')"}) - time.Sleep(1 * time.Second) -} - -func execStatements(t *testing.T, queries []string) { - t.Helper() - if err := env.Mysqld.ExecuteSuperQueryList(context.Background(), queries); err != nil { - t.Fatal(err) + resetDBClient() + return func() { + query := fmt.Sprintf("delete from _vt.vreplication where id = %d", qr.InsertID) + if _, err := pe.Exec(query); err != nil { + t.Fatal(err) + } } } @@ -86,40 +106,3 @@ func masterPosition(t *testing.T) string { } return mysql.EncodePosition(pos) } - -func realDBClientFactory() binlogplayer.DBClient { - return realDBClient{} -} - -type realDBClient struct{} - -func (dbc realDBClient) DBName() string { - return env.KeyspaceName -} - -func (dbc realDBClient) Connect() error { - return nil -} - -func (dbc realDBClient) Begin() error { - return env.Mysqld.ExecuteSuperQueryList(context.Background(), []string{"begin"}) -} - -func (dbc realDBClient) Commit() error { - return env.Mysqld.ExecuteSuperQueryList(context.Background(), []string{"commit"}) -} - -func (dbc realDBClient) Rollback() error { - return env.Mysqld.ExecuteSuperQueryList(context.Background(), []string{"rollback"}) -} - -func (dbc realDBClient) Close() { -} - -func (dbc realDBClient) ExecuteFetch(query string, maxrows int) (qr *sqltypes.Result, err error) { - fmt.Printf("executing: %v\n", query) - if strings.HasPrefix(query, "use") { - return nil, nil - } - return env.Mysqld.FetchSuperQuery(context.Background(), query) -} diff --git a/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go b/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go index 04e430bcb2b..f9920e791fb 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go +++ b/go/vt/vttablet/tabletserver/vstreamer/planbuilder.go @@ -120,6 +120,61 @@ func (plan *Plan) filter(values []sqltypes.Value) (bool, []sqltypes.Value, error return true, result, nil } +func mustSendDDL(query mysql.Query, dbname string, filter *binlogdatapb.Filter) bool { + if query.Database != "" && query.Database != dbname { + return false + } + ast, err := sqlparser.Parse(query.SQL) + // If there was a parsing error, we send it through. Hopefully, + // recipient can handle it. + if err != nil { + return true + } + switch stmt := ast.(type) { + case *sqlparser.DBDDL: + return false + case *sqlparser.DDL: + if !stmt.Table.IsEmpty() { + return tableMatches(stmt.Table, dbname, filter) + } + for _, table := range stmt.FromTables { + if tableMatches(table, dbname, filter) { + return true + } + } + for _, table := range stmt.ToTables { + if tableMatches(table, dbname, filter) { + return true + } + } + return false + } + return true +} + +func tableMatches(table sqlparser.TableName, dbname string, filter *binlogdatapb.Filter) bool { + if !table.Qualifier.IsEmpty() && table.Qualifier.String() != dbname { + return false + } + for _, rule := range filter.Rules { + switch { + case strings.HasPrefix(rule.Match, "/"): + expr := strings.Trim(rule.Match, "/") + result, err := regexp.MatchString(expr, table.Name.String()) + if err != nil { + return true + } + if !result { + continue + } + return true + case table.Name.String() == rule.Match: + return true + } + } + return false +} + func buildPlan(ti *Table, kschema *vindexes.KeyspaceSchema, filter *binlogdatapb.Filter) (*Plan, error) { for _, rule := range filter.Rules { switch { diff --git a/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go b/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go index f3bf8d4aaa4..f9f357e24e2 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/planbuilder_test.go @@ -67,6 +67,87 @@ func init() { testKSChema = kschema } +func TestMustSendDDL(t *testing.T) { + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "/t1.*/", + }, { + Match: "t2", + }}, + } + testcases := []struct { + sql string + db string + output bool + }{{ + sql: "create database db", + output: false, + }, { + sql: "create table foo(id int)", + output: false, + }, { + sql: "create table db.foo(id int)", + output: false, + }, { + sql: "create table mydb.foo(id int)", + output: false, + }, { + sql: "create table t1a(id int)", + output: true, + }, { + sql: "create table db.t1a(id int)", + output: false, + }, { + sql: "create table mydb.t1a(id int)", + output: true, + }, { + sql: "rename table t1a to foo, foo to bar", + output: true, + }, { + sql: "rename table foo to t1a, foo to bar", + output: true, + }, { + sql: "rename table foo to bar, t1a to bar", + output: true, + }, { + sql: "rename table foo to bar, bar to foo", + output: false, + }, { + sql: "drop table t1a, foo", + output: true, + }, { + sql: "drop table foo, t1a", + output: true, + }, { + sql: "drop table foo, bar", + output: false, + }, { + sql: "bad query", + output: true, + }, { + sql: "select * from t", + output: true, + }, { + sql: "drop table t2", + output: true, + }, { + sql: "create table t1a(id int)", + db: "db", + output: false, + }, { + sql: "create table t1a(id int)", + db: "mydb", + output: true, + }} + for _, tcase := range testcases { + q := mysql.Query{SQL: tcase.sql, Database: tcase.db} + got := mustSendDDL(q, "mydb", filter) + if got != tcase.output { + t.Errorf("%v: %v, want %v", q, got, tcase.output) + } + } +} + func TestPlanbuilder(t *testing.T) { t1 := &Table{ TableMap: &mysql.TableMap{ diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go index df3947f4b45..0ae019c8326 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go +++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go @@ -138,9 +138,6 @@ func (vs *vstreamer) parseEvents(ctx context.Context, events <-chan mysql.Binlog vevents := bufferedEvents bufferedEvents = nil curSize = 0 - if len(vevents) == 3 && vevents[0].Type == binlogdatapb.VEventType_GTID && vevents[1].Type == binlogdatapb.VEventType_BEGIN && vevents[2].Type == binlogdatapb.VEventType_COMMIT { - vevents = vevents[:1] - } return vs.send(vevents) case binlogdatapb.VEventType_ROW: // ROW events happen inside transactions. So, we can chunk them. @@ -236,7 +233,7 @@ func (vs *vstreamer) parseEvent(ev mysql.BinlogEvent) ([]*binlogdatapb.VEvent, e } var vevents []*binlogdatapb.VEvent switch { - case ev.IsPseudo() || ev.IsGTID(): + case ev.IsGTID(): gtid, hasBegin, err := ev.GTID(vs.format) if err != nil { return nil, fmt.Errorf("can't get GTID from binlog event: %v, event data: %#v", err, ev) @@ -274,10 +271,20 @@ func (vs *vstreamer) parseEvent(ev mysql.BinlogEvent) ([]*binlogdatapb.VEvent, e Type: binlogdatapb.VEventType_ROLLBACK, }) case sqlparser.StmtDDL: - vevents = append(vevents, &binlogdatapb.VEvent{ - Type: binlogdatapb.VEventType_DDL, - Ddl: q.SQL, - }) + if mustSendDDL(q, vs.cp.DbName, vs.filter) { + vevents = append(vevents, &binlogdatapb.VEvent{ + Type: binlogdatapb.VEventType_DDL, + Ddl: q.SQL, + }) + } else { + vevents = append(vevents, + &binlogdatapb.VEvent{ + Type: binlogdatapb.VEventType_BEGIN, + }, + &binlogdatapb.VEvent{ + Type: binlogdatapb.VEventType_COMMIT, + }) + } // Proactively reload schema. // If the DDL adds a column, comparing with an older snapshot of the // schema will make us think that a column was dropped and error out. diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go index 03dfe5470ba..7c345cd3cdd 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go @@ -472,6 +472,33 @@ func TestDDLDropColumn(t *testing.T) { } } +func TestUnsentDDL(t *testing.T) { + if testing.Short() { + t.Skip() + } + + execStatement(t, "create table unsent(id int, val varbinary(128), primary key(id))") + + testcases := []testcase{{ + input: []string{ + "drop table unsent", + }, + // An unsent DDL is sent as an empty transaction. + output: [][]string{{ + `gtid|begin`, + `gtid|begin`, + `commit`, + }}, + }} + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "/none/", + }}, + } + runCases(t, filter, testcases) +} + func TestBuffering(t *testing.T) { if testing.Short() { t.Skip() From cf26d48bc0b65f8c1aec041aa334ec86783d5c7c Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Mon, 14 Jan 2019 22:25:32 -0800 Subject: [PATCH 07/27] vreplication: relayLog functionality Signed-off-by: Sugu Sougoumarane --- .../vreplication/framework_test.go | 25 ++- .../tabletmanager/vreplication/relaylog.go | 145 ++++++++++++++++++ .../tabletmanager/vreplication/vplayer.go | 112 ++++++++++---- .../vreplication/vplayer_test.go | 30 ++-- 4 files changed, 268 insertions(+), 44 deletions(-) create mode 100644 go/vt/vttablet/tabletmanager/vreplication/relaylog.go diff --git a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go index 275f1763306..51537b96af3 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go @@ -311,6 +311,18 @@ func (dbc *realDBClient) ExecuteFetch(query string, maxrows int) (qr *sqltypes.R return env.Mysqld.FetchSuperQuery(context.Background(), query) } +func printQueries(t *testing.T) { + t.Helper() + for { + select { + case got := <-globalDBClient.queries: + t.Errorf("%s", got) + default: + return + } + } +} + func expectDBClientQueries(t *testing.T, queries []string) { t.Helper() for i, query := range queries { @@ -322,15 +334,18 @@ func expectDBClientQueries(t *testing.T, queries []string) { panic(err) } if !match { - t.Fatalf("query:\n%s, does not match query %d:\n%s", got, i, query) + t.Errorf("query:\n%s, does not match query %d:\n%s", got, i, query) } case <-time.After(5 * time.Second): t.Fatalf("no query received, expecting %s", query) } } - select { - case got := <-globalDBClient.queries: - t.Fatalf("unexpected query: %s", got) - default: + for { + select { + case got := <-globalDBClient.queries: + t.Errorf("unexpected query: %s", got) + default: + return + } } } diff --git a/go/vt/vttablet/tabletmanager/vreplication/relaylog.go b/go/vt/vttablet/tabletmanager/vreplication/relaylog.go new file mode 100644 index 00000000000..635bf1a76c3 --- /dev/null +++ b/go/vt/vttablet/tabletmanager/vreplication/relaylog.go @@ -0,0 +1,145 @@ +/* +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 vreplication + +import ( + "io" + "sync" + + "golang.org/x/net/context" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" +) + +type relayLog struct { + ctx context.Context + maxItems int + maxSize int + + // mu controls all variables below and is shared by canAccept and hasItems. + // Broadcasting must be done while holding mu. This is mainly necessary because both + // conditions depend on ctx.Done(), which can change state asynchronously. + mu sync.Mutex + curSize int + items [][]*binlogdatapb.VEvent + err error + // canAccept is true if: curSize<=maxSize, len(items)0, err==nil and ctx is not Done. + hasItems sync.Cond +} + +func newRelayLog(ctx context.Context, maxItems, maxSize int) *relayLog { + rl := &relayLog{ + ctx: ctx, + maxItems: maxItems, + maxSize: maxSize, + } + rl.canAccept.L = &rl.mu + rl.hasItems.L = &rl.mu + + // Any time context is done, wake up all waiters to make them exit. + go func() { + select { + case <-ctx.Done(): + rl.mu.Lock() + defer rl.mu.Unlock() + rl.canAccept.Broadcast() + rl.hasItems.Broadcast() + default: + } + }() + return rl +} + +func (rl *relayLog) Send(events []*binlogdatapb.VEvent) error { + rl.mu.Lock() + defer rl.mu.Unlock() + + if err := rl.checkDone(); err != nil { + return err + } + for rl.curSize > rl.maxSize || len(rl.items) >= rl.maxItems { + rl.canAccept.Wait() + if err := rl.checkDone(); err != nil { + return err + } + } + rl.items = append(rl.items, events) + rl.curSize += eventsSize(events) + rl.hasItems.Broadcast() + return nil +} + +func (rl *relayLog) Fetch() ([][]*binlogdatapb.VEvent, error) { + rl.mu.Lock() + defer rl.mu.Unlock() + + if err := rl.checkDone(); err != nil { + return nil, err + } + for len(rl.items) == 0 { + rl.hasItems.Wait() + if err := rl.checkDone(); err != nil { + return nil, err + } + } + items := rl.items + rl.items = nil + rl.curSize = 0 + rl.canAccept.Broadcast() + return items, nil +} + +func (rl *relayLog) checkDone() error { + if rl.err != nil { + return rl.err + } + select { + case <-rl.ctx.Done(): + return io.EOF + default: + } + return nil +} + +func (rl *relayLog) SetError(err error) { + rl.mu.Lock() + defer rl.mu.Unlock() + + rl.err = err + rl.canAccept.Broadcast() + rl.hasItems.Broadcast() +} + +func eventsSize(events []*binlogdatapb.VEvent) int { + size := 0 + for _, event := range events { + if event.Type != binlogdatapb.VEventType_ROW { + continue + } + for _, rowChange := range event.RowEvent.RowChanges { + if rowChange.Before != nil { + size += len(rowChange.Before.Values) + } + if rowChange.After != nil { + size += len(rowChange.After.Values) + } + } + } + return size +} diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index 02938c7479e..27aca2e525a 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -19,7 +19,6 @@ package vreplication import ( "bytes" "fmt" - "io" "time" "golang.org/x/net/context" @@ -46,10 +45,13 @@ type vplayer struct { dbClient binlogplayer.DBClient mysqld mysqlctl.MysqlDaemon - pos mysql.Position - stopPos mysql.Position - pplan *playerPlan - tplans map[string]*tablePlan + pos mysql.Position + stopPos mysql.Position + externalizePos []mysql.Position + mustCommit bool + + pplan *playerPlan + tplans map[string]*tablePlan retryDelay time.Duration } @@ -114,24 +116,69 @@ func (vp *vplayer) play(ctx context.Context) error { if err != nil { return fmt.Errorf("error dialing tablet: %v", err) } + ctx, cancel := context.WithCancel(ctx) + defer cancel() + + relay := newRelayLog(ctx, 10, 10) + go vp.applyEvents(relay) + target := &querypb.Target{ Keyspace: vp.sourceTablet.Keyspace, Shard: vp.sourceTablet.Shard, TabletType: vp.sourceTablet.Type, } return vsClient.VStream(ctx, target, startPos, plan.vstreamFilter, func(events []*binlogdatapb.VEvent) error { - for _, event := range events { - select { - case <-ctx.Done(): - return io.EOF - default: + return relay.Send(events) + }) +} + +func (vp *vplayer) applyEvents(relay *relayLog) { + for { + items, err := relay.Fetch() + if err != nil { + return + } + events := linearizeEvents(items) + for i, event := range events { + switch event.Type { + case binlogdatapb.VEventType_COMMIT: + if vp.pos.Equal(vp.stopPos) { + break + } + // Check for externalizePos + foundAnotherCommit := false + searchCommit: + for j := i + 1; j < len(events); j++ { + switch event.Type { + case binlogdatapb.VEventType_COMMIT: + foundAnotherCommit = true + break searchCommit + case binlogdatapb.VEventType_ROLLBACK, binlogdatapb.VEventType_DDL: + break searchCommit + } + } + if foundAnotherCommit { + continue + } } if err := vp.applyEvent(event); err != nil { - return err + relay.SetError(err) + return } } - return nil - }) + } +} + +func linearizeEvents(items [][]*binlogdatapb.VEvent) []*binlogdatapb.VEvent { + length := 0 + for _, events := range items { + length += len(events) + } + linear := make([]*binlogdatapb.VEvent, 0, length) + for _, events := range items { + linear = append(linear, events...) + } + return linear } func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { @@ -149,28 +196,46 @@ func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { return fmt.Errorf("next event position %v exceeds stop pos %v, exiting without applying", vp.pos, vp.stopPos) } case binlogdatapb.VEventType_BEGIN: - if err := vp.dbClient.Begin(); err != nil { - return err - } + // No-op: begin is called as needed. case binlogdatapb.VEventType_COMMIT: + if !vp.mustCommit { + return nil + } if err := vp.updatePos(event.Timestamp); err != nil { return err } if err := vp.dbClient.Commit(); err != nil { return err } + vp.mustCommit = false case binlogdatapb.VEventType_ROLLBACK: // This code is unreachable. It's just here as failsafe. + vp.mustCommit = false _ = vp.dbClient.Rollback() case binlogdatapb.VEventType_FIELD: + if !vp.mustCommit { + if err := vp.dbClient.Begin(); err != nil { + return err + } + vp.mustCommit = true + } if err := vp.updatePlan(event.FieldEvent); err != nil { return err } case binlogdatapb.VEventType_ROW: + if !vp.mustCommit { + if err := vp.dbClient.Begin(); err != nil { + return err + } + vp.mustCommit = true + } if err := vp.applyRowEvent(event.RowEvent); err != nil { return err } case binlogdatapb.VEventType_DDL: + if vp.mustCommit { + return fmt.Errorf("unexpected state: DDL encountered in the middle of a transaction: %v", event.Ddl) + } switch vp.source.OnDdl { case binlogdatapb.OnDDLAction_IGNORE: // no-op @@ -189,14 +254,14 @@ func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { if err := vp.exec(event.Ddl); err != nil { return err } - if err := vp.savePos(event.Timestamp); err != nil { + if err := vp.updatePos(event.Timestamp); err != nil { return err } case binlogdatapb.OnDDLAction_EXEC_IGNORE: if err := vp.exec(event.Ddl); err != nil { log.Infof("Ignoring error: %v for DDL: %s", err, event.Ddl) } - if err := vp.savePos(event.Timestamp); err != nil { + if err := vp.updatePos(event.Timestamp); err != nil { return err } } @@ -416,17 +481,6 @@ func (vp *vplayer) writeWhereValues(sql *sqlparser.TrackedBuffer, tplan *tablePl } } -// savePos performs an updatePos in its own transaction. -func (vp *vplayer) savePos(ts int64) error { - if err := vp.dbClient.Begin(); err != nil { - return err - } - if err := vp.updatePos(ts); err != nil { - return err - } - return vp.dbClient.Commit() -} - func (vp *vplayer) updatePos(ts int64) error { updatePos := binlogplayer.GenerateUpdatePos(vp.id, vp.pos, time.Now().Unix(), ts) if _, err := vp.dbClient.ExecuteFetch(updatePos, 0); err != nil { diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go index 11e2d8bc5da..a349b026204 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go @@ -19,6 +19,7 @@ package vreplication import ( "fmt" "testing" + "time" "golang.org/x/net/context" "vitess.io/vitess/go/mysql" @@ -49,17 +50,26 @@ func TestSimple(t *testing.T) { cancel := startVReplication(t, playerEngine, filter, "") defer cancel() - execStatements(t, []string{"insert into t1 values(1, 'aaa')"}) - expectDBClientQueries(t, []string{ - "update _vt.vreplication set state='Running'.*", - "begin", - "update _vt.vreplication set pos=.*", - "commit", - "begin", - "insert into t2 set id=1, val='aaa'", - "update _vt.vreplication set pos=.*", - "commit", + execStatements(t, []string{ + "insert into t1 values(1, 'aaa')", + "insert into t1 values(2, 'aaa')", + "insert into t1 values(3, 'aaa')", + "insert into t1 values(4, 'aaa')", }) + time.Sleep(1 * time.Second) + printQueries(t) + /* + expectDBClientQueries(t, []string{ + "update _vt.vreplication set state='Running'.*", + "begin", + "update _vt.vreplication set pos=.*", + "commit", + "begin", + "insert into t2 set id=1, val='aaa'", + "update _vt.vreplication set pos=.*", + "commit", + }) + */ } func execStatements(t *testing.T, queries []string) { From 8b56b338986afc93a4e722cd226c2ab22cfd12a6 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Thu, 17 Jan 2019 11:52:00 -0800 Subject: [PATCH 08/27] vplayer: more features Signed-off-by: Sugu Sougoumarane --- .../tabletmanager/vreplication/controller.go | 14 +- .../tabletmanager/vreplication/engine.go | 31 ++- .../tabletmanager/vreplication/engine_test.go | 11 + .../tabletmanager/vreplication/relaylog.go | 20 +- .../vreplication/retryableClient.go | 82 +++++++ .../tabletmanager/vreplication/vplayer.go | 200 ++++++++++++------ .../tabletserver/vstreamer/vstreamer.go | 8 +- 7 files changed, 288 insertions(+), 78 deletions(-) create mode 100644 go/vt/vttablet/tabletmanager/vreplication/retryableClient.go diff --git a/go/vt/vttablet/tabletmanager/vreplication/controller.go b/go/vt/vttablet/tabletmanager/vreplication/controller.go index 200492e0cdf..574b1484d21 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/controller.go +++ b/go/vt/vttablet/tabletmanager/vreplication/controller.go @@ -22,6 +22,7 @@ import ( "strconv" "time" + "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/vt/vterrors" "github.com/golang/protobuf/proto" @@ -47,6 +48,9 @@ type controller struct { mysqld mysqlctl.MysqlDaemon blpStats *binlogplayer.Stats + // vplayer is set only if we launch vplayer. + vplayer *vplayer + id uint32 source binlogdatapb.BinlogSource stopPos string @@ -183,12 +187,18 @@ 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: - player := newVPlayer(ct.id, &ct.source, tablet, ct.blpStats, dbClient, ct.mysqld) - return player.Play(ctx) + ct.vplayer = newVPlayer(ct.id, &ct.source, tablet, ct.blpStats, dbClient, ct.mysqld) + return ct.vplayer.Play(ctx) } return fmt.Errorf("missing source") } +func (ct *controller) exportPosition(pos mysql.Position) { + if ct.vplayer != nil { + ct.vplayer.exportPosition(pos) + } +} + func (ct *controller) Stop() { ct.cancel() <-ct.done diff --git a/go/vt/vttablet/tabletmanager/vreplication/engine.go b/go/vt/vttablet/tabletmanager/vreplication/engine.go index f214876de88..3a664b131f5 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/engine.go +++ b/go/vt/vttablet/tabletmanager/vreplication/engine.go @@ -281,14 +281,31 @@ func (vre *Engine) WaitForPos(ctx context.Context, id int, pos string) error { return err } - vre.mu.Lock() - if !vre.isOpen { - vre.mu.Unlock() - return errors.New("vreplication engine is closed") + if err := func() error { + vre.mu.Lock() + defer vre.mu.Unlock() + + if !vre.isOpen { + return errors.New("vreplication engine is closed") + } + ct, ok := vre.controllers[id] + if !ok { + return fmt.Errorf("vreplication stream %d not found", id) + } + mpos, err := mysql.DecodePosition(pos) + if err != nil { + return err + } + // vplayer doesn't export all the positions it receives unless + // we specifically request it for one. + ct.exportPosition(mpos) + + // Ensure that the engine won't be closed while this is running. + vre.wg.Add(1) + return nil + }(); err != nil { + return err } - // Ensure that the engine won't be closed while this is running. - vre.wg.Add(1) - vre.mu.Unlock() defer vre.wg.Done() dbClient := vre.dbClientFactory() diff --git a/go/vt/vttablet/tabletmanager/vreplication/engine_test.go b/go/vt/vttablet/tabletmanager/vreplication/engine_test.go index 5b335a05a9a..8394a8328f0 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/engine_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/engine_test.go @@ -283,6 +283,8 @@ func TestWaitForPos(t *testing.T) { if err := vre.Open(context.Background()); err != nil { t.Fatal(err) } + // hack a controller into vre. + vre.controllers[1] = &controller{} dbClient.ExpectRequest("select pos from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ sqltypes.NewVarBinary("MariaDB/0-1-1083"), @@ -315,6 +317,8 @@ func TestWaitForPosError(t *testing.T) { if err := vre.Open(context.Background()); err != nil { t.Fatal(err) } + // hack a controller into vre. + vre.controllers[1] = &controller{} err = vre.WaitForPos(context.Background(), 1, "BadFlavor/0-1-1084") want = `parse error: unknown GTIDSet flavor "BadFlavor"` @@ -357,6 +361,13 @@ func TestWaitForPosCancel(t *testing.T) { }}}, nil) ctx, cancel := context.WithCancel(context.Background()) cancel() + // hack a controller that can handle Close + done := make(chan struct{}) + close(done) + vre.controllers[1] = &controller{ + cancel: func() {}, + done: done, + } err := vre.WaitForPos(ctx, 1, "MariaDB/0-1-1084") if err == nil || err != context.Canceled { t.Errorf("WaitForPos: %v, want %v", err, context.Canceled) diff --git a/go/vt/vttablet/tabletmanager/vreplication/relaylog.go b/go/vt/vttablet/tabletmanager/vreplication/relaylog.go index 635bf1a76c3..b8544e81164 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/relaylog.go +++ b/go/vt/vttablet/tabletmanager/vreplication/relaylog.go @@ -19,9 +19,11 @@ package vreplication import ( "io" "sync" + "time" "golang.org/x/net/context" + "vitess.io/vitess/go/sync2" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" ) @@ -39,7 +41,7 @@ type relayLog struct { err error // canAccept is true if: curSize<=maxSize, len(items)0, err==nil and ctx is not Done. + // hasItems is true if len(items)>0, err==nil, ctx is not Done, and call has not timedout. hasItems sync.Cond } @@ -92,11 +94,27 @@ func (rl *relayLog) Fetch() ([][]*binlogdatapb.VEvent, error) { if err := rl.checkDone(); err != nil { return nil, err } + timer := time.NewTimer(idleTimeout) + defer timer.Stop() + var timedout sync2.AtomicBool + go func() { + select { + case <-timer.C: + rl.mu.Lock() + defer rl.mu.Unlock() + timedout.Set(true) + rl.hasItems.Broadcast() + default: + } + }() for len(rl.items) == 0 { rl.hasItems.Wait() if err := rl.checkDone(); err != nil { return nil, err } + if timedout.Get() { + return nil, nil + } } items := rl.items rl.items = nil diff --git a/go/vt/vttablet/tabletmanager/vreplication/retryableClient.go b/go/vt/vttablet/tabletmanager/vreplication/retryableClient.go new file mode 100644 index 00000000000..30deacf155b --- /dev/null +++ b/go/vt/vttablet/tabletmanager/vreplication/retryableClient.go @@ -0,0 +1,82 @@ +/* +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 vreplication + +import ( + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/vt/binlog/binlogplayer" +) + +type retryableClient struct { + binlogplayer.DBClient + inTransaction bool + queries []string +} + +func (rt *retryableClient) Begin() error { + if err := rt.DBClient.Begin(); err != nil { + return err + } + rt.inTransaction = true + return nil +} + +func (rt *retryableClient) Commit() error { + if err := rt.DBClient.Commit(); err != nil { + return err + } + rt.inTransaction = false + rt.queries = nil + return nil +} + +func (rt *retryableClient) Rollback() error { + if err := rt.DBClient.Rollback(); err != nil { + return err + } + rt.inTransaction = false + rt.queries = nil + return nil +} + +func (rt *retryableClient) ExecuteFetch(query string, maxrows int) (*sqltypes.Result, error) { + if !rt.inTransaction { + rt.queries = []string{query} + } else { + rt.queries = append(rt.queries, query) + } + return rt.DBClient.ExecuteFetch(query, maxrows) +} + +func (rt *retryableClient) Retry() error { + if !rt.inTransaction { + _, err := rt.DBClient.ExecuteFetch(rt.queries[0], 10000) + return err + } + if err := rt.DBClient.Rollback(); err != nil { + return err + } + if err := rt.DBClient.Begin(); err != nil { + return err + } + for _, q := range rt.queries { + if _, err := rt.DBClient.ExecuteFetch(q, 10000); err != nil { + return err + } + } + return nil +} diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index 27aca2e525a..d79640bbcd0 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -19,6 +19,8 @@ package vreplication import ( "bytes" "fmt" + "io" + "sync" "time" "golang.org/x/net/context" @@ -37,47 +39,67 @@ import ( topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) +var ( + idleTimeout = 1 * time.Hour + dblockRetryDelay = 1 * time.Second +) + type vplayer struct { id uint32 source *binlogdatapb.BinlogSource sourceTablet *topodatapb.Tablet stats *binlogplayer.Stats - dbClient binlogplayer.DBClient + dbClient *retryableClient mysqld mysqlctl.MysqlDaemon - pos mysql.Position - stopPos mysql.Position - externalizePos []mysql.Position - mustCommit bool + pos mysql.Position + unsavedGTID *binlogdatapb.VEvent + timeLastSaved time.Time + stopPos mysql.Position + mustCommit bool + + // mu protects exportPositions. + // exportPositions specifies a list of positions. + // Anytime the vplayer reaches or exceeds one of these + // positions, it will update the position even if it's + // an empty transaction. This is used for cases where + // a target is stopped, and a request is initiated to + // wait for vreplication to reach the source's position. + // Since vreplication only updates state on non-empty + // transactions, this list is used to force such updates. + mu sync.Mutex + exportPositions []mysql.Position pplan *playerPlan tplans map[string]*tablePlan - - retryDelay time.Duration } func newVPlayer(id uint32, source *binlogdatapb.BinlogSource, sourceTablet *topodatapb.Tablet, stats *binlogplayer.Stats, dbClient binlogplayer.DBClient, mysqld mysqlctl.MysqlDaemon) *vplayer { return &vplayer{ - id: id, - source: source, - sourceTablet: sourceTablet, - stats: stats, - dbClient: dbClient, - mysqld: mysqld, - retryDelay: 1 * time.Second, - tplans: make(map[string]*tablePlan), + id: id, + source: source, + sourceTablet: sourceTablet, + stats: stats, + dbClient: &retryableClient{DBClient: dbClient}, + mysqld: mysqld, + timeLastSaved: time.Now(), + tplans: make(map[string]*tablePlan), } } func (vp *vplayer) Play(ctx context.Context) error { - vp.setState(binlogplayer.BlpRunning, "") + if err := vp.setState(binlogplayer.BlpRunning, ""); err != nil { + return err + } if err := vp.play(ctx); err != nil { msg := err.Error() vp.stats.History.Add(&binlogplayer.StatsHistoryRecord{ Time: time.Now(), Message: msg, }) - vp.setState(binlogplayer.BlpError, msg) + if err := vp.setState(binlogplayer.BlpError, msg); err != nil { + return err + } return err } return nil @@ -100,8 +122,7 @@ func (vp *vplayer) play(ctx context.Context) error { } if !vp.stopPos.IsZero() { if vp.pos.AtLeast(vp.stopPos) { - vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("Stop position %v already reached: %v", vp.pos, vp.stopPos)) - return nil + return vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("Stop position %v already reached: %v", vp.pos, vp.stopPos)) } } log.Infof("Starting VReplication player id: %v, startPos: %v, stop: %v, source: %v", vp.id, startPos, vp.stopPos, vp.sourceTablet) @@ -136,49 +157,81 @@ func (vp *vplayer) applyEvents(relay *relayLog) { for { items, err := relay.Fetch() if err != nil { + relay.SetError(err) return } - events := linearizeEvents(items) - for i, event := range events { - switch event.Type { - case binlogdatapb.VEventType_COMMIT: - if vp.pos.Equal(vp.stopPos) { - break - } - // Check for externalizePos - foundAnotherCommit := false - searchCommit: - for j := i + 1; j < len(events); j++ { - switch event.Type { - case binlogdatapb.VEventType_COMMIT: - foundAnotherCommit = true - break searchCommit - case binlogdatapb.VEventType_ROLLBACK, binlogdatapb.VEventType_DDL: - break searchCommit + // This covers two situations: + // 1. Fetch was idle for idleTimeout. + // 2. We've been receiving empty events for longer than idleTimeout. + // In both cases, now > timeLastSaved. If so, any unsaved GTID should be saved + // to prevent us from falling behind on tracking the source binlog position. + if time.Now().Sub(vp.timeLastSaved) >= idleTimeout && vp.unsavedGTID != nil { + if err := vp.updatePos(vp.unsavedGTID.Timestamp); err != nil { + relay.SetError(err) + return + } + } + for i, events := range items { + for j, event := range events { + switch event.Type { + case binlogdatapb.VEventType_COMMIT: + if vp.pos.Equal(vp.stopPos) { + // If stop pos is reached, we have to commit. + break + } + if vp.mustExport() { + break + } + if hasAnotherCommit(items, i, j+1) { + continue } } - if foundAnotherCommit { - continue + if err := vp.applyEvent(event); err != nil { + relay.SetError(err) + return } } - if err := vp.applyEvent(event); err != nil { - relay.SetError(err) - return - } } } } -func linearizeEvents(items [][]*binlogdatapb.VEvent) []*binlogdatapb.VEvent { - length := 0 - for _, events := range items { - length += len(events) +func (vp *vplayer) mustExport() bool { + vp.mu.Lock() + defer vp.mu.Unlock() + + mustExport := false + for { + found := false + for i, export := range vp.exportPositions { + if vp.pos.AtLeast(export) { + mustExport = true + found = true + vp.exportPositions = append(vp.exportPositions[:i], vp.exportPositions[i+1:]...) + break + } + } + if !found { + return mustExport + } } - linear := make([]*binlogdatapb.VEvent, 0, length) - for _, events := range items { - linear = append(linear, events...) +} + +func hasAnotherCommit(items [][]*binlogdatapb.VEvent, i, j int) bool { + for i < len(items) { + for j < len(items[i]) { + // We ignore GTID, BEGIN, FIELD and ROW. + switch items[i][j].Type { + case binlogdatapb.VEventType_COMMIT: + return true + case binlogdatapb.VEventType_DDL: + return false + } + j++ + } + j = 0 + i++ } - return linear + return false } func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { @@ -189,6 +242,7 @@ func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { return err } vp.pos = pos + vp.unsavedGTID = event if vp.stopPos.IsZero() { return nil } @@ -204,14 +258,19 @@ func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { if err := vp.updatePos(event.Timestamp); err != nil { return err } + posReached := !vp.stopPos.IsZero() && vp.pos.Equal(vp.stopPos) + if posReached { + if err := vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("Stopped at position %v", vp.stopPos)); err != nil { + return err + } + } if err := vp.dbClient.Commit(); err != nil { return err } vp.mustCommit = false - case binlogdatapb.VEventType_ROLLBACK: - // This code is unreachable. It's just here as failsafe. - vp.mustCommit = false - _ = vp.dbClient.Rollback() + if posReached { + return io.EOF + } case binlogdatapb.VEventType_FIELD: if !vp.mustCommit { if err := vp.dbClient.Begin(); err != nil { @@ -246,10 +305,13 @@ func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { if err := vp.updatePos(event.Timestamp); err != nil { return err } - vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("stopped at DDL %s", event.Ddl)) + if err := vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("Stopped at DDL %s", event.Ddl)); err != nil { + return err + } if err := vp.dbClient.Commit(); err != nil { return err } + return io.EOF case binlogdatapb.OnDDLAction_EXEC: if err := vp.exec(event.Ddl); err != nil { return err @@ -269,12 +331,8 @@ func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { return nil } -func (vp *vplayer) setState(state, message string) { - if state == binlogplayer.BlpStopped { - } - if err := binlogplayer.SetVReplicationState(vp.dbClient, vp.id, state, message); err != nil { - log.Errorf("Error writing state: %s, msg: %s, err: %v", state, message, err) - } +func (vp *vplayer) setState(state, message string) error { + return binlogplayer.SetVReplicationState(vp.dbClient, vp.id, state, message) } func (vp *vplayer) updatePlan(fieldEvent *binlogdatapb.FieldEvent) error { @@ -487,13 +545,31 @@ func (vp *vplayer) updatePos(ts int64) error { _ = vp.dbClient.Rollback() return fmt.Errorf("error %v updating position", err) } + vp.unsavedGTID = nil + vp.timeLastSaved = time.Now() return nil } func (vp *vplayer) exec(sql string) error { vp.stats.Timings.Record("query", time.Now()) _, err := vp.dbClient.ExecuteFetch(sql, 0) - return err + for err != nil { + // 1213: deadlock, 1205: lock wait timeout + if sqlErr, ok := err.(*mysql.SQLError); ok && sqlErr.Number() == 1213 || sqlErr.Number() == 1205 { + log.Infof("retryable error: %v, waiting for %v and retrying", sqlErr, dblockRetryDelay) + time.Sleep(dblockRetryDelay) + err = vp.dbClient.Retry() + continue + } + return err + } + return nil +} + +func (vp *vplayer) exportPosition(pos mysql.Position) { + vp.mu.Lock() + defer vp.mu.Unlock() + vp.exportPositions = append(vp.exportPositions, pos) } func encodeValue(sql *sqlparser.TrackedBuffer, value sqltypes.Value) { diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go index 0ae019c8326..6c8631a49ba 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go +++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go @@ -131,8 +131,8 @@ func (vs *vstreamer) parseEvents(ctx context.Context, events <-chan mysql.Binlog case binlogdatapb.VEventType_GTID, binlogdatapb.VEventType_BEGIN, binlogdatapb.VEventType_FIELD: // We never have to send GTID, BEGIN or FIELD events on their own. bufferedEvents = append(bufferedEvents, vevent) - case binlogdatapb.VEventType_COMMIT, binlogdatapb.VEventType_ROLLBACK, binlogdatapb.VEventType_DDL: - // COMMIT, ROLLBACK and DDL are terminal. There may be no more events after + case binlogdatapb.VEventType_COMMIT, binlogdatapb.VEventType_DDL: + // COMMIT and DDL are terminal. There may be no more events after // these for a long time. So, we have to send whatever we have. bufferedEvents = append(bufferedEvents, vevent) vevents := bufferedEvents @@ -266,10 +266,6 @@ func (vs *vstreamer) parseEvent(ev mysql.BinlogEvent) ([]*binlogdatapb.VEvent, e vevents = append(vevents, &binlogdatapb.VEvent{ Type: binlogdatapb.VEventType_COMMIT, }) - case sqlparser.StmtRollback: - vevents = append(vevents, &binlogdatapb.VEvent{ - Type: binlogdatapb.VEventType_ROLLBACK, - }) case sqlparser.StmtDDL: if mustSendDDL(q, vs.cp.DbName, vs.filter) { vevents = append(vevents, &binlogdatapb.VEvent{ From cff8cda6ac4d46d1f81d745321d09655576e2b39 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Thu, 17 Jan 2019 23:26:15 -0800 Subject: [PATCH 09/27] vplayer: more tests and a few bug fixes Signed-off-by: Sugu Sougoumarane --- .../vreplication/framework_test.go | 81 ++++++++++--------- .../tabletmanager/vreplication/relaylog.go | 16 +--- .../tabletmanager/vreplication/vplayer.go | 60 +++++++------- .../vreplication/vplayer_test.go | 74 +++++++++++------ 4 files changed, 127 insertions(+), 104 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go index 51537b96af3..139313e3393 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go @@ -29,6 +29,7 @@ import ( "github.com/golang/protobuf/proto" "golang.org/x/net/context" + "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/sqltypes" "vitess.io/vitess/go/vt/binlog/binlogplayer" "vitess.io/vitess/go/vt/grpcclient" @@ -45,11 +46,11 @@ import ( ) var ( - playerEngine *Engine - streamerEngine *vstreamer.Engine - env *testenv.Env - globalFBC = &fakeBinlogClient{} - globalDBClient = &realDBClient{} + playerEngine *Engine + streamerEngine *vstreamer.Engine + env *testenv.Env + globalFBC = &fakeBinlogClient{} + globalDBQueries = make(chan string, 1000) ) func init() { @@ -263,17 +264,12 @@ func expectFBCRequest(t *testing.T, tablet *topodatapb.Tablet, pos string, table //-------------------------------------- // DBCLient wrapper -func resetDBClient() { - globalDBClient.queries = make(chan string, 1000) -} - func realDBClientFactory() binlogplayer.DBClient { - resetDBClient() - return globalDBClient + return &realDBClient{} } type realDBClient struct { - queries chan string + conn *mysql.Conn } func (dbc *realDBClient) DBName() string { @@ -281,17 +277,24 @@ func (dbc *realDBClient) DBName() string { } func (dbc *realDBClient) Connect() error { + conn, err := mysql.Connect(context.Background(), env.Dbcfgs.AppWithDB()) + if err != nil { + return err + } + dbc.conn = conn return nil } func (dbc *realDBClient) Begin() error { - dbc.queries <- "begin" - return env.Mysqld.ExecuteSuperQueryList(context.Background(), []string{"begin"}) + globalDBQueries <- "begin" + _, err := dbc.conn.ExecuteFetch("begin", 10000, true) + return err } func (dbc *realDBClient) Commit() error { - dbc.queries <- "commit" - return env.Mysqld.ExecuteSuperQueryList(context.Background(), []string{"commit"}) + globalDBQueries <- "commit" + _, err := dbc.conn.ExecuteFetch("commit", 10000, true) + return err } func (dbc *realDBClient) Rollback() error { @@ -299,50 +302,52 @@ func (dbc *realDBClient) Rollback() error { } func (dbc *realDBClient) Close() { + dbc.conn.Close() + dbc.conn = nil } -func (dbc *realDBClient) ExecuteFetch(query string, maxrows int) (qr *sqltypes.Result, err error) { +func (dbc *realDBClient) ExecuteFetch(query string, maxrows int) (*sqltypes.Result, error) { if !strings.HasPrefix(query, "select") { - dbc.queries <- query + globalDBQueries <- query } if strings.HasPrefix(query, "use") { return nil, nil } - return env.Mysqld.FetchSuperQuery(context.Background(), query) -} - -func printQueries(t *testing.T) { - t.Helper() - for { - select { - case got := <-globalDBClient.queries: - t.Errorf("%s", got) - default: - return - } - } + return dbc.conn.ExecuteFetch(query, 10000, true) } func expectDBClientQueries(t *testing.T, queries []string) { t.Helper() + failed := false for i, query := range queries { + if failed { + t.Errorf("no query received, expecting %s", query) + continue + } var got string select { - case got = <-globalDBClient.queries: - match, err := regexp.MatchString(query, got) - if err != nil { - panic(err) + case got = <-globalDBQueries: + var match bool + if query[0] == '/' { + result, err := regexp.MatchString(query[1:], got) + if err != nil { + panic(err) + } + match = result + } else { + match = (got == query) } if !match { - t.Errorf("query:\n%s, does not match query %d:\n%s", got, i, query) + t.Errorf("query:\n%q, does not match query %d:\n%q", got, i, query) } case <-time.After(5 * time.Second): - t.Fatalf("no query received, expecting %s", query) + t.Errorf("no query received, expecting %s", query) + failed = true } } for { select { - case got := <-globalDBClient.queries: + case got := <-globalDBQueries: t.Errorf("unexpected query: %s", got) default: return diff --git a/go/vt/vttablet/tabletmanager/vreplication/relaylog.go b/go/vt/vttablet/tabletmanager/vreplication/relaylog.go index b8544e81164..9be66dd75c4 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/relaylog.go +++ b/go/vt/vttablet/tabletmanager/vreplication/relaylog.go @@ -39,9 +39,9 @@ type relayLog struct { curSize int items [][]*binlogdatapb.VEvent err error - // canAccept is true if: curSize<=maxSize, len(items)0, err==nil, ctx is not Done, and call has not timedout. + // hasItems is true if len(items)>0, ctx is not Done, and call has not timedout. hasItems sync.Cond } @@ -124,9 +124,6 @@ func (rl *relayLog) Fetch() ([][]*binlogdatapb.VEvent, error) { } func (rl *relayLog) checkDone() error { - if rl.err != nil { - return rl.err - } select { case <-rl.ctx.Done(): return io.EOF @@ -135,15 +132,6 @@ func (rl *relayLog) checkDone() error { return nil } -func (rl *relayLog) SetError(err error) { - rl.mu.Lock() - defer rl.mu.Unlock() - - rl.err = err - rl.canAccept.Broadcast() - rl.hasItems.Broadcast() -} - func eventsSize(events []*binlogdatapb.VEvent) int { size := 0 for _, event := range events { diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index d79640bbcd0..9f21b3f2f50 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -125,7 +125,7 @@ func (vp *vplayer) play(ctx context.Context) error { return vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("Stop position %v already reached: %v", vp.pos, vp.stopPos)) } } - log.Infof("Starting VReplication player id: %v, startPos: %v, stop: %v, source: %v", vp.id, startPos, vp.stopPos, vp.sourceTablet) + log.Infof("Starting VReplication player id: %v, startPos: %v, stop: %v, source: %v, filter: %v", vp.id, startPos, vp.stopPos, vp.sourceTablet, vp.source) plan, err := buildPlayerPlan(vp.source.Filter) if err != nil { @@ -141,24 +141,40 @@ func (vp *vplayer) play(ctx context.Context) error { defer cancel() relay := newRelayLog(ctx, 10, 10) - go vp.applyEvents(relay) target := &querypb.Target{ Keyspace: vp.sourceTablet.Keyspace, Shard: vp.sourceTablet.Shard, TabletType: vp.sourceTablet.Type, } - return vsClient.VStream(ctx, target, startPos, plan.vstreamFilter, func(events []*binlogdatapb.VEvent) error { - return relay.Send(events) - }) + log.Infof("Sending vstream command: %v", plan.vstreamFilter) + streamErr := make(chan error, 1) + go func() { + streamErr <- vsClient.VStream(ctx, target, startPos, plan.vstreamFilter, func(events []*binlogdatapb.VEvent) error { + return relay.Send(events) + }) + }() + + applyErr := make(chan error, 1) + go func() { + applyErr <- vp.applyEvents(relay) + }() + + select { + case err = <-applyErr: + case err = <-streamErr: + } + if err == io.EOF { + return nil + } + return err } -func (vp *vplayer) applyEvents(relay *relayLog) { +func (vp *vplayer) applyEvents(relay *relayLog) error { for { items, err := relay.Fetch() if err != nil { - relay.SetError(err) - return + return err } // This covers two situations: // 1. Fetch was idle for idleTimeout. @@ -167,8 +183,7 @@ func (vp *vplayer) applyEvents(relay *relayLog) { // to prevent us from falling behind on tracking the source binlog position. if time.Now().Sub(vp.timeLastSaved) >= idleTimeout && vp.unsavedGTID != nil { if err := vp.updatePos(vp.unsavedGTID.Timestamp); err != nil { - relay.SetError(err) - return + return err } } for i, events := range items { @@ -187,8 +202,7 @@ func (vp *vplayer) applyEvents(relay *relayLog) { } } if err := vp.applyEvent(event); err != nil { - relay.SetError(err) - return + return err } } } @@ -354,25 +368,15 @@ func (vp *vplayer) updatePlan(fieldEvent *binlogdatapb.FieldEvent) error { } } } else { - if len(tplan.fields) != len(tplan.colExprs) { - return fmt.Errorf("columns received from vreplication: %v, do not match expected: %v", tplan.fields, tplan.colExprs) - } - for i, field := range tplan.fields { - if tplan.colExprs[i].colname.EqualString(field.Name) { - return fmt.Errorf("column name from vreplication field %d: %s, does not match expected: %s", i, field.Name, tplan.colExprs[i].colname) + for _, cExpr := range tplan.colExprs { + if cExpr.colnum >= len(tplan.fields) { + // Unreachable code. + return fmt.Errorf("columns received from vreplication: %v, do not match expected: %v", tplan.fields, tplan.colExprs) } } } - qr, err := vp.dbClient.ExecuteFetch("select database()", 1) - if err != nil { - return err - } - if len(qr.Rows) == 0 || len(qr.Rows[0]) == 0 { - return fmt.Errorf("unexpected result from 'select database()': %v", qr) - } - dbname := qr.Rows[0][0].ToString() - pkcols, err := vp.mysqld.GetPrimaryKeyColumns(dbname, tplan.name) + pkcols, err := vp.mysqld.GetPrimaryKeyColumns(vp.dbClient.DBName(), tplan.name) if err != nil { return fmt.Errorf("error fetching pk columns for %s: %v", tplan.name, err) } @@ -499,7 +503,7 @@ func (vp *vplayer) writeUpdateValues(sql *sqlparser.TrackedBuffer, tplan *tableP separator = ", " } if cExpr.op == opCount || cExpr.op == opSum { - sql.Myprintf("%s", cExpr.colname) + sql.Myprintf("%v", cExpr.colname) } if len(before) != 0 { switch cExpr.op { diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go index a349b026204..a59168f435c 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go @@ -18,8 +18,8 @@ package vreplication import ( "fmt" + "strings" "testing" - "time" "golang.org/x/net/context" "vitess.io/vitess/go/mysql" @@ -32,44 +32,65 @@ func TestSimple(t *testing.T) { defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) execStatements(t, []string{ - "create table t1(id int, val varbinary(128), primary key(id))", - "create table t2(id int, val varbinary(128), primary key(id))", + "create table src1(id int, val varbinary(128), primary key(id))", + "create table dst1(id int, val varbinary(128), primary key(id))", + "create table src2(id int, val1 int, val2 int, primary key(id))", + "create table dst2(id int, val1 int, sval2 int, rcount int, primary key(id))", }) defer execStatements(t, []string{ - "drop table t1", - "drop table t2", + "drop table src1", + "drop table dst1", + "drop table src2", + //"drop table dst2", }) env.SchemaEngine.Reload(context.Background()) filter := &binlogdatapb.Filter{ Rules: []*binlogdatapb.Rule{{ - Match: "t2", - Filter: "select * from t1", + Match: "dst1", + Filter: "select * from src1", + }, { + Match: "dst2", + Filter: "select id, val1, sum(val2) as sval2, count(*) as rcount from src2 group by id", }}, } cancel := startVReplication(t, playerEngine, filter, "") defer cancel() - execStatements(t, []string{ - "insert into t1 values(1, 'aaa')", - "insert into t1 values(2, 'aaa')", - "insert into t1 values(3, 'aaa')", - "insert into t1 values(4, 'aaa')", - }) - time.Sleep(1 * time.Second) - printQueries(t) - /* + testcases := []struct { + input string + output string + }{{ + input: "insert into src1 values(1, 'aaa')", + output: "insert into dst1 set id=1, val='aaa'", + }, { + input: "update src1 set val='bbb'", + output: "update dst1 set id=1, val='bbb' where id=1", + }, { + input: "delete from src1 where id=1", + output: "delete from dst1 where id=1", + }, { + input: "insert into src2 values(1, 2, 3)", + output: "insert into dst2 set id=1, val1=2, sval2=3, rcount=1 on duplicate key update val1=2, sval2=sval2+3, rcount=rcount+1", + }, { + input: "update src2 set val1=5, val2=1 where id=1", + output: "update dst2 set val1=5, sval2=sval2-3+1, rcount=rcount-1+1 where id=1", + }, { + input: "delete from src2 where id=1", + output: "update dst2 set val1=NULL, sval2=sval2-1, rcount=rcount-1 where id=1", + }} + + for _, tcases := range testcases { + execStatements(t, []string{ + tcases.input, + }) expectDBClientQueries(t, []string{ - "update _vt.vreplication set state='Running'.*", "begin", - "update _vt.vreplication set pos=.*", - "commit", - "begin", - "insert into t2 set id=1, val='aaa'", - "update _vt.vreplication set pos=.*", + tcases.output, + "/update _vt.vreplication set pos=.*", "commit", }) - */ + } } func execStatements(t *testing.T, queries []string) { @@ -99,7 +120,12 @@ func startVReplication(t *testing.T, pe *Engine, filter *binlogdatapb.Filter, po if err != nil { t.Fatal(err) } - resetDBClient() + // Eat all the initialization queries + for q := range globalDBQueries { + if strings.HasPrefix(q, "update") { + break + } + } return func() { query := fmt.Sprintf("delete from _vt.vreplication where id = %d", qr.InsertID) if _, err := pe.Exec(query); err != nil { From ed9216df2f4633b91f6cceccb043a98fe1324658 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Sat, 19 Jan 2019 14:46:47 -0800 Subject: [PATCH 10/27] vplayer: ddl tests and some fixes Signed-off-by: Sugu Sougoumarane --- go/vt/binlog/binlogplayer/binlog_player.go | 8 +- .../binlog/binlogplayer/binlog_player_test.go | 10 +- .../tabletmanager/vreplication/engine.go | 14 +- .../vreplication/framework_test.go | 34 ++- .../tabletmanager/vreplication/relaylog.go | 13 +- .../tabletmanager/vreplication/vplayer.go | 65 +++-- .../vreplication/vplayer_test.go | 246 +++++++++++++++--- .../tabletserver/vstreamer/vstreamer.go | 4 + .../tabletserver/vstreamer/vstreamer_test.go | 2 +- 9 files changed, 326 insertions(+), 70 deletions(-) diff --git a/go/vt/binlog/binlogplayer/binlog_player.go b/go/vt/binlog/binlogplayer/binlog_player.go index da81c6f1053..419bacf1624 100644 --- a/go/vt/binlog/binlogplayer/binlog_player.go +++ b/go/vt/binlog/binlogplayer/binlog_player.go @@ -601,11 +601,17 @@ func encodeString(in string) string { } // ReadVReplicationPos returns a statement to query the gtid for a -// given shard from the _vt.vreplication table. +// given stream from the _vt.vreplication table. func ReadVReplicationPos(index uint32) string { return fmt.Sprintf("select pos from _vt.vreplication where id=%v", index) } +// ReadVReplicationStatus returns a statement to query the status fields for a +// given stream from the _vt.vreplication table. +func ReadVReplicationStatus(index uint32) string { + return fmt.Sprintf("select pos, state, message from _vt.vreplication where id=%v", index) +} + // StatsHistoryRecord is used to store a Message with timestamp type StatsHistoryRecord struct { Time time.Time diff --git a/go/vt/binlog/binlogplayer/binlog_player_test.go b/go/vt/binlog/binlogplayer/binlog_player_test.go index 8db24607dab..4493dc33a37 100644 --- a/go/vt/binlog/binlogplayer/binlog_player_test.go +++ b/go/vt/binlog/binlogplayer/binlog_player_test.go @@ -377,6 +377,14 @@ func TestReadVReplicationPos(t *testing.T) { want := "select pos from _vt.vreplication where id=482821" got := ReadVReplicationPos(482821) if got != want { - t.Errorf("ReadVReplicationThrottlerSettings(482821) = %#v, want %#v", got, want) + t.Errorf("ReadVReplicationPos(482821) = %#v, want %#v", got, want) + } +} + +func TestReadVReplicationStatus(t *testing.T) { + want := "select pos, state, message from _vt.vreplication where id=482821" + got := ReadVReplicationStatus(482821) + if got != want { + t.Errorf("ReadVReplicationStatus(482821) = %#v, want %#v", got, want) } } diff --git a/go/vt/vttablet/tabletmanager/vreplication/engine.go b/go/vt/vttablet/tabletmanager/vreplication/engine.go index 3a664b131f5..e5f47456cd7 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/engine.go +++ b/go/vt/vttablet/tabletmanager/vreplication/engine.go @@ -315,13 +315,13 @@ func (vre *Engine) WaitForPos(ctx context.Context, id int, pos string) error { defer dbClient.Close() for { - qr, err := dbClient.ExecuteFetch(binlogplayer.ReadVReplicationPos(uint32(id)), 10) + qr, err := dbClient.ExecuteFetch(binlogplayer.ReadVReplicationStatus(uint32(id)), 10) switch { case err != nil: return err case len(qr.Rows) == 0: return fmt.Errorf("vreplication stream %d not found", id) - case len(qr.Rows) > 1 || len(qr.Rows[0]) != 1: + case len(qr.Rows) > 1 || len(qr.Rows[0]) != 3: return fmt.Errorf("unexpected result: %v", qr) } current, err := mysql.DecodePosition(qr.Rows[0][0].ToString()) @@ -329,10 +329,18 @@ func (vre *Engine) WaitForPos(ctx context.Context, id int, pos string) error { return err } - if current.AtLeast(mPos) { + if current.Equal(mPos) { return nil } + if current.AtLeast(mPos) { + return fmt.Errorf("postion %v has been overshot, current position is %v", mPos, current) + } + + if qr.Rows[0][1].ToString() == binlogplayer.BlpStopped { + return fmt.Errorf("replication has stopped at %v before reaching position %v, message: %s", current, mPos, qr.Rows[0][2].ToString()) + } + select { case <-ctx.Done(): return ctx.Err() diff --git a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go index 139313e3393..5149b3bf00d 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go @@ -50,6 +50,7 @@ var ( streamerEngine *vstreamer.Engine env *testenv.Env globalFBC = &fakeBinlogClient{} + vrepldb = "vrepl" globalDBQueries = make(chan string, 1000) ) @@ -89,8 +90,16 @@ func TestMain(m *testing.M) { streamerEngine.Open(env.KeyspaceName, env.Cells[0]) defer streamerEngine.Close() + if err := env.Mysqld.ExecuteSuperQuery(context.Background(), fmt.Sprintf("create database %s", vrepldb)); err != nil { + fmt.Fprintf(os.Stderr, "%v", err) + return 1 + } + playerEngine = NewEngine(env.TopoServ, env.Cells[0], env.Mysqld, realDBClientFactory) - playerEngine.Open(context.Background()) + if err := playerEngine.Open(context.Background()); err != nil { + fmt.Fprintf(os.Stderr, "%v", err) + return 1 + } defer playerEngine.Close() return m.Run() @@ -273,11 +282,13 @@ type realDBClient struct { } func (dbc *realDBClient) DBName() string { - return env.KeyspaceName + return vrepldb } func (dbc *realDBClient) Connect() error { - conn, err := mysql.Connect(context.Background(), env.Dbcfgs.AppWithDB()) + app := env.Dbcfgs.AppWithDB() + app.DbName = vrepldb + conn, err := mysql.Connect(context.Background(), app) if err != nil { return err } @@ -286,19 +297,21 @@ func (dbc *realDBClient) Connect() error { } func (dbc *realDBClient) Begin() error { - globalDBQueries <- "begin" _, err := dbc.conn.ExecuteFetch("begin", 10000, true) + globalDBQueries <- "begin" return err } func (dbc *realDBClient) Commit() error { - globalDBQueries <- "commit" _, err := dbc.conn.ExecuteFetch("commit", 10000, true) + globalDBQueries <- "commit" return err } func (dbc *realDBClient) Rollback() error { - panic("rollback should never be called") + _, err := dbc.conn.ExecuteFetch("rollback", 10000, true) + globalDBQueries <- "rollback" + return err } func (dbc *realDBClient) Close() { @@ -307,13 +320,14 @@ func (dbc *realDBClient) Close() { } func (dbc *realDBClient) ExecuteFetch(query string, maxrows int) (*sqltypes.Result, error) { - if !strings.HasPrefix(query, "select") { - globalDBQueries <- query - } if strings.HasPrefix(query, "use") { return nil, nil } - return dbc.conn.ExecuteFetch(query, 10000, true) + qr, err := dbc.conn.ExecuteFetch(query, 10000, true) + if !strings.HasPrefix(query, "select") { + globalDBQueries <- query + } + return qr, err } func expectDBClientQueries(t *testing.T, queries []string) { diff --git a/go/vt/vttablet/tabletmanager/vreplication/relaylog.go b/go/vt/vttablet/tabletmanager/vreplication/relaylog.go index 9be66dd75c4..3830d632a4e 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/relaylog.go +++ b/go/vt/vttablet/tabletmanager/vreplication/relaylog.go @@ -56,14 +56,11 @@ func newRelayLog(ctx context.Context, maxItems, maxSize int) *relayLog { // Any time context is done, wake up all waiters to make them exit. go func() { - select { - case <-ctx.Done(): - rl.mu.Lock() - defer rl.mu.Unlock() - rl.canAccept.Broadcast() - rl.hasItems.Broadcast() - default: - } + <-ctx.Done() + rl.mu.Lock() + defer rl.mu.Unlock() + rl.canAccept.Broadcast() + rl.hasItems.Broadcast() }() return rl } diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index 9f21b3f2f50..c53973a28e0 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -18,6 +18,7 @@ package vreplication import ( "bytes" + "errors" "fmt" "io" "sync" @@ -108,7 +109,7 @@ func (vp *vplayer) Play(ctx context.Context) error { func (vp *vplayer) play(ctx context.Context) error { startPos, stopPos, _, _, err := binlogplayer.ReadVRSettings(vp.dbClient, vp.id) if err != nil { - return fmt.Errorf("error reading VReplication settings: %v", err) + return vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("error reading VReplication settings: %v", err)) } vp.pos, err = mysql.DecodePosition(startPos) if err != nil { @@ -117,7 +118,7 @@ func (vp *vplayer) play(ctx context.Context) error { if stopPos != "" { vp.stopPos, err = mysql.DecodePosition(stopPos) if err != nil { - return fmt.Errorf("error decoding stop position %v: %v", stopPos, err) + return vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("error decoding stop position %v: %v", stopPos, err)) } } if !vp.stopPos.IsZero() { @@ -161,13 +162,39 @@ func (vp *vplayer) play(ctx context.Context) error { }() select { - case err = <-applyErr: - case err = <-streamErr: - } - if err == io.EOF { - return nil + case err := <-applyErr: + defer func() { + // cancel and wait for the other thread to finish. + cancel() + <-streamErr + }() + // If the apply thread ends with io.EOF, it means the context + // was canceled, which can only happen if Engine is shutting down. + // If so, we return nil which will cause the vplayer to shut down. + if err == io.EOF { + return nil + } + return err + case err := <-streamErr: + defer func() { + // cancel and wait for the other thread to finish. + cancel() + <-applyErr + }() + // If context is done, don't return an error. + select { + case <-ctx.Done(): + return nil + default: + } + // If the stream ends normally without context being canceled, + // we have to return an error indicating that the controller + // has to retry a different vttablet. + if err == nil { + return errors.New("vstream ended") + } + return err } - return err } func (vp *vplayer) applyEvents(relay *relayLog) error { @@ -261,7 +288,7 @@ func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { return nil } if !vp.pos.Equal(vp.stopPos) && vp.pos.AtLeast(vp.stopPos) { - return fmt.Errorf("next event position %v exceeds stop pos %v, exiting without applying", vp.pos, vp.stopPos) + return vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("next event position %v exceeds stop pos %v, exiting without applying", vp.pos, vp.stopPos)) } case binlogdatapb.VEventType_BEGIN: // No-op: begin is called as needed. @@ -433,6 +460,9 @@ func (vp *vplayer) applyRowChange(tplan *tablePlan, rowChange *binlogdatapb.RowC case before == nil && after == nil: // unreachable } + if query == "" { + return nil + } return vp.exec(query) } @@ -453,7 +483,7 @@ func (vp *vplayer) generateInsert(tplan *tablePlan, after []sqltypes.Value) stri func (vp *vplayer) generateUpdate(tplan *tablePlan, before, after []sqltypes.Value) string { if tplan.onInsert == insertIgnore { - return "" + return vp.generateInsert(tplan, after) } sql := sqlparser.NewTrackedBuffer(nil) sql.Myprintf("update %v set ", sqlparser.NewTableIdent(tplan.name)) @@ -465,15 +495,18 @@ func (vp *vplayer) generateUpdate(tplan *tablePlan, before, after []sqltypes.Val func (vp *vplayer) generateDelete(tplan *tablePlan, before []sqltypes.Value) string { sql := sqlparser.NewTrackedBuffer(nil) - if tplan.onInsert == insertNormal { + switch tplan.onInsert { + case insertOndup: + sql.Myprintf("update %v set ", sqlparser.NewTableIdent(tplan.name)) + vp.writeUpdateValues(sql, tplan, before, nil) + sql.Myprintf(" where ") + vp.writeWhereValues(sql, tplan, before) + case insertIgnore: + return "" + default: // insertNormal sql.Myprintf("delete from %v where ", sqlparser.NewTableIdent(tplan.name)) vp.writeWhereValues(sql, tplan, before) - return sql.String() } - sql.Myprintf("update %v set ", sqlparser.NewTableIdent(tplan.name)) - vp.writeUpdateValues(sql, tplan, before, nil) - sql.Myprintf(" where ") - vp.writeWhereValues(sql, tplan, before) return sql.String() } diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go index a59168f435c..c0082879697 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go @@ -28,20 +28,30 @@ import ( topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) -func TestSimple(t *testing.T) { +func TestFilters(t *testing.T) { defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) execStatements(t, []string{ "create table src1(id int, val varbinary(128), primary key(id))", - "create table dst1(id int, val varbinary(128), primary key(id))", + fmt.Sprintf("create table %s.dst1(id int, val varbinary(128), primary key(id))", vrepldb), "create table src2(id int, val1 int, val2 int, primary key(id))", - "create table dst2(id int, val1 int, sval2 int, rcount int, primary key(id))", + fmt.Sprintf("create table %s.dst2(id int, val1 int, sval2 int, rcount int, primary key(id))", vrepldb), + "create table src3(id int, val varbinary(128), primary key(id))", + fmt.Sprintf("create table %s.dst3(id int, val varbinary(128), 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))", }) defer execStatements(t, []string{ "drop table src1", - "drop table dst1", + fmt.Sprintf("drop table %s.dst1", vrepldb), "drop table src2", - //"drop table dst2", + fmt.Sprintf("drop table %s.dst2", vrepldb), + "drop table src3", + fmt.Sprintf("drop table %s.dst3", vrepldb), + "drop table yes", + fmt.Sprintf("drop table %s.yes", vrepldb), + "drop table no", }) env.SchemaEngine.Reload(context.Background()) @@ -52,45 +62,216 @@ func TestSimple(t *testing.T) { }, { Match: "dst2", Filter: "select id, val1, sum(val2) as sval2, count(*) as rcount from src2 group by id", + }, { + Match: "dst3", + Filter: "select id, val from src3 group by id, val", + }, { + Match: "/yes", }}, } - cancel := startVReplication(t, playerEngine, filter, "") + cancel, _ := startVReplication(t, playerEngine, filter, binlogdatapb.OnDDLAction_IGNORE, "") defer cancel() testcases := []struct { input string - output string + output []string }{{ - input: "insert into src1 values(1, 'aaa')", - output: "insert into dst1 set id=1, val='aaa'", + // insert with insertNormal + input: "insert into src1 values(1, 'aaa')", + output: []string{ + "begin", + "insert into dst1 set id=1, val='aaa'", + "/update _vt.vreplication set pos=", + "commit", + }, + }, { + // update with insertNormal + input: "update src1 set val='bbb'", + output: []string{ + "begin", + "update dst1 set id=1, val='bbb' where id=1", + "/update _vt.vreplication set pos=", + "commit", + }, + }, { + // delete with insertNormal + input: "delete from src1 where id=1", + output: []string{ + "begin", + "delete from dst1 where id=1", + "/update _vt.vreplication set pos=", + "commit", + }, + }, { + // insert with insertOnDup + input: "insert into src2 values(1, 2, 3)", + output: []string{ + "begin", + "insert into dst2 set id=1, val1=2, sval2=3, rcount=1 on duplicate key update val1=2, sval2=sval2+3, rcount=rcount+1", + "/update _vt.vreplication set pos=", + "commit", + }, + }, { + // update with insertOnDup + input: "update src2 set val1=5, val2=1 where id=1", + output: []string{ + "begin", + "update dst2 set val1=5, sval2=sval2-3+1, rcount=rcount-1+1 where id=1", + "/update _vt.vreplication set pos=", + "commit", + }, }, { - input: "update src1 set val='bbb'", - output: "update dst1 set id=1, val='bbb' where id=1", + // delete with insertOnDup + input: "delete from src2 where id=1", + output: []string{ + "begin", + "update dst2 set val1=NULL, sval2=sval2-1, rcount=rcount-1 where id=1", + "/update _vt.vreplication set pos=", + "commit", + }, + }, { + // insert with insertIgnore + input: "insert into src3 values(1, 'aaa')", + output: []string{ + "begin", + "insert ignore into dst3 set id=1, val='aaa'", + "/update _vt.vreplication set pos=", + "commit", + }, + }, { + // update with insertIgnore + input: "update src3 set val='bbb'", + output: []string{ + "begin", + "insert ignore into dst3 set id=1, val='bbb'", + "/update _vt.vreplication set pos=", + "commit", + }, }, { - input: "delete from src1 where id=1", - output: "delete from dst1 where id=1", + // delete with insertIgnore + input: "delete from src3 where id=1", + output: []string{ + "begin", + "/update _vt.vreplication set pos=", + "commit", + }, }, { - input: "insert into src2 values(1, 2, 3)", - output: "insert into dst2 set id=1, val1=2, sval2=3, rcount=1 on duplicate key update val1=2, sval2=sval2+3, rcount=rcount+1", + // insert: regular expression filter + input: "insert into yes values(1, 'aaa')", + output: []string{ + "begin", + "insert into yes set id=1, val='aaa'", + "/update _vt.vreplication set pos=", + "commit", + }, }, { - input: "update src2 set val1=5, val2=1 where id=1", - output: "update dst2 set val1=5, sval2=sval2-3+1, rcount=rcount-1+1 where id=1", + // update: regular expression filter + input: "update yes set val='bbb'", + output: []string{ + "begin", + "update yes set id=1, val='bbb' where id=1", + "/update _vt.vreplication set pos=", + "commit", + }, }, { - input: "delete from src2 where id=1", - output: "update dst2 set val1=NULL, sval2=sval2-1, rcount=rcount-1 where id=1", + // table should not match a rule + input: "insert into no values(1, 'aaa')", + output: []string{}, }} for _, tcases := range testcases { - execStatements(t, []string{ - tcases.input, - }) - expectDBClientQueries(t, []string{ - "begin", - tcases.output, - "/update _vt.vreplication set pos=.*", - "commit", - }) + execStatements(t, []string{tcases.input}) + expectDBClientQueries(t, tcases.output) + } +} + +func TestDDL(t *testing.T) { + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "/.*", + }}, + } + + cancel, _ := startVReplication(t, playerEngine, filter, binlogdatapb.OnDDLAction_IGNORE, "") + execStatements(t, []string{"create table t1(id int, primary key(id))"}) + execStatements(t, []string{"drop table t1"}) + expectDBClientQueries(t, []string{}) + cancel() + + cancel, id := startVReplication(t, playerEngine, filter, binlogdatapb.OnDDLAction_STOP, "") + execStatements(t, []string{"create table t1(id int, primary key(id))"}) + pos1 := masterPosition(t) + execStatements(t, []string{"drop table t1"}) + pos2 := masterPosition(t) + // The stop position must be the GTID of the first DDL + expectDBClientQueries(t, []string{ + "begin", + fmt.Sprintf("/update _vt.vreplication set pos='%s'", pos1), + "/update _vt.vreplication set state='Stopped'", + "commit", + }) + // Restart vreplication + if _, err := playerEngine.Exec(fmt.Sprintf(`update _vt.vreplication set state = 'Running', message='' where id=%d`, id)); err != nil { + t.Fatal(err) } + // It should stop at the next DDL + expectDBClientQueries(t, []string{ + "/update.*'Running'", + "/update.*'Running'", + "begin", + fmt.Sprintf("/update.*'%s'", pos2), + "/update _vt.vreplication set state='Stopped'", + "commit", + }) + cancel() + + execStatements(t, []string{fmt.Sprintf("create table %s.t2(id int, primary key(id))", vrepldb)}) + cancel, _ = startVReplication(t, playerEngine, filter, binlogdatapb.OnDDLAction_EXEC, "") + execStatements(t, []string{"create table t1(id int, primary key(id))"}) + expectDBClientQueries(t, []string{ + "create table t1(id int, primary key(id))", + "/update _vt.vreplication set pos=", + }) + execStatements(t, []string{"create table t2(id int, primary key(id))"}) + expectDBClientQueries(t, []string{ + "create table t2(id int, primary key(id))", + "/update _vt.vreplication set state='Error'", + }) + cancel() + + // Don't test drop. + // MySQL rewrites them by uppercasing, which may be version specific. + execStatements(t, []string{ + "drop table t1", + fmt.Sprintf("drop table %s.t1", vrepldb), + "drop table t2", + fmt.Sprintf("drop table %s.t2", vrepldb), + }) + + execStatements(t, []string{fmt.Sprintf("create table %s.t2(id int, primary key(id))", vrepldb)}) + cancel, _ = startVReplication(t, playerEngine, filter, binlogdatapb.OnDDLAction_EXEC_IGNORE, "") + execStatements(t, []string{"create table t1(id int, primary key(id))"}) + expectDBClientQueries(t, []string{ + "create table t1(id int, primary key(id))", + "/update _vt.vreplication set pos=", + }) + execStatements(t, []string{"create table t2(id int, primary key(id))"}) + expectDBClientQueries(t, []string{ + "create table t2(id int, primary key(id))", + "/update _vt.vreplication set pos=", + }) + cancel() + + // Don't test drop. + // MySQL rewrites them by uppercasing, which may be version specific. + execStatements(t, []string{ + "drop table t1", + fmt.Sprintf("drop table %s.t1", vrepldb), + "drop table t2", + fmt.Sprintf("drop table %s.t2", vrepldb), + }) } func execStatements(t *testing.T, queries []string) { @@ -100,13 +281,14 @@ func execStatements(t *testing.T, queries []string) { } } -func startVReplication(t *testing.T, pe *Engine, filter *binlogdatapb.Filter, pos string) (cancelFunc func()) { +func startVReplication(t *testing.T, pe *Engine, filter *binlogdatapb.Filter, onddl binlogdatapb.OnDDLAction, pos string) (cancelFunc func(), id int) { t.Helper() bls := &binlogdatapb.BinlogSource{ Keyspace: env.KeyspaceName, Shard: env.ShardName, Filter: filter, + OnDdl: onddl, } if pos == "" { pos = masterPosition(t) @@ -127,11 +309,15 @@ func startVReplication(t *testing.T, pe *Engine, filter *binlogdatapb.Filter, po } } return func() { + t.Helper() query := fmt.Sprintf("delete from _vt.vreplication where id = %d", qr.InsertID) if _, err := pe.Exec(query); err != nil { t.Fatal(err) } - } + expectDBClientQueries(t, []string{ + "/delete", + }) + }, int(qr.InsertID) } func masterPosition(t *testing.T) string { diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go index 6c8631a49ba..4bf378d9f81 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go +++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer.go @@ -20,6 +20,7 @@ import ( "context" "flag" "fmt" + "io" "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/sqltypes" @@ -183,6 +184,9 @@ func (vs *vstreamer) parseEvents(ctx context.Context, events <-chan mysql.Binlog } for _, vevent := range vevents { if err := bufferAndTransmit(vevent); err != nil { + if err == io.EOF { + return nil + } return fmt.Errorf("error sending event: %v", err) } } diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go index 7c345cd3cdd..9cf671a3b7b 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go @@ -954,7 +954,7 @@ func startStream(ctx context.Context, t *testing.T, filter *binlogdatapb.Filter) go func() { defer close(ch) if err := vstream(ctx, t, pos, filter, ch); err != nil { - t.Fatal(err) + t.Error(err) } }() return ch From 4b6d22f034902b861fd3d0d568558ab25f4b36ae Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Sat, 19 Jan 2019 16:00:11 -0800 Subject: [PATCH 11/27] vplayer: fix tests and mariadb103 issues Signed-off-by: Sugu Sougoumarane --- go/vt/vttablet/endtoend/compatibility_test.go | 9 ++++++- go/vt/vttablet/endtoend/misc_test.go | 14 ++++++++--- .../tabletmanager/vreplication/engine.go | 24 +++++++------------ .../tabletmanager/vreplication/engine_test.go | 20 +++++++++++----- .../tabletserver/vstreamer/vstreamer_test.go | 4 +--- .../testlib/migrate_served_from_test.go | 6 +++-- .../testlib/migrate_served_types_test.go | 12 ++++++---- 7 files changed, 55 insertions(+), 34 deletions(-) diff --git a/go/vt/vttablet/endtoend/compatibility_test.go b/go/vt/vttablet/endtoend/compatibility_test.go index 6edc6628e4b..1bf90904633 100644 --- a/go/vt/vttablet/endtoend/compatibility_test.go +++ b/go/vt/vttablet/endtoend/compatibility_test.go @@ -794,7 +794,14 @@ func TestJSONType(t *testing.T) { }, } if !reflect.DeepEqual(*qr, want) { - t.Errorf("Execute: \n%v, want \n%v", prettyPrint(*qr), prettyPrint(want)) + // MariaDB 10.3 has different behavior. + want2 := want.Copy() + want2.Fields[1].Type = sqltypes.Blob + want2.Fields[1].Charset = 33 + want2.Rows[0][1] = sqltypes.TestValue(sqltypes.Blob, "{\"foo\": \"bar\"}") + if !reflect.DeepEqual(*qr, *want2) { + t.Errorf("Execute:\n%v, want\n%v or\n%v", prettyPrint(*qr), prettyPrint(want), prettyPrint(*want2)) + } } } diff --git a/go/vt/vttablet/endtoend/misc_test.go b/go/vt/vttablet/endtoend/misc_test.go index 5e961a9ae32..81fb606ba0a 100644 --- a/go/vt/vttablet/endtoend/misc_test.go +++ b/go/vt/vttablet/endtoend/misc_test.go @@ -364,7 +364,13 @@ func TestBindInSelect(t *testing.T) { }, } if !qr.Equal(want) { - t.Errorf("Execute: \n%#v, want \n%#v", prettyPrint(*qr), prettyPrint(*want)) + // MariaDB 10.3 has different behavior. + want2 := want.Copy() + want2.Fields[0].Type = sqltypes.Int32 + want2.Rows[0][0] = sqltypes.NewInt32(1) + if !qr.Equal(want2) { + t.Errorf("Execute:\n%v, want\n%v or\n%v", prettyPrint(*qr), prettyPrint(*want), prettyPrint(*want2)) + } } // String bind var. @@ -382,7 +388,6 @@ func TestBindInSelect(t *testing.T) { Type: sqltypes.VarChar, ColumnLength: 12, Charset: 33, - Decimals: 31, Flags: 1, }}, RowsAffected: 1, @@ -392,6 +397,8 @@ func TestBindInSelect(t *testing.T) { }, }, } + // MariaDB 10.3 has different behavior. + qr.Fields[0].Decimals = 0 if !qr.Equal(want) { t.Errorf("Execute: \n%#v, want \n%#v", prettyPrint(*qr), prettyPrint(*want)) } @@ -411,7 +418,6 @@ func TestBindInSelect(t *testing.T) { Type: sqltypes.VarChar, ColumnLength: 6, Charset: 33, - Decimals: 31, Flags: 1, }}, RowsAffected: 1, @@ -421,6 +427,8 @@ func TestBindInSelect(t *testing.T) { }, }, } + // MariaDB 10.3 has different behavior. + qr.Fields[0].Decimals = 0 if !qr.Equal(want) { t.Errorf("Execute: \n%#v, want \n%#v", prettyPrint(*qr), prettyPrint(*want)) } diff --git a/go/vt/vttablet/tabletmanager/vreplication/engine.go b/go/vt/vttablet/tabletmanager/vreplication/engine.go index e5f47456cd7..740c497626d 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/engine.go +++ b/go/vt/vttablet/tabletmanager/vreplication/engine.go @@ -288,17 +288,15 @@ func (vre *Engine) WaitForPos(ctx context.Context, id int, pos string) error { if !vre.isOpen { return errors.New("vreplication engine is closed") } - ct, ok := vre.controllers[id] - if !ok { - return fmt.Errorf("vreplication stream %d not found", id) - } - mpos, err := mysql.DecodePosition(pos) - if err != nil { - return err + if ct, ok := vre.controllers[id]; ok { + mpos, err := mysql.DecodePosition(pos) + if err != nil { + return err + } + // vplayer doesn't export all the positions it receives unless + // we specifically request it for one. + ct.exportPosition(mpos) } - // vplayer doesn't export all the positions it receives unless - // we specifically request it for one. - ct.exportPosition(mpos) // Ensure that the engine won't be closed while this is running. vre.wg.Add(1) @@ -329,12 +327,8 @@ func (vre *Engine) WaitForPos(ctx context.Context, id int, pos string) error { return err } - if current.Equal(mPos) { - return nil - } - if current.AtLeast(mPos) { - return fmt.Errorf("postion %v has been overshot, current position is %v", mPos, current) + return nil } if qr.Rows[0][1].ToString() == binlogplayer.BlpStopped { diff --git a/go/vt/vttablet/tabletmanager/vreplication/engine_test.go b/go/vt/vttablet/tabletmanager/vreplication/engine_test.go index 8394a8328f0..9bf21378e85 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/engine_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/engine_test.go @@ -286,11 +286,15 @@ func TestWaitForPos(t *testing.T) { // hack a controller into vre. vre.controllers[1] = &controller{} - dbClient.ExpectRequest("select pos from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ + dbClient.ExpectRequest("select pos, state, message from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ sqltypes.NewVarBinary("MariaDB/0-1-1083"), + sqltypes.NewVarBinary("Running"), + sqltypes.NewVarBinary(""), }}}, nil) - dbClient.ExpectRequest("select pos from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ + dbClient.ExpectRequest("select pos, state, message from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ sqltypes.NewVarBinary("MariaDB/0-1-1084"), + sqltypes.NewVarBinary("Running"), + sqltypes.NewVarBinary(""), }}}, nil) start := time.Now() if err := vre.WaitForPos(context.Background(), 1, "MariaDB/0-1-1084"); err != nil { @@ -326,14 +330,14 @@ func TestWaitForPosError(t *testing.T) { t.Errorf("WaitForPos: %v, want %v", err, want) } - dbClient.ExpectRequest("select pos from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{}}}, nil) + dbClient.ExpectRequest("select pos, state, message from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{}}}, nil) err = vre.WaitForPos(context.Background(), 1, "MariaDB/0-1-1084") want = "unexpected result: &{[] 0 0 [[]] }" if err == nil || err.Error() != want { t.Errorf("WaitForPos: %v, want %v", err, want) } - dbClient.ExpectRequest("select pos from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ + dbClient.ExpectRequest("select pos, state, message from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ sqltypes.NewVarBinary("MariaDB/0-1-1083"), }, { sqltypes.NewVarBinary("MariaDB/0-1-1083"), @@ -356,8 +360,10 @@ func TestWaitForPosCancel(t *testing.T) { t.Fatal(err) } - dbClient.ExpectRequest("select pos from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ + dbClient.ExpectRequest("select pos, state, message from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ sqltypes.NewVarBinary("MariaDB/0-1-1083"), + sqltypes.NewVarBinary("Running"), + sqltypes.NewVarBinary(""), }}}, nil) ctx, cancel := context.WithCancel(context.Background()) cancel() @@ -378,8 +384,10 @@ func TestWaitForPosCancel(t *testing.T) { time.Sleep(5 * time.Millisecond) vre.Close() }() - dbClient.ExpectRequest("select pos from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ + dbClient.ExpectRequest("select pos, state, message from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ sqltypes.NewVarBinary("MariaDB/0-1-1083"), + sqltypes.NewVarBinary("Running"), + sqltypes.NewVarBinary(""), }}}, nil) err = vre.WaitForPos(context.Background(), 1, "MariaDB/0-1-1084") want := "vreplication is closing: context canceled" diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go index 9cf671a3b7b..5d39189e9f2 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go @@ -741,9 +741,7 @@ func TestTypes(t *testing.T) { } func TestJSON(t *testing.T) { - if testing.Short() { - t.Skip() - } + t.Skip("This test is disabled because every flavor of mysql has a different behavior.") // JSON is supported only after mysql57. if err := env.Mysqld.ExecuteSuperQuery(context.Background(), "create table vitess_json(id int default 1, val json, primary key(id))"); err != nil { diff --git a/go/vt/wrangler/testlib/migrate_served_from_test.go b/go/vt/wrangler/testlib/migrate_served_from_test.go index 6194e2150ed..303e820e5b5 100644 --- a/go/vt/wrangler/testlib/migrate_served_from_test.go +++ b/go/vt/wrangler/testlib/migrate_served_from_test.go @@ -111,9 +111,11 @@ func TestMigrateServedFrom(t *testing.T) { if err := destMaster.Agent.VREngine.Open(context.Background()); err != nil { t.Fatal(err) } - // select pos from _vt.vreplication - dbClient.ExpectRequest("select pos from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ + // select pos, state, message from _vt.vreplication + dbClient.ExpectRequest("select pos, state, message from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ sqltypes.NewVarBinary("MariaDB/5-456-892"), + sqltypes.NewVarBinary("Running"), + sqltypes.NewVarBinary(""), }}}, nil) dbClient.ExpectRequest("use _vt", &sqltypes.Result{}, nil) dbClient.ExpectRequest("delete from _vt.vreplication where id = 1", &sqltypes.Result{RowsAffected: 1}, nil) diff --git a/go/vt/wrangler/testlib/migrate_served_types_test.go b/go/vt/wrangler/testlib/migrate_served_types_test.go index 28ae7789ffe..e0e8b849420 100644 --- a/go/vt/wrangler/testlib/migrate_served_types_test.go +++ b/go/vt/wrangler/testlib/migrate_served_types_test.go @@ -147,9 +147,11 @@ func TestMigrateServedTypes(t *testing.T) { if err := dest1Master.Agent.VREngine.Open(context.Background()); err != nil { t.Fatal(err) } - // select pos from _vt.vreplication - dbClient1.ExpectRequest("select pos from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ + // select pos, state, message from _vt.vreplication + dbClient1.ExpectRequest("select pos, state, message from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ sqltypes.NewVarBinary("MariaDB/5-456-892"), + sqltypes.NewVarBinary("Running"), + sqltypes.NewVarBinary(""), }}}, nil) dbClient1.ExpectRequest("use _vt", &sqltypes.Result{}, nil) dbClient1.ExpectRequest("delete from _vt.vreplication where id = 1", &sqltypes.Result{RowsAffected: 1}, nil) @@ -174,9 +176,11 @@ func TestMigrateServedTypes(t *testing.T) { if err := dest2Master.Agent.VREngine.Open(context.Background()); err != nil { t.Fatal(err) } - // select pos from _vt.vreplication - dbClient2.ExpectRequest("select pos from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ + // select pos, state, message from _vt.vreplication + dbClient2.ExpectRequest("select pos, state, message from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ sqltypes.NewVarBinary("MariaDB/5-456-892"), + sqltypes.NewVarBinary("Running"), + sqltypes.NewVarBinary(""), }}}, nil) dbClient2.ExpectRequest("use _vt", &sqltypes.Result{}, nil) dbClient2.ExpectRequest("delete from _vt.vreplication where id = 1", &sqltypes.Result{RowsAffected: 1}, nil) From 4126ddbf45e9c6ce06e45a55cb6ce90046c8a838 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Sun, 20 Jan 2019 01:20:05 -0800 Subject: [PATCH 12/27] vplayer: test stopPos Signed-off-by: Sugu Sougoumarane --- .../tabletmanager/vreplication/vplayer.go | 33 ++++-- .../vreplication/vplayer_test.go | 101 +++++++++++++++++- 2 files changed, 119 insertions(+), 15 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index c53973a28e0..0168c063898 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -113,7 +113,7 @@ func (vp *vplayer) play(ctx context.Context) error { } vp.pos, err = mysql.DecodePosition(startPos) if err != nil { - return fmt.Errorf("error decoding start position %v: %v", startPos, err) + return vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("error decoding start position %v: %v", startPos, err)) } if stopPos != "" { vp.stopPos, err = mysql.DecodePosition(stopPos) @@ -168,9 +168,9 @@ func (vp *vplayer) play(ctx context.Context) error { cancel() <-streamErr }() - // If the apply thread ends with io.EOF, it means the context - // was canceled, which can only happen if Engine is shutting down. - // If so, we return nil which will cause the vplayer to shut down. + // If the apply thread ends with io.EOF, it means either the Engine + // is shutting down and canceled the context, or stop position was reached. + // If so, we return nil which will cause the controller to not retry. if err == io.EOF { return nil } @@ -187,10 +187,9 @@ func (vp *vplayer) play(ctx context.Context) error { return nil default: } - // If the stream ends normally without context being canceled, - // we have to return an error indicating that the controller - // has to retry a different vttablet. - if err == nil { + // If the stream ends normally we have to return an error indicating + // that the controller has to retry a different vttablet. + if err == nil || err == io.EOF { return errors.New("vstream ended") } return err @@ -288,18 +287,32 @@ func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { return nil } if !vp.pos.Equal(vp.stopPos) && vp.pos.AtLeast(vp.stopPos) { - return vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("next event position %v exceeds stop pos %v, exiting without applying", vp.pos, vp.stopPos)) + // Code is unreachable, but bad data can cause this to happen. + if err := vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("next event position %v exceeds stop pos %v, exiting without applying", vp.pos, vp.stopPos)); err != nil { + return err + } + return io.EOF } case binlogdatapb.VEventType_BEGIN: // No-op: begin is called as needed. case binlogdatapb.VEventType_COMMIT: + posReached := !vp.stopPos.IsZero() && vp.pos.Equal(vp.stopPos) + // If stop pos is reached, then we must commit. + // So, if we haven't started a transaction (vp.mustCommit==false), + // we must start one. + if posReached && !vp.mustCommit { + if err := vp.dbClient.Begin(); err != nil { + return err + } + vp.mustCommit = true + } + if !vp.mustCommit { return nil } if err := vp.updatePos(event.Timestamp); err != nil { return err } - posReached := !vp.stopPos.IsZero() && vp.pos.Equal(vp.stopPos) if posReached { if err := vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("Stopped at position %v", vp.stopPos)); err != nil { return err diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go index c0082879697..2d9d796a431 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go @@ -24,6 +24,7 @@ import ( "golang.org/x/net/context" "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/vt/binlog/binlogplayer" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) @@ -274,6 +275,100 @@ func TestDDL(t *testing.T) { }) } +func TestStopPos(t *testing.T) { + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + + 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))", + }) + defer execStatements(t, []string{ + "drop table yes", + fmt.Sprintf("drop table %s.yes", vrepldb), + "drop table no", + }) + env.SchemaEngine.Reload(context.Background()) + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "/yes", + }}, + } + bls := &binlogdatapb.BinlogSource{ + Keyspace: env.KeyspaceName, + Shard: env.ShardName, + Filter: filter, + OnDdl: binlogdatapb.OnDDLAction_IGNORE, + } + startPos := masterPosition(t) + query := binlogplayer.CreateVReplicationStopped("test", bls, startPos) + qr, err := playerEngine.Exec(query) + if err != nil { + t.Fatal(err) + } + id := uint32(qr.InsertID) + for q := range globalDBQueries { + if strings.HasPrefix(q, "insert into _vt.vreplication") { + break + } + } + + // Test normal stop. + execStatements(t, []string{ + "insert into yes values(1, 'aaa')", + }) + stopPos := masterPosition(t) + query = binlogplayer.StartVReplicationUntil(id, stopPos) + if _, err := playerEngine.Exec(query); err != nil { + t.Fatal(err) + } + expectDBClientQueries(t, []string{ + "/update.*'Running'", // done by Engine + "/update.*'Running'", // done by vplayer on start + "begin", + "insert into yes set id=1, val='aaa'", + fmt.Sprintf("/update.*'%s'", stopPos), + "/update.*'Stopped'", + "commit", + }) + + // Test stopping at empty transaction. + execStatements(t, []string{ + "insert into no values(2, 'aaa')", + "insert into no values(3, 'aaa')", + }) + stopPos = masterPosition(t) + execStatements(t, []string{ + "insert into no values(4, 'aaa')", + }) + query = binlogplayer.StartVReplicationUntil(id, stopPos) + if _, err := playerEngine.Exec(query); err != nil { + t.Fatal(err) + } + expectDBClientQueries(t, []string{ + "/update.*'Running'", // done by Engine + "/update.*'Running'", // done by vplayer on start + "begin", + // Since 'no' generates empty transactions that are skipped by + // vplayer, a commit is done only for the stop position event. + fmt.Sprintf("/update.*'%s'", stopPos), + "/update.*'Stopped'", + "commit", + }) + + // Test stopping when position is already reached. + query = binlogplayer.StartVReplicationUntil(id, stopPos) + if _, err := playerEngine.Exec(query); err != nil { + t.Fatal(err) + } + expectDBClientQueries(t, []string{ + "/update.*'Running'", // done by Engine + "/update.*'Running'", // done by vplayer on start + "/update.*'Stopped'.*already reached", + }) +} + func execStatements(t *testing.T, queries []string) { t.Helper() if err := env.Mysqld.ExecuteSuperQueryList(context.Background(), queries); err != nil { @@ -293,11 +388,7 @@ func startVReplication(t *testing.T, pe *Engine, filter *binlogdatapb.Filter, on if pos == "" { pos = masterPosition(t) } - query := fmt.Sprintf(`insert into _vt.vreplication`+ - `(workflow, source, pos, max_tps, max_replication_lag, time_updated, transaction_timestamp, state)`+ - `values('test', '%v', '%s', 9223372036854775807, 9223372036854775807, 481823, 0, 'Running')`, - bls, pos, - ) + query := binlogplayer.CreateVReplication("test", bls, pos, 9223372036854775807, 9223372036854775807, 0) qr, err := pe.Exec(query) if err != nil { t.Fatal(err) From 9a63e3864748dc665d29bb83a320e895cc011b84 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Sun, 20 Jan 2019 13:29:06 -0800 Subject: [PATCH 13/27] vplayer: improved state management Signed-off-by: Sugu Sougoumarane --- .../tabletmanager/vreplication/controller.go | 8 +- .../tabletmanager/vreplication/relaylog.go | 56 ++++++----- .../tabletmanager/vreplication/vplayer.go | 93 ++++++++++++------- .../vreplication/vplayer_test.go | 23 ++++- 4 files changed, 120 insertions(+), 60 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vreplication/controller.go b/go/vt/vttablet/tabletmanager/vreplication/controller.go index 574b1484d21..37186eb21f7 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/controller.go +++ b/go/vt/vttablet/tabletmanager/vreplication/controller.go @@ -136,7 +136,13 @@ func (ct *controller) run(ctx context.Context) { default: } log.Errorf("stream %v: %v, retrying after %v", ct.id, err, *retryDelay) - time.Sleep(*retryDelay) + timer := time.NewTimer(*retryDelay) + select { + case <-ctx.Done(): + timer.Stop() + return + case <-timer.C: + } } } diff --git a/go/vt/vttablet/tabletmanager/vreplication/relaylog.go b/go/vt/vttablet/tabletmanager/vreplication/relaylog.go index 3830d632a4e..64504b5457a 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/relaylog.go +++ b/go/vt/vttablet/tabletmanager/vreplication/relaylog.go @@ -23,7 +23,6 @@ import ( "golang.org/x/net/context" - "vitess.io/vitess/go/sync2" binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" ) @@ -35,13 +34,14 @@ type relayLog struct { // mu controls all variables below and is shared by canAccept and hasItems. // Broadcasting must be done while holding mu. This is mainly necessary because both // conditions depend on ctx.Done(), which can change state asynchronously. - mu sync.Mutex - curSize int - items [][]*binlogdatapb.VEvent - err error + mu sync.Mutex + curSize int + items [][]*binlogdatapb.VEvent + interruptFetch bool + err error // canAccept is true if: curSize<=maxSize, len(items)0, ctx is not Done, and call has not timedout. + // hasItems is true if len(items)>0, ctx is not Done, and interuptFetch is false. hasItems sync.Cond } @@ -91,28 +91,15 @@ func (rl *relayLog) Fetch() ([][]*binlogdatapb.VEvent, error) { if err := rl.checkDone(); err != nil { return nil, err } - timer := time.NewTimer(idleTimeout) - defer timer.Stop() - var timedout sync2.AtomicBool - go func() { - select { - case <-timer.C: - rl.mu.Lock() - defer rl.mu.Unlock() - timedout.Set(true) - rl.hasItems.Broadcast() - default: - } - }() - for len(rl.items) == 0 { + cancelTimer := rl.startTimer() + defer cancelTimer() + for len(rl.items) == 0 && !rl.interruptFetch { rl.hasItems.Wait() if err := rl.checkDone(); err != nil { return nil, err } - if timedout.Get() { - return nil, nil - } } + rl.interruptFetch = false items := rl.items rl.items = nil rl.curSize = 0 @@ -129,6 +116,29 @@ func (rl *relayLog) checkDone() error { return nil } +func (rl *relayLog) startTimer() (cancel func()) { + timer := time.NewTimer(idleTimeout) + timerDone := make(chan struct{}) + go func() { + select { + case <-timer.C: + rl.InterruptFetch() + case <-timerDone: + } + }() + return func() { + timer.Stop() + close(timerDone) + } +} + +func (rl *relayLog) InterruptFetch() { + rl.mu.Lock() + defer rl.mu.Unlock() + rl.interruptFetch = true + rl.hasItems.Broadcast() +} + func eventsSize(events []*binlogdatapb.VEvent) int { size := 0 for _, event := range events { diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index 0168c063898..0528c920230 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -57,7 +57,9 @@ type vplayer struct { unsavedGTID *binlogdatapb.VEvent timeLastSaved time.Time stopPos mysql.Position - mustCommit bool + // inTransaction is true if we've started a transaction. + // It remains true until the next commit or rollback. + inTransaction bool // mu protects exportPositions. // exportPositions specifies a list of positions. @@ -202,32 +204,30 @@ func (vp *vplayer) applyEvents(relay *relayLog) error { if err != nil { return err } - // This covers two situations: - // 1. Fetch was idle for idleTimeout. - // 2. We've been receiving empty events for longer than idleTimeout. - // In both cases, now > timeLastSaved. If so, any unsaved GTID should be saved - // to prevent us from falling behind on tracking the source binlog position. - if time.Now().Sub(vp.timeLastSaved) >= idleTimeout && vp.unsavedGTID != nil { + // Don't do special things in the middle of a transaction. + if !vp.inTransaction && vp.mustExport() { if err := vp.updatePos(vp.unsavedGTID.Timestamp); err != nil { return err } } for i, events := range items { for j, event := range events { + mustSave := false switch event.Type { case binlogdatapb.VEventType_COMMIT: if vp.pos.Equal(vp.stopPos) { - // If stop pos is reached, we have to commit. + mustSave = true break } if vp.mustExport() { + mustSave = true break } if hasAnotherCommit(items, i, j+1) { continue } } - if err := vp.applyEvent(event); err != nil { + if err := vp.applyEvent(event, mustSave); err != nil { return err } } @@ -236,6 +236,15 @@ func (vp *vplayer) applyEvents(relay *relayLog) error { } func (vp *vplayer) mustExport() bool { + // This covers two situations: + // 1. Fetch was idle for idleTimeout. + // 2. We've been receiving empty events for longer than idleTimeout. + // In both cases, now > timeLastSaved. If so, any unsaved GTID should be saved + // to prevent us from falling behind on tracking the source binlog position. + if time.Now().Sub(vp.timeLastSaved) >= idleTimeout && vp.unsavedGTID != nil { + return true + } + vp.mu.Lock() defer vp.mu.Unlock() @@ -274,7 +283,7 @@ func hasAnotherCommit(items [][]*binlogdatapb.VEvent, i, j int) bool { return false } -func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { +func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent, mustSave bool) error { switch event.Type { case binlogdatapb.VEventType_GTID: pos, err := mysql.DecodePosition(event.Gtid) @@ -296,64 +305,53 @@ func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { case binlogdatapb.VEventType_BEGIN: // No-op: begin is called as needed. case binlogdatapb.VEventType_COMMIT: - posReached := !vp.stopPos.IsZero() && vp.pos.Equal(vp.stopPos) - // If stop pos is reached, then we must commit. - // So, if we haven't started a transaction (vp.mustCommit==false), - // we must start one. - if posReached && !vp.mustCommit { - if err := vp.dbClient.Begin(); err != nil { + if mustSave { + if err := vp.begin(); err != nil { return err } - vp.mustCommit = true } - if !vp.mustCommit { + if !vp.inTransaction { return nil } if err := vp.updatePos(event.Timestamp); err != nil { return err } + posReached := !vp.stopPos.IsZero() && vp.pos.Equal(vp.stopPos) if posReached { if err := vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("Stopped at position %v", vp.stopPos)); err != nil { return err } } - if err := vp.dbClient.Commit(); err != nil { + if err := vp.commit(); err != nil { return err } - vp.mustCommit = false if posReached { return io.EOF } case binlogdatapb.VEventType_FIELD: - if !vp.mustCommit { - if err := vp.dbClient.Begin(); err != nil { - return err - } - vp.mustCommit = true + if err := vp.begin(); err != nil { + return err } if err := vp.updatePlan(event.FieldEvent); err != nil { return err } case binlogdatapb.VEventType_ROW: - if !vp.mustCommit { - if err := vp.dbClient.Begin(); err != nil { - return err - } - vp.mustCommit = true + if err := vp.begin(); err != nil { + return err } if err := vp.applyRowEvent(event.RowEvent); err != nil { return err } case binlogdatapb.VEventType_DDL: - if vp.mustCommit { + if vp.inTransaction { return fmt.Errorf("unexpected state: DDL encountered in the middle of a transaction: %v", event.Ddl) } switch vp.source.OnDdl { case binlogdatapb.OnDDLAction_IGNORE: // no-op case binlogdatapb.OnDDLAction_STOP: - if err := vp.dbClient.Begin(); err != nil { + if err := vp.begin(); err != nil { return err } if err := vp.updatePos(event.Timestamp); err != nil { @@ -362,7 +360,7 @@ func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { if err := vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("Stopped at DDL %s", event.Ddl)); err != nil { return err } - if err := vp.dbClient.Commit(); err != nil { + if err := vp.commit(); err != nil { return err } return io.EOF @@ -385,6 +383,33 @@ func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent) error { return nil } +func (vp *vplayer) begin() error { + if vp.inTransaction { + return nil + } + if err := vp.dbClient.Begin(); err != nil { + return err + } + vp.inTransaction = true + return nil +} + +func (vp *vplayer) commit() error { + if !vp.inTransaction { + return nil + } + if err := vp.dbClient.Commit(); err != nil { + return err + } + vp.inTransaction = false + return nil +} + +func (vp *vplayer) rollback() { + vp.inTransaction = false + _ = vp.dbClient.Rollback() +} + func (vp *vplayer) setState(state, message string) error { return binlogplayer.SetVReplicationState(vp.dbClient, vp.id, state, message) } @@ -592,7 +617,7 @@ func (vp *vplayer) writeWhereValues(sql *sqlparser.TrackedBuffer, tplan *tablePl func (vp *vplayer) updatePos(ts int64) error { updatePos := binlogplayer.GenerateUpdatePos(vp.id, vp.pos, time.Now().Unix(), ts) if _, err := vp.dbClient.ExecuteFetch(updatePos, 0); err != nil { - _ = vp.dbClient.Rollback() + vp.rollback() return fmt.Errorf("error %v updating position", err) } vp.unsavedGTID = nil diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go index 2d9d796a431..04f11689974 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go @@ -188,6 +188,15 @@ func TestFilters(t *testing.T) { func TestDDL(t *testing.T) { defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + execStatements(t, []string{ + "create table dummy(id int, primary key(id))", + fmt.Sprintf("create table %s.dummy(id int, primary key(id))", vrepldb), + }) + defer execStatements(t, []string{ + "drop table dummy", + fmt.Sprintf("drop table %s.dummy", vrepldb), + }) + env.SchemaEngine.Reload(context.Background()) filter := &binlogdatapb.Filter{ Rules: []*binlogdatapb.Rule{{ @@ -196,6 +205,18 @@ func TestDDL(t *testing.T) { } cancel, _ := startVReplication(t, playerEngine, filter, binlogdatapb.OnDDLAction_IGNORE, "") + // Issue a dummy change to ensure vreplication is initialized. Otherwise there + // is a race between the DDLs and the schema loader of vstreamer. + // Root cause seems to be with MySQL where t1 shows up in information_schema before + // the actual table is created. + execStatements(t, []string{"insert into dummy values(1)"}) + expectDBClientQueries(t, []string{ + "begin", + "insert into dummy set id=1", + "/update _vt.vreplication set pos=", + "commit", + }) + execStatements(t, []string{"create table t1(id int, primary key(id))"}) execStatements(t, []string{"drop table t1"}) expectDBClientQueries(t, []string{}) @@ -265,8 +286,6 @@ func TestDDL(t *testing.T) { }) cancel() - // Don't test drop. - // MySQL rewrites them by uppercasing, which may be version specific. execStatements(t, []string{ "drop table t1", fmt.Sprintf("drop table %s.t1", vrepldb), From 28392444a563351d4320a7ef97d3c847b7dec170 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Sun, 20 Jan 2019 22:40:34 -0800 Subject: [PATCH 14/27] vplayer: go with shorter idle timeouts This allows us to avoid the whole mess about WaitForPos asking for specific positions to be exported. Signed-off-by: Sugu Sougoumarane --- .../tabletmanager/vreplication/controller.go | 14 +--- .../tabletmanager/vreplication/engine.go | 10 --- .../tabletmanager/vreplication/engine_test.go | 11 --- .../tabletmanager/vreplication/vplayer.go | 76 ++++--------------- 4 files changed, 17 insertions(+), 94 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vreplication/controller.go b/go/vt/vttablet/tabletmanager/vreplication/controller.go index 37186eb21f7..665d0f8a2a7 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/controller.go +++ b/go/vt/vttablet/tabletmanager/vreplication/controller.go @@ -22,7 +22,6 @@ import ( "strconv" "time" - "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/vt/vterrors" "github.com/golang/protobuf/proto" @@ -48,9 +47,6 @@ type controller struct { mysqld mysqlctl.MysqlDaemon blpStats *binlogplayer.Stats - // vplayer is set only if we launch vplayer. - vplayer *vplayer - id uint32 source binlogdatapb.BinlogSource stopPos string @@ -193,18 +189,12 @@ 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: - ct.vplayer = newVPlayer(ct.id, &ct.source, tablet, ct.blpStats, dbClient, ct.mysqld) - return ct.vplayer.Play(ctx) + vplayer := newVPlayer(ct.id, &ct.source, tablet, ct.blpStats, dbClient, ct.mysqld) + return vplayer.Play(ctx) } return fmt.Errorf("missing source") } -func (ct *controller) exportPosition(pos mysql.Position) { - if ct.vplayer != nil { - ct.vplayer.exportPosition(pos) - } -} - func (ct *controller) Stop() { ct.cancel() <-ct.done diff --git a/go/vt/vttablet/tabletmanager/vreplication/engine.go b/go/vt/vttablet/tabletmanager/vreplication/engine.go index 740c497626d..3577508abdc 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/engine.go +++ b/go/vt/vttablet/tabletmanager/vreplication/engine.go @@ -284,19 +284,9 @@ func (vre *Engine) WaitForPos(ctx context.Context, id int, pos string) error { if err := func() error { vre.mu.Lock() defer vre.mu.Unlock() - if !vre.isOpen { return errors.New("vreplication engine is closed") } - if ct, ok := vre.controllers[id]; ok { - mpos, err := mysql.DecodePosition(pos) - if err != nil { - return err - } - // vplayer doesn't export all the positions it receives unless - // we specifically request it for one. - ct.exportPosition(mpos) - } // Ensure that the engine won't be closed while this is running. vre.wg.Add(1) diff --git a/go/vt/vttablet/tabletmanager/vreplication/engine_test.go b/go/vt/vttablet/tabletmanager/vreplication/engine_test.go index 9bf21378e85..eb106370b56 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/engine_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/engine_test.go @@ -283,8 +283,6 @@ func TestWaitForPos(t *testing.T) { if err := vre.Open(context.Background()); err != nil { t.Fatal(err) } - // hack a controller into vre. - vre.controllers[1] = &controller{} dbClient.ExpectRequest("select pos, state, message from _vt.vreplication where id=1", &sqltypes.Result{Rows: [][]sqltypes.Value{{ sqltypes.NewVarBinary("MariaDB/0-1-1083"), @@ -321,8 +319,6 @@ func TestWaitForPosError(t *testing.T) { if err := vre.Open(context.Background()); err != nil { t.Fatal(err) } - // hack a controller into vre. - vre.controllers[1] = &controller{} err = vre.WaitForPos(context.Background(), 1, "BadFlavor/0-1-1084") want = `parse error: unknown GTIDSet flavor "BadFlavor"` @@ -367,13 +363,6 @@ func TestWaitForPosCancel(t *testing.T) { }}}, nil) ctx, cancel := context.WithCancel(context.Background()) cancel() - // hack a controller that can handle Close - done := make(chan struct{}) - close(done) - vre.controllers[1] = &controller{ - cancel: func() {}, - done: done, - } err := vre.WaitForPos(ctx, 1, "MariaDB/0-1-1084") if err == nil || err != context.Canceled { t.Errorf("WaitForPos: %v, want %v", err, context.Canceled) diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index 0528c920230..6d06ab802fa 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -21,7 +21,6 @@ import ( "errors" "fmt" "io" - "sync" "time" "golang.org/x/net/context" @@ -41,8 +40,8 @@ import ( ) var ( - idleTimeout = 1 * time.Hour - dblockRetryDelay = 1 * time.Second + idleTimeout = 1 * time.Second + dbLockRetryDelay = 1 * time.Second ) type vplayer struct { @@ -61,18 +60,6 @@ type vplayer struct { // It remains true until the next commit or rollback. inTransaction bool - // mu protects exportPositions. - // exportPositions specifies a list of positions. - // Anytime the vplayer reaches or exceeds one of these - // positions, it will update the position even if it's - // an empty transaction. This is used for cases where - // a target is stopped, and a request is initiated to - // wait for vreplication to reach the source's position. - // Since vreplication only updates state on non-empty - // transactions, this list is used to force such updates. - mu sync.Mutex - exportPositions []mysql.Position - pplan *playerPlan tplans map[string]*tablePlan } @@ -204,10 +191,17 @@ func (vp *vplayer) applyEvents(relay *relayLog) error { if err != nil { return err } - // Don't do special things in the middle of a transaction. - if !vp.inTransaction && vp.mustExport() { - if err := vp.updatePos(vp.unsavedGTID.Timestamp); err != nil { - return err + // This covers two situations: + // 1. Fetch was idle for idleTimeout. + // 2. We've been receiving empty events for longer than idleTimeout. + // In both cases, now > timeLastSaved. If so, any unsaved GTID should be saved. + if time.Now().Sub(vp.timeLastSaved) >= idleTimeout && vp.unsavedGTID != nil { + // Although unlikely, we should not save if a transaction is still open. + // This can happen if a large transaction is split as multiple events. + if !vp.inTransaction { + if err := vp.updatePos(vp.unsavedGTID.Timestamp); err != nil { + return err + } } } for i, events := range items { @@ -219,10 +213,6 @@ func (vp *vplayer) applyEvents(relay *relayLog) error { mustSave = true break } - if vp.mustExport() { - mustSave = true - break - } if hasAnotherCommit(items, i, j+1) { continue } @@ -235,36 +225,6 @@ func (vp *vplayer) applyEvents(relay *relayLog) error { } } -func (vp *vplayer) mustExport() bool { - // This covers two situations: - // 1. Fetch was idle for idleTimeout. - // 2. We've been receiving empty events for longer than idleTimeout. - // In both cases, now > timeLastSaved. If so, any unsaved GTID should be saved - // to prevent us from falling behind on tracking the source binlog position. - if time.Now().Sub(vp.timeLastSaved) >= idleTimeout && vp.unsavedGTID != nil { - return true - } - - vp.mu.Lock() - defer vp.mu.Unlock() - - mustExport := false - for { - found := false - for i, export := range vp.exportPositions { - if vp.pos.AtLeast(export) { - mustExport = true - found = true - vp.exportPositions = append(vp.exportPositions[:i], vp.exportPositions[i+1:]...) - break - } - } - if !found { - return mustExport - } - } -} - func hasAnotherCommit(items [][]*binlogdatapb.VEvent, i, j int) bool { for i < len(items) { for j < len(items[i]) { @@ -631,8 +591,8 @@ func (vp *vplayer) exec(sql string) error { for err != nil { // 1213: deadlock, 1205: lock wait timeout if sqlErr, ok := err.(*mysql.SQLError); ok && sqlErr.Number() == 1213 || sqlErr.Number() == 1205 { - log.Infof("retryable error: %v, waiting for %v and retrying", sqlErr, dblockRetryDelay) - time.Sleep(dblockRetryDelay) + log.Infof("retryable error: %v, waiting for %v and retrying", sqlErr, dbLockRetryDelay) + time.Sleep(dbLockRetryDelay) err = vp.dbClient.Retry() continue } @@ -641,12 +601,6 @@ func (vp *vplayer) exec(sql string) error { return nil } -func (vp *vplayer) exportPosition(pos mysql.Position) { - vp.mu.Lock() - defer vp.mu.Unlock() - vp.exportPositions = append(vp.exportPositions, pos) -} - func encodeValue(sql *sqlparser.TrackedBuffer, value sqltypes.Value) { if value.Type() == querypb.Type_TIMESTAMP && !bytes.HasPrefix(value.ToBytes(), mysql.ZeroTimestamp) { // Values in the binary log are UTC. Let's convert them From 8199e401e5effa0027158e53936b819a7268bd08 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Mon, 21 Jan 2019 01:54:21 -0800 Subject: [PATCH 15/27] vplayer: more tests Signed-off-by: Sugu Sougoumarane --- .../tabletmanager/vreplication/relaylog.go | 26 +++-- .../vreplication/vplayer_test.go | 98 ++++++++++++++++++- 2 files changed, 105 insertions(+), 19 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vreplication/relaylog.go b/go/vt/vttablet/tabletmanager/vreplication/relaylog.go index 64504b5457a..f0714390d2e 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/relaylog.go +++ b/go/vt/vttablet/tabletmanager/vreplication/relaylog.go @@ -34,11 +34,11 @@ type relayLog struct { // mu controls all variables below and is shared by canAccept and hasItems. // Broadcasting must be done while holding mu. This is mainly necessary because both // conditions depend on ctx.Done(), which can change state asynchronously. - mu sync.Mutex - curSize int - items [][]*binlogdatapb.VEvent - interruptFetch bool - err error + mu sync.Mutex + curSize int + items [][]*binlogdatapb.VEvent + timedout bool + err error // canAccept is true if: curSize<=maxSize, len(items)0, ctx is not Done, and interuptFetch is false. @@ -93,13 +93,13 @@ func (rl *relayLog) Fetch() ([][]*binlogdatapb.VEvent, error) { } cancelTimer := rl.startTimer() defer cancelTimer() - for len(rl.items) == 0 && !rl.interruptFetch { + for len(rl.items) == 0 && !rl.timedout { rl.hasItems.Wait() if err := rl.checkDone(); err != nil { return nil, err } } - rl.interruptFetch = false + rl.timedout = false items := rl.items rl.items = nil rl.curSize = 0 @@ -122,7 +122,10 @@ func (rl *relayLog) startTimer() (cancel func()) { go func() { select { case <-timer.C: - rl.InterruptFetch() + rl.mu.Lock() + defer rl.mu.Unlock() + rl.timedout = true + rl.hasItems.Broadcast() case <-timerDone: } }() @@ -132,13 +135,6 @@ func (rl *relayLog) startTimer() (cancel func()) { } } -func (rl *relayLog) InterruptFetch() { - rl.mu.Lock() - defer rl.mu.Unlock() - rl.interruptFetch = true - rl.hasItems.Broadcast() -} - func eventsSize(events []*binlogdatapb.VEvent) int { size := 0 for _, event := range events { diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go index 04f11689974..38006ff3d56 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go @@ -17,9 +17,11 @@ limitations under the License. package vreplication import ( + "flag" "fmt" "strings" "testing" + "time" "golang.org/x/net/context" "vitess.io/vitess/go/mysql" @@ -29,7 +31,7 @@ import ( topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) -func TestFilters(t *testing.T) { +func TestPlayerFilters(t *testing.T) { defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) execStatements(t, []string{ @@ -186,7 +188,7 @@ func TestFilters(t *testing.T) { } } -func TestDDL(t *testing.T) { +func TestPlayerDDL(t *testing.T) { defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) execStatements(t, []string{ "create table dummy(id int, primary key(id))", @@ -294,7 +296,7 @@ func TestDDL(t *testing.T) { }) } -func TestStopPos(t *testing.T) { +func TestPlayerStopPos(t *testing.T) { defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) execStatements(t, []string{ @@ -388,10 +390,98 @@ func TestStopPos(t *testing.T) { }) } +func TestPlayerIdleUpdate(t *testing.T) { + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + + savedIdleTimeout := idleTimeout + defer func() { idleTimeout = savedIdleTimeout }() + 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), + }) + defer execStatements(t, []string{ + "drop table t1", + fmt.Sprintf("drop table %s.t1", vrepldb), + }) + env.SchemaEngine.Reload(context.Background()) + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "/.*", + }}, + } + cancel, _ := startVReplication(t, playerEngine, filter, binlogdatapb.OnDDLAction_IGNORE, "") + defer cancel() + + execStatements(t, []string{ + "insert into t1 values(1, 'aaa')", + }) + start := time.Now() + expectDBClientQueries(t, []string{ + "begin", + "insert into t1 set id=1, val='aaa'", + "/update _vt.vreplication set pos=", + "commit", + }) + // The above write will generate a new binlog event, and + // that event will loopback into player as an empty event. + // But it must not get saved until idleTimeout has passed. + // The exact positions are hard to verify because of this + // loopback mechanism. + expectDBClientQueries(t, []string{ + "/update _vt.vreplication set pos=", + }) + if duration := time.Now().Sub(start); duration < idleTimeout { + t.Errorf("duration: %v, must be at least %v", duration, idleTimeout) + } +} + +func TestPlayerSplitTransaction(t *testing.T) { + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + flag.Set("vstream_packet_size", "10") + defer flag.Set("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), + }) + defer execStatements(t, []string{ + "drop table t1", + fmt.Sprintf("drop table %s.t1", vrepldb), + }) + env.SchemaEngine.Reload(context.Background()) + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "/.*", + }}, + } + cancel, _ := startVReplication(t, playerEngine, filter, binlogdatapb.OnDDLAction_IGNORE, "") + defer cancel() + + execStatements(t, []string{ + "begin", + "insert into t1 values(1, '123456')", + "insert into t1 values(2, '789012')", + "commit", + }) + // Because the packet size is 10, this is received as two events, + // but still combined as one transaction. + expectDBClientQueries(t, []string{ + "begin", + "insert into t1 set id=1, val='123456'", + "insert into t1 set id=2, val='789012'", + "/update _vt.vreplication set pos=", + "commit", + }) +} + func execStatements(t *testing.T, queries []string) { t.Helper() if err := env.Mysqld.ExecuteSuperQueryList(context.Background(), queries); err != nil { - t.Fatal(err) + t.Error(err) } } From c98ab1af2d5e25b750bcdeb59d27d2131531b908 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Tue, 22 Jan 2019 01:26:17 -0800 Subject: [PATCH 16/27] vplayer: more tests and timestamp fixes Signed-off-by: Sugu Sougoumarane --- .../vreplication/framework_test.go | 19 +- ...retryableClient.go => retryable_client.go} | 31 +- .../tabletmanager/vreplication/vplayer.go | 72 +-- .../vreplication/vplayer_test.go | 450 +++++++++++++++++- 4 files changed, 514 insertions(+), 58 deletions(-) rename go/vt/vttablet/tabletmanager/vreplication/{retryableClient.go => retryable_client.go} (77%) diff --git a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go index 5149b3bf00d..ea1cd2dc63d 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go @@ -95,6 +95,11 @@ func TestMain(m *testing.M) { return 1 } + if err := env.Mysqld.ExecuteSuperQuery(context.Background(), "set @@global.innodb_lock_wait_timeout=1"); err != nil { + fmt.Fprintf(os.Stderr, "%v", err) + return 1 + } + playerEngine = NewEngine(env.TopoServ, env.Cells[0], env.Mysqld, realDBClientFactory) if err := playerEngine.Open(context.Background()); err != nil { fmt.Fprintf(os.Stderr, "%v", err) @@ -278,7 +283,8 @@ func realDBClientFactory() binlogplayer.DBClient { } type realDBClient struct { - conn *mysql.Conn + conn *mysql.Conn + nolog bool } func (dbc *realDBClient) DBName() string { @@ -297,20 +303,17 @@ func (dbc *realDBClient) Connect() error { } func (dbc *realDBClient) Begin() error { - _, err := dbc.conn.ExecuteFetch("begin", 10000, true) - globalDBQueries <- "begin" + _, err := dbc.ExecuteFetch("begin", 10000) return err } func (dbc *realDBClient) Commit() error { - _, err := dbc.conn.ExecuteFetch("commit", 10000, true) - globalDBQueries <- "commit" + _, err := dbc.ExecuteFetch("commit", 10000) return err } func (dbc *realDBClient) Rollback() error { - _, err := dbc.conn.ExecuteFetch("rollback", 10000, true) - globalDBQueries <- "rollback" + _, err := dbc.ExecuteFetch("rollback", 10000) return err } @@ -324,7 +327,7 @@ func (dbc *realDBClient) ExecuteFetch(query string, maxrows int) (*sqltypes.Resu return nil, nil } qr, err := dbc.conn.ExecuteFetch(query, 10000, true) - if !strings.HasPrefix(query, "select") { + if !strings.HasPrefix(query, "select") && !dbc.nolog { globalDBQueries <- query } return qr, err diff --git a/go/vt/vttablet/tabletmanager/vreplication/retryableClient.go b/go/vt/vttablet/tabletmanager/vreplication/retryable_client.go similarity index 77% rename from go/vt/vttablet/tabletmanager/vreplication/retryableClient.go rename to go/vt/vttablet/tabletmanager/vreplication/retryable_client.go index 30deacf155b..f08cd99cf37 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/retryableClient.go +++ b/go/vt/vttablet/tabletmanager/vreplication/retryable_client.go @@ -21,16 +21,29 @@ import ( "vitess.io/vitess/go/vt/binlog/binlogplayer" ) +// retryableClient is a wrapper on binlogplayer.DBClient. +// It allows us to retry a failed transactions on lock errors. type retryableClient struct { binlogplayer.DBClient inTransaction bool queries []string } +func (rt *retryableClient) Connect() error { + if err := rt.Connect(); err != nil { + return err + } + if _, err := rt.DBClient.ExecuteFetch("set @@session.time_zone = '+00:00'", 10000); err != nil { + return err + } + return nil +} + func (rt *retryableClient) Begin() error { if err := rt.DBClient.Begin(); err != nil { return err } + rt.queries = append(rt.queries, "begin") rt.inTransaction = true return nil } @@ -49,7 +62,7 @@ func (rt *retryableClient) Rollback() error { return err } rt.inTransaction = false - rt.queries = nil + // Don't reset queries to allow for vplayer to retry. return nil } @@ -63,17 +76,13 @@ func (rt *retryableClient) ExecuteFetch(query string, maxrows int) (*sqltypes.Re } func (rt *retryableClient) Retry() error { - if !rt.inTransaction { - _, err := rt.DBClient.ExecuteFetch(rt.queries[0], 10000) - return err - } - if err := rt.DBClient.Rollback(); err != nil { - return err - } - if err := rt.DBClient.Begin(); err != nil { - return err - } for _, q := range rt.queries { + if q == "begin" { + if err := rt.Begin(); err != nil { + return err + } + continue + } if _, err := rt.DBClient.ExecuteFetch(q, 10000); err != nil { return err } diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index 6d06ab802fa..66b2937d7a4 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -17,7 +17,6 @@ limitations under the License. package vreplication import ( - "bytes" "errors" "fmt" "io" @@ -42,6 +41,8 @@ import ( var ( idleTimeout = 1 * time.Second dbLockRetryDelay = 1 * time.Second + relayLogMaxSize = 10000 + relayLogMaxItems = 1000 ) type vplayer struct { @@ -50,17 +51,25 @@ type vplayer struct { sourceTablet *topodatapb.Tablet stats *binlogplayer.Stats dbClient *retryableClient - mysqld mysqlctl.MysqlDaemon - - pos mysql.Position - unsavedGTID *binlogdatapb.VEvent + // mysqld is used to fetch the local schema. + mysqld mysqlctl.MysqlDaemon + + pos mysql.Position + // unsavedGTID when we receive a GTID event and reset + // if it gets saved. If Fetch returns on idleTimeout, + // we save the last unsavedGTID. + unsavedGTID *binlogdatapb.VEvent + // timeLastSaved is set every time a GTID is saved. timeLastSaved time.Time stopPos mysql.Position // inTransaction is true if we've started a transaction. // It remains true until the next commit or rollback. inTransaction bool - pplan *playerPlan + // pplan is built based on the source Filter at the beginning. + pplan *playerPlan + // tplans[table] is built for each table based on pplan and schema info + // about the table. tplans map[string]*tablePlan } @@ -130,7 +139,7 @@ func (vp *vplayer) play(ctx context.Context) error { ctx, cancel := context.WithCancel(ctx) defer cancel() - relay := newRelayLog(ctx, 10, 10) + relay := newRelayLog(ctx, relayLogMaxItems, relayLogMaxSize) target := &querypb.Target{ Keyspace: vp.sourceTablet.Keyspace, @@ -147,7 +156,7 @@ func (vp *vplayer) play(ctx context.Context) error { applyErr := make(chan error, 1) go func() { - applyErr <- vp.applyEvents(relay) + applyErr <- vp.applyEvents(ctx, relay) }() select { @@ -185,7 +194,7 @@ func (vp *vplayer) play(ctx context.Context) error { } } -func (vp *vplayer) applyEvents(relay *relayLog) error { +func (vp *vplayer) applyEvents(ctx context.Context, relay *relayLog) error { for { items, err := relay.Fetch() if err != nil { @@ -217,7 +226,7 @@ func (vp *vplayer) applyEvents(relay *relayLog) error { continue } } - if err := vp.applyEvent(event, mustSave); err != nil { + if err := vp.applyEvent(ctx, event, mustSave); err != nil { return err } } @@ -243,7 +252,7 @@ func hasAnotherCommit(items [][]*binlogdatapb.VEvent, i, j int) bool { return false } -func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent, mustSave bool) error { +func (vp *vplayer) applyEvent(ctx context.Context, event *binlogdatapb.VEvent, mustSave bool) error { switch event.Type { case binlogdatapb.VEventType_GTID: pos, err := mysql.DecodePosition(event.Gtid) @@ -300,7 +309,7 @@ func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent, mustSave bool) error { if err := vp.begin(); err != nil { return err } - if err := vp.applyRowEvent(event.RowEvent); err != nil { + if err := vp.applyRowEvent(ctx, event.RowEvent); err != nil { return err } case binlogdatapb.VEventType_DDL: @@ -325,14 +334,14 @@ func (vp *vplayer) applyEvent(event *binlogdatapb.VEvent, mustSave bool) error { } return io.EOF case binlogdatapb.OnDDLAction_EXEC: - if err := vp.exec(event.Ddl); err != nil { + if err := vp.exec(ctx, event.Ddl); err != nil { return err } if err := vp.updatePos(event.Timestamp); err != nil { return err } case binlogdatapb.OnDDLAction_EXEC_IGNORE: - if err := vp.exec(event.Ddl); err != nil { + if err := vp.exec(ctx, event.Ddl); err != nil { log.Infof("Ignoring error: %v for DDL: %s", err, event.Ddl) } if err := vp.updatePos(event.Timestamp); err != nil { @@ -425,20 +434,20 @@ func (vp *vplayer) updatePlan(fieldEvent *binlogdatapb.FieldEvent) error { return nil } -func (vp *vplayer) applyRowEvent(rowEvent *binlogdatapb.RowEvent) error { +func (vp *vplayer) applyRowEvent(ctx context.Context, rowEvent *binlogdatapb.RowEvent) error { tplan := vp.tplans[rowEvent.TableName] if tplan == nil { return fmt.Errorf("unexpected event on table %s", rowEvent.TableName) } for _, change := range rowEvent.RowChanges { - if err := vp.applyRowChange(tplan, change); err != nil { + if err := vp.applyRowChange(ctx, tplan, change); err != nil { return err } } return nil } -func (vp *vplayer) applyRowChange(tplan *tablePlan, rowChange *binlogdatapb.RowChange) error { +func (vp *vplayer) applyRowChange(ctx context.Context, tplan *tablePlan, rowChange *binlogdatapb.RowChange) error { // MakeRowTrusted is needed here because because Proto3ToResult is not convenient. var before, after []sqltypes.Value if rowChange.Before != nil { @@ -461,7 +470,7 @@ func (vp *vplayer) applyRowChange(tplan *tablePlan, rowChange *binlogdatapb.RowC if query == "" { return nil } - return vp.exec(query) + return vp.exec(ctx, query) } func (vp *vplayer) generateInsert(tplan *tablePlan, after []sqltypes.Value) string { @@ -585,14 +594,23 @@ func (vp *vplayer) updatePos(ts int64) error { return nil } -func (vp *vplayer) exec(sql string) error { +func (vp *vplayer) exec(ctx context.Context, sql string) error { vp.stats.Timings.Record("query", time.Now()) _, err := vp.dbClient.ExecuteFetch(sql, 0) for err != nil { // 1213: deadlock, 1205: lock wait timeout if sqlErr, ok := err.(*mysql.SQLError); ok && sqlErr.Number() == 1213 || sqlErr.Number() == 1205 { log.Infof("retryable error: %v, waiting for %v and retrying", sqlErr, dbLockRetryDelay) + if err := vp.dbClient.Rollback(); err != nil { + return err + } time.Sleep(dbLockRetryDelay) + // Check context here. Otherwise this can become an infinite loop. + select { + case <-ctx.Done(): + return io.EOF + default: + } err = vp.dbClient.Retry() continue } @@ -602,14 +620,10 @@ func (vp *vplayer) exec(sql string) error { } func encodeValue(sql *sqlparser.TrackedBuffer, value sqltypes.Value) { - if value.Type() == querypb.Type_TIMESTAMP && !bytes.HasPrefix(value.ToBytes(), mysql.ZeroTimestamp) { - // Values in the binary log are UTC. Let's convert them - // to whatever timezone the connection is using, - // so MySQL properly converts them back to UTC. - sql.WriteString("convert_tz(") - value.EncodeSQL(sql) - sql.WriteString(", '+00:00', @@session.time_zone)") - } else { - value.EncodeSQL(sql) - } + // This is currently a separate function because special handling + // may be needed for certain types. + // Previously, this function used to convert timestamp to the session + // time zone, but we now set the session timezone to UTC. So, the timestamp + // value we receive as UTC can be sent as is. + value.EncodeSQL(sql) } diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go index 38006ff3d56..a1b06f4b609 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go @@ -72,7 +72,7 @@ func TestPlayerFilters(t *testing.T) { Match: "/yes", }}, } - cancel, _ := startVReplication(t, playerEngine, filter, binlogdatapb.OnDDLAction_IGNORE, "") + cancel, _ := startVReplication(t, filter, binlogdatapb.OnDDLAction_IGNORE, "") defer cancel() testcases := []struct { @@ -206,7 +206,7 @@ func TestPlayerDDL(t *testing.T) { }}, } - cancel, _ := startVReplication(t, playerEngine, filter, binlogdatapb.OnDDLAction_IGNORE, "") + cancel, _ := startVReplication(t, filter, binlogdatapb.OnDDLAction_IGNORE, "") // Issue a dummy change to ensure vreplication is initialized. Otherwise there // is a race between the DDLs and the schema loader of vstreamer. // Root cause seems to be with MySQL where t1 shows up in information_schema before @@ -224,7 +224,7 @@ func TestPlayerDDL(t *testing.T) { expectDBClientQueries(t, []string{}) cancel() - cancel, id := startVReplication(t, playerEngine, filter, binlogdatapb.OnDDLAction_STOP, "") + cancel, id := startVReplication(t, filter, binlogdatapb.OnDDLAction_STOP, "") execStatements(t, []string{"create table t1(id int, primary key(id))"}) pos1 := masterPosition(t) execStatements(t, []string{"drop table t1"}) @@ -252,7 +252,7 @@ func TestPlayerDDL(t *testing.T) { cancel() execStatements(t, []string{fmt.Sprintf("create table %s.t2(id int, primary key(id))", vrepldb)}) - cancel, _ = startVReplication(t, playerEngine, filter, binlogdatapb.OnDDLAction_EXEC, "") + cancel, _ = startVReplication(t, filter, binlogdatapb.OnDDLAction_EXEC, "") execStatements(t, []string{"create table t1(id int, primary key(id))"}) expectDBClientQueries(t, []string{ "create table t1(id int, primary key(id))", @@ -275,7 +275,7 @@ func TestPlayerDDL(t *testing.T) { }) execStatements(t, []string{fmt.Sprintf("create table %s.t2(id int, primary key(id))", vrepldb)}) - cancel, _ = startVReplication(t, playerEngine, filter, binlogdatapb.OnDDLAction_EXEC_IGNORE, "") + cancel, _ = startVReplication(t, filter, binlogdatapb.OnDDLAction_EXEC_IGNORE, "") execStatements(t, []string{"create table t1(id int, primary key(id))"}) expectDBClientQueries(t, []string{ "create table t1(id int, primary key(id))", @@ -412,7 +412,7 @@ func TestPlayerIdleUpdate(t *testing.T) { Match: "/.*", }}, } - cancel, _ := startVReplication(t, playerEngine, filter, binlogdatapb.OnDDLAction_IGNORE, "") + cancel, _ := startVReplication(t, filter, binlogdatapb.OnDDLAction_IGNORE, "") defer cancel() execStatements(t, []string{ @@ -458,7 +458,7 @@ func TestPlayerSplitTransaction(t *testing.T) { Match: "/.*", }}, } - cancel, _ := startVReplication(t, playerEngine, filter, binlogdatapb.OnDDLAction_IGNORE, "") + cancel, _ := startVReplication(t, filter, binlogdatapb.OnDDLAction_IGNORE, "") defer cancel() execStatements(t, []string{ @@ -478,6 +478,436 @@ func TestPlayerSplitTransaction(t *testing.T) { }) } +func TestPlayerLockErrors(t *testing.T) { + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + + 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), + }) + defer execStatements(t, []string{ + "drop table t1", + fmt.Sprintf("drop table %s.t1", vrepldb), + }) + env.SchemaEngine.Reload(context.Background()) + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "/.*", + }}, + } + cancel, _ := startVReplication(t, filter, binlogdatapb.OnDDLAction_IGNORE, "") + defer cancel() + + execStatements(t, []string{ + "begin", + "insert into t1 values(1, 'aaa')", + "insert into t1 values(2, 'bbb')", + "commit", + }) + expectDBClientQueries(t, []string{ + "begin", + "insert into t1 set id=1, val='aaa'", + "insert into t1 set id=2, val='bbb'", + "/update _vt.vreplication set pos=", + "commit", + }) + + vconn := &realDBClient{nolog: true} + if err := vconn.Connect(); err != nil { + t.Error(err) + } + defer vconn.Close() + + // Start a transaction and lock the second row. + if _, err := vconn.ExecuteFetch("begin", 1); err != nil { + t.Error(err) + } + if _, err := vconn.ExecuteFetch("update t1 set val='bbb' where id=2", 1); err != nil { + t.Error(err) + } + + execStatements(t, []string{ + "begin", + "update t1 set val='ccc' where id=1", + "update t1 set val='ccc' where id=2", + "commit", + }) + // The innodb lock wait timeout is set to 1s. + expectDBClientQueries(t, []string{ + "begin", + "update t1 set id=1, val='ccc' where id=1", + "update t1 set id=2, val='ccc' where id=2", + "rollback", + }) + + // Release the lock, and watch the retry go through. + _, _ = vconn.ExecuteFetch("rollback", 1) + expectDBClientQueries(t, []string{ + "begin", + "update t1 set id=1, val='ccc' where id=1", + "update t1 set id=2, val='ccc' where id=2", + "/update _vt.vreplication set pos=", + "commit", + }) +} + +func TestPlayerCancelOnLock(t *testing.T) { + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + + 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), + }) + defer execStatements(t, []string{ + "drop table t1", + fmt.Sprintf("drop table %s.t1", vrepldb), + }) + env.SchemaEngine.Reload(context.Background()) + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "/.*", + }}, + } + cancel, _ := startVReplication(t, filter, binlogdatapb.OnDDLAction_IGNORE, "") + defer cancel() + + execStatements(t, []string{ + "begin", + "insert into t1 values(1, 'aaa')", + "commit", + }) + expectDBClientQueries(t, []string{ + "begin", + "insert into t1 set id=1, val='aaa'", + "/update _vt.vreplication set pos=", + "commit", + }) + + vconn := &realDBClient{nolog: true} + if err := vconn.Connect(); err != nil { + t.Error(err) + } + defer vconn.Close() + + // Start a transaction and lock the row. + if _, err := vconn.ExecuteFetch("begin", 1); err != nil { + t.Error(err) + } + if _, err := vconn.ExecuteFetch("update t1 set val='bbb' where id=1", 1); err != nil { + t.Error(err) + } + + execStatements(t, []string{ + "begin", + "update t1 set val='ccc' where id=1", + "commit", + }) + // The innodb lock wait timeout is set to 1s. + expectDBClientQueries(t, []string{ + "begin", + "update t1 set id=1, val='ccc' where id=1", + "rollback", + }) + + // VReplication should not get stuck if you cancel now. + done := make(chan bool) + go func() { + cancel() + close(done) + }() + select { + case <-done: + case <-time.After(5 * time.Second): + t.Error("cancel is hung") + } +} + +func TestPlayerBatching(t *testing.T) { + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + + 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), + }) + defer execStatements(t, []string{ + "drop table t1", + fmt.Sprintf("drop table %s.t1", vrepldb), + }) + env.SchemaEngine.Reload(context.Background()) + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "/.*", + }}, + } + cancel, _ := startVReplication(t, filter, binlogdatapb.OnDDLAction_IGNORE, "") + defer cancel() + + execStatements(t, []string{ + "insert into t1 values(1, 'aaa')", + }) + expectDBClientQueries(t, []string{ + "begin", + "insert into t1 set id=1, val='aaa'", + "/update _vt.vreplication set pos=", + "commit", + }) + + vconn := &realDBClient{nolog: true} + if err := vconn.Connect(); err != nil { + t.Error(err) + } + defer vconn.Close() + + // Start a transaction and lock the row. + if _, err := vconn.ExecuteFetch("begin", 1); err != nil { + t.Error(err) + } + if _, err := vconn.ExecuteFetch("update t1 set val='bbb' where id=1", 1); err != nil { + t.Error(err) + } + + // create one transaction + execStatements(t, []string{ + "update t1 set val='ccc' where id=1", + }) + // Wait for the begin. The update will be blocked. + expectDBClientQueries(t, []string{ + "begin", + }) + + // Create two more transactions. They will go and wait in the relayLog. + execStatements(t, []string{ + "insert into t1 values(2, 'aaa')", + "insert into t1 values(3, 'aaa')", + }) + + // Release the lock. + _, _ = vconn.ExecuteFetch("rollback", 1) + // First transaction will complete. The other two + // transactions must be batched into one + expectDBClientQueries(t, []string{ + "update t1 set id=1, val='ccc' where id=1", + "/update _vt.vreplication set pos=", + "commit", + "begin", + "insert into t1 set id=2, val='aaa'", + "insert into t1 set id=3, val='aaa'", + "/update _vt.vreplication set pos=", + "commit", + }) +} + +func TestPlayerRelayLogMaxSize(t *testing.T) { + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + + for i := 0; i < 2; i++ { + // First iteration checks max size, second checks max items + func() { + switch i { + case 0: + savedSize := relayLogMaxSize + defer func() { relayLogMaxSize = savedSize }() + relayLogMaxSize = 10 + case 1: + savedLen := relayLogMaxItems + defer func() { relayLogMaxItems = savedLen }() + 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), + }) + defer execStatements(t, []string{ + "drop table t1", + fmt.Sprintf("drop table %s.t1", vrepldb), + }) + env.SchemaEngine.Reload(context.Background()) + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "/.*", + }}, + } + cancel, _ := startVReplication(t, filter, binlogdatapb.OnDDLAction_IGNORE, "") + defer cancel() + + execStatements(t, []string{ + "insert into t1 values(1, '123456')", + }) + expectDBClientQueries(t, []string{ + "begin", + "insert into t1 set id=1, val='123456'", + "/update _vt.vreplication set pos=", + "commit", + }) + + vconn := &realDBClient{nolog: true} + if err := vconn.Connect(); err != nil { + t.Error(err) + } + defer vconn.Close() + + // Start a transaction and lock the row. + if _, err := vconn.ExecuteFetch("begin", 1); err != nil { + t.Error(err) + } + if _, err := vconn.ExecuteFetch("update t1 set val='bbb' where id=1", 1); err != nil { + t.Error(err) + } + + // create one transaction + execStatements(t, []string{ + "update t1 set val='ccc' where id=1", + }) + // Wait for the begin. The update will be blocked. + expectDBClientQueries(t, []string{ + "begin", + }) + + // Create two more transactions. They will go and wait in the relayLog. + execStatements(t, []string{ + "insert into t1 values(2, '789012')", + "insert into t1 values(3, '345678')", + "insert into t1 values(4, '901234')", + }) + + // Release the lock. + _, _ = vconn.ExecuteFetch("rollback", 1) + // First transaction will complete. The other two + // transactions must be batched into one. The last transaction + // will wait to be sent to the relay until the player fetches + // them. + expectDBClientQueries(t, []string{ + "update t1 set id=1, val='ccc' where id=1", + "/update _vt.vreplication set pos=", + "commit", + "begin", + "insert into t1 set id=2, val='789012'", + "insert into t1 set id=3, val='345678'", + "/update _vt.vreplication set pos=", + "commit", + "begin", + "insert into t1 set id=4, val='901234'", + "/update _vt.vreplication set pos=", + "commit", + }) + }() + } +} + +func TestRestartOnVStreamEnd(t *testing.T) { + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + + savedDelay := *retryDelay + defer func() { *retryDelay = savedDelay }() + *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), + }) + defer execStatements(t, []string{ + "drop table t1", + fmt.Sprintf("drop table %s.t1", vrepldb), + }) + env.SchemaEngine.Reload(context.Background()) + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "/.*", + }}, + } + cancel, _ := startVReplication(t, filter, binlogdatapb.OnDDLAction_IGNORE, "") + defer cancel() + + execStatements(t, []string{ + "insert into t1 values(1, 'aaa')", + }) + expectDBClientQueries(t, []string{ + "begin", + "insert into t1 set id=1, val='aaa'", + "/update _vt.vreplication set pos=", + "commit", + }) + + streamerEngine.Close() + expectDBClientQueries(t, []string{ + "/update.*'Error'.*vstream ended", + }) + if err := streamerEngine.Open(env.KeyspaceName, env.ShardName); err != nil { + t.Fatal(err) + } + + execStatements(t, []string{ + "insert into t1 values(2, 'aaa')", + }) + expectDBClientQueries(t, []string{ + "/update.*'Running'", + "begin", + "insert into t1 set id=2, val='aaa'", + "/update _vt.vreplication set pos=", + "commit", + }) +} + +func TestTimestamp(t *testing.T) { + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + + execStatements(t, []string{ + "create table t1(id int, ts timestamp, dt datetime)", + fmt.Sprintf("create table %s.t1(id int, ts timestamp, dt datetime)", vrepldb), + }) + defer execStatements(t, []string{ + "drop table t1", + fmt.Sprintf("drop table %s.t1", vrepldb), + }) + env.SchemaEngine.Reload(context.Background()) + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "/.*", + }}, + } + cancel, _ := startVReplication(t, filter, binlogdatapb.OnDDLAction_IGNORE, "") + defer cancel() + + qr, err := env.Mysqld.FetchSuperQuery(context.Background(), "select now()") + if err != nil { + t.Fatal(err) + } + want := qr.Rows[0][0].ToString() + t.Logf("want: %s", want) + + execStatements(t, []string{ + fmt.Sprintf("insert into t1 values(1, '%s', '%s')", want, want), + }) + expectDBClientQueries(t, []string{ + "begin", + // The insert value for ts will be in UTC. + // We'll check the row instead. + "/insert into t1 set id=", + "/update _vt.vreplication set pos=", + "commit", + }) + + qr, err = env.Mysqld.FetchSuperQuery(context.Background(), "select ts, dt from t1 where id=1") + if err != nil { + t.Fatal(err) + } + // The value for dt should come back in the local timezone. + if got := qr.Rows[0][0].ToString(); got != want { + t.Errorf("ts: %s, want %s", got, want) + } + // The value for dt should be as is. + if got := qr.Rows[0][1].ToString(); got != want { + t.Errorf("ts: %s, want %s", got, want) + } +} + func execStatements(t *testing.T, queries []string) { t.Helper() if err := env.Mysqld.ExecuteSuperQueryList(context.Background(), queries); err != nil { @@ -485,7 +915,7 @@ func execStatements(t *testing.T, queries []string) { } } -func startVReplication(t *testing.T, pe *Engine, filter *binlogdatapb.Filter, onddl binlogdatapb.OnDDLAction, pos string) (cancelFunc func(), id int) { +func startVReplication(t *testing.T, filter *binlogdatapb.Filter, onddl binlogdatapb.OnDDLAction, pos string) (cancelFunc func(), id int) { t.Helper() bls := &binlogdatapb.BinlogSource{ @@ -498,7 +928,7 @@ func startVReplication(t *testing.T, pe *Engine, filter *binlogdatapb.Filter, on pos = masterPosition(t) } query := binlogplayer.CreateVReplication("test", bls, pos, 9223372036854775807, 9223372036854775807, 0) - qr, err := pe.Exec(query) + qr, err := playerEngine.Exec(query) if err != nil { t.Fatal(err) } @@ -511,7 +941,7 @@ func startVReplication(t *testing.T, pe *Engine, filter *binlogdatapb.Filter, on return func() { t.Helper() query := fmt.Sprintf("delete from _vt.vreplication where id = %d", qr.InsertID) - if _, err := pe.Exec(query); err != nil { + if _, err := playerEngine.Exec(query); err != nil { t.Fatal(err) } expectDBClientQueries(t, []string{ From f9c0f3122b8624c2fc064bff0dd80558a86e9972 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Tue, 22 Jan 2019 02:14:12 -0800 Subject: [PATCH 17/27] vplayer: timezone fix and more tests Signed-off-by: Sugu Sougoumarane --- .../tabletmanager/vreplication/controller.go | 4 ++++ .../tabletmanager/vreplication/framework_test.go | 2 +- .../tabletmanager/vreplication/retryable_client.go | 10 ---------- .../vttablet/tabletmanager/vreplication/vplayer.go | 1 + .../tabletmanager/vreplication/vplayer_test.go | 13 ++++++++++--- 5 files changed, 16 insertions(+), 14 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vreplication/controller.go b/go/vt/vttablet/tabletmanager/vreplication/controller.go index 665d0f8a2a7..75d1c7c32de 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/controller.go +++ b/go/vt/vttablet/tabletmanager/vreplication/controller.go @@ -189,6 +189,10 @@ 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: + // VPlayer requires the timezone to be UTC. + if _, err := dbClient.ExecuteFetch("set @@session.time_zone = '+00:00'", 10000); err != nil { + return err + } vplayer := newVPlayer(ct.id, &ct.source, tablet, ct.blpStats, dbClient, ct.mysqld) return vplayer.Play(ctx) } diff --git a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go index ea1cd2dc63d..421a046ccfa 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go @@ -327,7 +327,7 @@ func (dbc *realDBClient) ExecuteFetch(query string, maxrows int) (*sqltypes.Resu return nil, nil } qr, err := dbc.conn.ExecuteFetch(query, 10000, true) - if !strings.HasPrefix(query, "select") && !dbc.nolog { + if !strings.HasPrefix(query, "select") && !strings.HasPrefix(query, "set") && !dbc.nolog { globalDBQueries <- query } return qr, err diff --git a/go/vt/vttablet/tabletmanager/vreplication/retryable_client.go b/go/vt/vttablet/tabletmanager/vreplication/retryable_client.go index f08cd99cf37..e1edaf78966 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/retryable_client.go +++ b/go/vt/vttablet/tabletmanager/vreplication/retryable_client.go @@ -29,16 +29,6 @@ type retryableClient struct { queries []string } -func (rt *retryableClient) Connect() error { - if err := rt.Connect(); err != nil { - return err - } - if _, err := rt.DBClient.ExecuteFetch("set @@session.time_zone = '+00:00'", 10000); err != nil { - return err - } - return nil -} - func (rt *retryableClient) Begin() error { if err := rt.DBClient.Begin(); err != nil { return err diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index 66b2937d7a4..328fa34b4e6 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -625,5 +625,6 @@ func encodeValue(sql *sqlparser.TrackedBuffer, value sqltypes.Value) { // Previously, this function used to convert timestamp to the session // time zone, but we now set the session timezone to UTC. So, the timestamp // value we receive as UTC can be sent as is. + // TODO(sougou): handle BIT data type here? value.EncodeSQL(sql) } diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go index a1b06f4b609..bd98ed9ecc4 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go @@ -642,7 +642,7 @@ func TestPlayerBatching(t *testing.T) { Match: "/.*", }}, } - cancel, _ := startVReplication(t, filter, binlogdatapb.OnDDLAction_IGNORE, "") + cancel, _ := startVReplication(t, filter, binlogdatapb.OnDDLAction_EXEC, "") defer cancel() execStatements(t, []string{ @@ -682,12 +682,15 @@ func TestPlayerBatching(t *testing.T) { execStatements(t, []string{ "insert into t1 values(2, 'aaa')", "insert into t1 values(3, 'aaa')", + "create table t2(id int, val varbinary(128), primary key(id))", + "drop table t2", }) // Release the lock. _, _ = vconn.ExecuteFetch("rollback", 1) // First transaction will complete. The other two - // transactions must be batched into one + // transactions must be batched into one. But the + // DDLs should be on their own. expectDBClientQueries(t, []string{ "update t1 set id=1, val='ccc' where id=1", "/update _vt.vreplication set pos=", @@ -697,6 +700,10 @@ func TestPlayerBatching(t *testing.T) { "insert into t1 set id=3, val='aaa'", "/update _vt.vreplication set pos=", "commit", + "create table t2(id int, val varbinary(128), primary key(id))", + "/update _vt.vreplication set pos=", + "/", // drop table is rewritten by mysql. Don't check. + "/update _vt.vreplication set pos=", }) } @@ -894,7 +901,7 @@ func TestTimestamp(t *testing.T) { "commit", }) - qr, err = env.Mysqld.FetchSuperQuery(context.Background(), "select ts, dt from t1 where id=1") + qr, err = env.Mysqld.FetchSuperQuery(context.Background(), fmt.Sprintf("select ts, dt from %s.t1 where id=1", vrepldb)) if err != nil { t.Fatal(err) } From 8a963318320524cf3cd6ec6d38066ca9f5c68b02 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Tue, 22 Jan 2019 16:40:01 -0800 Subject: [PATCH 18/27] vplayer: tables with no pk and more tests Signed-off-by: Sugu Sougoumarane --- .../fakemysqldaemon/fakemysqldaemon.go | 5 + go/vt/mysqlctl/mysql_daemon.go | 1 + .../vreplication/framework_test.go | 23 ++++ .../vreplication/retryable_client.go | 13 ++- .../tabletmanager/vreplication/vplayer.go | 59 +++------- .../vreplication/vplayer_test.go | 103 ++++++++++++++++-- 6 files changed, 146 insertions(+), 58 deletions(-) diff --git a/go/vt/mysqlctl/fakemysqldaemon/fakemysqldaemon.go b/go/vt/mysqlctl/fakemysqldaemon/fakemysqldaemon.go index e2dda296fb0..5688624ba5e 100644 --- a/go/vt/mysqlctl/fakemysqldaemon/fakemysqldaemon.go +++ b/go/vt/mysqlctl/fakemysqldaemon/fakemysqldaemon.go @@ -383,6 +383,11 @@ func (fmd *FakeMysqlDaemon) GetSchema(dbName string, tables, excludeTables []str return tmutils.FilterTables(fmd.Schema, tables, excludeTables, includeViews) } +// GetColumns is part of the MysqlDaemon interface +func (fmd *FakeMysqlDaemon) GetColumns(dbName, table string) ([]string, error) { + return []string{}, nil +} + // GetPrimaryKeyColumns is part of the MysqlDaemon interface func (fmd *FakeMysqlDaemon) GetPrimaryKeyColumns(dbName, table string) ([]string, error) { return []string{}, nil diff --git a/go/vt/mysqlctl/mysql_daemon.go b/go/vt/mysqlctl/mysql_daemon.go index 49b30dbf65e..60e59ee3b71 100644 --- a/go/vt/mysqlctl/mysql_daemon.go +++ b/go/vt/mysqlctl/mysql_daemon.go @@ -69,6 +69,7 @@ type MysqlDaemon interface { // Schema related methods GetSchema(dbName string, tables, excludeTables []string, includeViews bool) (*tabletmanagerdatapb.SchemaDefinition, error) + GetColumns(dbName, table string) ([]string, error) GetPrimaryKeyColumns(dbName, table string) ([]string, error) PreflightSchemaChange(dbName string, changes []string) ([]*tabletmanagerdatapb.SchemaChangeResult, error) ApplySchemaChange(dbName string, change *tmutils.SchemaChange) (*tabletmanagerdatapb.SchemaChangeResult, error) diff --git a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go index 421a046ccfa..9d44d314cb2 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go @@ -371,3 +371,26 @@ func expectDBClientQueries(t *testing.T, queries []string) { } } } + +func expectData(t *testing.T, table string, values [][]string) { + t.Helper() + + qr, err := env.Mysqld.FetchSuperQuery(context.Background(), fmt.Sprintf("select * from %s.%s", vrepldb, table)) + if err != nil { + t.Error(err) + return + } + for i, row := range values { + if i >= len(qr.Rows) { + t.Errorf("Result too short, row: %d, want: %v", i, row) + } + for j, val := range row { + if j >= len(qr.Rows[i]) { + t.Errorf("Too few columns, result: %v, row: %d, want: %v", qr.Rows[i], i, row) + } + if got := qr.Rows[i][j].ToString(); got != val { + t.Errorf("Mismatch at (%d, %d): %v, want %s", i, j, qr.Rows[i][j], val) + } + } + } +} diff --git a/go/vt/vttablet/tabletmanager/vreplication/retryable_client.go b/go/vt/vttablet/tabletmanager/vreplication/retryable_client.go index e1edaf78966..f5e8eaa4efc 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/retryable_client.go +++ b/go/vt/vttablet/tabletmanager/vreplication/retryable_client.go @@ -25,16 +25,19 @@ import ( // It allows us to retry a failed transactions on lock errors. type retryableClient struct { binlogplayer.DBClient - inTransaction bool + InTransaction bool queries []string } func (rt *retryableClient) Begin() error { + if rt.InTransaction { + return nil + } if err := rt.DBClient.Begin(); err != nil { return err } rt.queries = append(rt.queries, "begin") - rt.inTransaction = true + rt.InTransaction = true return nil } @@ -42,7 +45,7 @@ func (rt *retryableClient) Commit() error { if err := rt.DBClient.Commit(); err != nil { return err } - rt.inTransaction = false + rt.InTransaction = false rt.queries = nil return nil } @@ -51,13 +54,13 @@ func (rt *retryableClient) Rollback() error { if err := rt.DBClient.Rollback(); err != nil { return err } - rt.inTransaction = false + rt.InTransaction = false // Don't reset queries to allow for vplayer to retry. return nil } func (rt *retryableClient) ExecuteFetch(query string, maxrows int) (*sqltypes.Result, error) { - if !rt.inTransaction { + if !rt.InTransaction { rt.queries = []string{query} } else { rt.queries = append(rt.queries, query) diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index 328fa34b4e6..c9c90a1b2b9 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -62,9 +62,6 @@ type vplayer struct { // timeLastSaved is set every time a GTID is saved. timeLastSaved time.Time stopPos mysql.Position - // inTransaction is true if we've started a transaction. - // It remains true until the next commit or rollback. - inTransaction bool // pplan is built based on the source Filter at the beginning. pplan *playerPlan @@ -207,7 +204,7 @@ func (vp *vplayer) applyEvents(ctx context.Context, relay *relayLog) error { if time.Now().Sub(vp.timeLastSaved) >= idleTimeout && vp.unsavedGTID != nil { // Although unlikely, we should not save if a transaction is still open. // This can happen if a large transaction is split as multiple events. - if !vp.inTransaction { + if !vp.dbClient.InTransaction { if err := vp.updatePos(vp.unsavedGTID.Timestamp); err != nil { return err } @@ -275,12 +272,12 @@ func (vp *vplayer) applyEvent(ctx context.Context, event *binlogdatapb.VEvent, m // No-op: begin is called as needed. case binlogdatapb.VEventType_COMMIT: if mustSave { - if err := vp.begin(); err != nil { + if err := vp.dbClient.Begin(); err != nil { return err } } - if !vp.inTransaction { + if !vp.dbClient.InTransaction { return nil } if err := vp.updatePos(event.Timestamp); err != nil { @@ -292,35 +289,35 @@ func (vp *vplayer) applyEvent(ctx context.Context, event *binlogdatapb.VEvent, m return err } } - if err := vp.commit(); err != nil { + if err := vp.dbClient.Commit(); err != nil { return err } if posReached { return io.EOF } case binlogdatapb.VEventType_FIELD: - if err := vp.begin(); err != nil { + if err := vp.dbClient.Begin(); err != nil { return err } if err := vp.updatePlan(event.FieldEvent); err != nil { return err } case binlogdatapb.VEventType_ROW: - if err := vp.begin(); err != nil { + if err := vp.dbClient.Begin(); err != nil { return err } if err := vp.applyRowEvent(ctx, event.RowEvent); err != nil { return err } case binlogdatapb.VEventType_DDL: - if vp.inTransaction { + if vp.dbClient.InTransaction { return fmt.Errorf("unexpected state: DDL encountered in the middle of a transaction: %v", event.Ddl) } switch vp.source.OnDdl { case binlogdatapb.OnDDLAction_IGNORE: // no-op case binlogdatapb.OnDDLAction_STOP: - if err := vp.begin(); err != nil { + if err := vp.dbClient.Begin(); err != nil { return err } if err := vp.updatePos(event.Timestamp); err != nil { @@ -329,7 +326,7 @@ func (vp *vplayer) applyEvent(ctx context.Context, event *binlogdatapb.VEvent, m if err := vp.setState(binlogplayer.BlpStopped, fmt.Sprintf("Stopped at DDL %s", event.Ddl)); err != nil { return err } - if err := vp.commit(); err != nil { + if err := vp.dbClient.Commit(); err != nil { return err } return io.EOF @@ -352,33 +349,6 @@ func (vp *vplayer) applyEvent(ctx context.Context, event *binlogdatapb.VEvent, m return nil } -func (vp *vplayer) begin() error { - if vp.inTransaction { - return nil - } - if err := vp.dbClient.Begin(); err != nil { - return err - } - vp.inTransaction = true - return nil -} - -func (vp *vplayer) commit() error { - if !vp.inTransaction { - return nil - } - if err := vp.dbClient.Commit(); err != nil { - return err - } - vp.inTransaction = false - return nil -} - -func (vp *vplayer) rollback() { - vp.inTransaction = false - _ = vp.dbClient.Rollback() -} - func (vp *vplayer) setState(state, message string) error { return binlogplayer.SetVReplicationState(vp.dbClient, vp.id, state, message) } @@ -414,6 +384,13 @@ func (vp *vplayer) updatePlan(fieldEvent *binlogdatapb.FieldEvent) error { if err != nil { return fmt.Errorf("error fetching pk columns for %s: %v", tplan.name, err) } + if len(pkcols) == 0 { + // If the table doesn't have a PK, then we treat all columns as PK. + pkcols, err = vp.mysqld.GetColumns(vp.dbClient.DBName(), tplan.name) + if err != nil { + return fmt.Errorf("error fetching pk columns for %s: %v", tplan.name, err) + } + } for _, pkcol := range pkcols { found := false for i, cExpr := range tplan.colExprs { @@ -577,7 +554,7 @@ func (vp *vplayer) writeWhereValues(sql *sqlparser.TrackedBuffer, tplan *tablePl for _, cExpr := range tplan.pkCols { sql.Myprintf("%s%v=", separator, cExpr.colname) if separator == "" { - separator = " AND " + separator = " and " } encodeValue(sql, before[cExpr.colnum]) } @@ -586,7 +563,7 @@ func (vp *vplayer) writeWhereValues(sql *sqlparser.TrackedBuffer, tplan *tablePl func (vp *vplayer) updatePos(ts int64) error { updatePos := binlogplayer.GenerateUpdatePos(vp.id, vp.pos, time.Now().Unix(), ts) if _, err := vp.dbClient.ExecuteFetch(updatePos, 0); err != nil { - vp.rollback() + vp.dbClient.Rollback() return fmt.Errorf("error %v updating position", err) } vp.unsavedGTID = nil diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go index bd98ed9ecc4..d8be7ddf287 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go @@ -44,6 +44,8 @@ func TestPlayerFilters(t *testing.T) { "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 nopk(id int, val varbinary(128))", + fmt.Sprintf("create table %s.nopk(id int, val varbinary(128))", vrepldb), }) defer execStatements(t, []string{ "drop table src1", @@ -55,6 +57,8 @@ func TestPlayerFilters(t *testing.T) { "drop table yes", fmt.Sprintf("drop table %s.yes", vrepldb), "drop table no", + "drop table nopk", + fmt.Sprintf("drop table %s.nopk", vrepldb), }) env.SchemaEngine.Reload(context.Background()) @@ -70,6 +74,8 @@ func TestPlayerFilters(t *testing.T) { Filter: "select id, val from src3 group by id, val", }, { Match: "/yes", + }, { + Match: "/nopk", }}, } cancel, _ := startVReplication(t, filter, binlogdatapb.OnDDLAction_IGNORE, "") @@ -78,6 +84,8 @@ func TestPlayerFilters(t *testing.T) { testcases := []struct { input string output []string + table string + data [][]string }{{ // insert with insertNormal input: "insert into src1 values(1, 'aaa')", @@ -87,6 +95,10 @@ func TestPlayerFilters(t *testing.T) { "/update _vt.vreplication set pos=", "commit", }, + table: "dst1", + data: [][]string{ + {"1", "aaa"}, + }, }, { // update with insertNormal input: "update src1 set val='bbb'", @@ -96,6 +108,10 @@ func TestPlayerFilters(t *testing.T) { "/update _vt.vreplication set pos=", "commit", }, + table: "dst1", + data: [][]string{ + {"1", "bbb"}, + }, }, { // delete with insertNormal input: "delete from src1 where id=1", @@ -105,6 +121,8 @@ func TestPlayerFilters(t *testing.T) { "/update _vt.vreplication set pos=", "commit", }, + table: "dst1", + data: [][]string{}, }, { // insert with insertOnDup input: "insert into src2 values(1, 2, 3)", @@ -114,6 +132,10 @@ func TestPlayerFilters(t *testing.T) { "/update _vt.vreplication set pos=", "commit", }, + table: "dst2", + data: [][]string{ + {"1", "2", "3"}, + }, }, { // update with insertOnDup input: "update src2 set val1=5, val2=1 where id=1", @@ -123,6 +145,10 @@ func TestPlayerFilters(t *testing.T) { "/update _vt.vreplication set pos=", "commit", }, + table: "dst2", + data: [][]string{ + {"1", "5", "1"}, + }, }, { // delete with insertOnDup input: "delete from src2 where id=1", @@ -132,6 +158,10 @@ func TestPlayerFilters(t *testing.T) { "/update _vt.vreplication set pos=", "commit", }, + table: "dst2", + data: [][]string{ + {"1", "", "0"}, + }, }, { // insert with insertIgnore input: "insert into src3 values(1, 'aaa')", @@ -141,6 +171,10 @@ func TestPlayerFilters(t *testing.T) { "/update _vt.vreplication set pos=", "commit", }, + table: "dst3", + data: [][]string{ + {"1", "aaa"}, + }, }, { // update with insertIgnore input: "update src3 set val='bbb'", @@ -150,6 +184,10 @@ func TestPlayerFilters(t *testing.T) { "/update _vt.vreplication set pos=", "commit", }, + table: "dst3", + data: [][]string{ + {"1", "aaa"}, + }, }, { // delete with insertIgnore input: "delete from src3 where id=1", @@ -158,6 +196,10 @@ func TestPlayerFilters(t *testing.T) { "/update _vt.vreplication set pos=", "commit", }, + table: "dst3", + data: [][]string{ + {"1", "aaa"}, + }, }, { // insert: regular expression filter input: "insert into yes values(1, 'aaa')", @@ -167,6 +209,10 @@ func TestPlayerFilters(t *testing.T) { "/update _vt.vreplication set pos=", "commit", }, + table: "yes", + data: [][]string{ + {"1", "aaa"}, + }, }, { // update: regular expression filter input: "update yes set val='bbb'", @@ -176,15 +222,59 @@ func TestPlayerFilters(t *testing.T) { "/update _vt.vreplication set pos=", "commit", }, + table: "yes", + data: [][]string{ + {"1", "bbb"}, + }, }, { // table should not match a rule input: "insert into no values(1, 'aaa')", output: []string{}, + }, { + // nopk: insert + input: "insert into nopk values(1, 'aaa')", + output: []string{ + "begin", + "insert into nopk set id=1, val='aaa'", + "/update _vt.vreplication set pos=", + "commit", + }, + table: "nopk", + data: [][]string{ + {"1", "aaa"}, + }, + }, { + // nopk: update + input: "update nopk set val='bbb' where id=1", + output: []string{ + "begin", + "update nopk set id=1, val='bbb' where id=1 and val='aaa'", + "/update _vt.vreplication set pos=", + "commit", + }, + table: "nopk", + data: [][]string{ + {"1", "bbb"}, + }, + }, { + // nopk: delete + input: "delete from nopk where id=1", + output: []string{ + "begin", + "delete from nopk where id=1 and val='bbb'", + "/update _vt.vreplication set pos=", + "commit", + }, + table: "nopk", + data: [][]string{}, }} for _, tcases := range testcases { execStatements(t, []string{tcases.input}) expectDBClientQueries(t, tcases.output) + if tcases.table != "" { + expectData(t, tcases.table, tcases.data) + } } } @@ -901,18 +991,7 @@ func TestTimestamp(t *testing.T) { "commit", }) - qr, err = env.Mysqld.FetchSuperQuery(context.Background(), fmt.Sprintf("select ts, dt from %s.t1 where id=1", vrepldb)) - if err != nil { - t.Fatal(err) - } - // The value for dt should come back in the local timezone. - if got := qr.Rows[0][0].ToString(); got != want { - t.Errorf("ts: %s, want %s", got, want) - } - // The value for dt should be as is. - if got := qr.Rows[0][1].ToString(); got != want { - t.Errorf("ts: %s, want %s", got, want) - } + expectData(t, "t1", [][]string{{"1", want, want}}) } func execStatements(t *testing.T, queries []string) { From c0dbf473e730ddbfc6a1dfba9256bf7b48a19ef3 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Sat, 26 Jan 2019 03:07:19 -0800 Subject: [PATCH 19/27] vplayer: PlayerPlan tests Signed-off-by: Sugu Sougoumarane --- .../vreplication/controller_plan_test.go | 2 +- .../tabletmanager/vreplication/player_plan.go | 132 +++--- .../vreplication/player_plan_test.go | 435 ++++++++++++++++++ .../tabletmanager/vreplication/vplayer.go | 142 +++--- 4 files changed, 580 insertions(+), 131 deletions(-) create mode 100644 go/vt/vttablet/tabletmanager/vreplication/player_plan_test.go diff --git a/go/vt/vttablet/tabletmanager/vreplication/controller_plan_test.go b/go/vt/vttablet/tabletmanager/vreplication/controller_plan_test.go index 6d9a7c54caf..533668a2955 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/controller_plan_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/controller_plan_test.go @@ -21,7 +21,7 @@ import ( "testing" ) -func TestPlanBuilder(t *testing.T) { +func TestControllerPlan(t *testing.T) { tcases := []struct { in string plan *controllerPlan diff --git a/go/vt/vttablet/tabletmanager/vreplication/player_plan.go b/go/vt/vttablet/tabletmanager/vreplication/player_plan.go index 25100ae3773..77ecde44d5d 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/player_plan.go +++ b/go/vt/vttablet/tabletmanager/vreplication/player_plan.go @@ -26,78 +26,89 @@ import ( querypb "vitess.io/vitess/go/vt/proto/query" ) -type playerPlan struct { - vstreamFilter *binlogdatapb.Filter - tablePlans map[string]*tablePlan +// PlayerPlan is the execution plan for a player stream. +type PlayerPlan struct { + VStreamFilter *binlogdatapb.Filter + TablePlans map[string]*TablePlan } -type tablePlan struct { - name string - colExprs []*colExpr - onInsert insertType - - fields []*querypb.Field - pkCols []*colExpr +// TablePlan is the execution plan for a table within a player stream. +// There are two incarantions of this per table. The first one is built +// while analyzing the inital stream request. A tentative plan is built +// without knowing the table info. The second incarnation is built when +// we receive the field info for a table. At that time, we copy the +// original TablePlan into a separtae map and populate the Fields and +// PKCols members. +type TablePlan struct { + Name string + ColExprs []*ColExpr `json:",omitempty"` + OnInsert InsertType `json:",omitempty"` + + Fields []*querypb.Field `json:",omitempty"` + PKCols []*ColExpr `json:",omitempty"` } -func (tp *tablePlan) findCol(name sqlparser.ColIdent) *colExpr { - for _, cExpr := range tp.colExprs { - if cExpr.colname.Equal(name) { +func (tp *TablePlan) findCol(name sqlparser.ColIdent) *ColExpr { + for _, cExpr := range tp.ColExprs { + if cExpr.ColName.Equal(name) { return cExpr } } return nil } -type colExpr struct { - colname sqlparser.ColIdent - colnum int - op operation - isGrouped bool +// ColExpr describes the processing to be performed to +// compute the value of the target table column. +type ColExpr struct { + ColName sqlparser.ColIdent + ColNum int + Operation Operation `json:",omitempty"` + IsGrouped bool `json:",omitempty"` } -type operation int +// Operation is the opcode for the ColExpr. +type Operation int +// The following values are the various ColExpr opcodes. const ( - opNone = operation(iota) - opCount - opSum + OpNone = Operation(iota) + OpCount + OpSum ) -type insertType int +// InsertType describes the type of insert statement to generate. +type InsertType int +// The following values are the various insert types. const ( - insertNormal = insertType(iota) - insertOndup - insertIgnore + InsertNormal = InsertType(iota) + InsertOndup + InsertIgnore ) -func buildPlayerPlan(filter *binlogdatapb.Filter) (*playerPlan, error) { - plan := &playerPlan{ - vstreamFilter: &binlogdatapb.Filter{ +func buildPlayerPlan(filter *binlogdatapb.Filter) (*PlayerPlan, error) { + plan := &PlayerPlan{ + VStreamFilter: &binlogdatapb.Filter{ Rules: make([]*binlogdatapb.Rule, len(filter.Rules)), }, - tablePlans: make(map[string]*tablePlan), + TablePlans: make(map[string]*TablePlan), } for i, rule := range filter.Rules { if strings.HasPrefix(rule.Match, "/") { - plan.vstreamFilter.Rules[i] = rule + plan.VStreamFilter.Rules[i] = rule continue } sendRule, tplan, err := buildTablePlan(rule) if err != nil { return nil, err } - if tplan == nil { - continue - } - plan.vstreamFilter.Rules[i] = sendRule - plan.tablePlans[sendRule.Match] = tplan + plan.VStreamFilter.Rules[i] = sendRule + plan.TablePlans[sendRule.Match] = tplan } return plan, nil } -func buildTablePlan(rule *binlogdatapb.Rule) (*binlogdatapb.Rule, *tablePlan, error) { +func buildTablePlan(rule *binlogdatapb.Rule) (*binlogdatapb.Rule, *TablePlan, error) { statement, err := sqlparser.Parse(rule.Filter) if err != nil { return nil, nil, err @@ -106,6 +117,9 @@ func buildTablePlan(rule *binlogdatapb.Rule) (*binlogdatapb.Rule, *tablePlan, er if !ok { return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(statement)) } + if sel.Distinct != "" { + return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(sel)) + } if len(sel.From) > 1 { return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(sel)) } @@ -126,11 +140,11 @@ func buildTablePlan(rule *binlogdatapb.Rule) (*binlogdatapb.Rule, *tablePlan, er Match: fromTable.String(), Filter: rule.Filter, } - return sendRule, &tablePlan{name: rule.Match}, nil + return sendRule, &TablePlan{Name: rule.Match}, nil } - tplan := &tablePlan{ - name: rule.Match, + tplan := &TablePlan{ + Name: rule.Match, } sendSelect := &sqlparser.Select{ From: sel.From, @@ -143,19 +157,19 @@ func buildTablePlan(rule *binlogdatapb.Rule) (*binlogdatapb.Rule, *tablePlan, er } if selExpr != nil { sendSelect.SelectExprs = append(sendSelect.SelectExprs, selExpr) - cExpr.colnum = len(sendSelect.SelectExprs) - 1 + cExpr.ColNum = len(sendSelect.SelectExprs) - 1 } - tplan.colExprs = append(tplan.colExprs, cExpr) + tplan.ColExprs = append(tplan.ColExprs, cExpr) } if sel.GroupBy != nil { if err := analyzeGroupBy(sel.GroupBy, tplan); err != nil { return nil, nil, err } - tplan.onInsert = insertIgnore - for _, cExpr := range tplan.colExprs { - if !cExpr.isGrouped { - tplan.onInsert = insertOndup + tplan.OnInsert = InsertIgnore + for _, cExpr := range tplan.ColExprs { + if !cExpr.IsGrouped { + tplan.OnInsert = InsertOndup break } } @@ -167,7 +181,7 @@ func buildTablePlan(rule *binlogdatapb.Rule) (*binlogdatapb.Rule, *tablePlan, er return sendRule, tplan, nil } -func analyzeExpr(selExpr sqlparser.SelectExpr) (sqlparser.SelectExpr, *colExpr, error) { +func analyzeExpr(selExpr sqlparser.SelectExpr) (sqlparser.SelectExpr, *ColExpr, error) { aliased, ok := selExpr.(*sqlparser.AliasedExpr) if !ok { return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(selExpr)) @@ -178,54 +192,54 @@ func analyzeExpr(selExpr sqlparser.SelectExpr) (sqlparser.SelectExpr, *colExpr, } switch expr := aliased.Expr.(type) { case *sqlparser.ColName: - return selExpr, &colExpr{colname: as}, nil + return selExpr, &ColExpr{ColName: as}, nil case *sqlparser.FuncExpr: if expr.Distinct || len(expr.Exprs) != 1 { - return nil, nil, fmt.Errorf("unsupported: %v", sqlparser.String(expr)) + return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(expr)) } if aliased.As.IsEmpty() { return nil, nil, fmt.Errorf("expression needs an alias: %v", sqlparser.String(expr)) } switch fname := expr.Name.Lowered(); fname { case "month", "day", "hour": - return selExpr, &colExpr{colname: as}, nil + return selExpr, &ColExpr{ColName: as}, nil case "count": if _, ok := expr.Exprs[0].(*sqlparser.StarExpr); !ok { return nil, nil, fmt.Errorf("only count(*) is supported: %v", sqlparser.String(expr)) } - return nil, &colExpr{colname: as, op: opCount}, nil + return nil, &ColExpr{ColName: as, Operation: OpCount}, nil case "sum": aInner, ok := expr.Exprs[0].(*sqlparser.AliasedExpr) if !ok { - return nil, nil, fmt.Errorf("unsupported: %v", sqlparser.String(expr)) + return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(expr)) } innerCol, ok := aInner.Expr.(*sqlparser.ColName) if !ok { - return nil, nil, fmt.Errorf("unsupported: %v", sqlparser.String(expr)) + return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(expr)) } - return &sqlparser.AliasedExpr{Expr: innerCol}, &colExpr{colname: as, op: opSum}, nil + return &sqlparser.AliasedExpr{Expr: innerCol}, &ColExpr{ColName: as, Operation: OpSum}, nil default: - return nil, nil, fmt.Errorf("unsupported: %v", sqlparser.String(expr)) + return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(expr)) } default: return nil, nil, fmt.Errorf("unexpected: %v", sqlparser.String(expr)) } } -func analyzeGroupBy(groupBy sqlparser.GroupBy, tplan *tablePlan) error { +func analyzeGroupBy(groupBy sqlparser.GroupBy, tplan *TablePlan) error { for _, expr := range groupBy { colname, ok := expr.(*sqlparser.ColName) if !ok { - return fmt.Errorf("unsupported: %v", sqlparser.String(expr)) + return fmt.Errorf("unexpected: %v", sqlparser.String(expr)) } cExpr := tplan.findCol(colname.Name) if cExpr == nil { return fmt.Errorf("group by expression does not reference an alias in the select list: %v", sqlparser.String(expr)) } - if cExpr.op != opNone { + if cExpr.Operation != OpNone { return fmt.Errorf("group by expression is not allowed to reference an aggregate expression: %v", sqlparser.String(expr)) } - cExpr.isGrouped = true + cExpr.IsGrouped = true } return nil } diff --git a/go/vt/vttablet/tabletmanager/vreplication/player_plan_test.go b/go/vt/vttablet/tabletmanager/vreplication/player_plan_test.go new file mode 100644 index 00000000000..bf5066002f2 --- /dev/null +++ b/go/vt/vttablet/tabletmanager/vreplication/player_plan_test.go @@ -0,0 +1,435 @@ +/* +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 vreplication + +import ( + "encoding/json" + "testing" + + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" + "vitess.io/vitess/go/vt/sqlparser" +) + +func TestPlayerPlan(t *testing.T) { + testcases := []struct { + input *binlogdatapb.Filter + plan *PlayerPlan + err string + }{{ + // Regular expression + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "/.*", + }}, + }, + plan: &PlayerPlan{ + VStreamFilter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "/.*", + }}, + }, + TablePlans: map[string]*TablePlan{}, + }, + }, { + // '*' expression + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select * from t2", + }}, + }, + plan: &PlayerPlan{ + VStreamFilter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t2", + Filter: "select * from t2", + }}, + }, + TablePlans: map[string]*TablePlan{ + "t2": { + Name: "t1", + }, + }, + }, + }, { + // Explicit columns + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select c1, c2 from t2", + }}, + }, + plan: &PlayerPlan{ + VStreamFilter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t2", + Filter: "select c1, c2 from t2", + }}, + }, + TablePlans: map[string]*TablePlan{ + "t2": { + Name: "t1", + ColExprs: []*ColExpr{{ + ColName: sqlparser.NewColIdent("c1"), + ColNum: 0, + }, { + ColName: sqlparser.NewColIdent("c2"), + ColNum: 1, + }}, + }, + }, + }, + }, { + // func expr + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select hour(c1) as hc1, day(c2) as dc2 from t2", + }}, + }, + plan: &PlayerPlan{ + VStreamFilter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t2", + Filter: "select hour(c1) as hc1, day(c2) as dc2 from t2", + }}, + }, + TablePlans: map[string]*TablePlan{ + "t2": { + Name: "t1", + ColExprs: []*ColExpr{{ + ColName: sqlparser.NewColIdent("hc1"), + ColNum: 0, + }, { + ColName: sqlparser.NewColIdent("dc2"), + ColNum: 1, + }}, + }, + }, + }, + }, { + // count expr + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select hour(c1) as hc1, count(*) as c, day(c2) as dc2 from t2", + }}, + }, + plan: &PlayerPlan{ + VStreamFilter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t2", + Filter: "select hour(c1) as hc1, day(c2) as dc2 from t2", + }}, + }, + TablePlans: map[string]*TablePlan{ + "t2": { + Name: "t1", + ColExprs: []*ColExpr{{ + ColName: sqlparser.NewColIdent("hc1"), + ColNum: 0, + }, { + ColName: sqlparser.NewColIdent("c"), + Operation: OpCount, + }, { + ColName: sqlparser.NewColIdent("dc2"), + ColNum: 1, + }}, + }, + }, + }, + }, { + // sum expr + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select hour(c1) as hc1, sum(c3) as s, day(c2) as dc2 from t2", + }}, + }, + plan: &PlayerPlan{ + VStreamFilter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t2", + Filter: "select hour(c1) as hc1, c3, day(c2) as dc2 from t2", + }}, + }, + TablePlans: map[string]*TablePlan{ + "t2": { + Name: "t1", + ColExprs: []*ColExpr{{ + ColName: sqlparser.NewColIdent("hc1"), + ColNum: 0, + }, { + ColName: sqlparser.NewColIdent("s"), + ColNum: 1, + Operation: OpSum, + }, { + ColName: sqlparser.NewColIdent("dc2"), + ColNum: 2, + }}, + }, + }, + }, + }, { + // partial group by + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select c1, c2, c3 from t2 group by c3, c1", + }}, + }, + plan: &PlayerPlan{ + VStreamFilter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t2", + Filter: "select c1, c2, c3 from t2", + }}, + }, + TablePlans: map[string]*TablePlan{ + "t2": { + Name: "t1", + ColExprs: []*ColExpr{{ + ColName: sqlparser.NewColIdent("c1"), + ColNum: 0, + IsGrouped: true, + }, { + ColName: sqlparser.NewColIdent("c2"), + ColNum: 1, + }, { + ColName: sqlparser.NewColIdent("c3"), + ColNum: 2, + IsGrouped: true, + }}, + OnInsert: InsertOndup, + }, + }, + }, + }, { + // full group by + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select c1, c2, c3 from t2 group by c3, c1, c2", + }}, + }, + plan: &PlayerPlan{ + VStreamFilter: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t2", + Filter: "select c1, c2, c3 from t2", + }}, + }, + TablePlans: map[string]*TablePlan{ + "t2": { + Name: "t1", + ColExprs: []*ColExpr{{ + ColName: sqlparser.NewColIdent("c1"), + ColNum: 0, + IsGrouped: true, + }, { + ColName: sqlparser.NewColIdent("c2"), + ColNum: 1, + IsGrouped: true, + }, { + ColName: sqlparser.NewColIdent("c3"), + ColNum: 2, + IsGrouped: true, + }}, + OnInsert: InsertIgnore, + }, + }, + }, + }, { + // syntax error + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "bad query", + }}, + }, + err: "syntax error at position 4 near 'bad'", + }, { + // not a select + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "update t1 set val=1", + }}, + }, + err: "unexpected: update t1 set val = 1", + }, { + // no distinct + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select distinct c1 from t1", + }}, + }, + err: "unexpected: select distinct c1 from t1", + }, { + // no ',' join + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select * from t1, t2", + }}, + }, + err: "unexpected: select * from t1, t2", + }, { + // no join + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select * from t1 join t2", + }}, + }, + err: "unexpected: select * from t1 join t2", + }, { + // no subqueries + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select * from (select * from t2) as a", + }}, + }, + err: "unexpected: select * from (select * from t2) as a", + }, { + // cannot combine '*' with other + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select *, c1 from t1", + }}, + }, + err: "unexpected: select *, c1 from t1", + }, { + // cannot combine '*' with other (different code path) + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select c1, * from t1", + }}, + }, + err: "unexpected: *", + }, { + // no distinct in func + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select hour(distinct c1) from t1", + }}, + }, + err: "unexpected: hour(distinct c1)", + }, { + // funcs need alias + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select hour(c1) from t1", + }}, + }, + err: "expression needs an alias: hour(c1)", + }, { + // only count(*) + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select count(c1) as c from t1", + }}, + }, + err: "only count(*) is supported: count(c1)", + }, { + // no sum(*) + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select sum(*) as c from t1", + }}, + }, + err: "unexpected: sum(*)", + }, { + // no complex expr in sum + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select sum(a + b) as c from t1", + }}, + }, + err: "unexpected: sum(a + b)", + }, { + // unsupported func + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select foo(a) as c from t1", + }}, + }, + err: "unexpected: foo(a)", + }, { + // no complex expr in select + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select a + b from t1", + }}, + }, + err: "unexpected: a + b", + }, { + // no complex expr in group by + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select a from t1 group by a + 1", + }}, + }, + err: "unexpected: a + 1", + }, { + // group by does not reference alias + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select a as b from t1 group by a", + }}, + }, + err: "group by expression does not reference an alias in the select list: a", + }, { + // cannot group by aggr + input: &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select count(*) as a from t1 group by a", + }}, + }, + err: "group by expression is not allowed to reference an aggregate expression: a", + }} + + for _, tcase := range testcases { + plan, err := buildPlayerPlan(tcase.input) + gotPlan, _ := json.Marshal(plan) + wantPlan, _ := json.Marshal(tcase.plan) + if string(gotPlan) != string(wantPlan) { + t.Errorf("Filter(%v):\n%s, want\n%s", tcase.input, gotPlan, wantPlan) + } + gotErr := "" + if err != nil { + gotErr = err.Error() + } + if gotErr != tcase.err { + t.Errorf("Filter err(%v): %s, want %v", tcase.input, gotErr, tcase.err) + } + } +} diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index c9c90a1b2b9..4315c10c8cc 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -64,10 +64,10 @@ type vplayer struct { stopPos mysql.Position // pplan is built based on the source Filter at the beginning. - pplan *playerPlan + pplan *PlayerPlan // tplans[table] is built for each table based on pplan and schema info // about the table. - tplans map[string]*tablePlan + tplans map[string]*TablePlan } func newVPlayer(id uint32, source *binlogdatapb.BinlogSource, sourceTablet *topodatapb.Tablet, stats *binlogplayer.Stats, dbClient binlogplayer.DBClient, mysqld mysqlctl.MysqlDaemon) *vplayer { @@ -79,7 +79,7 @@ func newVPlayer(id uint32, source *binlogdatapb.BinlogSource, sourceTablet *topo dbClient: &retryableClient{DBClient: dbClient}, mysqld: mysqld, timeLastSaved: time.Now(), - tplans: make(map[string]*tablePlan), + tplans: make(map[string]*TablePlan), } } @@ -143,10 +143,10 @@ func (vp *vplayer) play(ctx context.Context) error { Shard: vp.sourceTablet.Shard, TabletType: vp.sourceTablet.Type, } - log.Infof("Sending vstream command: %v", plan.vstreamFilter) + log.Infof("Sending vstream command: %v", plan.VStreamFilter) streamErr := make(chan error, 1) go func() { - streamErr <- vsClient.VStream(ctx, target, startPos, plan.vstreamFilter, func(events []*binlogdatapb.VEvent) error { + streamErr <- vsClient.VStream(ctx, target, startPos, plan.VStreamFilter, func(events []*binlogdatapb.VEvent) error { return relay.Send(events) }) }() @@ -354,57 +354,57 @@ func (vp *vplayer) setState(state, message string) error { } func (vp *vplayer) updatePlan(fieldEvent *binlogdatapb.FieldEvent) error { - prelim := vp.pplan.tablePlans[fieldEvent.TableName] - tplan := &tablePlan{ - name: fieldEvent.TableName, + prelim := vp.pplan.TablePlans[fieldEvent.TableName] + tplan := &TablePlan{ + Name: fieldEvent.TableName, } if prelim != nil { *tplan = *prelim } - tplan.fields = fieldEvent.Fields + tplan.Fields = fieldEvent.Fields - if tplan.colExprs == nil { - tplan.colExprs = make([]*colExpr, len(tplan.fields)) - for i, field := range tplan.fields { - tplan.colExprs[i] = &colExpr{ - colname: sqlparser.NewColIdent(field.Name), - colnum: i, + if tplan.ColExprs == nil { + tplan.ColExprs = make([]*ColExpr, len(tplan.Fields)) + for i, field := range tplan.Fields { + tplan.ColExprs[i] = &ColExpr{ + ColName: sqlparser.NewColIdent(field.Name), + ColNum: i, } } } else { - for _, cExpr := range tplan.colExprs { - if cExpr.colnum >= len(tplan.fields) { + for _, cExpr := range tplan.ColExprs { + if cExpr.ColNum >= len(tplan.Fields) { // Unreachable code. - return fmt.Errorf("columns received from vreplication: %v, do not match expected: %v", tplan.fields, tplan.colExprs) + return fmt.Errorf("columns received from vreplication: %v, do not match expected: %v", tplan.Fields, tplan.ColExprs) } } } - pkcols, err := vp.mysqld.GetPrimaryKeyColumns(vp.dbClient.DBName(), tplan.name) + pkcols, err := vp.mysqld.GetPrimaryKeyColumns(vp.dbClient.DBName(), tplan.Name) if err != nil { - return fmt.Errorf("error fetching pk columns for %s: %v", tplan.name, err) + return fmt.Errorf("error fetching pk columns for %s: %v", tplan.Name, err) } if len(pkcols) == 0 { // If the table doesn't have a PK, then we treat all columns as PK. - pkcols, err = vp.mysqld.GetColumns(vp.dbClient.DBName(), tplan.name) + pkcols, err = vp.mysqld.GetColumns(vp.dbClient.DBName(), tplan.Name) if err != nil { - return fmt.Errorf("error fetching pk columns for %s: %v", tplan.name, err) + return fmt.Errorf("error fetching pk columns for %s: %v", tplan.Name, err) } } for _, pkcol := range pkcols { found := false - for i, cExpr := range tplan.colExprs { - if cExpr.colname.EqualString(pkcol) { + for i, cExpr := range tplan.ColExprs { + if cExpr.ColName.EqualString(pkcol) { found = true - tplan.pkCols = append(tplan.pkCols, &colExpr{ - colname: cExpr.colname, - colnum: i, + tplan.PKCols = append(tplan.PKCols, &ColExpr{ + ColName: cExpr.ColName, + ColNum: i, }) break } } if !found { - return fmt.Errorf("primary key column %s missing from select list for table %s", pkcol, tplan.name) + return fmt.Errorf("primary key column %s missing from select list for table %s", pkcol, tplan.Name) } } vp.tplans[fieldEvent.TableName] = tplan @@ -424,14 +424,14 @@ func (vp *vplayer) applyRowEvent(ctx context.Context, rowEvent *binlogdatapb.Row return nil } -func (vp *vplayer) applyRowChange(ctx context.Context, tplan *tablePlan, rowChange *binlogdatapb.RowChange) error { +func (vp *vplayer) applyRowChange(ctx context.Context, tplan *TablePlan, rowChange *binlogdatapb.RowChange) error { // MakeRowTrusted is needed here because because Proto3ToResult is not convenient. var before, after []sqltypes.Value if rowChange.Before != nil { - before = sqltypes.MakeRowTrusted(tplan.fields, rowChange.Before) + before = sqltypes.MakeRowTrusted(tplan.Fields, rowChange.Before) } if rowChange.After != nil { - after = sqltypes.MakeRowTrusted(tplan.fields, rowChange.After) + after = sqltypes.MakeRowTrusted(tplan.Fields, rowChange.After) } var query string switch { @@ -450,113 +450,113 @@ func (vp *vplayer) applyRowChange(ctx context.Context, tplan *tablePlan, rowChan return vp.exec(ctx, query) } -func (vp *vplayer) generateInsert(tplan *tablePlan, after []sqltypes.Value) string { +func (vp *vplayer) generateInsert(tplan *TablePlan, after []sqltypes.Value) string { sql := sqlparser.NewTrackedBuffer(nil) - if tplan.onInsert == insertIgnore { - sql.Myprintf("insert ignore into %v set ", sqlparser.NewTableIdent(tplan.name)) + if tplan.OnInsert == InsertIgnore { + sql.Myprintf("insert ignore into %v set ", sqlparser.NewTableIdent(tplan.Name)) } else { - sql.Myprintf("insert into %v set ", sqlparser.NewTableIdent(tplan.name)) + sql.Myprintf("insert into %v set ", sqlparser.NewTableIdent(tplan.Name)) } vp.writeInsertValues(sql, tplan, after) - if tplan.onInsert == insertOndup { + if tplan.OnInsert == InsertOndup { sql.Myprintf(" on duplicate key update ") vp.writeUpdateValues(sql, tplan, nil, after) } return sql.String() } -func (vp *vplayer) generateUpdate(tplan *tablePlan, before, after []sqltypes.Value) string { - if tplan.onInsert == insertIgnore { +func (vp *vplayer) generateUpdate(tplan *TablePlan, before, after []sqltypes.Value) string { + if tplan.OnInsert == InsertIgnore { return vp.generateInsert(tplan, after) } sql := sqlparser.NewTrackedBuffer(nil) - sql.Myprintf("update %v set ", sqlparser.NewTableIdent(tplan.name)) + sql.Myprintf("update %v set ", sqlparser.NewTableIdent(tplan.Name)) vp.writeUpdateValues(sql, tplan, before, after) sql.Myprintf(" where ") vp.writeWhereValues(sql, tplan, before) return sql.String() } -func (vp *vplayer) generateDelete(tplan *tablePlan, before []sqltypes.Value) string { +func (vp *vplayer) generateDelete(tplan *TablePlan, before []sqltypes.Value) string { sql := sqlparser.NewTrackedBuffer(nil) - switch tplan.onInsert { - case insertOndup: - sql.Myprintf("update %v set ", sqlparser.NewTableIdent(tplan.name)) + switch tplan.OnInsert { + case InsertOndup: + sql.Myprintf("update %v set ", sqlparser.NewTableIdent(tplan.Name)) vp.writeUpdateValues(sql, tplan, before, nil) sql.Myprintf(" where ") vp.writeWhereValues(sql, tplan, before) - case insertIgnore: + case InsertIgnore: return "" default: // insertNormal - sql.Myprintf("delete from %v where ", sqlparser.NewTableIdent(tplan.name)) + sql.Myprintf("delete from %v where ", sqlparser.NewTableIdent(tplan.Name)) vp.writeWhereValues(sql, tplan, before) } return sql.String() } -func (vp *vplayer) writeInsertValues(sql *sqlparser.TrackedBuffer, tplan *tablePlan, after []sqltypes.Value) { +func (vp *vplayer) writeInsertValues(sql *sqlparser.TrackedBuffer, tplan *TablePlan, after []sqltypes.Value) { separator := "" - for _, cExpr := range tplan.colExprs { - sql.Myprintf("%s%v=", separator, cExpr.colname) + for _, cExpr := range tplan.ColExprs { + sql.Myprintf("%s%v=", separator, cExpr.ColName) if separator == "" { separator = ", " } - if cExpr.op == opCount { + if cExpr.Operation == OpCount { sql.WriteString("1") } else { - encodeValue(sql, after[cExpr.colnum]) + encodeValue(sql, after[cExpr.ColNum]) } } } -func (vp *vplayer) writeUpdateValues(sql *sqlparser.TrackedBuffer, tplan *tablePlan, before, after []sqltypes.Value) { +func (vp *vplayer) writeUpdateValues(sql *sqlparser.TrackedBuffer, tplan *TablePlan, before, after []sqltypes.Value) { separator := "" - for _, cExpr := range tplan.colExprs { - if cExpr.isGrouped { + for _, cExpr := range tplan.ColExprs { + if cExpr.IsGrouped { continue } - sql.Myprintf("%s%v=", separator, cExpr.colname) + sql.Myprintf("%s%v=", separator, cExpr.ColName) if separator == "" { separator = ", " } - if cExpr.op == opCount || cExpr.op == opSum { - sql.Myprintf("%v", cExpr.colname) + if cExpr.Operation == OpCount || cExpr.Operation == OpSum { + sql.Myprintf("%v", cExpr.ColName) } if len(before) != 0 { - switch cExpr.op { - case opNone: + switch cExpr.Operation { + case OpNone: if len(after) == 0 { sql.WriteString("NULL") } - case opCount: + case OpCount: sql.WriteString("-1") - case opSum: + case OpSum: sql.WriteString("-") - encodeValue(sql, before[cExpr.colnum]) + encodeValue(sql, before[cExpr.ColNum]) } } if len(after) != 0 { - switch cExpr.op { - case opNone: - encodeValue(sql, after[cExpr.colnum]) - case opCount: + switch cExpr.Operation { + case OpNone: + encodeValue(sql, after[cExpr.ColNum]) + case OpCount: sql.WriteString("+1") - case opSum: + case OpSum: sql.WriteString("+") - encodeValue(sql, after[cExpr.colnum]) + encodeValue(sql, after[cExpr.ColNum]) } } } } -func (vp *vplayer) writeWhereValues(sql *sqlparser.TrackedBuffer, tplan *tablePlan, before []sqltypes.Value) { +func (vp *vplayer) writeWhereValues(sql *sqlparser.TrackedBuffer, tplan *TablePlan, before []sqltypes.Value) { separator := "" - for _, cExpr := range tplan.pkCols { - sql.Myprintf("%s%v=", separator, cExpr.colname) + for _, cExpr := range tplan.PKCols { + sql.Myprintf("%s%v=", separator, cExpr.ColName) if separator == "" { separator = " and " } - encodeValue(sql, before[cExpr.colnum]) + encodeValue(sql, before[cExpr.ColNum]) } } From fab917f9ec8251a7fe8fc9111719d283bc302641 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Sat, 26 Jan 2019 04:18:43 -0800 Subject: [PATCH 20/27] vplayer: types test Signed-off-by: Sugu Sougoumarane --- .../vreplication/framework_test.go | 4 +- .../vreplication/vplayer_test.go | 95 +++++++++++++++++++ 2 files changed, 97 insertions(+), 2 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go index 9d44d314cb2..c3044b3bbcc 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go @@ -382,11 +382,11 @@ func expectData(t *testing.T, table string, values [][]string) { } for i, row := range values { if i >= len(qr.Rows) { - t.Errorf("Result too short, row: %d, want: %v", i, row) + t.Fatalf("Result too short, row: %d, want: %v", i, row) } for j, val := range row { if j >= len(qr.Rows[i]) { - t.Errorf("Too few columns, result: %v, row: %d, want: %v", qr.Rows[i], i, row) + t.Fatalf("Too few columns, result: %v, row: %d, want: %v", qr.Rows[i], i, row) } if got := qr.Rows[i][j].ToString(); got != val { t.Errorf("Mismatch at (%d, %d): %v, want %s", i, j, qr.Rows[i][j], val) diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go index d8be7ddf287..7b9e6c7c8f1 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go @@ -278,6 +278,101 @@ func TestPlayerFilters(t *testing.T) { } } +func TestPlayerTypes(t *testing.T) { + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + + execStatements(t, []string{ + "create table vitess_ints(tiny tinyint, tinyu tinyint unsigned, small smallint, smallu smallint unsigned, medium mediumint, mediumu mediumint unsigned, normal int, normalu int unsigned, big bigint, bigu bigint unsigned, y year, primary key(tiny))", + fmt.Sprintf("create table %s.vitess_ints(tiny tinyint, tinyu tinyint unsigned, small smallint, smallu smallint unsigned, medium mediumint, mediumu mediumint unsigned, normal int, normalu int unsigned, big bigint, bigu bigint unsigned, y year, primary key(tiny))", vrepldb), + "create table vitess_fracts(id int, deci decimal(5,2), num numeric(5,2), f float, d double, primary key(id))", + fmt.Sprintf("create table %s.vitess_fracts(id int, deci decimal(5,2), num numeric(5,2), f float, d double, primary key(id))", vrepldb), + "create table vitess_strings(vb varbinary(16), c char(16), vc varchar(16), b binary(4), tb tinyblob, bl blob, ttx tinytext, tx text, en enum('a','b'), s set('a','b'), primary key(vb))", + fmt.Sprintf("create table %s.vitess_strings(vb varbinary(16), c char(16), vc varchar(16), b binary(4), tb tinyblob, bl blob, ttx tinytext, tx text, en enum('a','b'), s set('a','b'), primary key(vb))", vrepldb), + "create table vitess_misc(id int, b bit(8), d date, dt datetime, t time, g geometry, primary key(id))", + fmt.Sprintf("create table %s.vitess_misc(id int, b bit(8), d date, dt datetime, t time, g geometry, primary key(id))", vrepldb), + "create table vitess_null(id int, val varbinary(128), primary key(id))", + fmt.Sprintf("create table %s.vitess_null(id int, val varbinary(128), primary key(id))", vrepldb), + "create table src1(id int, val varbinary(128), primary key(id))", + fmt.Sprintf("create table %s.src1(id int, val varbinary(128), primary key(id))", vrepldb), + }) + defer execStatements(t, []string{ + "drop table vitess_ints", + fmt.Sprintf("drop table %s.vitess_ints", vrepldb), + "drop table vitess_fracts", + fmt.Sprintf("drop table %s.vitess_fracts", vrepldb), + "drop table vitess_strings", + fmt.Sprintf("drop table %s.vitess_strings", vrepldb), + "drop table vitess_misc", + fmt.Sprintf("drop table %s.vitess_misc", vrepldb), + "drop table vitess_null", + fmt.Sprintf("drop table %s.vitess_null", vrepldb), + }) + env.SchemaEngine.Reload(context.Background()) + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "/.*", + }}, + } + cancel, _ := startVReplication(t, filter, binlogdatapb.OnDDLAction_IGNORE, "") + defer cancel() + testcases := []struct { + input string + output string + table string + data [][]string + }{{ + input: "insert into vitess_ints values(-128, 255, -32768, 65535, -8388608, 16777215, -2147483648, 4294967295, -9223372036854775808, 18446744073709551615, 2012)", + output: "insert into vitess_ints set tiny=-128, tinyu=255, small=-32768, smallu=65535, medium=-8388608, mediumu=16777215, normal=-2147483648, normalu=4294967295, big=-9223372036854775808, bigu=18446744073709551615, y=2012", + table: "vitess_ints", + data: [][]string{ + {"-128", "255", "-32768", "65535", "-8388608", "16777215", "-2147483648", "4294967295", "-9223372036854775808", "18446744073709551615", "2012"}, + }, + }, { + input: "insert into vitess_fracts values(1, 1.99, 2.99, 3.99, 4.99)", + output: "insert into vitess_fracts set id=1, deci=1.99, num=2.99, f=3.99E+00, d=4.99E+00", + table: "vitess_fracts", + data: [][]string{ + {"1", "1.99", "2.99", "3.99", "4.99"}, + }, + }, { + input: "insert into vitess_strings values('a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'a', 'a,b')", + output: "insert into vitess_strings set vb='a', c='b', vc='c', b='d', tb='e', bl='f', ttx='g', tx='h', en='1', s='3'", + table: "vitess_strings", + data: [][]string{ + {"a", "b", "c", "d\x00\x00\x00", "e", "f", "g", "h", "a", "a,b"}, + }, + }, { + input: "insert into vitess_misc values(1, '\x01', '2012-01-01', '2012-01-01 15:45:45', '15:45:45', point(1, 2))", + output: "insert into vitess_misc set id=1, b=b'00000001', d='2012-01-01', dt='2012-01-01 15:45:45', t='15:45:45', g='\\0\\0\\0\\0\x01\x01\\0\\0\\0\\0\\0\\0\\0\\0\\0\xf0?\\0\\0\\0\\0\\0\\0\\0@'", + table: "vitess_misc", + data: [][]string{ + {"1", "\x01", "2012-01-01", "2012-01-01 15:45:45", "15:45:45", "\x00\x00\x00\x00\x01\x01\x00\x00\x00\x00\x00\x00\x00\x00\x00\xf0?\x00\x00\x00\x00\x00\x00\x00@"}, + }, + }, { + input: "insert into vitess_null values(1, null)", + output: "insert into vitess_null set id=1, val=null", + table: "vitess_null", + data: [][]string{ + {"1", ""}, + }, + }} + + for _, tcases := range testcases { + execStatements(t, []string{tcases.input}) + want := []string{ + "begin", + tcases.output, + "/update _vt.vreplication set pos=", + "commit", + } + expectDBClientQueries(t, want) + if tcases.table != "" { + expectData(t, tcases.table, tcases.data) + } + } +} + func TestPlayerDDL(t *testing.T) { defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) execStatements(t, []string{ From d74e120530dde9899bc3bee987d3f33ed03c095c Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Sat, 26 Jan 2019 17:48:45 -0800 Subject: [PATCH 21/27] vplayer: fix for binary column data type Fixes #3984 Signed-off-by: Sugu Sougoumarane --- go/mysql/binlog_event_rbr.go | 16 ++++++++++++-- .../vreplication/vplayer_test.go | 21 ++++++++++++++++++- .../tabletserver/vstreamer/vstreamer_test.go | 4 ++-- 3 files changed, 36 insertions(+), 5 deletions(-) diff --git a/go/mysql/binlog_event_rbr.go b/go/mysql/binlog_event_rbr.go index 7e61f83b205..a8ec52dd05b 100644 --- a/go/mysql/binlog_event_rbr.go +++ b/go/mysql/binlog_event_rbr.go @@ -853,14 +853,26 @@ func CellValue(data []byte, pos int, typ byte, metadata uint16, styp querypb.Typ max := int((((metadata >> 4) & 0x300) ^ 0x300) + (metadata & 0xff)) // Length is encoded in 1 or 2 bytes. if max > 255 { + // This code path exists due to https://bugs.mysql.com/bug.php?id=37426. + // CHAR types need to allocate 3 bytes per char. So, the length for CHAR(255) + // cannot be represented in 1 byte. This also means that this rule does not + // apply to BINARY data. l := int(uint64(data[pos]) | uint64(data[pos+1])<<8) return sqltypes.MakeTrusted(querypb.Type_VARCHAR, data[pos+2:pos+2+l]), l + 2, nil } l := int(data[pos]) - return sqltypes.MakeTrusted(querypb.Type_VARCHAR, - data[pos+1:pos+1+l]), l + 1, nil + mdata := data[pos+1 : pos+1+l] + if sqltypes.IsBinary(styp) { + // Fixed length binaries have to be padded with zeroes + // up to the length of the field. Otherwise, equality checks + // fail against saved data. See https://github.com/vitessio/vitess/issues/3984. + ret := make([]byte, max) + copy(ret, mdata) + return sqltypes.MakeTrusted(querypb.Type_BINARY, ret), l + 1, nil + } + return sqltypes.MakeTrusted(querypb.Type_VARCHAR, mdata), l + 1, nil case TypeGeometry: l := 0 diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go index 7b9e6c7c8f1..660b0a1da96 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go @@ -294,6 +294,8 @@ func TestPlayerTypes(t *testing.T) { fmt.Sprintf("create table %s.vitess_null(id int, val varbinary(128), primary key(id))", vrepldb), "create table src1(id int, val varbinary(128), primary key(id))", fmt.Sprintf("create table %s.src1(id int, val varbinary(128), primary key(id))", vrepldb), + "create table binary_pk(b binary(4), val varbinary(4), primary key(b))", + fmt.Sprintf("create table %s.binary_pk(b binary(4), val varbinary(4), primary key(b))", vrepldb), }) defer execStatements(t, []string{ "drop table vitess_ints", @@ -306,6 +308,8 @@ func TestPlayerTypes(t *testing.T) { fmt.Sprintf("drop table %s.vitess_misc", vrepldb), "drop table vitess_null", fmt.Sprintf("drop table %s.vitess_null", vrepldb), + "drop table binary_pk", + fmt.Sprintf("drop table %s.binary_pk", vrepldb), }) env.SchemaEngine.Reload(context.Background()) @@ -337,7 +341,7 @@ func TestPlayerTypes(t *testing.T) { }, }, { input: "insert into vitess_strings values('a', 'b', 'c', 'd', 'e', 'f', 'g', 'h', 'a', 'a,b')", - output: "insert into vitess_strings set vb='a', c='b', vc='c', b='d', tb='e', bl='f', ttx='g', tx='h', en='1', s='3'", + output: "insert into vitess_strings set vb='a', c='b', vc='c', b='d\\0\\0\\0', tb='e', bl='f', ttx='g', tx='h', en='1', s='3'", table: "vitess_strings", data: [][]string{ {"a", "b", "c", "d\x00\x00\x00", "e", "f", "g", "h", "a", "a,b"}, @@ -356,6 +360,21 @@ func TestPlayerTypes(t *testing.T) { data: [][]string{ {"1", ""}, }, + }, { + input: "insert into binary_pk values('a', 'aaa')", + output: "insert into binary_pk set b='a\\0\\0\\0', val='aaa'", + table: "binary_pk", + data: [][]string{ + {"a\x00\x00\x00", "aaa"}, + }, + }, { + // 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 b='a\\0\\0\\0', val='bbb' where b='a\\0\\0\\0'", + table: "binary_pk", + data: [][]string{ + {"a\x00\x00\x00", "bbb"}, + }, }} for _, tcases := range testcases { diff --git a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go index 5d39189e9f2..517e19534ac 100644 --- a/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go +++ b/go/vt/vttablet/tabletserver/vstreamer/vstreamer_test.go @@ -696,8 +696,8 @@ func TestTypes(t *testing.T) { `fields: ` + `fields: ` + `fields: > `, - `type:ROW row_event: > > `, + `type:ROW row_event: > > `, `commit`, }}, }, { From a7f0dca59e042160bfd11ea5e768bc42b1ce4565 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Sat, 26 Jan 2019 20:33:30 -0800 Subject: [PATCH 22/27] vplayer: update only what has changed Signed-off-by: Sugu Sougoumarane --- .../vreplication/framework_test.go | 10 +- .../tabletmanager/vreplication/vplayer.go | 67 ++++++--- .../vreplication/vplayer_test.go | 142 ++++++++++++++++-- 3 files changed, 177 insertions(+), 42 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go index c3044b3bbcc..23c019bd638 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/framework_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/framework_test.go @@ -380,14 +380,14 @@ func expectData(t *testing.T, table string, values [][]string) { t.Error(err) return } + if len(values) != len(qr.Rows) { + t.Fatalf("row counts don't match: %v, want %v", qr.Rows, values) + } for i, row := range values { - if i >= len(qr.Rows) { - t.Fatalf("Result too short, row: %d, want: %v", i, row) + if len(row) != len(qr.Rows[i]) { + t.Fatalf("Too few columns, result: %v, row: %d, want: %v", qr.Rows[i], i, row) } for j, val := range row { - if j >= len(qr.Rows[i]) { - t.Fatalf("Too few columns, result: %v, row: %d, want: %v", qr.Rows[i], i, row) - } if got := qr.Rows[i][j].ToString(); got != val { t.Errorf("Mismatch at (%d, %d): %v, want %s", i, j, qr.Rows[i][j], val) } diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index 4315c10c8cc..f49c2b16583 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -460,7 +460,7 @@ func (vp *vplayer) generateInsert(tplan *TablePlan, after []sqltypes.Value) stri vp.writeInsertValues(sql, tplan, after) if tplan.OnInsert == InsertOndup { sql.Myprintf(" on duplicate key update ") - vp.writeUpdateValues(sql, tplan, nil, after) + _ = vp.writeUpdateValues(sql, tplan, nil, after) } return sql.String() } @@ -471,7 +471,9 @@ func (vp *vplayer) generateUpdate(tplan *TablePlan, before, after []sqltypes.Val } sql := sqlparser.NewTrackedBuffer(nil) sql.Myprintf("update %v set ", sqlparser.NewTableIdent(tplan.Name)) - vp.writeUpdateValues(sql, tplan, before, after) + if ok := vp.writeUpdateValues(sql, tplan, before, after); !ok { + return "" + } sql.Myprintf(" where ") vp.writeWhereValues(sql, tplan, before) return sql.String() @@ -481,10 +483,7 @@ func (vp *vplayer) generateDelete(tplan *TablePlan, before []sqltypes.Value) str sql := sqlparser.NewTrackedBuffer(nil) switch tplan.OnInsert { case InsertOndup: - sql.Myprintf("update %v set ", sqlparser.NewTableIdent(tplan.Name)) - vp.writeUpdateValues(sql, tplan, before, nil) - sql.Myprintf(" where ") - vp.writeWhereValues(sql, tplan, before) + return vp.generateUpdate(tplan, before, nil) case InsertIgnore: return "" default: // insertNormal @@ -498,27 +497,50 @@ func (vp *vplayer) writeInsertValues(sql *sqlparser.TrackedBuffer, tplan *TableP separator := "" for _, cExpr := range tplan.ColExprs { sql.Myprintf("%s%v=", separator, cExpr.ColName) - if separator == "" { - separator = ", " - } + separator = ", " if cExpr.Operation == OpCount { sql.WriteString("1") } else { - encodeValue(sql, after[cExpr.ColNum]) + if cExpr.Operation == OpSum && after[cExpr.ColNum].IsNull() { + sql.WriteString("0") + } else { + encodeValue(sql, after[cExpr.ColNum]) + } } } } -func (vp *vplayer) writeUpdateValues(sql *sqlparser.TrackedBuffer, tplan *TablePlan, before, after []sqltypes.Value) { +// writeUpdateValues returns true if at least one value was set. Otherwise, it returns false. +func (vp *vplayer) writeUpdateValues(sql *sqlparser.TrackedBuffer, tplan *TablePlan, before, after []sqltypes.Value) bool { separator := "" + hasSet := false for _, cExpr := range tplan.ColExprs { if cExpr.IsGrouped { continue } - sql.Myprintf("%s%v=", separator, cExpr.ColName) - if separator == "" { - separator = ", " + if len(before) != 0 && len(after) != 0 { + if cExpr.Operation == OpCount { + continue + } + bef := before[cExpr.ColNum] + aft := after[cExpr.ColNum] + // If both are null, there's no change + if bef.IsNull() && aft.IsNull() { + continue + } + // If any one of them is null, something has changed. + if bef.IsNull() || aft.IsNull() { + goto mustSet + } + // Compare content only if none are null. + if bef.ToString() == aft.ToString() { + continue + } } + mustSet: + sql.Myprintf("%s%v=", separator, cExpr.ColName) + separator = ", " + hasSet = true if cExpr.Operation == OpCount || cExpr.Operation == OpSum { sql.Myprintf("%v", cExpr.ColName) } @@ -531,8 +553,10 @@ func (vp *vplayer) writeUpdateValues(sql *sqlparser.TrackedBuffer, tplan *TableP case OpCount: sql.WriteString("-1") case OpSum: - sql.WriteString("-") - encodeValue(sql, before[cExpr.ColNum]) + if !before[cExpr.ColNum].IsNull() { + sql.WriteString("-") + encodeValue(sql, before[cExpr.ColNum]) + } } } if len(after) != 0 { @@ -542,20 +566,21 @@ func (vp *vplayer) writeUpdateValues(sql *sqlparser.TrackedBuffer, tplan *TableP case OpCount: sql.WriteString("+1") case OpSum: - sql.WriteString("+") - encodeValue(sql, after[cExpr.ColNum]) + if !after[cExpr.ColNum].IsNull() { + sql.WriteString("+") + encodeValue(sql, after[cExpr.ColNum]) + } } } } + return hasSet } func (vp *vplayer) writeWhereValues(sql *sqlparser.TrackedBuffer, tplan *TablePlan, before []sqltypes.Value) { separator := "" for _, cExpr := range tplan.PKCols { sql.Myprintf("%s%v=", separator, cExpr.ColName) - if separator == "" { - separator = " and " - } + separator = " and " encodeValue(sql, before[cExpr.ColNum]) } } diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go index 660b0a1da96..1996015d82a 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer_test.go @@ -24,9 +24,10 @@ import ( "time" "golang.org/x/net/context" - "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/mysql" "vitess.io/vitess/go/vt/binlog/binlogplayer" + binlogdatapb "vitess.io/vitess/go/vt/proto/binlogdata" topodatapb "vitess.io/vitess/go/vt/proto/topodata" ) @@ -104,7 +105,7 @@ func TestPlayerFilters(t *testing.T) { input: "update src1 set val='bbb'", output: []string{ "begin", - "update dst1 set id=1, val='bbb' where id=1", + "update dst1 set val='bbb' where id=1", "/update _vt.vreplication set pos=", "commit", }, @@ -134,20 +135,20 @@ func TestPlayerFilters(t *testing.T) { }, table: "dst2", data: [][]string{ - {"1", "2", "3"}, + {"1", "2", "3", "1"}, }, }, { // update with insertOnDup input: "update src2 set val1=5, val2=1 where id=1", output: []string{ "begin", - "update dst2 set val1=5, sval2=sval2-3+1, rcount=rcount-1+1 where id=1", + "update dst2 set val1=5, sval2=sval2-3+1 where id=1", "/update _vt.vreplication set pos=", "commit", }, table: "dst2", data: [][]string{ - {"1", "5", "1"}, + {"1", "5", "1", "1"}, }, }, { // delete with insertOnDup @@ -160,7 +161,7 @@ func TestPlayerFilters(t *testing.T) { }, table: "dst2", data: [][]string{ - {"1", "", "0"}, + {"1", "", "0", "0"}, }, }, { // insert with insertIgnore @@ -218,7 +219,7 @@ func TestPlayerFilters(t *testing.T) { input: "update yes set val='bbb'", output: []string{ "begin", - "update yes set id=1, val='bbb' where id=1", + "update yes set val='bbb' where id=1", "/update _vt.vreplication set pos=", "commit", }, @@ -248,7 +249,7 @@ func TestPlayerFilters(t *testing.T) { input: "update nopk set val='bbb' where id=1", output: []string{ "begin", - "update nopk set id=1, val='bbb' where id=1 and val='aaa'", + "update nopk set val='bbb' where id=1 and val='aaa'", "/update _vt.vreplication set pos=", "commit", }, @@ -278,6 +279,115 @@ func TestPlayerFilters(t *testing.T) { } } +func TestPlayerUpdates(t *testing.T) { + defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) + + execStatements(t, []string{ + "create table t1(id int, grouped int, ungrouped int, summed int, primary key(id))", + fmt.Sprintf("create table %s.t1(id int, grouped int, ungrouped int, summed int, rcount int, primary key(id))", vrepldb), + }) + defer execStatements(t, []string{ + "drop table t1", + fmt.Sprintf("drop table %s.t1", vrepldb), + }) + env.SchemaEngine.Reload(context.Background()) + + filter := &binlogdatapb.Filter{ + Rules: []*binlogdatapb.Rule{{ + Match: "t1", + Filter: "select id, grouped, ungrouped, sum(summed) as summed, count(*) as rcount from t1 group by id, grouped", + }}, + } + cancel, _ := startVReplication(t, filter, binlogdatapb.OnDDLAction_IGNORE, "") + defer cancel() + + testcases := []struct { + input string + output string + table string + data [][]string + }{{ + // Start with all nulls + input: "insert into t1 values(1, null, null, null)", + output: "insert into t1 set id=1, grouped=null, ungrouped=null, summed=0, rcount=1 on duplicate key update ungrouped=null, summed=summed, rcount=rcount+1", + table: "t1", + data: [][]string{ + {"1", "", "", "0", "1"}, + }, + }, { + // null to null values + input: "update t1 set grouped=1 where id=1", + output: "", + table: "t1", + data: [][]string{ + {"1", "", "", "0", "1"}, + }, + }, { + // null to non-null values + input: "update t1 set ungrouped=1, summed=1 where id=1", + output: "update t1 set ungrouped=1, summed=summed+1 where id=1", + table: "t1", + data: [][]string{ + {"1", "", "1", "1", "1"}, + }, + }, { + // non-null to non-null values + input: "update t1 set ungrouped=2, summed=2 where id=1", + output: "update t1 set ungrouped=2, summed=summed-1+2 where id=1", + table: "t1", + data: [][]string{ + {"1", "", "2", "2", "1"}, + }, + }, { + // non-null to null values + input: "update t1 set ungrouped=null, summed=null where id=1", + output: "update t1 set ungrouped=null, summed=summed-2 where id=1", + table: "t1", + data: [][]string{ + {"1", "", "", "0", "1"}, + }, + }, { + // insert non-null values + input: "insert into t1 values(2, 2, 3, 4)", + output: "insert into t1 set id=2, grouped=2, ungrouped=3, summed=4, rcount=1 on duplicate key update ungrouped=3, summed=summed+4, rcount=rcount+1", + table: "t1", + data: [][]string{ + {"1", "", "", "0", "1"}, + {"2", "2", "3", "4", "1"}, + }, + }, { + // delete non-null values + input: "delete from t1 where id=2", + output: "update t1 set ungrouped=NULL, summed=summed-4, rcount=rcount-1 where id=2", + table: "t1", + data: [][]string{ + {"1", "", "", "0", "1"}, + {"2", "2", "", "0", "0"}, + }, + }} + + for _, tcases := range testcases { + execStatements(t, []string{tcases.input}) + output := []string{ + "begin", + tcases.output, + "/update _vt.vreplication set pos=", + "commit", + } + if tcases.output == "" { + output = []string{ + "begin", + "/update _vt.vreplication set pos=", + "commit", + } + } + expectDBClientQueries(t, output) + if tcases.table != "" { + expectData(t, tcases.table, tcases.data) + } + } +} + func TestPlayerTypes(t *testing.T) { defer deleteTablet(addTablet(100, "0", topodatapb.TabletType_REPLICA, true, true)) @@ -370,7 +480,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 b='a\\0\\0\\0', val='bbb' where b='a\\0\\0\\0'", + output: "update binary_pk set val='bbb' where b='a\\0\\0\\0'", table: "binary_pk", data: [][]string{ {"a\x00\x00\x00", "bbb"}, @@ -740,8 +850,8 @@ func TestPlayerLockErrors(t *testing.T) { // The innodb lock wait timeout is set to 1s. expectDBClientQueries(t, []string{ "begin", - "update t1 set id=1, val='ccc' where id=1", - "update t1 set id=2, val='ccc' where id=2", + "update t1 set val='ccc' where id=1", + "update t1 set val='ccc' where id=2", "rollback", }) @@ -749,8 +859,8 @@ func TestPlayerLockErrors(t *testing.T) { _, _ = vconn.ExecuteFetch("rollback", 1) expectDBClientQueries(t, []string{ "begin", - "update t1 set id=1, val='ccc' where id=1", - "update t1 set id=2, val='ccc' where id=2", + "update t1 set val='ccc' where id=1", + "update t1 set val='ccc' where id=2", "/update _vt.vreplication set pos=", "commit", }) @@ -811,7 +921,7 @@ func TestPlayerCancelOnLock(t *testing.T) { // The innodb lock wait timeout is set to 1s. expectDBClientQueries(t, []string{ "begin", - "update t1 set id=1, val='ccc' where id=1", + "update t1 set val='ccc' where id=1", "rollback", }) @@ -896,7 +1006,7 @@ func TestPlayerBatching(t *testing.T) { // transactions must be batched into one. But the // DDLs should be on their own. expectDBClientQueries(t, []string{ - "update t1 set id=1, val='ccc' where id=1", + "update t1 set val='ccc' where id=1", "/update _vt.vreplication set pos=", "commit", "begin", @@ -993,7 +1103,7 @@ func TestPlayerRelayLogMaxSize(t *testing.T) { // will wait to be sent to the relay until the player fetches // them. expectDBClientQueries(t, []string{ - "update t1 set id=1, val='ccc' where id=1", + "update t1 set val='ccc' where id=1", "/update _vt.vreplication set pos=", "commit", "begin", From 20d7100c5aa37dd9418409c042eb4fb66592ef28 Mon Sep 17 00:00:00 2001 From: xichengliudui Date: Wed, 30 Jan 2019 06:49:48 -0500 Subject: [PATCH 23/27] update .go and .py files Signed-off-by: xichengliudui --- go/vt/vttablet/tabletserver/connpool/dbconn.go | 2 +- test/base_sharding.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/go/vt/vttablet/tabletserver/connpool/dbconn.go b/go/vt/vttablet/tabletserver/connpool/dbconn.go index 2a189676a44..abeef3e68b2 100644 --- a/go/vt/vttablet/tabletserver/connpool/dbconn.go +++ b/go/vt/vttablet/tabletserver/connpool/dbconn.go @@ -37,7 +37,7 @@ import ( querypb "vitess.io/vitess/go/vt/proto/query" ) -// BinlogFormat is used for for specifying the binlog format. +// BinlogFormat is used for specifying the binlog format. type BinlogFormat int // The following constants specify the possible binlog format values. diff --git a/test/base_sharding.py b/test/base_sharding.py index 3fcc92c1fb1..acc6d3a6f3d 100644 --- a/test/base_sharding.py +++ b/test/base_sharding.py @@ -383,7 +383,7 @@ def check_throttler_service_configuration(self, throttler_server, names): auto_log=True, trap_output=True) for name in names: # The max should be set and have a non-zero value. - # We test only the the first field 'target_replication_lag_sec'. + # We test only the first field 'target_replication_lag_sec'. self.assertIn('| %s | target_replication_lag_sec:12345 ' % (name), stdout) # protobuf omits fields with a zero value in the text output. self.assertNotIn('ignore_n_slowest_replicas', stdout) From dee0fe35d9075158e8f413c89391282c305b2ff7 Mon Sep 17 00:00:00 2001 From: Sugu Sougoumarane Date: Wed, 30 Jan 2019 05:39:04 -0800 Subject: [PATCH 24/27] vplayer: address review comments Signed-off-by: Sugu Sougoumarane --- go/vt/vttablet/tabletmanager/vreplication/vplayer.go | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go index f49c2b16583..fd4bc4746e9 100644 --- a/go/vt/vttablet/tabletmanager/vreplication/vplayer.go +++ b/go/vt/vttablet/tabletmanager/vreplication/vplayer.go @@ -197,6 +197,15 @@ func (vp *vplayer) applyEvents(ctx context.Context, relay *relayLog) error { if err != nil { return err } + // Filtered replication often ends up receiving a large number of empty transactions. + // This is required because the player needs to know the latest position of the source. + // This allows it to stop at that position if requested. + // This position also needs to be saved, which will allow an external request + // to check if a required position has been reached. + // However, this leads to a large number of empty commits which not only slow + // down the replay, but also generate binlog bloat on the target. + // In order to mitigate this problem, empty transactions are saved at most + // once every idleTimeout. // This covers two situations: // 1. Fetch was idle for idleTimeout. // 2. We've been receiving empty events for longer than idleTimeout. From 16284e34f1b0fbc97b50f426aec5334f6c0d3a37 Mon Sep 17 00:00:00 2001 From: deepthi Date: Wed, 30 Jan 2019 14:41:55 -0800 Subject: [PATCH 25/27] Revert "change ClusterAlias in _vt.local_metadata to be keyspace/shard instead of keyspace.shard" This reverts commit 8ea39c31a45ff97385247b32535ca181b0e2e5e2. Signed-off-by: deepthi --- go/vt/vttablet/tabletmanager/restore.go | 2 +- test/backup.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/go/vt/vttablet/tabletmanager/restore.go b/go/vt/vttablet/tabletmanager/restore.go index d61bfff2d85..55a3a57285d 100644 --- a/go/vt/vttablet/tabletmanager/restore.go +++ b/go/vt/vttablet/tabletmanager/restore.go @@ -189,7 +189,7 @@ func (agent *ActionAgent) getLocalMetadataValues(tabletType topodatapb.TabletTyp tablet := agent.Tablet() values := map[string]string{ "Alias": topoproto.TabletAliasString(tablet.Alias), - "ClusterAlias": fmt.Sprintf("%s/%s", tablet.Keyspace, tablet.Shard), + "ClusterAlias": fmt.Sprintf("%s.%s", tablet.Keyspace, tablet.Shard), "DataCenter": tablet.Alias.Cell, "PromotionRule": "must_not", } diff --git a/test/backup.py b/test/backup.py index 7de8b683851..78e8735fb9b 100755 --- a/test/backup.py +++ b/test/backup.py @@ -297,7 +297,7 @@ def _test_backup(self, tablet_type): for row in result: metadata[row[0]] = row[1] self.assertEqual(metadata['Alias'], 'test_nj-0000062346') - self.assertEqual(metadata['ClusterAlias'], 'test_keyspace/0') + self.assertEqual(metadata['ClusterAlias'], 'test_keyspace.0') self.assertEqual(metadata['DataCenter'], 'test_nj') if tablet_type == 'replica': self.assertEqual(metadata['PromotionRule'], 'neutral') From fd87a7ebe231f382fd240248a1d64ce65bc9ca98 Mon Sep 17 00:00:00 2001 From: deepthi Date: Wed, 30 Jan 2019 16:21:17 -0800 Subject: [PATCH 26/27] fix comments in tests Signed-off-by: deepthi --- go/vt/vtgate/executor_select_test.go | 4 ++-- go/vt/vtgate/executor_test.go | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/go/vt/vtgate/executor_select_test.go b/go/vt/vtgate/executor_select_test.go index e83090f1eec..997757330df 100644 --- a/go/vt/vtgate/executor_select_test.go +++ b/go/vt/vtgate/executor_select_test.go @@ -596,7 +596,7 @@ func TestSelectKeyRangeUnique(t *testing.T) { func TestSelectIN(t *testing.T) { executor, sbc1, sbc2, sbclookup := createExecutorEnv() - // Constant in IN clause, not a bind variable. + // Constant in IN clause is just a number, not a bind variable. _, err := executorExec(executor, "select id from user where id in (1)", nil) if err != nil { t.Error(err) @@ -614,7 +614,7 @@ func TestSelectIN(t *testing.T) { t.Errorf("sbc2.Queries: %+v, want nil\n", sbc2.Queries) } - // Constant IN is just a couple numbers, not bind variables. + // Constants in IN clause are just numbers, not bind variables. // They result in two different queries on two shards. sbc1.Queries = nil sbc2.Queries = nil diff --git a/go/vt/vtgate/executor_test.go b/go/vt/vtgate/executor_test.go index 5e3349db10f..8af068afeed 100644 --- a/go/vt/vtgate/executor_test.go +++ b/go/vt/vtgate/executor_test.go @@ -1841,7 +1841,7 @@ func TestExecutorUnrecognized(t *testing.T) { func TestExecutorMessageAckSharded(t *testing.T) { executor, sbc1, sbc2, _ := createExecutorEnv() - // Constant in IN clause, not a bind variable. + // Constant in IN clause is just a number, not a bind variable. ids := []*querypb.Value{{ Type: sqltypes.VarChar, Value: []byte("1"), @@ -1860,7 +1860,7 @@ func TestExecutorMessageAckSharded(t *testing.T) { t.Errorf("sbc2.MessageIDs: %+v, want nil\n", sbc2.MessageIDs) } - // Constant IN is just a couple numbers, not bind variables. + // Constants in IN clause are just numbers, not bind variables. // They result in two different MessageIDs on two shards. sbc1.MessageIDs = nil sbc2.MessageIDs = nil From 1c69d83d8d23303613cd042c18e5963a15ca8981 Mon Sep 17 00:00:00 2001 From: David Weitzman Date: Wed, 30 Jan 2019 17:15:04 -0800 Subject: [PATCH 27/27] Use a more specific "in use" message for the tx killer's rollback Someone who accidentally made a huge transaction asked what this error meant: "vttablet: rpc error: code = Aborted desc = transaction 1548379105148705713: in use: for rollback" It's less clear than the typical transaction killed error because the transaction was so large that rolling it back probably took some time. This diff changes the in use reason to "for tx killer rollback" which will hopefully be more clear. Signed-off-by: David Weitzman --- go/vt/vttablet/tabletserver/tx_pool.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/go/vt/vttablet/tabletserver/tx_pool.go b/go/vt/vttablet/tabletserver/tx_pool.go index 44b07e125ef..0c47f93f33c 100644 --- a/go/vt/vttablet/tabletserver/tx_pool.go +++ b/go/vt/vttablet/tabletserver/tx_pool.go @@ -178,7 +178,7 @@ func (axp *TxPool) RollbackNonBusy(ctx context.Context) { func (axp *TxPool) transactionKiller() { defer tabletenv.LogError() - for _, v := range axp.activePool.GetOutdated(time.Duration(axp.Timeout()), "for rollback") { + for _, v := range axp.activePool.GetOutdated(time.Duration(axp.Timeout()), "for tx killer rollback") { conn := v.(*TxConnection) log.Warningf("killing transaction (exceeded timeout: %v): %s", axp.Timeout(), conn.Format(nil)) tabletenv.KillStats.Add("Transactions", 1)