Merge pull request #2224 from sougou/2pc

2pc: commit failure handling
This commit is contained in:
sougou 2016-11-07 16:30:42 -08:00 коммит произвёл GitHub
Родитель 973b258381 323bcdfd04
Коммит 1f5ba0d1e4
12 изменённых файлов: 348 добавлений и 83 удалений

Просмотреть файл

@ -248,8 +248,12 @@ func (qe *QueryEngine) IsMySQLReachable() bool {
} }
// PrepareFromRedo replays and prepares the transactions // PrepareFromRedo replays and prepares the transactions
// from the redo log. This is called when a tablet becomes // from the redo log. It also loads previously failed transactions
// into the reserved list. This is called when a tablet becomes
// a master. // a master.
// TODO(sougou): Make this function set the lastId for tx pool to be
// greater than all those used by dtids. This will prevent dtid
// collisions.
func (qe *QueryEngine) PrepareFromRedo() error { func (qe *QueryEngine) PrepareFromRedo() error {
ctx := context.Background() ctx := context.Background()
var allErr concurrency.AllErrorRecorder var allErr concurrency.AllErrorRecorder
@ -258,13 +262,13 @@ func (qe *QueryEngine) PrepareFromRedo() error {
return err return err
} }
defer readConn.Recycle() defer readConn.Recycle()
transactions, err := qe.twoPC.ReadPrepared(ctx, readConn) prepared, failed, err := qe.twoPC.ReadAllRedo(ctx, readConn)
if err != nil { if err != nil {
return err return err
} }
outer: outer:
for dtid, tx := range transactions { for dtid, tx := range prepared {
conn, err := qe.txPool.LocalBegin(ctx) conn, err := qe.txPool.LocalBegin(ctx)
if err != nil { if err != nil {
allErr.RecordError(err) allErr.RecordError(err)
@ -287,6 +291,9 @@ outer:
continue continue
} }
} }
for _, dtid := range failed {
qe.preparedPool.SetFailed(dtid)
}
return allErr.Error() return allErr.Error()
} }
@ -301,7 +308,7 @@ func (qe *QueryEngine) RollbackTransactions() {
// this function. In case of any such change, this will // this function. In case of any such change, this will
// have to be revisited. // have to be revisited.
qe.txPool.RollbackNonBusy(ctx) qe.txPool.RollbackNonBusy(ctx)
for _, c := range qe.preparedPool.GetAll() { for _, c := range qe.preparedPool.FetchAll() {
qe.txPool.LocalConclude(ctx, c) qe.txPool.LocalConclude(ctx, c)
} }

Просмотреть файл

@ -1175,6 +1175,7 @@ func getQueryExecutorSupportedQueries() map[string]*sqltypes.Result {
sqlTurnoffBinlog: {}, sqlTurnoffBinlog: {},
fmt.Sprintf(sqlCreateSidecarDB, "_vt"): {}, fmt.Sprintf(sqlCreateSidecarDB, "_vt"): {},
fmt.Sprintf(sqlCreateTableRedoLogTransaction, "_vt"): {}, fmt.Sprintf(sqlCreateTableRedoLogTransaction, "_vt"): {},
fmt.Sprintf(sqlAlterTableRedoLogTransaction, "_vt"): {},
fmt.Sprintf(sqlCreateTableRedoLogStatement, "_vt"): {}, fmt.Sprintf(sqlCreateTableRedoLogStatement, "_vt"): {},
fmt.Sprintf(sqlCreateTableTransaction, "_vt"): {}, fmt.Sprintf(sqlCreateTableTransaction, "_vt"): {},
fmt.Sprintf(sqlCreateTableParticipant, "_vt"): {}, fmt.Sprintf(sqlCreateTableParticipant, "_vt"): {},
@ -1390,5 +1391,6 @@ func getQueryExecutorSupportedQueries() map[string]*sqltypes.Result {
}, },
}, },
}, },
fmt.Sprintf(sqlReadAllRedo, "_vt", "_vt"): {},
} }
} }

Просмотреть файл

@ -73,14 +73,13 @@ func NewQueryServiceStats(statsPrefix string, enablePublishStats bool) *QuerySer
resultBuckets := []int64{0, 1, 5, 10, 50, 100, 500, 1000, 5000, 10000} resultBuckets := []int64{0, 1, 5, 10, 50, 100, 500, 1000, 5000, 10000}
queryStats := stats.NewTimings(queryStatsName) queryStats := stats.NewTimings(queryStatsName)
return &QueryServiceStats{ return &QueryServiceStats{
MySQLStats: stats.NewTimings(mysqlStatsName), MySQLStats: stats.NewTimings(mysqlStatsName),
QueryStats: queryStats, QueryStats: queryStats,
WaitStats: stats.NewTimings(waitStatsName), WaitStats: stats.NewTimings(waitStatsName),
KillStats: stats.NewCounters(killStatsName, "Transactions", "Queries"), KillStats: stats.NewCounters(killStatsName, "Transactions", "Queries"),
InfoErrors: stats.NewCounters(infoErrorsName, "Retry", "Fatal", "DupKey"), InfoErrors: stats.NewCounters(infoErrorsName, "Retry", "Fatal", "DupKey"),
ErrorStats: stats.NewCounters(errorStatsName, "Fail", "TxPoolFull", "NotInTx", "Deadlock"), ErrorStats: stats.NewCounters(errorStatsName, "Fail", "TxPoolFull", "NotInTx", "Deadlock"),
InternalErrors: stats.NewCounters(internalErrorsName, "Task", InternalErrors: stats.NewCounters(internalErrorsName, "Task", "StrayTransactions", "Panic", "HungQuery", "Schema", "TwopcCommit", "TwopcResurrection"),
"Mismatch", "StrayTransactions", "Invalidation", "Panic", "HungQuery", "Schema"),
UserTableQueryCount: stats.NewMultiCounters( UserTableQueryCount: stats.NewMultiCounters(
userTableQueryCountName, []string{"TableName", "CallerID", "Type"}), userTableQueryCountName, []string{"TableName", "CallerID", "Type"}),
UserTableQueryTimesNs: stats.NewMultiCounters( UserTableQueryTimesNs: stats.NewMultiCounters(

Просмотреть файл

@ -412,7 +412,10 @@ func (tsv *TabletServer) serveNewType() (err error) {
if tsv.target.TabletType == topodatapb.TabletType_MASTER { if tsv.target.TabletType == topodatapb.TabletType_MASTER {
err = tsv.qe.PrepareFromRedo() err = tsv.qe.PrepareFromRedo()
if err != nil { if err != nil {
// TODO(sougou): raise alarms. // If this operation fails, we choose to raise an alert and
// continue anyway. Serving traffic is considered more important
// than blocking everything for the sake of a few transactions.
tsv.qe.queryServiceStats.InternalErrors.Add("TwopcResurrection", 1)
log.Errorf("Could not prepare transactions: %v", err) log.Errorf("Could not prepare transactions: %v", err)
} }
} else { } else {

Просмотреть файл

@ -692,16 +692,17 @@ func TestTabletServerReplicaToMaster(t *testing.T) {
tsv.SetServingType(topodatapb.TabletType_REPLICA, true, nil) tsv.SetServingType(topodatapb.TabletType_REPLICA, true, nil)
tpc := tsv.qe.twoPC tpc := tsv.qe.twoPC
db.AddQuery(tpc.readPrepared, &sqltypes.Result{}) db.AddQuery(tpc.readAllRedo, &sqltypes.Result{})
tsv.SetServingType(topodatapb.TabletType_MASTER, true, nil) tsv.SetServingType(topodatapb.TabletType_MASTER, true, nil)
if len(tsv.qe.preparedPool.conns) != 0 { if len(tsv.qe.preparedPool.conns) != 0 {
t.Errorf("len(tsv.qe.preparedPool.conns): %d, want 0", len(tsv.qe.preparedPool.conns)) t.Errorf("len(tsv.qe.preparedPool.conns): %d, want 0", len(tsv.qe.preparedPool.conns))
} }
tsv.SetServingType(topodatapb.TabletType_REPLICA, true, nil) tsv.SetServingType(topodatapb.TabletType_REPLICA, true, nil)
db.AddQuery(tpc.readPrepared, &sqltypes.Result{ db.AddQuery(tpc.readAllRedo, &sqltypes.Result{
Rows: [][]sqltypes.Value{{ Rows: [][]sqltypes.Value{{
sqltypes.MakeString([]byte("dtid0")), sqltypes.MakeString([]byte("dtid0")),
sqltypes.MakeString([]byte("Prepared")),
sqltypes.MakeString([]byte("")), sqltypes.MakeString([]byte("")),
sqltypes.MakeString([]byte("update test_table set name = 2 where pk in (1) /* _stream test_table (pk ) (1 ); */")), sqltypes.MakeString([]byte("update test_table set name = 2 where pk in (1) /* _stream test_table (pk ) (1 ); */")),
}}, }},
@ -718,15 +719,22 @@ func TestTabletServerReplicaToMaster(t *testing.T) {
tsv.SetServingType(topodatapb.TabletType_REPLICA, true, nil) tsv.SetServingType(topodatapb.TabletType_REPLICA, true, nil)
// Ensure we continue past errors. // Ensure we continue past errors.
db.AddQuery(tpc.readPrepared, &sqltypes.Result{ db.AddQuery(tpc.readAllRedo, &sqltypes.Result{
Rows: [][]sqltypes.Value{{ Rows: [][]sqltypes.Value{{
sqltypes.MakeString([]byte("bogus")), sqltypes.MakeString([]byte("bogus")),
sqltypes.MakeString([]byte("Prepared")),
sqltypes.MakeString([]byte("")), sqltypes.MakeString([]byte("")),
sqltypes.MakeString([]byte("bogus")), sqltypes.MakeString([]byte("bogus")),
}, { }, {
sqltypes.MakeString([]byte("dtid0")), sqltypes.MakeString([]byte("dtid0")),
sqltypes.MakeString([]byte("Prepared")),
sqltypes.MakeString([]byte("")), sqltypes.MakeString([]byte("")),
sqltypes.MakeString([]byte("update test_table set name = 2 where pk in (1) /* _stream test_table (pk ) (1 ); */")), sqltypes.MakeString([]byte("update test_table set name = 2 where pk in (1) /* _stream test_table (pk ) (1 ); */")),
}, {
sqltypes.MakeString([]byte("dtid1")),
sqltypes.MakeString([]byte("Failed")),
sqltypes.MakeString([]byte("")),
sqltypes.MakeString([]byte("unused")),
}}, }},
}) })
tsv.SetServingType(topodatapb.TabletType_MASTER, true, nil) tsv.SetServingType(topodatapb.TabletType_MASTER, true, nil)
@ -738,6 +746,10 @@ func TestTabletServerReplicaToMaster(t *testing.T) {
if !reflect.DeepEqual(got, want) { if !reflect.DeepEqual(got, want) {
t.Errorf("Prepared queries: %v, want %v", got, want) t.Errorf("Prepared queries: %v, want %v", got, want)
} }
wantFailed := map[string]error{"dtid1": errPrepFailed}
if !reflect.DeepEqual(tsv.qe.preparedPool.reserved, wantFailed) {
t.Errorf("Failed dtids: %v, want %v", tsv.qe.preparedPool.reserved, wantFailed)
}
tsv.SetServingType(topodatapb.TabletType_REPLICA, true, nil) tsv.SetServingType(topodatapb.TabletType_REPLICA, true, nil)
} }
@ -1671,6 +1683,7 @@ func getSupportedQueries() map[string]*sqltypes.Result {
sqlTurnoffBinlog: {}, sqlTurnoffBinlog: {},
fmt.Sprintf(sqlCreateSidecarDB, "_vt"): {}, fmt.Sprintf(sqlCreateSidecarDB, "_vt"): {},
fmt.Sprintf(sqlCreateTableRedoLogTransaction, "_vt"): {}, fmt.Sprintf(sqlCreateTableRedoLogTransaction, "_vt"): {},
fmt.Sprintf(sqlAlterTableRedoLogTransaction, "_vt"): {},
fmt.Sprintf(sqlCreateTableRedoLogStatement, "_vt"): {}, fmt.Sprintf(sqlCreateTableRedoLogStatement, "_vt"): {},
fmt.Sprintf(sqlCreateTableTransaction, "_vt"): {}, fmt.Sprintf(sqlCreateTableTransaction, "_vt"): {},
fmt.Sprintf(sqlCreateTableParticipant, "_vt"): {}, fmt.Sprintf(sqlCreateTableParticipant, "_vt"): {},
@ -1787,6 +1800,7 @@ func getSupportedQueries() map[string]*sqltypes.Result {
}, },
}, },
}, },
fmt.Sprintf(sqlReadAllRedo, "_vt", "_vt"): {},
} }
} }

Просмотреть файл

@ -36,6 +36,11 @@ const (
index state_time_idx(state, time_created) index state_time_idx(state, time_created)
) engine=InnoDB` ) engine=InnoDB`
// Due to possible legacy issues, do not modify the create
// tablet statement. Alter it instead.
sqlAlterTableRedoLogTransaction = `alter table ` + "`%s`" + `.redo_log_transaction
modify state enum('Prepared', 'Failed')`
sqlCreateTableRedoLogStatement = `create table if not exists ` + "`%s`" + `.redo_log_statement( sqlCreateTableRedoLogStatement = `create table if not exists ` + "`%s`" + `.redo_log_statement(
dtid varbinary(512), dtid varbinary(512),
id bigint, id bigint,
@ -58,15 +63,20 @@ const (
shard varchar(256), shard varchar(256),
primary key(dtid, id) primary key(dtid, id)
) engine=InnoDB` ) engine=InnoDB`
sqlReadAllRedo = `select t.dtid, t.state, s.id, s.statement from ` + "`%s`" + `.redo_log_transaction t
join ` + "`%s`" + `.redo_log_statement s on t.dtid = s.dtid
where t.state = 'Prepared' order by t.dtid, s.id`
) )
// TwoPC performs 2PC metadata management (MM) functions. // TwoPC performs 2PC metadata management (MM) functions.
type TwoPC struct { type TwoPC struct {
insertRedoTx *sqlparser.ParsedQuery insertRedoTx *sqlparser.ParsedQuery
insertRedoStmt *sqlparser.ParsedQuery insertRedoStmt *sqlparser.ParsedQuery
updateRedoTx *sqlparser.ParsedQuery
deleteRedoTx *sqlparser.ParsedQuery deleteRedoTx *sqlparser.ParsedQuery
deleteRedoStmt *sqlparser.ParsedQuery deleteRedoStmt *sqlparser.ParsedQuery
readPrepared string readAllRedo string
insertTransaction *sqlparser.ParsedQuery insertTransaction *sqlparser.ParsedQuery
insertParticipants *sqlparser.ParsedQuery insertParticipants *sqlparser.ParsedQuery
@ -94,6 +104,7 @@ func (tpc *TwoPC) Open(sidecarDBName string, dbaparams *sqldb.ConnParams) {
sqlTurnoffBinlog, sqlTurnoffBinlog,
fmt.Sprintf(sqlCreateSidecarDB, sidecarDBName), fmt.Sprintf(sqlCreateSidecarDB, sidecarDBName),
fmt.Sprintf(sqlCreateTableRedoLogTransaction, sidecarDBName), fmt.Sprintf(sqlCreateTableRedoLogTransaction, sidecarDBName),
fmt.Sprintf(sqlAlterTableRedoLogTransaction, sidecarDBName),
fmt.Sprintf(sqlCreateTableRedoLogStatement, sidecarDBName), fmt.Sprintf(sqlCreateTableRedoLogStatement, sidecarDBName),
fmt.Sprintf(sqlCreateTableTransaction, sidecarDBName), fmt.Sprintf(sqlCreateTableTransaction, sidecarDBName),
fmt.Sprintf(sqlCreateTableParticipant, sidecarDBName), fmt.Sprintf(sqlCreateTableParticipant, sidecarDBName),
@ -109,17 +120,16 @@ func (tpc *TwoPC) Open(sidecarDBName string, dbaparams *sqldb.ConnParams) {
tpc.insertRedoStmt = buildParsedQuery( tpc.insertRedoStmt = buildParsedQuery(
"insert into `%s`.redo_log_statement(dtid, id, statement) values %a", "insert into `%s`.redo_log_statement(dtid, id, statement) values %a",
sidecarDBName, ":vals") sidecarDBName, ":vals")
tpc.updateRedoTx = buildParsedQuery(
"update `%s`.redo_log_transaction set state = %a where dtid = %a",
sidecarDBName, ":state", ":dtid")
tpc.deleteRedoTx = buildParsedQuery( tpc.deleteRedoTx = buildParsedQuery(
"delete from `%s`.redo_log_transaction where dtid = %a", "delete from `%s`.redo_log_transaction where dtid = %a",
sidecarDBName, ":dtid") sidecarDBName, ":dtid")
tpc.deleteRedoStmt = buildParsedQuery( tpc.deleteRedoStmt = buildParsedQuery(
"delete from `%s`.redo_log_statement where dtid = %a", "delete from `%s`.redo_log_statement where dtid = %a",
sidecarDBName, ":dtid") sidecarDBName, ":dtid")
tpc.readPrepared = fmt.Sprintf( tpc.readAllRedo = fmt.Sprintf(sqlReadAllRedo, sidecarDBName, sidecarDBName)
"select s.dtid, s.id, s.statement from `%s`.redo_log_transaction t "+
"join `%s`.redo_log_statement s on t.dtid = s.dtid "+
"where t.state = 'Prepared' order by s.dtid, s.id",
sidecarDBName, sidecarDBName)
tpc.insertTransaction = buildParsedQuery( tpc.insertTransaction = buildParsedQuery(
"insert into `%s`.transaction(dtid, state, time_created, time_updated) values (%a, 'Prepare', %a, %a)", "insert into `%s`.transaction(dtid, state, time_created, time_updated) values (%a, 'Prepare', %a, %a)",
@ -180,6 +190,16 @@ func (tpc *TwoPC) SaveRedo(ctx context.Context, conn *TxConnection, dtid string,
return err return err
} }
// UpdateRedo changes the state of the redo log for the dtid.
func (tpc *TwoPC) UpdateRedo(ctx context.Context, conn *TxConnection, dtid, state string) error {
bindVars := map[string]interface{}{
"dtid": sqltypes.MakeTrusted(sqltypes.VarBinary, []byte(dtid)),
"state": sqltypes.MakeTrusted(sqltypes.VarBinary, []byte(state)),
}
_, err := tpc.exec(ctx, conn, tpc.updateRedoTx, bindVars)
return err
}
// DeleteRedo deletes the redo log for the dtid. // DeleteRedo deletes the redo log for the dtid.
func (tpc *TwoPC) DeleteRedo(ctx context.Context, conn *TxConnection, dtid string) error { func (tpc *TwoPC) DeleteRedo(ctx context.Context, conn *TxConnection, dtid string) error {
bindVars := map[string]interface{}{ bindVars := map[string]interface{}{
@ -193,32 +213,38 @@ func (tpc *TwoPC) DeleteRedo(ctx context.Context, conn *TxConnection, dtid strin
return err return err
} }
// ReadPrepared returns all the prepared transactions from the redo logs. // ReadAllRedo returns all the prepared transactions from the redo logs.
func (tpc *TwoPC) ReadPrepared(ctx context.Context, conn *DBConn) (map[string][]string, error) { func (tpc *TwoPC) ReadAllRedo(ctx context.Context, conn *DBConn) (prepared map[string][]string, failed []string, err error) {
qr, err := conn.Exec(ctx, tpc.readPrepared, 10000, false) qr, err := conn.Exec(ctx, tpc.readAllRedo, 10000, false)
if err != nil { if err != nil {
return nil, err return nil, nil, err
} }
transactions := make(map[string][]string)
var stmts []string // Do this as two loops for better readability.
var dtid string // Load prepared transactions.
for i, row := range qr.Rows { prepared = make(map[string][]string)
curdtid := row[0].String() for _, row := range qr.Rows {
if i == 0 { if row[1].String() != "Prepared" {
dtid = curdtid
}
if dtid == curdtid {
stmts = append(stmts, row[2].String())
continue continue
} }
transactions[dtid] = stmts dtid := row[0].String()
dtid = curdtid prepared[dtid] = append(prepared[dtid], row[3].String())
stmts = []string{row[2].String()}
} }
if stmts != nil {
transactions[dtid] = stmts // Load failed transactions.
lastdtid := ""
for _, row := range qr.Rows {
if row[1].String() != "Failed" {
continue
}
dtid := row[0].String()
if dtid == lastdtid {
continue
}
failed = append(failed, dtid)
lastdtid = dtid
} }
return transactions, nil return prepared, failed, nil
} }
// CreateTransaction saves the metadata of a 2pc transaction as Prepared. // CreateTransaction saves the metadata of a 2pc transaction as Prepared.

Просмотреть файл

@ -12,7 +12,7 @@ import (
"github.com/youtube/vitess/go/sqltypes" "github.com/youtube/vitess/go/sqltypes"
) )
func TestReadPrepared(t *testing.T) { func TestReadAllRedo(t *testing.T) {
// Reuse code from tx_executor_test. // Reuse code from tx_executor_test.
_, tsv, db := newTestTxExecutor() _, tsv, db := newTestTxExecutor()
defer tsv.StopService() defer tsv.StopService()
@ -25,68 +25,83 @@ func TestReadPrepared(t *testing.T) {
} }
defer conn.Recycle() defer conn.Recycle()
db.AddQuery(tpc.readPrepared, &sqltypes.Result{}) db.AddQuery(tpc.readAllRedo, &sqltypes.Result{})
got, err := tpc.ReadPrepared(ctx, conn) prepared, failed, err := tpc.ReadAllRedo(ctx, conn)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
want := map[string][]string{} want := map[string][]string{}
if !reflect.DeepEqual(got, want) { if !reflect.DeepEqual(prepared, want) {
t.Errorf("ReadPrepared: %#v, want %#v", got, want) t.Errorf("ReadAllRedo: %#v, want %#v", prepared, want)
}
if len(failed) != 0 {
t.Errorf("ReadAllRedo (failed): %v, must be empty", failed)
} }
db.AddQuery(tpc.readPrepared, &sqltypes.Result{ db.AddQuery(tpc.readAllRedo, &sqltypes.Result{
Rows: [][]sqltypes.Value{{ Rows: [][]sqltypes.Value{{
sqltypes.MakeString([]byte("dtid0")), sqltypes.MakeString([]byte("dtid0")),
sqltypes.MakeString([]byte("Prepared")),
sqltypes.MakeString([]byte("")), sqltypes.MakeString([]byte("")),
sqltypes.MakeString([]byte("stmt01")), sqltypes.MakeString([]byte("stmt01")),
}}, }},
}) })
got, err = tpc.ReadPrepared(ctx, conn) prepared, failed, err = tpc.ReadAllRedo(ctx, conn)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
want = map[string][]string{"dtid0": {"stmt01"}} want = map[string][]string{"dtid0": {"stmt01"}}
if !reflect.DeepEqual(got, want) { if !reflect.DeepEqual(prepared, want) {
t.Errorf("ReadPrepared: %#v, want %#v", got, want) t.Errorf("ReadAllRedo: %#v, want %#v", prepared, want)
}
if len(failed) != 0 {
t.Errorf("ReadAllRedo (failed): %v, must be empty", failed)
} }
db.AddQuery(tpc.readPrepared, &sqltypes.Result{ db.AddQuery(tpc.readAllRedo, &sqltypes.Result{
Rows: [][]sqltypes.Value{{ Rows: [][]sqltypes.Value{{
sqltypes.MakeString([]byte("dtid0")), sqltypes.MakeString([]byte("dtid0")),
sqltypes.MakeString([]byte("Prepared")),
sqltypes.MakeString([]byte("")), sqltypes.MakeString([]byte("")),
sqltypes.MakeString([]byte("stmt01")), sqltypes.MakeString([]byte("stmt01")),
}, { }, {
sqltypes.MakeString([]byte("dtid0")), sqltypes.MakeString([]byte("dtid0")),
sqltypes.MakeString([]byte("Prepared")),
sqltypes.MakeString([]byte("")), sqltypes.MakeString([]byte("")),
sqltypes.MakeString([]byte("stmt02")), sqltypes.MakeString([]byte("stmt02")),
}}, }},
}) })
got, err = tpc.ReadPrepared(ctx, conn) prepared, failed, err = tpc.ReadAllRedo(ctx, conn)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
want = map[string][]string{"dtid0": {"stmt01", "stmt02"}} want = map[string][]string{"dtid0": {"stmt01", "stmt02"}}
if !reflect.DeepEqual(got, want) { if !reflect.DeepEqual(prepared, want) {
t.Errorf("ReadPrepared: %#v, want %#v", got, want) t.Errorf("ReadAllRedo: %#v, want %#v", prepared, want)
}
if len(failed) != 0 {
t.Errorf("ReadAllRedo (failed): %v, must be empty", failed)
} }
db.AddQuery(tpc.readPrepared, &sqltypes.Result{ db.AddQuery(tpc.readAllRedo, &sqltypes.Result{
Rows: [][]sqltypes.Value{{ Rows: [][]sqltypes.Value{{
sqltypes.MakeString([]byte("dtid0")), sqltypes.MakeString([]byte("dtid0")),
sqltypes.MakeString([]byte("Prepared")),
sqltypes.MakeString([]byte("")), sqltypes.MakeString([]byte("")),
sqltypes.MakeString([]byte("stmt01")), sqltypes.MakeString([]byte("stmt01")),
}, { }, {
sqltypes.MakeString([]byte("dtid0")), sqltypes.MakeString([]byte("dtid0")),
sqltypes.MakeString([]byte("Prepared")),
sqltypes.MakeString([]byte("")), sqltypes.MakeString([]byte("")),
sqltypes.MakeString([]byte("stmt02")), sqltypes.MakeString([]byte("stmt02")),
}, { }, {
sqltypes.MakeString([]byte("dtid1")), sqltypes.MakeString([]byte("dtid1")),
sqltypes.MakeString([]byte("Prepared")),
sqltypes.MakeString([]byte("")), sqltypes.MakeString([]byte("")),
sqltypes.MakeString([]byte("stmt11")), sqltypes.MakeString([]byte("stmt11")),
}}, }},
}) })
got, err = tpc.ReadPrepared(ctx, conn) prepared, failed, err = tpc.ReadAllRedo(ctx, conn)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -94,7 +109,59 @@ func TestReadPrepared(t *testing.T) {
"dtid0": {"stmt01", "stmt02"}, "dtid0": {"stmt01", "stmt02"},
"dtid1": {"stmt11"}, "dtid1": {"stmt11"},
} }
if !reflect.DeepEqual(got, want) { if !reflect.DeepEqual(prepared, want) {
t.Errorf("ReadPrepared: %#v, want %#v", got, want) t.Errorf("ReadAllRedo: %#v, want %#v", prepared, want)
}
if len(failed) != 0 {
t.Errorf("ReadAllRedo (failed): %v, must be empty", failed)
}
db.AddQuery(tpc.readAllRedo, &sqltypes.Result{
Rows: [][]sqltypes.Value{{
sqltypes.MakeString([]byte("dtid0")),
sqltypes.MakeString([]byte("Prepared")),
sqltypes.MakeString([]byte("")),
sqltypes.MakeString([]byte("stmt01")),
}, {
sqltypes.MakeString([]byte("dtid0")),
sqltypes.MakeString([]byte("Prepared")),
sqltypes.MakeString([]byte("")),
sqltypes.MakeString([]byte("stmt02")),
}, {
sqltypes.MakeString([]byte("dtid1")),
sqltypes.MakeString([]byte("Failed")),
sqltypes.MakeString([]byte("")),
sqltypes.MakeString([]byte("stmt11")),
}, {
sqltypes.MakeString([]byte("dtid2")),
sqltypes.MakeString([]byte("Failed")),
sqltypes.MakeString([]byte("")),
sqltypes.MakeString([]byte("stmt21")),
}, {
sqltypes.MakeString([]byte("dtid2")),
sqltypes.MakeString([]byte("Failed")),
sqltypes.MakeString([]byte("")),
sqltypes.MakeString([]byte("stmt22")),
}, {
sqltypes.MakeString([]byte("dtid3")),
sqltypes.MakeString([]byte("Prepared")),
sqltypes.MakeString([]byte("")),
sqltypes.MakeString([]byte("stmt31")),
}},
})
prepared, failed, err = tpc.ReadAllRedo(ctx, conn)
if err != nil {
t.Fatal(err)
}
want = map[string][]string{
"dtid0": {"stmt01", "stmt02"},
"dtid3": {"stmt31"},
}
if !reflect.DeepEqual(prepared, want) {
t.Errorf("ReadAllRedo: %#v, want %#v", prepared, want)
}
wantFailed := []string{"dtid1", "dtid2"}
if !reflect.DeepEqual(failed, wantFailed) {
t.Errorf("ReadAllRedo failed): %#v, want %#v", failed, wantFailed)
} }
} }

Просмотреть файл

@ -9,6 +9,7 @@ import (
"golang.org/x/net/context" "golang.org/x/net/context"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/trace" "github.com/youtube/vitess/go/trace"
querypb "github.com/youtube/vitess/go/vt/proto/query" querypb "github.com/youtube/vitess/go/vt/proto/query"
@ -17,6 +18,7 @@ import (
// TxExecutor is used for executing a transactional request. // TxExecutor is used for executing a transactional request.
type TxExecutor struct { type TxExecutor struct {
// TODO(sougou): Parameterize this.
ctx context.Context ctx context.Context
logStats *LogStats logStats *LogStats
qe *QueryEngine qe *QueryEngine
@ -68,12 +70,13 @@ func (txe *TxExecutor) Prepare(transactionID int64, dtid string) error {
// CommitPrepared commits a prepared transaction. If the operation // CommitPrepared commits a prepared transaction. If the operation
// fails, an error counter is incremented and the transaction is // fails, an error counter is incremented and the transaction is
// marked as defunct in the redo log. If the marking fails, a // marked as failed in the redo log.
// different error counter is incremented indicating a more
// severe condition.
func (txe *TxExecutor) CommitPrepared(dtid string) error { func (txe *TxExecutor) CommitPrepared(dtid string) error {
defer txe.qe.queryServiceStats.QueryStats.Record("COMMIT_PREPARED", time.Now()) defer txe.qe.queryServiceStats.QueryStats.Record("COMMIT_PREPARED", time.Now())
conn := txe.qe.preparedPool.Get(dtid) conn, err := txe.qe.preparedPool.FetchForCommit(dtid)
if err != nil {
return NewTabletError(vtrpcpb.ErrorCode_BAD_INPUT, "cannot commit dtid %s, state: %v", dtid, err)
}
if conn == nil { if conn == nil {
return nil return nil
} }
@ -81,19 +84,47 @@ func (txe *TxExecutor) CommitPrepared(dtid string) error {
// even if the original context expires. // even if the original context expires.
ctx := trace.CopySpan(context.Background(), txe.ctx) ctx := trace.CopySpan(context.Background(), txe.ctx)
defer txe.qe.txPool.LocalConclude(ctx, conn) defer txe.qe.txPool.LocalConclude(ctx, conn)
err := txe.qe.twoPC.DeleteRedo(ctx, conn, dtid) err = txe.qe.twoPC.DeleteRedo(ctx, conn, dtid)
if err != nil { if err != nil {
// TODO(sougou): raise alarms & mark as defunct. txe.markFailed(ctx, dtid)
return err return err
} }
err = txe.qe.txPool.LocalCommit(ctx, conn) err = txe.qe.txPool.LocalCommit(ctx, conn)
if err != nil { if err != nil {
// TODO(sougou): raise alarms & mark as defunct. txe.markFailed(ctx, dtid)
return err return err
} }
txe.qe.preparedPool.Forget(dtid)
return nil return nil
} }
// markFailed does the necessary work to mark a CommitPrepared
// as failed. It marks the dtid as failed in the prepared pool,
// increments the InternalErros counter, and also changes the
// state of the transaction in the redo log as failed. If the
// state change does not succeed, it just logs the event.
// The function uses the passed in context that has no timeout
// instead of TxExecutor's context.
func (txe *TxExecutor) markFailed(ctx context.Context, dtid string) {
txe.qe.queryServiceStats.InternalErrors.Add("TwopcCommit", 1)
txe.qe.preparedPool.SetFailed(dtid)
conn, err := txe.qe.txPool.LocalBegin(ctx)
if err != nil {
log.Errorf("markFailed: Begin failed for dtid %s: %v", dtid, err)
return
}
defer txe.qe.txPool.LocalConclude(ctx, conn)
if err = txe.qe.twoPC.UpdateRedo(ctx, conn, dtid, "Failed"); err != nil {
log.Errorf("markFailed: UpdateRedo failed for dtid %s: %v", dtid, err)
return
}
if err = txe.qe.txPool.LocalCommit(ctx, conn); err != nil {
log.Errorf("markFailed: Commit failed for dtid %s: %v", dtid, err)
}
}
// RollbackPrepared rolls back a prepared transaction. This function handles // RollbackPrepared rolls back a prepared transaction. This function handles
// the case of an incomplete prepare. // the case of an incomplete prepare.
// //
@ -128,7 +159,7 @@ func (txe *TxExecutor) RollbackPrepared(dtid string, originalID int64) error {
err = txe.qe.txPool.LocalCommit(txe.ctx, conn) err = txe.qe.txPool.LocalCommit(txe.ctx, conn)
returnConn: returnConn:
if preparedConn := txe.qe.preparedPool.Get(dtid); preparedConn != nil { if preparedConn := txe.qe.preparedPool.FetchForRollback(dtid); preparedConn != nil {
txe.qe.txPool.LocalConclude(txe.ctx, preparedConn) txe.qe.txPool.LocalConclude(txe.ctx, preparedConn)
} }
if originalID != 0 { if originalID != 0 {

Просмотреть файл

@ -150,10 +150,17 @@ func TestTxExecutorCommitRedoFail(t *testing.T) {
t.Error(err) t.Error(err)
} }
defer txe.RollbackPrepared("bb", 0) defer txe.RollbackPrepared("bb", 0)
db.AddQuery("update `_vt`.redo_log_transaction set state = 'Failed' where dtid = 'bb'", &sqltypes.Result{})
err = txe.CommitPrepared("bb") err = txe.CommitPrepared("bb")
want := "is not supported" want := "is not supported"
if err == nil || !strings.Contains(err.Error(), want) { if err == nil || !strings.Contains(err.Error(), want) {
t.Errorf("Prepare err: %v, must contain %s", err, want) t.Errorf("txe.CommitPrepared err: %v, must contain %s", err, want)
}
// A retry should fail differently.
err = txe.CommitPrepared("bb")
want = "cannot commit dtid bb, state: failed"
if err == nil || !strings.Contains(err.Error(), want) {
t.Errorf("txe.CommitPrepared err: %v, must contain %s", err, want)
} }
} }

Просмотреть файл

@ -10,12 +10,18 @@ import (
"sync" "sync"
) )
var (
errPrepCommiting = errors.New("commiting")
errPrepFailed = errors.New("failed")
)
// TxPreparedPool manages connections for prepared transactions. // TxPreparedPool manages connections for prepared transactions.
// The Prepare functionality and associated orchestration // The Prepare functionality and associated orchestration
// is done by TxPool. // is done by TxPool.
type TxPreparedPool struct { type TxPreparedPool struct {
mu sync.Mutex mu sync.Mutex
conns map[string]*TxConnection conns map[string]*TxConnection
reserved map[string]error
capacity int capacity int
} }
@ -23,15 +29,19 @@ type TxPreparedPool struct {
func NewTxPreparedPool(capacity int) *TxPreparedPool { func NewTxPreparedPool(capacity int) *TxPreparedPool {
return &TxPreparedPool{ return &TxPreparedPool{
conns: make(map[string]*TxConnection, capacity), conns: make(map[string]*TxConnection, capacity),
reserved: make(map[string]error),
capacity: capacity, capacity: capacity,
} }
} }
// Put adds the connection to the pool. It returns an error // Put adds the connection to the pool. It returns an error
// if the pool is full, and panics on duplicate key. // if the pool is full or on duplicate key.
func (pp *TxPreparedPool) Put(c *TxConnection, dtid string) error { func (pp *TxPreparedPool) Put(c *TxConnection, dtid string) error {
pp.mu.Lock() pp.mu.Lock()
defer pp.mu.Unlock() defer pp.mu.Unlock()
if _, ok := pp.reserved[dtid]; ok {
return errors.New("duplicate DTID in Prepare: " + dtid)
}
if _, ok := pp.conns[dtid]; ok { if _, ok := pp.conns[dtid]; ok {
return errors.New("duplicate DTID in Prepare: " + dtid) return errors.New("duplicate DTID in Prepare: " + dtid)
} }
@ -42,18 +52,62 @@ func (pp *TxPreparedPool) Put(c *TxConnection, dtid string) error {
return nil return nil
} }
// Get returns the connection and removes it from the pool. // FetchForRollback returns the connection and removes it from the pool.
// If the connection is not found, it returns nil. // If the connection is not found, it returns nil. If the dtid
func (pp *TxPreparedPool) Get(dtid string) *TxConnection { // is in the reserved list, it means that an operator is trying
// to resolve a previously failed commit. So, it removes the entry
// and returns nil.
func (pp *TxPreparedPool) FetchForRollback(dtid string) *TxConnection {
pp.mu.Lock() pp.mu.Lock()
defer pp.mu.Unlock() defer pp.mu.Unlock()
if _, ok := pp.reserved[dtid]; ok {
delete(pp.reserved, dtid)
return nil
}
c := pp.conns[dtid] c := pp.conns[dtid]
delete(pp.conns, dtid) delete(pp.conns, dtid)
return c return c
} }
// GetAll removes all connections and returns them as a list. // FetchForCommit returns the connection for commit. Before returning,
func (pp *TxPreparedPool) GetAll() []*TxConnection { // it remembers the dtid in its reserved list as "committing". If
// the dtid is already in the reserved list, it returns an error.
// If the commit is successful, the dtid can be removed from the
// reserved list by calling Forget. If the commit failed, SetFailed
// must be called. This will inform future retries that the previous
// commit failed.
func (pp *TxPreparedPool) FetchForCommit(dtid string) (*TxConnection, error) {
pp.mu.Lock()
defer pp.mu.Unlock()
if err, ok := pp.reserved[dtid]; ok {
return nil, err
}
c, ok := pp.conns[dtid]
if ok {
delete(pp.conns, dtid)
pp.reserved[dtid] = errPrepCommiting
}
return c, nil
}
// SetFailed marks the reserved dtid as failed.
// If there was no previous entry, one is created.
func (pp *TxPreparedPool) SetFailed(dtid string) {
pp.mu.Lock()
defer pp.mu.Unlock()
pp.reserved[dtid] = errPrepFailed
}
// Forget removes the dtid from the reserved list.
func (pp *TxPreparedPool) Forget(dtid string) {
pp.mu.Lock()
defer pp.mu.Unlock()
delete(pp.reserved, dtid)
}
// FetchAll removes all connections and returns them as a list.
// It also forgets all reserved dtids.
func (pp *TxPreparedPool) FetchAll() []*TxConnection {
pp.mu.Lock() pp.mu.Lock()
defer pp.mu.Unlock() defer pp.mu.Unlock()
conns := make([]*TxConnection, 0, len(pp.conns)) conns := make([]*TxConnection, 0, len(pp.conns))
@ -61,5 +115,6 @@ func (pp *TxPreparedPool) GetAll() []*TxConnection {
conns = append(conns, c) conns = append(conns, c)
} }
pp.conns = make(map[string]*TxConnection, pp.capacity) pp.conns = make(map[string]*TxConnection, pp.capacity)
pp.reserved = make(map[string]error)
return conns return conns
} }

Просмотреть файл

@ -37,35 +37,91 @@ func TestPrepPut(t *testing.T) {
if err == nil || err.Error() != want { if err == nil || err.Error() != want {
t.Errorf("Put err: %v, want %s", err, want) t.Errorf("Put err: %v, want %s", err, want)
} }
_, err = pp.FetchForCommit("aa")
err = pp.Put(nil, "aa")
want = "duplicate DTID in Prepare: aa"
if err == nil || err.Error() != want {
t.Errorf("Put err: %v, want %s", err, want)
}
pp.Forget("aa")
err = pp.Put(nil, "aa")
if err != nil {
t.Error(err)
}
} }
func TestPrepGet(t *testing.T) { func TestPrepFetchForRollback(t *testing.T) {
pp := NewTxPreparedPool(2) pp := NewTxPreparedPool(2)
conn := &TxConnection{} conn := &TxConnection{}
pp.Put(conn, "aa") pp.Put(conn, "aa")
got := pp.Get("bb") got := pp.FetchForRollback("bb")
if got != nil { if got != nil {
t.Errorf("Get(bb): %v, want nil", got) t.Errorf("Get(bb): %v, want nil", got)
} }
got = pp.Get("aa") got = pp.FetchForRollback("aa")
if got != conn { if got != conn {
t.Errorf("pp.Get(aa): %p, want %p", got, conn) t.Errorf("pp.Get(aa): %p, want %p", got, conn)
} }
got = pp.Get("aa") got = pp.FetchForRollback("aa")
if got != nil { if got != nil {
t.Errorf("Get(aa): %v, want nil", got) t.Errorf("Get(aa): %v, want nil", got)
} }
} }
func TestPrepGetAll(t *testing.T) { func TestPrepFetchForCommit(t *testing.T) {
pp := NewTxPreparedPool(2)
conn := &TxConnection{}
got, err := pp.FetchForCommit("aa")
if err != nil {
t.Error(err)
}
if got != nil {
t.Errorf("Get(aa): %v, want nil", got)
}
pp.Put(conn, "aa")
got, err = pp.FetchForCommit("aa")
if err != nil {
t.Error(err)
}
if got != conn {
t.Errorf("pp.Get(aa): %p, want %p", got, conn)
}
got, err = pp.FetchForCommit("aa")
want := "commiting"
if err == nil || err.Error() != want {
t.Errorf("FetchForCommit err: %v, want %s", err, want)
}
pp.SetFailed("aa")
got, err = pp.FetchForCommit("aa")
want = "failed"
if err == nil || err.Error() != want {
t.Errorf("FetchForCommit err: %v, want %s", err, want)
}
pp.SetFailed("bb")
got, err = pp.FetchForCommit("bb")
want = "failed"
if err == nil || err.Error() != want {
t.Errorf("FetchForCommit err: %v, want %s", err, want)
}
pp.Forget("aa")
got, err = pp.FetchForCommit("aa")
if err != nil {
t.Error(err)
}
if got != nil {
t.Errorf("Get(aa): %v, want nil", got)
}
}
func TestPrepFetchAll(t *testing.T) {
pp := NewTxPreparedPool(2) pp := NewTxPreparedPool(2)
conn1 := &TxConnection{} conn1 := &TxConnection{}
conn2 := &TxConnection{} conn2 := &TxConnection{}
pp.Put(conn1, "aa") pp.Put(conn1, "aa")
pp.Put(conn2, "bb") pp.Put(conn2, "bb")
got := pp.GetAll() got := pp.FetchAll()
if len(got) != 2 { if len(got) != 2 {
t.Errorf("GetAll len: %d, want 2", len(got)) t.Errorf("FetchAll len: %d, want 2", len(got))
} }
if len(pp.conns) != 0 { if len(pp.conns) != 0 {
t.Errorf("len(pp.conns): %d, want 0", len(pp.conns)) t.Errorf("len(pp.conns): %d, want 0", len(pp.conns))

Просмотреть файл

@ -194,8 +194,6 @@ func (txc *TxConn) resumeCommit(ctx context.Context, target *querypb.Target, tra
} }
func (txc *TxConn) generateDTID(mmShard *vtgatepb.Session_ShardSession) string { func (txc *TxConn) generateDTID(mmShard *vtgatepb.Session_ShardSession) string {
// TODO(sougou): Change query_engine to start off transaction id counting
// above the highest number used by dtids. This will prevent collisions.
return fmt.Sprintf("%s:%s:0:%d", mmShard.Target.Keyspace, mmShard.Target.Shard, mmShard.TransactionId) return fmt.Sprintf("%s:%s:0:%d", mmShard.Target.Keyspace, mmShard.Target.Shard, mmShard.TransactionId)
} }