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"
"time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/acl"
"github.com/youtube/vitess/go/vt/logutil"
@ -402,7 +404,7 @@ func main() {
if err != nil {
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,

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

@ -12,6 +12,8 @@ import (
"sort"
"time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/rpcplus"
"github.com/youtube/vitess/go/rpcwrap/bsonrpc"
@ -52,7 +54,7 @@ func get(rpcClient *rpcplus.Client, path string, verbose bool) {
// it's a get
zkPath := &zk.ZkPath{Path: path}
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)
}
if verbose {
@ -67,7 +69,7 @@ func getv(rpcClient *rpcplus.Client, paths []string, verbose bool) {
zkPathV.Paths[i] = v
}
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)
}
if verbose {
@ -81,7 +83,7 @@ func children(rpcClient *rpcplus.Client, paths []string, verbose bool) {
for _, v := range paths {
zkPath := &zk.ZkPath{Path: v}
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)
}
if verbose {
@ -102,7 +104,7 @@ func getSrvKeyspaceNames(rpcClient *rpcplus.Client, cell string, verbose bool) {
Cell: cell,
}
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)
}
if verbose {
@ -118,7 +120,7 @@ func getSrvKeyspace(rpcClient *rpcplus.Client, cell, keyspace string, verbose bo
Keyspace: keyspace,
}
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)
}
if verbose {
@ -150,7 +152,7 @@ func getEndPoints(rpcClient *rpcplus.Client, cell, keyspace, shard, tabletType s
TabletType: topo.TabletType(tabletType),
}
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)
}
if verbose {

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

@ -14,6 +14,9 @@ import (
"net/http"
"reflect"
"sync"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/trace"
)
// 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 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.
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.ServiceMethod = serviceMethod
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.
func (client *Client) Call(serviceMethod string, args interface{}, reply interface{}) error {
call := <-client.Go(serviceMethod, args, reply, make(chan *Call, 1)).Done
func (client *Client) Call(ctx context.Context, serviceMethod string, args interface{}, reply interface{}) error {
call := <-client.Go(ctx, serviceMethod, args, reply, make(chan *Call, 1)).Done
return call.Error
}

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

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

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

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

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

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

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

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

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

@ -10,6 +10,8 @@ import (
"fmt"
"time"
"code.google.com/p/go.net/context"
rpc "github.com/youtube/vitess/go/rpcplus"
"github.com/youtube/vitess/go/rpcwrap/bsonrpc"
"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.
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.
@ -39,7 +41,7 @@ func (c *goRpcMysqlctlClient) Shutdown(waitForMysqld bool, mysqlWaitTime time.Du
if !waitForMysqld {
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.

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

@ -10,7 +10,9 @@ package actionnode
import (
"time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/trace"
"github.com/youtube/vitess/go/vt/topo"
)
@ -22,8 +24,15 @@ var (
// LockKeyspace will lock the keyspace in the topology server.
// 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)
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)
}
@ -52,8 +61,16 @@ func (n *ActionNode) UnlockKeyspace(ts topo.Server, keyspace string, lockPath st
// LockShard will lock the shard in the topology server.
// 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)
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)
}
@ -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.
// 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)
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)
}

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

@ -30,6 +30,8 @@ import (
"sync"
"time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/history"
"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)
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")
return nil
}

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

@ -44,91 +44,91 @@ type RpcAgent interface {
// 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
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
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
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
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
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.
// Should be called under RpcWrap.
func (agent *ActionAgent) Ping(args string) string {
func (agent *ActionAgent) Ping(ctx context.Context, args string) string {
return args
}
// GetSchema returns the schema.
// 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)
}
// GetPermissions returns the db permissions.
// 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()
}
// SetReadOnly makes the mysql instance read-only or read-write
// 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)
if err != nil {
return err
@ -178,48 +178,48 @@ func (agent *ActionAgent) SetReadOnly(rdonly bool) error {
} else {
tablet.State = topo.STATE_READ_WRITE
}
return topo.UpdateTablet(agent.TopoServer, tablet)
return topo.UpdateTablet(ctx, agent.TopoServer, tablet)
}
// ChangeType changes the tablet type
// 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*/)
}
// Scrap scraps the live running tablet
// 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)
}
// Sleep sleeps for the duration
// 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)
}
// ExecuteHook executes the provided hook locally, and returns the result.
// 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)
return hk.Execute()
}
// RefreshState reload the tablet record from the topo server.
// 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
// 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)
}
// ReloadSchema will reload the schema
// Should be called under RpcWrapLockAction.
func (agent *ActionAgent) ReloadSchema() {
func (agent *ActionAgent) ReloadSchema(ctx context.Context) {
if agent.DBConfigs == nil {
// we skip this for test instances that can't connect to the DB anyway
return
@ -233,7 +233,7 @@ func (agent *ActionAgent) ReloadSchema() {
// PreflightSchema will try out the schema change
// 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
tablet := agent.Tablet()
@ -243,7 +243,7 @@ func (agent *ActionAgent) PreflightSchema(change string) (*myproto.SchemaChangeR
// ApplySchema will apply a schema change
// 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
tablet := agent.Tablet()
@ -254,13 +254,13 @@ func (agent *ActionAgent) ApplySchema(change *myproto.SchemaChange) (*myproto.Sc
}
// and if it worked, reload the schema
agent.ReloadSchema()
agent.ReloadSchema(ctx)
return scr, nil
}
// ExecuteFetch will execute the given query, possibly disabling binlogs.
// 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
conn, err := agent.MysqlDaemon.GetDbaConnection()
if err != nil {
@ -294,14 +294,14 @@ func (agent *ActionAgent) ExecuteFetch(query string, maxrows int, wantFields, di
// SlaveStatus returns the replication status
// 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()
}
// WaitSlavePosition waits until we reach the provided position,
// and returns the current position
// 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 {
return nil, err
}
@ -311,14 +311,14 @@ func (agent *ActionAgent) WaitSlavePosition(position myproto.ReplicationPosition
// MasterPosition returns the master position
// 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()
}
// ReparentPosition returns the RestartSlaveData for the provided
// ReplicationPosition.
// 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)
if err != nil {
return nil, err
@ -333,13 +333,13 @@ func (agent *ActionAgent) ReparentPosition(rp *myproto.ReplicationPosition) (*ac
// StopSlave will stop the replication
// Should be called under RpcWrapLock.
func (agent *ActionAgent) StopSlave() error {
func (agent *ActionAgent) StopSlave(ctx context.Context) error {
return agent.MysqlDaemon.StopSlave(agent.hookExtraEnv())
}
// StopSlaveMinimum will stop the slave after it reaches at least the
// 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 {
return nil, err
}
@ -351,14 +351,14 @@ func (agent *ActionAgent) StopSlaveMinimum(position myproto.ReplicationPosition,
// StartSlave will start the replication
// Should be called under RpcWrapLock.
func (agent *ActionAgent) StartSlave() error {
func (agent *ActionAgent) StartSlave(ctx context.Context) error {
return agent.MysqlDaemon.StartSlave(agent.hookExtraEnv())
}
// TabletExternallyReparented updates all topo records so the current
// tablet is the new master for this shard.
// 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()
// 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
actionNode := actionnode.ShardExternallyReparented(agent.TabletAlias)
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 {
log.Warningf("TabletExternallyReparented: Cannot lock shard %v/%v: %v", tablet.Keyspace, tablet.Shard, err)
return err
}
// do the work
runAfterAction, err := agent.tabletExternallyReparentedLocked(actionTimeout, interrupted)
runAfterAction, err := agent.tabletExternallyReparentedLocked(ctx, actionTimeout, interrupted)
if err != nil {
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.
// It returns if agent.refreshTablet should be called, and the 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
tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias)
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
// by the agent.
event.DispatchUpdate(ev, "mark ourself as new master")
err = agent.updateReplicationGraphForPromotedSlave(tablet)
err = agent.updateReplicationGraphForPromotedSlave(ctx, tablet)
if err != nil {
// This suggests we can't talk to topo server. This is bad.
return true, fmt.Errorf("updateReplicationGraphForPromotedSlave failed: %v", err)
@ -493,7 +493,7 @@ func (agent *ActionAgent) tabletExternallyReparentedLocked(actionTimeout time.Du
logger := logutil.NewConsoleLogger()
tmc := tmclient.NewTabletManagerClient()
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
@ -507,14 +507,14 @@ func (agent *ActionAgent) tabletExternallyReparentedLocked(actionTimeout time.Du
event.DispatchUpdate(ev, "updating shard record")
log.Infof("Updating Shard's MasterAlias record")
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
}
// and rebuild the shard serving graph
event.DispatchUpdate(ev, "rebuilding shard serving graph")
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
}
@ -524,20 +524,20 @@ func (agent *ActionAgent) tabletExternallyReparentedLocked(actionTimeout time.Du
// GetSlaves returns the address of all the slaves
// Should be called under RpcWrap.
func (agent *ActionAgent) GetSlaves() ([]string, error) {
func (agent *ActionAgent) GetSlaves(ctx context.Context) ([]string, error) {
return agent.Mysqld.FindSlaves()
}
// WaitBlpPosition waits until a specific filtered replication position is
// reached.
// 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)
}
// StopBlp stops the binlog players, and return their positions.
// 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 {
return nil, fmt.Errorf("No BinlogPlayerMap configured")
}
@ -547,7 +547,7 @@ func (agent *ActionAgent) StopBlp() (*blproto.BlpPositionList, error) {
// StartBlp starts the binlog players
// Should be called under RpcWrapLockAction.
func (agent *ActionAgent) StartBlp() error {
func (agent *ActionAgent) StartBlp(ctx context.Context) error {
if agent.BinlogPlayerMap == nil {
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,
// 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 {
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.
// Should be called under RpcWrapLockAction.
func (agent *ActionAgent) DemoteMaster() error {
func (agent *ActionAgent) DemoteMaster(ctx context.Context) error {
_, err := agent.Mysqld.DemoteMaster()
if err != nil {
return err
@ -593,7 +593,7 @@ func (agent *ActionAgent) DemoteMaster() error {
// PromoteSlave transforms the current tablet from a slave to a master.
// It returns the data needed for other tablets to become a slave.
// 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)
if err != nil {
return nil, err
@ -610,23 +610,23 @@ func (agent *ActionAgent) PromoteSlave() (*actionnode.RestartSlaveData, error) {
}
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.
// 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)
if err != nil {
return err
}
return agent.updateReplicationGraphForPromotedSlave(tablet)
return agent.updateReplicationGraphForPromotedSlave(ctx, tablet)
}
// RestartSlave tells the tablet it has a new master
// 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)
if err != nil {
return err
@ -655,7 +655,7 @@ func (agent *ActionAgent) RestartSlave(rsd *actionnode.RestartSlaveData) error {
}
// Once this action completes, update authoritive tablet node first.
tablet.Parent = rsd.Parent
err = topo.UpdateTablet(agent.TopoServer, tablet)
err = topo.UpdateTablet(ctx, agent.TopoServer, tablet)
if err != nil {
return err
}
@ -667,7 +667,7 @@ func (agent *ActionAgent) RestartSlave(rsd *actionnode.RestartSlaveData) error {
// Complete the special orphan accounting.
if tablet.Type == topo.TYPE_LAG_ORPHAN {
tablet.Type = topo.TYPE_LAG
err = topo.UpdateTablet(agent.TopoServer, tablet)
err = topo.UpdateTablet(ctx, agent.TopoServer, tablet)
if err != nil {
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
// 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 {
return err
}
@ -696,7 +696,7 @@ func (agent *ActionAgent) RestartSlave(rsd *actionnode.RestartSlaveData) error {
// SlaveWasRestarted updates the parent record for a tablet.
// 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)
if err != nil {
return err
@ -708,14 +708,14 @@ func (agent *ActionAgent) SlaveWasRestarted(swrd *actionnode.SlaveWasRestartedAr
tablet.Type = topo.TYPE_SPARE
tablet.State = topo.STATE_READ_ONLY
}
err = topo.UpdateTablet(agent.TopoServer, tablet)
err = topo.UpdateTablet(ctx, agent.TopoServer, tablet)
if err != nil {
return err
}
// Update the new tablet location in the replication graph now that
// 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 {
return err
}
@ -726,20 +726,20 @@ func (agent *ActionAgent) SlaveWasRestarted(swrd *actionnode.SlaveWasRestartedAr
// BreakSlaves will tinker with the replication stream in a way that
// will stop all the slaves.
// Should be called under RpcWrapLockAction.
func (agent *ActionAgent) BreakSlaves() error {
func (agent *ActionAgent) BreakSlaves(ctx context.Context) error {
return agent.Mysqld.BreakSlaves()
}
// updateReplicationGraphForPromotedSlave makes sure the newly promoted slave
// 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.
tablet.State = topo.STATE_READ_WRITE
tablet.Type = topo.TYPE_MASTER
tablet.Parent.Cell = ""
tablet.Parent.Uid = topo.NO_TABLET
tablet.Health = nil
err := topo.UpdateTablet(agent.TopoServer, tablet)
err := topo.UpdateTablet(ctx, agent.TopoServer, tablet)
if err != nil {
return err
}
@ -750,7 +750,7 @@ func (agent *ActionAgent) updateReplicationGraphForPromotedSlave(tablet *topo.Ta
// Insert the new tablet location in the replication graph now that
// 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 {
return err
}
@ -764,7 +764,7 @@ func (agent *ActionAgent) updateReplicationGraphForPromotedSlave(tablet *topo.Ta
// Snapshot takes a db snapshot
// 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
tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias)
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
// master.
tablet.Tablet.Type = topo.TYPE_BACKUP
err = topo.UpdateTablet(agent.TopoServer, tablet)
err = topo.UpdateTablet(ctx, agent.TopoServer, tablet)
} else {
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 {
log.Infof("force change type backup -> master: %v", tablet.Alias)
tablet.Tablet.Type = topo.TYPE_MASTER
err = topo.UpdateTablet(agent.TopoServer, tablet)
err = topo.UpdateTablet(ctx, agent.TopoServer, tablet)
} else {
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
// Snapshot(server_mode =true)
// 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)
if err != nil {
return err
@ -868,7 +868,7 @@ func (agent *ActionAgent) SnapshotSourceEnd(args *actionnode.SnapshotSourceEndAr
if args.OriginalType == topo.TYPE_MASTER {
// force the master update
tablet.Tablet.Type = topo.TYPE_MASTER
err = topo.UpdateTablet(agent.TopoServer, tablet)
err = topo.UpdateTablet(ctx, agent.TopoServer, tablet)
} else {
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)
// - to SCRAP if something in the process on the target host fails
// - 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
hk := hook.NewSimpleHook("preflight_assigned")
topotools.ConfigureTabletHook(hk, agent.TabletAlias)
@ -897,18 +897,18 @@ func (agent *ActionAgent) changeTypeToRestore(tablet, sourceTablet *topo.TabletI
tablet.Type = topo.TYPE_RESTORE
tablet.KeyRange = keyRange
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
}
// 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
// restore operation.
// 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
if err := agent.Mysqld.ValidateCloneTarget(agent.hookExtraEnv()); err != nil {
return err
@ -939,7 +939,7 @@ func (agent *ActionAgent) ReserveForRestore(args *actionnode.ReserveForRestoreAr
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 {
@ -969,7 +969,7 @@ func fetchAndParseJsonFile(addr, filename string, result interface{}) error {
// Restart mysqld and replication.
// Put tablet into the replication graph as a spare.
// 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
tablet, err := agent.TopoServer.GetTablet(agent.TabletAlias)
if err != nil {
@ -1009,7 +1009,7 @@ func (agent *ActionAgent) Restore(args *actionnode.RestoreArgs, logger logutil.L
}
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
}
}
@ -1028,7 +1028,7 @@ func (agent *ActionAgent) Restore(args *actionnode.RestoreArgs, logger logutil.L
}
// reload the schema
agent.ReloadSchema()
agent.ReloadSchema(ctx)
// change to TYPE_SPARE, we're done!
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
// 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)
if err != nil {
return nil, err
@ -1071,7 +1071,7 @@ func (agent *ActionAgent) MultiSnapshot(args *actionnode.MultiSnapshotArgs, logg
// MultiRestore performs the multi-part restore.
// 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
// we only support restoring to the master or active replicas
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
originalType := tablet.Type
tablet.Type = topo.TYPE_RESTORE
err = topo.UpdateTablet(agent.TopoServer, tablet)
err = topo.UpdateTablet(ctx, agent.TopoServer, tablet)
if err != nil {
return err
}
@ -1157,7 +1157,7 @@ func (agent *ActionAgent) MultiRestore(args *actionnode.MultiRestoreArgs, logger
}
// reload the schema
agent.ReloadSchema()
agent.ReloadSchema(ctx)
// restart replication
if topo.IsSlaveType(originalType) {
@ -1168,5 +1168,5 @@ func (agent *ActionAgent) MultiRestore(args *actionnode.MultiRestoreArgs, logger
// restore type back
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
//
func (fra *fakeRpcAgent) Ping(args string) string {
func (fra *fakeRpcAgent) Ping(ctx context.Context, args string) string {
return args
}
func agentRpcTestPing(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.Ping(ti, time.Minute)
func agentRpcTestPing(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.Ping(ctx, ti, time.Minute)
if err != nil {
t.Errorf("Ping failed: %v", err)
}
@ -130,15 +130,15 @@ var testGetSchemaReply = &myproto.SchemaDefinition{
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 excludeTables", excludeTables, testGetSchemaExcludeTables)
compareBool(fra.t, "GetSchema includeViews", includeViews)
return testGetSchemaReply, nil
}
func agentRpcTestGetSchema(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
result, err := client.GetSchema(ti, testGetSchemaTables, testGetSchemaExcludeTables, true, time.Minute)
func agentRpcTestGetSchema(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
result, err := client.GetSchema(ctx, ti, testGetSchemaTables, testGetSchemaExcludeTables, true, time.Minute)
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
}
func agentRpcTestGetPermissions(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
result, err := client.GetPermissions(ti, time.Minute)
func agentRpcTestGetPermissions(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
result, err := client.GetPermissions(ctx, ti, time.Minute)
compareError(t, "GetPermissions", err, result, testGetPermissionsReply)
}
@ -192,21 +192,21 @@ func agentRpcTestGetPermissions(t *testing.T, client tmclient.TabletManagerClien
var testSetReadOnlyExpectedValue bool
func (fra *fakeRpcAgent) SetReadOnly(rdonly bool) error {
func (fra *fakeRpcAgent) SetReadOnly(ctx context.Context, rdonly bool) error {
if rdonly != testSetReadOnlyExpectedValue {
fra.t.Errorf("Wrong SetReadOnly value: got %v expected %v", rdonly, testSetReadOnlyExpectedValue)
}
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
err := client.SetReadOnly(ti, time.Minute)
err := client.SetReadOnly(ctx, ti, time.Minute)
if err != nil {
t.Errorf("SetReadOnly failed: %v", err)
}
testSetReadOnlyExpectedValue = false
err = client.SetReadWrite(ti, time.Minute)
err = client.SetReadWrite(ctx, ti, time.Minute)
if err != nil {
t.Errorf("SetReadWrite failed: %v", err)
}
@ -214,13 +214,13 @@ func agentRpcTestSetReadOnly(t *testing.T, client tmclient.TabletManagerClient,
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)
return nil
}
func agentRpcTestChangeType(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.ChangeType(ti, testChangeTypeValue, time.Minute)
func agentRpcTestChangeType(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.ChangeType(ctx, ti, testChangeTypeValue, time.Minute)
if err != nil {
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!")
func (fra *fakeRpcAgent) Scrap() error {
func (fra *fakeRpcAgent) Scrap(ctx context.Context) error {
return testScrapError
}
func agentRpcTestScrap(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.Scrap(ti, time.Minute)
func agentRpcTestScrap(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.Scrap(ctx, ti, time.Minute)
if strings.Index(err.Error(), testScrapError.Error()) == -1 {
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
func (fra *fakeRpcAgent) Sleep(duration time.Duration) {
func (fra *fakeRpcAgent) Sleep(ctx context.Context, duration time.Duration) {
compare(fra.t, "Sleep duration", duration, testSleepDuration)
}
func agentRpcTestSleep(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.Sleep(ti, testSleepDuration, time.Minute)
func agentRpcTestSleep(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.Sleep(ctx, ti, testSleepDuration, time.Minute)
if err != nil {
t.Errorf("Sleep failed: %v", err)
}
@ -266,27 +266,27 @@ var testExecuteHookHookResult = &hook.HookResult{
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)
return testExecuteHookHookResult
}
func agentRpcTestExecuteHook(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
hr, err := client.ExecuteHook(ti, testExecuteHookHook, time.Minute)
func agentRpcTestExecuteHook(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
hr, err := client.ExecuteHook(ctx, ti, testExecuteHookHook, time.Minute)
compareError(t, "ExecuteHook", err, hr, testExecuteHookHookResult)
}
var testRefreshStateCalled = false
func (fra *fakeRpcAgent) RefreshState() {
func (fra *fakeRpcAgent) RefreshState(ctx context.Context) {
if testRefreshStateCalled {
fra.t.Errorf("RefreshState called multiple times?")
}
testRefreshStateCalled = true
}
func agentRpcTestRefreshState(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.RefreshState(ti, time.Minute)
func agentRpcTestRefreshState(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.RefreshState(ctx, ti, time.Minute)
if err != nil {
t.Errorf("RefreshState failed: %v", err)
}
@ -297,12 +297,12 @@ func agentRpcTestRefreshState(t *testing.T, client tmclient.TabletManagerClient,
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)
}
func agentRpcTestRunHealthCheck(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.RunHealthCheck(ti, testRunHealthCheckValue, time.Minute)
func agentRpcTestRunHealthCheck(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.RunHealthCheck(ctx, ti, testRunHealthCheckValue, time.Minute)
if err != nil {
t.Errorf("RunHealthCheck failed: %v", err)
}
@ -310,15 +310,15 @@ func agentRpcTestRunHealthCheck(t *testing.T, client tmclient.TabletManagerClien
var testReloadSchemaCalled = false
func (fra *fakeRpcAgent) ReloadSchema() {
func (fra *fakeRpcAgent) ReloadSchema(ctx context.Context) {
if testReloadSchemaCalled {
fra.t.Errorf("ReloadSchema called multiple times?")
}
testReloadSchemaCalled = true
}
func agentRpcTestReloadSchema(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.ReloadSchema(ti, time.Minute)
func agentRpcTestReloadSchema(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.ReloadSchema(ctx, ti, time.Minute)
if err != nil {
t.Errorf("ReloadSchema failed: %v", err)
}
@ -333,13 +333,13 @@ var testSchemaChangeResult = &myproto.SchemaChangeResult{
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)
return testSchemaChangeResult, nil
}
func agentRpcTestPreflightSchema(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
scr, err := client.PreflightSchema(ti, testPreflightSchema, time.Minute)
func agentRpcTestPreflightSchema(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
scr, err := client.PreflightSchema(ctx, ti, testPreflightSchema, time.Minute)
compareError(t, "PreflightSchema", err, scr, testSchemaChangeResult)
}
@ -351,13 +351,13 @@ var testSchemaChange = &myproto.SchemaChange{
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)
return testSchemaChangeResult, nil
}
func agentRpcTestApplySchema(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
scr, err := client.ApplySchema(ti, testSchemaChange, time.Minute)
func agentRpcTestApplySchema(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
scr, err := client.ApplySchema(ctx, ti, testSchemaChange, time.Minute)
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 maxrows", maxrows, testExecuteFetchMaxRows)
compareBool(fra.t, "ExecuteFetch wantFields", wantFields)
@ -391,8 +391,8 @@ func (fra *fakeRpcAgent) ExecuteFetch(query string, maxrows int, wantFields, dis
return testExecuteFetchResult, nil
}
func agentRpcTestExecuteFetch(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
qr, err := client.ExecuteFetch(ti, testExecuteFetchQuery, testExecuteFetchMaxRows, true, true, time.Minute)
func agentRpcTestExecuteFetch(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
qr, err := client.ExecuteFetch(ctx, ti, testExecuteFetchQuery, testExecuteFetchMaxRows, true, true, time.Minute)
compareError(t, "ExecuteFetch", err, qr, testExecuteFetchResult)
}
@ -415,12 +415,12 @@ var testReplicationStatus = &myproto.ReplicationStatus{
MasterConnectRetry: 12,
}
func (fra *fakeRpcAgent) SlaveStatus() (*myproto.ReplicationStatus, error) {
func (fra *fakeRpcAgent) SlaveStatus(ctx context.Context) (*myproto.ReplicationStatus, error) {
return testReplicationStatus, nil
}
func agentRpcTestSlaveStatus(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rs, err := client.SlaveStatus(ti, time.Minute)
func agentRpcTestSlaveStatus(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rs, err := client.SlaveStatus(ctx, ti, time.Minute)
compareError(t, "SlaveStatus", err, rs, testReplicationStatus)
}
@ -433,23 +433,23 @@ var testReplicationPosition = myproto.ReplicationPosition{
}
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 waitTimeout", waitTimeout, testWaitSlavePositionWaitTimeout)
return testReplicationStatus, nil
}
func agentRpcTestWaitSlavePosition(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rs, err := client.WaitSlavePosition(ti, testReplicationPosition, testWaitSlavePositionWaitTimeout)
func agentRpcTestWaitSlavePosition(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rs, err := client.WaitSlavePosition(ctx, ti, testReplicationPosition, testWaitSlavePositionWaitTimeout)
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
}
func agentRpcTestMasterPosition(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rs, err := client.MasterPosition(ti, time.Minute)
func agentRpcTestMasterPosition(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rs, err := client.MasterPosition(ctx, ti, time.Minute)
compareError(t, "MasterPosition", err, rs, testReplicationPosition)
}
@ -464,73 +464,73 @@ var testRestartSlaveData = &actionnode.RestartSlaveData{
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)
return testRestartSlaveData, nil
}
func agentRpcTestReparentPosition(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rsd, err := client.ReparentPosition(ti, &testReplicationPosition, time.Minute)
func agentRpcTestReparentPosition(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rsd, err := client.ReparentPosition(ctx, ti, &testReplicationPosition, time.Minute)
compareError(t, "ReparentPosition", err, rsd, testRestartSlaveData)
}
var testStopSlaveCalled = false
func (fra *fakeRpcAgent) StopSlave() error {
func (fra *fakeRpcAgent) StopSlave(ctx context.Context) error {
testStopSlaveCalled = true
return nil
}
func agentRpcTestStopSlave(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.StopSlave(ti, time.Minute)
func agentRpcTestStopSlave(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.StopSlave(ctx, ti, time.Minute)
compareError(t, "StopSlave", err, true, testStopSlaveCalled)
}
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 waitTime", waitTime, testStopSlaveMinimumWaitTime)
return testReplicationStatus, nil
}
func agentRpcTestStopSlaveMinimum(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rs, err := client.StopSlaveMinimum(ti, testReplicationPosition, testStopSlaveMinimumWaitTime)
func agentRpcTestStopSlaveMinimum(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rs, err := client.StopSlaveMinimum(ctx, ti, testReplicationPosition, testStopSlaveMinimumWaitTime)
compareError(t, "StopSlave", err, rs, testReplicationStatus)
}
var testStartSlaveCalled = false
func (fra *fakeRpcAgent) StartSlave() error {
func (fra *fakeRpcAgent) StartSlave(ctx context.Context) error {
testStartSlaveCalled = true
return nil
}
func agentRpcTestStartSlave(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.StartSlave(ti, time.Minute)
func agentRpcTestStartSlave(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.StartSlave(ctx, ti, time.Minute)
compareError(t, "StartSlave", err, true, testStartSlaveCalled)
}
var testTabletExternallyReparentedCalled = false
func (fra *fakeRpcAgent) TabletExternallyReparented(actionTimeout time.Duration) error {
func (fra *fakeRpcAgent) TabletExternallyReparented(ctx context.Context, actionTimeout time.Duration) error {
testTabletExternallyReparentedCalled = true
return nil
}
func agentRpcTestTabletExternallyReparented(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.TabletExternallyReparented(ti, time.Minute)
func agentRpcTestTabletExternallyReparented(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.TabletExternallyReparented(ctx, ti, time.Minute)
compareError(t, "TabletExternallyReparented", err, true, testTabletExternallyReparentedCalled)
}
var testGetSlavesResult = []string{"slave1", "slave2"}
func (fra *fakeRpcAgent) GetSlaves() ([]string, error) {
func (fra *fakeRpcAgent) GetSlaves(ctx context.Context) ([]string, error) {
return testGetSlavesResult, nil
}
func agentRpcTestGetSlaves(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
s, err := client.GetSlaves(ti, time.Minute)
func agentRpcTestGetSlaves(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
s, err := client.GetSlaves(ctx, ti, time.Minute)
compareError(t, "GetSlaves", err, s, testGetSlavesResult)
}
@ -541,15 +541,15 @@ var testBlpPosition = &blproto.BlpPosition{
var testWaitBlpPositionWaitTime = time.Hour
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 waitTime", waitTime, testWaitBlpPositionWaitTime)
testWaitBlpPositionCalled = true
return nil
}
func agentRpcTestWaitBlpPosition(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.WaitBlpPosition(ti, *testBlpPosition, testWaitBlpPositionWaitTime)
func agentRpcTestWaitBlpPosition(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.WaitBlpPosition(ctx, ti, *testBlpPosition, testWaitBlpPositionWaitTime)
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
}
func agentRpcTestStopBlp(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
bpl, err := client.StopBlp(ti, time.Minute)
func agentRpcTestStopBlp(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
bpl, err := client.StopBlp(ctx, ti, time.Minute)
compareError(t, "StopBlp", err, bpl, testBlpPositionList)
}
var testStartBlpCalled = false
func (fra *fakeRpcAgent) StartBlp() error {
func (fra *fakeRpcAgent) StartBlp(ctx context.Context) error {
testStartBlpCalled = true
return nil
}
func agentRpcTestStartBlp(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.StartBlp(ti, time.Minute)
func agentRpcTestStartBlp(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.StartBlp(ctx, ti, time.Minute)
compareError(t, "StartBlp", err, true, testStartBlpCalled)
}
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 waitTime", waitTime, testRunBlpUntilWaitTime)
return &testReplicationPosition, nil
}
func agentRpcTestRunBlpUntil(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rp, err := client.RunBlpUntil(ti, testBlpPositionList, testRunBlpUntilWaitTime)
func agentRpcTestRunBlpUntil(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rp, err := client.RunBlpUntil(ctx, ti, testBlpPositionList, testRunBlpUntilWaitTime)
compareError(t, "RunBlpUntil", err, rp, testReplicationPosition)
}
@ -599,47 +599,47 @@ func agentRpcTestRunBlpUntil(t *testing.T, client tmclient.TabletManagerClient,
var testDemoteMasterCalled = false
func (fra *fakeRpcAgent) DemoteMaster() error {
func (fra *fakeRpcAgent) DemoteMaster(ctx context.Context) error {
testDemoteMasterCalled = true
return nil
}
func agentRpcTestDemoteMaster(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.DemoteMaster(ti, time.Minute)
func agentRpcTestDemoteMaster(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.DemoteMaster(ctx, ti, time.Minute)
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
}
func agentRpcTestPromoteSlave(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rsd, err := client.PromoteSlave(ti, time.Minute)
func agentRpcTestPromoteSlave(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
rsd, err := client.PromoteSlave(ctx, ti, time.Minute)
compareError(t, "PromoteSlave", err, rsd, testRestartSlaveData)
}
var testSlaveWasPromotedCalled = false
func (fra *fakeRpcAgent) SlaveWasPromoted() error {
func (fra *fakeRpcAgent) SlaveWasPromoted(ctx context.Context) error {
testSlaveWasPromotedCalled = true
return nil
}
func agentRpcTestSlaveWasPromoted(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.SlaveWasPromoted(ti, time.Minute)
func agentRpcTestSlaveWasPromoted(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.SlaveWasPromoted(ctx, ti, time.Minute)
compareError(t, "SlaveWasPromoted", err, true, testSlaveWasPromotedCalled)
}
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)
testRestartSlaveCalled = true
return nil
}
func agentRpcTestRestartSlave(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.RestartSlave(ti, testRestartSlaveData, time.Minute)
func agentRpcTestRestartSlave(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.RestartSlave(ctx, ti, testRestartSlaveData, time.Minute)
compareError(t, "RestartSlave", err, true, testRestartSlaveCalled)
}
@ -651,26 +651,26 @@ var testSlaveWasRestartedArgs = &actionnode.SlaveWasRestartedArgs{
}
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)
testSlaveWasRestartedCalled = true
return nil
}
func agentRpcTestSlaveWasRestarted(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.SlaveWasRestarted(ti, testSlaveWasRestartedArgs, time.Minute)
func agentRpcTestSlaveWasRestarted(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.SlaveWasRestarted(ctx, ti, testSlaveWasRestartedArgs, time.Minute)
compareError(t, "RestartSlave", err, true, testRestartSlaveCalled)
}
var testBreakSlavesCalled = false
func (fra *fakeRpcAgent) BreakSlaves() error {
func (fra *fakeRpcAgent) BreakSlaves(ctx context.Context) error {
testBreakSlavesCalled = true
return nil
}
func agentRpcTestBreakSlaves(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.BreakSlaves(ti, time.Minute)
func agentRpcTestBreakSlaves(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.BreakSlaves(ctx, ti, time.Minute)
compareError(t, "BreakSlaves", err, true, testBreakSlavesCalled)
}
@ -693,14 +693,14 @@ var testSnapshotReply = &actionnode.SnapshotReply{
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)
logStuff(logger, 0)
return testSnapshotReply, nil
}
func agentRpcTestSnapshot(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
logChannel, errFunc, err := client.Snapshot(ti, testSnapshotArgs, time.Minute)
func agentRpcTestSnapshot(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
logChannel, errFunc, err := client.Snapshot(ctx, ti, testSnapshotArgs, time.Minute)
if err != nil {
t.Fatalf("Snapshot failed: %v", err)
}
@ -716,14 +716,14 @@ var testSnapshotSourceEndArgs = &actionnode.SnapshotSourceEndArgs{
}
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)
testSnapshotSourceEndCalled = true
return nil
}
func agentRpcTestSnapshotSourceEnd(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.SnapshotSourceEnd(ti, testSnapshotSourceEndArgs, time.Minute)
func agentRpcTestSnapshotSourceEnd(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.SnapshotSourceEnd(ctx, ti, testSnapshotSourceEndArgs, time.Minute)
compareError(t, "SnapshotSourceEnd", err, true, testSnapshotSourceEndCalled)
}
@ -735,14 +735,14 @@ var testReserveForRestoreArgs = &actionnode.ReserveForRestoreArgs{
}
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)
testReserveForRestoreCalled = true
return nil
}
func agentRpcTestReserveForRestore(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.ReserveForRestore(ti, testReserveForRestoreArgs, time.Minute)
func agentRpcTestReserveForRestore(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
err := client.ReserveForRestore(ctx, ti, testReserveForRestoreArgs, time.Minute)
compareError(t, "ReserveForRestore", err, true, testReserveForRestoreCalled)
}
@ -763,15 +763,15 @@ var testRestoreArgs = &actionnode.RestoreArgs{
}
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)
logStuff(logger, 10)
testRestoreCalled = true
return nil
}
func agentRpcTestRestore(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
logChannel, errFunc, err := client.Restore(ti, testRestoreArgs, time.Minute)
func agentRpcTestRestore(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
logChannel, errFunc, err := client.Restore(ctx, ti, testRestoreArgs, time.Minute)
if err != nil {
t.Fatalf("Restore failed: %v", err)
}
@ -801,14 +801,14 @@ var testMultiSnapshotReply = &actionnode.MultiSnapshotReply{
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)
logStuff(logger, 100)
return testMultiSnapshotReply, nil
}
func agentRpcTestMultiSnapshot(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
logChannel, errFunc, err := client.MultiSnapshot(ti, testMultiSnapshotArgs, time.Minute)
func agentRpcTestMultiSnapshot(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
logChannel, errFunc, err := client.MultiSnapshot(ctx, ti, testMultiSnapshotArgs, time.Minute)
if err != nil {
t.Fatalf("MultiSnapshot failed: %v", err)
}
@ -836,15 +836,15 @@ var testMultiRestoreArgs = &actionnode.MultiRestoreArgs{
}
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)
logStuff(logger, 1000)
testMultiRestoreCalled = true
return nil
}
func agentRpcTestMultiRestore(t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
logChannel, errFunc, err := client.MultiRestore(ti, testMultiRestoreArgs, time.Minute)
func agentRpcTestMultiRestore(ctx context.Context, t *testing.T, client tmclient.TabletManagerClient, ti *topo.TabletInfo) {
logChannel, errFunc, err := client.MultiRestore(ctx, ti, testMultiRestoreArgs, time.Minute)
if err != nil {
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 client will connect to a server backed by our RpcAgent (returned
// 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
agentRpcTestPing(t, client, ti)
agentRpcTestGetSchema(t, client, ti)
agentRpcTestGetPermissions(t, client, ti)
agentRpcTestPing(ctx, t, client, ti)
agentRpcTestGetSchema(ctx, t, client, ti)
agentRpcTestGetPermissions(ctx, t, client, ti)
// Various read-write methods
agentRpcTestSetReadOnly(t, client, ti)
agentRpcTestChangeType(t, client, ti)
agentRpcTestScrap(t, client, ti)
agentRpcTestSleep(t, client, ti)
agentRpcTestExecuteHook(t, client, ti)
agentRpcTestRefreshState(t, client, ti)
agentRpcTestRunHealthCheck(t, client, ti)
agentRpcTestReloadSchema(t, client, ti)
agentRpcTestPreflightSchema(t, client, ti)
agentRpcTestApplySchema(t, client, ti)
agentRpcTestExecuteFetch(t, client, ti)
agentRpcTestSetReadOnly(ctx, t, client, ti)
agentRpcTestChangeType(ctx, t, client, ti)
agentRpcTestScrap(ctx, t, client, ti)
agentRpcTestSleep(ctx, t, client, ti)
agentRpcTestExecuteHook(ctx, t, client, ti)
agentRpcTestRefreshState(ctx, t, client, ti)
agentRpcTestRunHealthCheck(ctx, t, client, ti)
agentRpcTestReloadSchema(ctx, t, client, ti)
agentRpcTestPreflightSchema(ctx, t, client, ti)
agentRpcTestApplySchema(ctx, t, client, ti)
agentRpcTestExecuteFetch(ctx, t, client, ti)
// Replication related methods
agentRpcTestSlaveStatus(t, client, ti)
agentRpcTestWaitSlavePosition(t, client, ti)
agentRpcTestMasterPosition(t, client, ti)
agentRpcTestReparentPosition(t, client, ti)
agentRpcTestStopSlave(t, client, ti)
agentRpcTestStopSlaveMinimum(t, client, ti)
agentRpcTestStartSlave(t, client, ti)
agentRpcTestTabletExternallyReparented(t, client, ti)
agentRpcTestGetSlaves(t, client, ti)
agentRpcTestWaitBlpPosition(t, client, ti)
agentRpcTestStopBlp(t, client, ti)
agentRpcTestStartBlp(t, client, ti)
agentRpcTestRunBlpUntil(t, client, ti)
agentRpcTestSlaveStatus(ctx, t, client, ti)
agentRpcTestWaitSlavePosition(ctx, t, client, ti)
agentRpcTestMasterPosition(ctx, t, client, ti)
agentRpcTestReparentPosition(ctx, t, client, ti)
agentRpcTestStopSlave(ctx, t, client, ti)
agentRpcTestStopSlaveMinimum(ctx, t, client, ti)
agentRpcTestStartSlave(ctx, t, client, ti)
agentRpcTestTabletExternallyReparented(ctx, t, client, ti)
agentRpcTestGetSlaves(ctx, t, client, ti)
agentRpcTestWaitBlpPosition(ctx, t, client, ti)
agentRpcTestStopBlp(ctx, t, client, ti)
agentRpcTestStartBlp(ctx, t, client, ti)
agentRpcTestRunBlpUntil(ctx, t, client, ti)
// Reparenting related functions
agentRpcTestDemoteMaster(t, client, ti)
agentRpcTestPromoteSlave(t, client, ti)
agentRpcTestSlaveWasPromoted(t, client, ti)
agentRpcTestRestartSlave(t, client, ti)
agentRpcTestSlaveWasRestarted(t, client, ti)
agentRpcTestBreakSlaves(t, client, ti)
agentRpcTestDemoteMaster(ctx, t, client, ti)
agentRpcTestPromoteSlave(ctx, t, client, ti)
agentRpcTestSlaveWasPromoted(ctx, t, client, ti)
agentRpcTestRestartSlave(ctx, t, client, ti)
agentRpcTestSlaveWasRestarted(ctx, t, client, ti)
agentRpcTestBreakSlaves(ctx, t, client, ti)
// Backup / restore related methods
agentRpcTestSnapshot(t, client, ti)
agentRpcTestSnapshotSourceEnd(t, client, ti)
agentRpcTestReserveForRestore(t, client, ti)
agentRpcTestRestore(t, client, ti)
agentRpcTestMultiSnapshot(t, client, ti)
agentRpcTestMultiRestore(t, client, ti)
agentRpcTestSnapshot(ctx, t, client, ti)
agentRpcTestSnapshotSourceEnd(ctx, t, client, ti)
agentRpcTestReserveForRestore(ctx, t, client, ti)
agentRpcTestRestore(ctx, t, client, ti)
agentRpcTestMultiSnapshot(ctx, t, client, ti)
agentRpcTestMultiRestore(ctx, t, client, ti)
}

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

@ -8,6 +8,7 @@ import (
"fmt"
"time"
"code.google.com/p/go.net/context"
mproto "github.com/youtube/vitess/go/mysql/proto"
"github.com/youtube/vitess/go/rpcwrap/bsonrpc"
blproto "github.com/youtube/vitess/go/vt/binlog/proto"
@ -30,7 +31,7 @@ func init() {
// GoRpcTabletManagerClient implements tmclient.TabletManagerClient
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
// timeout, and starting the overall timeout as well
tmr := time.NewTimer(waitTime)
@ -42,7 +43,7 @@ func (client *GoRpcTabletManagerClient) rpcCallTablet(tablet *topo.TabletInfo, n
defer rpcClient.Close()
// 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 {
case <-tmr.C:
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
//
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
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 {
return err
}
@ -71,29 +72,29 @@ func (client *GoRpcTabletManagerClient) Ping(tablet *topo.TabletInfo, waitTime t
return nil
}
func (client *GoRpcTabletManagerClient) Sleep(tablet *topo.TabletInfo, duration, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_SLEEP, &duration, &rpc.Unused{}, waitTime)
func (client *GoRpcTabletManagerClient) Sleep(ctx context.Context, tablet *topo.TabletInfo, duration, waitTime time.Duration) error {
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
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 &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
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 &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
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 &p, nil
@ -103,53 +104,53 @@ func (client *GoRpcTabletManagerClient) GetPermissions(tablet *topo.TabletInfo,
// Various read-write methods
//
func (client *GoRpcTabletManagerClient) SetReadOnly(tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_SET_RDONLY, "", &rpc.Unused{}, waitTime)
func (client *GoRpcTabletManagerClient) SetReadOnly(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_SET_RDONLY, "", &rpc.Unused{}, waitTime)
}
func (client *GoRpcTabletManagerClient) SetReadWrite(tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_SET_RDWR, "", &rpc.Unused{}, waitTime)
func (client *GoRpcTabletManagerClient) SetReadWrite(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
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 {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_CHANGE_TYPE, &dbType, &rpc.Unused{}, waitTime)
func (client *GoRpcTabletManagerClient) ChangeType(ctx context.Context, tablet *topo.TabletInfo, dbType topo.TabletType, waitTime time.Duration) error {
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 {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_SCRAP, "", &rpc.Unused{}, waitTime)
func (client *GoRpcTabletManagerClient) Scrap(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_SCRAP, "", &rpc.Unused{}, waitTime)
}
func (client *GoRpcTabletManagerClient) RefreshState(tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_REFRESH_STATE, "", &rpc.Unused{}, waitTime)
func (client *GoRpcTabletManagerClient) RefreshState(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
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 {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_RUN_HEALTH_CHECK, &targetTabletType, &rpc.Unused{}, waitTime)
func (client *GoRpcTabletManagerClient) RunHealthCheck(ctx context.Context, tablet *topo.TabletInfo, targetTabletType topo.TabletType, waitTime time.Duration) error {
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 {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_RELOAD_SCHEMA, "", &rpc.Unused{}, waitTime)
func (client *GoRpcTabletManagerClient) ReloadSchema(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
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
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 &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
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 &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
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 &qr, nil
@ -159,17 +160,17 @@ func (client *GoRpcTabletManagerClient) ExecuteFetch(tablet *topo.TabletInfo, qu
// 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
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 &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
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,
WaitTimeout: waitTime,
}, &status, waitTime); err != nil {
@ -178,29 +179,29 @@ func (client *GoRpcTabletManagerClient) WaitSlavePosition(tablet *topo.TabletInf
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
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, 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
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 &rsd, nil
}
func (client *GoRpcTabletManagerClient) StopSlave(tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_STOP_SLAVE, "", &rpc.Unused{}, waitTime)
func (client *GoRpcTabletManagerClient) StopSlave(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
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
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,
WaitTime: waitTime,
}, &status, waitTime); err != nil {
@ -209,44 +210,44 @@ func (client *GoRpcTabletManagerClient) StopSlaveMinimum(tablet *topo.TabletInfo
return &status, nil
}
func (client *GoRpcTabletManagerClient) StartSlave(tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_START_SLAVE, "", &rpc.Unused{}, waitTime)
func (client *GoRpcTabletManagerClient) StartSlave(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_START_SLAVE, "", &rpc.Unused{}, waitTime)
}
func (client *GoRpcTabletManagerClient) TabletExternallyReparented(tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_EXTERNALLY_REPARENTED, "", &rpc.Unused{}, waitTime)
func (client *GoRpcTabletManagerClient) TabletExternallyReparented(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
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
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 sl.Addrs, nil
}
func (client *GoRpcTabletManagerClient) WaitBlpPosition(tablet *topo.TabletInfo, blpPosition blproto.BlpPosition, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_WAIT_BLP_POSITION, &gorpcproto.WaitBlpPositionArgs{
func (client *GoRpcTabletManagerClient) WaitBlpPosition(ctx context.Context, tablet *topo.TabletInfo, blpPosition blproto.BlpPosition, waitTime time.Duration) error {
return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_WAIT_BLP_POSITION, &gorpcproto.WaitBlpPositionArgs{
BlpPosition: blpPosition,
WaitTimeout: 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
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 &bpl, nil
}
func (client *GoRpcTabletManagerClient) StartBlp(tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_START_BLP, "", &rpc.Unused{}, waitTime)
func (client *GoRpcTabletManagerClient) StartBlp(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
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
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,
WaitTimeout: waitTime,
}, &pos, waitTime); err != nil {
@ -259,39 +260,39 @@ func (client *GoRpcTabletManagerClient) RunBlpUntil(tablet *topo.TabletInfo, pos
// Reparenting related functions
//
func (client *GoRpcTabletManagerClient) DemoteMaster(tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_DEMOTE_MASTER, "", &rpc.Unused{}, waitTime)
func (client *GoRpcTabletManagerClient) DemoteMaster(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
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
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 &rsd, nil
}
func (client *GoRpcTabletManagerClient) SlaveWasPromoted(tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_SLAVE_WAS_PROMOTED, "", &rpc.Unused{}, waitTime)
func (client *GoRpcTabletManagerClient) SlaveWasPromoted(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
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 {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_RESTART_SLAVE, rsd, &rpc.Unused{}, waitTime)
func (client *GoRpcTabletManagerClient) RestartSlave(ctx context.Context, tablet *topo.TabletInfo, rsd *actionnode.RestartSlaveData, waitTime time.Duration) error {
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 {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_SLAVE_WAS_RESTARTED, args, &rpc.Unused{}, waitTime)
func (client *GoRpcTabletManagerClient) SlaveWasRestarted(ctx context.Context, tablet *topo.TabletInfo, args *actionnode.SlaveWasRestartedArgs, waitTime time.Duration) error {
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 {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_BREAK_SLAVES, "", &rpc.Unused{}, waitTime)
func (client *GoRpcTabletManagerClient) BreakSlaves(ctx context.Context, tablet *topo.TabletInfo, waitTime time.Duration) error {
return client.rpcCallTablet(ctx, tablet, actionnode.TABLET_ACTION_BREAK_SLAVES, "", &rpc.Unused{}, waitTime)
}
//
// 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)
if err != nil {
return nil, nil, err
@ -319,15 +320,15 @@ func (client *GoRpcTabletManagerClient) Snapshot(tablet *topo.TabletInfo, sa *ac
}, nil
}
func (client *GoRpcTabletManagerClient) SnapshotSourceEnd(tablet *topo.TabletInfo, args *actionnode.SnapshotSourceEndArgs, waitTime time.Duration) error {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_SNAPSHOT_SOURCE_END, args, &rpc.Unused{}, waitTime)
func (client *GoRpcTabletManagerClient) SnapshotSourceEnd(ctx context.Context, tablet *topo.TabletInfo, args *actionnode.SnapshotSourceEndArgs, waitTime time.Duration) error {
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 {
return client.rpcCallTablet(tablet, actionnode.TABLET_ACTION_RESERVE_FOR_RESTORE, args, &rpc.Unused{}, waitTime)
func (client *GoRpcTabletManagerClient) ReserveForRestore(ctx context.Context, tablet *topo.TabletInfo, args *actionnode.ReserveForRestoreArgs, waitTime time.Duration) error {
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)
if err != nil {
return nil, nil, err
@ -341,7 +342,7 @@ func (client *GoRpcTabletManagerClient) Restore(tablet *topo.TabletInfo, sa *act
}, 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)
if err != nil {
return nil, nil, err
@ -369,7 +370,7 @@ func (client *GoRpcTabletManagerClient) MultiSnapshot(tablet *topo.TabletInfo, s
}, 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)
if err != nil {
return nil, nil, err

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

@ -35,28 +35,28 @@ type TabletManager struct {
func (tm *TabletManager) Ping(ctx context.Context, args, reply *string) 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
})
}
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 {
tm.agent.Sleep(*args)
tm.agent.Sleep(ctx, *args)
return nil
})
}
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 {
*reply = *tm.agent.ExecuteHook(args)
*reply = *tm.agent.ExecuteHook(ctx, args)
return nil
})
}
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 {
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 {
*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 {
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 {
*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 {
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 {
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 {
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 {
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 {
return tm.agent.RpcWrapLockAction(ctx, actionnode.TABLET_ACTION_REFRESH_STATE, args, reply, true, func() error {
tm.agent.RefreshState()
tm.agent.RefreshState(ctx)
return nil
})
}
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 {
tm.agent.RunHealthCheck(*args)
tm.agent.RunHealthCheck(ctx, *args)
return nil
})
}
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 {
tm.agent.ReloadSchema()
tm.agent.ReloadSchema(ctx)
return nil
})
}
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 {
scr, err := tm.agent.PreflightSchema(*args)
scr, err := tm.agent.PreflightSchema(ctx, *args)
if err == nil {
*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 {
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 {
*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 {
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 {
*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 {
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 {
*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 {
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 {
*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 {
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 {
*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 {
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 {
*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 {
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 {
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 {
*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 {
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
// reasonnable hard-coded value.
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 {
return tm.agent.RpcWrap(ctx, actionnode.TABLET_ACTION_GET_SLAVES, args, reply, func() error {
var err error
reply.Addrs, err = tm.agent.GetSlaves()
reply.Addrs, err = tm.agent.GetSlaves(ctx)
return err
})
}
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.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 {
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 {
*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 {
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 {
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 {
*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 {
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 {
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 {
*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 {
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 {
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 {
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 {
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()
}()
sr, err := tm.agent.Snapshot(args, logger)
sr, err := tm.agent.Snapshot(ctx, args, logger)
close(logger)
wg.Wait()
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 {
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 {
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()
}()
err := tm.agent.Restore(args, logger)
err := tm.agent.Restore(ctx, args, logger)
close(logger)
wg.Wait()
return err
@ -404,7 +404,7 @@ func (tm *TabletManager) MultiSnapshot(ctx context.Context, args *actionnode.Mul
wg.Done()
}()
sr, err := tm.agent.MultiSnapshot(args, logger)
sr, err := tm.agent.MultiSnapshot(ctx, args, logger)
close(logger)
wg.Wait()
if err != nil {
@ -435,7 +435,7 @@ func (tm *TabletManager) MultiRestore(ctx context.Context, args *actionnode.Mult
wg.Done()
}()
err := tm.agent.MultiRestore(args, logger)
err := tm.agent.MultiRestore(ctx, args, logger)
close(logger)
wg.Wait()
return err

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

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

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

@ -14,6 +14,8 @@ import (
"reflect"
"time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/timer"
"github.com/youtube/vitess/go/vt/health"
@ -264,7 +266,7 @@ func (agent *ActionAgent) rebuildShardIfNeeded(tablet *topo.TabletInfo, targetTa
interrupted := make(chan struct{})
// 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)
}
}

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

@ -8,6 +8,7 @@ import (
"flag"
"time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog"
mproto "github.com/youtube/vitess/go/mysql/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(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(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(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
//
// 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(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(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(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(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(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(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(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(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(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(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(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
//
// 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
// 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(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
// 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(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
// 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(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(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(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
// 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,
// 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(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
// 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
//
// 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(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(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(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(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
// 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
//
// 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(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(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(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(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(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

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

@ -41,7 +41,7 @@ type TabletBson struct {
}
// 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 config *tls.Config
if *tabletBsonEncrypted {
@ -64,7 +64,7 @@ func DialTablet(context context.Context, endPoint topo.EndPoint, keyspace, shard
}
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()
return nil, tabletError(err)
}
@ -73,7 +73,7 @@ func DialTablet(context context.Context, endPoint topo.EndPoint, keyspace, shard
}
// 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()
defer conn.mu.RUnlock()
if conn.rpcClient == nil {
@ -87,14 +87,14 @@ func (conn *TabletBson) Execute(context context.Context, query string, bindVars
SessionId: conn.sessionID,
}
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 qr, nil
}
// 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()
defer conn.mu.RUnlock()
if conn.rpcClient == nil {
@ -107,14 +107,14 @@ func (conn *TabletBson) ExecuteBatch(context context.Context, queries []tproto.B
SessionId: conn.sessionID,
}
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 qrs, nil
}
// 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()
defer conn.mu.RUnlock()
if conn.rpcClient == nil {
@ -145,7 +145,7 @@ func (conn *TabletBson) StreamExecute(context context.Context, query string, bin
}
// 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()
defer conn.mu.RUnlock()
if conn.rpcClient == nil {
@ -156,12 +156,12 @@ func (conn *TabletBson) Begin(context context.Context) (transactionID int64, err
SessionId: conn.sessionID,
}
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)
}
// 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()
defer conn.mu.RUnlock()
if conn.rpcClient == nil {
@ -172,11 +172,11 @@ func (conn *TabletBson) Commit(context context.Context, transactionID int64) err
SessionId: conn.sessionID,
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.
func (conn *TabletBson) Rollback(context context.Context, transactionID int64) error {
func (conn *TabletBson) Rollback(ctx context.Context, transactionID int64) error {
conn.mu.RLock()
defer conn.mu.RUnlock()
if conn.rpcClient == nil {
@ -187,11 +187,11 @@ func (conn *TabletBson) Rollback(context context.Context, transactionID int64) e
SessionId: conn.sessionID,
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
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()
defer conn.mu.RUnlock()
if conn.rpcClient == nil {
@ -203,7 +203,7 @@ func (conn *TabletBson) SplitQuery(context context.Context, query tproto.BoundQu
SplitCount: splitCount,
}
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 reply.Queries, nil

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

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

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

@ -5,8 +5,10 @@
package topo
import (
"code.google.com/p/go.net/context"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/trace"
"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
// 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 {
// not very efficient, but easy to read
links := make([]ReplicationLink, 0, len(sr.ReplicationLinks)+1)

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

@ -11,8 +11,11 @@ import (
"strings"
"sync"
"code.google.com/p/go.net/context"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/trace"
"github.com/youtube/vitess/go/vt/concurrency"
"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
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
if si.version != 0 {
version = si.version

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

@ -12,8 +12,11 @@ import (
"strings"
"sync"
"code.google.com/p/go.net/context"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/jscfg"
"github.com/youtube/vitess/go/trace"
"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.
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
if tablet.version != 0 {
version = tablet.version
@ -628,13 +636,13 @@ func CreateTablet(ts Server, tablet *Tablet) error {
return nil
}
return UpdateTabletReplicationData(ts, tablet)
return UpdateTabletReplicationData(context.TODO(), ts, tablet)
}
// UpdateTabletReplicationData creates or updates the replication
// graph data for a tablet
func UpdateTabletReplicationData(ts Server, tablet *Tablet) error {
return UpdateShardReplicationRecord(ts, tablet.Keyspace, tablet.Shard, tablet.Alias, tablet.Parent)
func UpdateTabletReplicationData(ctx context.Context, ts Server, tablet *Tablet) error {
return UpdateShardReplicationRecord(ctx, ts, tablet.Keyspace, tablet.Shard, tablet.Alias, tablet.Parent)
}
// DeleteTabletReplicationData deletes replication data.

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

@ -8,6 +8,8 @@ import (
"encoding/json"
"testing"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/vt/topo"
)
@ -72,7 +74,7 @@ func CheckShard(t *testing.T, ts topo.Server) {
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)
}

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

@ -9,6 +9,8 @@ import (
"testing"
"time"
"code.google.com/p/go.net/context"
"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
}
func CheckTablet(t *testing.T, ts topo.Server) {
func CheckTablet(ctx context.Context, t *testing.T, ts topo.Server) {
cell := getLocalCell(t, ts)
tablet := &topo.Tablet{
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
if err := topo.UpdateTablet(ts, ti); err != nil {
if err := topo.UpdateTablet(ctx, ts, ti); err != nil {
t.Errorf("UpdateTablet: %v", err)
}

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

@ -10,6 +10,8 @@ import (
"sync"
"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/logutil"
"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
// 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)
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.
shardInfo, err := ts.GetShard(keyspace, shard)
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
// shard in the same cell (e.g. from our peer tablets).
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 {
rec.RecordError(err)
return
@ -98,7 +105,7 @@ func RebuildShard(log logutil.Logger, ts topo.Server, keyspace, shard string, ce
}
// write the data we need to
rebuildErr := rebuildCellSrvShard(log, ts, shardInfo, cell, tablets)
rebuildErr := rebuildCellSrvShard(ctx, log, ts, shardInfo, cell, tablets)
// and unlock
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
// 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)
// 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)
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)
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 {
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()
}(tabletType, addrs)
}
@ -192,9 +203,13 @@ func rebuildCellSrvShard(log logutil.Logger, ts topo.Server, shardInfo *topo.Sha
wg.Add(1)
go func(tabletType topo.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 {
log.Warningf("unable to remove stale db type %v from serving graph: %v", tabletType, err)
}
span.Finish()
wg.Done()
}(tabletType)
}
@ -215,9 +230,15 @@ func rebuildCellSrvShard(log logutil.Logger, ts topo.Server, shardInfo *topo.Sha
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 {
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()
}()

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

@ -9,6 +9,8 @@ import (
"testing"
"time"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/test/faketopo"
@ -19,6 +21,7 @@ import (
)
func TestRebuildShardRace(t *testing.T) {
ctx := context.Background()
cells := []string{"test_cell"}
logger := logutil.NewMemoryLogger()
timeout := 10 * time.Second
@ -35,7 +38,7 @@ func TestRebuildShardRace(t *testing.T) {
f.AddTablet(2, "test_cell", topo.TYPE_REPLICA, master)
// 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)
}
@ -73,11 +76,11 @@ func TestRebuildShardRace(t *testing.T) {
// the SrvShard lock.
masterInfo := f.GetTablet(1)
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)
}
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)
}
close(done)
@ -90,10 +93,10 @@ func TestRebuildShardRace(t *testing.T) {
// that doesn't stall.
replicaInfo := f.GetTablet(2)
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)
}
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)
}

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

@ -11,6 +11,8 @@ package topotools
import (
"sync"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
"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.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)
}
}

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

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

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

@ -8,6 +8,8 @@ import (
"flag"
"fmt"
"code.google.com/p/go.net/context"
_ "github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/wrangler"
)
@ -45,7 +47,7 @@ func commandDemoteMaster(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []s
if err != nil {
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 {

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

@ -15,6 +15,7 @@ import (
"strings"
"time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/flagutil"
"github.com/youtube/vitess/go/jscfg"
@ -779,7 +780,7 @@ func commandSetReadOnly(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []st
if err != nil {
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 {
@ -798,7 +799,7 @@ func commandSetReadWrite(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []s
if err != nil {
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 {
@ -851,7 +852,7 @@ func commandPing(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string) e
if err != nil {
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 {
@ -869,7 +870,7 @@ func commandRefreshState(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []s
if err != nil {
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 {
@ -891,7 +892,7 @@ func commandRunHealthCheck(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args [
if err != nil {
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 {
@ -923,7 +924,7 @@ func commandSleep(wr *wrangler.Wrangler, subFlags *flag.FlagSet, args []string)
if err != nil {
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 {
@ -1236,7 +1237,7 @@ func commandShardExternallyReparented(wr *wrangler.Wrangler, subFlags *flag.Flag
if err != nil {
return err
}
return wr.TabletManagerClient().TabletExternallyReparented(ti, wr.ActionTimeout())
return wr.TabletManagerClient().TabletExternallyReparented(context.TODO(), ti, wr.ActionTimeout())
}
return wr.ShardExternallyReparented(keyspace, shard, tabletAlias)
}
@ -1464,7 +1465,7 @@ func commandShardReplicationAdd(wr *wrangler.Wrangler, subFlags *flag.FlagSet, a
if err != nil {
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 {

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

@ -13,6 +13,8 @@ import (
"text/template"
"time"
"code.google.com/p/go.net/context"
mproto "github.com/youtube/vitess/go/mysql/proto"
"github.com/youtube/vitess/go/sqltypes"
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)
}
_, 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 {
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
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 {
wr.Logger().Infof("Not splitting table %v into multiple chunks: %v", td.Name, err)
return result, nil
@ -295,7 +297,7 @@ func executeFetchLoop(wr *wrangler.Wrangler, ti *topo.TabletInfo, insertChannel
return nil
}
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 {
return fmt.Errorf("ExecuteFetch failed: %v", err)
}

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

@ -10,6 +10,8 @@ import (
"strings"
"time"
"code.google.com/p/go.net/context"
mproto "github.com/youtube/vitess/go/mysql/proto"
rpc "github.com/youtube/vitess/go/rpcplus"
"github.com/youtube/vitess/go/rpcwrap/bsonrpc"
@ -44,7 +46,7 @@ func NewQueryResultReaderForTablet(ts topo.Server, tabletAlias topo.TabletAlias,
}
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
}

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

@ -11,6 +11,8 @@ import (
"sync"
"time"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/event"
"github.com/youtube/vitess/go/sync2"
"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)
}
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)
}
@ -578,7 +580,7 @@ func (scw *SplitCloneWorker) copy() error {
// get the current position from the sources
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 {
return err
}
@ -628,7 +630,7 @@ func (scw *SplitCloneWorker) copy() error {
go func(ti *topo.TabletInfo) {
defer destinationWaitGroup.Done()
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)
}
}(scw.destinationTablets[shardIndex][tabletAlias])

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

@ -10,6 +10,8 @@ import (
"sync"
"time"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/sync2"
blproto "github.com/youtube/vitess/go/vt/binlog/proto"
"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
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 {
return fmt.Errorf("StopBlp for %v failed: %v", sdw.shardInfo.MasterAlias, err)
}
@ -292,7 +294,7 @@ func (sdw *SplitDiffWorker) synchronizeReplication() error {
// stop replication
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 {
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
// 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)
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 {
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 {
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 {
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
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 {
sdw.wr.Logger().Warningf("Cannot find cleaning action %v/%v: %v", wrangler.StartBlpActionName, sdw.shardInfo.MasterAlias.String(), err)
}

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

@ -10,6 +10,8 @@ import (
"sync"
"time"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/wrangler"
)
@ -223,7 +225,7 @@ func (worker *SQLDiffWorker) synchronizeReplication() error {
if err != nil {
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)
}
if worker.CheckInterrupted() {
@ -251,7 +253,7 @@ func (worker *SQLDiffWorker) synchronizeReplication() error {
if err != nil {
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)
}

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

@ -11,6 +11,8 @@ import (
"sync"
"time"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/event"
"github.com/youtube/vitess/go/sync2"
"github.com/youtube/vitess/go/vt/binlog/binlogplayer"
@ -279,7 +281,7 @@ func (vscw *VerticalSplitCloneWorker) findTargets() error {
}
// 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)
}
@ -505,7 +507,7 @@ func (vscw *VerticalSplitCloneWorker) copy() error {
// then create and populate the blp_checkpoint table
if strings.Index(vscw.strategy, "populateBlpCheckpoint") != -1 {
// 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 {
return err
}
@ -551,7 +553,7 @@ func (vscw *VerticalSplitCloneWorker) copy() error {
go func(ti *topo.TabletInfo) {
defer destinationWaitGroup.Done()
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)
}
}(vscw.destinationTablets[tabletAlias])

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

@ -11,6 +11,8 @@ import (
"sync"
"time"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/sync2"
blproto "github.com/youtube/vitess/go/vt/binlog/proto"
"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
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 {
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 {
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 {
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
// 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)
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 {
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 {
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 {
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
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 {
vsdw.wr.Logger().Warningf("Cannot find cleaning action %v/%v: %v", wrangler.StartBlpActionName, vsdw.shardInfo.MasterAlias.String(), err)
}

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

@ -9,6 +9,8 @@ import (
"sync"
"time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/vt/concurrency"
"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.
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.
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"
"sync"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/vt/concurrency"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
"github.com/youtube/vitess/go/vt/topo"
@ -42,7 +44,7 @@ func (wr *Wrangler) Snapshot(tabletAlias topo.TabletAlias, forceMasterSnapshot b
ServerMode: serverMode,
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 {
return nil, "", err
}
@ -67,7 +69,7 @@ func (wr *Wrangler) SnapshotSourceEnd(tabletAlias topo.TabletAlias, slaveStartRe
ReadOnly: !readWrite,
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
@ -86,7 +88,7 @@ func (wr *Wrangler) ReserveForRestore(srcTabletAlias, dstTabletAlias topo.Tablet
args := &actionnode.ReserveForRestoreArgs{
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,
@ -145,7 +147,7 @@ func (wr *Wrangler) Restore(srcTabletAlias topo.TabletAlias, srcFilePath string,
WasReserved: wasReserved,
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 {
return err
}

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

@ -8,6 +8,8 @@ import (
"fmt"
"strings"
"code.google.com/p/go.net/context"
log "github.com/golang/glog"
hk "github.com/youtube/vitess/go/vt/hook"
"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) {
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

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

@ -8,6 +8,7 @@ import (
"fmt"
"sync"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/event"
blproto "github.com/youtube/vitess/go/vt/binlog/proto"
"github.com/youtube/vitess/go/vt/concurrency"
@ -22,7 +23,7 @@ import (
// keyspace related methods for Wrangler
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 {
@ -211,7 +212,7 @@ func (wr *Wrangler) getMastersPosition(shards []*topo.ShardInfo) (map[*topo.Shar
return
}
pos, err := wr.tmc.MasterPosition(ti, wr.ActionTimeout())
pos, err := wr.tmc.MasterPosition(context.TODO(), ti, wr.ActionTimeout())
if err != nil {
rec.RecordError(err)
return
@ -255,7 +256,7 @@ func (wr *Wrangler) waitForFilteredReplication(sourcePositions map[*topo.ShardIn
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)
} else {
wr.Logger().Infof("%v caught up", si.MasterAlias)
@ -282,7 +283,7 @@ func (wr *Wrangler) refreshMasters(shards []*topo.ShardInfo) error {
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)
} else {
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 {
return err
}
if err := topo.UpdateShard(wr.ts, si); err != nil {
if err := topo.UpdateShard(context.TODO(), wr.ts, si); err != nil {
return err
}
}
@ -393,7 +394,7 @@ func (wr *Wrangler) migrateServedTypes(keyspace string, sourceShards, destinatio
// All is good, we can save the shards now
event.DispatchUpdate(ev, "updating source shards")
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
}
}
@ -405,7 +406,7 @@ func (wr *Wrangler) migrateServedTypes(keyspace string, sourceShards, destinatio
}
event.DispatchUpdate(ev, "updating destination shards")
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
}
}
@ -559,7 +560,7 @@ func (wr *Wrangler) replicaMigrateServedFrom(ki *topo.KeyspaceInfo, sourceShard
if err := sourceShard.UpdateSourceBlacklistedTables(servedType, cells, reverse, tables); err != nil {
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)
}
@ -599,26 +600,26 @@ func (wr *Wrangler) masterMigrateServedFrom(ki *topo.KeyspaceInfo, sourceShard *
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)
}
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)
}
// Now refresh the blacklisted table list on the source master
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
}
// get the 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 {
return err
}
// wait for it
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,
Position: masterPosition,
}, wr.ActionTimeout()); err != nil {
@ -634,7 +635,7 @@ func (wr *Wrangler) masterMigrateServedFrom(ki *topo.KeyspaceInfo, sourceShard *
// Update the destination shard (no more source shard)
event.DispatchUpdate(ev, "updating destination shard")
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
}
@ -695,7 +696,7 @@ func (wr *Wrangler) RefreshTablesByShard(si *topo.ShardInfo, tabletType topo.Tab
wg.Add(1)
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)
}
wg.Done()

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

@ -9,6 +9,8 @@ import (
"sort"
"sync"
"code.google.com/p/go.net/context"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/vt/concurrency"
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 wr.tmc.GetPermissions(tablet, wr.ActionTimeout())
return wr.tmc.GetPermissions(context.TODO(), tablet, wr.ActionTimeout())
}
// helper method to asynchronously diff a permissions

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

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

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

@ -67,6 +67,8 @@ On X: (promoted slave)
import (
"fmt"
"code.google.com/p/go.net/context"
myproto "github.com/youtube/vitess/go/vt/mysqlctl/proto"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
"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)
}
status, err := wr.tmc.SlaveStatus(ti, wr.ActionTimeout())
status, err := wr.tmc.SlaveStatus(context.TODO(), ti, wr.ActionTimeout())
if err != nil {
return err
}
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 {
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
// disconnected, hence we have to force this action.
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"
"time"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/vt/hook"
myproto "github.com/youtube/vitess/go/vt/mysqlctl/proto"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
@ -64,7 +65,7 @@ func (wr *Wrangler) checkSlaveReplication(tabletMap map[topo.TabletAlias]*topo.T
return
}
status, err := wr.tmc.SlaveStatus(tablet, wr.ActionTimeout())
status, err := wr.tmc.SlaveStatus(context.TODO(), tablet, wr.ActionTimeout())
if err != nil {
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)
@ -112,7 +113,7 @@ func (wr *Wrangler) checkSlaveConsistency(tabletMap map[uint32]*topo.TabletInfo,
if !masterPosition.IsZero() {
// 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 {
ctx.err = err
return
@ -120,7 +121,7 @@ func (wr *Wrangler) checkSlaveConsistency(tabletMap map[uint32]*topo.TabletInfo,
ctx.status = status
} else {
// 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 {
ctx.err = err
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 {
errs := make(chan error, len(tabletMap))
f := func(ti *topo.TabletInfo) {
err := wr.tmc.StopSlave(ti, wr.ActionTimeout())
err := wr.tmc.StopSlave(context.TODO(), ti, wr.ActionTimeout())
if err != nil {
wr.logger.Infof("StopSlave failed: %v", err)
}
@ -218,13 +219,13 @@ func (wr *Wrangler) tabletReplicationStatuses(tablets []*topo.TabletInfo) ([]*my
ctx := &rpcContext{tablet: ti}
calls[idx] = ctx
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
if err == nil {
ctx.status = &myproto.ReplicationStatus{Position: pos}
}
} 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) {
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 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) {
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) {
@ -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) {
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 {
@ -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)
} else {
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)
}
}
@ -348,7 +349,7 @@ func (wr *Wrangler) finishReparent(si *topo.ShardInfo, masterElect *topo.TabletI
// save the new master in the shard info
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)
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
// out of the graph.
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
}
@ -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
// the replication graph.
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 {

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

@ -7,6 +7,7 @@ package wrangler
import (
"fmt"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/event"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
"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")
wr.logger.Infof("Updating Shard's MasterAlias record")
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
}
// and rebuild the shard serving graph
event.DispatchUpdate(ev, "rebuilding shard serving graph")
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
}
@ -143,16 +144,15 @@ func (wr *Wrangler) reparentShardExternal(ev *events.Reparent, slaveTabletMap, m
// timeout is executed, so even if we got to the timeout,
// we're still good.
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
}
func (wr *Wrangler) slaveWasPromoted(ti *topo.TabletInfo) error {
wr.logger.Infof("slaveWasPromoted(%v)", ti.Alias)
return wr.tmc.SlaveWasPromoted(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())
return wr.tmc.SlaveWasPromoted(context.TODO(), ti, wr.ActionTimeout())
}

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

@ -8,6 +8,8 @@ import (
"fmt"
"strings"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/event"
"github.com/youtube/vitess/go/vt/topo"
"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.
event.DispatchUpdate(ev, "scrapping old master")
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.
wr.logger.Warningf("scrap demoted master failed: %v", scrapErr)
}

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

@ -10,6 +10,8 @@ import (
"strings"
"sync"
"code.google.com/p/go.net/context"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/vt/concurrency"
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 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.
@ -36,7 +38,7 @@ func (wr *Wrangler) ReloadSchema(tabletAlias topo.TabletAlias) error {
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
@ -184,7 +186,7 @@ func (wr *Wrangler) PreflightSchema(tabletAlias topo.TabletAlias, change string)
if err != nil {
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.
@ -193,7 +195,7 @@ func (wr *Wrangler) ApplySchema(tabletAlias topo.TabletAlias, sc *myproto.Schema
if err != nil {
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
@ -285,7 +287,7 @@ func (wr *Wrangler) applySchemaShard(shardInfo *topo.ShardInfo, preflight *mypro
for _, status := range statusArray {
wg.Add(1)
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()
}(status)
}

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

@ -7,6 +7,7 @@ package wrangler
import (
"fmt"
"code.google.com/p/go.net/context"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/vt/key"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
@ -16,7 +17,7 @@ import (
// shard related methods for Wrangler
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 {
@ -70,7 +71,7 @@ func (wr *Wrangler) updateShardCellsAndMaster(si *topo.ShardInfo, tabletAlias to
if wasUpdated {
// 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)
}
}
@ -102,7 +103,7 @@ func (wr *Wrangler) setShardServedTypes(keyspace, shard string, cells []string,
if err := si.UpdateServedTypesMap(servedType, cells, remove); err != nil {
return err
}
return topo.UpdateShard(wr.ts, si)
return topo.UpdateShard(context.TODO(), wr.ts, si)
}
// 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 topo.UpdateShard(wr.ts, shardInfo)
return topo.UpdateShard(context.TODO(), wr.ts, shardInfo)
}
// 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
return topo.UpdateShard(wr.ts, shardInfo)
return topo.UpdateShard(context.TODO(), wr.ts, shardInfo)
}
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
}
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 {
@ -321,5 +322,5 @@ func (wr *Wrangler) sourceShardAdd(keyspace, shard string, uid uint32, skeyspace
KeyRange: keyRange,
Tables: tables,
})
return topo.UpdateShard(wr.ts, si)
return topo.UpdateShard(context.TODO(), wr.ts, si)
}

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

@ -8,6 +8,8 @@ import (
"fmt"
"sync"
"code.google.com/p/go.net/context"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/event"
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
// master.
ti.Tablet.Type = topo.TYPE_BACKUP
err = topo.UpdateTablet(wr.ts, ti)
err = topo.UpdateTablet(context.TODO(), wr.ts, ti)
} else {
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 {
log.Infof("force change type backup -> master: %v", ti.Alias)
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)
@ -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 {
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
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 {
return nil, topo.TabletAlias{}, err
}
@ -243,7 +245,7 @@ func (wr *Wrangler) SetSourceShards(keyspace, shard string, sources []topo.Table
}
// 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
}

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

@ -7,6 +7,7 @@ package wrangler
import (
"fmt"
"code.google.com/p/go.net/context"
log "github.com/golang/glog"
mproto "github.com/youtube/vitess/go/mysql/proto"
"github.com/youtube/vitess/go/vt/tabletmanager/actionnode"
@ -76,7 +77,7 @@ func (wr *Wrangler) InitTablet(tablet *topo.Tablet, force, createShardAndKeyspac
} else {
if oldTablet.Keyspace == tablet.Keyspace && oldTablet.Shard == tablet.Shard {
*(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)
// now fall through the Scrap case
} else {
@ -84,7 +85,7 @@ func (wr *Wrangler) InitTablet(tablet *topo.Tablet, force, createShardAndKeyspac
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)
// now fall through the Scrap case
} else {
@ -126,7 +127,7 @@ func (wr *Wrangler) Scrap(tabletAlias topo.TabletAlias, force, skipRebuild bool)
if force {
err = topotools.Scrap(wr.ts, ti.Alias, force)
} else {
err = wr.tmc.Scrap(ti, wr.ActionTimeout())
err = wr.tmc.Scrap(context.TODO(), ti, wr.ActionTimeout())
}
if err != nil {
return err
@ -160,7 +161,7 @@ func (wr *Wrangler) Scrap(tabletAlias topo.TabletAlias, force, skipRebuild bool)
si.MasterAlias = topo.TabletAlias{}
// 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)
}
} else {
@ -219,7 +220,7 @@ func (wr *Wrangler) ChangeTypeNoRebuild(tabletAlias topo.TabletAlias, tabletType
return false, "", "", "", err
}
} 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
}
}
@ -248,13 +249,13 @@ func (wr *Wrangler) changeTypeInternal(tabletAlias topo.TabletAlias, dbType topo
rebuildRequired := ti.Tablet.IsInServingGraph()
// 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
}
// rebuild if necessary
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 {
return err
}
@ -282,5 +283,5 @@ func (wr *Wrangler) ExecuteFetch(tabletAlias topo.TabletAlias, query string, max
if err != nil {
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"
"time"
"code.google.com/p/go.net/context"
"github.com/youtube/vitess/go/vt/logutil"
_ "github.com/youtube/vitess/go/vt/tabletmanager/gorpctmclient"
"github.com/youtube/vitess/go/vt/topo"
@ -18,6 +20,7 @@ import (
)
func TestShardExternallyReparented(t *testing.T) {
ctx := context.Background()
ts := zktopo.NewTestServer(t, []string{"cell1", "cell2"})
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)
}
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)
}

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

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

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

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

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

@ -10,6 +10,8 @@ import (
"strings"
"time"
"code.google.com/p/go.net/context"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/rpcplus"
"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) {
zkPath := &ZkPath{path}
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 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) {
zkPath := &ZkPath{path}
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 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) {
zkPath := &ZkPath{path}
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 &zkNode.Stat, nil