Add tracing in rpcplus and other places downstream from tabletmanager.

The goal is to start getting useful traces out of tabletmanager.
To start, we trace outgoing RPC calls on the client side.
Certain high-level TopoServer operations are also traced at this point.

To facilitate tracing, I've begun plumbing Context through tabletmanager
and all its users. At this time, it is simply a few regexes adding the
parameter so it can be used by tracing; no timeouts or cancellation are
using Context yet. Those use cases will require more careful
consideration.

Other than tabletmanager, most callers use context.TODO() since they
aren't plumbed yet.
This commit is contained in:
Anthony Yeh 2014-11-11 22:56:57 -08:00
Родитель e797fc38e3
Коммит b09ca93ea3
55 изменённых файлов: 814 добавлений и 643 удалений

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

@ -13,6 +13,8 @@ import (
"strings" "strings"
"time" "time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
"github.com/youtube/vitess/go/acl" "github.com/youtube/vitess/go/acl"
"github.com/youtube/vitess/go/vt/logutil" "github.com/youtube/vitess/go/vt/logutil"
@ -402,7 +404,7 @@ func main() {
if err != nil { if err != nil {
return "", err return "", err
} }
return "", wr.TabletManagerClient().Ping(ti, 10*time.Second) return "", wr.TabletManagerClient().Ping(context.TODO(), ti, 10*time.Second)
}) })
actionRepo.RegisterTabletAction("ScrapTablet", acl.ADMIN, actionRepo.RegisterTabletAction("ScrapTablet", acl.ADMIN,

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

@ -12,6 +12,8 @@ import (
"sort" "sort"
"time" "time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
"github.com/youtube/vitess/go/rpcplus" "github.com/youtube/vitess/go/rpcplus"
"github.com/youtube/vitess/go/rpcwrap/bsonrpc" "github.com/youtube/vitess/go/rpcwrap/bsonrpc"
@ -52,7 +54,7 @@ func get(rpcClient *rpcplus.Client, path string, verbose bool) {
// it's a get // it's a get
zkPath := &zk.ZkPath{Path: path} zkPath := &zk.ZkPath{Path: path}
zkNode := &zk.ZkNode{} zkNode := &zk.ZkNode{}
if err := rpcClient.Call("ZkReader.Get", zkPath, zkNode); err != nil { if err := rpcClient.Call(context.TODO(), "ZkReader.Get", zkPath, zkNode); err != nil {
log.Fatalf("ZkReader.Get error: %v", err) log.Fatalf("ZkReader.Get error: %v", err)
} }
if verbose { if verbose {
@ -67,7 +69,7 @@ func getv(rpcClient *rpcplus.Client, paths []string, verbose bool) {
zkPathV.Paths[i] = v zkPathV.Paths[i] = v
} }
zkNodeV := &zk.ZkNodeV{} zkNodeV := &zk.ZkNodeV{}
if err := rpcClient.Call("ZkReader.GetV", zkPathV, zkNodeV); err != nil { if err := rpcClient.Call(context.TODO(), "ZkReader.GetV", zkPathV, zkNodeV); err != nil {
log.Fatalf("ZkReader.GetV error: %v", err) log.Fatalf("ZkReader.GetV error: %v", err)
} }
if verbose { if verbose {
@ -81,7 +83,7 @@ func children(rpcClient *rpcplus.Client, paths []string, verbose bool) {
for _, v := range paths { for _, v := range paths {
zkPath := &zk.ZkPath{Path: v} zkPath := &zk.ZkPath{Path: v}
zkNode := &zk.ZkNode{} zkNode := &zk.ZkNode{}
if err := rpcClient.Call("ZkReader.Children", zkPath, zkNode); err != nil { if err := rpcClient.Call(context.TODO(), "ZkReader.Children", zkPath, zkNode); err != nil {
log.Fatalf("ZkReader.Children error: %v", err) log.Fatalf("ZkReader.Children error: %v", err)
} }
if verbose { if verbose {
@ -102,7 +104,7 @@ func getSrvKeyspaceNames(rpcClient *rpcplus.Client, cell string, verbose bool) {
Cell: cell, Cell: cell,
} }
reply := &topo.SrvKeyspaceNames{} reply := &topo.SrvKeyspaceNames{}
if err := rpcClient.Call("TopoReader.GetSrvKeyspaceNames", req, reply); err != nil { if err := rpcClient.Call(context.TODO(), "TopoReader.GetSrvKeyspaceNames", req, reply); err != nil {
log.Fatalf("TopoReader.GetSrvKeyspaceNames error: %v", err) log.Fatalf("TopoReader.GetSrvKeyspaceNames error: %v", err)
} }
if verbose { if verbose {
@ -118,7 +120,7 @@ func getSrvKeyspace(rpcClient *rpcplus.Client, cell, keyspace string, verbose bo
Keyspace: keyspace, Keyspace: keyspace,
} }
reply := &topo.SrvKeyspace{} reply := &topo.SrvKeyspace{}
if err := rpcClient.Call("TopoReader.GetSrvKeyspace", req, reply); err != nil { if err := rpcClient.Call(context.TODO(), "TopoReader.GetSrvKeyspace", req, reply); err != nil {
log.Fatalf("TopoReader.GetSrvKeyspace error: %v", err) log.Fatalf("TopoReader.GetSrvKeyspace error: %v", err)
} }
if verbose { if verbose {
@ -150,7 +152,7 @@ func getEndPoints(rpcClient *rpcplus.Client, cell, keyspace, shard, tabletType s
TabletType: topo.TabletType(tabletType), TabletType: topo.TabletType(tabletType),
} }
reply := &topo.EndPoints{} reply := &topo.EndPoints{}
if err := rpcClient.Call("TopoReader.GetEndPoints", req, reply); err != nil { if err := rpcClient.Call(context.TODO(), "TopoReader.GetEndPoints", req, reply); err != nil {
log.Fatalf("TopoReader.GetEndPoints error: %v", err) log.Fatalf("TopoReader.GetEndPoints error: %v", err)
} }
if verbose { if verbose {

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

@ -14,6 +14,9 @@ import (
"net/http" "net/http"
"reflect" "reflect"
"sync" "sync"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/trace"
) )
// ServerError represents an error that has been returned from // ServerError represents an error that has been returned from
@ -315,7 +318,11 @@ func (client *Client) Close() error {
// the invocation. The done channel will signal when the call is complete by returning // the invocation. The done channel will signal when the call is complete by returning
// the same Call object. If done is nil, Go will allocate a new channel. // the same Call object. If done is nil, Go will allocate a new channel.
// If non-nil, done must be buffered or Go will deliberately crash. // If non-nil, done must be buffered or Go will deliberately crash.
func (client *Client) Go(serviceMethod string, args interface{}, reply interface{}, done chan *Call) *Call { func (client *Client) Go(ctx context.Context, serviceMethod string, args interface{}, reply interface{}, done chan *Call) *Call {
span := trace.NewSpanFromContext(ctx)
span.StartClient(serviceMethod)
defer span.Finish()
call := new(Call) call := new(Call)
call.ServiceMethod = serviceMethod call.ServiceMethod = serviceMethod
call.Args = args call.Args = args
@ -358,7 +365,7 @@ func (client *Client) StreamGo(serviceMethod string, args interface{}, replyStre
} }
// Call invokes the named function, waits for it to complete, and returns its error status. // Call invokes the named function, waits for it to complete, and returns its error status.
func (client *Client) Call(serviceMethod string, args interface{}, reply interface{}) error { func (client *Client) Call(ctx context.Context, serviceMethod string, args interface{}, reply interface{}) error {
call := <-client.Go(serviceMethod, args, reply, make(chan *Call, 1)).Done call := <-client.Go(ctx, serviceMethod, args, reply, make(chan *Call, 1)).Done
return call.Error return call.Error
} }

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

@ -12,6 +12,8 @@ import (
"net" "net"
"testing" "testing"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/rpcplus" "github.com/youtube/vitess/go/rpcplus"
) )
@ -105,6 +107,7 @@ func TestServer(t *testing.T) {
} }
func TestClient(t *testing.T) { func TestClient(t *testing.T) {
ctx := context.Background()
// Assume server is okay (TestServer is above). // Assume server is okay (TestServer is above).
// Test client against server. // Test client against server.
cli, srv := net.Pipe() cli, srv := net.Pipe()
@ -116,7 +119,7 @@ func TestClient(t *testing.T) {
// Synchronous calls // Synchronous calls
args := &Args{7, 8} args := &Args{7, 8}
reply := new(Reply) reply := new(Reply)
err := client.Call("Arith.Add", args, reply) err := client.Call(ctx, "Arith.Add", args, reply)
if err != nil { if err != nil {
t.Errorf("Add: expected no error but got string %q", err.Error()) t.Errorf("Add: expected no error but got string %q", err.Error())
} }
@ -126,7 +129,7 @@ func TestClient(t *testing.T) {
args = &Args{7, 8} args = &Args{7, 8}
reply = new(Reply) reply = new(Reply)
err = client.Call("Arith.Mul", args, reply) err = client.Call(ctx, "Arith.Mul", args, reply)
if err != nil { if err != nil {
t.Errorf("Mul: expected no error but got string %q", err.Error()) t.Errorf("Mul: expected no error but got string %q", err.Error())
} }
@ -137,9 +140,9 @@ func TestClient(t *testing.T) {
// Out of order. // Out of order.
args = &Args{7, 8} args = &Args{7, 8}
mulReply := new(Reply) mulReply := new(Reply)
mulCall := client.Go("Arith.Mul", args, mulReply, nil) mulCall := client.Go(ctx, "Arith.Mul", args, mulReply, nil)
addReply := new(Reply) addReply := new(Reply)
addCall := client.Go("Arith.Add", args, addReply, nil) addCall := client.Go(ctx, "Arith.Add", args, addReply, nil)
addCall = <-addCall.Done addCall = <-addCall.Done
if addCall.Error != nil { if addCall.Error != nil {
@ -160,7 +163,7 @@ func TestClient(t *testing.T) {
// Error test // Error test
args = &Args{7, 0} args = &Args{7, 0}
reply = new(Reply) reply = new(Reply)
err = client.Call("Arith.Div", args, reply) err = client.Call(ctx, "Arith.Div", args, reply)
// expect an error: zero divide // expect an error: zero divide
if err == nil { if err == nil {
t.Error("Div: expected error") t.Error("Div: expected error")

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

@ -17,6 +17,8 @@ import (
"sync/atomic" "sync/atomic"
"testing" "testing"
"time" "time"
"code.google.com/p/go.net/context"
) )
var ( var (
@ -118,13 +120,14 @@ func startHttpServer() {
} }
func TestRPC(t *testing.T) { func TestRPC(t *testing.T) {
ctx := context.Background()
once.Do(startServer) once.Do(startServer)
testRPC(t, serverAddr) testRPC(ctx, t, serverAddr)
newOnce.Do(startNewServer) newOnce.Do(startNewServer)
testRPC(t, newServerAddr) testRPC(ctx, t, newServerAddr)
} }
func testRPC(t *testing.T, addr string) { func testRPC(ctx context.Context, t *testing.T, addr string) {
client, err := Dial("tcp", addr) client, err := Dial("tcp", addr)
if err != nil { if err != nil {
t.Fatal("dialing", err) t.Fatal("dialing", err)
@ -133,7 +136,7 @@ func testRPC(t *testing.T, addr string) {
// Synchronous calls // Synchronous calls
args := &Args{7, 8} args := &Args{7, 8}
reply := new(Reply) reply := new(Reply)
err = client.Call("Arith.Add", args, reply) err = client.Call(ctx, "Arith.Add", args, reply)
if err != nil { if err != nil {
t.Errorf("Add: expected no error but got string %q", err.Error()) t.Errorf("Add: expected no error but got string %q", err.Error())
} }
@ -144,7 +147,7 @@ func testRPC(t *testing.T, addr string) {
// Nonexistent method // Nonexistent method
args = &Args{7, 0} args = &Args{7, 0}
reply = new(Reply) reply = new(Reply)
err = client.Call("Arith.BadOperation", args, reply) err = client.Call(ctx, "Arith.BadOperation", args, reply)
// expect an error // expect an error
if err == nil { if err == nil {
t.Error("BadOperation: expected error") t.Error("BadOperation: expected error")
@ -155,7 +158,7 @@ func testRPC(t *testing.T, addr string) {
// Unknown service // Unknown service
args = &Args{7, 8} args = &Args{7, 8}
reply = new(Reply) reply = new(Reply)
err = client.Call("Arith.Unknown", args, reply) err = client.Call(ctx, "Arith.Unknown", args, reply)
if err == nil { if err == nil {
t.Error("expected error calling unknown service") t.Error("expected error calling unknown service")
} else if strings.Index(err.Error(), "method") < 0 { } else if strings.Index(err.Error(), "method") < 0 {
@ -165,9 +168,9 @@ func testRPC(t *testing.T, addr string) {
// Out of order. // Out of order.
args = &Args{7, 8} args = &Args{7, 8}
mulReply := new(Reply) mulReply := new(Reply)
mulCall := client.Go("Arith.Mul", args, mulReply, nil) mulCall := client.Go(ctx, "Arith.Mul", args, mulReply, nil)
addReply := new(Reply) addReply := new(Reply)
addCall := client.Go("Arith.Add", args, addReply, nil) addCall := client.Go(ctx, "Arith.Add", args, addReply, nil)
addCall = <-addCall.Done addCall = <-addCall.Done
if addCall.Error != nil { if addCall.Error != nil {
@ -188,7 +191,7 @@ func testRPC(t *testing.T, addr string) {
// Error test // Error test
args = &Args{7, 0} args = &Args{7, 0}
reply = new(Reply) reply = new(Reply)
err = client.Call("Arith.Div", args, reply) err = client.Call(ctx, "Arith.Div", args, reply)
// expect an error: zero divide // expect an error: zero divide
if err == nil { if err == nil {
t.Error("Div: expected error") t.Error("Div: expected error")
@ -198,7 +201,7 @@ func testRPC(t *testing.T, addr string) {
// Bad type. // Bad type.
reply = new(Reply) reply = new(Reply)
err = client.Call("Arith.Add", reply, reply) // args, reply would be the correct thing to use err = client.Call(ctx, "Arith.Add", reply, reply) // args, reply would be the correct thing to use
if err == nil { if err == nil {
t.Error("expected error calling Arith.Add with wrong arg type") t.Error("expected error calling Arith.Add with wrong arg type")
} else if strings.Index(err.Error(), "type") < 0 { } else if strings.Index(err.Error(), "type") < 0 {
@ -209,7 +212,7 @@ func testRPC(t *testing.T, addr string) {
const Val = 12345 const Val = 12345
str := fmt.Sprint(Val) str := fmt.Sprint(Val)
reply = new(Reply) reply = new(Reply)
err = client.Call("Arith.Scan", &str, reply) err = client.Call(ctx, "Arith.Scan", &str, reply)
if err != nil { if err != nil {
t.Errorf("Scan: expected no error but got string %q", err.Error()) t.Errorf("Scan: expected no error but got string %q", err.Error())
} else if reply.C != Val { } else if reply.C != Val {
@ -219,7 +222,7 @@ func testRPC(t *testing.T, addr string) {
// Non-struct reply // Non-struct reply
args = &Args{27, 35} args = &Args{27, 35}
str = "" str = ""
err = client.Call("Arith.String", args, &str) err = client.Call(ctx, "Arith.String", args, &str)
if err != nil { if err != nil {
t.Errorf("String: expected no error but got string %q", err.Error()) t.Errorf("String: expected no error but got string %q", err.Error())
} }
@ -230,7 +233,7 @@ func testRPC(t *testing.T, addr string) {
args = &Args{7, 8} args = &Args{7, 8}
reply = new(Reply) reply = new(Reply)
err = client.Call("Arith.Mul", args, reply) err = client.Call(ctx, "Arith.Mul", args, reply)
if err != nil { if err != nil {
t.Errorf("Mul: expected no error but got string %q", err.Error()) t.Errorf("Mul: expected no error but got string %q", err.Error())
} }
@ -240,20 +243,21 @@ func testRPC(t *testing.T, addr string) {
// Takes context // Takes context
emptyString := "" emptyString := ""
err = client.Call("Arith.TakesContext", "", &emptyString) err = client.Call(ctx, "Arith.TakesContext", "", &emptyString)
if err != nil { if err != nil {
t.Errorf("TakesContext: expected no error but got string %q", err.Error()) t.Errorf("TakesContext: expected no error but got string %q", err.Error())
} }
} }
func TestHTTP(t *testing.T) { func TestHTTP(t *testing.T) {
ctx := context.Background()
once.Do(startServer) once.Do(startServer)
testHTTPRPC(t, "") testHTTPRPC(ctx, t, "")
newOnce.Do(startNewServer) newOnce.Do(startNewServer)
testHTTPRPC(t, newHttpPath) testHTTPRPC(ctx, t, newHttpPath)
} }
func testHTTPRPC(t *testing.T, path string) { func testHTTPRPC(ctx context.Context, t *testing.T, path string) {
var client *Client var client *Client
var err error var err error
if path == "" { if path == "" {
@ -268,7 +272,7 @@ func testHTTPRPC(t *testing.T, path string) {
// Synchronous calls // Synchronous calls
args := &Args{7, 8} args := &Args{7, 8}
reply := new(Reply) reply := new(Reply)
err = client.Call("Arith.Add", args, reply) err = client.Call(ctx, "Arith.Add", args, reply)
if err != nil { if err != nil {
t.Errorf("Add: expected no error but got string %q", err.Error()) t.Errorf("Add: expected no error but got string %q", err.Error())
} }
@ -287,7 +291,7 @@ type CodecEmulator struct {
err error err error
} }
func (codec *CodecEmulator) Call(serviceMethod string, args *Args, reply *Reply) error { func (codec *CodecEmulator) Call(ctx context.Context, serviceMethod string, args *Args, reply *Reply) error {
codec.serviceMethod = serviceMethod codec.serviceMethod = serviceMethod
codec.args = args codec.args = args
codec.reply = reply codec.reply = reply
@ -332,18 +336,19 @@ func (codec *CodecEmulator) Close() error {
} }
func TestServeRequest(t *testing.T) { func TestServeRequest(t *testing.T) {
ctx := context.Background()
once.Do(startServer) once.Do(startServer)
testServeRequest(t, nil) testServeRequest(ctx, t, nil)
newOnce.Do(startNewServer) newOnce.Do(startNewServer)
testServeRequest(t, newServer) testServeRequest(ctx, t, newServer)
} }
func testServeRequest(t *testing.T, server *Server) { func testServeRequest(ctx context.Context, t *testing.T, server *Server) {
client := CodecEmulator{server: server} client := CodecEmulator{server: server}
args := &Args{7, 8} args := &Args{7, 8}
reply := new(Reply) reply := new(Reply)
err := client.Call("Arith.Add", args, reply) err := client.Call(ctx, "Arith.Add", args, reply)
if err != nil { if err != nil {
t.Errorf("Add: expected no error but got string %q", err.Error()) t.Errorf("Add: expected no error but got string %q", err.Error())
} }
@ -351,7 +356,7 @@ func testServeRequest(t *testing.T, server *Server) {
t.Errorf("Add: expected %d got %d", reply.C, args.A+args.B) t.Errorf("Add: expected %d got %d", reply.C, args.A+args.B)
} }
err = client.Call("Arith.Add", nil, reply) err = client.Call(ctx, "Arith.Add", nil, reply)
if err == nil { if err == nil {
t.Errorf("expected error calling Arith.Add with nil arg") t.Errorf("expected error calling Arith.Add with nil arg")
} }
@ -410,12 +415,13 @@ func (WriteFailCodec) Close() error {
} }
func TestSendDeadlock(t *testing.T) { func TestSendDeadlock(t *testing.T) {
ctx := context.Background()
client := NewClientWithCodec(WriteFailCodec(0)) client := NewClientWithCodec(WriteFailCodec(0))
done := make(chan bool) done := make(chan bool)
go func() { go func() {
testSendDeadlock(client) testSendDeadlock(ctx, client)
testSendDeadlock(client) testSendDeadlock(ctx, client)
done <- true done <- true
}() }()
select { select {
@ -426,13 +432,13 @@ func TestSendDeadlock(t *testing.T) {
} }
} }
func testSendDeadlock(client *Client) { func testSendDeadlock(ctx context.Context, client *Client) {
defer func() { defer func() {
recover() recover()
}() }()
args := &Args{7, 8} args := &Args{7, 8}
reply := new(Reply) reply := new(Reply)
client.Call("Arith.Add", args, reply) client.Call(ctx, "Arith.Add", args, reply)
} }
func dialDirect() (*Client, error) { func dialDirect() (*Client, error) {
@ -443,7 +449,7 @@ func dialHTTP() (*Client, error) {
return DialHTTP("tcp", httpServerAddr) return DialHTTP("tcp", httpServerAddr)
} }
func countMallocs(dial func() (*Client, error), t *testing.T) uint64 { func countMallocs(ctx context.Context, dial func() (*Client, error), t *testing.T) uint64 {
once.Do(startServer) once.Do(startServer)
client, err := dial() client, err := dial()
if err != nil { if err != nil {
@ -456,7 +462,7 @@ func countMallocs(dial func() (*Client, error), t *testing.T) uint64 {
mallocs := 0 - memstats.Mallocs mallocs := 0 - memstats.Mallocs
const count = 100 const count = 100
for i := 0; i < count; i++ { for i := 0; i < count; i++ {
err := client.Call("Arith.Add", args, reply) err := client.Call(ctx, "Arith.Add", args, reply)
if err != nil { if err != nil {
t.Errorf("Add: expected no error but got string %q", err.Error()) t.Errorf("Add: expected no error but got string %q", err.Error())
} }
@ -470,11 +476,13 @@ func countMallocs(dial func() (*Client, error), t *testing.T) uint64 {
} }
func TestCountMallocs(t *testing.T) { func TestCountMallocs(t *testing.T) {
fmt.Printf("mallocs per rpc round trip: %d\n", countMallocs(dialDirect, t)) ctx := context.Background()
fmt.Printf("mallocs per rpc round trip: %d\n", countMallocs(ctx, dialDirect, t))
} }
func TestCountMallocsOverHTTP(t *testing.T) { func TestCountMallocsOverHTTP(t *testing.T) {
fmt.Printf("mallocs per HTTP rpc round trip: %d\n", countMallocs(dialHTTP, t)) ctx := context.Background()
fmt.Printf("mallocs per HTTP rpc round trip: %d\n", countMallocs(ctx, dialHTTP, t))
} }
type writeCrasher struct { type writeCrasher struct {
@ -495,10 +503,11 @@ func (writeCrasher) Write(p []byte) (int, error) {
} }
func TestClientWriteError(t *testing.T) { func TestClientWriteError(t *testing.T) {
ctx := context.Background()
w := &writeCrasher{done: make(chan bool)} w := &writeCrasher{done: make(chan bool)}
c := NewClient(w) client := NewClient(w)
res := false res := false
err := c.Call("foo", 1, &res) err := client.Call(ctx, "foo", 1, &res)
if err == nil { if err == nil {
t.Fatal("expected error") t.Fatal("expected error")
} }
@ -508,7 +517,7 @@ func TestClientWriteError(t *testing.T) {
w.done <- true w.done <- true
} }
func benchmarkEndToEnd(dial func() (*Client, error), b *testing.B) { func benchmarkEndToEnd(ctx context.Context, dial func() (*Client, error), b *testing.B) {
b.StopTimer() b.StopTimer()
once.Do(startServer) once.Do(startServer)
client, err := dial() client, err := dial()
@ -528,7 +537,7 @@ func benchmarkEndToEnd(dial func() (*Client, error), b *testing.B) {
go func() { go func() {
reply := new(Reply) reply := new(Reply)
for atomic.AddInt32(&N, -1) >= 0 { for atomic.AddInt32(&N, -1) >= 0 {
err := client.Call("Arith.Add", args, reply) err := client.Call(ctx, "Arith.Add", args, reply)
if err != nil { if err != nil {
b.Fatalf("rpc error: Add: expected no error but got string %q", err.Error()) b.Fatalf("rpc error: Add: expected no error but got string %q", err.Error())
} }
@ -542,7 +551,7 @@ func benchmarkEndToEnd(dial func() (*Client, error), b *testing.B) {
wg.Wait() wg.Wait()
} }
func benchmarkEndToEndAsync(dial func() (*Client, error), b *testing.B) { func benchmarkEndToEndAsync(ctx context.Context, dial func() (*Client, error), b *testing.B) {
const MaxConcurrentCalls = 100 const MaxConcurrentCalls = 100
b.StopTimer() b.StopTimer()
once.Do(startServer) once.Do(startServer)
@ -567,7 +576,7 @@ func benchmarkEndToEndAsync(dial func() (*Client, error), b *testing.B) {
for atomic.AddInt32(&send, -1) >= 0 { for atomic.AddInt32(&send, -1) >= 0 {
gate <- true gate <- true
reply := new(Reply) reply := new(Reply)
client.Go("Arith.Add", args, reply, res) client.Go(ctx, "Arith.Add", args, reply, res)
} }
}() }()
go func() { go func() {
@ -590,17 +599,17 @@ func benchmarkEndToEndAsync(dial func() (*Client, error), b *testing.B) {
} }
func BenchmarkEndToEnd(b *testing.B) { func BenchmarkEndToEnd(b *testing.B) {
benchmarkEndToEnd(dialDirect, b) benchmarkEndToEnd(context.Background(), dialDirect, b)
} }
func BenchmarkEndToEndHTTP(b *testing.B) { func BenchmarkEndToEndHTTP(b *testing.B) {
benchmarkEndToEnd(dialHTTP, b) benchmarkEndToEnd(context.Background(), dialHTTP, b)
} }
func BenchmarkEndToEndAsync(b *testing.B) { func BenchmarkEndToEndAsync(b *testing.B) {
benchmarkEndToEndAsync(dialDirect, b) benchmarkEndToEndAsync(context.Background(), dialDirect, b)
} }
func BenchmarkEndToEndAsyncHTTP(b *testing.B) { func BenchmarkEndToEndAsyncHTTP(b *testing.B) {
benchmarkEndToEndAsync(dialHTTP, b) benchmarkEndToEndAsync(context.Background(), dialHTTP, b)
} }

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

@ -7,6 +7,8 @@ import (
"strings" "strings"
"testing" "testing"
"time" "time"
"code.google.com/p/go.net/context"
) )
const ( const (
@ -105,6 +107,7 @@ func callOnceAndCheck(t *testing.T, client *Client) {
} }
func TestStreamingRpc(t *testing.T) { func TestStreamingRpc(t *testing.T) {
ctx := context.Background()
if testing.Short() { if testing.Short() {
t.Skip("skipping wait-based test in short mode.") t.Skip("skipping wait-based test in short mode.")
} }
@ -114,7 +117,7 @@ func TestStreamingRpc(t *testing.T) {
// Nonexistent method // Nonexistent method
args := &StreamingArgs{7, 10, -1, -1} args := &StreamingArgs{7, 10, -1, -1}
reply := new(StreamingReply) reply := new(StreamingReply)
err := client.Call("StreamingArith.BadOperation", args, reply) err := client.Call(ctx, "StreamingArith.BadOperation", args, reply)
// expect an error // expect an error
if err == nil { if err == nil {
t.Error("BadOperation: expected error") t.Error("BadOperation: expected error")

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

@ -13,6 +13,8 @@ import (
"net/http" "net/http"
"time" "time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
rpc "github.com/youtube/vitess/go/rpcplus" rpc "github.com/youtube/vitess/go/rpcplus"
"github.com/youtube/vitess/go/rpcwrap/auth" "github.com/youtube/vitess/go/rpcwrap/auth"
@ -68,12 +70,13 @@ func DialAuthHTTP(network, address, user, password, codecName string, cFactory C
return return
} }
reply := new(auth.GetNewChallengeReply) reply := new(auth.GetNewChallengeReply)
if err = conn.Call("AuthenticatorCRAMMD5.GetNewChallenge", "", reply); err != nil { if err = conn.Call(context.TODO(), "AuthenticatorCRAMMD5.GetNewChallenge", "", reply); err != nil {
return return
} }
proof := auth.CRAMMD5GetExpected(user, password, reply.Challenge) proof := auth.CRAMMD5GetExpected(user, password, reply.Challenge)
if err = conn.Call( if err = conn.Call(
context.TODO(),
"AuthenticatorCRAMMD5.Authenticate", "AuthenticatorCRAMMD5.Authenticate",
auth.AuthenticateRequest{Proof: proof}, new(auth.AuthenticateReply)); err != nil { auth.AuthenticateRequest{Proof: proof}, new(auth.AuthenticateReply)); err != nil {
return return

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

@ -10,6 +10,8 @@ import (
"fmt" "fmt"
"time" "time"
"code.google.com/p/go.net/context"
rpc "github.com/youtube/vitess/go/rpcplus" rpc "github.com/youtube/vitess/go/rpcplus"
"github.com/youtube/vitess/go/rpcwrap/bsonrpc" "github.com/youtube/vitess/go/rpcwrap/bsonrpc"
"github.com/youtube/vitess/go/vt/mysqlctl/mysqlctlclient" "github.com/youtube/vitess/go/vt/mysqlctl/mysqlctlclient"
@ -31,7 +33,7 @@ func goRpcMysqlctlClientFactory(network, addr string, dialTimeout time.Duration)
// Start is part of the MysqlctlClient interface. // Start is part of the MysqlctlClient interface.
func (c *goRpcMysqlctlClient) Start(mysqlWaitTime time.Duration) error { func (c *goRpcMysqlctlClient) Start(mysqlWaitTime time.Duration) error {
return c.rpcClient.Call("MysqlctlServer.Start", &mysqlWaitTime, nil) return c.rpcClient.Call(context.TODO(), "MysqlctlServer.Start", &mysqlWaitTime, nil)
} }
// Shutdown is part of the MysqlctlClient interface. // Shutdown is part of the MysqlctlClient interface.
@ -39,7 +41,7 @@ func (c *goRpcMysqlctlClient) Shutdown(waitForMysqld bool, mysqlWaitTime time.Du
if !waitForMysqld { if !waitForMysqld {
mysqlWaitTime = 0 mysqlWaitTime = 0
} }
return c.rpcClient.Call("MysqlctlServer.Shutdown", &mysqlWaitTime, nil) return c.rpcClient.Call(context.TODO(), "MysqlctlServer.Shutdown", &mysqlWaitTime, nil)
} }
// Close is part of the MysqlctlClient interface. // Close is part of the MysqlctlClient interface.

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

@ -10,7 +10,9 @@ package actionnode
import ( import (
"time" "time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
"github.com/youtube/vitess/go/trace"
"github.com/youtube/vitess/go/vt/topo" "github.com/youtube/vitess/go/vt/topo"
) )
@ -22,8 +24,15 @@ var (
// LockKeyspace will lock the keyspace in the topology server. // LockKeyspace will lock the keyspace in the topology server.
// UnlockKeyspace should be called if this returns no error. // UnlockKeyspace should be called if this returns no error.
func (n *ActionNode) LockKeyspace(ts topo.Server, keyspace string, lockTimeout time.Duration, interrupted chan struct{}) (lockPath string, err error) { func (n *ActionNode) LockKeyspace(ctx context.Context, ts topo.Server, keyspace string, lockTimeout time.Duration, interrupted chan struct{}) (lockPath string, err error) {
log.Infof("Locking keyspace %v for action %v", keyspace, n.Action) log.Infof("Locking keyspace %v for action %v", keyspace, n.Action)
span := trace.NewSpanFromContext(ctx)
span.StartClient("TopoServer.LockKeyspaceForAction")
span.Annotate("action", n.Action)
span.Annotate("keyspace", keyspace)
defer span.Finish()
return ts.LockKeyspaceForAction(keyspace, n.ToJson(), lockTimeout, interrupted) return ts.LockKeyspaceForAction(keyspace, n.ToJson(), lockTimeout, interrupted)
} }
@ -52,8 +61,16 @@ func (n *ActionNode) UnlockKeyspace(ts topo.Server, keyspace string, lockPath st
// LockShard will lock the shard in the topology server. // LockShard will lock the shard in the topology server.
// UnlockShard should be called if this returns no error. // UnlockShard should be called if this returns no error.
func (n *ActionNode) LockShard(ts topo.Server, keyspace, shard string, lockTimeout time.Duration, interrupted chan struct{}) (lockPath string, err error) { func (n *ActionNode) LockShard(ctx context.Context, ts topo.Server, keyspace, shard string, lockTimeout time.Duration, interrupted chan struct{}) (lockPath string, err error) {
log.Infof("Locking shard %v/%v for action %v", keyspace, shard, n.Action) log.Infof("Locking shard %v/%v for action %v", keyspace, shard, n.Action)
span := trace.NewSpanFromContext(ctx)
span.StartClient("TopoServer.LockShardForAction")
span.Annotate("action", n.Action)
span.Annotate("keyspace", keyspace)
span.Annotate("shard", shard)
defer span.Finish()
return ts.LockShardForAction(keyspace, shard, n.ToJson(), lockTimeout, interrupted) return ts.LockShardForAction(keyspace, shard, n.ToJson(), lockTimeout, interrupted)
} }
@ -82,8 +99,17 @@ func (n *ActionNode) UnlockShard(ts topo.Server, keyspace, shard string, lockPat
// LockSrvShard will lock the serving shard in the topology server. // LockSrvShard will lock the serving shard in the topology server.
// UnlockSrvShard should be called if this returns no error. // UnlockSrvShard should be called if this returns no error.
func (n *ActionNode) LockSrvShard(ts topo.Server, cell, keyspace, shard string, lockTimeout time.Duration, interrupted chan struct{}) (lockPath string, err error) { func (n *ActionNode) LockSrvShard(ctx context.Context, ts topo.Server, cell, keyspace, shard string, lockTimeout time.Duration, interrupted chan struct{}) (lockPath string, err error) {
log.Infof("Locking serving shard %v/%v/%v for action %v", cell, keyspace, shard, n.Action) log.Infof("Locking serving shard %v/%v/%v for action %v", cell, keyspace, shard, n.Action)
span := trace.NewSpanFromContext(ctx)
span.StartClient("TopoServer.LockSrvShardForAction")
span.Annotate("action", n.Action)
span.Annotate("keyspace", keyspace)
span.Annotate("shard", shard)
span.Annotate("cell", cell)
defer span.Finish()
return ts.LockSrvShardForAction(cell, keyspace, shard, n.ToJson(), lockTimeout, interrupted) return ts.LockSrvShardForAction(cell, keyspace, shard, n.ToJson(), lockTimeout, interrupted)
} }

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

@ -30,6 +30,8 @@ import (
"sync" "sync"
"time" "time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
"github.com/youtube/vitess/go/history" "github.com/youtube/vitess/go/history"
"github.com/youtube/vitess/go/jscfg" "github.com/youtube/vitess/go/jscfg"
@ -380,7 +382,7 @@ func (agent *ActionAgent) checkTabletMysqlPort(tablet *topo.TabletInfo) *topo.Ta
log.Warningf("MySQL port has changed from %v to %v, updating it in tablet record", tablet.Portmap["mysql"], mport) log.Warningf("MySQL port has changed from %v to %v, updating it in tablet record", tablet.Portmap["mysql"], mport)
tablet.Portmap["mysql"] = mport tablet.Portmap["mysql"] = mport
if err := topo.UpdateTablet(agent.TopoServer, tablet); err != nil { if err := topo.UpdateTablet(context.TODO(), agent.TopoServer, tablet); err != nil {
log.Warningf("Failed to update tablet record, may use old mysql port") log.Warningf("Failed to update tablet record, may use old mysql port")
return nil return nil
} }

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

@ -44,91 +44,91 @@ type RpcAgent interface {
// Various read-only methods // Various read-only methods
Ping(args string) string Ping(ctx context.Context, args string) string
GetSchema(tables, excludeTables []string, includeViews bool) (*myproto.SchemaDefinition, error) GetSchema(ctx context.Context, tables, excludeTables []string, includeViews bool) (*myproto.SchemaDefinition, error)
GetPermissions() (*myproto.Permissions, error) GetPermissions(ctx context.Context) (*myproto.Permissions, error)
// Various read-write methods // Various read-write methods
SetReadOnly(rdonly bool) error SetReadOnly(ctx context.Context, rdonly bool) error
ChangeType(topo.TabletType) error ChangeType(ctx context.Context, tabletType topo.TabletType) error
Scrap() error Scrap(ctx context.Context) error
Sleep(duration time.Duration) Sleep(ctx context.Context, duration time.Duration)
ExecuteHook(hk *hook.Hook) *hook.HookResult ExecuteHook(ctx context.Context, hk *hook.Hook) *hook.HookResult
RefreshState() RefreshState(ctx context.Context)
RunHealthCheck(targetTabletType topo.TabletType) RunHealthCheck(ctx context.Context, targetTabletType topo.TabletType)
ReloadSchema() ReloadSchema(ctx context.Context)
PreflightSchema(change string) (*myproto.SchemaChangeResult, error) PreflightSchema(ctx context.Context, change string) (*myproto.SchemaChangeResult, error)
ApplySchema(change *myproto.SchemaChange) (*myproto.SchemaChangeResult, error) ApplySchema(ctx context.Context, change *myproto.SchemaChange) (*myproto.SchemaChangeResult, error)
ExecuteFetch(query string, maxrows int, wantFields, disableBinlogs bool) (*proto.QueryResult, error) ExecuteFetch(ctx context.Context, query string, maxrows int, wantFields, disableBinlogs bool) (*proto.QueryResult, error)
// Replication related methods // Replication related methods
SlaveStatus() (*myproto.ReplicationStatus, error) SlaveStatus(ctx context.Context) (*myproto.ReplicationStatus, error)
WaitSlavePosition(position myproto.ReplicationPosition, waitTimeout time.Duration) (*myproto.ReplicationStatus, error) WaitSlavePosition(ctx context.Context, position myproto.ReplicationPosition, waitTimeout time.Duration) (*myproto.ReplicationStatus, error)
MasterPosition() (myproto.ReplicationPosition, error) MasterPosition(ctx context.Context) (myproto.ReplicationPosition, error)
ReparentPosition(rp *myproto.ReplicationPosition) (*actionnode.RestartSlaveData, error) ReparentPosition(ctx context.Context, rp *myproto.ReplicationPosition) (*actionnode.RestartSlaveData, error)
StopSlave() error StopSlave(ctx context.Context) error
StopSlaveMinimum(position myproto.ReplicationPosition, waitTime time.Duration) (*myproto.ReplicationStatus, error) StopSlaveMinimum(ctx context.Context, position myproto.ReplicationPosition, waitTime time.Duration) (*myproto.ReplicationStatus, error)
StartSlave() error StartSlave(ctx context.Context) error
TabletExternallyReparented(actionTimeout time.Duration) error TabletExternallyReparented(ctx context.Context, actionTimeout time.Duration) error
GetSlaves() ([]string, error) GetSlaves(ctx context.Context) ([]string, error)
WaitBlpPosition(blpPosition *blproto.BlpPosition, waitTime time.Duration) error WaitBlpPosition(ctx context.Context, blpPosition *blproto.BlpPosition, waitTime time.Duration) error
StopBlp() (*blproto.BlpPositionList, error) StopBlp(ctx context.Context) (*blproto.BlpPositionList, error)
StartBlp() error StartBlp(ctx context.Context) error
RunBlpUntil(bpl *blproto.BlpPositionList, waitTime time.Duration) (*myproto.ReplicationPosition, error) RunBlpUntil(ctx context.Context, bpl *blproto.BlpPositionList, waitTime time.Duration) (*myproto.ReplicationPosition, error)
// Reparenting related functions // Reparenting related functions
DemoteMaster() error DemoteMaster(ctx context.Context) error
PromoteSlave() (*actionnode.RestartSlaveData, error) PromoteSlave(ctx context.Context) (*actionnode.RestartSlaveData, error)
SlaveWasPromoted() error SlaveWasPromoted(ctx context.Context) error
RestartSlave(rsd *actionnode.RestartSlaveData) error RestartSlave(ctx context.Context, rsd *actionnode.RestartSlaveData) error
SlaveWasRestarted(swrd *actionnode.SlaveWasRestartedArgs) error SlaveWasRestarted(ctx context.Context, swrd *actionnode.SlaveWasRestartedArgs) error
BreakSlaves() error BreakSlaves(ctx context.Context) error
// Backup / restore related methods // Backup / restore related methods
Snapshot(args *actionnode.SnapshotArgs, logger logutil.Logger) (*actionnode.SnapshotReply, error) Snapshot(ctx context.Context, args *actionnode.SnapshotArgs, logger logutil.Logger) (*actionnode.SnapshotReply, error)
SnapshotSourceEnd(args *actionnode.SnapshotSourceEndArgs) error SnapshotSourceEnd(ctx context.Context, args *actionnode.SnapshotSourceEndArgs) error
ReserveForRestore(args *actionnode.ReserveForRestoreArgs) error ReserveForRestore(ctx context.Context, args *actionnode.ReserveForRestoreArgs) error
Restore(args *actionnode.RestoreArgs, logger logutil.Logger) error Restore(ctx context.Context, args *actionnode.RestoreArgs, logger logutil.Logger) error
MultiSnapshot(args *actionnode.MultiSnapshotArgs, logger logutil.Logger) (*actionnode.MultiSnapshotReply, error) MultiSnapshot(ctx context.Context, args *actionnode.MultiSnapshotArgs, logger logutil.Logger) (*actionnode.MultiSnapshotReply, error)
MultiRestore(args *actionnode.MultiRestoreArgs, logger logutil.Logger) error MultiRestore(ctx context.Context, args *actionnode.MultiRestoreArgs, logger logutil.Logger) error
// RPC helpers // RPC helpers
RpcWrap(ctx context.Context, name string, args, reply interface{}, f func() error) error RpcWrap(ctx context.Context, name string, args, reply interface{}, f func() error) error
@ -145,25 +145,25 @@ type RpcAgent interface {
// Ping makes sure RPCs work, and refreshes the tablet record. // Ping makes sure RPCs work, and refreshes the tablet record.
// Should be called under RpcWrap. // Should be called under RpcWrap.
func (agent *ActionAgent) Ping(args string) string { func (agent *ActionAgent) Ping(ctx context.Context, args string) string {
return args return args
} }
// GetSchema returns the schema. // GetSchema returns the schema.
// Should be called under RpcWrap. // Should be called under RpcWrap.
func (agent *ActionAgent) GetSchema(tables, excludeTables []string, includeViews bool) (*myproto.SchemaDefinition, error) { func (agent *ActionAgent) GetSchema(ctx context.Context, tables, excludeTables []string, includeViews bool) (*myproto.SchemaDefinition, error) {
return agent.MysqlDaemon.GetSchema(agent.Tablet().DbName(), tables, excludeTables, includeViews) return agent.MysqlDaemon.GetSchema(agent.Tablet().DbName(), tables, excludeTables, includeViews)
} }
// GetPermissions returns the db permissions. // GetPermissions returns the db permissions.
// Should be called under RpcWrap. // Should be called under RpcWrap.
func (agent *ActionAgent) GetPermissions() (*myproto.Permissions, error) { func (agent *ActionAgent) GetPermissions(ctx context.Context) (*myproto.Permissions, error) {
return agent.Mysqld.GetPermissions() return agent.Mysqld.GetPermissions()
} }
// SetReadOnly makes the mysql instance read-only or read-write // SetReadOnly makes the mysql instance read-only or read-write
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) SetReadOnly(rdonly bool) error { func (agent *ActionAgent) SetReadOnly(ctx context.Context, rdonly bool) error {
err := agent.Mysqld.SetReadOnly(rdonly) err := agent.Mysqld.SetReadOnly(rdonly)
if err != nil { if err != nil {
return err return err
@ -178,48 +178,48 @@ func (agent *ActionAgent) SetReadOnly(rdonly bool) error {
} else { } else {
tablet.State = topo.STATE_READ_WRITE tablet.State = topo.STATE_READ_WRITE
} }
return topo.UpdateTablet(agent.TopoServer, tablet) return topo.UpdateTablet(ctx, agent.TopoServer, tablet)
} }
// ChangeType changes the tablet type // ChangeType changes the tablet type
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) ChangeType(tabletType topo.TabletType) error { func (agent *ActionAgent) ChangeType(ctx context.Context, tabletType topo.TabletType) error {
return topotools.ChangeType(agent.TopoServer, agent.TabletAlias, tabletType, nil, true /*runHooks*/) return topotools.ChangeType(agent.TopoServer, agent.TabletAlias, tabletType, nil, true /*runHooks*/)
} }
// Scrap scraps the live running tablet // Scrap scraps the live running tablet
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) Scrap() error { func (agent *ActionAgent) Scrap(ctx context.Context) error {
return topotools.Scrap(agent.TopoServer, agent.TabletAlias, false) return topotools.Scrap(agent.TopoServer, agent.TabletAlias, false)
} }
// Sleep sleeps for the duration // Sleep sleeps for the duration
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) Sleep(duration time.Duration) { func (agent *ActionAgent) Sleep(ctx context.Context, duration time.Duration) {
time.Sleep(duration) time.Sleep(duration)
} }
// ExecuteHook executes the provided hook locally, and returns the result. // ExecuteHook executes the provided hook locally, and returns the result.
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) ExecuteHook(hk *hook.Hook) *hook.HookResult { func (agent *ActionAgent) ExecuteHook(ctx context.Context, hk *hook.Hook) *hook.HookResult {
topotools.ConfigureTabletHook(hk, agent.TabletAlias) topotools.ConfigureTabletHook(hk, agent.TabletAlias)
return hk.Execute() return hk.Execute()
} }
// RefreshState reload the tablet record from the topo server. // RefreshState reload the tablet record from the topo server.
// Should be called under RpcWrapLockAction, so it actually works. // Should be called under RpcWrapLockAction, so it actually works.
func (agent *ActionAgent) RefreshState() { func (agent *ActionAgent) RefreshState(ctx context.Context) {
} }
// RunHealthCheck will manually run the health check on the tablet // RunHealthCheck will manually run the health check on the tablet
// Should be called under RpcWrap. // Should be called under RpcWrap.
func (agent *ActionAgent) RunHealthCheck(targetTabletType topo.TabletType) { func (agent *ActionAgent) RunHealthCheck(ctx context.Context, targetTabletType topo.TabletType) {
agent.runHealthCheck(targetTabletType) agent.runHealthCheck(targetTabletType)
} }
// ReloadSchema will reload the schema // ReloadSchema will reload the schema
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) ReloadSchema() { func (agent *ActionAgent) ReloadSchema(ctx context.Context) {
if agent.DBConfigs == nil { if agent.DBConfigs == nil {
// we skip this for test instances that can't connect to the DB anyway // we skip this for test instances that can't connect to the DB anyway
return return
@ -233,7 +233,7 @@ func (agent *ActionAgent) ReloadSchema() {
// PreflightSchema will try out the schema change // PreflightSchema will try out the schema change
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) PreflightSchema(change string) (*myproto.SchemaChangeResult, error) { func (agent *ActionAgent) PreflightSchema(ctx context.Context, change string) (*myproto.SchemaChangeResult, error) {
// get the db name from the tablet // get the db name from the tablet
tablet := agent.Tablet() tablet := agent.Tablet()
@ -243,7 +243,7 @@ func (agent *ActionAgent) PreflightSchema(change string) (*myproto.SchemaChangeR
// ApplySchema will apply a schema change // ApplySchema will apply a schema change
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) ApplySchema(change *myproto.SchemaChange) (*myproto.SchemaChangeResult, error) { func (agent *ActionAgent) ApplySchema(ctx context.Context, change *myproto.SchemaChange) (*myproto.SchemaChangeResult, error) {
// get the db name from the tablet // get the db name from the tablet
tablet := agent.Tablet() tablet := agent.Tablet()
@ -254,13 +254,13 @@ func (agent *ActionAgent) ApplySchema(change *myproto.SchemaChange) (*myproto.Sc
} }
// and if it worked, reload the schema // and if it worked, reload the schema
agent.ReloadSchema() agent.ReloadSchema(ctx)
return scr, nil return scr, nil
} }
// ExecuteFetch will execute the given query, possibly disabling binlogs. // ExecuteFetch will execute the given query, possibly disabling binlogs.
// Should be called under RpcWrap. // Should be called under RpcWrap.
func (agent *ActionAgent) ExecuteFetch(query string, maxrows int, wantFields, disableBinlogs bool) (*proto.QueryResult, error) { func (agent *ActionAgent) ExecuteFetch(ctx context.Context, query string, maxrows int, wantFields, disableBinlogs bool) (*proto.QueryResult, error) {
// get a connection // get a connection
conn, err := agent.MysqlDaemon.GetDbaConnection() conn, err := agent.MysqlDaemon.GetDbaConnection()
if err != nil { if err != nil {
@ -294,14 +294,14 @@ func (agent *ActionAgent) ExecuteFetch(query string, maxrows int, wantFields, di
// SlaveStatus returns the replication status // SlaveStatus returns the replication status
// Should be called under RpcWrap. // Should be called under RpcWrap.
func (agent *ActionAgent) SlaveStatus() (*myproto.ReplicationStatus, error) { func (agent *ActionAgent) SlaveStatus(ctx context.Context) (*myproto.ReplicationStatus, error) {
return agent.MysqlDaemon.SlaveStatus() return agent.MysqlDaemon.SlaveStatus()
} }
// WaitSlavePosition waits until we reach the provided position, // WaitSlavePosition waits until we reach the provided position,
// and returns the current position // and returns the current position
// Should be called under RpcWrapLock. // Should be called under RpcWrapLock.
func (agent *ActionAgent) WaitSlavePosition(position myproto.ReplicationPosition, waitTimeout time.Duration) (*myproto.ReplicationStatus, error) { func (agent *ActionAgent) WaitSlavePosition(ctx context.Context, position myproto.ReplicationPosition, waitTimeout time.Duration) (*myproto.ReplicationStatus, error) {
if err := agent.Mysqld.WaitMasterPos(position, waitTimeout); err != nil { if err := agent.Mysqld.WaitMasterPos(position, waitTimeout); err != nil {
return nil, err return nil, err
} }
@ -311,14 +311,14 @@ func (agent *ActionAgent) WaitSlavePosition(position myproto.ReplicationPosition
// MasterPosition returns the master position // MasterPosition returns the master position
// Should be called under RpcWrap. // Should be called under RpcWrap.
func (agent *ActionAgent) MasterPosition() (myproto.ReplicationPosition, error) { func (agent *ActionAgent) MasterPosition(ctx context.Context) (myproto.ReplicationPosition, error) {
return agent.Mysqld.MasterPosition() return agent.Mysqld.MasterPosition()
} }
// ReparentPosition returns the RestartSlaveData for the provided // ReparentPosition returns the RestartSlaveData for the provided
// ReplicationPosition. // ReplicationPosition.
// Should be called under RpcWrap. // Should be called under RpcWrap.
func (agent *ActionAgent) ReparentPosition(rp *myproto.ReplicationPosition) (*actionnode.RestartSlaveData, error) { func (agent *ActionAgent) ReparentPosition(ctx context.Context, rp *myproto.ReplicationPosition) (*actionnode.RestartSlaveData, error) {
replicationStatus, waitPosition, timePromoted, err := agent.Mysqld.ReparentPosition(*rp) replicationStatus, waitPosition, timePromoted, err := agent.Mysqld.ReparentPosition(*rp)
if err != nil { if err != nil {
return nil, err return nil, err
@ -333,13 +333,13 @@ func (agent *ActionAgent) ReparentPosition(rp *myproto.ReplicationPosition) (*ac
// StopSlave will stop the replication // StopSlave will stop the replication
// Should be called under RpcWrapLock. // Should be called under RpcWrapLock.
func (agent *ActionAgent) StopSlave() error { func (agent *ActionAgent) StopSlave(ctx context.Context) error {
return agent.MysqlDaemon.StopSlave(agent.hookExtraEnv()) return agent.MysqlDaemon.StopSlave(agent.hookExtraEnv())
} }
// StopSlaveMinimum will stop the slave after it reaches at least the // StopSlaveMinimum will stop the slave after it reaches at least the
// provided position. // provided position.
func (agent *ActionAgent) StopSlaveMinimum(position myproto.ReplicationPosition, waitTime time.Duration) (*myproto.ReplicationStatus, error) { func (agent *ActionAgent) StopSlaveMinimum(ctx context.Context, position myproto.ReplicationPosition, waitTime time.Duration) (*myproto.ReplicationStatus, error) {
if err := agent.Mysqld.WaitMasterPos(position, waitTime); err != nil { if err := agent.Mysqld.WaitMasterPos(position, waitTime); err != nil {
return nil, err return nil, err
} }
@ -351,14 +351,14 @@ func (agent *ActionAgent) StopSlaveMinimum(position myproto.ReplicationPosition,
// StartSlave will start the replication // StartSlave will start the replication
// Should be called under RpcWrapLock. // Should be called under RpcWrapLock.
func (agent *ActionAgent) StartSlave() error { func (agent *ActionAgent) StartSlave(ctx context.Context) error {
return agent.MysqlDaemon.StartSlave(agent.hookExtraEnv()) return agent.MysqlDaemon.StartSlave(agent.hookExtraEnv())
} }
// TabletExternallyReparented updates all topo records so the current // TabletExternallyReparented updates all topo records so the current
// tablet is the new master for this shard. // tablet is the new master for this shard.
// Should be called under RpcWrapLock. // Should be called under RpcWrapLock.
func (agent *ActionAgent) TabletExternallyReparented(actionTimeout time.Duration) error { func (agent *ActionAgent) TabletExternallyReparented(ctx context.Context, actionTimeout time.Duration) error {
tablet := agent.Tablet() tablet := agent.Tablet()
// fast quick check on the shard to see if we're not the master already // fast quick check on the shard to see if we're not the master already
@ -375,14 +375,14 @@ func (agent *ActionAgent) TabletExternallyReparented(actionTimeout time.Duration
// grab the shard lock // grab the shard lock
actionNode := actionnode.ShardExternallyReparented(agent.TabletAlias) actionNode := actionnode.ShardExternallyReparented(agent.TabletAlias)
interrupted := make(chan struct{}) interrupted := make(chan struct{})
lockPath, err := actionNode.LockShard(agent.TopoServer, tablet.Keyspace, tablet.Shard, agent.LockTimeout, interrupted) lockPath, err := actionNode.LockShard(ctx, agent.TopoServer, tablet.Keyspace, tablet.Shard, agent.LockTimeout, interrupted)
if err != nil { if err != nil {
log.Warningf("TabletExternallyReparented: Cannot lock shard %v/%v: %v", tablet.Keyspace, tablet.Shard, err) log.Warningf("TabletExternallyReparented: Cannot lock shard %v/%v: %v", tablet.Keyspace, tablet.Shard, err)
return err return err
} }
// do the work // do the work
runAfterAction, err := agent.tabletExternallyReparentedLocked(actionTimeout, interrupted) runAfterAction, err := agent.tabletExternallyReparentedLocked(ctx, actionTimeout, interrupted)
if err != nil { if err != nil {
log.Warningf("TabletExternallyReparented: internal error: %v", err) log.Warningf("TabletExternallyReparented: internal error: %v", err)
} }
@ -406,7 +406,7 @@ func (agent *ActionAgent) TabletExternallyReparented(actionTimeout time.Duration
// tabletExternallyReparentedLocked is called with the shard lock. // tabletExternallyReparentedLocked is called with the shard lock.
// It returns if agent.refreshTablet should be called, and the error. // It returns if agent.refreshTablet should be called, and the error.
// Note both are set independently (can have both true and an error). // Note both are set independently (can have both true and an error).
func (agent *ActionAgent) tabletExternallyReparentedLocked(actionTimeout time.Duration, interrupted chan struct{}) (bool, error) { func (agent *ActionAgent) tabletExternallyReparentedLocked(ctx context.Context, actionTimeout time.Duration, interrupted chan struct{}) (bool, error) {
// re-read the tablet record to be sure we have the latest version // re-read the tablet record to be sure we have the latest version
tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias) tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias)
if err != nil { if err != nil {
@ -472,7 +472,7 @@ func (agent *ActionAgent) tabletExternallyReparentedLocked(actionTimeout time.Du
// so we will always return true, so the tablet record is re-read // so we will always return true, so the tablet record is re-read
// by the agent. // by the agent.
event.DispatchUpdate(ev, "mark ourself as new master") event.DispatchUpdate(ev, "mark ourself as new master")
err = agent.updateReplicationGraphForPromotedSlave(tablet) err = agent.updateReplicationGraphForPromotedSlave(ctx, tablet)
if err != nil { if err != nil {
// This suggests we can't talk to topo server. This is bad. // This suggests we can't talk to topo server. This is bad.
return true, fmt.Errorf("updateReplicationGraphForPromotedSlave failed: %v", err) return true, fmt.Errorf("updateReplicationGraphForPromotedSlave failed: %v", err)
@ -493,7 +493,7 @@ func (agent *ActionAgent) tabletExternallyReparentedLocked(actionTimeout time.Du
logger := logutil.NewConsoleLogger() logger := logutil.NewConsoleLogger()
tmc := tmclient.NewTabletManagerClient() tmc := tmclient.NewTabletManagerClient()
topotools.RestartSlavesExternal(agent.TopoServer, logger, slaveTabletMap, masterTabletMap, masterElectTablet.Alias, func(ti *topo.TabletInfo, swrd *actionnode.SlaveWasRestartedArgs) error { topotools.RestartSlavesExternal(agent.TopoServer, logger, slaveTabletMap, masterTabletMap, masterElectTablet.Alias, func(ti *topo.TabletInfo, swrd *actionnode.SlaveWasRestartedArgs) error {
return tmc.SlaveWasRestarted(ti, swrd, actionTimeout) return tmc.SlaveWasRestarted(ctx, ti, swrd, actionTimeout)
}) })
// Compute the list of Cells we need to rebuild: old master and // Compute the list of Cells we need to rebuild: old master and
@ -507,14 +507,14 @@ func (agent *ActionAgent) tabletExternallyReparentedLocked(actionTimeout time.Du
event.DispatchUpdate(ev, "updating shard record") event.DispatchUpdate(ev, "updating shard record")
log.Infof("Updating Shard's MasterAlias record") log.Infof("Updating Shard's MasterAlias record")
shardInfo.MasterAlias = tablet.Alias shardInfo.MasterAlias = tablet.Alias
if err = topo.UpdateShard(agent.TopoServer, shardInfo); err != nil { if err = topo.UpdateShard(ctx, agent.TopoServer, shardInfo); err != nil {
return true, err return true, err
} }
// and rebuild the shard serving graph // and rebuild the shard serving graph
event.DispatchUpdate(ev, "rebuilding shard serving graph") event.DispatchUpdate(ev, "rebuilding shard serving graph")
log.Infof("Rebuilding shard serving graph data") log.Infof("Rebuilding shard serving graph data")
if _, err = topotools.RebuildShard(logger, agent.TopoServer, tablet.Keyspace, tablet.Shard, cells, agent.LockTimeout, interrupted); err != nil { if _, err = topotools.RebuildShard(ctx, logger, agent.TopoServer, tablet.Keyspace, tablet.Shard, cells, agent.LockTimeout, interrupted); err != nil {
return true, err return true, err
} }
@ -524,20 +524,20 @@ func (agent *ActionAgent) tabletExternallyReparentedLocked(actionTimeout time.Du
// GetSlaves returns the address of all the slaves // GetSlaves returns the address of all the slaves
// Should be called under RpcWrap. // Should be called under RpcWrap.
func (agent *ActionAgent) GetSlaves() ([]string, error) { func (agent *ActionAgent) GetSlaves(ctx context.Context) ([]string, error) {
return agent.Mysqld.FindSlaves() return agent.Mysqld.FindSlaves()
} }
// WaitBlpPosition waits until a specific filtered replication position is // WaitBlpPosition waits until a specific filtered replication position is
// reached. // reached.
// Should be called under RpcWrapLock. // Should be called under RpcWrapLock.
func (agent *ActionAgent) WaitBlpPosition(blpPosition *blproto.BlpPosition, waitTime time.Duration) error { func (agent *ActionAgent) WaitBlpPosition(ctx context.Context, blpPosition *blproto.BlpPosition, waitTime time.Duration) error {
return agent.Mysqld.WaitBlpPosition(blpPosition, waitTime) return agent.Mysqld.WaitBlpPosition(blpPosition, waitTime)
} }
// StopBlp stops the binlog players, and return their positions. // StopBlp stops the binlog players, and return their positions.
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) StopBlp() (*blproto.BlpPositionList, error) { func (agent *ActionAgent) StopBlp(ctx context.Context) (*blproto.BlpPositionList, error) {
if agent.BinlogPlayerMap == nil { if agent.BinlogPlayerMap == nil {
return nil, fmt.Errorf("No BinlogPlayerMap configured") return nil, fmt.Errorf("No BinlogPlayerMap configured")
} }
@ -547,7 +547,7 @@ func (agent *ActionAgent) StopBlp() (*blproto.BlpPositionList, error) {
// StartBlp starts the binlog players // StartBlp starts the binlog players
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) StartBlp() error { func (agent *ActionAgent) StartBlp(ctx context.Context) error {
if agent.BinlogPlayerMap == nil { if agent.BinlogPlayerMap == nil {
return fmt.Errorf("No BinlogPlayerMap configured") return fmt.Errorf("No BinlogPlayerMap configured")
} }
@ -557,7 +557,7 @@ func (agent *ActionAgent) StartBlp() error {
// RunBlpUntil runs the binlog player server until the position is reached, // RunBlpUntil runs the binlog player server until the position is reached,
// and returns the current mysql master replication position. // and returns the current mysql master replication position.
func (agent *ActionAgent) RunBlpUntil(bpl *blproto.BlpPositionList, waitTime time.Duration) (*myproto.ReplicationPosition, error) { func (agent *ActionAgent) RunBlpUntil(ctx context.Context, bpl *blproto.BlpPositionList, waitTime time.Duration) (*myproto.ReplicationPosition, error) {
if agent.BinlogPlayerMap == nil { if agent.BinlogPlayerMap == nil {
return nil, fmt.Errorf("No BinlogPlayerMap configured") return nil, fmt.Errorf("No BinlogPlayerMap configured")
} }
@ -574,7 +574,7 @@ func (agent *ActionAgent) RunBlpUntil(bpl *blproto.BlpPositionList, waitTime tim
// DemoteMaster demotes the current master, and marks it read-only in the topo. // DemoteMaster demotes the current master, and marks it read-only in the topo.
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) DemoteMaster() error { func (agent *ActionAgent) DemoteMaster(ctx context.Context) error {
_, err := agent.Mysqld.DemoteMaster() _, err := agent.Mysqld.DemoteMaster()
if err != nil { if err != nil {
return err return err
@ -593,7 +593,7 @@ func (agent *ActionAgent) DemoteMaster() error {
// PromoteSlave transforms the current tablet from a slave to a master. // PromoteSlave transforms the current tablet from a slave to a master.
// It returns the data needed for other tablets to become a slave. // It returns the data needed for other tablets to become a slave.
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) PromoteSlave() (*actionnode.RestartSlaveData, error) { func (agent *ActionAgent) PromoteSlave(ctx context.Context) (*actionnode.RestartSlaveData, error) {
tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias) tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias)
if err != nil { if err != nil {
return nil, err return nil, err
@ -610,23 +610,23 @@ func (agent *ActionAgent) PromoteSlave() (*actionnode.RestartSlaveData, error) {
} }
log.Infof("PromoteSlave response: %v", *rsd) log.Infof("PromoteSlave response: %v", *rsd)
return rsd, agent.updateReplicationGraphForPromotedSlave(tablet) return rsd, agent.updateReplicationGraphForPromotedSlave(ctx, tablet)
} }
// SlaveWasPromoted promotes a slave to master, no questions asked. // SlaveWasPromoted promotes a slave to master, no questions asked.
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) SlaveWasPromoted() error { func (agent *ActionAgent) SlaveWasPromoted(ctx context.Context) error {
tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias) tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias)
if err != nil { if err != nil {
return err return err
} }
return agent.updateReplicationGraphForPromotedSlave(tablet) return agent.updateReplicationGraphForPromotedSlave(ctx, tablet)
} }
// RestartSlave tells the tablet it has a new master // RestartSlave tells the tablet it has a new master
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) RestartSlave(rsd *actionnode.RestartSlaveData) error { func (agent *ActionAgent) RestartSlave(ctx context.Context, rsd *actionnode.RestartSlaveData) error {
tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias) tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias)
if err != nil { if err != nil {
return err return err
@ -655,7 +655,7 @@ func (agent *ActionAgent) RestartSlave(rsd *actionnode.RestartSlaveData) error {
} }
// Once this action completes, update authoritive tablet node first. // Once this action completes, update authoritive tablet node first.
tablet.Parent = rsd.Parent tablet.Parent = rsd.Parent
err = topo.UpdateTablet(agent.TopoServer, tablet) err = topo.UpdateTablet(ctx, agent.TopoServer, tablet)
if err != nil { if err != nil {
return err return err
} }
@ -667,7 +667,7 @@ func (agent *ActionAgent) RestartSlave(rsd *actionnode.RestartSlaveData) error {
// Complete the special orphan accounting. // Complete the special orphan accounting.
if tablet.Type == topo.TYPE_LAG_ORPHAN { if tablet.Type == topo.TYPE_LAG_ORPHAN {
tablet.Type = topo.TYPE_LAG tablet.Type = topo.TYPE_LAG
err = topo.UpdateTablet(agent.TopoServer, tablet) err = topo.UpdateTablet(ctx, agent.TopoServer, tablet)
if err != nil { if err != nil {
return err return err
} }
@ -686,7 +686,7 @@ func (agent *ActionAgent) RestartSlave(rsd *actionnode.RestartSlaveData) error {
// Insert the new tablet location in the replication graph now that // Insert the new tablet location in the replication graph now that
// we've updated the tablet. // we've updated the tablet.
err = topo.UpdateTabletReplicationData(agent.TopoServer, tablet.Tablet) err = topo.UpdateTabletReplicationData(ctx, agent.TopoServer, tablet.Tablet)
if err != nil && err != topo.ErrNodeExists { if err != nil && err != topo.ErrNodeExists {
return err return err
} }
@ -696,7 +696,7 @@ func (agent *ActionAgent) RestartSlave(rsd *actionnode.RestartSlaveData) error {
// SlaveWasRestarted updates the parent record for a tablet. // SlaveWasRestarted updates the parent record for a tablet.
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) SlaveWasRestarted(swrd *actionnode.SlaveWasRestartedArgs) error { func (agent *ActionAgent) SlaveWasRestarted(ctx context.Context, swrd *actionnode.SlaveWasRestartedArgs) error {
tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias) tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias)
if err != nil { if err != nil {
return err return err
@ -708,14 +708,14 @@ func (agent *ActionAgent) SlaveWasRestarted(swrd *actionnode.SlaveWasRestartedAr
tablet.Type = topo.TYPE_SPARE tablet.Type = topo.TYPE_SPARE
tablet.State = topo.STATE_READ_ONLY tablet.State = topo.STATE_READ_ONLY
} }
err = topo.UpdateTablet(agent.TopoServer, tablet) err = topo.UpdateTablet(ctx, agent.TopoServer, tablet)
if err != nil { if err != nil {
return err return err
} }
// Update the new tablet location in the replication graph now that // Update the new tablet location in the replication graph now that
// we've updated the tablet. // we've updated the tablet.
err = topo.UpdateTabletReplicationData(agent.TopoServer, tablet.Tablet) err = topo.UpdateTabletReplicationData(ctx, agent.TopoServer, tablet.Tablet)
if err != nil && err != topo.ErrNodeExists { if err != nil && err != topo.ErrNodeExists {
return err return err
} }
@ -726,20 +726,20 @@ func (agent *ActionAgent) SlaveWasRestarted(swrd *actionnode.SlaveWasRestartedAr
// BreakSlaves will tinker with the replication stream in a way that // BreakSlaves will tinker with the replication stream in a way that
// will stop all the slaves. // will stop all the slaves.
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) BreakSlaves() error { func (agent *ActionAgent) BreakSlaves(ctx context.Context) error {
return agent.Mysqld.BreakSlaves() return agent.Mysqld.BreakSlaves()
} }
// updateReplicationGraphForPromotedSlave makes sure the newly promoted slave // updateReplicationGraphForPromotedSlave makes sure the newly promoted slave
// is correctly represented in the replication graph // is correctly represented in the replication graph
func (agent *ActionAgent) updateReplicationGraphForPromotedSlave(tablet *topo.TabletInfo) error { func (agent *ActionAgent) updateReplicationGraphForPromotedSlave(ctx context.Context, tablet *topo.TabletInfo) error {
// Update tablet regardless - trend towards consistency. // Update tablet regardless - trend towards consistency.
tablet.State = topo.STATE_READ_WRITE tablet.State = topo.STATE_READ_WRITE
tablet.Type = topo.TYPE_MASTER tablet.Type = topo.TYPE_MASTER
tablet.Parent.Cell = "" tablet.Parent.Cell = ""
tablet.Parent.Uid = topo.NO_TABLET tablet.Parent.Uid = topo.NO_TABLET
tablet.Health = nil tablet.Health = nil
err := topo.UpdateTablet(agent.TopoServer, tablet) err := topo.UpdateTablet(ctx, agent.TopoServer, tablet)
if err != nil { if err != nil {
return err return err
} }
@ -750,7 +750,7 @@ func (agent *ActionAgent) updateReplicationGraphForPromotedSlave(tablet *topo.Ta
// Insert the new tablet location in the replication graph now that // Insert the new tablet location in the replication graph now that
// we've updated the tablet. // we've updated the tablet.
err = topo.UpdateTabletReplicationData(agent.TopoServer, tablet.Tablet) err = topo.UpdateTabletReplicationData(ctx, agent.TopoServer, tablet.Tablet)
if err != nil && err != topo.ErrNodeExists { if err != nil && err != topo.ErrNodeExists {
return err return err
} }
@ -764,7 +764,7 @@ func (agent *ActionAgent) updateReplicationGraphForPromotedSlave(tablet *topo.Ta
// Snapshot takes a db snapshot // Snapshot takes a db snapshot
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) Snapshot(args *actionnode.SnapshotArgs, logger logutil.Logger) (*actionnode.SnapshotReply, error) { func (agent *ActionAgent) Snapshot(ctx context.Context, args *actionnode.SnapshotArgs, logger logutil.Logger) (*actionnode.SnapshotReply, error) {
// update our type to TYPE_BACKUP // update our type to TYPE_BACKUP
tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias) tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias)
if err != nil { if err != nil {
@ -782,7 +782,7 @@ func (agent *ActionAgent) Snapshot(args *actionnode.SnapshotArgs, logger logutil
// There is a legitimate reason to force in the case of a single // There is a legitimate reason to force in the case of a single
// master. // master.
tablet.Tablet.Type = topo.TYPE_BACKUP tablet.Tablet.Type = topo.TYPE_BACKUP
err = topo.UpdateTablet(agent.TopoServer, tablet) err = topo.UpdateTablet(ctx, agent.TopoServer, tablet)
} else { } else {
err = topotools.ChangeType(agent.TopoServer, tablet.Alias, topo.TYPE_BACKUP, make(map[string]string), true /*runHooks*/) err = topotools.ChangeType(agent.TopoServer, tablet.Alias, topo.TYPE_BACKUP, make(map[string]string), true /*runHooks*/)
} }
@ -816,7 +816,7 @@ func (agent *ActionAgent) Snapshot(args *actionnode.SnapshotArgs, logger logutil
if tablet.Parent.Uid == topo.NO_TABLET && args.ForceMasterSnapshot && newType != topo.TYPE_SNAPSHOT_SOURCE { if tablet.Parent.Uid == topo.NO_TABLET && args.ForceMasterSnapshot && newType != topo.TYPE_SNAPSHOT_SOURCE {
log.Infof("force change type backup -> master: %v", tablet.Alias) log.Infof("force change type backup -> master: %v", tablet.Alias)
tablet.Tablet.Type = topo.TYPE_MASTER tablet.Tablet.Type = topo.TYPE_MASTER
err = topo.UpdateTablet(agent.TopoServer, tablet) err = topo.UpdateTablet(ctx, agent.TopoServer, tablet)
} else { } else {
err = topotools.ChangeType(agent.TopoServer, tablet.Alias, newType, nil, true /*runHooks*/) err = topotools.ChangeType(agent.TopoServer, tablet.Alias, newType, nil, true /*runHooks*/)
} }
@ -850,7 +850,7 @@ func (agent *ActionAgent) Snapshot(args *actionnode.SnapshotArgs, logger logutil
// SnapshotSourceEnd restores the state of the server after a // SnapshotSourceEnd restores the state of the server after a
// Snapshot(server_mode =true) // Snapshot(server_mode =true)
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) SnapshotSourceEnd(args *actionnode.SnapshotSourceEndArgs) error { func (agent *ActionAgent) SnapshotSourceEnd(ctx context.Context, args *actionnode.SnapshotSourceEndArgs) error {
tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias) tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias)
if err != nil { if err != nil {
return err return err
@ -868,7 +868,7 @@ func (agent *ActionAgent) SnapshotSourceEnd(args *actionnode.SnapshotSourceEndAr
if args.OriginalType == topo.TYPE_MASTER { if args.OriginalType == topo.TYPE_MASTER {
// force the master update // force the master update
tablet.Tablet.Type = topo.TYPE_MASTER tablet.Tablet.Type = topo.TYPE_MASTER
err = topo.UpdateTablet(agent.TopoServer, tablet) err = topo.UpdateTablet(ctx, agent.TopoServer, tablet)
} else { } else {
err = topotools.ChangeType(agent.TopoServer, tablet.Alias, args.OriginalType, make(map[string]string), true /*runHooks*/) err = topotools.ChangeType(agent.TopoServer, tablet.Alias, args.OriginalType, make(map[string]string), true /*runHooks*/)
} }
@ -882,7 +882,7 @@ func (agent *ActionAgent) SnapshotSourceEnd(args *actionnode.SnapshotSourceEndAr
// a successful ReserveForRestore but a failed Snapshot) // a successful ReserveForRestore but a failed Snapshot)
// - to SCRAP if something in the process on the target host fails // - to SCRAP if something in the process on the target host fails
// - to SPARE if the clone works // - to SPARE if the clone works
func (agent *ActionAgent) changeTypeToRestore(tablet, sourceTablet *topo.TabletInfo, parentAlias topo.TabletAlias, keyRange key.KeyRange) error { func (agent *ActionAgent) changeTypeToRestore(ctx context.Context, tablet, sourceTablet *topo.TabletInfo, parentAlias topo.TabletAlias, keyRange key.KeyRange) error {
// run the optional preflight_assigned hook // run the optional preflight_assigned hook
hk := hook.NewSimpleHook("preflight_assigned") hk := hook.NewSimpleHook("preflight_assigned")
topotools.ConfigureTabletHook(hk, agent.TabletAlias) topotools.ConfigureTabletHook(hk, agent.TabletAlias)
@ -897,18 +897,18 @@ func (agent *ActionAgent) changeTypeToRestore(tablet, sourceTablet *topo.TabletI
tablet.Type = topo.TYPE_RESTORE tablet.Type = topo.TYPE_RESTORE
tablet.KeyRange = keyRange tablet.KeyRange = keyRange
tablet.DbNameOverride = sourceTablet.DbNameOverride tablet.DbNameOverride = sourceTablet.DbNameOverride
if err := topo.UpdateTablet(agent.TopoServer, tablet); err != nil { if err := topo.UpdateTablet(ctx, agent.TopoServer, tablet); err != nil {
return err return err
} }
// and create the replication graph items // and create the replication graph items
return topo.UpdateTabletReplicationData(agent.TopoServer, tablet.Tablet) return topo.UpdateTabletReplicationData(ctx, agent.TopoServer, tablet.Tablet)
} }
// ReserveForRestore reserves the current tablet for an upcoming // ReserveForRestore reserves the current tablet for an upcoming
// restore operation. // restore operation.
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) ReserveForRestore(args *actionnode.ReserveForRestoreArgs) error { func (agent *ActionAgent) ReserveForRestore(ctx context.Context, args *actionnode.ReserveForRestoreArgs) error {
// first check mysql, no need to go further if we can't restore // first check mysql, no need to go further if we can't restore
if err := agent.Mysqld.ValidateCloneTarget(agent.hookExtraEnv()); err != nil { if err := agent.Mysqld.ValidateCloneTarget(agent.hookExtraEnv()); err != nil {
return err return err
@ -939,7 +939,7 @@ func (agent *ActionAgent) ReserveForRestore(args *actionnode.ReserveForRestoreAr
parentAlias = sourceTablet.Parent parentAlias = sourceTablet.Parent
} }
return agent.changeTypeToRestore(tablet, sourceTablet, parentAlias, sourceTablet.KeyRange) return agent.changeTypeToRestore(ctx, tablet, sourceTablet, parentAlias, sourceTablet.KeyRange)
} }
func fetchAndParseJsonFile(addr, filename string, result interface{}) error { func fetchAndParseJsonFile(addr, filename string, result interface{}) error {
@ -969,7 +969,7 @@ func fetchAndParseJsonFile(addr, filename string, result interface{}) error {
// Restart mysqld and replication. // Restart mysqld and replication.
// Put tablet into the replication graph as a spare. // Put tablet into the replication graph as a spare.
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) Restore(args *actionnode.RestoreArgs, logger logutil.Logger) error { func (agent *ActionAgent) Restore(ctx context.Context, args *actionnode.RestoreArgs, logger logutil.Logger) error {
// read our current tablet, verify its state // read our current tablet, verify its state
tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias) tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias)
if err != nil { if err != nil {
@ -1009,7 +1009,7 @@ func (agent *ActionAgent) Restore(args *actionnode.RestoreArgs, logger logutil.L
} }
if !args.WasReserved { if !args.WasReserved {
if err := agent.changeTypeToRestore(tablet, sourceTablet, parentTablet.Alias, sourceTablet.KeyRange); err != nil { if err := agent.changeTypeToRestore(ctx, tablet, sourceTablet, parentTablet.Alias, sourceTablet.KeyRange); err != nil {
return err return err
} }
} }
@ -1028,7 +1028,7 @@ func (agent *ActionAgent) Restore(args *actionnode.RestoreArgs, logger logutil.L
} }
// reload the schema // reload the schema
agent.ReloadSchema() agent.ReloadSchema(ctx)
// change to TYPE_SPARE, we're done! // change to TYPE_SPARE, we're done!
return topotools.ChangeType(agent.TopoServer, agent.TabletAlias, topo.TYPE_SPARE, nil, true) return topotools.ChangeType(agent.TopoServer, agent.TabletAlias, topo.TYPE_SPARE, nil, true)
@ -1036,7 +1036,7 @@ func (agent *ActionAgent) Restore(args *actionnode.RestoreArgs, logger logutil.L
// MultiSnapshot takes a multi-part snapshot // MultiSnapshot takes a multi-part snapshot
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) MultiSnapshot(args *actionnode.MultiSnapshotArgs, logger logutil.Logger) (*actionnode.MultiSnapshotReply, error) { func (agent *ActionAgent) MultiSnapshot(ctx context.Context, args *actionnode.MultiSnapshotArgs, logger logutil.Logger) (*actionnode.MultiSnapshotReply, error) {
tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias) tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias)
if err != nil { if err != nil {
return nil, err return nil, err
@ -1071,7 +1071,7 @@ func (agent *ActionAgent) MultiSnapshot(args *actionnode.MultiSnapshotArgs, logg
// MultiRestore performs the multi-part restore. // MultiRestore performs the multi-part restore.
// Should be called under RpcWrapLockAction. // Should be called under RpcWrapLockAction.
func (agent *ActionAgent) MultiRestore(args *actionnode.MultiRestoreArgs, logger logutil.Logger) error { func (agent *ActionAgent) MultiRestore(ctx context.Context, args *actionnode.MultiRestoreArgs, logger logutil.Logger) error {
// read our current tablet, verify its state // read our current tablet, verify its state
// we only support restoring to the master or active replicas // we only support restoring to the master or active replicas
tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias) tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias)
@ -1104,7 +1104,7 @@ func (agent *ActionAgent) MultiRestore(args *actionnode.MultiRestoreArgs, logger
// change type to restore, no change to replication graph // change type to restore, no change to replication graph
originalType := tablet.Type originalType := tablet.Type
tablet.Type = topo.TYPE_RESTORE tablet.Type = topo.TYPE_RESTORE
err = topo.UpdateTablet(agent.TopoServer, tablet) err = topo.UpdateTablet(ctx, agent.TopoServer, tablet)
if err != nil { if err != nil {
return err return err
} }
@ -1157,7 +1157,7 @@ func (agent *ActionAgent) MultiRestore(args *actionnode.MultiRestoreArgs, logger
} }
// reload the schema // reload the schema
agent.ReloadSchema() agent.ReloadSchema(ctx)
// restart replication // restart replication
if topo.IsSlaveType(originalType) { if topo.IsSlaveType(originalType) {
@ -1168,5 +1168,5 @@ func (agent *ActionAgent) MultiRestore(args *actionnode.MultiRestoreArgs, logger
// restore type back // restore type back
tablet.Type = originalType tablet.Type = originalType
return topo.UpdateTablet(agent.TopoServer, tablet) return topo.UpdateTablet(ctx, agent.TopoServer, tablet)
} }

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

@ -92,12 +92,12 @@ func compareLoggedStuff(t *testing.T, name string, logChannel <-chan *logutil.Lo
// Various read-only methods // Various read-only methods
// //
func (fra *fakeRpcAgent) Ping(args string) string { func (fra *fakeRpcAgent) Ping(ctx context.Context, args string) string {
return args return args
} }
func agentRpcTestPing(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestPing(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.Ping(ti, time.Minute) err := client.Ping(ctx, ti, time.Minute)
if err != nil { if err != nil {
t.Errorf("Ping failed: %v", err) t.Errorf("Ping failed: %v", err)
} }
@ -130,15 +130,15 @@ var testGetSchemaReply = &myproto.SchemaDefinition{
Version: "xxx", Version: "xxx",
} }
func (fra *fakeRpcAgent) GetSchema(tables, excludeTables []string, includeViews bool) (*myproto.SchemaDefinition, error) { func (fra *fakeRpcAgent) GetSchema(ctx context.Context, tables, excludeTables []string, includeViews bool) (*myproto.SchemaDefinition, error) {
compare(fra.t, "GetSchema tables", tables, testGetSchemaTables) compare(fra.t, "GetSchema tables", tables, testGetSchemaTables)
compare(fra.t, "GetSchema excludeTables", excludeTables, testGetSchemaExcludeTables) compare(fra.t, "GetSchema excludeTables", excludeTables, testGetSchemaExcludeTables)
compareBool(fra.t, "GetSchema includeViews", includeViews) compareBool(fra.t, "GetSchema includeViews", includeViews)
return testGetSchemaReply, nil return testGetSchemaReply, nil
} }
func agentRpcTestGetSchema(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestGetSchema(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
result, err := client.GetSchema(ti, testGetSchemaTables, testGetSchemaExcludeTables, true, time.Minute) result, err := client.GetSchema(ctx, ti, testGetSchemaTables, testGetSchemaExcludeTables, true, time.Minute)
compareError(t, "GetSchema", err, result, testGetSchemaReply) compareError(t, "GetSchema", err, result, testGetSchemaReply)
} }
@ -177,12 +177,12 @@ var testGetPermissionsReply = &myproto.Permissions{
}, },
} }
func (fra *fakeRpcAgent) GetPermissions() (*myproto.Permissions, error) { func (fra *fakeRpcAgent) GetPermissions(ctx context.Context) (*myproto.Permissions, error) {
return testGetPermissionsReply, nil return testGetPermissionsReply, nil
} }
func agentRpcTestGetPermissions(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestGetPermissions(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
result, err := client.GetPermissions(ti, time.Minute) result, err := client.GetPermissions(ctx, ti, time.Minute)
compareError(t, "GetPermissions", err, result, testGetPermissionsReply) compareError(t, "GetPermissions", err, result, testGetPermissionsReply)
} }
@ -192,21 +192,21 @@ func agentRpcTestGetPermissions(t *testing.T, client tmclient.TabletManagerClien
var testSetReadOnlyExpectedValue bool var testSetReadOnlyExpectedValue bool
func (fra *fakeRpcAgent) SetReadOnly(rdonly bool) error { func (fra *fakeRpcAgent) SetReadOnly(ctx context.Context, rdonly bool) error {
if rdonly != testSetReadOnlyExpectedValue { if rdonly != testSetReadOnlyExpectedValue {
fra.t.Errorf("Wrong SetReadOnly value: got %v expected %v", rdonly, testSetReadOnlyExpectedValue) fra.t.Errorf("Wrong SetReadOnly value: got %v expected %v", rdonly, testSetReadOnlyExpectedValue)
} }
return nil return nil
} }
func agentRpcTestSetReadOnly(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestSetReadOnly(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
testSetReadOnlyExpectedValue = true testSetReadOnlyExpectedValue = true
err := client.SetReadOnly(ti, time.Minute) err := client.SetReadOnly(ctx, ti, time.Minute)
if err != nil { if err != nil {
t.Errorf("SetReadOnly failed: %v", err) t.Errorf("SetReadOnly failed: %v", err)
} }
testSetReadOnlyExpectedValue = false testSetReadOnlyExpectedValue = false
err = client.SetReadWrite(ti, time.Minute) err = client.SetReadWrite(ctx, ti, time.Minute)
if err != nil { if err != nil {
t.Errorf("SetReadWrite failed: %v", err) t.Errorf("SetReadWrite failed: %v", err)
} }
@ -214,13 +214,13 @@ func agentRpcTestSetReadOnly(t *testing.T, client tmclient.TabletManagerClient,
var testChangeTypeValue = topo.TYPE_REPLICA var testChangeTypeValue = topo.TYPE_REPLICA
func (fra *fakeRpcAgent) ChangeType(tabletType topo.TabletType) error { func (fra *fakeRpcAgent) ChangeType(ctx context.Context, tabletType topo.TabletType) error {
compare(fra.t, "ChangeType tabletType", tabletType, testChangeTypeValue) compare(fra.t, "ChangeType tabletType", tabletType, testChangeTypeValue)
return nil return nil
} }
func agentRpcTestChangeType(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestChangeType(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.ChangeType(ti, testChangeTypeValue, time.Minute) err := client.ChangeType(ctx, ti, testChangeTypeValue, time.Minute)
if err != nil { if err != nil {
t.Errorf("ChangeType failed: %v", err) t.Errorf("ChangeType failed: %v", err)
} }
@ -228,12 +228,12 @@ func agentRpcTestChangeType(t *testing.T, client tmclient.TabletManagerClient, t
var testScrapError = fmt.Errorf("Scrap Failed!") var testScrapError = fmt.Errorf("Scrap Failed!")
func (fra *fakeRpcAgent) Scrap() error { func (fra *fakeRpcAgent) Scrap(ctx context.Context) error {
return testScrapError return testScrapError
} }
func agentRpcTestScrap(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestScrap(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.Scrap(ti, time.Minute) err := client.Scrap(ctx, ti, time.Minute)
if strings.Index(err.Error(), testScrapError.Error()) == -1 { if strings.Index(err.Error(), testScrapError.Error()) == -1 {
t.Errorf("Unexpected Scrap result: got %v expected %v", err, testScrapError) t.Errorf("Unexpected Scrap result: got %v expected %v", err, testScrapError)
} }
@ -241,12 +241,12 @@ func agentRpcTestScrap(t *testing.T, client tmclient.TabletManagerClient, ti *to
var testSleepDuration = time.Minute var testSleepDuration = time.Minute
func (fra *fakeRpcAgent) Sleep(duration time.Duration) { func (fra *fakeRpcAgent) Sleep(ctx context.Context, duration time.Duration) {
compare(fra.t, "Sleep duration", duration, testSleepDuration) compare(fra.t, "Sleep duration", duration, testSleepDuration)
} }
func agentRpcTestSleep(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestSleep(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.Sleep(ti, testSleepDuration, time.Minute) err := client.Sleep(ctx, ti, testSleepDuration, time.Minute)
if err != nil { if err != nil {
t.Errorf("Sleep failed: %v", err) t.Errorf("Sleep failed: %v", err)
} }
@ -266,27 +266,27 @@ var testExecuteHookHookResult = &hook.HookResult{
Stderr: "err", Stderr: "err",
} }
func (fra *fakeRpcAgent) ExecuteHook(hk *hook.Hook) *hook.HookResult { func (fra *fakeRpcAgent) ExecuteHook(ctx context.Context, hk *hook.Hook) *hook.HookResult {
compare(fra.t, "ExecuteHook hook", hk, testExecuteHookHook) compare(fra.t, "ExecuteHook hook", hk, testExecuteHookHook)
return testExecuteHookHookResult return testExecuteHookHookResult
} }
func agentRpcTestExecuteHook(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestExecuteHook(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
hr, err := client.ExecuteHook(ti, testExecuteHookHook, time.Minute) hr, err := client.ExecuteHook(ctx, ti, testExecuteHookHook, time.Minute)
compareError(t, "ExecuteHook", err, hr, testExecuteHookHookResult) compareError(t, "ExecuteHook", err, hr, testExecuteHookHookResult)
} }
var testRefreshStateCalled = false var testRefreshStateCalled = false
func (fra *fakeRpcAgent) RefreshState() { func (fra *fakeRpcAgent) RefreshState(ctx context.Context) {
if testRefreshStateCalled { if testRefreshStateCalled {
fra.t.Errorf("RefreshState called multiple times?") fra.t.Errorf("RefreshState called multiple times?")
} }
testRefreshStateCalled = true testRefreshStateCalled = true
} }
func agentRpcTestRefreshState(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestRefreshState(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.RefreshState(ti, time.Minute) err := client.RefreshState(ctx, ti, time.Minute)
if err != nil { if err != nil {
t.Errorf("RefreshState failed: %v", err) t.Errorf("RefreshState failed: %v", err)
} }
@ -297,12 +297,12 @@ func agentRpcTestRefreshState(t *testing.T, client tmclient.TabletManagerClient,
var testRunHealthCheckValue = topo.TYPE_RDONLY var testRunHealthCheckValue = topo.TYPE_RDONLY
func (fra *fakeRpcAgent) RunHealthCheck(targetTabletType topo.TabletType) { func (fra *fakeRpcAgent) RunHealthCheck(ctx context.Context, targetTabletType topo.TabletType) {
compare(fra.t, "RunHealthCheck tabletType", targetTabletType, testRunHealthCheckValue) compare(fra.t, "RunHealthCheck tabletType", targetTabletType, testRunHealthCheckValue)
} }
func agentRpcTestRunHealthCheck(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestRunHealthCheck(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.RunHealthCheck(ti, testRunHealthCheckValue, time.Minute) err := client.RunHealthCheck(ctx, ti, testRunHealthCheckValue, time.Minute)
if err != nil { if err != nil {
t.Errorf("RunHealthCheck failed: %v", err) t.Errorf("RunHealthCheck failed: %v", err)
} }
@ -310,15 +310,15 @@ func agentRpcTestRunHealthCheck(t *testing.T, client tmclient.TabletManagerClien
var testReloadSchemaCalled = false var testReloadSchemaCalled = false
func (fra *fakeRpcAgent) ReloadSchema() { func (fra *fakeRpcAgent) ReloadSchema(ctx context.Context) {
if testReloadSchemaCalled { if testReloadSchemaCalled {
fra.t.Errorf("ReloadSchema called multiple times?") fra.t.Errorf("ReloadSchema called multiple times?")
} }
testReloadSchemaCalled = true testReloadSchemaCalled = true
} }
func agentRpcTestReloadSchema(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestReloadSchema(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.ReloadSchema(ti, time.Minute) err := client.ReloadSchema(ctx, ti, time.Minute)
if err != nil { if err != nil {
t.Errorf("ReloadSchema failed: %v", err) t.Errorf("ReloadSchema failed: %v", err)
} }
@ -333,13 +333,13 @@ var testSchemaChangeResult = &myproto.SchemaChangeResult{
AfterSchema: testGetSchemaReply, AfterSchema: testGetSchemaReply,
} }
func (fra *fakeRpcAgent) PreflightSchema(change string) (*myproto.SchemaChangeResult, error) { func (fra *fakeRpcAgent) PreflightSchema(ctx context.Context, change string) (*myproto.SchemaChangeResult, error) {
compare(fra.t, "PreflightSchema result", change, testPreflightSchema) compare(fra.t, "PreflightSchema result", change, testPreflightSchema)
return testSchemaChangeResult, nil return testSchemaChangeResult, nil
} }
func agentRpcTestPreflightSchema(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestPreflightSchema(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
scr, err := client.PreflightSchema(ti, testPreflightSchema, time.Minute) scr, err := client.PreflightSchema(ctx, ti, testPreflightSchema, time.Minute)
compareError(t, "PreflightSchema", err, scr, testSchemaChangeResult) compareError(t, "PreflightSchema", err, scr, testSchemaChangeResult)
} }
@ -351,13 +351,13 @@ var testSchemaChange = &myproto.SchemaChange{
AfterSchema: testGetSchemaReply, AfterSchema: testGetSchemaReply,
} }
func (fra *fakeRpcAgent) ApplySchema(change *myproto.SchemaChange) (*myproto.SchemaChangeResult, error) { func (fra *fakeRpcAgent) ApplySchema(ctx context.Context, change *myproto.SchemaChange) (*myproto.SchemaChangeResult, error) {
compare(fra.t, "ApplySchema change", change, testSchemaChange) compare(fra.t, "ApplySchema change", change, testSchemaChange)
return testSchemaChangeResult, nil return testSchemaChangeResult, nil
} }
func agentRpcTestApplySchema(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestApplySchema(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
scr, err := client.ApplySchema(ti, testSchemaChange, time.Minute) scr, err := client.ApplySchema(ctx, ti, testSchemaChange, time.Minute)
compareError(t, "ApplySchema", err, scr, testSchemaChangeResult) compareError(t, "ApplySchema", err, scr, testSchemaChangeResult)
} }
@ -383,7 +383,7 @@ var testExecuteFetchResult = &mproto.QueryResult{
}, },
} }
func (fra *fakeRpcAgent) ExecuteFetch(query string, maxrows int, wantFields, disableBinlogs bool) (*mproto.QueryResult, error) { func (fra *fakeRpcAgent) ExecuteFetch(ctx context.Context, query string, maxrows int, wantFields, disableBinlogs bool) (*mproto.QueryResult, error) {
compare(fra.t, "ExecuteFetch query", query, testExecuteFetchQuery) compare(fra.t, "ExecuteFetch query", query, testExecuteFetchQuery)
compare(fra.t, "ExecuteFetch maxrows", maxrows, testExecuteFetchMaxRows) compare(fra.t, "ExecuteFetch maxrows", maxrows, testExecuteFetchMaxRows)
compareBool(fra.t, "ExecuteFetch wantFields", wantFields) compareBool(fra.t, "ExecuteFetch wantFields", wantFields)
@ -391,8 +391,8 @@ func (fra *fakeRpcAgent) ExecuteFetch(query string, maxrows int, wantFields, dis
return testExecuteFetchResult, nil return testExecuteFetchResult, nil
} }
func agentRpcTestExecuteFetch(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestExecuteFetch(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
qr, err := client.ExecuteFetch(ti, testExecuteFetchQuery, testExecuteFetchMaxRows, true, true, time.Minute) qr, err := client.ExecuteFetch(ctx, ti, testExecuteFetchQuery, testExecuteFetchMaxRows, true, true, time.Minute)
compareError(t, "ExecuteFetch", err, qr, testExecuteFetchResult) compareError(t, "ExecuteFetch", err, qr, testExecuteFetchResult)
} }
@ -415,12 +415,12 @@ var testReplicationStatus = &myproto.ReplicationStatus{
MasterConnectRetry: 12, MasterConnectRetry: 12,
} }
func (fra *fakeRpcAgent) SlaveStatus() (*myproto.ReplicationStatus, error) { func (fra *fakeRpcAgent) SlaveStatus(ctx context.Context) (*myproto.ReplicationStatus, error) {
return testReplicationStatus, nil return testReplicationStatus, nil
} }
func agentRpcTestSlaveStatus(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestSlaveStatus(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rs, err := client.SlaveStatus(ti, time.Minute) rs, err := client.SlaveStatus(ctx, ti, time.Minute)
compareError(t, "SlaveStatus", err, rs, testReplicationStatus) compareError(t, "SlaveStatus", err, rs, testReplicationStatus)
} }
@ -433,23 +433,23 @@ var testReplicationPosition = myproto.ReplicationPosition{
} }
var testWaitSlavePositionWaitTimeout = time.Hour var testWaitSlavePositionWaitTimeout = time.Hour
func (fra *fakeRpcAgent) WaitSlavePosition(position myproto.ReplicationPosition, waitTimeout time.Duration) (*myproto.ReplicationStatus, error) { func (fra *fakeRpcAgent) WaitSlavePosition(ctx context.Context, position myproto.ReplicationPosition, waitTimeout time.Duration) (*myproto.ReplicationStatus, error) {
compare(fra.t, "WaitSlavePosition position", position, testReplicationPosition) compare(fra.t, "WaitSlavePosition position", position, testReplicationPosition)
compare(fra.t, "WaitSlavePosition waitTimeout", waitTimeout, testWaitSlavePositionWaitTimeout) compare(fra.t, "WaitSlavePosition waitTimeout", waitTimeout, testWaitSlavePositionWaitTimeout)
return testReplicationStatus, nil return testReplicationStatus, nil
} }
func agentRpcTestWaitSlavePosition(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestWaitSlavePosition(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rs, err := client.WaitSlavePosition(ti, testReplicationPosition, testWaitSlavePositionWaitTimeout) rs, err := client.WaitSlavePosition(ctx, ti, testReplicationPosition, testWaitSlavePositionWaitTimeout)
compareError(t, "WaitSlavePosition", err, rs, testReplicationStatus) compareError(t, "WaitSlavePosition", err, rs, testReplicationStatus)
} }
func (fra *fakeRpcAgent) MasterPosition() (myproto.ReplicationPosition, error) { func (fra *fakeRpcAgent) MasterPosition(ctx context.Context) (myproto.ReplicationPosition, error) {
return testReplicationPosition, nil return testReplicationPosition, nil
} }
func agentRpcTestMasterPosition(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestMasterPosition(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rs, err := client.MasterPosition(ti, time.Minute) rs, err := client.MasterPosition(ctx, ti, time.Minute)
compareError(t, "MasterPosition", err, rs, testReplicationPosition) compareError(t, "MasterPosition", err, rs, testReplicationPosition)
} }
@ -464,73 +464,73 @@ var testRestartSlaveData = &actionnode.RestartSlaveData{
Force: true, Force: true,
} }
func (fra *fakeRpcAgent) ReparentPosition(rp *myproto.ReplicationPosition) (*actionnode.RestartSlaveData, error) { func (fra *fakeRpcAgent) ReparentPosition(ctx context.Context, rp *myproto.ReplicationPosition) (*actionnode.RestartSlaveData, error) {
compare(fra.t, "ReparentPosition position", rp.GTIDSet, testReplicationPosition.GTIDSet) compare(fra.t, "ReparentPosition position", rp.GTIDSet, testReplicationPosition.GTIDSet)
return testRestartSlaveData, nil return testRestartSlaveData, nil
} }
func agentRpcTestReparentPosition(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestReparentPosition(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rsd, err := client.ReparentPosition(ti, &testReplicationPosition, time.Minute) rsd, err := client.ReparentPosition(ctx, ti, &testReplicationPosition, time.Minute)
compareError(t, "ReparentPosition", err, rsd, testRestartSlaveData) compareError(t, "ReparentPosition", err, rsd, testRestartSlaveData)
} }
var testStopSlaveCalled = false var testStopSlaveCalled = false
func (fra *fakeRpcAgent) StopSlave() error { func (fra *fakeRpcAgent) StopSlave(ctx context.Context) error {
testStopSlaveCalled = true testStopSlaveCalled = true
return nil return nil
} }
func agentRpcTestStopSlave(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestStopSlave(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.StopSlave(ti, time.Minute) err := client.StopSlave(ctx, ti, time.Minute)
compareError(t, "StopSlave", err, true, testStopSlaveCalled) compareError(t, "StopSlave", err, true, testStopSlaveCalled)
} }
var testStopSlaveMinimumWaitTime = time.Hour var testStopSlaveMinimumWaitTime = time.Hour
func (fra *fakeRpcAgent) StopSlaveMinimum(position myproto.ReplicationPosition, waitTime time.Duration) (*myproto.ReplicationStatus, error) { func (fra *fakeRpcAgent) StopSlaveMinimum(ctx context.Context, position myproto.ReplicationPosition, waitTime time.Duration) (*myproto.ReplicationStatus, error) {
compare(fra.t, "StopSlaveMinimum position", position.GTIDSet, testReplicationPosition.GTIDSet) compare(fra.t, "StopSlaveMinimum position", position.GTIDSet, testReplicationPosition.GTIDSet)
compare(fra.t, "StopSlaveMinimum waitTime", waitTime, testStopSlaveMinimumWaitTime) compare(fra.t, "StopSlaveMinimum waitTime", waitTime, testStopSlaveMinimumWaitTime)
return testReplicationStatus, nil return testReplicationStatus, nil
} }
func agentRpcTestStopSlaveMinimum(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestStopSlaveMinimum(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rs, err := client.StopSlaveMinimum(ti, testReplicationPosition, testStopSlaveMinimumWaitTime) rs, err := client.StopSlaveMinimum(ctx, ti, testReplicationPosition, testStopSlaveMinimumWaitTime)
compareError(t, "StopSlave", err, rs, testReplicationStatus) compareError(t, "StopSlave", err, rs, testReplicationStatus)
} }
var testStartSlaveCalled = false var testStartSlaveCalled = false
func (fra *fakeRpcAgent) StartSlave() error { func (fra *fakeRpcAgent) StartSlave(ctx context.Context) error {
testStartSlaveCalled = true testStartSlaveCalled = true
return nil return nil
} }
func agentRpcTestStartSlave(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestStartSlave(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.StartSlave(ti, time.Minute) err := client.StartSlave(ctx, ti, time.Minute)
compareError(t, "StartSlave", err, true, testStartSlaveCalled) compareError(t, "StartSlave", err, true, testStartSlaveCalled)
} }
var testTabletExternallyReparentedCalled = false var testTabletExternallyReparentedCalled = false
func (fra *fakeRpcAgent) TabletExternallyReparented(actionTimeout time.Duration) error { func (fra *fakeRpcAgent) TabletExternallyReparented(ctx context.Context, actionTimeout time.Duration) error {
testTabletExternallyReparentedCalled = true testTabletExternallyReparentedCalled = true
return nil return nil
} }
func agentRpcTestTabletExternallyReparented(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestTabletExternallyReparented(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.TabletExternallyReparented(ti, time.Minute) err := client.TabletExternallyReparented(ctx, ti, time.Minute)
compareError(t, "TabletExternallyReparented", err, true, testTabletExternallyReparentedCalled) compareError(t, "TabletExternallyReparented", err, true, testTabletExternallyReparentedCalled)
} }
var testGetSlavesResult = []string{"slave1", "slave2"} var testGetSlavesResult = []string{"slave1", "slave2"}
func (fra *fakeRpcAgent) GetSlaves() ([]string, error) { func (fra *fakeRpcAgent) GetSlaves(ctx context.Context) ([]string, error) {
return testGetSlavesResult, nil return testGetSlavesResult, nil
} }
func agentRpcTestGetSlaves(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestGetSlaves(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
s, err := client.GetSlaves(ti, time.Minute) s, err := client.GetSlaves(ctx, ti, time.Minute)
compareError(t, "GetSlaves", err, s, testGetSlavesResult) compareError(t, "GetSlaves", err, s, testGetSlavesResult)
} }
@ -541,15 +541,15 @@ var testBlpPosition = &blproto.BlpPosition{
var testWaitBlpPositionWaitTime = time.Hour var testWaitBlpPositionWaitTime = time.Hour
var testWaitBlpPositionCalled = false var testWaitBlpPositionCalled = false
func (fra *fakeRpcAgent) WaitBlpPosition(blpPosition *blproto.BlpPosition, waitTime time.Duration) error { func (fra *fakeRpcAgent) WaitBlpPosition(ctx context.Context, blpPosition *blproto.BlpPosition, waitTime time.Duration) error {
compare(fra.t, "WaitBlpPosition blpPosition", blpPosition, testBlpPosition) compare(fra.t, "WaitBlpPosition blpPosition", blpPosition, testBlpPosition)
compare(fra.t, "WaitBlpPosition waitTime", waitTime, testWaitBlpPositionWaitTime) compare(fra.t, "WaitBlpPosition waitTime", waitTime, testWaitBlpPositionWaitTime)
testWaitBlpPositionCalled = true testWaitBlpPositionCalled = true
return nil return nil
} }
func agentRpcTestWaitBlpPosition(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestWaitBlpPosition(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.WaitBlpPosition(ti, *testBlpPosition, testWaitBlpPositionWaitTime) err := client.WaitBlpPosition(ctx, ti, *testBlpPosition, testWaitBlpPositionWaitTime)
compareError(t, "WaitBlpPosition", err, true, testWaitBlpPositionCalled) compareError(t, "WaitBlpPosition", err, true, testWaitBlpPositionCalled)
} }
@ -559,37 +559,37 @@ var testBlpPositionList = &blproto.BlpPositionList{
}, },
} }
func (fra *fakeRpcAgent) StopBlp() (*blproto.BlpPositionList, error) { func (fra *fakeRpcAgent) StopBlp(ctx context.Context) (*blproto.BlpPositionList, error) {
return testBlpPositionList, nil return testBlpPositionList, nil
} }
func agentRpcTestStopBlp(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestStopBlp(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
bpl, err := client.StopBlp(ti, time.Minute) bpl, err := client.StopBlp(ctx, ti, time.Minute)
compareError(t, "StopBlp", err, bpl, testBlpPositionList) compareError(t, "StopBlp", err, bpl, testBlpPositionList)
} }
var testStartBlpCalled = false var testStartBlpCalled = false
func (fra *fakeRpcAgent) StartBlp() error { func (fra *fakeRpcAgent) StartBlp(ctx context.Context) error {
testStartBlpCalled = true testStartBlpCalled = true
return nil return nil
} }
func agentRpcTestStartBlp(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestStartBlp(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.StartBlp(ti, time.Minute) err := client.StartBlp(ctx, ti, time.Minute)
compareError(t, "StartBlp", err, true, testStartBlpCalled) compareError(t, "StartBlp", err, true, testStartBlpCalled)
} }
var testRunBlpUntilWaitTime = 3 * time.Minute var testRunBlpUntilWaitTime = 3 * time.Minute
func (fra *fakeRpcAgent) RunBlpUntil(bpl *blproto.BlpPositionList, waitTime time.Duration) (*myproto.ReplicationPosition, error) { func (fra *fakeRpcAgent) RunBlpUntil(ctx context.Context, bpl *blproto.BlpPositionList, waitTime time.Duration) (*myproto.ReplicationPosition, error) {
compare(fra.t, "RunBlpUntil bpl", bpl, testBlpPositionList) compare(fra.t, "RunBlpUntil bpl", bpl, testBlpPositionList)
compare(fra.t, "RunBlpUntil waitTime", waitTime, testRunBlpUntilWaitTime) compare(fra.t, "RunBlpUntil waitTime", waitTime, testRunBlpUntilWaitTime)
return &testReplicationPosition, nil return &testReplicationPosition, nil
} }
func agentRpcTestRunBlpUntil(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestRunBlpUntil(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rp, err := client.RunBlpUntil(ti, testBlpPositionList, testRunBlpUntilWaitTime) rp, err := client.RunBlpUntil(ctx, ti, testBlpPositionList, testRunBlpUntilWaitTime)
compareError(t, "RunBlpUntil", err, rp, testReplicationPosition) compareError(t, "RunBlpUntil", err, rp, testReplicationPosition)
} }
@ -599,47 +599,47 @@ func agentRpcTestRunBlpUntil(t *testing.T, client tmclient.TabletManagerClient,
var testDemoteMasterCalled = false var testDemoteMasterCalled = false
func (fra *fakeRpcAgent) DemoteMaster() error { func (fra *fakeRpcAgent) DemoteMaster(ctx context.Context) error {
testDemoteMasterCalled = true testDemoteMasterCalled = true
return nil return nil
} }
func agentRpcTestDemoteMaster(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestDemoteMaster(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.DemoteMaster(ti, time.Minute) err := client.DemoteMaster(ctx, ti, time.Minute)
compareError(t, "DemoteMaster", err, true, testDemoteMasterCalled) compareError(t, "DemoteMaster", err, true, testDemoteMasterCalled)
} }
func (fra *fakeRpcAgent) PromoteSlave() (*actionnode.RestartSlaveData, error) { func (fra *fakeRpcAgent) PromoteSlave(ctx context.Context) (*actionnode.RestartSlaveData, error) {
return testRestartSlaveData, nil return testRestartSlaveData, nil
} }
func agentRpcTestPromoteSlave(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestPromoteSlave(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rsd, err := client.PromoteSlave(ti, time.Minute) rsd, err := client.PromoteSlave(ctx, ti, time.Minute)
compareError(t, "PromoteSlave", err, rsd, testRestartSlaveData) compareError(t, "PromoteSlave", err, rsd, testRestartSlaveData)
} }
var testSlaveWasPromotedCalled = false var testSlaveWasPromotedCalled = false
func (fra *fakeRpcAgent) SlaveWasPromoted() error { func (fra *fakeRpcAgent) SlaveWasPromoted(ctx context.Context) error {
testSlaveWasPromotedCalled = true testSlaveWasPromotedCalled = true
return nil return nil
} }
func agentRpcTestSlaveWasPromoted(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestSlaveWasPromoted(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.SlaveWasPromoted(ti, time.Minute) err := client.SlaveWasPromoted(ctx, ti, time.Minute)
compareError(t, "SlaveWasPromoted", err, true, testSlaveWasPromotedCalled) compareError(t, "SlaveWasPromoted", err, true, testSlaveWasPromotedCalled)
} }
var testRestartSlaveCalled = false var testRestartSlaveCalled = false
func (fra *fakeRpcAgent) RestartSlave(rsd *actionnode.RestartSlaveData) error { func (fra *fakeRpcAgent) RestartSlave(ctx context.Context, rsd *actionnode.RestartSlaveData) error {
compare(fra.t, "RestartSlave rsd", rsd, testRestartSlaveData) compare(fra.t, "RestartSlave rsd", rsd, testRestartSlaveData)
testRestartSlaveCalled = true testRestartSlaveCalled = true
return nil return nil
} }
func agentRpcTestRestartSlave(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestRestartSlave(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.RestartSlave(ti, testRestartSlaveData, time.Minute) err := client.RestartSlave(ctx, ti, testRestartSlaveData, time.Minute)
compareError(t, "RestartSlave", err, true, testRestartSlaveCalled) compareError(t, "RestartSlave", err, true, testRestartSlaveCalled)
} }
@ -651,26 +651,26 @@ var testSlaveWasRestartedArgs = &actionnode.SlaveWasRestartedArgs{
} }
var testSlaveWasRestartedCalled = false var testSlaveWasRestartedCalled = false
func (fra *fakeRpcAgent) SlaveWasRestarted(swrd *actionnode.SlaveWasRestartedArgs) error { func (fra *fakeRpcAgent) SlaveWasRestarted(ctx context.Context, swrd *actionnode.SlaveWasRestartedArgs) error {
compare(fra.t, "SlaveWasRestarted swrd", swrd, testSlaveWasRestartedArgs) compare(fra.t, "SlaveWasRestarted swrd", swrd, testSlaveWasRestartedArgs)
testSlaveWasRestartedCalled = true testSlaveWasRestartedCalled = true
return nil return nil
} }
func agentRpcTestSlaveWasRestarted(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestSlaveWasRestarted(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.SlaveWasRestarted(ti, testSlaveWasRestartedArgs, time.Minute) err := client.SlaveWasRestarted(ctx, ti, testSlaveWasRestartedArgs, time.Minute)
compareError(t, "RestartSlave", err, true, testRestartSlaveCalled) compareError(t, "RestartSlave", err, true, testRestartSlaveCalled)
} }
var testBreakSlavesCalled = false var testBreakSlavesCalled = false
func (fra *fakeRpcAgent) BreakSlaves() error { func (fra *fakeRpcAgent) BreakSlaves(ctx context.Context) error {
testBreakSlavesCalled = true testBreakSlavesCalled = true
return nil return nil
} }
func agentRpcTestBreakSlaves(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestBreakSlaves(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.BreakSlaves(ti, time.Minute) err := client.BreakSlaves(ctx, ti, time.Minute)
compareError(t, "BreakSlaves", err, true, testBreakSlavesCalled) compareError(t, "BreakSlaves", err, true, testBreakSlavesCalled)
} }
@ -693,14 +693,14 @@ var testSnapshotReply = &actionnode.SnapshotReply{
ReadOnly: true, ReadOnly: true,
} }
func (fra *fakeRpcAgent) Snapshot(args *actionnode.SnapshotArgs, logger logutil.Logger) (*actionnode.SnapshotReply, error) { func (fra *fakeRpcAgent) Snapshot(ctx context.Context, args *actionnode.SnapshotArgs, logger logutil.Logger) (*actionnode.SnapshotReply, error) {
compare(fra.t, "Snapshot args", args, testSnapshotArgs) compare(fra.t, "Snapshot args", args, testSnapshotArgs)
logStuff(logger, 0) logStuff(logger, 0)
return testSnapshotReply, nil return testSnapshotReply, nil
} }
func agentRpcTestSnapshot(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestSnapshot(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
logChannel, errFunc, err := client.Snapshot(ti, testSnapshotArgs, time.Minute) logChannel, errFunc, err := client.Snapshot(ctx, ti, testSnapshotArgs, time.Minute)
if err != nil { if err != nil {
t.Fatalf("Snapshot failed: %v", err) t.Fatalf("Snapshot failed: %v", err)
} }
@ -716,14 +716,14 @@ var testSnapshotSourceEndArgs = &actionnode.SnapshotSourceEndArgs{
} }
var testSnapshotSourceEndCalled = false var testSnapshotSourceEndCalled = false
func (fra *fakeRpcAgent) SnapshotSourceEnd(args *actionnode.SnapshotSourceEndArgs) error { func (fra *fakeRpcAgent) SnapshotSourceEnd(ctx context.Context, args *actionnode.SnapshotSourceEndArgs) error {
compare(fra.t, "SnapshotSourceEnd args", args, testSnapshotSourceEndArgs) compare(fra.t, "SnapshotSourceEnd args", args, testSnapshotSourceEndArgs)
testSnapshotSourceEndCalled = true testSnapshotSourceEndCalled = true
return nil return nil
} }
func agentRpcTestSnapshotSourceEnd(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestSnapshotSourceEnd(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.SnapshotSourceEnd(ti, testSnapshotSourceEndArgs, time.Minute) err := client.SnapshotSourceEnd(ctx, ti, testSnapshotSourceEndArgs, time.Minute)
compareError(t, "SnapshotSourceEnd", err, true, testSnapshotSourceEndCalled) compareError(t, "SnapshotSourceEnd", err, true, testSnapshotSourceEndCalled)
} }
@ -735,14 +735,14 @@ var testReserveForRestoreArgs = &actionnode.ReserveForRestoreArgs{
} }
var testReserveForRestoreCalled = false var testReserveForRestoreCalled = false
func (fra *fakeRpcAgent) ReserveForRestore(args *actionnode.ReserveForRestoreArgs) error { func (fra *fakeRpcAgent) ReserveForRestore(ctx context.Context, args *actionnode.ReserveForRestoreArgs) error {
compare(fra.t, "ReserveForRestore args", args, testReserveForRestoreArgs) compare(fra.t, "ReserveForRestore args", args, testReserveForRestoreArgs)
testReserveForRestoreCalled = true testReserveForRestoreCalled = true
return nil return nil
} }
func agentRpcTestReserveForRestore(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestReserveForRestore(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.ReserveForRestore(ti, testReserveForRestoreArgs, time.Minute) err := client.ReserveForRestore(ctx, ti, testReserveForRestoreArgs, time.Minute)
compareError(t, "ReserveForRestore", err, true, testReserveForRestoreCalled) compareError(t, "ReserveForRestore", err, true, testReserveForRestoreCalled)
} }
@ -763,15 +763,15 @@ var testRestoreArgs = &actionnode.RestoreArgs{
} }
var testRestoreCalled = false var testRestoreCalled = false
func (fra *fakeRpcAgent) Restore(args *actionnode.RestoreArgs, logger logutil.Logger) error { func (fra *fakeRpcAgent) Restore(ctx context.Context, args *actionnode.RestoreArgs, logger logutil.Logger) error {
compare(fra.t, "Restore args", args, testRestoreArgs) compare(fra.t, "Restore args", args, testRestoreArgs)
logStuff(logger, 10) logStuff(logger, 10)
testRestoreCalled = true testRestoreCalled = true
return nil return nil
} }
func agentRpcTestRestore(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestRestore(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
logChannel, errFunc, err := client.Restore(ti, testRestoreArgs, time.Minute) logChannel, errFunc, err := client.Restore(ctx, ti, testRestoreArgs, time.Minute)
if err != nil { if err != nil {
t.Fatalf("Restore failed: %v", err) t.Fatalf("Restore failed: %v", err)
} }
@ -801,14 +801,14 @@ var testMultiSnapshotReply = &actionnode.MultiSnapshotReply{
ManifestPaths: []string{"path1", "path2"}, ManifestPaths: []string{"path1", "path2"},
} }
func (fra *fakeRpcAgent) MultiSnapshot(args *actionnode.MultiSnapshotArgs, logger logutil.Logger) (*actionnode.MultiSnapshotReply, error) { func (fra *fakeRpcAgent) MultiSnapshot(ctx context.Context, args *actionnode.MultiSnapshotArgs, logger logutil.Logger) (*actionnode.MultiSnapshotReply, error) {
compare(fra.t, "MultiSnapshot args", args, testMultiSnapshotArgs) compare(fra.t, "MultiSnapshot args", args, testMultiSnapshotArgs)
logStuff(logger, 100) logStuff(logger, 100)
return testMultiSnapshotReply, nil return testMultiSnapshotReply, nil
} }
func agentRpcTestMultiSnapshot(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestMultiSnapshot(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
logChannel, errFunc, err := client.MultiSnapshot(ti, testMultiSnapshotArgs, time.Minute) logChannel, errFunc, err := client.MultiSnapshot(ctx, ti, testMultiSnapshotArgs, time.Minute)
if err != nil { if err != nil {
t.Fatalf("MultiSnapshot failed: %v", err) t.Fatalf("MultiSnapshot failed: %v", err)
} }
@ -836,15 +836,15 @@ var testMultiRestoreArgs = &actionnode.MultiRestoreArgs{
} }
var testMultiRestoreCalled = false var testMultiRestoreCalled = false
func (fra *fakeRpcAgent) MultiRestore(args *actionnode.MultiRestoreArgs, logger logutil.Logger) error { func (fra *fakeRpcAgent) MultiRestore(ctx context.Context, args *actionnode.MultiRestoreArgs, logger logutil.Logger) error {
compare(fra.t, "MultiRestore args", args, testMultiRestoreArgs) compare(fra.t, "MultiRestore args", args, testMultiRestoreArgs)
logStuff(logger, 1000) logStuff(logger, 1000)
testMultiRestoreCalled = true testMultiRestoreCalled = true
return nil return nil
} }
func agentRpcTestMultiRestore(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func agentRpcTestMultiRestore(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
logChannel, errFunc, err := client.MultiRestore(ti, testMultiRestoreArgs, time.Minute) logChannel, errFunc, err := client.MultiRestore(ctx, ti, testMultiRestoreArgs, time.Minute)
if err != nil { if err != nil {
t.Fatalf("MultiRestore failed: %v", err) t.Fatalf("MultiRestore failed: %v", err)
} }
@ -878,53 +878,53 @@ func (fra *fakeRpcAgent) RpcWrapLockAction(ctx context.Context, name string, arg
// the provided tablet. Tablet's vt address needs to be configured so // the provided tablet. Tablet's vt address needs to be configured so
// the client will connect to a server backed by our RpcAgent (returned // the client will connect to a server backed by our RpcAgent (returned
// by NewFakeRpcAgent) // by NewFakeRpcAgent)
func AgentRpcTestSuite(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) { func AgentRpcTestSuite(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
// Various read-only methods // Various read-only methods
agentRpcTestPing(t, client, ti) agentRpcTestPing(ctx, t, client, ti)
agentRpcTestGetSchema(t, client, ti) agentRpcTestGetSchema(ctx, t, client, ti)
agentRpcTestGetPermissions(t, client, ti) agentRpcTestGetPermissions(ctx, t, client, ti)
// Various read-write methods // Various read-write methods
agentRpcTestSetReadOnly(t, client, ti) agentRpcTestSetReadOnly(ctx, t, client, ti)
agentRpcTestChangeType(t, client, ti) agentRpcTestChangeType(ctx, t, client, ti)
agentRpcTestScrap(t, client, ti) agentRpcTestScrap(ctx, t, client, ti)
agentRpcTestSleep(t, client, ti) agentRpcTestSleep(ctx, t, client, ti)
agentRpcTestExecuteHook(t, client, ti) agentRpcTestExecuteHook(ctx, t, client, ti)
agentRpcTestRefreshState(t, client, ti) agentRpcTestRefreshState(ctx, t, client, ti)
agentRpcTestRunHealthCheck(t, client, ti) agentRpcTestRunHealthCheck(ctx, t, client, ti)
agentRpcTestReloadSchema(t, client, ti) agentRpcTestReloadSchema(ctx, t, client, ti)
agentRpcTestPreflightSchema(t, client, ti) agentRpcTestPreflightSchema(ctx, t, client, ti)
agentRpcTestApplySchema(t, client, ti) agentRpcTestApplySchema(ctx, t, client, ti)
agentRpcTestExecuteFetch(t, client, ti) agentRpcTestExecuteFetch(ctx, t, client, ti)
// Replication related methods // Replication related methods
agentRpcTestSlaveStatus(t, client, ti) agentRpcTestSlaveStatus(ctx, t, client, ti)
agentRpcTestWaitSlavePosition(t, client, ti) agentRpcTestWaitSlavePosition(ctx, t, client, ti)
agentRpcTestMasterPosition(t, client, ti) agentRpcTestMasterPosition(ctx, t, client, ti)
agentRpcTestReparentPosition(t, client, ti) agentRpcTestReparentPosition(ctx, t, client, ti)
agentRpcTestStopSlave(t, client, ti) agentRpcTestStopSlave(ctx, t, client, ti)
agentRpcTestStopSlaveMinimum(t, client, ti) agentRpcTestStopSlaveMinimum(ctx, t, client, ti)
agentRpcTestStartSlave(t, client, ti) agentRpcTestStartSlave(ctx, t, client, ti)
agentRpcTestTabletExternallyReparented(t, client, ti) agentRpcTestTabletExternallyReparented(ctx, t, client, ti)
agentRpcTestGetSlaves(t, client, ti) agentRpcTestGetSlaves(ctx, t, client, ti)
agentRpcTestWaitBlpPosition(t, client, ti) agentRpcTestWaitBlpPosition(ctx, t, client, ti)
agentRpcTestStopBlp(t, client, ti) agentRpcTestStopBlp(ctx, t, client, ti)
agentRpcTestStartBlp(t, client, ti) agentRpcTestStartBlp(ctx, t, client, ti)
agentRpcTestRunBlpUntil(t, client, ti) agentRpcTestRunBlpUntil(ctx, t, client, ti)
// Reparenting related functions // Reparenting related functions
agentRpcTestDemoteMaster(t, client, ti) agentRpcTestDemoteMaster(ctx, t, client, ti)
agentRpcTestPromoteSlave(t, client, ti) agentRpcTestPromoteSlave(ctx, t, client, ti)
agentRpcTestSlaveWasPromoted(t, client, ti) agentRpcTestSlaveWasPromoted(ctx, t, client, ti)
agentRpcTestRestartSlave(t, client, ti) agentRpcTestRestartSlave(ctx, t, client, ti)
agentRpcTestSlaveWasRestarted(t, client, ti) agentRpcTestSlaveWasRestarted(ctx, t, client, ti)
agentRpcTestBreakSlaves(t, client, ti) agentRpcTestBreakSlaves(ctx, t, client, ti)
// Backup / restore related methods // Backup / restore related methods
agentRpcTestSnapshot(t, client, ti) agentRpcTestSnapshot(ctx, t, client, ti)
agentRpcTestSnapshotSourceEnd(t, client, ti) agentRpcTestSnapshotSourceEnd(ctx, t, client, ti)
agentRpcTestReserveForRestore(t, client, ti) agentRpcTestReserveForRestore(ctx, t, client, ti)
agentRpcTestRestore(t, client, ti) agentRpcTestRestore(ctx, t, client, ti)
agentRpcTestMultiSnapshot(t, client, ti) agentRpcTestMultiSnapshot(ctx, t, client, ti)
agentRpcTestMultiRestore(t, client, ti) agentRpcTestMultiRestore(ctx, t, client, ti)
} }

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

@ -8,6 +8,7 @@ import (
"fmt" "fmt"
"time" "time"
"code.google.com/p/go.net/context"
mproto "github.com/youtube/vitess/go/mysql/proto" mproto "github.com/youtube/vitess/go/mysql/proto"
"github.com/youtube/vitess/go/rpcwrap/bsonrpc" "github.com/youtube/vitess/go/rpcwrap/bsonrpc"
blproto "github.com/youtube/vitess/go/vt/binlog/proto" blproto "github.com/youtube/vitess/go/vt/binlog/proto"
@ -30,7 +31,7 @@ func init() {
// GoRpcTabletManagerClient implements tmclient.TabletManagerClient // GoRpcTabletManagerClient implements tmclient.TabletManagerClient
type GoRpcTabletManagerClient struct{} type GoRpcTabletManagerClient struct{}
func (client *GoRpcTabletManagerClient) rpcCallTablet(tablet *topo.TabletInfo, name string, args, reply interface{}, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) rpcCallTablet(ctx context.Context, tablet *topo.TabletInfo, name string, args, reply interface{}, waitTime time.Duration) error {
// create the RPC client, using waitTime as the connect // create the RPC client, using waitTime as the connect
// timeout, and starting the overall timeout as well // timeout, and starting the overall timeout as well
tmr := time.NewTimer(waitTime) tmr := time.NewTimer(waitTime)
@ -42,7 +43,7 @@ func (client *GoRpcTabletManagerClient) rpcCallTablet(tablet *topo.TabletInfo, n
defer rpcClient.Close() defer rpcClient.Close()
// do the call in the remaining time // do the call in the remaining time
call := rpcClient.Go("TabletManager."+name, args, reply, nil) call := rpcClient.Go(ctx, "TabletManager."+name, args, reply, nil)
select { select {
case <-tmr.C: case <-tmr.C:
return fmt.Errorf("Timeout waiting for TabletManager.%v to %v", name, tablet.Alias) return fmt.Errorf("Timeout waiting for TabletManager.%v to %v", name, tablet.Alias)
@ -59,9 +60,9 @@ func (client *GoRpcTabletManagerClient) rpcCallTablet(tablet *topo.TabletInfo, n
// Various read-only methods // Various read-only methods
// //
func (client *GoRpcTabletManagerClient) Ping(tablet *topo.TabletInfo, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) Ping(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
var result string var result string
err := client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_PING, "payload", &result, waitTime) err := client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_PING, "payload", &result, waitTime)
if err != nil { if err != nil {
return err return err
} }
@ -71,29 +72,29 @@ func (client *GoRpcTabletManagerClient) Ping(tablet *topo.TabletInfo, waitTime t
return nil return nil
} }
func (client *GoRpcTabletManagerClient) Sleep(tablet *topo.TabletInfo, duration, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) Sleep(ctx context.Context, tablet *topo.TabletInfo, duration, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_SLEEP, &duration, &rpc.Unused{}, waitTime) return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_SLEEP, &duration, &rpc.Unused{}, waitTime)
} }
func (client *GoRpcTabletManagerClient) ExecuteHook(tablet *topo.TabletInfo, hk *hook.Hook, waitTime time.Duration) (*hook.HookResult, error) { func (client *GoRpcTabletManagerClient) ExecuteHook(ctx context.Context, tablet *topo.TabletInfo, hk *hook.Hook, waitTime time.Duration) (*hook.HookResult, error) {
var hr hook.HookResult var hr hook.HookResult
if err := client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_EXECUTE_HOOK, hk, &hr, waitTime); err != nil { if err := client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_EXECUTE_HOOK, hk, &hr, waitTime); err != nil {
return nil, err return nil, err
} }
return &hr, nil return &hr, nil
} }
func (client *GoRpcTabletManagerClient) GetSchema(tablet *topo.TabletInfo, tables, excludeTables []string, includeViews bool, waitTime time.Duration) (*myproto.SchemaDefinition, error) { func (client *GoRpcTabletManagerClient) GetSchema(ctx context.Context, tablet *topo.TabletInfo, tables, excludeTables []string, includeViews bool, waitTime time.Duration) (*myproto.SchemaDefinition, error) {
var sd myproto.SchemaDefinition var sd myproto.SchemaDefinition
if err := client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_GET_SCHEMA, &gorpcproto.GetSchemaArgs{Tables: tables, ExcludeTables: excludeTables, IncludeViews: includeViews}, &sd, waitTime); err != nil { if err := client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_GET_SCHEMA, &gorpcproto.GetSchemaArgs{Tables: tables, ExcludeTables: excludeTables, IncludeViews: includeViews}, &sd, waitTime); err != nil {
return nil, err return nil, err
} }
return &sd, nil return &sd, nil
} }
func (client *GoRpcTabletManagerClient) GetPermissions(tablet *topo.TabletInfo, waitTime time.Duration) (*myproto.Permissions, error) { func (client *GoRpcTabletManagerClient) GetPermissions(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) (*myproto.Permissions, error) {
var p myproto.Permissions var p myproto.Permissions
if err := client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_GET_PERMISSIONS, "", &p, waitTime); err != nil { if err := client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_GET_PERMISSIONS, "", &p, waitTime); err != nil {
return nil, err return nil, err
} }
return &p, nil return &p, nil
@ -103,53 +104,53 @@ func (client *GoRpcTabletManagerClient) GetPermissions(tablet *topo.TabletInfo,
// Various read-write methods // Various read-write methods
// //
func (client *GoRpcTabletManagerClient) SetReadOnly(tablet *topo.TabletInfo, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) SetReadOnly(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_SET_RDONLY, "", &rpc.Unused{}, waitTime) return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_SET_RDONLY, "", &rpc.Unused{}, waitTime)
} }
func (client *GoRpcTabletManagerClient) SetReadWrite(tablet *topo.TabletInfo, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) SetReadWrite(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_SET_RDWR, "", &rpc.Unused{}, waitTime) return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_SET_RDWR, "", &rpc.Unused{}, waitTime)
} }
func (client *GoRpcTabletManagerClient) ChangeType(tablet *topo.TabletInfo, dbType topo.TabletType, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) ChangeType(ctx context.Context, tablet *topo.TabletInfo, dbType topo.TabletType, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_CHANGE_TYPE, &dbType, &rpc.Unused{}, waitTime) return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_CHANGE_TYPE, &dbType, &rpc.Unused{}, waitTime)
} }
func (client *GoRpcTabletManagerClient) Scrap(tablet *topo.TabletInfo, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) Scrap(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_SCRAP, "", &rpc.Unused{}, waitTime) return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_SCRAP, "", &rpc.Unused{}, waitTime)
} }
func (client *GoRpcTabletManagerClient) RefreshState(tablet *topo.TabletInfo, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) RefreshState(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_REFRESH_STATE, "", &rpc.Unused{}, waitTime) return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_REFRESH_STATE, "", &rpc.Unused{}, waitTime)
} }
func (client *GoRpcTabletManagerClient) RunHealthCheck(tablet *topo.TabletInfo, targetTabletType topo.TabletType, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) RunHealthCheck(ctx context.Context, tablet *topo.TabletInfo, targetTabletType topo.TabletType, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_RUN_HEALTH_CHECK, &targetTabletType, &rpc.Unused{}, waitTime) return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_RUN_HEALTH_CHECK, &targetTabletType, &rpc.Unused{}, waitTime)
} }
func (client *GoRpcTabletManagerClient) ReloadSchema(tablet *topo.TabletInfo, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) ReloadSchema(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_RELOAD_SCHEMA, "", &rpc.Unused{}, waitTime) return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_RELOAD_SCHEMA, "", &rpc.Unused{}, waitTime)
} }
func (client *GoRpcTabletManagerClient) PreflightSchema(tablet *topo.TabletInfo, change string, waitTime time.Duration) (*myproto.SchemaChangeResult, error) { func (client *GoRpcTabletManagerClient) PreflightSchema(ctx context.Context, tablet *topo.TabletInfo, change string, waitTime time.Duration) (*myproto.SchemaChangeResult, error) {
var scr myproto.SchemaChangeResult var scr myproto.SchemaChangeResult
if err := client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_PREFLIGHT_SCHEMA, change, &scr, waitTime); err != nil { if err := client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_PREFLIGHT_SCHEMA, change, &scr, waitTime); err != nil {
return nil, err return nil, err
} }
return &scr, nil return &scr, nil
} }
func (client *GoRpcTabletManagerClient) ApplySchema(tablet *topo.TabletInfo, change *myproto.SchemaChange, waitTime time.Duration) (*myproto.SchemaChangeResult, error) { func (client *GoRpcTabletManagerClient) ApplySchema(ctx context.Context, tablet *topo.TabletInfo, change *myproto.SchemaChange, waitTime time.Duration) (*myproto.SchemaChangeResult, error) {
var scr myproto.SchemaChangeResult var scr myproto.SchemaChangeResult
if err := client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_APPLY_SCHEMA, change, &scr, waitTime); err != nil { if err := client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_APPLY_SCHEMA, change, &scr, waitTime); err != nil {
return nil, err return nil, err
} }
return &scr, nil return &scr, nil
} }
func (client *GoRpcTabletManagerClient) ExecuteFetch(tablet *topo.TabletInfo, query string, maxRows int, wantFields, disableBinlogs bool, waitTime time.Duration) (*mproto.QueryResult, error) { func (client *GoRpcTabletManagerClient) ExecuteFetch(ctx context.Context, tablet *topo.TabletInfo, query string, maxRows int, wantFields, disableBinlogs bool, waitTime time.Duration) (*mproto.QueryResult, error) {
var qr mproto.QueryResult var qr mproto.QueryResult
if err := client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_EXECUTE_FETCH, &gorpcproto.ExecuteFetchArgs{Query: query, MaxRows: maxRows, WantFields: wantFields, DisableBinlogs: disableBinlogs}, &qr, waitTime); err != nil { if err := client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_EXECUTE_FETCH, &gorpcproto.ExecuteFetchArgs{Query: query, MaxRows: maxRows, WantFields: wantFields, DisableBinlogs: disableBinlogs}, &qr, waitTime); err != nil {
return nil, err return nil, err
} }
return &qr, nil return &qr, nil
@ -159,17 +160,17 @@ func (client *GoRpcTabletManagerClient) ExecuteFetch(tablet *topo.TabletInfo, qu
// Replication related methods // Replication related methods
// //
func (client *GoRpcTabletManagerClient) SlaveStatus(tablet *topo.TabletInfo, waitTime time.Duration) (*myproto.ReplicationStatus, error) { func (client *GoRpcTabletManagerClient) SlaveStatus(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) (*myproto.ReplicationStatus, error) {
var status myproto.ReplicationStatus var status myproto.ReplicationStatus
if err := client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_SLAVE_STATUS, "", &status, waitTime); err != nil { if err := client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_SLAVE_STATUS, "", &status, waitTime); err != nil {
return nil, err return nil, err
} }
return &status, nil return &status, nil
} }
func (client *GoRpcTabletManagerClient) WaitSlavePosition(tablet *topo.TabletInfo, waitPos myproto.ReplicationPosition, waitTime time.Duration) (*myproto.ReplicationStatus, error) { func (client *GoRpcTabletManagerClient) WaitSlavePosition(ctx context.Context, tablet *topo.TabletInfo, waitPos myproto.ReplicationPosition, waitTime time.Duration) (*myproto.ReplicationStatus, error) {
var status myproto.ReplicationStatus var status myproto.ReplicationStatus
if err := client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_WAIT_SLAVE_POSITION, &gorpcproto.WaitSlavePositionArgs{ if err := client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_WAIT_SLAVE_POSITION, &gorpcproto.WaitSlavePositionArgs{
Position: waitPos, Position: waitPos,
WaitTimeout: waitTime, WaitTimeout: waitTime,
}, &status, waitTime); err != nil { }, &status, waitTime); err != nil {
@ -178,29 +179,29 @@ func (client *GoRpcTabletManagerClient) WaitSlavePosition(tablet *topo.TabletInf
return &status, nil return &status, nil
} }
func (client *GoRpcTabletManagerClient) MasterPosition(tablet *topo.TabletInfo, waitTime time.Duration) (myproto.ReplicationPosition, error) { func (client *GoRpcTabletManagerClient) MasterPosition(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) (myproto.ReplicationPosition, error) {
var rp myproto.ReplicationPosition var rp myproto.ReplicationPosition
if err := client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_MASTER_POSITION, "", &rp, waitTime); err != nil { if err := client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_MASTER_POSITION, "", &rp, waitTime); err != nil {
return rp, err return rp, err
} }
return rp, nil return rp, nil
} }
func (client *GoRpcTabletManagerClient) ReparentPosition(tablet *topo.TabletInfo, rp *myproto.ReplicationPosition, waitTime time.Duration) (*actionnode.RestartSlaveData, error) { func (client *GoRpcTabletManagerClient) ReparentPosition(ctx context.Context, tablet *topo.TabletInfo, rp *myproto.ReplicationPosition, waitTime time.Duration) (*actionnode.RestartSlaveData, error) {
var rsd actionnode.RestartSlaveData var rsd actionnode.RestartSlaveData
if err := client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_REPARENT_POSITION, rp, &rsd, waitTime); err != nil { if err := client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_REPARENT_POSITION, rp, &rsd, waitTime); err != nil {
return nil, err return nil, err
} }
return &rsd, nil return &rsd, nil
} }
func (client *GoRpcTabletManagerClient) StopSlave(tablet *topo.TabletInfo, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) StopSlave(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_STOP_SLAVE, "", &rpc.Unused{}, waitTime) return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_STOP_SLAVE, "", &rpc.Unused{}, waitTime)
} }
func (client *GoRpcTabletManagerClient) StopSlaveMinimum(tablet *topo.TabletInfo, minPos myproto.ReplicationPosition, waitTime time.Duration) (*myproto.ReplicationStatus, error) { func (client *GoRpcTabletManagerClient) StopSlaveMinimum(ctx context.Context, tablet *topo.TabletInfo, minPos myproto.ReplicationPosition, waitTime time.Duration) (*myproto.ReplicationStatus, error) {
var status myproto.ReplicationStatus var status myproto.ReplicationStatus
if err := client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_STOP_SLAVE_MINIMUM, &gorpcproto.StopSlaveMinimumArgs{ if err := client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_STOP_SLAVE_MINIMUM, &gorpcproto.StopSlaveMinimumArgs{
Position: minPos, Position: minPos,
WaitTime: waitTime, WaitTime: waitTime,
}, &status, waitTime); err != nil { }, &status, waitTime); err != nil {
@ -209,44 +210,44 @@ func (client *GoRpcTabletManagerClient) StopSlaveMinimum(tablet *topo.TabletInfo
return &status, nil return &status, nil
} }
func (client *GoRpcTabletManagerClient) StartSlave(tablet *topo.TabletInfo, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) StartSlave(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_START_SLAVE, "", &rpc.Unused{}, waitTime) return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_START_SLAVE, "", &rpc.Unused{}, waitTime)
} }
func (client *GoRpcTabletManagerClient) TabletExternallyReparented(tablet *topo.TabletInfo, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) TabletExternallyReparented(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_EXTERNALLY_REPARENTED, "", &rpc.Unused{}, waitTime) return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_EXTERNALLY_REPARENTED, "", &rpc.Unused{}, waitTime)
} }
func (client *GoRpcTabletManagerClient) GetSlaves(tablet *topo.TabletInfo, waitTime time.Duration) ([]string, error) { func (client *GoRpcTabletManagerClient) GetSlaves(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) ([]string, error) {
var sl gorpcproto.GetSlavesReply var sl gorpcproto.GetSlavesReply
if err := client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_GET_SLAVES, "", &sl, waitTime); err != nil { if err := client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_GET_SLAVES, "", &sl, waitTime); err != nil {
return nil, err return nil, err
} }
return sl.Addrs, nil return sl.Addrs, nil
} }
func (client *GoRpcTabletManagerClient) WaitBlpPosition(tablet *topo.TabletInfo, blpPosition blproto.BlpPosition, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) WaitBlpPosition(ctx context.Context, tablet *topo.TabletInfo, blpPosition blproto.BlpPosition, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_WAIT_BLP_POSITION, &gorpcproto.WaitBlpPositionArgs{ return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_WAIT_BLP_POSITION, &gorpcproto.WaitBlpPositionArgs{
BlpPosition: blpPosition, BlpPosition: blpPosition,
WaitTimeout: waitTime, WaitTimeout: waitTime,
}, &rpc.Unused{}, waitTime) }, &rpc.Unused{}, waitTime)
} }
func (client *GoRpcTabletManagerClient) StopBlp(tablet *topo.TabletInfo, waitTime time.Duration) (*blproto.BlpPositionList, error) { func (client *GoRpcTabletManagerClient) StopBlp(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) (*blproto.BlpPositionList, error) {
var bpl blproto.BlpPositionList var bpl blproto.BlpPositionList
if err := client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_STOP_BLP, "", &bpl, waitTime); err != nil { if err := client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_STOP_BLP, "", &bpl, waitTime); err != nil {
return nil, err return nil, err
} }
return &bpl, nil return &bpl, nil
} }
func (client *GoRpcTabletManagerClient) StartBlp(tablet *topo.TabletInfo, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) StartBlp(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_START_BLP, "", &rpc.Unused{}, waitTime) return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_START_BLP, "", &rpc.Unused{}, waitTime)
} }
func (client *GoRpcTabletManagerClient) RunBlpUntil(tablet *topo.TabletInfo, positions *blproto.BlpPositionList, waitTime time.Duration) (myproto.ReplicationPosition, error) { func (client *GoRpcTabletManagerClient) RunBlpUntil(ctx context.Context, tablet *topo.TabletInfo, positions *blproto.BlpPositionList, waitTime time.Duration) (myproto.ReplicationPosition, error) {
var pos myproto.ReplicationPosition var pos myproto.ReplicationPosition
if err := client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_RUN_BLP_UNTIL, &gorpcproto.RunBlpUntilArgs{ if err := client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_RUN_BLP_UNTIL, &gorpcproto.RunBlpUntilArgs{
BlpPositionList: positions, BlpPositionList: positions,
WaitTimeout: waitTime, WaitTimeout: waitTime,
}, &pos, waitTime); err != nil { }, &pos, waitTime); err != nil {
@ -259,39 +260,39 @@ func (client *GoRpcTabletManagerClient) RunBlpUntil(tablet *topo.TabletInfo, pos
// Reparenting related functions // Reparenting related functions
// //
func (client *GoRpcTabletManagerClient) DemoteMaster(tablet *topo.TabletInfo, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) DemoteMaster(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_DEMOTE_MASTER, "", &rpc.Unused{}, waitTime) return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_DEMOTE_MASTER, "", &rpc.Unused{}, waitTime)
} }
func (client *GoRpcTabletManagerClient) PromoteSlave(tablet *topo.TabletInfo, waitTime time.Duration) (*actionnode.RestartSlaveData, error) { func (client *GoRpcTabletManagerClient) PromoteSlave(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) (*actionnode.RestartSlaveData, error) {
var rsd actionnode.RestartSlaveData var rsd actionnode.RestartSlaveData
if err := client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_PROMOTE_SLAVE, "", &rsd, waitTime); err != nil { if err := client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_PROMOTE_SLAVE, "", &rsd, waitTime); err != nil {
return nil, err return nil, err
} }
return &rsd, nil return &rsd, nil
} }
func (client *GoRpcTabletManagerClient) SlaveWasPromoted(tablet *topo.TabletInfo, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) SlaveWasPromoted(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_SLAVE_WAS_PROMOTED, "", &rpc.Unused{}, waitTime) return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_SLAVE_WAS_PROMOTED, "", &rpc.Unused{}, waitTime)
} }
func (client *GoRpcTabletManagerClient) RestartSlave(tablet *topo.TabletInfo, rsd *actionnode.RestartSlaveData, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) RestartSlave(ctx context.Context, tablet *topo.TabletInfo, rsd *actionnode.RestartSlaveData, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_RESTART_SLAVE, rsd, &rpc.Unused{}, waitTime) return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_RESTART_SLAVE, rsd, &rpc.Unused{}, waitTime)
} }
func (client *GoRpcTabletManagerClient) SlaveWasRestarted(tablet *topo.TabletInfo, args *actionnode.SlaveWasRestartedArgs, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) SlaveWasRestarted(ctx context.Context, tablet *topo.TabletInfo, args *actionnode.SlaveWasRestartedArgs, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_SLAVE_WAS_RESTARTED, args, &rpc.Unused{}, waitTime) return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_SLAVE_WAS_RESTARTED, args, &rpc.Unused{}, waitTime)
} }
func (client *GoRpcTabletManagerClient) BreakSlaves(tablet *topo.TabletInfo, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) BreakSlaves(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_BREAK_SLAVES, "", &rpc.Unused{}, waitTime) return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_BREAK_SLAVES, "", &rpc.Unused{}, waitTime)
} }
// //
// Backup related methods // Backup related methods
// //
func (client *GoRpcTabletManagerClient) Snapshot(tablet *topo.TabletInfo, sa *actionnode.SnapshotArgs, waitTime time.Duration) (<-chan *logutil.LoggerEvent, tmclient.SnapshotReplyFunc, error) { func (client *GoRpcTabletManagerClient) Snapshot(ctx context.Context, tablet *topo.TabletInfo, sa *actionnode.SnapshotArgs, waitTime time.Duration) (<-chan *logutil.LoggerEvent, tmclient.SnapshotReplyFunc, error) {
rpcClient, err := bsonrpc.DialHTTP("tcp", tablet.Addr(), waitTime, nil) rpcClient, err := bsonrpc.DialHTTP("tcp", tablet.Addr(), waitTime, nil)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
@ -319,15 +320,15 @@ func (client *GoRpcTabletManagerClient) Snapshot(tablet *topo.TabletInfo, sa *ac
}, nil }, nil
} }
func (client *GoRpcTabletManagerClient) SnapshotSourceEnd(tablet *topo.TabletInfo, args *actionnode.SnapshotSourceEndArgs, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) SnapshotSourceEnd(ctx context.Context, tablet *topo.TabletInfo, args *actionnode.SnapshotSourceEndArgs, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_SNAPSHOT_SOURCE_END, args, &rpc.Unused{}, waitTime) return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_SNAPSHOT_SOURCE_END, args, &rpc.Unused{}, waitTime)
} }
func (client *GoRpcTabletManagerClient) ReserveForRestore(tablet *topo.TabletInfo, args *actionnode.ReserveForRestoreArgs, waitTime time.Duration) error { func (client *GoRpcTabletManagerClient) ReserveForRestore(ctx context.Context, tablet *topo.TabletInfo, args *actionnode.ReserveForRestoreArgs, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_RESERVE_FOR_RESTORE, args, &rpc.Unused{}, waitTime) return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_RESERVE_FOR_RESTORE, args, &rpc.Unused{}, waitTime)
} }
func (client *GoRpcTabletManagerClient) Restore(tablet *topo.TabletInfo, sa *actionnode.RestoreArgs, waitTime time.Duration) (<-chan *logutil.LoggerEvent, tmclient.ErrFunc, error) { func (client *GoRpcTabletManagerClient) Restore(ctx context.Context, tablet *topo.TabletInfo, sa *actionnode.RestoreArgs, waitTime time.Duration) (<-chan *logutil.LoggerEvent, tmclient.ErrFunc, error) {
rpcClient, err := bsonrpc.DialHTTP("tcp", tablet.Addr(), waitTime, nil) rpcClient, err := bsonrpc.DialHTTP("tcp", tablet.Addr(), waitTime, nil)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
@ -341,7 +342,7 @@ func (client *GoRpcTabletManagerClient) Restore(tablet *topo.TabletInfo, sa *act
}, nil }, nil
} }
func (client *GoRpcTabletManagerClient) MultiSnapshot(tablet *topo.TabletInfo, sa *actionnode.MultiSnapshotArgs, waitTime time.Duration) (<-chan *logutil.LoggerEvent, tmclient.MultiSnapshotReplyFunc, error) { func (client *GoRpcTabletManagerClient) MultiSnapshot(ctx context.Context, tablet *topo.TabletInfo, sa *actionnode.MultiSnapshotArgs, waitTime time.Duration) (<-chan *logutil.LoggerEvent, tmclient.MultiSnapshotReplyFunc, error) {
rpcClient, err := bsonrpc.DialHTTP("tcp", tablet.Addr(), waitTime, nil) rpcClient, err := bsonrpc.DialHTTP("tcp", tablet.Addr(), waitTime, nil)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err
@ -369,7 +370,7 @@ func (client *GoRpcTabletManagerClient) MultiSnapshot(tablet *topo.TabletInfo, s
}, nil }, nil
} }
func (client *GoRpcTabletManagerClient) MultiRestore(tablet *topo.TabletInfo, sa *actionnode.MultiRestoreArgs, waitTime time.Duration) (<-chan *logutil.LoggerEvent, tmclient.ErrFunc, error) { func (client *GoRpcTabletManagerClient) MultiRestore(ctx context.Context, tablet *topo.TabletInfo, sa *actionnode.MultiRestoreArgs, waitTime time.Duration) (<-chan *logutil.LoggerEvent, tmclient.ErrFunc, error) {
rpcClient, err := bsonrpc.DialHTTP("tcp", tablet.Addr(), waitTime, nil) rpcClient, err := bsonrpc.DialHTTP("tcp", tablet.Addr(), waitTime, nil)
if err != nil { if err != nil {
return nil, nil, err return nil, nil, err

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

@ -35,28 +35,28 @@ type TabletManager struct {
func (tm *TabletManager) Ping(ctx context.Context, args, reply *string) error { func (tm *TabletManager) Ping(ctx context.Context, args, reply *string) error {
return tm.agent.RpcWrap(ctx, actionnode.TABLET_ACTION_PING, args, reply, func() error { return tm.agent.RpcWrap(ctx, actionnode.TABLET_ACTION_PING, args, reply, func() error {
*reply = tm.agent.Ping(*args) *reply = tm.agent.Ping(ctx, *args)
return nil return nil
}) })
} }
func (tm *TabletManager) Sleep(ctx context.Context, args *time.Duration, reply *rpc.Unused) error { func (tm *TabletManager) Sleep(ctx context.Context, args *time.Duration, reply *rpc.Unused) error {
return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_SLEEP, args, reply, true, func() error { return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_SLEEP, args, reply, true, func() error {
tm.agent.Sleep(*args) tm.agent.Sleep(ctx, *args)
return nil return nil
}) })
} }
func (tm *TabletManager) ExecuteHook(ctx context.Context, args *hook.Hook, reply *hook.HookResult) error { func (tm *TabletManager) ExecuteHook(ctx context.Context, args *hook.Hook, reply *hook.HookResult) error {
return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_EXECUTE_HOOK, args, reply, true, func() error { return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_EXECUTE_HOOK, args, reply, true, func() error {
*reply = *tm.agent.ExecuteHook(args) *reply = *tm.agent.ExecuteHook(ctx, args)
return nil return nil
}) })
} }
func (tm *TabletManager) GetSchema(ctx context.Context, args *gorpcproto.GetSchemaArgs, reply *myproto.SchemaDefinition) error { func (tm *TabletManager) GetSchema(ctx context.Context, args *gorpcproto.GetSchemaArgs, reply *myproto.SchemaDefinition) error {
return tm.agent.RpcWrap(ctx, actionnode.TABLET_ACTION_GET_SCHEMA, args, reply, func() error { return tm.agent.RpcWrap(ctx, actionnode.TABLET_ACTION_GET_SCHEMA, args, reply, func() error {
sd, err := tm.agent.GetSchema(args.Tables, args.ExcludeTables, args.IncludeViews) sd, err := tm.agent.GetSchema(ctx, args.Tables, args.ExcludeTables, args.IncludeViews)
if err == nil { if err == nil {
*reply = *sd *reply = *sd
} }
@ -66,7 +66,7 @@ func (tm *TabletManager) GetSchema(ctx context.Context, args *gorpcproto.GetSche
func (tm *TabletManager) GetPermissions(ctx context.Context, args *rpc.Unused, reply *myproto.Permissions) error { func (tm *TabletManager) GetPermissions(ctx context.Context, args *rpc.Unused, reply *myproto.Permissions) error {
return tm.agent.RpcWrap(ctx, actionnode.TABLET_ACTION_GET_PERMISSIONS, args, reply, func() error { return tm.agent.RpcWrap(ctx, actionnode.TABLET_ACTION_GET_PERMISSIONS, args, reply, func() error {
p, err := tm.agent.GetPermissions() p, err := tm.agent.GetPermissions(ctx)
if err == nil { if err == nil {
*reply = *p *reply = *p
} }
@ -80,52 +80,52 @@ func (tm *TabletManager) GetPermissions(ctx context.Context, args *rpc.Unused, r
func (tm *TabletManager) SetReadOnly(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error { func (tm *TabletManager) SetReadOnly(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error {
return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_SET_RDONLY, args, reply, true, func() error { return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_SET_RDONLY, args, reply, true, func() error {
return tm.agent.SetReadOnly(true) return tm.agent.SetReadOnly(ctx, true)
}) })
} }
func (tm *TabletManager) SetReadWrite(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error { func (tm *TabletManager) SetReadWrite(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error {
return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_SET_RDWR, args, reply, true, func() error { return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_SET_RDWR, args, reply, true, func() error {
return tm.agent.SetReadOnly(false) return tm.agent.SetReadOnly(ctx, false)
}) })
} }
func (tm *TabletManager) ChangeType(ctx context.Context, args *topo.TabletType, reply *rpc.Unused) error { func (tm *TabletManager) ChangeType(ctx context.Context, args *topo.TabletType, reply *rpc.Unused) error {
return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_CHANGE_TYPE, args, reply, true, func() error { return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_CHANGE_TYPE, args, reply, true, func() error {
return tm.agent.ChangeType(*args) return tm.agent.ChangeType(ctx, *args)
}) })
} }
func (tm *TabletManager) Scrap(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error { func (tm *TabletManager) Scrap(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error {
return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_SCRAP, args, reply, true, func() error { return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_SCRAP, args, reply, true, func() error {
return tm.agent.Scrap() return tm.agent.Scrap(ctx)
}) })
} }
func (tm *TabletManager) RefreshState(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error { func (tm *TabletManager) RefreshState(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error {
return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_REFRESH_STATE, args, reply, true, func() error { return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_REFRESH_STATE, args, reply, true, func() error {
tm.agent.RefreshState() tm.agent.RefreshState(ctx)
return nil return nil
}) })
} }
func (tm *TabletManager) RunHealthCheck(ctx context.Context, args *topo.TabletType, reply *rpc.Unused) error { func (tm *TabletManager) RunHealthCheck(ctx context.Context, args *topo.TabletType, reply *rpc.Unused) error {
return tm.agent.RpcWrap(ctx, actionnode.TABLET_ACTION_RUN_HEALTH_CHECK, args, reply, func() error { return tm.agent.RpcWrap(ctx, actionnode.TABLET_ACTION_RUN_HEALTH_CHECK, args, reply, func() error {
tm.agent.RunHealthCheck(*args) tm.agent.RunHealthCheck(ctx, *args)
return nil return nil
}) })
} }
func (tm *TabletManager) ReloadSchema(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error { func (tm *TabletManager) ReloadSchema(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error {
return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_RELOAD_SCHEMA, args, reply, true, func() error { return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_RELOAD_SCHEMA, args, reply, true, func() error {
tm.agent.ReloadSchema() tm.agent.ReloadSchema(ctx)
return nil return nil
}) })
} }
func (tm *TabletManager) PreflightSchema(ctx context.Context, args *string, reply *myproto.SchemaChangeResult) error { func (tm *TabletManager) PreflightSchema(ctx context.Context, args *string, reply *myproto.SchemaChangeResult) error {
return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_PREFLIGHT_SCHEMA, args, reply, true, func() error { return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_PREFLIGHT_SCHEMA, args, reply, true, func() error {
scr, err := tm.agent.PreflightSchema(*args) scr, err := tm.agent.PreflightSchema(ctx, *args)
if err == nil { if err == nil {
*reply = *scr *reply = *scr
} }
@ -135,7 +135,7 @@ func (tm *TabletManager) PreflightSchema(ctx context.Context, args *string, repl
func (tm *TabletManager) ApplySchema(ctx context.Context, args *myproto.SchemaChange, reply *myproto.SchemaChangeResult) error { func (tm *TabletManager) ApplySchema(ctx context.Context, args *myproto.SchemaChange, reply *myproto.SchemaChangeResult) error {
return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_APPLY_SCHEMA, args, reply, true, func() error { return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_APPLY_SCHEMA, args, reply, true, func() error {
scr, err := tm.agent.ApplySchema(args) scr, err := tm.agent.ApplySchema(ctx, args)
if err == nil { if err == nil {
*reply = *scr *reply = *scr
} }
@ -145,7 +145,7 @@ func (tm *TabletManager) ApplySchema(ctx context.Context, args *myproto.SchemaCh
func (tm *TabletManager) ExecuteFetch(ctx context.Context, args *gorpcproto.ExecuteFetchArgs, reply *mproto.QueryResult) error { func (tm *TabletManager) ExecuteFetch(ctx context.Context, args *gorpcproto.ExecuteFetchArgs, reply *mproto.QueryResult) error {
return tm.agent.RpcWrap(ctx, actionnode.TABLET_ACTION_EXECUTE_FETCH, args, reply, func() error { return tm.agent.RpcWrap(ctx, actionnode.TABLET_ACTION_EXECUTE_FETCH, args, reply, func() error {
qr, err := tm.agent.ExecuteFetch(args.Query, args.MaxRows, args.WantFields, args.DisableBinlogs) qr, err := tm.agent.ExecuteFetch(ctx, args.Query, args.MaxRows, args.WantFields, args.DisableBinlogs)
if err == nil { if err == nil {
*reply = *qr *reply = *qr
} }
@ -159,7 +159,7 @@ func (tm *TabletManager) ExecuteFetch(ctx context.Context, args *gorpcproto.Exec
func (tm *TabletManager) SlaveStatus(ctx context.Context, args *rpc.Unused, reply *myproto.ReplicationStatus) error { func (tm *TabletManager) SlaveStatus(ctx context.Context, args *rpc.Unused, reply *myproto.ReplicationStatus) error {
return tm.agent.RpcWrap(ctx, actionnode.TABLET_ACTION_SLAVE_STATUS, args, reply, func() error { return tm.agent.RpcWrap(ctx, actionnode.TABLET_ACTION_SLAVE_STATUS, args, reply, func() error {
status, err := tm.agent.SlaveStatus() status, err := tm.agent.SlaveStatus(ctx)
if err == nil { if err == nil {
*reply = *status *reply = *status
} }
@ -169,7 +169,7 @@ func (tm *TabletManager) SlaveStatus(ctx context.Context, args *rpc.Unused, repl
func (tm *TabletManager) WaitSlavePosition(ctx context.Context, args *gorpcproto.WaitSlavePositionArgs, reply *myproto.ReplicationStatus) error { func (tm *TabletManager) WaitSlavePosition(ctx context.Context, args *gorpcproto.WaitSlavePositionArgs, reply *myproto.ReplicationStatus) error {
return tm.agent.RpcWrapLock(ctx, actionnode.TABLET_ACTION_WAIT_SLAVE_POSITION, args, reply, true, func() error { return tm.agent.RpcWrapLock(ctx, actionnode.TABLET_ACTION_WAIT_SLAVE_POSITION, args, reply, true, func() error {
status, err := tm.agent.WaitSlavePosition(args.Position, args.WaitTimeout) status, err := tm.agent.WaitSlavePosition(ctx, args.Position, args.WaitTimeout)
if err == nil { if err == nil {
*reply = *status *reply = *status
} }
@ -179,7 +179,7 @@ func (tm *TabletManager) WaitSlavePosition(ctx context.Context, args *gorpcproto
func (tm *TabletManager) MasterPosition(ctx context.Context, args *rpc.Unused, reply *myproto.ReplicationPosition) error { func (tm *TabletManager) MasterPosition(ctx context.Context, args *rpc.Unused, reply *myproto.ReplicationPosition) error {
return tm.agent.RpcWrap(ctx, actionnode.TABLET_ACTION_MASTER_POSITION, args, reply, func() error { return tm.agent.RpcWrap(ctx, actionnode.TABLET_ACTION_MASTER_POSITION, args, reply, func() error {
position, err := tm.agent.MasterPosition() position, err := tm.agent.MasterPosition(ctx)
if err == nil { if err == nil {
*reply = position *reply = position
} }
@ -189,7 +189,7 @@ func (tm *TabletManager) MasterPosition(ctx context.Context, args *rpc.Unused, r
func (tm *TabletManager) ReparentPosition(ctx context.Context, args *myproto.ReplicationPosition, reply *actionnode.RestartSlaveData) error { func (tm *TabletManager) ReparentPosition(ctx context.Context, args *myproto.ReplicationPosition, reply *actionnode.RestartSlaveData) error {
return tm.agent.RpcWrap(ctx, actionnode.TABLET_ACTION_REPARENT_POSITION, args, reply, func() error { return tm.agent.RpcWrap(ctx, actionnode.TABLET_ACTION_REPARENT_POSITION, args, reply, func() error {
rsd, err := tm.agent.ReparentPosition(args) rsd, err := tm.agent.ReparentPosition(ctx, args)
if err == nil { if err == nil {
*reply = *rsd *reply = *rsd
} }
@ -199,13 +199,13 @@ func (tm *TabletManager) ReparentPosition(ctx context.Context, args *myproto.Rep
func (tm *TabletManager) StopSlave(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error { func (tm *TabletManager) StopSlave(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error {
return tm.agent.RpcWrapLock(ctx, actionnode.TABLET_ACTION_STOP_SLAVE, args, reply, true, func() error { return tm.agent.RpcWrapLock(ctx, actionnode.TABLET_ACTION_STOP_SLAVE, args, reply, true, func() error {
return tm.agent.StopSlave() return tm.agent.StopSlave(ctx)
}) })
} }
func (tm *TabletManager) StopSlaveMinimum(ctx context.Context, args *gorpcproto.StopSlaveMinimumArgs, reply *myproto.ReplicationStatus) error { func (tm *TabletManager) StopSlaveMinimum(ctx context.Context, args *gorpcproto.StopSlaveMinimumArgs, reply *myproto.ReplicationStatus) error {
return tm.agent.RpcWrapLock(ctx, actionnode.TABLET_ACTION_STOP_SLAVE_MINIMUM, args, reply, true, func() error { return tm.agent.RpcWrapLock(ctx, actionnode.TABLET_ACTION_STOP_SLAVE_MINIMUM, args, reply, true, func() error {
status, err := tm.agent.StopSlaveMinimum(args.Position, args.WaitTime) status, err := tm.agent.StopSlaveMinimum(ctx, args.Position, args.WaitTime)
if err == nil { if err == nil {
*reply = *status *reply = *status
} }
@ -215,7 +215,7 @@ func (tm *TabletManager) StopSlaveMinimum(ctx context.Context, args *gorpcproto.
func (tm *TabletManager) StartSlave(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error { func (tm *TabletManager) StartSlave(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error {
return tm.agent.RpcWrapLock(ctx, actionnode.TABLET_ACTION_START_SLAVE, args, reply, true, func() error { return tm.agent.RpcWrapLock(ctx, actionnode.TABLET_ACTION_START_SLAVE, args, reply, true, func() error {
return tm.agent.StartSlave() return tm.agent.StartSlave(ctx)
}) })
} }
@ -224,27 +224,27 @@ func (tm *TabletManager) TabletExternallyReparented(ctx context.Context, args *r
// the original gorpc call. Until we support that, use a // the original gorpc call. Until we support that, use a
// reasonnable hard-coded value. // reasonnable hard-coded value.
return tm.agent.RpcWrapLock(ctx, actionnode.TABLET_ACTION_EXTERNALLY_REPARENTED, args, reply, false, func() error { return tm.agent.RpcWrapLock(ctx, actionnode.TABLET_ACTION_EXTERNALLY_REPARENTED, args, reply, false, func() error {
return tm.agent.TabletExternallyReparented(30 * time.Second) return tm.agent.TabletExternallyReparented(ctx, 30*time.Second)
}) })
} }
func (tm *TabletManager) GetSlaves(ctx context.Context, args *rpc.Unused, reply *gorpcproto.GetSlavesReply) error { func (tm *TabletManager) GetSlaves(ctx context.Context, args *rpc.Unused, reply *gorpcproto.GetSlavesReply) error {
return tm.agent.RpcWrap(ctx, actionnode.TABLET_ACTION_GET_SLAVES, args, reply, func() error { return tm.agent.RpcWrap(ctx, actionnode.TABLET_ACTION_GET_SLAVES, args, reply, func() error {
var err error var err error
reply.Addrs, err = tm.agent.GetSlaves() reply.Addrs, err = tm.agent.GetSlaves(ctx)
return err return err
}) })
} }
func (tm *TabletManager) WaitBlpPosition(ctx context.Context, args *gorpcproto.WaitBlpPositionArgs, reply *rpc.Unused) error { func (tm *TabletManager) WaitBlpPosition(ctx context.Context, args *gorpcproto.WaitBlpPositionArgs, reply *rpc.Unused) error {
return tm.agent.RpcWrapLock(ctx, actionnode.TABLET_ACTION_WAIT_BLP_POSITION, args, reply, true, func() error { return tm.agent.RpcWrapLock(ctx, actionnode.TABLET_ACTION_WAIT_BLP_POSITION, args, reply, true, func() error {
return tm.agent.WaitBlpPosition(&args.BlpPosition, args.WaitTimeout) return tm.agent.WaitBlpPosition(ctx, &args.BlpPosition, args.WaitTimeout)
}) })
} }
func (tm *TabletManager) StopBlp(ctx context.Context, args *rpc.Unused, reply *blproto.BlpPositionList) error { func (tm *TabletManager) StopBlp(ctx context.Context, args *rpc.Unused, reply *blproto.BlpPositionList) error {
return tm.agent.RpcWrapLock(ctx, actionnode.TABLET_ACTION_STOP_BLP, args, reply, true, func() error { return tm.agent.RpcWrapLock(ctx, actionnode.TABLET_ACTION_STOP_BLP, args, reply, true, func() error {
positions, err := tm.agent.StopBlp() positions, err := tm.agent.StopBlp(ctx)
if err == nil { if err == nil {
*reply = *positions *reply = *positions
} }
@ -254,13 +254,13 @@ func (tm *TabletManager) StopBlp(ctx context.Context, args *rpc.Unused, reply *b
func (tm *TabletManager) StartBlp(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error { func (tm *TabletManager) StartBlp(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error {
return tm.agent.RpcWrapLock(ctx, actionnode.TABLET_ACTION_START_BLP, args, reply, true, func() error { return tm.agent.RpcWrapLock(ctx, actionnode.TABLET_ACTION_START_BLP, args, reply, true, func() error {
return tm.agent.StartBlp() return tm.agent.StartBlp(ctx)
}) })
} }
func (tm *TabletManager) RunBlpUntil(ctx context.Context, args *gorpcproto.RunBlpUntilArgs, reply *myproto.ReplicationPosition) error { func (tm *TabletManager) RunBlpUntil(ctx context.Context, args *gorpcproto.RunBlpUntilArgs, reply *myproto.ReplicationPosition) error {
return tm.agent.RpcWrapLock(ctx, actionnode.TABLET_ACTION_RUN_BLP_UNTIL, args, reply, true, func() error { return tm.agent.RpcWrapLock(ctx, actionnode.TABLET_ACTION_RUN_BLP_UNTIL, args, reply, true, func() error {
position, err := tm.agent.RunBlpUntil(args.BlpPositionList, args.WaitTimeout) position, err := tm.agent.RunBlpUntil(ctx, args.BlpPositionList, args.WaitTimeout)
if err == nil { if err == nil {
*reply = *position *reply = *position
} }
@ -274,13 +274,13 @@ func (tm *TabletManager) RunBlpUntil(ctx context.Context, args *gorpcproto.RunBl
func (tm *TabletManager) DemoteMaster(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error { func (tm *TabletManager) DemoteMaster(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error {
return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_DEMOTE_MASTER, args, reply, true, func() error { return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_DEMOTE_MASTER, args, reply, true, func() error {
return tm.agent.DemoteMaster() return tm.agent.DemoteMaster(ctx)
}) })
} }
func (tm *TabletManager) PromoteSlave(ctx context.Context, args *rpc.Unused, reply *actionnode.RestartSlaveData) error { func (tm *TabletManager) PromoteSlave(ctx context.Context, args *rpc.Unused, reply *actionnode.RestartSlaveData) error {
return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_PROMOTE_SLAVE, args, reply, true, func() error { return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_PROMOTE_SLAVE, args, reply, true, func() error {
rsd, err := tm.agent.PromoteSlave() rsd, err := tm.agent.PromoteSlave(ctx)
if err == nil { if err == nil {
*reply = *rsd *reply = *rsd
} }
@ -290,25 +290,25 @@ func (tm *TabletManager) PromoteSlave(ctx context.Context, args *rpc.Unused, rep
func (tm *TabletManager) SlaveWasPromoted(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error { func (tm *TabletManager) SlaveWasPromoted(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error {
return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_SLAVE_WAS_PROMOTED, args, reply, true, func() error { return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_SLAVE_WAS_PROMOTED, args, reply, true, func() error {
return tm.agent.SlaveWasPromoted() return tm.agent.SlaveWasPromoted(ctx)
}) })
} }
func (tm *TabletManager) RestartSlave(ctx context.Context, args *actionnode.RestartSlaveData, reply *rpc.Unused) error { func (tm *TabletManager) RestartSlave(ctx context.Context, args *actionnode.RestartSlaveData, reply *rpc.Unused) error {
return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_RESTART_SLAVE, args, reply, true, func() error { return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_RESTART_SLAVE, args, reply, true, func() error {
return tm.agent.RestartSlave(args) return tm.agent.RestartSlave(ctx, args)
}) })
} }
func (tm *TabletManager) SlaveWasRestarted(ctx context.Context, args *actionnode.SlaveWasRestartedArgs, reply *rpc.Unused) error { func (tm *TabletManager) SlaveWasRestarted(ctx context.Context, args *actionnode.SlaveWasRestartedArgs, reply *rpc.Unused) error {
return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_SLAVE_WAS_RESTARTED, args, reply, true, func() error { return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_SLAVE_WAS_RESTARTED, args, reply, true, func() error {
return tm.agent.SlaveWasRestarted(args) return tm.agent.SlaveWasRestarted(ctx, args)
}) })
} }
func (tm *TabletManager) BreakSlaves(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error { func (tm *TabletManager) BreakSlaves(ctx context.Context, args *rpc.Unused, reply *rpc.Unused) error {
return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_BREAK_SLAVES, args, reply, true, func() error { return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_BREAK_SLAVES, args, reply, true, func() error {
return tm.agent.BreakSlaves() return tm.agent.BreakSlaves(ctx)
}) })
} }
@ -334,7 +334,7 @@ func (tm *TabletManager) Snapshot(ctx context.Context, args *actionnode.Snapshot
wg.Done() wg.Done()
}() }()
sr, err := tm.agent.Snapshot(args, logger) sr, err := tm.agent.Snapshot(ctx, args, logger)
close(logger) close(logger)
wg.Wait() wg.Wait()
if err != nil { if err != nil {
@ -350,13 +350,13 @@ func (tm *TabletManager) Snapshot(ctx context.Context, args *actionnode.Snapshot
func (tm *TabletManager) SnapshotSourceEnd(ctx context.Context, args *actionnode.SnapshotSourceEndArgs, reply *rpc.Unused) error { func (tm *TabletManager) SnapshotSourceEnd(ctx context.Context, args *actionnode.SnapshotSourceEndArgs, reply *rpc.Unused) error {
return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_SNAPSHOT_SOURCE_END, args, reply, true, func() error { return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_SNAPSHOT_SOURCE_END, args, reply, true, func() error {
return tm.agent.SnapshotSourceEnd(args) return tm.agent.SnapshotSourceEnd(ctx, args)
}) })
} }
func (tm *TabletManager) ReserveForRestore(ctx context.Context, args *actionnode.ReserveForRestoreArgs, reply *rpc.Unused) error { func (tm *TabletManager) ReserveForRestore(ctx context.Context, args *actionnode.ReserveForRestoreArgs, reply *rpc.Unused) error {
return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_RESERVE_FOR_RESTORE, args, reply, true, func() error { return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_RESERVE_FOR_RESTORE, args, reply, true, func() error {
return tm.agent.ReserveForRestore(args) return tm.agent.ReserveForRestore(ctx, args)
}) })
} }
@ -377,7 +377,7 @@ func (tm *TabletManager) Restore(ctx context.Context, args *actionnode.RestoreAr
wg.Done() wg.Done()
}() }()
err := tm.agent.Restore(args, logger) err := tm.agent.Restore(ctx, args, logger)
close(logger) close(logger)
wg.Wait() wg.Wait()
return err return err
@ -404,7 +404,7 @@ func (tm *TabletManager) MultiSnapshot(ctx context.Context, args *actionnode.Mul
wg.Done() wg.Done()
}() }()
sr, err := tm.agent.MultiSnapshot(args, logger) sr, err := tm.agent.MultiSnapshot(ctx, args, logger)
close(logger) close(logger)
wg.Wait() wg.Wait()
if err != nil { if err != nil {
@ -435,7 +435,7 @@ func (tm *TabletManager) MultiRestore(ctx context.Context, args *actionnode.Mult
wg.Done() wg.Done()
}() }()
err := tm.agent.MultiRestore(args, logger) err := tm.agent.MultiRestore(ctx, args, logger)
close(logger) close(logger)
wg.Wait() wg.Wait()
return err return err

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

@ -9,6 +9,7 @@ import (
"net/http" "net/http"
"testing" "testing"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/rpcplus" "github.com/youtube/vitess/go/rpcplus"
"github.com/youtube/vitess/go/rpcwrap/bsonrpc" "github.com/youtube/vitess/go/rpcwrap/bsonrpc"
"github.com/youtube/vitess/go/vt/tabletmanager/agentrpctest" "github.com/youtube/vitess/go/vt/tabletmanager/agentrpctest"
@ -52,5 +53,5 @@ func TestGoRpcTMServer(t *testing.T) {
}, 0) }, 0)
// and run the test suite // and run the test suite
agentrpctest.AgentRpcTestSuite(t, client, ti) agentrpctest.AgentRpcTestSuite(context.Background(), t, client, ti)
} }

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

@ -14,6 +14,8 @@ import (
"reflect" "reflect"
"time" "time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
"github.com/youtube/vitess/go/timer" "github.com/youtube/vitess/go/timer"
"github.com/youtube/vitess/go/vt/health" "github.com/youtube/vitess/go/vt/health"
@ -264,7 +266,7 @@ func (agent *ActionAgent) rebuildShardIfNeeded(tablet *topo.TabletInfo, targetTa
interrupted := make(chan struct{}) interrupted := make(chan struct{})
// no need to take the shard lock in this case // no need to take the shard lock in this case
if _, err := topotools.RebuildShard(logutil.NewConsoleLogger(), agent.TopoServer, tablet.Keyspace, tablet.Shard, []string{tablet.Alias.Cell}, agent.LockTimeout, interrupted); err != nil { if _, err := topotools.RebuildShard(context.TODO(), logutil.NewConsoleLogger(), agent.TopoServer, tablet.Keyspace, tablet.Shard, []string{tablet.Alias.Cell}, agent.LockTimeout, interrupted); err != nil {
return fmt.Errorf("topotools.RebuildShard returned an error: %v", err) return fmt.Errorf("topotools.RebuildShard returned an error: %v", err)
} }
} }

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

@ -8,6 +8,7 @@ import (
"flag" "flag"
"time" "time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
mproto "github.com/youtube/vitess/go/mysql/proto" mproto "github.com/youtube/vitess/go/mysql/proto"
blproto "github.com/youtube/vitess/go/vt/binlog/proto" blproto "github.com/youtube/vitess/go/vt/binlog/proto"
@ -36,147 +37,147 @@ type TabletManagerClient interface {
// //
// Ping will try to ping the remote tablet // Ping will try to ping the remote tablet
Ping(tablet *topo.TabletInfo, waitTime time.Duration) error Ping(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error
// GetSchema asks the remote tablet for its database schema // GetSchema asks the remote tablet for its database schema
GetSchema(tablet *topo.TabletInfo, tables, excludeTables []string, includeViews bool, waitTime time.Duration) (*myproto.SchemaDefinition, error) GetSchema(ctx context.Context, tablet *topo.TabletInfo, tables, excludeTables []string, includeViews bool, waitTime time.Duration) (*myproto.SchemaDefinition, error)
// GetPermissions asks the remote tablet for its permissions list // GetPermissions asks the remote tablet for its permissions list
GetPermissions(tablet *topo.TabletInfo, waitTime time.Duration) (*myproto.Permissions, error) GetPermissions(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) (*myproto.Permissions, error)
// //
// Various read-write methods // Various read-write methods
// //
// SetReadOnly makes the mysql instance read-only // SetReadOnly makes the mysql instance read-only
SetReadOnly(tablet *topo.TabletInfo, waitTime time.Duration) error SetReadOnly(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error
// SetReadWrite makes the mysql instance read-write // SetReadWrite makes the mysql instance read-write
SetReadWrite(tablet *topo.TabletInfo, waitTime time.Duration) error SetReadWrite(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error
// ChangeType asks the remote tablet to change its type // ChangeType asks the remote tablet to change its type
ChangeType(tablet *topo.TabletInfo, dbType topo.TabletType, waitTime time.Duration) error ChangeType(ctx context.Context, tablet *topo.TabletInfo, dbType topo.TabletType, waitTime time.Duration) error
// Scrap scraps the live running tablet // Scrap scraps the live running tablet
Scrap(tablet *topo.TabletInfo, waitTime time.Duration) error Scrap(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error
// Sleep will sleep for a duration (used for tests) // Sleep will sleep for a duration (used for tests)
Sleep(tablet *topo.TabletInfo, duration, waitTime time.Duration) error Sleep(ctx context.Context, tablet *topo.TabletInfo, duration, waitTime time.Duration) error
// ExecuteHook executes the provided hook remotely // ExecuteHook executes the provided hook remotely
ExecuteHook(tablet *topo.TabletInfo, hk *hook.Hook, waitTime time.Duration) (*hook.HookResult, error) ExecuteHook(ctx context.Context, tablet *topo.TabletInfo, hk *hook.Hook, waitTime time.Duration) (*hook.HookResult, error)
// RefreshState asks the remote tablet to reload its tablet record // RefreshState asks the remote tablet to reload its tablet record
RefreshState(tablet *topo.TabletInfo, waitTime time.Duration) error RefreshState(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error
// RunHealthCheck asks the remote tablet to run a health check cycle // RunHealthCheck asks the remote tablet to run a health check cycle
RunHealthCheck(tablet *topo.TabletInfo, targetTabletType topo.TabletType, waitTime time.Duration) error RunHealthCheck(ctx context.Context, tablet *topo.TabletInfo, targetTabletType topo.TabletType, waitTime time.Duration) error
// ReloadSchema asks the remote tablet to reload its schema // ReloadSchema asks the remote tablet to reload its schema
ReloadSchema(tablet *topo.TabletInfo, waitTime time.Duration) error ReloadSchema(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error
// PreflightSchema will test a schema change // PreflightSchema will test a schema change
PreflightSchema(tablet *topo.TabletInfo, change string, waitTime time.Duration) (*myproto.SchemaChangeResult, error) PreflightSchema(ctx context.Context, tablet *topo.TabletInfo, change string, waitTime time.Duration) (*myproto.SchemaChangeResult, error)
// ApplySchema will apply a schema change // ApplySchema will apply a schema change
ApplySchema(tablet *topo.TabletInfo, change *myproto.SchemaChange, waitTime time.Duration) (*myproto.SchemaChangeResult, error) ApplySchema(ctx context.Context, tablet *topo.TabletInfo, change *myproto.SchemaChange, waitTime time.Duration) (*myproto.SchemaChangeResult, error)
// ExecuteFetch executes a query remotely using the DBA pool // ExecuteFetch executes a query remotely using the DBA pool
ExecuteFetch(tablet *topo.TabletInfo, query string, maxRows int, wantFields, disableBinlogs bool, waitTime time.Duration) (*mproto.QueryResult, error) ExecuteFetch(ctx context.Context, tablet *topo.TabletInfo, query string, maxRows int, wantFields, disableBinlogs bool, waitTime time.Duration) (*mproto.QueryResult, error)
// //
// Replication related methods // Replication related methods
// //
// SlaveStatus returns the tablet's mysql slave status. // SlaveStatus returns the tablet's mysql slave status.
SlaveStatus(tablet *topo.TabletInfo, waitTime time.Duration) (*myproto.ReplicationStatus, error) SlaveStatus(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) (*myproto.ReplicationStatus, error)
// WaitSlavePosition asks the tablet to wait until it reaches that // WaitSlavePosition asks the tablet to wait until it reaches that
// position in mysql replication // position in mysql replication
WaitSlavePosition(tablet *topo.TabletInfo, waitPos myproto.ReplicationPosition, waitTime time.Duration) (*myproto.ReplicationStatus, error) WaitSlavePosition(ctx context.Context, tablet *topo.TabletInfo, waitPos myproto.ReplicationPosition, waitTime time.Duration) (*myproto.ReplicationStatus, error)
// MasterPosition returns the tablet's master position // MasterPosition returns the tablet's master position
MasterPosition(tablet *topo.TabletInfo, waitTime time.Duration) (myproto.ReplicationPosition, error) MasterPosition(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) (myproto.ReplicationPosition, error)
// ReparentPosition returns the data for a slave to use to reparent // ReparentPosition returns the data for a slave to use to reparent
// to the target tablet at the given position. // to the target tablet at the given position.
ReparentPosition(tablet *topo.TabletInfo, rp *myproto.ReplicationPosition, waitTime time.Duration) (*actionnode.RestartSlaveData, error) ReparentPosition(ctx context.Context, tablet *topo.TabletInfo, rp *myproto.ReplicationPosition, waitTime time.Duration) (*actionnode.RestartSlaveData, error)
// StopSlave stops the mysql replication // StopSlave stops the mysql replication
StopSlave(tablet *topo.TabletInfo, waitTime time.Duration) error StopSlave(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error
// StopSlaveMinimum stops the mysql replication after it reaches // StopSlaveMinimum stops the mysql replication after it reaches
// the provided minimum point // the provided minimum point
StopSlaveMinimum(tablet *topo.TabletInfo, stopPos myproto.ReplicationPosition, waitTime time.Duration) (*myproto.ReplicationStatus, error) StopSlaveMinimum(ctx context.Context, tablet *topo.TabletInfo, stopPos myproto.ReplicationPosition, waitTime time.Duration) (*myproto.ReplicationStatus, error)
// StartSlave starts the mysql replication // StartSlave starts the mysql replication
StartSlave(tablet *topo.TabletInfo, waitTime time.Duration) error StartSlave(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error
// TabletExternallyReparented tells a tablet it is now the master // TabletExternallyReparented tells a tablet it is now the master
TabletExternallyReparented(tablet *topo.TabletInfo, waitTime time.Duration) error TabletExternallyReparented(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error
// GetSlaves returns the addresses of the slaves // GetSlaves returns the addresses of the slaves
GetSlaves(tablet *topo.TabletInfo, waitTime time.Duration) ([]string, error) GetSlaves(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) ([]string, error)
// WaitBlpPosition asks the tablet to wait until it reaches that // WaitBlpPosition asks the tablet to wait until it reaches that
// position in replication // position in replication
WaitBlpPosition(tablet *topo.TabletInfo, blpPosition blproto.BlpPosition, waitTime time.Duration) error WaitBlpPosition(ctx context.Context, tablet *topo.TabletInfo, blpPosition blproto.BlpPosition, waitTime time.Duration) error
// StopBlp asks the tablet to stop all its binlog players, // StopBlp asks the tablet to stop all its binlog players,
// and returns the current position for all of them // and returns the current position for all of them
StopBlp(tablet *topo.TabletInfo, waitTime time.Duration) (*blproto.BlpPositionList, error) StopBlp(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) (*blproto.BlpPositionList, error)
// StartBlp asks the tablet to restart its binlog players // StartBlp asks the tablet to restart its binlog players
StartBlp(tablet *topo.TabletInfo, waitTime time.Duration) error StartBlp(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error
// RunBlpUntil asks the tablet to restart its binlog players until // RunBlpUntil asks the tablet to restart its binlog players until
// it reaches the given positions, if not there yet. // it reaches the given positions, if not there yet.
RunBlpUntil(tablet *topo.TabletInfo, positions *blproto.BlpPositionList, waitTime time.Duration) (myproto.ReplicationPosition, error) RunBlpUntil(ctx context.Context, tablet *topo.TabletInfo, positions *blproto.BlpPositionList, waitTime time.Duration) (myproto.ReplicationPosition, error)
// //
// Reparenting related functions // Reparenting related functions
// //
// DemoteMaster tells the soon-to-be-former master it's gonna change // DemoteMaster tells the soon-to-be-former master it's gonna change
DemoteMaster(tablet *topo.TabletInfo, waitTime time.Duration) error DemoteMaster(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error
// PromoteSlave transforms the tablet from a slave to a master. // PromoteSlave transforms the tablet from a slave to a master.
PromoteSlave(tablet *topo.TabletInfo, waitTime time.Duration) (*actionnode.RestartSlaveData, error) PromoteSlave(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) (*actionnode.RestartSlaveData, error)
// SlaveWasPromoted tells the remote tablet it is now the master // SlaveWasPromoted tells the remote tablet it is now the master
SlaveWasPromoted(tablet *topo.TabletInfo, waitTime time.Duration) error SlaveWasPromoted(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error
// RestartSlave tells the remote tablet it has a new master // RestartSlave tells the remote tablet it has a new master
RestartSlave(tablet *topo.TabletInfo, rsd *actionnode.RestartSlaveData, waitTime time.Duration) error RestartSlave(ctx context.Context, tablet *topo.TabletInfo, rsd *actionnode.RestartSlaveData, waitTime time.Duration) error
// SlaveWasRestarted tells the remote tablet its master has changed // SlaveWasRestarted tells the remote tablet its master has changed
SlaveWasRestarted(tablet *topo.TabletInfo, args *actionnode.SlaveWasRestartedArgs, waitTime time.Duration) error SlaveWasRestarted(ctx context.Context, tablet *topo.TabletInfo, args *actionnode.SlaveWasRestartedArgs, waitTime time.Duration) error
// BreakSlaves will tinker with the replication stream in a // BreakSlaves will tinker with the replication stream in a
// way that will stop all the slaves. // way that will stop all the slaves.
BreakSlaves(tablet *topo.TabletInfo, waitTime time.Duration) error BreakSlaves(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error
// //
// Backup / restore related methods // Backup / restore related methods
// //
// Snapshot takes a database snapshot // Snapshot takes a database snapshot
Snapshot(tablet *topo.TabletInfo, sa *actionnode.SnapshotArgs, waitTime time.Duration) (<-chan *logutil.LoggerEvent, SnapshotReplyFunc, error) Snapshot(ctx context.Context, tablet *topo.TabletInfo, sa *actionnode.SnapshotArgs, waitTime time.Duration) (<-chan *logutil.LoggerEvent, SnapshotReplyFunc, error)
// SnapshotSourceEnd restarts the mysql server // SnapshotSourceEnd restarts the mysql server
SnapshotSourceEnd(tablet *topo.TabletInfo, ssea *actionnode.SnapshotSourceEndArgs, waitTime time.Duration) error SnapshotSourceEnd(ctx context.Context, tablet *topo.TabletInfo, ssea *actionnode.SnapshotSourceEndArgs, waitTime time.Duration) error
// ReserveForRestore will prepare a server for restore // ReserveForRestore will prepare a server for restore
ReserveForRestore(tablet *topo.TabletInfo, rfra *actionnode.ReserveForRestoreArgs, waitTime time.Duration) error ReserveForRestore(ctx context.Context, tablet *topo.TabletInfo, rfra *actionnode.ReserveForRestoreArgs, waitTime time.Duration) error
// Restore restores a database snapshot // Restore restores a database snapshot
Restore(tablet *topo.TabletInfo, sa *actionnode.RestoreArgs, waitTime time.Duration) (<-chan *logutil.LoggerEvent, ErrFunc, error) Restore(ctx context.Context, tablet *topo.TabletInfo, sa *actionnode.RestoreArgs, waitTime time.Duration) (<-chan *logutil.LoggerEvent, ErrFunc, error)
// MultiSnapshot takes a database snapshot // MultiSnapshot takes a database snapshot
MultiSnapshot(tablet *topo.TabletInfo, sa *actionnode.MultiSnapshotArgs, waitTime time.Duration) (<-chan *logutil.LoggerEvent, MultiSnapshotReplyFunc, error) MultiSnapshot(ctx context.Context, tablet *topo.TabletInfo, sa *actionnode.MultiSnapshotArgs, waitTime time.Duration) (<-chan *logutil.LoggerEvent, MultiSnapshotReplyFunc, error)
// MultiRestore restores a database snapshot // MultiRestore restores a database snapshot
MultiRestore(tablet *topo.TabletInfo, sa *actionnode.MultiRestoreArgs, waitTime time.Duration) (<-chan *logutil.LoggerEvent, ErrFunc, error) MultiRestore(ctx context.Context, tablet *topo.TabletInfo, sa *actionnode.MultiRestoreArgs, waitTime time.Duration) (<-chan *logutil.LoggerEvent, ErrFunc, error)
} }
type TabletManagerClientFactory func() TabletManagerClient type TabletManagerClientFactory func() TabletManagerClient

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

@ -41,7 +41,7 @@ type TabletBson struct {
} }
// DialTablet creates and initializes TabletBson. // DialTablet creates and initializes TabletBson.
func DialTablet(context context.Context, endPoint topo.EndPoint, keyspace, shard string, timeout time.Duration) (tabletconn.TabletConn, error) { func DialTablet(ctx context.Context, endPoint topo.EndPoint, keyspace, shard string, timeout time.Duration) (tabletconn.TabletConn, error) {
var addr string var addr string
var config *tls.Config var config *tls.Config
if *tabletBsonEncrypted { if *tabletBsonEncrypted {
@ -64,7 +64,7 @@ func DialTablet(context context.Context, endPoint topo.EndPoint, keyspace, shard
} }
var sessionInfo tproto.SessionInfo var sessionInfo tproto.SessionInfo
if err = conn.rpcClient.Call("SqlQuery.GetSessionId", tproto.SessionParams{Keyspace: keyspace, Shard: shard}, &sessionInfo); err != nil { if err = conn.rpcClient.Call(ctx, "SqlQuery.GetSessionId", tproto.SessionParams{Keyspace: keyspace, Shard: shard}, &sessionInfo); err != nil {
conn.rpcClient.Close() conn.rpcClient.Close()
return nil, tabletError(err) return nil, tabletError(err)
} }
@ -73,7 +73,7 @@ func DialTablet(context context.Context, endPoint topo.EndPoint, keyspace, shard
} }
// Execute sends the query to VTTablet. // Execute sends the query to VTTablet.
func (conn *TabletBson) Execute(context context.Context, query string, bindVars map[string]interface{}, transactionID int64) (*mproto.QueryResult, error) { func (conn *TabletBson) Execute(ctx context.Context, query string, bindVars map[string]interface{}, transactionID int64) (*mproto.QueryResult, error) {
conn.mu.RLock() conn.mu.RLock()
defer conn.mu.RUnlock() defer conn.mu.RUnlock()
if conn.rpcClient == nil { if conn.rpcClient == nil {
@ -87,14 +87,14 @@ func (conn *TabletBson) Execute(context context.Context, query string, bindVars
SessionId: conn.sessionID, SessionId: conn.sessionID,
} }
qr := new(mproto.QueryResult) qr := new(mproto.QueryResult)
if err := conn.rpcClient.Call("SqlQuery.Execute", req, qr); err != nil { if err := conn.rpcClient.Call(ctx, "SqlQuery.Execute", req, qr); err != nil {
return nil, tabletError(err) return nil, tabletError(err)
} }
return qr, nil return qr, nil
} }
// ExecuteBatch sends a batch query to VTTablet. // ExecuteBatch sends a batch query to VTTablet.
func (conn *TabletBson) ExecuteBatch(context context.Context, queries []tproto.BoundQuery, transactionID int64) (*tproto.QueryResultList, error) { func (conn *TabletBson) ExecuteBatch(ctx context.Context, queries []tproto.BoundQuery, transactionID int64) (*tproto.QueryResultList, error) {
conn.mu.RLock() conn.mu.RLock()
defer conn.mu.RUnlock() defer conn.mu.RUnlock()
if conn.rpcClient == nil { if conn.rpcClient == nil {
@ -107,14 +107,14 @@ func (conn *TabletBson) ExecuteBatch(context context.Context, queries []tproto.B
SessionId: conn.sessionID, SessionId: conn.sessionID,
} }
qrs := new(tproto.QueryResultList) qrs := new(tproto.QueryResultList)
if err := conn.rpcClient.Call("SqlQuery.ExecuteBatch", req, qrs); err != nil { if err := conn.rpcClient.Call(ctx, "SqlQuery.ExecuteBatch", req, qrs); err != nil {
return nil, tabletError(err) return nil, tabletError(err)
} }
return qrs, nil return qrs, nil
} }
// StreamExecute starts a streaming query to VTTablet. // StreamExecute starts a streaming query to VTTablet.
func (conn *TabletBson) StreamExecute(context context.Context, query string, bindVars map[string]interface{}, transactionID int64) (<-chan *mproto.QueryResult, tabletconn.ErrFunc, error) { func (conn *TabletBson) StreamExecute(ctx context.Context, query string, bindVars map[string]interface{}, transactionID int64) (<-chan *mproto.QueryResult, tabletconn.ErrFunc, error) {
conn.mu.RLock() conn.mu.RLock()
defer conn.mu.RUnlock() defer conn.mu.RUnlock()
if conn.rpcClient == nil { if conn.rpcClient == nil {
@ -145,7 +145,7 @@ func (conn *TabletBson) StreamExecute(context context.Context, query string, bin
} }
// Begin starts a transaction. // Begin starts a transaction.
func (conn *TabletBson) Begin(context context.Context) (transactionID int64, err error) { func (conn *TabletBson) Begin(ctx context.Context) (transactionID int64, err error) {
conn.mu.RLock() conn.mu.RLock()
defer conn.mu.RUnlock() defer conn.mu.RUnlock()
if conn.rpcClient == nil { if conn.rpcClient == nil {
@ -156,12 +156,12 @@ func (conn *TabletBson) Begin(context context.Context) (transactionID int64, err
SessionId: conn.sessionID, SessionId: conn.sessionID,
} }
var txInfo tproto.TransactionInfo var txInfo tproto.TransactionInfo
err = conn.rpcClient.Call("SqlQuery.Begin", req, &txInfo) err = conn.rpcClient.Call(ctx, "SqlQuery.Begin", req, &txInfo)
return txInfo.TransactionId, tabletError(err) return txInfo.TransactionId, tabletError(err)
} }
// Commit commits the ongoing transaction. // Commit commits the ongoing transaction.
func (conn *TabletBson) Commit(context context.Context, transactionID int64) error { func (conn *TabletBson) Commit(ctx context.Context, transactionID int64) error {
conn.mu.RLock() conn.mu.RLock()
defer conn.mu.RUnlock() defer conn.mu.RUnlock()
if conn.rpcClient == nil { if conn.rpcClient == nil {
@ -172,11 +172,11 @@ func (conn *TabletBson) Commit(context context.Context, transactionID int64) err
SessionId: conn.sessionID, SessionId: conn.sessionID,
TransactionId: transactionID, TransactionId: transactionID,
} }
return tabletError(conn.rpcClient.Call("SqlQuery.Commit", req, &rpc.Unused{})) return tabletError(conn.rpcClient.Call(ctx, "SqlQuery.Commit", req, &rpc.Unused{}))
} }
// Rollback rolls back the ongoing transaction. // Rollback rolls back the ongoing transaction.
func (conn *TabletBson) Rollback(context context.Context, transactionID int64) error { func (conn *TabletBson) Rollback(ctx context.Context, transactionID int64) error {
conn.mu.RLock() conn.mu.RLock()
defer conn.mu.RUnlock() defer conn.mu.RUnlock()
if conn.rpcClient == nil { if conn.rpcClient == nil {
@ -187,11 +187,11 @@ func (conn *TabletBson) Rollback(context context.Context, transactionID int64) e
SessionId: conn.sessionID, SessionId: conn.sessionID,
TransactionId: transactionID, TransactionId: transactionID,
} }
return tabletError(conn.rpcClient.Call("SqlQuery.Rollback", req, &rpc.Unused{})) return tabletError(conn.rpcClient.Call(ctx, "SqlQuery.Rollback", req, &rpc.Unused{}))
} }
// SplitQuery is the stub for SqlQuery.SplitQuery RPC // SplitQuery is the stub for SqlQuery.SplitQuery RPC
func (conn *TabletBson) SplitQuery(context context.Context, query tproto.BoundQuery, splitCount int) (queries []tproto.QuerySplit, err error) { func (conn *TabletBson) SplitQuery(ctx context.Context, query tproto.BoundQuery, splitCount int) (queries []tproto.QuerySplit, err error) {
conn.mu.RLock() conn.mu.RLock()
defer conn.mu.RUnlock() defer conn.mu.RUnlock()
if conn.rpcClient == nil { if conn.rpcClient == nil {
@ -203,7 +203,7 @@ func (conn *TabletBson) SplitQuery(context context.Context, query tproto.BoundQu
SplitCount: splitCount, SplitCount: splitCount,
} }
reply := new(tproto.SplitQueryResult) reply := new(tproto.SplitQueryResult)
if err := conn.rpcClient.Call("SqlQuery.SplitQuery", req, reply); err != nil { if err := conn.rpcClient.Call(ctx, "SqlQuery.SplitQuery", req, reply); err != nil {
return nil, tabletError(err) return nil, tabletError(err)
} }
return reply.Queries, nil return reply.Queries, nil

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

@ -8,6 +8,8 @@ import (
"fmt" "fmt"
"testing" "testing"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/vt/topo" "github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/test" "github.com/youtube/vitess/go/vt/topo/test"
"github.com/youtube/vitess/go/vt/zktopo" "github.com/youtube/vitess/go/vt/zktopo"
@ -57,7 +59,7 @@ func TestShard(t *testing.T) {
func TestTablet(t *testing.T) { func TestTablet(t *testing.T) {
ts := newFakeTeeServer(t) ts := newFakeTeeServer(t)
test.CheckTablet(t, ts) test.CheckTablet(context.Background(), t, ts)
} }
func TestServingGraph(t *testing.T) { func TestServingGraph(t *testing.T) {

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

@ -5,8 +5,10 @@
package topo package topo
import ( import (
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
"github.com/youtube/vitess/go/trace"
"github.com/youtube/vitess/go/vt/logutil" "github.com/youtube/vitess/go/vt/logutil"
) )
@ -72,7 +74,14 @@ func (sri *ShardReplicationInfo) Shard() string {
// UpdateShardReplicationRecord is a low level function to add / update an // UpdateShardReplicationRecord is a low level function to add / update an
// entry to the ShardReplication object. // entry to the ShardReplication object.
func UpdateShardReplicationRecord(ts Server, keyspace, shard string, tabletAlias, parent TabletAlias) error { func UpdateShardReplicationRecord(ctx context.Context, ts Server, keyspace, shard string, tabletAlias, parent TabletAlias) error {
span := trace.NewSpanFromContext(ctx)
span.StartClient("TopoServer.UpdateShardReplicationFields")
span.Annotate("keyspace", keyspace)
span.Annotate("shard", shard)
span.Annotate("tablet", tabletAlias.String())
defer span.Finish()
return ts.UpdateShardReplicationFields(tabletAlias.Cell, keyspace, shard, func(sr *ShardReplication) error { return ts.UpdateShardReplicationFields(tabletAlias.Cell, keyspace, shard, func(sr *ShardReplication) error {
// not very efficient, but easy to read // not very efficient, but easy to read
links := make([]ReplicationLink, 0, len(sr.ReplicationLinks)+1) links := make([]ReplicationLink, 0, len(sr.ReplicationLinks)+1)

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

@ -11,8 +11,11 @@ import (
"strings" "strings"
"sync" "sync"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
"github.com/youtube/vitess/go/trace"
"github.com/youtube/vitess/go/vt/concurrency" "github.com/youtube/vitess/go/vt/concurrency"
"github.com/youtube/vitess/go/vt/key" "github.com/youtube/vitess/go/vt/key"
) )
@ -234,7 +237,13 @@ func NewShardInfo(keyspace, shard string, value *Shard, version int64) *ShardInf
} }
// UpdateShard updates the shard data, with the right version // UpdateShard updates the shard data, with the right version
func UpdateShard(ts Server, si *ShardInfo) error { func UpdateShard(ctx context.Context, ts Server, si *ShardInfo) error {
span := trace.NewSpanFromContext(ctx)
span.StartClient("TopoServer.UpdateShard")
span.Annotate("keyspace", si.Keyspace())
span.Annotate("shard", si.ShardName())
defer span.Finish()
var version int64 = -1 var version int64 = -1
if si.version != 0 { if si.version != 0 {
version = si.version version = si.version

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

@ -12,8 +12,11 @@ import (
"strings" "strings"
"sync" "sync"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
"github.com/youtube/vitess/go/jscfg" "github.com/youtube/vitess/go/jscfg"
"github.com/youtube/vitess/go/trace"
"github.com/youtube/vitess/go/vt/key" "github.com/youtube/vitess/go/vt/key"
) )
@ -523,7 +526,12 @@ func NewTabletInfo(tablet *Tablet, version int64) *TabletInfo {
} }
// UpdateTablet updates the tablet data only - not associated replication paths. // UpdateTablet updates the tablet data only - not associated replication paths.
func UpdateTablet(ts Server, tablet *TabletInfo) error { func UpdateTablet(ctx context.Context, ts Server, tablet *TabletInfo) error {
span := trace.NewSpanFromContext(ctx)
span.StartClient("TopoServer.UpdateTablet")
span.Annotate("tablet", tablet.Alias.String())
defer span.Finish()
var version int64 = -1 var version int64 = -1
if tablet.version != 0 { if tablet.version != 0 {
version = tablet.version version = tablet.version
@ -628,13 +636,13 @@ func CreateTablet(ts Server, tablet *Tablet) error {
return nil return nil
} }
return UpdateTabletReplicationData(ts, tablet) return UpdateTabletReplicationData(context.TODO(), ts, tablet)
} }
// UpdateTabletReplicationData creates or updates the replication // UpdateTabletReplicationData creates or updates the replication
// graph data for a tablet // graph data for a tablet
func UpdateTabletReplicationData(ts Server, tablet *Tablet) error { func UpdateTabletReplicationData(ctx context.Context, ts Server, tablet *Tablet) error {
return UpdateShardReplicationRecord(ts, tablet.Keyspace, tablet.Shard, tablet.Alias, tablet.Parent) return UpdateShardReplicationRecord(ctx, ts, tablet.Keyspace, tablet.Shard, tablet.Alias, tablet.Parent)
} }
// DeleteTabletReplicationData deletes replication data. // DeleteTabletReplicationData deletes replication data.

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

@ -8,6 +8,8 @@ import (
"encoding/json" "encoding/json"
"testing" "testing"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/vt/topo" "github.com/youtube/vitess/go/vt/topo"
) )
@ -72,7 +74,7 @@ func CheckShard(t *testing.T, ts topo.Server) {
DisableQueryService: true, DisableQueryService: true,
}, },
} }
if err := topo.UpdateShard(ts, shardInfo); err != nil { if err := topo.UpdateShard(context.TODO(), ts, shardInfo); err != nil {
t.Errorf("UpdateShard: %v", err) t.Errorf("UpdateShard: %v", err)
} }

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

@ -9,6 +9,8 @@ import (
"testing" "testing"
"time" "time"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/vt/topo" "github.com/youtube/vitess/go/vt/topo"
) )
@ -24,7 +26,7 @@ func tabletEqual(left, right *topo.Tablet) (bool, error) {
return string(lj) == string(rj), nil return string(lj) == string(rj), nil
} }
func CheckTablet(t *testing.T, ts topo.Server) { func CheckTablet(ctx context.Context, t *testing.T, ts topo.Server) {
cell := getLocalCell(t, ts) cell := getLocalCell(t, ts)
tablet := &topo.Tablet{ tablet := &topo.Tablet{
Alias: topo.TabletAlias{Cell: cell, Uid: 1}, Alias: topo.TabletAlias{Cell: cell, Uid: 1},
@ -75,7 +77,7 @@ func CheckTablet(t *testing.T, ts topo.Server) {
} }
ti.State = topo.STATE_READ_ONLY ti.State = topo.STATE_READ_ONLY
if err := topo.UpdateTablet(ts, ti); err != nil { if err := topo.UpdateTablet(ctx, ts, ti); err != nil {
t.Errorf("UpdateTablet: %v", err) t.Errorf("UpdateTablet: %v", err)
} }

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

@ -10,6 +10,8 @@ import (
"sync" "sync"
"time" "time"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/trace"
"github.com/youtube/vitess/go/vt/concurrency" "github.com/youtube/vitess/go/vt/concurrency"
"github.com/youtube/vitess/go/vt/logutil" "github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode" "github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
@ -27,9 +29,14 @@ var UseSrvShardLocks = flag.Bool("use_srv_shard_locks", true, "DEPRECATED: If tr
// //
// This function locks individual SvrShard paths, so it doesn't need a lock // This function locks individual SvrShard paths, so it doesn't need a lock
// on the shard. // on the shard.
func RebuildShard(log logutil.Logger, ts topo.Server, keyspace, shard string, cells []string, timeout time.Duration, interrupted chan struct{}) (*topo.ShardInfo, error) { func RebuildShard(ctx context.Context, log logutil.Logger, ts topo.Server, keyspace, shard string, cells []string, timeout time.Duration, interrupted chan struct{}) (*topo.ShardInfo, error) {
log.Infof("RebuildShard %v/%v", keyspace, shard) log.Infof("RebuildShard %v/%v", keyspace, shard)
span := trace.NewSpanFromContext(ctx)
span.StartLocal("topotools.RebuildShard")
defer span.Finish()
ctx = trace.NewContext(ctx, span)
// read the existing shard info. It has to exist. // read the existing shard info. It has to exist.
shardInfo, err := ts.GetShard(keyspace, shard) shardInfo, err := ts.GetShard(keyspace, shard)
if err != nil { if err != nil {
@ -58,7 +65,7 @@ func RebuildShard(log logutil.Logger, ts topo.Server, keyspace, shard string, ce
// Lock the SrvShard so we don't race with other rebuilds of the same // Lock the SrvShard so we don't race with other rebuilds of the same
// shard in the same cell (e.g. from our peer tablets). // shard in the same cell (e.g. from our peer tablets).
actionNode := actionnode.RebuildSrvShard() actionNode := actionnode.RebuildSrvShard()
lockPath, err := actionNode.LockSrvShard(ts, cell, keyspace, shard, timeout, interrupted) lockPath, err := actionNode.LockSrvShard(ctx, ts, cell, keyspace, shard, timeout, interrupted)
if err != nil { if err != nil {
rec.RecordError(err) rec.RecordError(err)
return return
@ -98,7 +105,7 @@ func RebuildShard(log logutil.Logger, ts topo.Server, keyspace, shard string, ce
} }
// write the data we need to // write the data we need to
rebuildErr := rebuildCellSrvShard(log, ts, shardInfo, cell, tablets) rebuildErr := rebuildCellSrvShard(ctx, log, ts, shardInfo, cell, tablets)
// and unlock // and unlock
if err := actionNode.UnlockSrvShard(ts, cell, keyspace, shard, lockPath, rebuildErr); err != nil { if err := actionNode.UnlockSrvShard(ts, cell, keyspace, shard, lockPath, rebuildErr); err != nil {
@ -113,7 +120,7 @@ func RebuildShard(log logutil.Logger, ts topo.Server, keyspace, shard string, ce
// rebuildCellSrvShard computes and writes the serving graph data to a // rebuildCellSrvShard computes and writes the serving graph data to a
// single cell // single cell
func rebuildCellSrvShard(log logutil.Logger, ts topo.Server, shardInfo *topo.ShardInfo, cell string, tablets map[topo.TabletAlias]*topo.TabletInfo) error { func rebuildCellSrvShard(ctx context.Context, log logutil.Logger, ts topo.Server, shardInfo *topo.ShardInfo, cell string, tablets map[topo.TabletAlias]*topo.TabletInfo) error {
log.Infof("rebuildCellSrvShard %v/%v in cell %v", shardInfo.Keyspace(), shardInfo.ShardName(), cell) log.Infof("rebuildCellSrvShard %v/%v in cell %v", shardInfo.Keyspace(), shardInfo.ShardName(), cell)
// Get all existing db types so they can be removed if nothing // Get all existing db types so they can be removed if nothing
@ -178,9 +185,13 @@ func rebuildCellSrvShard(log logutil.Logger, ts topo.Server, shardInfo *topo.Sha
wg.Add(1) wg.Add(1)
go func(tabletType topo.TabletType, addrs *topo.EndPoints) { go func(tabletType topo.TabletType, addrs *topo.EndPoints) {
log.Infof("saving serving graph for cell %v shard %v/%v tabletType %v", cell, shardInfo.Keyspace(), shardInfo.ShardName(), tabletType) log.Infof("saving serving graph for cell %v shard %v/%v tabletType %v", cell, shardInfo.Keyspace(), shardInfo.ShardName(), tabletType)
span := trace.NewSpanFromContext(ctx)
span.StartClient("TopoServer.UpdateEndPoints")
span.Annotate("tablet_type", string(tabletType))
if err := ts.UpdateEndPoints(cell, shardInfo.Keyspace(), shardInfo.ShardName(), tabletType, addrs); err != nil { if err := ts.UpdateEndPoints(cell, shardInfo.Keyspace(), shardInfo.ShardName(), tabletType, addrs); err != nil {
rec.RecordError(fmt.Errorf("writing endpoints for cell %v shard %v/%v tabletType %v failed: %v", cell, shardInfo.Keyspace(), shardInfo.ShardName(), tabletType, err)) rec.RecordError(fmt.Errorf("writing endpoints for cell %v shard %v/%v tabletType %v failed: %v", cell, shardInfo.Keyspace(), shardInfo.ShardName(), tabletType, err))
} }
span.Finish()
wg.Done() wg.Done()
}(tabletType, addrs) }(tabletType, addrs)
} }
@ -192,9 +203,13 @@ func rebuildCellSrvShard(log logutil.Logger, ts topo.Server, shardInfo *topo.Sha
wg.Add(1) wg.Add(1)
go func(tabletType topo.TabletType) { go func(tabletType topo.TabletType) {
log.Infof("removing stale db type from serving graph: %v", tabletType) log.Infof("removing stale db type from serving graph: %v", tabletType)
span := trace.NewSpanFromContext(ctx)
span.StartClient("TopoServer.DeleteEndPoints")
span.Annotate("tablet_type", string(tabletType))
if err := ts.DeleteEndPoints(cell, shardInfo.Keyspace(), shardInfo.ShardName(), tabletType); err != nil { if err := ts.DeleteEndPoints(cell, shardInfo.Keyspace(), shardInfo.ShardName(), tabletType); err != nil {
log.Warningf("unable to remove stale db type %v from serving graph: %v", tabletType, err) log.Warningf("unable to remove stale db type %v from serving graph: %v", tabletType, err)
} }
span.Finish()
wg.Done() wg.Done()
}(tabletType) }(tabletType)
} }
@ -215,9 +230,15 @@ func rebuildCellSrvShard(log logutil.Logger, ts topo.Server, shardInfo *topo.Sha
srvShard.TabletTypes = append(srvShard.TabletTypes, tabletType) srvShard.TabletTypes = append(srvShard.TabletTypes, tabletType)
} }
span := trace.NewSpanFromContext(ctx)
span.StartClient("TopoServer.UpdateSrvShard")
span.Annotate("keyspace", shardInfo.Keyspace())
span.Annotate("shard", shardInfo.ShardName())
span.Annotate("cell", cell)
if err := ts.UpdateSrvShard(cell, shardInfo.Keyspace(), shardInfo.ShardName(), srvShard); err != nil { if err := ts.UpdateSrvShard(cell, shardInfo.Keyspace(), shardInfo.ShardName(), srvShard); err != nil {
rec.RecordError(fmt.Errorf("writing serving data in cell %v for %v/%v failed: %v", cell, shardInfo.Keyspace(), shardInfo.ShardName(), err)) rec.RecordError(fmt.Errorf("writing serving data in cell %v for %v/%v failed: %v", cell, shardInfo.Keyspace(), shardInfo.ShardName(), err))
} }
span.Finish()
wg.Done() wg.Done()
}() }()

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

@ -9,6 +9,8 @@ import (
"testing" "testing"
"time" "time"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/vt/logutil" "github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/topo" "github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/test/faketopo" "github.com/youtube/vitess/go/vt/topo/test/faketopo"
@ -19,6 +21,7 @@ import (
) )
func TestRebuildShardRace(t *testing.T) { func TestRebuildShardRace(t *testing.T) {
ctx := context.Background()
cells := []string{"test_cell"} cells := []string{"test_cell"}
logger := logutil.NewMemoryLogger() logger := logutil.NewMemoryLogger()
timeout := 10 * time.Second timeout := 10 * time.Second
@ -35,7 +38,7 @@ func TestRebuildShardRace(t *testing.T) {
f.AddTablet(2, "test_cell", topo.TYPE_REPLICA, master) f.AddTablet(2, "test_cell", topo.TYPE_REPLICA, master)
// Do an initial rebuild. // Do an initial rebuild.
if _, err := RebuildShard(logger, f.Topo, keyspace, shard, cells, timeout, interrupted); err != nil { if _, err := RebuildShard(ctx, logger, f.Topo, keyspace, shard, cells, timeout, interrupted); err != nil {
t.Fatalf("RebuildShard: %v", err) t.Fatalf("RebuildShard: %v", err)
} }
@ -73,11 +76,11 @@ func TestRebuildShardRace(t *testing.T) {
// the SrvShard lock. // the SrvShard lock.
masterInfo := f.GetTablet(1) masterInfo := f.GetTablet(1)
masterInfo.Type = topo.TYPE_SPARE masterInfo.Type = topo.TYPE_SPARE
if err := topo.UpdateTablet(ts, masterInfo); err != nil { if err := topo.UpdateTablet(ctx, ts, masterInfo); err != nil {
t.Fatalf("UpdateTablet: %v", err) t.Fatalf("UpdateTablet: %v", err)
} }
go func() { go func() {
if _, err := RebuildShard(logger, f.Topo, keyspace, shard, cells, timeout, interrupted); err != nil { if _, err := RebuildShard(ctx, logger, f.Topo, keyspace, shard, cells, timeout, interrupted); err != nil {
t.Fatalf("RebuildShard: %v", err) t.Fatalf("RebuildShard: %v", err)
} }
close(done) close(done)
@ -90,10 +93,10 @@ func TestRebuildShardRace(t *testing.T) {
// that doesn't stall. // that doesn't stall.
replicaInfo := f.GetTablet(2) replicaInfo := f.GetTablet(2)
replicaInfo.Type = topo.TYPE_SPARE replicaInfo.Type = topo.TYPE_SPARE
if err := topo.UpdateTablet(ts, replicaInfo); err != nil { if err := topo.UpdateTablet(ctx, ts, replicaInfo); err != nil {
t.Fatalf("UpdateTablet: %v", err) t.Fatalf("UpdateTablet: %v", err)
} }
if _, err := RebuildShard(logger, f.Topo, keyspace, shard, cells, timeout, interrupted); err != nil { if _, err := RebuildShard(ctx, logger, f.Topo, keyspace, shard, cells, timeout, interrupted); err != nil {
t.Fatalf("RebuildShard: %v", err) t.Fatalf("RebuildShard: %v", err)
} }

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

@ -11,6 +11,8 @@ package topotools
import ( import (
"sync" "sync"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/vt/logutil" "github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode" "github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
"github.com/youtube/vitess/go/vt/topo" "github.com/youtube/vitess/go/vt/topo"
@ -54,7 +56,7 @@ func RestartSlavesExternal(ts topo.Server, log logutil.Logger, slaveTabletMap, m
ti.Type = topo.TYPE_SPARE ti.Type = topo.TYPE_SPARE
ti.Parent = masterElectTabletAlias ti.Parent = masterElectTabletAlias
if err := topo.UpdateTablet(ts, ti); err != nil { if err := topo.UpdateTablet(context.TODO(), ts, ti); err != nil {
log.Warningf("Failed to change old master %v to spare: %v", ti.Alias, err) log.Warningf("Failed to change old master %v to spare: %v", ti.Alias, err)
} }
} }

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

@ -25,6 +25,8 @@ import (
"fmt" "fmt"
"sync" "sync"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
"github.com/youtube/vitess/go/vt/concurrency" "github.com/youtube/vitess/go/vt/concurrency"
"github.com/youtube/vitess/go/vt/hook" "github.com/youtube/vitess/go/vt/hook"
@ -60,7 +62,7 @@ func Scrap(ts topo.Server, tabletAlias topo.TabletAlias, force bool) error {
tablet.Type = topo.TYPE_SCRAP tablet.Type = topo.TYPE_SCRAP
tablet.Parent = topo.TabletAlias{} tablet.Parent = topo.TabletAlias{}
// Update the tablet first, since that is canonical. // Update the tablet first, since that is canonical.
err = topo.UpdateTablet(ts, tablet) err = topo.UpdateTablet(context.TODO(), ts, tablet)
if err != nil { if err != nil {
return err return err
} }
@ -163,5 +165,5 @@ func ChangeType(ts topo.Server, tabletAlias topo.TabletAlias, newType topo.Table
tablet.Health = health tablet.Health = health
} }
} }
return topo.UpdateTablet(ts, tablet) return topo.UpdateTablet(context.TODO(), ts, tablet)
} }

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

@ -8,6 +8,8 @@ import (
"flag" "flag"
"fmt" "fmt"
"code.google.com/p/go.net/context"
_ "github.com/youtube/vitess/go/vt/logutil" _ "github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/wrangler" "github.com/youtube/vitess/go/vt/wrangler"
) )
@ -45,7 +47,7 @@ func commandDemoteMaster(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []s
if err != nil { if err != nil {
return err return err
} }
return wr.TabletManagerClient().DemoteMaster(tabletInfo, wr.ActionTimeout()) return wr.TabletManagerClient().DemoteMaster(context.TODO(), tabletInfo, wr.ActionTimeout())
} }
func commandReparentTablet(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { func commandReparentTablet(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error {

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

@ -15,6 +15,7 @@ import (
"strings" "strings"
"time" "time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
"github.com/youtube/vitess/go/flagutil" "github.com/youtube/vitess/go/flagutil"
"github.com/youtube/vitess/go/jscfg" "github.com/youtube/vitess/go/jscfg"
@ -779,7 +780,7 @@ func commandSetReadOnly(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []st
if err != nil { if err != nil {
return fmt.Errorf("failed reading tablet %v: %v", tabletAlias, err) return fmt.Errorf("failed reading tablet %v: %v", tabletAlias, err)
} }
return wr.TabletManagerClient().SetReadOnly(ti, wr.ActionTimeout()) return wr.TabletManagerClient().SetReadOnly(context.TODO(), ti, wr.ActionTimeout())
} }
func commandSetReadWrite(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { func commandSetReadWrite(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error {
@ -798,7 +799,7 @@ func commandSetReadWrite(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []s
if err != nil { if err != nil {
return fmt.Errorf("failed reading tablet %v: %v", tabletAlias, err) return fmt.Errorf("failed reading tablet %v: %v", tabletAlias, err)
} }
return wr.TabletManagerClient().SetReadWrite(ti, wr.ActionTimeout()) return wr.TabletManagerClient().SetReadWrite(context.TODO(), ti, wr.ActionTimeout())
} }
func commandChangeSlaveType(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { func commandChangeSlaveType(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error {
@ -851,7 +852,7 @@ func commandPing(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) e
if err != nil { if err != nil {
return err return err
} }
return wr.TabletManagerClient().Ping(tabletInfo, wr.ActionTimeout()) return wr.TabletManagerClient().Ping(context.TODO(), tabletInfo, wr.ActionTimeout())
} }
func commandRefreshState(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { func commandRefreshState(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error {
@ -869,7 +870,7 @@ func commandRefreshState(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []s
if err != nil { if err != nil {
return err return err
} }
return wr.TabletManagerClient().RefreshState(tabletInfo, wr.ActionTimeout()) return wr.TabletManagerClient().RefreshState(context.TODO(), tabletInfo, wr.ActionTimeout())
} }
func commandRunHealthCheck(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { func commandRunHealthCheck(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error {
@ -891,7 +892,7 @@ func commandRunHealthCheck(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args [
if err != nil { if err != nil {
return err return err
} }
return wr.TabletManagerClient().RunHealthCheck(tabletInfo, servedType, wr.ActionTimeout()) return wr.TabletManagerClient().RunHealthCheck(context.TODO(), tabletInfo, servedType, wr.ActionTimeout())
} }
func commandQuery(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { func commandQuery(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error {
@ -923,7 +924,7 @@ func commandSleep(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string)
if err != nil { if err != nil {
return err return err
} }
return wr.TabletManagerClient().Sleep(ti, duration, wr.ActionTimeout()) return wr.TabletManagerClient().Sleep(context.TODO(), ti, duration, wr.ActionTimeout())
} }
func commandSnapshotSourceEnd(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { func commandSnapshotSourceEnd(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error {
@ -1236,7 +1237,7 @@ func commandShardExternallyReparented(wr *wrangler.Wrangler, subFlags *flag.Flag
if err != nil { if err != nil {
return err return err
} }
return wr.TabletManagerClient().TabletExternallyReparented(ti, wr.ActionTimeout()) return wr.TabletManagerClient().TabletExternallyReparented(context.TODO(), ti, wr.ActionTimeout())
} }
return wr.ShardExternallyReparented(keyspace, shard, tabletAlias) return wr.ShardExternallyReparented(keyspace, shard, tabletAlias)
} }
@ -1464,7 +1465,7 @@ func commandShardReplicationAdd(wr *wrangler.Wrangler, subFlags *flag.FlagSet, a
if err != nil { if err != nil {
return err return err
} }
return topo.UpdateShardReplicationRecord(wr.TopoServer(), keyspace, shard, tabletAlias, parentAlias) return topo.UpdateShardReplicationRecord(context.TODO(), wr.TopoServer(), keyspace, shard, tabletAlias, parentAlias)
} }
func commandShardReplicationRemove(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error { func commandShardReplicationRemove(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) error {

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

@ -13,6 +13,8 @@ import (
"text/template" "text/template"
"time" "time"
"code.google.com/p/go.net/context"
mproto "github.com/youtube/vitess/go/mysql/proto" mproto "github.com/youtube/vitess/go/mysql/proto"
"github.com/youtube/vitess/go/sqltypes" "github.com/youtube/vitess/go/sqltypes"
myproto "github.com/youtube/vitess/go/vt/mysqlctl/proto" myproto "github.com/youtube/vitess/go/vt/mysqlctl/proto"
@ -93,7 +95,7 @@ func runSqlCommands(wr *wrangler.Wrangler, ti *topo.TabletInfo, commands []strin
return fmt.Errorf("fillStringTemplate failed: %v", err) return fmt.Errorf("fillStringTemplate failed: %v", err)
} }
_, err = wr.TabletManagerClient().ExecuteFetch(ti, command, 0, false, true, 30*time.Second) _, err = wr.TabletManagerClient().ExecuteFetch(context.TODO(), ti, command, 0, false, true, 30*time.Second)
if err != nil { if err != nil {
return err return err
} }
@ -132,7 +134,7 @@ func findChunks(wr *wrangler.Wrangler, ti *topo.TabletInfo, td *myproto.TableDef
// get the min and max of the leading column of the primary key // get the min and max of the leading column of the primary key
query := fmt.Sprintf("SELECT MIN(%v), MAX(%v) FROM %v.%v", td.PrimaryKeyColumns[0], td.PrimaryKeyColumns[0], ti.DbName(), td.Name) query := fmt.Sprintf("SELECT MIN(%v), MAX(%v) FROM %v.%v", td.PrimaryKeyColumns[0], td.PrimaryKeyColumns[0], ti.DbName(), td.Name)
qr, err := wr.TabletManagerClient().ExecuteFetch(ti, query, 1, true, false, 30*time.Second) qr, err := wr.TabletManagerClient().ExecuteFetch(context.TODO(), ti, query, 1, true, false, 30*time.Second)
if err != nil { if err != nil {
wr.Logger().Infof("Not splitting table %v into multiple chunks: %v", td.Name, err) wr.Logger().Infof("Not splitting table %v into multiple chunks: %v", td.Name, err)
return result, nil return result, nil
@ -295,7 +297,7 @@ func executeFetchLoop(wr *wrangler.Wrangler, ti *topo.TabletInfo, insertChannel
return nil return nil
} }
cmd = "INSERT INTO `" + ti.DbName() + "`." + cmd cmd = "INSERT INTO `" + ti.DbName() + "`." + cmd
_, err := wr.TabletManagerClient().ExecuteFetch(ti, cmd, 0, false, true, 30*time.Second) _, err := wr.TabletManagerClient().ExecuteFetch(context.TODO(), ti, cmd, 0, false, true, 30*time.Second)
if err != nil { if err != nil {
return fmt.Errorf("ExecuteFetch failed: %v", err) return fmt.Errorf("ExecuteFetch failed: %v", err)
} }

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

@ -10,6 +10,8 @@ import (
"strings" "strings"
"time" "time"
"code.google.com/p/go.net/context"
mproto "github.com/youtube/vitess/go/mysql/proto" mproto "github.com/youtube/vitess/go/mysql/proto"
rpc "github.com/youtube/vitess/go/rpcplus" rpc "github.com/youtube/vitess/go/rpcplus"
"github.com/youtube/vitess/go/rpcwrap/bsonrpc" "github.com/youtube/vitess/go/rpcwrap/bsonrpc"
@ -44,7 +46,7 @@ func NewQueryResultReaderForTablet(ts topo.Server, tabletAlias topo.TabletAlias,
} }
var sessionInfo tproto.SessionInfo var sessionInfo tproto.SessionInfo
if err := rpcClient.Call("SqlQuery.GetSessionId", tproto.SessionParams{Keyspace: tablet.Keyspace, Shard: tablet.Shard}, &sessionInfo); err != nil { if err := rpcClient.Call(context.TODO(), "SqlQuery.GetSessionId", tproto.SessionParams{Keyspace: tablet.Keyspace, Shard: tablet.Shard}, &sessionInfo); err != nil {
return nil, err return nil, err
} }

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

@ -11,6 +11,8 @@ import (
"sync" "sync"
"time" "time"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/event" "github.com/youtube/vitess/go/event"
"github.com/youtube/vitess/go/sync2" "github.com/youtube/vitess/go/sync2"
"github.com/youtube/vitess/go/vt/binlog/binlogplayer" "github.com/youtube/vitess/go/vt/binlog/binlogplayer"
@ -311,7 +313,7 @@ func (scw *SplitCloneWorker) findTargets() error {
return fmt.Errorf("cannot read tablet %v: %v", alias, err) return fmt.Errorf("cannot read tablet %v: %v", alias, err)
} }
if err := scw.wr.TabletManagerClient().StopSlave(scw.sourceTablets[i], 30*time.Second); err != nil { if err := scw.wr.TabletManagerClient().StopSlave(context.TODO(), scw.sourceTablets[i], 30*time.Second); err != nil {
return fmt.Errorf("cannot stop replication on tablet %v", alias) return fmt.Errorf("cannot stop replication on tablet %v", alias)
} }
@ -578,7 +580,7 @@ func (scw *SplitCloneWorker) copy() error {
// get the current position from the sources // get the current position from the sources
for shardIndex, _ := range scw.sourceShards { for shardIndex, _ := range scw.sourceShards {
status, err := scw.wr.TabletManagerClient().SlaveStatus(scw.sourceTablets[shardIndex], 30*time.Second) status, err := scw.wr.TabletManagerClient().SlaveStatus(context.TODO(), scw.sourceTablets[shardIndex], 30*time.Second)
if err != nil { if err != nil {
return err return err
} }
@ -628,7 +630,7 @@ func (scw *SplitCloneWorker) copy() error {
go func(ti *topo.TabletInfo) { go func(ti *topo.TabletInfo) {
defer destinationWaitGroup.Done() defer destinationWaitGroup.Done()
scw.wr.Logger().Infof("Reloading schema on tablet %v", ti.Alias) scw.wr.Logger().Infof("Reloading schema on tablet %v", ti.Alias)
if err := scw.wr.TabletManagerClient().ReloadSchema(ti, 30*time.Second); err != nil { if err := scw.wr.TabletManagerClient().ReloadSchema(context.TODO(), ti, 30*time.Second); err != nil {
processError("ReloadSchema failed on tablet %v: %v", ti.Alias, err) processError("ReloadSchema failed on tablet %v: %v", ti.Alias, err)
} }
}(scw.destinationTablets[shardIndex][tabletAlias]) }(scw.destinationTablets[shardIndex][tabletAlias])

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

@ -10,6 +10,8 @@ import (
"sync" "sync"
"time" "time"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/sync2" "github.com/youtube/vitess/go/sync2"
blproto "github.com/youtube/vitess/go/vt/binlog/proto" blproto "github.com/youtube/vitess/go/vt/binlog/proto"
"github.com/youtube/vitess/go/vt/concurrency" "github.com/youtube/vitess/go/vt/concurrency"
@ -266,7 +268,7 @@ func (sdw *SplitDiffWorker) synchronizeReplication() error {
// 1 - stop the master binlog replication, get its current position // 1 - stop the master binlog replication, get its current position
sdw.wr.Logger().Infof("Stopping master binlog replication on %v", sdw.shardInfo.MasterAlias) sdw.wr.Logger().Infof("Stopping master binlog replication on %v", sdw.shardInfo.MasterAlias)
blpPositionList, err := sdw.wr.TabletManagerClient().StopBlp(masterInfo, 30*time.Second) blpPositionList, err := sdw.wr.TabletManagerClient().StopBlp(context.TODO(), masterInfo, 30*time.Second)
if err != nil { if err != nil {
return fmt.Errorf("StopBlp for %v failed: %v", sdw.shardInfo.MasterAlias, err) return fmt.Errorf("StopBlp for %v failed: %v", sdw.shardInfo.MasterAlias, err)
} }
@ -292,7 +294,7 @@ func (sdw *SplitDiffWorker) synchronizeReplication() error {
// stop replication // stop replication
sdw.wr.Logger().Infof("Stopping slave[%v] %v at a minimum of %v", i, sdw.sourceAliases[i], blpPos.Position) sdw.wr.Logger().Infof("Stopping slave[%v] %v at a minimum of %v", i, sdw.sourceAliases[i], blpPos.Position)
stoppedAt, err := sdw.wr.TabletManagerClient().StopSlaveMinimum(sourceTablet, blpPos.Position, 30*time.Second) stoppedAt, err := sdw.wr.TabletManagerClient().StopSlaveMinimum(context.TODO(), sourceTablet, blpPos.Position, 30*time.Second)
if err != nil { if err != nil {
return fmt.Errorf("cannot stop slave %v at right binlog position %v: %v", sdw.sourceAliases[i], blpPos.Position, err) return fmt.Errorf("cannot stop slave %v at right binlog position %v: %v", sdw.sourceAliases[i], blpPos.Position, err)
} }
@ -312,7 +314,7 @@ func (sdw *SplitDiffWorker) synchronizeReplication() error {
// 3 - ask the master of the destination shard to resume filtered // 3 - ask the master of the destination shard to resume filtered
// replication up to the new list of positions // replication up to the new list of positions
sdw.wr.Logger().Infof("Restarting master %v until it catches up to %v", sdw.shardInfo.MasterAlias, stopPositionList) sdw.wr.Logger().Infof("Restarting master %v until it catches up to %v", sdw.shardInfo.MasterAlias, stopPositionList)
masterPos, err := sdw.wr.TabletManagerClient().RunBlpUntil(masterInfo, &stopPositionList, 30*time.Second) masterPos, err := sdw.wr.TabletManagerClient().RunBlpUntil(context.TODO(), masterInfo, &stopPositionList, 30*time.Second)
if err != nil { if err != nil {
return fmt.Errorf("RunBlpUntil for %v until %v failed: %v", sdw.shardInfo.MasterAlias, stopPositionList, err) return fmt.Errorf("RunBlpUntil for %v until %v failed: %v", sdw.shardInfo.MasterAlias, stopPositionList, err)
} }
@ -324,7 +326,7 @@ func (sdw *SplitDiffWorker) synchronizeReplication() error {
if err != nil { if err != nil {
return err return err
} }
_, err = sdw.wr.TabletManagerClient().StopSlaveMinimum(destinationTablet, masterPos, 30*time.Second) _, err = sdw.wr.TabletManagerClient().StopSlaveMinimum(context.TODO(), destinationTablet, masterPos, 30*time.Second)
if err != nil { if err != nil {
return fmt.Errorf("StopSlaveMinimum for %v at %v failed: %v", sdw.destinationAlias, masterPos, err) return fmt.Errorf("StopSlaveMinimum for %v at %v failed: %v", sdw.destinationAlias, masterPos, err)
} }
@ -337,7 +339,7 @@ func (sdw *SplitDiffWorker) synchronizeReplication() error {
// 5 - restart filtered replication on destination master // 5 - restart filtered replication on destination master
sdw.wr.Logger().Infof("Restarting filtered replication on master %v", sdw.shardInfo.MasterAlias) sdw.wr.Logger().Infof("Restarting filtered replication on master %v", sdw.shardInfo.MasterAlias)
err = sdw.wr.TabletManagerClient().StartBlp(masterInfo, 30*time.Second) err = sdw.wr.TabletManagerClient().StartBlp(context.TODO(), masterInfo, 30*time.Second)
if err := sdw.cleaner.RemoveActionByName(wrangler.StartBlpActionName, sdw.shardInfo.MasterAlias.String()); err != nil { if err := sdw.cleaner.RemoveActionByName(wrangler.StartBlpActionName, sdw.shardInfo.MasterAlias.String()); err != nil {
sdw.wr.Logger().Warningf("Cannot find cleaning action %v/%v: %v", wrangler.StartBlpActionName, sdw.shardInfo.MasterAlias.String(), err) sdw.wr.Logger().Warningf("Cannot find cleaning action %v/%v: %v", wrangler.StartBlpActionName, sdw.shardInfo.MasterAlias.String(), err)
} }

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

@ -10,6 +10,8 @@ import (
"sync" "sync"
"time" "time"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/vt/topo" "github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/wrangler" "github.com/youtube/vitess/go/vt/wrangler"
) )
@ -223,7 +225,7 @@ func (worker *SQLDiffWorker) synchronizeReplication() error {
if err != nil { if err != nil {
return err return err
} }
if err := worker.wr.TabletManagerClient().StopSlave(subsetTablet, 30*time.Second); err != nil { if err := worker.wr.TabletManagerClient().StopSlave(context.TODO(), subsetTablet, 30*time.Second); err != nil {
return fmt.Errorf("Cannot stop slave %v: %v", worker.subset.alias, err) return fmt.Errorf("Cannot stop slave %v: %v", worker.subset.alias, err)
} }
if worker.CheckInterrupted() { if worker.CheckInterrupted() {
@ -251,7 +253,7 @@ func (worker *SQLDiffWorker) synchronizeReplication() error {
if err != nil { if err != nil {
return err return err
} }
if err := worker.wr.TabletManagerClient().StopSlave(supersetTablet, 30*time.Second); err != nil { if err := worker.wr.TabletManagerClient().StopSlave(context.TODO(), supersetTablet, 30*time.Second); err != nil {
return fmt.Errorf("Cannot stop slave %v: %v", worker.superset.alias, err) return fmt.Errorf("Cannot stop slave %v: %v", worker.superset.alias, err)
} }

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

@ -11,6 +11,8 @@ import (
"sync" "sync"
"time" "time"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/event" "github.com/youtube/vitess/go/event"
"github.com/youtube/vitess/go/sync2" "github.com/youtube/vitess/go/sync2"
"github.com/youtube/vitess/go/vt/binlog/binlogplayer" "github.com/youtube/vitess/go/vt/binlog/binlogplayer"
@ -279,7 +281,7 @@ func (vscw *VerticalSplitCloneWorker) findTargets() error {
} }
// stop replication on it // stop replication on it
if err := vscw.wr.TabletManagerClient().StopSlave(vscw.sourceTablet, 30*time.Second); err != nil { if err := vscw.wr.TabletManagerClient().StopSlave(context.TODO(), vscw.sourceTablet, 30*time.Second); err != nil {
return fmt.Errorf("cannot stop replication on tablet %v", vscw.sourceAlias) return fmt.Errorf("cannot stop replication on tablet %v", vscw.sourceAlias)
} }
@ -505,7 +507,7 @@ func (vscw *VerticalSplitCloneWorker) copy() error {
// then create and populate the blp_checkpoint table // then create and populate the blp_checkpoint table
if strings.Index(vscw.strategy, "populateBlpCheckpoint") != -1 { if strings.Index(vscw.strategy, "populateBlpCheckpoint") != -1 {
// get the current position from the source // get the current position from the source
status, err := vscw.wr.TabletManagerClient().SlaveStatus(vscw.sourceTablet, 30*time.Second) status, err := vscw.wr.TabletManagerClient().SlaveStatus(context.TODO(), vscw.sourceTablet, 30*time.Second)
if err != nil { if err != nil {
return err return err
} }
@ -551,7 +553,7 @@ func (vscw *VerticalSplitCloneWorker) copy() error {
go func(ti *topo.TabletInfo) { go func(ti *topo.TabletInfo) {
defer destinationWaitGroup.Done() defer destinationWaitGroup.Done()
vscw.wr.Logger().Infof("Reloading schema on tablet %v", ti.Alias) vscw.wr.Logger().Infof("Reloading schema on tablet %v", ti.Alias)
if err := vscw.wr.TabletManagerClient().ReloadSchema(ti, 30*time.Second); err != nil { if err := vscw.wr.TabletManagerClient().ReloadSchema(context.TODO(), ti, 30*time.Second); err != nil {
processError("ReloadSchema failed on tablet %v: %v", ti.Alias, err) processError("ReloadSchema failed on tablet %v: %v", ti.Alias, err)
} }
}(vscw.destinationTablets[tabletAlias]) }(vscw.destinationTablets[tabletAlias])

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

@ -11,6 +11,8 @@ import (
"sync" "sync"
"time" "time"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/sync2" "github.com/youtube/vitess/go/sync2"
blproto "github.com/youtube/vitess/go/vt/binlog/proto" blproto "github.com/youtube/vitess/go/vt/binlog/proto"
"github.com/youtube/vitess/go/vt/concurrency" "github.com/youtube/vitess/go/vt/concurrency"
@ -272,7 +274,7 @@ func (vsdw *VerticalSplitDiffWorker) synchronizeReplication() error {
// 1 - stop the master binlog replication, get its current position // 1 - stop the master binlog replication, get its current position
vsdw.wr.Logger().Infof("Stopping master binlog replication on %v", vsdw.shardInfo.MasterAlias) vsdw.wr.Logger().Infof("Stopping master binlog replication on %v", vsdw.shardInfo.MasterAlias)
blpPositionList, err := vsdw.wr.TabletManagerClient().StopBlp(masterInfo, 30*time.Second) blpPositionList, err := vsdw.wr.TabletManagerClient().StopBlp(context.TODO(), masterInfo, 30*time.Second)
if err != nil { if err != nil {
return fmt.Errorf("StopBlp on master %v failed: %v", vsdw.shardInfo.MasterAlias, err) return fmt.Errorf("StopBlp on master %v failed: %v", vsdw.shardInfo.MasterAlias, err)
} }
@ -296,7 +298,7 @@ func (vsdw *VerticalSplitDiffWorker) synchronizeReplication() error {
if err != nil { if err != nil {
return err return err
} }
stoppedAt, err := vsdw.wr.TabletManagerClient().StopSlaveMinimum(sourceTablet, pos.Position, 30*time.Second) stoppedAt, err := vsdw.wr.TabletManagerClient().StopSlaveMinimum(context.TODO(), sourceTablet, pos.Position, 30*time.Second)
if err != nil { if err != nil {
return fmt.Errorf("cannot stop slave %v at right binlog position %v: %v", vsdw.sourceAlias, pos.Position, err) return fmt.Errorf("cannot stop slave %v at right binlog position %v: %v", vsdw.sourceAlias, pos.Position, err)
} }
@ -315,7 +317,7 @@ func (vsdw *VerticalSplitDiffWorker) synchronizeReplication() error {
// 3 - ask the master of the destination shard to resume filtered // 3 - ask the master of the destination shard to resume filtered
// replication up to the new list of positions // replication up to the new list of positions
vsdw.wr.Logger().Infof("Restarting master %v until it catches up to %v", vsdw.shardInfo.MasterAlias, stopPositionList) vsdw.wr.Logger().Infof("Restarting master %v until it catches up to %v", vsdw.shardInfo.MasterAlias, stopPositionList)
masterPos, err := vsdw.wr.TabletManagerClient().RunBlpUntil(masterInfo, &stopPositionList, 30*time.Second) masterPos, err := vsdw.wr.TabletManagerClient().RunBlpUntil(context.TODO(), masterInfo, &stopPositionList, 30*time.Second)
if err != nil { if err != nil {
return fmt.Errorf("RunBlpUntil on %v until %v failed: %v", vsdw.shardInfo.MasterAlias, stopPositionList, err) return fmt.Errorf("RunBlpUntil on %v until %v failed: %v", vsdw.shardInfo.MasterAlias, stopPositionList, err)
} }
@ -327,7 +329,7 @@ func (vsdw *VerticalSplitDiffWorker) synchronizeReplication() error {
if err != nil { if err != nil {
return err return err
} }
_, err = vsdw.wr.TabletManagerClient().StopSlaveMinimum(destinationTablet, masterPos, 30*time.Second) _, err = vsdw.wr.TabletManagerClient().StopSlaveMinimum(context.TODO(), destinationTablet, masterPos, 30*time.Second)
if err != nil { if err != nil {
return fmt.Errorf("StopSlaveMinimum on %v at %v failed: %v", vsdw.destinationAlias, masterPos, err) return fmt.Errorf("StopSlaveMinimum on %v at %v failed: %v", vsdw.destinationAlias, masterPos, err)
} }
@ -340,7 +342,7 @@ func (vsdw *VerticalSplitDiffWorker) synchronizeReplication() error {
// 5 - restart filtered replication on destination master // 5 - restart filtered replication on destination master
vsdw.wr.Logger().Infof("Restarting filtered replication on master %v", vsdw.shardInfo.MasterAlias) vsdw.wr.Logger().Infof("Restarting filtered replication on master %v", vsdw.shardInfo.MasterAlias)
err = vsdw.wr.TabletManagerClient().StartBlp(masterInfo, 30*time.Second) err = vsdw.wr.TabletManagerClient().StartBlp(context.TODO(), masterInfo, 30*time.Second)
if err := vsdw.cleaner.RemoveActionByName(wrangler.StartBlpActionName, vsdw.shardInfo.MasterAlias.String()); err != nil { if err := vsdw.cleaner.RemoveActionByName(wrangler.StartBlpActionName, vsdw.shardInfo.MasterAlias.String()); err != nil {
vsdw.wr.Logger().Warningf("Cannot find cleaning action %v/%v: %v", wrangler.StartBlpActionName, vsdw.shardInfo.MasterAlias.String(), err) vsdw.wr.Logger().Warningf("Cannot find cleaning action %v/%v: %v", wrangler.StartBlpActionName, vsdw.shardInfo.MasterAlias.String(), err)
} }

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

@ -9,6 +9,8 @@ import (
"sync" "sync"
"time" "time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
"github.com/youtube/vitess/go/vt/concurrency" "github.com/youtube/vitess/go/vt/concurrency"
"github.com/youtube/vitess/go/vt/topo" "github.com/youtube/vitess/go/vt/topo"
@ -221,7 +223,7 @@ func RecordStartSlaveAction(cleaner *Cleaner, tabletInfo *topo.TabletInfo, waitT
// CleanUp is part of CleanerAction interface. // CleanUp is part of CleanerAction interface.
func (sba StartSlaveAction) CleanUp(wr *Wrangler) error { func (sba StartSlaveAction) CleanUp(wr *Wrangler) error {
return wr.TabletManagerClient().StartSlave(sba.TabletInfo, sba.WaitTime) return wr.TabletManagerClient().StartSlave(context.TODO(), sba.TabletInfo, sba.WaitTime)
} }
// //
@ -247,5 +249,5 @@ func RecordStartBlpAction(cleaner *Cleaner, tabletInfo *topo.TabletInfo, waitTim
// CleanUp is part of CleanerAction interface. // CleanUp is part of CleanerAction interface.
func (sba StartBlpAction) CleanUp(wr *Wrangler) error { func (sba StartBlpAction) CleanUp(wr *Wrangler) error {
return wr.TabletManagerClient().StartBlp(sba.TabletInfo, sba.WaitTime) return wr.TabletManagerClient().StartBlp(context.TODO(), sba.TabletInfo, sba.WaitTime)
} }

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

@ -8,6 +8,8 @@ import (
"fmt" "fmt"
"sync" "sync"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/vt/concurrency" "github.com/youtube/vitess/go/vt/concurrency"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode" "github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
"github.com/youtube/vitess/go/vt/topo" "github.com/youtube/vitess/go/vt/topo"
@ -42,7 +44,7 @@ func (wr *Wrangler) Snapshot(tabletAlias topo.TabletAlias, forceMasterSnapshot b
ServerMode: serverMode, ServerMode: serverMode,
ForceMasterSnapshot: forceMasterSnapshot, ForceMasterSnapshot: forceMasterSnapshot,
} }
logStream, errFunc, err := wr.tmc.Snapshot(ti, args, wr.ActionTimeout()) logStream, errFunc, err := wr.tmc.Snapshot(context.TODO(), ti, args, wr.ActionTimeout())
if err != nil { if err != nil {
return nil, "", err return nil, "", err
} }
@ -67,7 +69,7 @@ func (wr *Wrangler) SnapshotSourceEnd(tabletAlias topo.TabletAlias, slaveStartRe
ReadOnly: !readWrite, ReadOnly: !readWrite,
OriginalType: originalType, OriginalType: originalType,
} }
return wr.tmc.SnapshotSourceEnd(ti, args, wr.ActionTimeout()) return wr.tmc.SnapshotSourceEnd(context.TODO(), ti, args, wr.ActionTimeout())
} }
// ReserveForRestore will make sure a tablet is ready to be used as a restore // ReserveForRestore will make sure a tablet is ready to be used as a restore
@ -86,7 +88,7 @@ func (wr *Wrangler) ReserveForRestore(srcTabletAlias, dstTabletAlias topo.Tablet
args := &actionnode.ReserveForRestoreArgs{ args := &actionnode.ReserveForRestoreArgs{
SrcTabletAlias: srcTabletAlias, SrcTabletAlias: srcTabletAlias,
} }
return wr.tmc.ReserveForRestore(tablet, args, wr.ActionTimeout()) return wr.tmc.ReserveForRestore(context.TODO(), tablet, args, wr.ActionTimeout())
} }
// UnreserveForRestore switches the tablet back to its original state, // UnreserveForRestore switches the tablet back to its original state,
@ -145,7 +147,7 @@ func (wr *Wrangler) Restore(srcTabletAlias topo.TabletAlias, srcFilePath string,
WasReserved: wasReserved, WasReserved: wasReserved,
DontWaitForSlaveStart: dontWaitForSlaveStart, DontWaitForSlaveStart: dontWaitForSlaveStart,
} }
logStream, errFunc, err := wr.tmc.Restore(tablet, args, wr.ActionTimeout()) logStream, errFunc, err := wr.tmc.Restore(context.TODO(), tablet, args, wr.ActionTimeout())
if err != nil { if err != nil {
return err return err
} }

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

@ -8,6 +8,8 @@ import (
"fmt" "fmt"
"strings" "strings"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
hk "github.com/youtube/vitess/go/vt/hook" hk "github.com/youtube/vitess/go/vt/hook"
"github.com/youtube/vitess/go/vt/topo" "github.com/youtube/vitess/go/vt/topo"
@ -25,7 +27,7 @@ func (wr *Wrangler) ExecuteHook(tabletAlias topo.TabletAlias, hook *hk.Hook) (ho
} }
func (wr *Wrangler) ExecuteTabletInfoHook(ti *topo.TabletInfo, hook *hk.Hook) (hookResult *hk.HookResult, err error) { func (wr *Wrangler) ExecuteTabletInfoHook(ti *topo.TabletInfo, hook *hk.Hook) (hookResult *hk.HookResult, err error) {
return wr.tmc.ExecuteHook(ti, hook, wr.ActionTimeout()) return wr.tmc.ExecuteHook(context.TODO(), ti, hook, wr.ActionTimeout())
} }
// Execute a hook and returns an error only if the hook failed, not if // Execute a hook and returns an error only if the hook failed, not if

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

@ -8,6 +8,7 @@ import (
"fmt" "fmt"
"sync" "sync"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/event" "github.com/youtube/vitess/go/event"
blproto "github.com/youtube/vitess/go/vt/binlog/proto" blproto "github.com/youtube/vitess/go/vt/binlog/proto"
"github.com/youtube/vitess/go/vt/concurrency" "github.com/youtube/vitess/go/vt/concurrency"
@ -22,7 +23,7 @@ import (
// keyspace related methods for Wrangler // keyspace related methods for Wrangler
func (wr *Wrangler) lockKeyspace(keyspace string, actionNode *actionnode.ActionNode) (lockPath string, err error) { func (wr *Wrangler) lockKeyspace(keyspace string, actionNode *actionnode.ActionNode) (lockPath string, err error) {
return actionNode.LockKeyspace(wr.ts, keyspace, wr.lockTimeout, interrupted) return actionNode.LockKeyspace(context.TODO(), wr.ts, keyspace, wr.lockTimeout, interrupted)
} }
func (wr *Wrangler) unlockKeyspace(keyspace string, actionNode *actionnode.ActionNode, lockPath string, actionError error) error { func (wr *Wrangler) unlockKeyspace(keyspace string, actionNode *actionnode.ActionNode, lockPath string, actionError error) error {
@ -211,7 +212,7 @@ func (wr *Wrangler) getMastersPosition(shards []*topo.ShardInfo) (map[*topo.Shar
return return
} }
pos, err := wr.tmc.MasterPosition(ti, wr.ActionTimeout()) pos, err := wr.tmc.MasterPosition(context.TODO(), ti, wr.ActionTimeout())
if err != nil { if err != nil {
rec.RecordError(err) rec.RecordError(err)
return return
@ -255,7 +256,7 @@ func (wr *Wrangler) waitForFilteredReplication(sourcePositions map[*topo.ShardIn
return return
} }
if err := wr.tmc.WaitBlpPosition(tablet, blpPosition, wr.ActionTimeout()); err != nil { if err := wr.tmc.WaitBlpPosition(context.TODO(), tablet, blpPosition, wr.ActionTimeout()); err != nil {
rec.RecordError(err) rec.RecordError(err)
} else { } else {
wr.Logger().Infof("%v caught up", si.MasterAlias) wr.Logger().Infof("%v caught up", si.MasterAlias)
@ -282,7 +283,7 @@ func (wr *Wrangler) refreshMasters(shards []*topo.ShardInfo) error {
return return
} }
if err := wr.tmc.RefreshState(ti, wr.ActionTimeout()); err != nil { if err := wr.tmc.RefreshState(context.TODO(), ti, wr.ActionTimeout()); err != nil {
rec.RecordError(err) rec.RecordError(err)
} else { } else {
wr.Logger().Infof("%v responded", si.MasterAlias) wr.Logger().Infof("%v responded", si.MasterAlias)
@ -334,7 +335,7 @@ func (wr *Wrangler) migrateServedTypes(keyspace string, sourceShards, destinatio
if err := si.UpdateDisableQueryService(topo.TYPE_MASTER, nil, true); err != nil { if err := si.UpdateDisableQueryService(topo.TYPE_MASTER, nil, true); err != nil {
return err return err
} }
if err := topo.UpdateShard(wr.ts, si); err != nil { if err := topo.UpdateShard(context.TODO(), wr.ts, si); err != nil {
return err return err
} }
} }
@ -393,7 +394,7 @@ func (wr *Wrangler) migrateServedTypes(keyspace string, sourceShards, destinatio
// All is good, we can save the shards now // All is good, we can save the shards now
event.DispatchUpdate(ev, "updating source shards") event.DispatchUpdate(ev, "updating source shards")
for _, si := range sourceShards { for _, si := range sourceShards {
if err := topo.UpdateShard(wr.ts, si); err != nil { if err := topo.UpdateShard(context.TODO(), wr.ts, si); err != nil {
return err return err
} }
} }
@ -405,7 +406,7 @@ func (wr *Wrangler) migrateServedTypes(keyspace string, sourceShards, destinatio
} }
event.DispatchUpdate(ev, "updating destination shards") event.DispatchUpdate(ev, "updating destination shards")
for _, si := range destinationShards { for _, si := range destinationShards {
if err := topo.UpdateShard(wr.ts, si); err != nil { if err := topo.UpdateShard(context.TODO(), wr.ts, si); err != nil {
return err return err
} }
} }
@ -559,7 +560,7 @@ func (wr *Wrangler) replicaMigrateServedFrom(ki *topo.KeyspaceInfo, sourceShard
if err := sourceShard.UpdateSourceBlacklistedTables(servedType, cells, reverse, tables); err != nil { if err := sourceShard.UpdateSourceBlacklistedTables(servedType, cells, reverse, tables); err != nil {
return fmt.Errorf("UpdateSourceBlacklistedTables(%v/%v) failed: %v", sourceShard.Keyspace(), sourceShard.ShardName(), err) return fmt.Errorf("UpdateSourceBlacklistedTables(%v/%v) failed: %v", sourceShard.Keyspace(), sourceShard.ShardName(), err)
} }
if err := topo.UpdateShard(wr.ts, sourceShard); err != nil { if err := topo.UpdateShard(context.TODO(), wr.ts, sourceShard); err != nil {
return fmt.Errorf("UpdateShard(%v/%v) failed: %v", sourceShard.Keyspace(), sourceShard.ShardName(), err) return fmt.Errorf("UpdateShard(%v/%v) failed: %v", sourceShard.Keyspace(), sourceShard.ShardName(), err)
} }
@ -599,26 +600,26 @@ func (wr *Wrangler) masterMigrateServedFrom(ki *topo.KeyspaceInfo, sourceShard *
if err := sourceShard.UpdateSourceBlacklistedTables(topo.TYPE_MASTER, nil, false, tables); err != nil { if err := sourceShard.UpdateSourceBlacklistedTables(topo.TYPE_MASTER, nil, false, tables); err != nil {
return fmt.Errorf("UpdateSourceBlacklistedTables(%v/%v) failed: %v", sourceShard.Keyspace(), sourceShard.ShardName(), err) return fmt.Errorf("UpdateSourceBlacklistedTables(%v/%v) failed: %v", sourceShard.Keyspace(), sourceShard.ShardName(), err)
} }
if err := topo.UpdateShard(wr.ts, sourceShard); err != nil { if err := topo.UpdateShard(context.TODO(), wr.ts, sourceShard); err != nil {
return fmt.Errorf("UpdateShard(%v/%v) failed: %v", sourceShard.Keyspace(), sourceShard.ShardName(), err) return fmt.Errorf("UpdateShard(%v/%v) failed: %v", sourceShard.Keyspace(), sourceShard.ShardName(), err)
} }
// Now refresh the blacklisted table list on the source master // Now refresh the blacklisted table list on the source master
event.DispatchUpdate(ev, "refreshing source master so it updates its blacklisted tables") event.DispatchUpdate(ev, "refreshing source master so it updates its blacklisted tables")
if err := wr.tmc.RefreshState(sourceMasterTabletInfo, wr.ActionTimeout()); err != nil { if err := wr.tmc.RefreshState(context.TODO(), sourceMasterTabletInfo, wr.ActionTimeout()); err != nil {
return err return err
} }
// get the position // get the position
event.DispatchUpdate(ev, "getting master position") event.DispatchUpdate(ev, "getting master position")
masterPosition, err := wr.tmc.MasterPosition(sourceMasterTabletInfo, wr.ActionTimeout()) masterPosition, err := wr.tmc.MasterPosition(context.TODO(), sourceMasterTabletInfo, wr.ActionTimeout())
if err != nil { if err != nil {
return err return err
} }
// wait for it // wait for it
event.DispatchUpdate(ev, "waiting for destination master to catch up to source master") event.DispatchUpdate(ev, "waiting for destination master to catch up to source master")
if err := wr.tmc.WaitBlpPosition(destinationMasterTabletInfo, blproto.BlpPosition{ if err := wr.tmc.WaitBlpPosition(context.TODO(), destinationMasterTabletInfo, blproto.BlpPosition{
Uid: 0, Uid: 0,
Position: masterPosition, Position: masterPosition,
}, wr.ActionTimeout()); err != nil { }, wr.ActionTimeout()); err != nil {
@ -634,7 +635,7 @@ func (wr *Wrangler) masterMigrateServedFrom(ki *topo.KeyspaceInfo, sourceShard *
// Update the destination shard (no more source shard) // Update the destination shard (no more source shard)
event.DispatchUpdate(ev, "updating destination shard") event.DispatchUpdate(ev, "updating destination shard")
destinationShard.SourceShards = nil destinationShard.SourceShards = nil
if err := topo.UpdateShard(wr.ts, destinationShard); err != nil { if err := topo.UpdateShard(context.TODO(), wr.ts, destinationShard); err != nil {
return err return err
} }
@ -695,7 +696,7 @@ func (wr *Wrangler) RefreshTablesByShard(si *topo.ShardInfo, tabletType topo.Tab
wg.Add(1) wg.Add(1)
go func(ti *topo.TabletInfo) { go func(ti *topo.TabletInfo) {
if err := wr.tmc.RefreshState(ti, wr.ActionTimeout()); err != nil { if err := wr.tmc.RefreshState(context.TODO(), ti, wr.ActionTimeout()); err != nil {
wr.Logger().Warningf("RefreshTablesByShard: failed to refresh %v: %v", ti.Alias, err) wr.Logger().Warningf("RefreshTablesByShard: failed to refresh %v: %v", ti.Alias, err)
} }
wg.Done() wg.Done()

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

@ -9,6 +9,8 @@ import (
"sort" "sort"
"sync" "sync"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
"github.com/youtube/vitess/go/vt/concurrency" "github.com/youtube/vitess/go/vt/concurrency"
myproto "github.com/youtube/vitess/go/vt/mysqlctl/proto" myproto "github.com/youtube/vitess/go/vt/mysqlctl/proto"
@ -21,7 +23,7 @@ func (wr *Wrangler) GetPermissions(tabletAlias topo.TabletAlias) (*myproto.Permi
return nil, err return nil, err
} }
return wr.tmc.GetPermissions(tablet, wr.ActionTimeout()) return wr.tmc.GetPermissions(context.TODO(), tablet, wr.ActionTimeout())
} }
// helper method to asynchronously diff a permissions // helper method to asynchronously diff a permissions

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

@ -8,6 +8,7 @@ import (
"fmt" "fmt"
"sync" "sync"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/vt/concurrency" "github.com/youtube/vitess/go/vt/concurrency"
"github.com/youtube/vitess/go/vt/key" "github.com/youtube/vitess/go/vt/key"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode" "github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
@ -18,7 +19,7 @@ import (
// Rebuild the serving and replication rollup data data while locking // Rebuild the serving and replication rollup data data while locking
// out other changes. // out other changes.
func (wr *Wrangler) RebuildShardGraph(keyspace, shard string, cells []string) (*topo.ShardInfo, error) { func (wr *Wrangler) RebuildShardGraph(keyspace, shard string, cells []string) (*topo.ShardInfo, error) {
return topotools.RebuildShard(wr.logger, wr.ts, keyspace, shard, cells, wr.lockTimeout, interrupted) return topotools.RebuildShard(context.TODO(), wr.logger, wr.ts, keyspace, shard, cells, wr.lockTimeout, interrupted)
} }
// Rebuild the serving graph data while locking out other changes. // Rebuild the serving graph data while locking out other changes.
@ -265,7 +266,7 @@ func (wr *Wrangler) RebuildReplicationGraph(cells []string, keyspaces []string)
} }
} }
mu.Unlock() mu.Unlock()
err := topo.UpdateTabletReplicationData(wr.ts, ti.Tablet) err := topo.UpdateTabletReplicationData(context.TODO(), wr.ts, ti.Tablet)
if err != nil { if err != nil {
mu.Lock() mu.Lock()
hasErr = true hasErr = true

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

@ -67,6 +67,8 @@ On X: (promoted slave)
import ( import (
"fmt" "fmt"
"code.google.com/p/go.net/context"
myproto "github.com/youtube/vitess/go/vt/mysqlctl/proto" myproto "github.com/youtube/vitess/go/vt/mysqlctl/proto"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode" "github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
"github.com/youtube/vitess/go/vt/topo" "github.com/youtube/vitess/go/vt/topo"
@ -211,13 +213,13 @@ func (wr *Wrangler) ReparentTablet(tabletAlias topo.TabletAlias) error {
return fmt.Errorf("master %v and potential slave not in same keyspace/shard", shardInfo.MasterAlias) return fmt.Errorf("master %v and potential slave not in same keyspace/shard", shardInfo.MasterAlias)
} }
status, err := wr.tmc.SlaveStatus(ti, wr.ActionTimeout()) status, err := wr.tmc.SlaveStatus(context.TODO(), ti, wr.ActionTimeout())
if err != nil { if err != nil {
return err return err
} }
wr.Logger().Infof("slave tablet position: %v %v %v", tabletAlias, ti.MysqlAddr(), status.Position) wr.Logger().Infof("slave tablet position: %v %v %v", tabletAlias, ti.MysqlAddr(), status.Position)
rsd, err := wr.tmc.ReparentPosition(masterTi, &status.Position, wr.ActionTimeout()) rsd, err := wr.tmc.ReparentPosition(context.TODO(), masterTi, &status.Position, wr.ActionTimeout())
if err != nil { if err != nil {
return err return err
} }
@ -226,5 +228,5 @@ func (wr *Wrangler) ReparentTablet(tabletAlias topo.TabletAlias) error {
// An orphan is already in the replication graph but it is // An orphan is already in the replication graph but it is
// disconnected, hence we have to force this action. // disconnected, hence we have to force this action.
rsd.Force = ti.Type == topo.TYPE_LAG_ORPHAN rsd.Force = ti.Type == topo.TYPE_LAG_ORPHAN
return wr.tmc.RestartSlave(ti, rsd, wr.ActionTimeout()) return wr.tmc.RestartSlave(context.TODO(), ti, rsd, wr.ActionTimeout())
} }

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

@ -7,6 +7,7 @@ import (
"sync" "sync"
"time" "time"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/vt/hook" "github.com/youtube/vitess/go/vt/hook"
myproto "github.com/youtube/vitess/go/vt/mysqlctl/proto" myproto "github.com/youtube/vitess/go/vt/mysqlctl/proto"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode" "github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
@ -64,7 +65,7 @@ func (wr *Wrangler) checkSlaveReplication(tabletMap map[topo.TabletAlias]*topo.T
return return
} }
status, err := wr.tmc.SlaveStatus(tablet, wr.ActionTimeout()) status, err := wr.tmc.SlaveStatus(context.TODO(), tablet, wr.ActionTimeout())
if err != nil { if err != nil {
if tablet.Type == topo.TYPE_BACKUP { if tablet.Type == topo.TYPE_BACKUP {
wr.logger.Warningf(" failed to get slave position from backup tablet %v, either wait for backup to finish or scrap tablet (%v)", tablet.Alias, err) wr.logger.Warningf(" failed to get slave position from backup tablet %v, either wait for backup to finish or scrap tablet (%v)", tablet.Alias, err)
@ -112,7 +113,7 @@ func (wr *Wrangler) checkSlaveConsistency(tabletMap map[uint32]*topo.TabletInfo,
if !masterPosition.IsZero() { if !masterPosition.IsZero() {
// If the master position is known, do our best to wait for replication to catch up. // If the master position is known, do our best to wait for replication to catch up.
status, err := wr.tmc.WaitSlavePosition(ti, masterPosition, wr.ActionTimeout()) status, err := wr.tmc.WaitSlavePosition(context.TODO(), ti, masterPosition, wr.ActionTimeout())
if err != nil { if err != nil {
ctx.err = err ctx.err = err
return return
@ -120,7 +121,7 @@ func (wr *Wrangler) checkSlaveConsistency(tabletMap map[uint32]*topo.TabletInfo,
ctx.status = status ctx.status = status
} else { } else {
// If the master is down, just get the slave status. // If the master is down, just get the slave status.
status, err := wr.tmc.SlaveStatus(ti, wr.ActionTimeout()) status, err := wr.tmc.SlaveStatus(context.TODO(), ti, wr.ActionTimeout())
if err != nil { if err != nil {
ctx.err = err ctx.err = err
return return
@ -182,7 +183,7 @@ func (wr *Wrangler) checkSlaveConsistency(tabletMap map[uint32]*topo.TabletInfo,
func (wr *Wrangler) stopSlaves(tabletMap map[topo.TabletAlias]*topo.TabletInfo) error { func (wr *Wrangler) stopSlaves(tabletMap map[topo.TabletAlias]*topo.TabletInfo) error {
errs := make(chan error, len(tabletMap)) errs := make(chan error, len(tabletMap))
f := func(ti *topo.TabletInfo) { f := func(ti *topo.TabletInfo) {
err := wr.tmc.StopSlave(ti, wr.ActionTimeout()) err := wr.tmc.StopSlave(context.TODO(), ti, wr.ActionTimeout())
if err != nil { if err != nil {
wr.logger.Infof("StopSlave failed: %v", err) wr.logger.Infof("StopSlave failed: %v", err)
} }
@ -218,13 +219,13 @@ func (wr *Wrangler) tabletReplicationStatuses(tablets []*topo.TabletInfo) ([]*my
ctx := &rpcContext{tablet: ti} ctx := &rpcContext{tablet: ti}
calls[idx] = ctx calls[idx] = ctx
if ti.Type == topo.TYPE_MASTER { if ti.Type == topo.TYPE_MASTER {
pos, err := wr.tmc.MasterPosition(ti, wr.ActionTimeout()) pos, err := wr.tmc.MasterPosition(context.TODO(), ti, wr.ActionTimeout())
ctx.err = err ctx.err = err
if err == nil { if err == nil {
ctx.status = &myproto.ReplicationStatus{Position: pos} ctx.status = &myproto.ReplicationStatus{Position: pos}
} }
} else if ti.IsSlaveType() { } else if ti.IsSlaveType() {
ctx.status, ctx.err = wr.tmc.SlaveStatus(ti, wr.ActionTimeout()) ctx.status, ctx.err = wr.tmc.SlaveStatus(context.TODO(), ti, wr.ActionTimeout())
} }
} }
@ -261,15 +262,15 @@ func (wr *Wrangler) tabletReplicationStatuses(tablets []*topo.TabletInfo) ([]*my
func (wr *Wrangler) demoteMaster(ti *topo.TabletInfo) (myproto.ReplicationPosition, error) { func (wr *Wrangler) demoteMaster(ti *topo.TabletInfo) (myproto.ReplicationPosition, error) {
wr.logger.Infof("demote master %v", ti.Alias) wr.logger.Infof("demote master %v", ti.Alias)
if err := wr.tmc.DemoteMaster(ti, wr.ActionTimeout()); err != nil { if err := wr.tmc.DemoteMaster(context.TODO(), ti, wr.ActionTimeout()); err != nil {
return myproto.ReplicationPosition{}, err return myproto.ReplicationPosition{}, err
} }
return wr.tmc.MasterPosition(ti, wr.ActionTimeout()) return wr.tmc.MasterPosition(context.TODO(), ti, wr.ActionTimeout())
} }
func (wr *Wrangler) promoteSlave(ti *topo.TabletInfo) (rsd *actionnode.RestartSlaveData, err error) { func (wr *Wrangler) promoteSlave(ti *topo.TabletInfo) (rsd *actionnode.RestartSlaveData, err error) {
wr.logger.Infof("promote slave %v", ti.Alias) wr.logger.Infof("promote slave %v", ti.Alias)
return wr.tmc.PromoteSlave(ti, wr.ActionTimeout()) return wr.tmc.PromoteSlave(context.TODO(), ti, wr.ActionTimeout())
} }
func (wr *Wrangler) restartSlaves(slaveTabletMap map[topo.TabletAlias]*topo.TabletInfo, rsd *actionnode.RestartSlaveData) (majorityRestart bool, err error) { func (wr *Wrangler) restartSlaves(slaveTabletMap map[topo.TabletAlias]*topo.TabletInfo, rsd *actionnode.RestartSlaveData) (majorityRestart bool, err error) {
@ -319,7 +320,7 @@ func (wr *Wrangler) restartSlaves(slaveTabletMap map[topo.TabletAlias]*topo.Tabl
func (wr *Wrangler) restartSlave(ti *topo.TabletInfo, rsd *actionnode.RestartSlaveData) (err error) { func (wr *Wrangler) restartSlave(ti *topo.TabletInfo, rsd *actionnode.RestartSlaveData) (err error) {
wr.logger.Infof("restart slave %v", ti.Alias) wr.logger.Infof("restart slave %v", ti.Alias)
return wr.tmc.RestartSlave(ti, rsd, wr.ActionTimeout()) return wr.tmc.RestartSlave(context.TODO(), ti, rsd, wr.ActionTimeout())
} }
func (wr *Wrangler) checkMasterElect(ti *topo.TabletInfo) error { func (wr *Wrangler) checkMasterElect(ti *topo.TabletInfo) error {
@ -338,7 +339,7 @@ func (wr *Wrangler) finishReparent(si *topo.ShardInfo, masterElect *topo.TabletI
wr.logger.Warningf("leaving master-elect read-only, change with: vtctl SetReadWrite %v", masterElect.Alias) wr.logger.Warningf("leaving master-elect read-only, change with: vtctl SetReadWrite %v", masterElect.Alias)
} else { } else {
wr.logger.Infof("marking master-elect read-write %v", masterElect.Alias) wr.logger.Infof("marking master-elect read-write %v", masterElect.Alias)
if err := wr.tmc.SetReadWrite(masterElect, wr.ActionTimeout()); err != nil { if err := wr.tmc.SetReadWrite(context.TODO(), masterElect, wr.ActionTimeout()); err != nil {
wr.logger.Warningf("master master-elect read-write failed, leaving master-elect read-only, change with: vtctl SetReadWrite %v", masterElect.Alias) wr.logger.Warningf("master master-elect read-write failed, leaving master-elect read-only, change with: vtctl SetReadWrite %v", masterElect.Alias)
} }
} }
@ -348,7 +349,7 @@ func (wr *Wrangler) finishReparent(si *topo.ShardInfo, masterElect *topo.TabletI
// save the new master in the shard info // save the new master in the shard info
si.MasterAlias = masterElect.Alias si.MasterAlias = masterElect.Alias
if err := topo.UpdateShard(wr.ts, si); err != nil { if err := topo.UpdateShard(context.TODO(), wr.ts, si); err != nil {
wr.logger.Errorf("Failed to save new master into shard: %v", err) wr.logger.Errorf("Failed to save new master into shard: %v", err)
return err return err
} }
@ -356,7 +357,7 @@ func (wr *Wrangler) finishReparent(si *topo.ShardInfo, masterElect *topo.TabletI
// We rebuild all the cells, as we may have taken tablets in and // We rebuild all the cells, as we may have taken tablets in and
// out of the graph. // out of the graph.
wr.logger.Infof("rebuilding shard serving graph data") wr.logger.Infof("rebuilding shard serving graph data")
_, err := topotools.RebuildShard(wr.logger, wr.ts, masterElect.Keyspace, masterElect.Shard, nil, wr.lockTimeout, interrupted) _, err := topotools.RebuildShard(context.TODO(), wr.logger, wr.ts, masterElect.Keyspace, masterElect.Shard, nil, wr.lockTimeout, interrupted)
return err return err
} }
@ -372,7 +373,7 @@ func (wr *Wrangler) breakReplication(slaveMap map[topo.TabletAlias]*topo.TabletI
// Force slaves to break, just in case they were not advertised in // Force slaves to break, just in case they were not advertised in
// the replication graph. // the replication graph.
wr.logger.Infof("break slaves %v", masterElect.Alias) wr.logger.Infof("break slaves %v", masterElect.Alias)
return wr.tmc.BreakSlaves(masterElect, wr.ActionTimeout()) return wr.tmc.BreakSlaves(context.TODO(), masterElect, wr.ActionTimeout())
} }
func (wr *Wrangler) restartableTabletMap(slaves map[topo.TabletAlias]*topo.TabletInfo) map[uint32]*topo.TabletInfo { func (wr *Wrangler) restartableTabletMap(slaves map[topo.TabletAlias]*topo.TabletInfo) map[uint32]*topo.TabletInfo {

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

@ -7,6 +7,7 @@ package wrangler
import ( import (
"fmt" "fmt"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/event" "github.com/youtube/vitess/go/event"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode" "github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
"github.com/youtube/vitess/go/vt/topo" "github.com/youtube/vitess/go/vt/topo"
@ -101,14 +102,14 @@ func (wr *Wrangler) shardExternallyReparentedLocked(keyspace, shard string, mast
event.DispatchUpdate(ev, "updating shard record") event.DispatchUpdate(ev, "updating shard record")
wr.logger.Infof("Updating Shard's MasterAlias record") wr.logger.Infof("Updating Shard's MasterAlias record")
shardInfo.MasterAlias = masterElectTabletAlias shardInfo.MasterAlias = masterElectTabletAlias
if err = topo.UpdateShard(wr.ts, shardInfo); err != nil { if err = topo.UpdateShard(context.TODO(), wr.ts, shardInfo); err != nil {
return err return err
} }
// and rebuild the shard serving graph // and rebuild the shard serving graph
event.DispatchUpdate(ev, "rebuilding shard serving graph") event.DispatchUpdate(ev, "rebuilding shard serving graph")
wr.logger.Infof("Rebuilding shard serving graph data") wr.logger.Infof("Rebuilding shard serving graph data")
if _, err = topotools.RebuildShard(wr.logger, wr.ts, masterElectTablet.Keyspace, masterElectTablet.Shard, cells, wr.lockTimeout, interrupted); err != nil { if _, err = topotools.RebuildShard(context.TODO(), wr.logger, wr.ts, masterElectTablet.Keyspace, masterElectTablet.Shard, cells, wr.lockTimeout, interrupted); err != nil {
return err return err
} }
@ -143,16 +144,15 @@ func (wr *Wrangler) reparentShardExternal(ev *events.Reparent, slaveTabletMap, m
// timeout is executed, so even if we got to the timeout, // timeout is executed, so even if we got to the timeout,
// we're still good. // we're still good.
event.DispatchUpdate(ev, "restarting slaves") event.DispatchUpdate(ev, "restarting slaves")
topotools.RestartSlavesExternal(wr.ts, wr.logger, slaveTabletMap, masterTabletMap, masterElectTablet.Alias, wr.slaveWasRestarted) ctx := context.TODO()
topotools.RestartSlavesExternal(wr.ts, wr.logger, slaveTabletMap, masterTabletMap, masterElectTablet.Alias, func(ti *topo.TabletInfo, swra *actionnode.SlaveWasRestartedArgs) error {
wr.logger.Infof("slaveWasRestarted(%v)", ti.Alias)
return wr.tmc.SlaveWasRestarted(ctx, ti, swra, wr.ActionTimeout())
})
return nil return nil
} }
func (wr *Wrangler) slaveWasPromoted(ti *topo.TabletInfo) error { func (wr *Wrangler) slaveWasPromoted(ti *topo.TabletInfo) error {
wr.logger.Infof("slaveWasPromoted(%v)", ti.Alias) wr.logger.Infof("slaveWasPromoted(%v)", ti.Alias)
return wr.tmc.SlaveWasPromoted(ti, wr.ActionTimeout()) return wr.tmc.SlaveWasPromoted(context.TODO(), ti, wr.ActionTimeout())
}
func (wr *Wrangler) slaveWasRestarted(ti *topo.TabletInfo, swra *actionnode.SlaveWasRestartedArgs) (err error) {
wr.logger.Infof("slaveWasRestarted(%v)", ti.Alias)
return wr.tmc.SlaveWasRestarted(ti, swra, wr.ActionTimeout())
} }

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

@ -8,6 +8,8 @@ import (
"fmt" "fmt"
"strings" "strings"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/event" "github.com/youtube/vitess/go/event"
"github.com/youtube/vitess/go/vt/topo" "github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topotools" "github.com/youtube/vitess/go/vt/topotools"
@ -111,7 +113,7 @@ func (wr *Wrangler) reparentShardGraceful(ev *events.Reparent, si *topo.ShardInf
// it as new replica. // it as new replica.
event.DispatchUpdate(ev, "scrapping old master") event.DispatchUpdate(ev, "scrapping old master")
wr.logger.Infof("scrap demoted master %v", masterTablet.Alias) wr.logger.Infof("scrap demoted master %v", masterTablet.Alias)
if scrapErr := wr.tmc.Scrap(masterTablet, wr.ActionTimeout()); scrapErr != nil { if scrapErr := wr.tmc.Scrap(context.TODO(), masterTablet, wr.ActionTimeout()); scrapErr != nil {
// The sub action is non-critical, so just warn. // The sub action is non-critical, so just warn.
wr.logger.Warningf("scrap demoted master failed: %v", scrapErr) wr.logger.Warningf("scrap demoted master failed: %v", scrapErr)
} }

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

@ -10,6 +10,8 @@ import (
"strings" "strings"
"sync" "sync"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
"github.com/youtube/vitess/go/vt/concurrency" "github.com/youtube/vitess/go/vt/concurrency"
myproto "github.com/youtube/vitess/go/vt/mysqlctl/proto" myproto "github.com/youtube/vitess/go/vt/mysqlctl/proto"
@ -26,7 +28,7 @@ func (wr *Wrangler) GetSchema(tabletAlias topo.TabletAlias, tables, excludeTable
return nil, err return nil, err
} }
return wr.tmc.GetSchema(ti, tables, excludeTables, includeViews, wr.ActionTimeout()) return wr.tmc.GetSchema(context.TODO(), ti, tables, excludeTables, includeViews, wr.ActionTimeout())
} }
// ReloadSchema forces the remote tablet to reload its schema. // ReloadSchema forces the remote tablet to reload its schema.
@ -36,7 +38,7 @@ func (wr *Wrangler) ReloadSchema(tabletAlias topo.TabletAlias) error {
return err return err
} }
return wr.tmc.ReloadSchema(ti, wr.ActionTimeout()) return wr.tmc.ReloadSchema(context.TODO(), ti, wr.ActionTimeout())
} }
// helper method to asynchronously diff a schema // helper method to asynchronously diff a schema
@ -184,7 +186,7 @@ func (wr *Wrangler) PreflightSchema(tabletAlias topo.TabletAlias, change string)
if err != nil { if err != nil {
return nil, err return nil, err
} }
return wr.tmc.PreflightSchema(ti, change, wr.ActionTimeout()) return wr.tmc.PreflightSchema(context.TODO(), ti, change, wr.ActionTimeout())
} }
// ApplySchema will apply a schema change on the remote tablet. // ApplySchema will apply a schema change on the remote tablet.
@ -193,7 +195,7 @@ func (wr *Wrangler) ApplySchema(tabletAlias topo.TabletAlias, sc *myproto.Schema
if err != nil { if err != nil {
return nil, err return nil, err
} }
return wr.tmc.ApplySchema(ti, sc, wr.ActionTimeout()) return wr.tmc.ApplySchema(context.TODO(), ti, sc, wr.ActionTimeout())
} }
// Note for 'complex' mode (the 'simple' mode is easy enough that we // Note for 'complex' mode (the 'simple' mode is easy enough that we
@ -285,7 +287,7 @@ func (wr *Wrangler) applySchemaShard(shardInfo *topo.ShardInfo, preflight *mypro
for _, status := range statusArray { for _, status := range statusArray {
wg.Add(1) wg.Add(1)
go func(status *TabletStatus) { go func(status *TabletStatus) {
status.beforeSchema, status.lastError = wr.tmc.GetSchema(status.ti, nil, nil, false, wr.ActionTimeout()) status.beforeSchema, status.lastError = wr.tmc.GetSchema(context.TODO(), status.ti, nil, nil, false, wr.ActionTimeout())
wg.Done() wg.Done()
}(status) }(status)
} }

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

@ -7,6 +7,7 @@ package wrangler
import ( import (
"fmt" "fmt"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
"github.com/youtube/vitess/go/vt/key" "github.com/youtube/vitess/go/vt/key"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode" "github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
@ -16,7 +17,7 @@ import (
// shard related methods for Wrangler // shard related methods for Wrangler
func (wr *Wrangler) lockShard(keyspace, shard string, actionNode *actionnode.ActionNode) (lockPath string, err error) { func (wr *Wrangler) lockShard(keyspace, shard string, actionNode *actionnode.ActionNode) (lockPath string, err error) {
return actionNode.LockShard(wr.ts, keyspace, shard, wr.lockTimeout, interrupted) return actionNode.LockShard(context.TODO(), wr.ts, keyspace, shard, wr.lockTimeout, interrupted)
} }
func (wr *Wrangler) unlockShard(keyspace, shard string, actionNode *actionnode.ActionNode, lockPath string, actionError error) error { func (wr *Wrangler) unlockShard(keyspace, shard string, actionNode *actionnode.ActionNode, lockPath string, actionError error) error {
@ -70,7 +71,7 @@ func (wr *Wrangler) updateShardCellsAndMaster(si *topo.ShardInfo, tabletAlias to
if wasUpdated { if wasUpdated {
// write it back // write it back
if err := topo.UpdateShard(wr.ts, si); err != nil { if err := topo.UpdateShard(context.TODO(), wr.ts, si); err != nil {
return wr.unlockShard(keyspace, shard, actionNode, lockPath, err) return wr.unlockShard(keyspace, shard, actionNode, lockPath, err)
} }
} }
@ -102,7 +103,7 @@ func (wr *Wrangler) setShardServedTypes(keyspace, shard string, cells []string,
if err := si.UpdateServedTypesMap(servedType, cells, remove); err != nil { if err := si.UpdateServedTypesMap(servedType, cells, remove); err != nil {
return err return err
} }
return topo.UpdateShard(wr.ts, si) return topo.UpdateShard(context.TODO(), wr.ts, si)
} }
// SetShardTabletControl changes the TabletControl records // SetShardTabletControl changes the TabletControl records
@ -144,7 +145,7 @@ func (wr *Wrangler) setShardTabletControl(keyspace, shard string, tabletType top
return fmt.Errorf("UpdateSourceBlacklistedTables(%v/%v) failed: %v", shardInfo.Keyspace(), shardInfo.ShardName(), err) return fmt.Errorf("UpdateSourceBlacklistedTables(%v/%v) failed: %v", shardInfo.Keyspace(), shardInfo.ShardName(), err)
} }
} }
return topo.UpdateShard(wr.ts, shardInfo) return topo.UpdateShard(context.TODO(), wr.ts, shardInfo)
} }
// DeleteShard will do all the necessary changes in the topology server // DeleteShard will do all the necessary changes in the topology server
@ -255,7 +256,7 @@ func (wr *Wrangler) removeShardCell(keyspace, shard, cell string, force bool) er
} }
shardInfo.Cells = newCells shardInfo.Cells = newCells
return topo.UpdateShard(wr.ts, shardInfo) return topo.UpdateShard(context.TODO(), wr.ts, shardInfo)
} }
func (wr *Wrangler) SourceShardDelete(keyspace, shard string, uid uint32) error { func (wr *Wrangler) SourceShardDelete(keyspace, shard string, uid uint32) error {
@ -287,7 +288,7 @@ func (wr *Wrangler) sourceShardDelete(keyspace, shard string, uid uint32) error
newSourceShards = nil newSourceShards = nil
} }
si.SourceShards = newSourceShards si.SourceShards = newSourceShards
return topo.UpdateShard(wr.ts, si) return topo.UpdateShard(context.TODO(), wr.ts, si)
} }
func (wr *Wrangler) SourceShardAdd(keyspace, shard string, uid uint32, skeyspace, sshard string, keyRange key.KeyRange, tables []string) error { func (wr *Wrangler) SourceShardAdd(keyspace, shard string, uid uint32, skeyspace, sshard string, keyRange key.KeyRange, tables []string) error {
@ -321,5 +322,5 @@ func (wr *Wrangler) sourceShardAdd(keyspace, shard string, uid uint32, skeyspace
KeyRange: keyRange, KeyRange: keyRange,
Tables: tables, Tables: tables,
}) })
return topo.UpdateShard(wr.ts, si) return topo.UpdateShard(context.TODO(), wr.ts, si)
} }

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

@ -8,6 +8,8 @@ import (
"fmt" "fmt"
"sync" "sync"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
"github.com/youtube/vitess/go/event" "github.com/youtube/vitess/go/event"
cc "github.com/youtube/vitess/go/vt/concurrency" cc "github.com/youtube/vitess/go/vt/concurrency"
@ -44,7 +46,7 @@ func (wr *Wrangler) prepareToSnapshot(ti *topo.TabletInfo, forceMasterSnapshot b
// There is a legitimate reason to force in the case of a single // There is a legitimate reason to force in the case of a single
// master. // master.
ti.Tablet.Type = topo.TYPE_BACKUP ti.Tablet.Type = topo.TYPE_BACKUP
err = topo.UpdateTablet(wr.ts, ti) err = topo.UpdateTablet(context.TODO(), wr.ts, ti)
} else { } else {
err = wr.ChangeType(ti.Alias, topo.TYPE_BACKUP, false) err = wr.ChangeType(ti.Alias, topo.TYPE_BACKUP, false)
} }
@ -59,7 +61,7 @@ func (wr *Wrangler) prepareToSnapshot(ti *topo.TabletInfo, forceMasterSnapshot b
if ti.Tablet.Parent.Uid == topo.NO_TABLET && forceMasterSnapshot { if ti.Tablet.Parent.Uid == topo.NO_TABLET && forceMasterSnapshot {
log.Infof("force change type backup -> master: %v", ti.Alias) log.Infof("force change type backup -> master: %v", ti.Alias)
ti.Tablet.Type = topo.TYPE_MASTER ti.Tablet.Type = topo.TYPE_MASTER
return topo.UpdateTablet(wr.ts, ti) return topo.UpdateTablet(context.TODO(), wr.ts, ti)
} }
return wr.ChangeType(ti.Alias, originalType, false) return wr.ChangeType(ti.Alias, originalType, false)
@ -87,7 +89,7 @@ func (wr *Wrangler) MultiRestore(dstTabletAlias topo.TabletAlias, sources []topo
} }
}() }()
logStream, errFunc, err := wr.tmc.MultiRestore(ti, args, wr.ActionTimeout()) logStream, errFunc, err := wr.tmc.MultiRestore(context.TODO(), ti, args, wr.ActionTimeout())
if err != nil { if err != nil {
return err return err
} }
@ -134,7 +136,7 @@ func (wr *Wrangler) MultiSnapshot(keyRanges []key.KeyRange, tabletAlias topo.Tab
}() }()
// execute the remote action, log the results, save the error // execute the remote action, log the results, save the error
logStream, errFunc, err := wr.tmc.MultiSnapshot(ti, args, wr.ActionTimeout()) logStream, errFunc, err := wr.tmc.MultiSnapshot(context.TODO(), ti, args, wr.ActionTimeout())
if err != nil { if err != nil {
return nil, topo.TabletAlias{}, err return nil, topo.TabletAlias{}, err
} }
@ -243,7 +245,7 @@ func (wr *Wrangler) SetSourceShards(keyspace, shard string, sources []topo.Table
} }
// and write the shard // and write the shard
if err = topo.UpdateShard(wr.ts, shardInfo); err != nil { if err = topo.UpdateShard(context.TODO(), wr.ts, shardInfo); err != nil {
return err return err
} }

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

@ -7,6 +7,7 @@ package wrangler
import ( import (
"fmt" "fmt"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
mproto "github.com/youtube/vitess/go/mysql/proto" mproto "github.com/youtube/vitess/go/mysql/proto"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode" "github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
@ -76,7 +77,7 @@ func (wr *Wrangler) InitTablet(tablet *topo.Tablet, force, createShardAndKeyspac
} else { } else {
if oldTablet.Keyspace == tablet.Keyspace && oldTablet.Shard == tablet.Shard { if oldTablet.Keyspace == tablet.Keyspace && oldTablet.Shard == tablet.Shard {
*(oldTablet.Tablet) = *tablet *(oldTablet.Tablet) = *tablet
if err := topo.UpdateTablet(wr.ts, oldTablet); err != nil { if err := topo.UpdateTablet(context.TODO(), wr.ts, oldTablet); err != nil {
log.Warningf("failed updating tablet %v: %v", tablet.Alias, err) log.Warningf("failed updating tablet %v: %v", tablet.Alias, err)
// now fall through the Scrap case // now fall through the Scrap case
} else { } else {
@ -84,7 +85,7 @@ func (wr *Wrangler) InitTablet(tablet *topo.Tablet, force, createShardAndKeyspac
return nil return nil
} }
if err := topo.UpdateTabletReplicationData(wr.ts, tablet); err != nil { if err := topo.UpdateTabletReplicationData(context.TODO(), wr.ts, tablet); err != nil {
log.Warningf("failed updating tablet replication data for %v: %v", tablet.Alias, err) log.Warningf("failed updating tablet replication data for %v: %v", tablet.Alias, err)
// now fall through the Scrap case // now fall through the Scrap case
} else { } else {
@ -126,7 +127,7 @@ func (wr *Wrangler) Scrap(tabletAlias topo.TabletAlias, force, skipRebuild bool)
if force { if force {
err = topotools.Scrap(wr.ts, ti.Alias, force) err = topotools.Scrap(wr.ts, ti.Alias, force)
} else { } else {
err = wr.tmc.Scrap(ti, wr.ActionTimeout()) err = wr.tmc.Scrap(context.TODO(), ti, wr.ActionTimeout())
} }
if err != nil { if err != nil {
return err return err
@ -160,7 +161,7 @@ func (wr *Wrangler) Scrap(tabletAlias topo.TabletAlias, force, skipRebuild bool)
si.MasterAlias = topo.TabletAlias{} si.MasterAlias = topo.TabletAlias{}
// write it back // write it back
if err := topo.UpdateShard(wr.ts, si); err != nil { if err := topo.UpdateShard(context.TODO(), wr.ts, si); err != nil {
return wr.unlockShard(ti.Keyspace, ti.Shard, actionNode, lockPath, err) return wr.unlockShard(ti.Keyspace, ti.Shard, actionNode, lockPath, err)
} }
} else { } else {
@ -219,7 +220,7 @@ func (wr *Wrangler) ChangeTypeNoRebuild(tabletAlias topo.TabletAlias, tabletType
return false, "", "", "", err return false, "", "", "", err
} }
} else { } else {
if err := wr.tmc.ChangeType(ti, tabletType, wr.ActionTimeout()); err != nil { if err := wr.tmc.ChangeType(context.TODO(), ti, tabletType, wr.ActionTimeout()); err != nil {
return false, "", "", "", err return false, "", "", "", err
} }
} }
@ -248,13 +249,13 @@ func (wr *Wrangler) changeTypeInternal(tabletAlias topo.TabletAlias, dbType topo
rebuildRequired := ti.Tablet.IsInServingGraph() rebuildRequired := ti.Tablet.IsInServingGraph()
// change the type // change the type
if err := wr.tmc.ChangeType(ti, dbType, wr.ActionTimeout()); err != nil { if err := wr.tmc.ChangeType(context.TODO(), ti, dbType, wr.ActionTimeout()); err != nil {
return err return err
} }
// rebuild if necessary // rebuild if necessary
if rebuildRequired { if rebuildRequired {
_, err = topotools.RebuildShard(wr.logger, wr.ts, ti.Keyspace, ti.Shard, []string{ti.Alias.Cell}, wr.lockTimeout, interrupted) _, err = topotools.RebuildShard(context.TODO(), wr.logger, wr.ts, ti.Keyspace, ti.Shard, []string{ti.Alias.Cell}, wr.lockTimeout, interrupted)
if err != nil { if err != nil {
return err return err
} }
@ -282,5 +283,5 @@ func (wr *Wrangler) ExecuteFetch(tabletAlias topo.TabletAlias, query string, max
if err != nil { if err != nil {
return nil, err return nil, err
} }
return wr.tmc.ExecuteFetch(ti, query, maxRows, wantFields, disableBinlogs, wr.ActionTimeout()) return wr.tmc.ExecuteFetch(context.TODO(), ti, query, maxRows, wantFields, disableBinlogs, wr.ActionTimeout())
} }

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

@ -9,6 +9,8 @@ import (
"testing" "testing"
"time" "time"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/vt/logutil" "github.com/youtube/vitess/go/vt/logutil"
_ "github.com/youtube/vitess/go/vt/tabletmanager/gorpctmclient" _ "github.com/youtube/vitess/go/vt/tabletmanager/gorpctmclient"
"github.com/youtube/vitess/go/vt/topo" "github.com/youtube/vitess/go/vt/topo"
@ -18,6 +20,7 @@ import (
) )
func TestShardExternallyReparented(t *testing.T) { func TestShardExternallyReparented(t *testing.T) {
ctx := context.Background()
ts := zktopo.NewTestServer(t, []string{"cell1", "cell2"}) ts := zktopo.NewTestServer(t, []string{"cell1", "cell2"})
wr := wrangler.New(logutil.NewConsoleLogger(), ts, time.Minute, time.Second) wr := wrangler.New(logutil.NewConsoleLogger(), ts, time.Minute, time.Second)
@ -39,7 +42,7 @@ func TestShardExternallyReparented(t *testing.T) {
t.Fatalf("GetShard failed: %v", err) t.Fatalf("GetShard failed: %v", err)
} }
si.Cells = append(si.Cells, "cell666") si.Cells = append(si.Cells, "cell666")
if err := topo.UpdateShard(ts, si); err != nil { if err := topo.UpdateShard(ctx, ts, si); err != nil {
t.Fatalf("UpdateShard failed: %v", err) t.Fatalf("UpdateShard failed: %v", err)
} }

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

@ -10,6 +10,8 @@ import (
"sync" "sync"
"time" "time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
"github.com/youtube/vitess/go/vt/topo" "github.com/youtube/vitess/go/vt/topo"
) )
@ -208,7 +210,7 @@ func (wr *Wrangler) validateReplication(shardInfo *topo.ShardInfo, tabletMap map
return return
} }
slaveList, err := wr.tmc.GetSlaves(masterTablet, wr.ActionTimeout()) slaveList, err := wr.tmc.GetSlaves(context.TODO(), masterTablet, wr.ActionTimeout())
if err != nil { if err != nil {
results <- vresult{shardInfo.MasterAlias.String(), err} results <- vresult{shardInfo.MasterAlias.String(), err}
return return
@ -261,7 +263,7 @@ func (wr *Wrangler) pingTablets(tabletMap map[topo.TabletAlias]*topo.TabletInfo,
return return
} }
if err := wr.tmc.Ping(tabletInfo, wr.ActionTimeout()); err != nil { if err := wr.tmc.Ping(context.TODO(), tabletInfo, wr.ActionTimeout()); err != nil {
results <- vresult{tabletAlias.String(), fmt.Errorf("Ping failed: %v %v", err, tabletInfo.Hostname)} results <- vresult{tabletAlias.String(), fmt.Errorf("Ping failed: %v %v", err, tabletInfo.Hostname)}
} }
}(tabletAlias, tabletInfo) }(tabletAlias, tabletInfo)

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

@ -3,6 +3,8 @@ package zktopo
import ( import (
"testing" "testing"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/vt/topo/test" "github.com/youtube/vitess/go/vt/topo/test"
) )
@ -21,7 +23,7 @@ func TestShard(t *testing.T) {
func TestTablet(t *testing.T) { func TestTablet(t *testing.T) {
ts := NewTestServer(t, []string{"test"}) ts := NewTestServer(t, []string{"test"})
defer ts.Close() defer ts.Close()
test.CheckTablet(t, ts) test.CheckTablet(context.Background(), t, ts)
} }
func TestShardReplication(t *testing.T) { func TestShardReplication(t *testing.T) {

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

@ -10,6 +10,8 @@ import (
"strings" "strings"
"time" "time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog" log "github.com/golang/glog"
"github.com/youtube/vitess/go/rpcplus" "github.com/youtube/vitess/go/rpcplus"
"github.com/youtube/vitess/go/rpcwrap/bsonrpc" "github.com/youtube/vitess/go/rpcwrap/bsonrpc"
@ -53,7 +55,7 @@ func DialZkocc(addr string, connectTimeout time.Duration) (zkocc *ZkoccConn, err
func (conn *ZkoccConn) Get(path string) (data string, stat Stat, err error) { func (conn *ZkoccConn) Get(path string) (data string, stat Stat, err error) {
zkPath := &ZkPath{path} zkPath := &ZkPath{path}
zkNode := &ZkNode{} zkNode := &ZkNode{}
if err := conn.rpcClient.Call("ZkReader.Get", zkPath, zkNode); err != nil { if err := conn.rpcClient.Call(context.TODO(), "ZkReader.Get", zkPath, zkNode); err != nil {
return "", nil, err return "", nil, err
} }
return zkNode.Data, &zkNode.Stat, nil return zkNode.Data, &zkNode.Stat, nil
@ -66,7 +68,7 @@ func (conn *ZkoccConn) GetW(path string) (data string, stat Stat, watch <-chan z
func (conn *ZkoccConn) Children(path string) (children []string, stat Stat, err error) { func (conn *ZkoccConn) Children(path string) (children []string, stat Stat, err error) {
zkPath := &ZkPath{path} zkPath := &ZkPath{path}
zkNode := &ZkNode{} zkNode := &ZkNode{}
if err := conn.rpcClient.Call("ZkReader.Children", zkPath, zkNode); err != nil { if err := conn.rpcClient.Call(context.TODO(), "ZkReader.Children", zkPath, zkNode); err != nil {
return nil, nil, err return nil, nil, err
} }
return zkNode.Children, &zkNode.Stat, nil return zkNode.Children, &zkNode.Stat, nil
@ -81,7 +83,7 @@ func (conn *ZkoccConn) ChildrenW(path string) (children []string, stat Stat, wat
func (conn *ZkoccConn) Exists(path string) (stat Stat, err error) { func (conn *ZkoccConn) Exists(path string) (stat Stat, err error) {
zkPath := &ZkPath{path} zkPath := &ZkPath{path}
zkNode := &ZkNode{} zkNode := &ZkNode{}
if err := conn.rpcClient.Call("ZkReader.Get", zkPath, zkNode); err != nil { if err := conn.rpcClient.Call(context.TODO(), "ZkReader.Get", zkPath, zkNode); err != nil {
return nil, err return nil, err
} }
return &zkNode.Stat, nil return &zkNode.Stat, nil