func (blp *BinlogPlayer) processTransaction(tx *binlogdatapb.BinlogTransaction) (ok bool, err error) { txnStartTime := time.Now() if err = blp.dbClient.Begin(); err != nil { return false, fmt.Errorf("failed query BEGIN, err: %s", err) } if err = blp.writeRecoveryPosition(tx); err != nil { return false, err } for i, stmt := range tx.Statements { // Make sure the statement is replayed in the proper charset. if dbClient, ok := blp.dbClient.(*DBClient); ok { var stmtCharset *binlogdatapb.Charset if stmt.Charset != nil { stmtCharset = stmt.Charset } else { // Streamer sends a nil Charset for statements that use the // charset we specified in the request. stmtCharset = blp.defaultCharset } if *blp.currentCharset != *stmtCharset { // In regular MySQL replication, the charset is silently adjusted as // needed during event playback. Here we also adjust so that playback // proceeds, but in Vitess-land this usually means a misconfigured // server or a misbehaving client, so we spam the logs with warnings. log.Warningf("BinlogPlayer changing charset from %v to %v for statement %d in transaction %v", blp.currentCharset, stmtCharset, i, *tx) err = sqldb.SetCharset(dbClient.dbConn, stmtCharset) if err != nil { return false, fmt.Errorf("can't set charset for statement %d in transaction %v: %v", i, *tx, err) } blp.currentCharset = stmtCharset } } if _, err = blp.exec(string(stmt.Sql)); err == nil { continue } if sqlErr, ok := err.(*sqldb.SQLError); ok && sqlErr.Number() == 1213 { // Deadlock: ask for retry log.Infof("Deadlock: %v", err) if err = blp.dbClient.Rollback(); err != nil { return false, err } return false, nil } return false, err } if err = blp.dbClient.Commit(); err != nil { return false, fmt.Errorf("failed query COMMIT, err: %s", err) } blp.blplStats.Timings.Record(BlplTransaction, txnStartTime) return true, nil }
// ApplyBinlogEvents makes an RPC request to BinlogServer // and processes the events. It will return nil if the provided context // was canceled, or if we reached the stopping point. // It will return io.EOF if the server stops sending us updates. // It may return any other error it encounters. func (blp *BinlogPlayer) ApplyBinlogEvents(ctx context.Context) error { // Instantiate the throttler based on the configuration stored in the db. maxTPS, maxReplicationLag, err := blp.readThrottlerSettings() if err != nil { log.Error(err) return err } t, err := throttler.NewThrottler( fmt.Sprintf("BinlogPlayer/%d", blp.uid), "transactions", 1 /* threadCount */, maxTPS, maxReplicationLag) if err != nil { err := fmt.Errorf("failed to instantiate throttler: %v", err) log.Error(err) return err } defer t.Close() // Log the mode of operation and when the player stops. if len(blp.tables) > 0 { log.Infof("BinlogPlayer client %v for tables %v starting @ '%v', server: %v", blp.uid, blp.tables, blp.position, blp.tablet, ) } else { log.Infof("BinlogPlayer client %v for keyrange '%v-%v' starting @ '%v', server: %v", blp.uid, hex.EncodeToString(blp.keyRange.Start), hex.EncodeToString(blp.keyRange.End), blp.position, blp.tablet, ) } if !blp.stopPosition.IsZero() { // We need to stop at some point. Sanity check the point. switch { case blp.position.Equal(blp.stopPosition): log.Infof("Not starting BinlogPlayer, we're already at the desired position %v", blp.stopPosition) return nil case blp.position.AtLeast(blp.stopPosition): return fmt.Errorf("starting point %v greater than stopping point %v", blp.position, blp.stopPosition) default: log.Infof("Will stop player when reaching %v", blp.stopPosition) } } clientFactory, ok := clientFactories[*binlogPlayerProtocol] if !ok { return fmt.Errorf("no binlog player client factory named %v", *binlogPlayerProtocol) } blplClient := clientFactory() err = blplClient.Dial(blp.tablet, *BinlogPlayerConnTimeout) if err != nil { err := fmt.Errorf("error dialing binlog server: %v", err) log.Error(err) return err } defer blplClient.Close() // Get the current charset of our connection, so we can ask the stream server // to check that they match. The streamer will also only send per-statement // charset data if that statement's charset is different from what we specify. if dbClient, ok := blp.dbClient.(*DBClient); ok { blp.defaultCharset, err = sqldb.GetCharset(dbClient.dbConn) if err != nil { return fmt.Errorf("can't get charset to request binlog stream: %v", err) } log.Infof("original charset: %v", blp.defaultCharset) blp.currentCharset = blp.defaultCharset // Restore original charset when we're done. defer func() { // If the connection has been closed, there's no need to restore // this connection-specific setting. if dbClient.dbConn == nil { return } log.Infof("restoring original charset %v", blp.defaultCharset) if csErr := sqldb.SetCharset(dbClient.dbConn, blp.defaultCharset); csErr != nil { log.Errorf("can't restore original charset %v: %v", blp.defaultCharset, csErr) } }() } var stream BinlogTransactionStream if len(blp.tables) > 0 { stream, err = blplClient.StreamTables(ctx, replication.EncodePosition(blp.position), blp.tables, blp.defaultCharset) } else { stream, err = blplClient.StreamKeyRange(ctx, replication.EncodePosition(blp.position), blp.keyRange, blp.defaultCharset) } if err != nil { err := fmt.Errorf("error sending streaming query to binlog server: %v", err) log.Error(err) return err } for { // Block if we are throttled. for { backoff := t.Throttle(0 /* threadID */) if backoff == throttler.NotThrottled { break } // We don't bother checking for context cancellation here because the // sleep will block only up to 1 second. (Usually, backoff is 1s / rate // e.g. a rate of 1000 TPS results into a backoff of 1 ms.) time.Sleep(backoff) } // get the response response, err := stream.Recv() if err != nil { switch err { case context.Canceled: return nil default: // if the context is canceled, we // return nil (some RPC // implementations will remap the // context error to their own errors) select { case <-ctx.Done(): if ctx.Err() == context.Canceled { return nil } default: } return fmt.Errorf("Error received from Stream %v", err) } } // process the transaction for { ok, err = blp.processTransaction(response) if err != nil { return fmt.Errorf("Error in processing binlog event %v", err) } if ok { if !blp.stopPosition.IsZero() { if blp.position.AtLeast(blp.stopPosition) { log.Infof("Reached stopping position, done playing logs") return nil } } break } log.Infof("Retrying txn") time.Sleep(1 * time.Second) } } }