import packages graduating from the experimental repo

This commit is contained in:
Mike Solomon 2012-07-23 22:56:17 -07:00
Родитель c3c8730983
Коммит 0f2dd36be2
45 изменённых файлов: 8660 добавлений и 0 удалений

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

@ -0,0 +1,67 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package main
import (
"code.google.com/p/vitess.x/go/vt/mysqlctl"
"code.google.com/p/vitess/go/relog"
"flag"
"log"
"os"
)
var port = flag.Int("port", 6612, "vtocc port")
var force = flag.Bool("force", false, "force action")
var mysqlPort = flag.Int("mysql-port", 3306, "mysql port")
var tabletUid = flag.Int("tablet-uid", 41983, "tablet uid")
var keyspace = flag.String("keyspace", "test", "keyspace name")
var logLevel = flag.String("log.level", "WARNING", "set log level")
func main() {
flag.Parse()
logger := relog.New(os.Stderr, "",
log.Ldate|log.Lmicroseconds|log.Lshortfile,
relog.LogNameToLogLevel(*logLevel))
relog.SetLogger(logger)
var vtRepl mysqlctl.VtReplParams
vtRepl.TabletHost = "localhost"
vtRepl.TabletPort = *port
vtRepl.StartKey = "\"\""
vtRepl.EndKey = "\"\""
mycnf := mysqlctl.NewMycnf(uint(*tabletUid), *mysqlPort, *keyspace, vtRepl)
dbaconfig := map[string]interface{}{
"uname": "vt_dba",
"unix_socket": mycnf.SocketPath,
"pass": "",
"dbname": "",
"charset": "utf8",
"host": "",
"port": 0,
}
mysqld := mysqlctl.NewMysqld(mycnf, dbaconfig)
action := flag.Arg(0)
switch action {
case "init":
if mysqlErr := mysqlctl.Init(mysqld); mysqlErr != nil {
log.Fatalf("failed init mysql: %v", mysqlErr)
}
case "shutdown":
if mysqlErr := mysqlctl.Shutdown(mysqld, true); mysqlErr != nil {
log.Fatalf("failed shutdown mysql: %v", mysqlErr)
}
case "start":
if mysqlErr := mysqlctl.Start(mysqld); mysqlErr != nil {
log.Fatalf("failed start mysql: %v", mysqlErr)
}
case "teardown":
if mysqlErr := mysqlctl.Teardown(mysqld, *force); mysqlErr != nil {
log.Fatalf("failed teardown mysql (forced? %v): %v", *force, mysqlErr)
}
default:
log.Fatalf("invalid action: %v", action)
}
}

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

@ -0,0 +1,99 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package main
import (
"expvar"
"flag"
"fmt"
"log"
"net/http"
_ "net/http/pprof"
"net/rpc"
"os"
"code.google.com/p/vitess.x/go/zk"
"code.google.com/p/vitess/go/relog"
"code.google.com/p/vitess/go/rpcwrap/bsonrpc"
"code.google.com/p/vitess/go/rpcwrap/jsonrpc"
_ "code.google.com/p/vitess/go/snitch"
"code.google.com/p/vitess.x/go/vt/mysqlctl"
"code.google.com/p/vitess.x/go/vt/tabletmanager"
)
var port = flag.Int("port", 0, "port for debug http server")
var action = flag.String("action", "", "management action to perform")
var actionNode = flag.String("action-node", "",
"path to zk node representing the action")
var actionGuid = flag.String("action-guid", "",
"a label to help track processes")
var logLevel = flag.String("log.level", "debug", "set log level")
var logFilename = flag.String("logfile", "/dev/stderr", "log path")
// FIXME(msolomon) temporary, until we are starting mysql ourselves
var mycnfPath = flag.String("mycnf-path", "/etc/my.cnf", "path to my.cnf")
func init() {
expvar.NewString("binary-name").Set("vtaction")
}
func main() {
flag.Parse()
rpc.HandleHTTP()
jsonrpc.ServeHTTP()
jsonrpc.ServeRPC()
bsonrpc.ServeHTTP()
bsonrpc.ServeRPC()
logFile, err := os.OpenFile(*logFilename,
os.O_APPEND|os.O_WRONLY|os.O_CREATE, 0666)
if err != nil {
panic(err)
}
logger := relog.New(logFile, fmt.Sprintf("vtaction [%v] ", os.Getpid()),
log.Ldate|log.Lmicroseconds|log.Lshortfile,
relog.LogNameToLogLevel(*logLevel))
relog.SetLogger(logger)
relog.Info("started vtaction %v", os.Args)
// we delegate out startup to the micromanagement server so these actions
// will occur after we have obtained our socket.
bindAddr := fmt.Sprintf(":%v", *port)
httpServer := &http.Server{Addr: bindAddr}
go func() {
if err := httpServer.ListenAndServe(); err != nil {
relog.Error("httpServer.ListenAndServe err: %v", err)
}
}()
mycnf, mycnfErr := mysqlctl.ReadMycnf(*mycnfPath)
if mycnfErr != nil {
relog.Error("mycnf read failed: %v", mycnfErr)
return
}
dbaconfig := map[string]interface{}{
"uname": "vt_dba",
"unix_socket": mycnf.SocketPath,
"pass": "",
"dbname": "",
"charset": "utf8",
"host": "",
"port": 0,
}
mysqld := mysqlctl.NewMysqld(mycnf, dbaconfig)
zconn := zk.NewMetaConn(5e9)
defer zconn.Close()
actor := tabletmanager.NewTabletActor(mysqld, zconn)
actionErr := actor.HandleAction(*actionNode, *action, *actionGuid)
if actionErr != nil {
relog.Fatal("action error: %v", actionErr)
}
relog.Info("finished vtaction %v", os.Args)
}

554
go/cmd/vtctl/vtctl.go Normal file
Просмотреть файл

@ -0,0 +1,554 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package main
import (
"bufio"
"flag"
"fmt"
"log"
"os"
"path"
"sort"
"strconv"
"strings"
"sync"
"time"
"code.google.com/p/vitess.x/go/vt/naming"
tm "code.google.com/p/vitess.x/go/vt/tabletmanager"
wr "code.google.com/p/vitess.x/go/vt/wrangler"
"code.google.com/p/vitess.x/go/zk"
"code.google.com/p/vitess/go/relog"
"launchpad.net/gozk/zookeeper"
)
var usage = `
Commands:
Tablets:
InitTablet <zk tablet path> <hostname> <mysql port> <vt port> <keyspace> <shard id> <tablet type> <zk parent alias>
ScrapTablet <zk tablet path>
-force writes the scrap state in to zk, no questions asked, if a tablet is offline.
SetReadOnly [<zk tablet path> | <zk shard/tablet path>]
SetReadWrite [<zk tablet path> | <zk shard/tablet path>]
DemoteMaster <zk tablet path>
ChangeType <zk tablet path> <db type>
Change the db type for this tablet if possible. this is mostly for arranging
replicas - it will not convert a master.
NOTE: This will automatically update the serving graph.
Ping <zk tablet path>
check that the agent is awake and responding - can be blocked by other in-flight
operations.
Shards:
RebuildShard <zk shard path>
rebuild the shard, this may trigger an update to all connected clients
ReparentShard <zk shard path> <zk tablet path>
specify which shard to reparent and which tablet should be the new master
Generic:
PurgeActions <zk action path>
remove all actions - be careful, this is powerful cleanup magic
WaitForAction <zk action path>
watch an action node, printing updates, until the action is complete
Resolve <keyspace>.<shard>.<db type>
read a list of addresses that can answer this query
Validate <zk vt path>
validate that all nodes in this cell are consistent with the global replication graph
ExportZkns <zk vt path>
export the serving graph entries to the legacy zkns format
ListIdle <zk vt path>
list all idle tablet paths
ListScrap <zk vt path>
list all scrap tablet paths
`
var noWaitForAction = flag.Bool("no-wait", false,
"don't wait for action completion, detach")
var waitTime = flag.Duration("wait-time", 24*time.Hour, "time to wait on an action")
var force = flag.Bool("force", false, "force action")
var verbose = flag.Bool("verbose", false, "verbose logging")
var pingTablets = flag.Bool("ping-tablets", false, "ping all tablets during validate")
var logLevel = flag.String("log.level", "WARNING", "set log level")
var stdin *bufio.Reader
func init() {
flag.Usage = func() {
fmt.Fprintf(os.Stderr, "Usage of %s:\n", os.Args[0])
flag.PrintDefaults()
fmt.Fprintf(os.Stderr, usage)
}
stdin = bufio.NewReader(os.Stdin)
}
func confirm(prompt string) bool {
if *force {
return true
}
fmt.Fprintf(os.Stderr, prompt+" [NO/yes] ")
line, _ := stdin.ReadString('\n')
return strings.ToLower(strings.TrimSpace(line)) == "yes"
}
func initTablet(zconn zk.Conn, path, hostname, mysqlPort, vtPort, keyspace, shardId, tabletType, parentAlias string, update bool) error {
tm.MustBeTabletPath(path)
pathParts := strings.Split(path, "/")
cell := zk.ZkCellFromZkPath(path)
uid, err := strconv.Atoi(pathParts[len(pathParts)-1])
if err != nil {
panic(err)
}
parent := tm.TabletAlias{}
if parentAlias != "" {
parent.Cell, parent.Uid = tm.ParseTabletReplicationPath(parentAlias)
}
tablet := tm.NewTablet(cell, uint(uid), parent, fmt.Sprintf("%v:%v", hostname, vtPort), fmt.Sprintf("%v:%v", hostname, mysqlPort), keyspace, shardId, tm.TabletType(tabletType))
err = tm.CreateTablet(zconn, path, tablet)
if err != nil {
if zkErr, ok := err.(*zookeeper.Error); ok && zkErr.Code == zookeeper.ZNODEEXISTS {
if update {
oldTablet, err := tm.ReadTablet(zconn, path)
if err != nil {
relog.Warning("failed reading tablet %v: %v", path, err)
} else {
if oldTablet.Keyspace == tablet.Keyspace && oldTablet.Shard == tablet.Shard {
*(oldTablet.Tablet) = *tablet
err := tm.UpdateTablet(zconn, path, oldTablet)
if err != nil {
relog.Warning("failed reading tablet %v: %v", path, err)
} else {
return nil
}
}
}
}
if *force {
zk.DeleteRecursive(zconn, path, -1)
err = tm.CreateTablet(zconn, path, tablet)
}
}
}
return err
}
func purgeActions(zconn zk.Conn, zkActionPath string) error {
if path.Base(zkActionPath) != "action" {
panic(fmt.Errorf("not action path: %v", zkActionPath))
}
children, _, err := zconn.Children(zkActionPath)
if err != nil {
return err
}
for _, child := range children {
err = zk.DeleteRecursive(zconn, path.Join(zkActionPath, child), -1)
if err != nil {
return err
}
}
return nil
}
func changeType(zconn zk.Conn, ai *tm.ActionInitiator, zkTabletPath, dbType string) error {
if *force {
return tm.ChangeType(zconn, zkTabletPath, tm.TabletType(dbType))
} else {
actionPath, err := ai.ChangeType(zkTabletPath, tm.TabletType(dbType))
if err != nil {
return err
}
// You don't have a choice - you must wait for completion before rebuilding.
err = ai.WaitForCompletion(actionPath, *waitTime)
if err != nil {
return err
}
}
tabletInfo, err := tm.ReadTablet(zconn, zkTabletPath)
if err != nil {
relog.Warning("%v: %v", zkTabletPath, err)
}
err = tm.RebuildShard(zconn, tabletInfo.ShardPath())
return err
}
func getTabletMap(zconn zk.Conn, tabletPaths []string) map[string]*tm.TabletInfo {
wg := sync.WaitGroup{}
mutex := sync.Mutex{}
tabletMap := make(map[string]*tm.TabletInfo)
for _, path := range tabletPaths {
tabletPath := path
wg.Add(1)
go func() {
tabletInfo, err := tm.ReadTablet(zconn, tabletPath)
if err != nil {
relog.Warning("%v: %v", tabletPath, err)
} else {
mutex.Lock()
tabletMap[tabletPath] = tabletInfo
mutex.Unlock()
}
wg.Done()
}()
}
wg.Wait()
mutex.Lock()
defer mutex.Unlock()
return tabletMap
}
func listScrap(zconn zk.Conn, zkVtPath string) error {
zkTabletsPath := path.Join(zkVtPath, "tablets")
children, _, err := zconn.Children(zkTabletsPath)
if err != nil {
return err
}
sort.Strings(children)
tabletPaths := make([]string, len(children))
for i, child := range children {
tabletPaths[i] = path.Join(zkTabletsPath, child)
}
tabletMap := getTabletMap(zconn, tabletPaths)
for _, tabletPath := range tabletPaths {
tabletInfo, ok := tabletMap[tabletPath]
if ok && tabletInfo.Type == tm.TYPE_SCRAP {
fmt.Println(tabletPath)
}
}
return nil
}
func listIdle(zconn zk.Conn, zkVtPath string) error {
zkTabletsPath := path.Join(zkVtPath, "tablets")
children, _, err := zconn.Children(zkTabletsPath)
if err != nil {
return err
}
sort.Strings(children)
tabletPaths := make([]string, len(children))
for i, child := range children {
tabletPaths[i] = path.Join(zkTabletsPath, child)
}
tabletMap := getTabletMap(zconn, tabletPaths)
for _, tabletPath := range tabletPaths {
tabletInfo, ok := tabletMap[tabletPath]
if ok && tabletInfo.Type == tm.TYPE_IDLE {
fmt.Println(tabletPath)
}
}
return nil
}
func validateZk(zconn zk.Conn, ai *tm.ActionInitiator, zkVtPath string) error {
// FIXME(msolomon) validate the replication view
zkTabletsPath := path.Join(zkVtPath, "tablets")
tabletUids, _, err := zconn.Children(zkTabletsPath)
if err != nil {
return err
}
someErrors := false
for _, tabletUid := range tabletUids {
tabletPath := path.Join(zkTabletsPath, tabletUid)
relog.Info("checking tablet %v", tabletPath)
err = tm.Validate(zconn, tabletPath, "")
if err != nil {
someErrors = true
relog.Error("%v: %v", tabletPath, err)
}
}
zkKeyspacesPath := path.Join("/zk/global/vt/keyspaces")
keyspaces, _, err := zconn.Children(zkKeyspacesPath)
if err != nil {
return err
}
for _, keyspace := range keyspaces {
zkShardsPath := path.Join(zkKeyspacesPath, keyspace, "shards")
shards, _, err := zconn.Children(zkShardsPath)
if err != nil {
return err
}
for _, shard := range shards {
zkShardPath := path.Join(zkShardsPath, shard)
shardInfo, err := tm.ReadShard(zconn, zkShardPath)
if err != nil {
return err
}
aliases, err := tm.FindAllTabletAliasesInShard(zconn, shardInfo)
if err != nil {
return err
}
var masterAlias tm.TabletAlias
shardTablets := make([]string, 0, 16)
for _, alias := range aliases {
shardTablets = append(shardTablets, tm.TabletPathForAlias(alias))
}
tabletMap := getTabletMap(zconn, shardTablets)
for _, alias := range aliases {
zkTabletPath := tm.TabletPathForAlias(alias)
tabletInfo, ok := tabletMap[zkTabletPath]
if !ok {
continue
}
if tabletInfo.Parent.Uid == tm.NO_TABLET {
if masterAlias.Cell != "" {
someErrors = true
relog.Error("%v: already have a master %v", zkTabletPath, masterAlias)
} else {
masterAlias = alias
}
}
}
// Need the master for this loop.
for _, alias := range aliases {
zkTabletPath := tm.TabletPathForAlias(alias)
zkTabletReplicationPath := zkShardPath + "/" + masterAlias.String()
if alias != masterAlias {
zkTabletReplicationPath += "/" + alias.String()
}
err = tm.Validate(zconn, zkTabletPath, zkTabletReplicationPath)
if err != nil {
someErrors = true
relog.Error("%v: %v", zkTabletReplicationPath, err)
}
}
if !*pingTablets {
continue
}
pingPaths := make([]string, 0, 128)
for _, alias := range aliases {
zkTabletPath := tm.TabletPathForAlias(alias)
tabletInfo := tabletMap[zkTabletPath]
zkTabletPid := path.Join(zkTabletPath, "pid")
_, _, err := zconn.Get(zkTabletPid)
if err != nil {
someErrors = true
relog.Error("no pid node %v: %v %v", zkTabletPid, err, tabletInfo.Hostname())
continue
}
actionPath, err := ai.Ping(zkTabletPath)
if err != nil {
someErrors = true
relog.Error("%v: %v %v", actionPath, err, tabletInfo.Hostname())
} else {
pingPaths = append(pingPaths, actionPath)
}
}
// FIXME(msolomon) this should be parallel
for _, actionPath := range pingPaths {
err := ai.WaitForCompletion(actionPath, *waitTime)
if err != nil {
someErrors = true
relog.Error("%v: %v", actionPath, err)
}
}
}
}
if someErrors {
return fmt.Errorf("some validation errors - see log")
}
return nil
}
func main() {
defer func() {
if panicErr := recover(); panicErr != nil {
relog.Fatal("%v", relog.NewPanicError(panicErr.(error)).String())
}
}()
flag.Parse()
args := flag.Args()
if len(args) == 0 {
flag.Usage()
os.Exit(1)
}
logger := relog.New(os.Stderr, "vtctl ",
log.Ldate|log.Lmicroseconds|log.Lshortfile,
relog.LogNameToLogLevel(*logLevel))
relog.SetLogger(logger)
zconn := zk.NewMetaConn(5e9)
defer zconn.Close()
ai := tm.NewActionInitiator(zconn)
wrangler := wr.NewWrangler(zconn, ai)
var actionPath string
var err error
switch args[0] {
case "InitTablet":
if len(args) != 9 {
relog.Fatal("action %v requires 8 args", args[0])
}
err = initTablet(zconn, args[1], args[2], args[3], args[4], args[5], args[6], args[7], args[8], false)
case "UpdateTablet":
if len(args) != 9 {
relog.Fatal("action %v requires 8 args", args[0])
}
err = initTablet(zconn, args[1], args[2], args[3], args[4], args[5], args[6], args[7], args[8], true)
case "Ping":
if len(args) != 2 {
relog.Fatal("action %v requires args", args[0])
}
actionPath, err = ai.Ping(args[1])
case tm.TABLET_ACTION_SET_RDONLY:
if len(args) != 2 {
relog.Fatal("action %v requires args", args[0])
}
actionPath, err = ai.SetReadOnly(args[1])
case tm.TABLET_ACTION_SET_RDWR:
if len(args) != 2 {
relog.Fatal("action %v requires args", args[0])
}
actionPath, err = ai.SetReadWrite(args[1])
case "ChangeType":
if len(args) != 3 {
relog.Fatal("action %v requires <zk tablet path> <db type>", args[0])
}
err = changeType(zconn, ai, args[1], args[2])
case "DemoteMaster":
if len(args) != 2 {
relog.Fatal("action %v requires <zk tablet path>", args[0])
}
actionPath, err = ai.DemoteMaster(args[1])
case "PurgeActions":
if len(args) != 2 {
relog.Fatal("action %v requires <zk shard path>", args[0])
}
err = purgeActions(zconn, args[1])
case "RebuildShard":
if len(args) != 2 {
relog.Fatal("action %v requires <zk shard path>", args[0])
}
err = tm.RebuildShard(zconn, args[1])
case "ReparentShard":
if len(args) != 3 {
relog.Fatal("action %v requires <zk shard path> <zk tablet path>", args[0])
}
actionPath, err = wrangler.ReparentShard(args[1], args[2], *force)
case "ExportZkns":
if len(args) != 2 {
relog.Fatal("action %v requires <zk vt root path>", args[0])
}
err = exportZkns(zconn, args[1])
case "Resolve":
if len(args) != 2 {
relog.Fatal("action %v requires <keyspace>.<shard>.<db type>:<port name>", args[0])
}
parts := strings.Split(args[1], ":")
if len(parts) != 2 {
relog.Fatal("action %v requires <keyspace>.<shard>.<db type>:<port name>", args[0])
}
namedPort := parts[1]
parts = strings.Split(parts[0], ".")
if len(parts) != 3 {
relog.Fatal("action %v requires <keyspace>.<shard>.<db type>:<port name>", args[0])
}
addrs, lookupErr := naming.LookupVtName(zconn, "", parts[0], parts[1], parts[2], namedPort)
if lookupErr == nil {
for _, addr := range addrs {
fmt.Printf("%v:%v\n", addr.Target, addr.Port)
}
} else {
err = lookupErr
}
case "ScrapTablet":
if len(args) != 2 {
relog.Fatal("action %v requires <zk tablet path>", args[0])
}
if *force {
err = tm.Scrap(zconn, args[1], *force)
} else {
actionPath, err = ai.Scrap(args[1])
}
case "Validate":
if len(args) != 2 {
relog.Fatal("action %v requires <zk vt path>", args[0])
}
err = validateZk(zconn, ai, args[1])
case "ListScrap":
if len(args) != 2 {
relog.Fatal("action %v requires <zk vt path>", args[0])
}
err = listScrap(zconn, args[1])
case "ListIdle":
if len(args) != 2 {
relog.Fatal("action %v requires <zk vt path>", args[0])
}
err = listIdle(zconn, args[1])
case "WaitForAction":
if len(args) != 2 {
relog.Fatal("action %v requires <zk action path>", args[0])
}
actionPath = args[1]
default:
fmt.Fprintf(os.Stderr, "Unknown command %#v\n\n", args[0])
flag.Usage()
os.Exit(1)
}
if err != nil {
relog.Fatal("action failed: %v %v", args[0], err)
}
if actionPath != "" {
relog.Info("action created: %v", actionPath)
if !*noWaitForAction {
err := ai.WaitForCompletion(actionPath, *waitTime)
if err != nil {
relog.Fatal(err.Error())
} else {
relog.Info("action completed: %v", actionPath)
}
}
}
}

87
go/cmd/vtctl/zkns.go Normal file
Просмотреть файл

@ -0,0 +1,87 @@
package main
import (
"encoding/json"
"fmt"
"path"
"code.google.com/p/vitess.x/go/vt/naming"
"code.google.com/p/vitess.x/go/zk"
"code.google.com/p/vitess.x/go/zk/zkns"
"launchpad.net/gozk/zookeeper"
)
/*
Export addresses from the VT serving graph to a legacy zkns server.
*/
func exportZkns(zconn zk.Conn, zkVtRoot string) error {
vtNsPath := path.Join(zkVtRoot, "ns")
zkCell := zk.ZkCellFromZkPath(zkVtRoot)
zknsRootPath := fmt.Sprintf("/zk/%v/zkns/vt", zkCell)
children, err := zk.ChildrenRecursive(zconn, vtNsPath)
if err != nil { return err }
for _, child := range children {
addrPath := path.Join(vtNsPath, child)
_, stat, err := zconn.Get(addrPath)
if err != nil {
return err
}
if stat.NumChildren() > 0 {
continue
}
addrs, err := naming.ReadAddrs(zconn, addrPath)
if err != nil { return err }
vtoccAddrs := LegacyZknsAddrs{make([]string, 0, 8)}
defaultAddrs := LegacyZknsAddrs{make([]string, 0, 8)}
// Write the individual endpoints
for i, entry := range addrs.Entries {
zknsAddrPath := fmt.Sprintf("%v/%v/%v", zknsRootPath, child, i)
zknsAddr := zkns.ZknsAddr{Host:entry.Host, Port:entry.NamedPortMap["_mysql"], NamedPortMap:entry.NamedPortMap}
err := WriteAddr(zconn, zknsAddrPath, &zknsAddr)
if err != nil { return err }
defaultAddrs.Endpoints = append(defaultAddrs.Endpoints, zknsAddrPath)
vtoccAddrs.Endpoints = append(vtoccAddrs.Endpoints, zknsAddrPath + ":_vtocc")
}
// Write the VDNS entries for both vtocc and mysql
vtoccVdnsPath := fmt.Sprintf("%v/%v/_vtocc.vdns", zknsRootPath, child)
err = WriteAddrs(zconn, vtoccVdnsPath, &vtoccAddrs)
if err != nil { return err }
defaultVdnsPath := fmt.Sprintf("%v/%v.vdns", zknsRootPath, child)
err = WriteAddrs(zconn, defaultVdnsPath, &defaultAddrs)
if err != nil { return err }
}
return nil
}
type LegacyZknsAddrs struct {
Endpoints []string `json:"endpoints"`
}
func toJson(x interface{}) string {
data, err := json.MarshalIndent(x, "", " ")
if err != nil {
panic(err)
}
return string(data)
}
func WriteAddr(zconn zk.Conn, zkPath string, addr *zkns.ZknsAddr) error {
data := toJson(addr)
_, err := zk.CreateOrUpdate(zconn, zkPath, data, 0, zookeeper.WorldACL(zookeeper.PERM_ALL), true)
return err
}
func WriteAddrs(zconn zk.Conn, zkPath string, addrs *LegacyZknsAddrs) error {
data := toJson(addrs)
_, err := zk.CreateOrUpdate(zconn, zkPath, data, 0, zookeeper.WorldACL(zookeeper.PERM_ALL), true)
return err
}

13
go/cmd/vttablet/Makefile Normal file
Просмотреть файл

@ -0,0 +1,13 @@
# Copyright 2012, Google Inc. All rights reserved.
# Use of this source code is governed by a BSD-style license that can
# be found in the LICENSE file.
MAKEFLAGS = -s
all:
cd $(GOTOP)/vt/sqlparser; $(MAKE)
go build
clean:
go clean
cd $(GOTOP)/vt/sqlparser; $(MAKE) clean

121
go/cmd/vttablet/vttablet.go Normal file
Просмотреть файл

@ -0,0 +1,121 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
// vt tablet server: Serves queries and performs housekeeping jobs.
package main
import (
"flag"
"fmt"
_ "net/http/pprof"
"net/rpc"
"syscall"
"code.google.com/p/vitess.x/go/vt/mysqlctl"
"code.google.com/p/vitess.x/go/vt/tabletmanager"
"code.google.com/p/vitess.x/go/zk"
"code.google.com/p/vitess/go/relog"
"code.google.com/p/vitess/go/rpcwrap/bsonrpc"
"code.google.com/p/vitess/go/rpcwrap/jsonrpc"
"code.google.com/p/vitess/go/sighandler"
_ "code.google.com/p/vitess/go/snitch"
"code.google.com/p/vitess/go/vt/servenv"
//ts "code.google.com/p/vitess/go/vt/tabletserver"
"code.google.com/p/vitess/go/umgmt"
)
const (
DefaultLameDuckPeriod = 30.0
DefaultRebindDelay = 0.0
)
var (
port = flag.Int("port", 0, "port for the http server")
lameDuckPeriod = flag.Float64("lame-duck-period", DefaultLameDuckPeriod,
"how long to give in-flight transactions to finish")
rebindDelay = flag.Float64("rebind-delay", DefaultRebindDelay,
"artificial delay before rebinding a hijacked listener")
tabletPath = flag.String("tablet-path", "",
"path to zk node representing the tablet")
)
func main() {
flag.Parse()
if *port == 0 {
panic("no -port supplied")
}
env.Init("vttablet")
// FIXME: (sougou) Integrate tabletserver initialization with zk
//config, dbconfig := ts.Init()
//ts.StartQueryService(config)
//ts.AllowQueries(dbconfig)
rpc.HandleHTTP()
jsonrpc.ServeHTTP()
jsonrpc.ServeRPC()
bsonrpc.ServeHTTP()
bsonrpc.ServeRPC()
zconn := zk.NewMetaConn(5e9)
defer zconn.Close()
bindAddr := fmt.Sprintf(":%v", *port)
// Action agent listens to changes in zookeeper and makes modifcations to this
// tablet.
mysqlAddr := "fixme:3306"
agent := tabletmanager.NewActionAgent(zconn, *tabletPath)
agent.Start(bindAddr, mysqlAddr)
relog.Info("mycnf: %v", agent.MycnfPath)
mycnf, mycnfErr := mysqlctl.ReadMycnf(agent.MycnfPath)
if mycnfErr != nil {
relog.Error("mycnf read failed: %v", mycnfErr)
return
}
dbaconfig := map[string]interface{}{
"uname": "vt_dba",
"unix_socket": mycnf.SocketPath,
"pass": "",
"dbname": "",
"charset": "utf8",
"host": "",
"port": 0,
}
mysqld := mysqlctl.NewMysqld(mycnf, dbaconfig)
// The TabletManager rpc service allow other processes to query for management
// related data. It might be co-registered with the query server.
tm := tabletmanager.NewTabletManager(bindAddr, nil, mysqld)
rpc.Register(tm)
// we delegate out startup to the micromanagement server so these actions
// will occur after we have obtained our socket.
/*usefulLameDuckPeriod := float64(config.QueryTimeout + 1)
if usefulLameDuckPeriod > *lameDuckPeriod {
*lameDuckPeriod = usefulLameDuckPeriod
relog.Info("readjusted -lame-duck-period to %f", *lameDuckPeriod)
}*/
umgmt.SetLameDuckPeriod(float32(*lameDuckPeriod))
umgmt.SetRebindDelay(float32(*rebindDelay))
umgmt.AddStartupCallback(func() {
umgmt.StartHttpServer(fmt.Sprintf(":%v", *port))
})
umgmt.AddStartupCallback(func() {
sighandler.SetSignalHandler(syscall.SIGTERM, umgmt.SigTermHandler)
})
/*umgmt.AddCloseCallback(func() {
ts.DisallowQueries()
})*/
relog.Info("started vttablet %v", *port)
umgmtSocket := fmt.Sprintf("/tmp/vttablet-%08x-umgmt.sock", *port)
if umgmtErr := umgmt.ListenAndServe(umgmtSocket); umgmtErr != nil {
relog.Error("umgmt.ListenAndServe err: %v", umgmtErr)
}
relog.Info("done")
}

818
go/cmd/zk/zkcmd.go Normal file
Просмотреть файл

@ -0,0 +1,818 @@
package main
import (
"archive/zip"
"fmt"
"io/ioutil"
"log"
"os"
"os/exec"
"os/signal"
"path"
"sort"
"strings"
"sync"
"syscall"
"time"
opts "code.google.com/p/opts-go"
"code.google.com/p/vitess.x/go/zk"
"launchpad.net/gozk/zookeeper"
)
var zkAddrs = opts.LongSingle("--zk.addrs",
"list of zookeeper servers (server1:port1,server2:port2,...)",
"")
var longListing = opts.ShortFlag("-l", "long listing")
var directoryListing = opts.ShortFlag("-d", "list directory instead of contents")
var force = opts.ShortFlag("-f", "no warning on nonexistent node")
var recursiveDelete = opts.ShortFlag("-r", "recursive delete")
var recursiveListing = opts.ShortFlag("-R", "recursive listing")
var createParents = opts.ShortFlag("-p", "create parents")
var touchOnly = opts.ShortFlag("-c", "touch only - don't create")
var doc = `zk - a tool for wrangling the zookeeper
This mimics unix file system commands wherever possible.
zk -h - provide help on overriding cell selection
zk cat /zk/path
zk chmod n-mode /zk/path
zk chmod n+mode /zk/path
zk cp /zk/path .
zk cp ./config /zk/path/config
zk cp ./config /zk/path/ (trailing slash indicates directory)
zk edit /zk/path (create a local copy, edit and save to quorum)
zk elock /zk/path (create an ephemeral node that lives as long as the process)
zk ls /zk
zk ls -l /zk
zk ls -ld /zk (list directory node itself)
zk ls -R /zk (recursive, expensive)
zk stat /zk/path
zk touch /zk/path
zk touch -c /zk/path (don't create, just touch timestamp)
zk touch -p /zk/path (create all parts necessary, think mkdir -p)
NOTE: there is no mkdir - just touch a node. The distinction
between file and directory is just not relevant in zookeeper.
zk rm /zk/path
zk rm -r /zk/path (recursive)
zk rm -f /zk/path (no error on nonexistent node)
zk wait /zk/path (wait for node change or creation)
zk wait /zk/path/children/ (trailing slash waits on children)
zk watch /zk/path (print changes)
zk unzip zktree.zip /
zk unzip zktree.zip /zk/prefix
zk zip /zk/root zktree.zip
NOTE: zip file can't be dumped to the file system since znodes
can have data and children.
The zk tool looks for the address of the cluster in /etc/zookeeper/zk_client.conf,
or the file specified in the ZK_CLIENT_CONFIG environment variable.
The local cell may be overridden with the ZK_CLIENT_LOCAL_CELL environment
variable.
--zk.addrs can override the value in the conf file.
`
const (
timeFmt = "2006-01-02 15:04:05"
timeFmtMicro = "2006-01-02 15:04:05.000000"
)
type cmdFunc func(args []string)
var cmdMap map[string]cmdFunc
var zconn zk.Conn
func init() {
opts.Description = doc
cmdMap = map[string]cmdFunc{
"cat": cmdCat,
"chmod": cmdChmod,
"cp": cmdCp,
"edit": cmdEdit,
"elock": cmdElock,
"ls": cmdLs,
"qlock": cmdQlock,
"rm": cmdRm,
"stat": cmdStat,
"touch": cmdTouch,
"unzip": cmdUnzip,
"wait": cmdWait,
"watch": cmdWatch,
"zip": cmdZip,
}
log.SetFlags(0)
zconn = zk.NewMetaConn(5 * time.Second)
}
func main() {
opts.Parse()
args := opts.Args
if len(args) == 0 {
opts.Help()
os.Exit(1)
}
if *zkAddrs != "" {
zc, session, err := zookeeper.Dial(*zkAddrs, 5*time.Second)
if err == nil {
// Wait for connection.
event := <-session
if event.State != zookeeper.STATE_CONNECTED {
panic(fmt.Errorf("zk connect failed: %v", event.State))
}
}
zconn = zc
}
cmdName := args[0]
args = args[1:]
if cmd, ok := cmdMap[cmdName]; ok {
cmd(args)
} else {
opts.Help()
}
}
func fixZkPath(zkPath string) string {
if zkPath != "/" {
zkPath = strings.TrimRight(zkPath, "/")
}
return path.Clean(zkPath)
}
func isZkFile(path string) bool {
return strings.HasPrefix(path, "/zk")
}
func cmdWait(args []string) {
zkPath := args[0]
isDir := zkPath[len(zkPath)-1] == '/'
zkPath = fixZkPath(zkPath)
var wait <-chan zookeeper.Event
var err error
if isDir {
_, _, wait, err = zconn.ChildrenW(zkPath)
} else {
_, _, wait, err = zconn.GetW(zkPath)
}
if err != nil {
if err.(*zookeeper.Error).Code == zookeeper.ZNONODE {
_, wait, err = zconn.ExistsW(zkPath)
} else {
log.Fatalf("wait: error %v: %v", zkPath, err)
}
}
event := <-wait
fmt.Printf("event: %v\n", event)
}
func cmdQlock(args []string) {
zkPath := fixZkPath(args[0])
locked, err := zk.ObtainQueueLock(zconn, zkPath, false)
if err != nil {
log.Fatalf("qlock: error %v: %v", zkPath, err)
}
fmt.Printf("qlock: %v %v\n", zkPath, locked)
}
// Create an ephemeral node an just wait.
func cmdElock(args []string) {
zkPath := fixZkPath(args[0])
// Speed up case where we die nicely, otherwise you have to wait for
// the server to notice the client's demise.
sigRecv := make(chan os.Signal, 1)
signal.Notify(sigRecv, os.Interrupt)
for {
_, err := zconn.Create(zkPath, "", zookeeper.EPHEMERAL, zookeeper.WorldACL(zookeeper.PERM_ALL))
if err != nil {
log.Fatalf("elock: error %v: %v", zkPath, err)
}
watchLoop:
for {
_, _, watch, err := zconn.GetW(zkPath)
if err != nil {
log.Fatalf("elock: error %v: %v", zkPath, err)
}
select {
case <-sigRecv:
zconn.Delete(zkPath, -1)
return
case event := <-watch:
log.Printf("elock: event %v: %v", zkPath, event)
if !event.Ok() {
//log.Fatalf("elock: error %v: %v", zkPath, event)
break watchLoop
}
}
}
}
}
// Watch for changes to the node.
func cmdWatch(args []string) {
// Speed up case where we die nicely, otherwise you have to wait for
// the server to notice the client's demise.
sigRecv := make(chan os.Signal, 1)
signal.Notify(sigRecv, os.Interrupt)
eventChan := make(chan zookeeper.Event, 16)
for _, arg := range args {
zkPath := fixZkPath(arg)
_, _, watch, err := zconn.GetW(zkPath)
if err != nil {
log.Fatalf("watch error: %v", err)
}
go func() {
eventChan <- <-watch
}()
}
for {
select {
case <-sigRecv:
return
case event := <-eventChan:
log.Printf("watch: event %v: %v", event.Path, event)
if event.Type == zookeeper.EVENT_CHANGED {
data, stat, watch, err := zconn.GetW(event.Path)
if err != nil {
log.Printf("ERROR: failed to watch %v", err)
}
log.Printf("watch: %v %v\n", event.Path, stat)
println(data)
go func() {
eventChan <- <-watch
}()
} else if event.State == zookeeper.STATE_CLOSED {
return
}
}
}
}
func cmdLs(args []string) {
if len(args) == 0 {
log.Fatal("ls: no path specified")
}
hasError := false
for _, arg := range args {
zkPath := fixZkPath(arg)
var children []string
var err error
isDir := true
if *directoryListing {
children = []string{""}
isDir = false
} else if *recursiveListing {
children, err = zk.ChildrenRecursive(zconn, zkPath)
} else {
children, _, err = zconn.Children(zkPath)
// Assume this is a file node if it has no children.
if len(children) == 0 {
children = []string{""}
isDir = false
}
}
if err != nil {
hasError = true
log.Printf("ls: cannot access %v: %v", zkPath, err)
}
// Show the full path when it helps.
showFullPath := false
if *recursiveListing {
showFullPath = true
} else if *longListing && (*directoryListing || !isDir) {
showFullPath = true
}
sort.Strings(children)
if len(children) > 0 {
if *longListing && isDir {
fmt.Printf("total: %v\n", len(children))
}
wg := sync.WaitGroup{}
mutex := sync.Mutex{}
statMap := make(map[string]*zookeeper.Stat)
for _, child := range children {
localPath := path.Join(zkPath, child)
wg.Add(1)
go func() {
stat, err := zconn.Exists(localPath)
if err != nil {
log.Printf("ls: cannot access: %v: %v", localPath, err)
} else {
mutex.Lock()
statMap[localPath] = stat
mutex.Unlock()
}
wg.Done()
}()
}
wg.Wait()
// Lock to read statMap consistently.
mutex.Lock()
defer mutex.Unlock()
for _, child := range children {
localPath := path.Join(zkPath, child)
fmtPath(statMap[localPath], localPath, showFullPath)
}
}
}
if hasError {
os.Exit(1)
}
}
func fmtPath(stat *zookeeper.Stat, zkPath string, showFullPath bool) {
var name, perms string
if !showFullPath {
name = path.Base(zkPath)
} else {
name = zkPath
}
if *longListing {
if stat.NumChildren() > 0 {
// FIXME(msolomon) do permissions check?
perms = "drwxrwxrwx"
if stat.DataLength() > 0 {
// give a visual indication that this node has data as well as children
perms = "nrw-rw-rw-"
}
} else if stat.EphemeralOwner() != 0 {
perms = "erw-rw-rw-"
} else {
perms = "-rw-rw-rw-"
}
fmt.Printf("%v %v %v % 8v % 20v %v\n", perms, "zk", "zk", stat.DataLength(), stat.MTime().Format(timeFmt), name)
} else {
fmt.Printf("%v\n", name)
}
}
func cmdTouch(args []string) {
if len(args) != 1 {
log.Fatal("touch: need to specify exactly one path")
}
zkPath := fixZkPath(args[0])
if !isZkFile(zkPath) {
log.Fatalf("touch: not a /zk file %v", zkPath)
}
data, stat, err := zconn.Get(zkPath)
version := -1
create := false
if err != nil {
if err.(*zookeeper.Error).Code == zookeeper.ZNONODE {
create = true
} else {
log.Fatalf("touch: cannot access %v: %v", zkPath, err)
}
} else {
version = stat.Version()
}
if create {
if *touchOnly {
log.Fatalf("touch: no such path %v", zkPath)
}
if *createParents {
_, err = zk.CreateRecursive(zconn, zkPath, data, 0, zookeeper.WorldACL(zookeeper.PERM_ALL))
} else {
_, err = zconn.Create(zkPath, data, 0, zookeeper.WorldACL(zookeeper.PERM_ALL))
}
} else {
_, err = zconn.Set(zkPath, data, version)
}
if err != nil {
log.Fatalf("touch: cannot modify %v: %v", zkPath, err)
}
}
func cmdRm(args []string) {
if len(args) == 0 {
log.Fatal("rm: no path specified")
}
hasError := false
for _, arg := range args {
zkPath := fixZkPath(arg)
var err error
if *recursiveDelete {
err = zk.DeleteRecursive(zconn, zkPath, -1)
} else {
err = zconn.Delete(zkPath, -1)
}
if err != nil {
hasError = true
log.Printf("rm: cannot delete %v: %v", zkPath, err)
}
}
if hasError && !*force {
os.Exit(1)
}
}
func cmdCat(args []string) {
if len(args) == 0 {
log.Fatal("cat: no path specified")
}
hasError := false
for _, arg := range args {
zkPath := fixZkPath(arg)
data, _, err := zconn.Get(zkPath)
if err != nil {
hasError = true
log.Printf("cat: cannot access %v: %v", zkPath, err)
} else {
fmt.Print(data)
}
}
if hasError {
os.Exit(1)
}
}
func cmdEdit(args []string) {
if len(args) == 0 {
log.Fatal("edit: no path specified")
}
arg := args[0]
zkPath := fixZkPath(arg)
data, stat, err := zconn.Get(zkPath)
if err != nil {
log.Printf("edit: cannot access %v: %v", zkPath, err)
os.Exit(1)
}
name := path.Base(zkPath)
tmpPath := fmt.Sprintf("/tmp/zk-edit-%v-%v", name, time.Now().UnixNano())
f, err := os.Create(tmpPath)
if err == nil {
_, err = f.WriteString(data)
f.Close()
}
if err != nil {
log.Printf("edit: cannot write file %v", err)
os.Exit(1)
}
cmd := exec.Command(os.Getenv("EDITOR"), tmpPath)
cmd.Stdin = os.Stdin
cmd.Stdout = os.Stdout
cmd.Stderr = os.Stderr
err = cmd.Run()
if err != nil {
os.Remove(tmpPath)
log.Printf("edit: cannot start $EDITOR: %v", err)
os.Exit(1)
}
fileData, err := ioutil.ReadFile(tmpPath)
if err != nil {
os.Remove(tmpPath)
log.Printf("edit: cannot read file %v", err)
os.Exit(1)
}
if string(fileData) != data {
// data changed - update if we can
_, err = zconn.Set(zkPath, string(fileData), stat.Version())
if err != nil {
os.Remove(tmpPath)
log.Printf("edit: cannot write zk file %v", err)
os.Exit(1)
}
}
os.Remove(tmpPath)
}
func cmdStat(args []string) {
if len(args) == 0 {
log.Fatal("stat: no path specified")
}
hasError := false
for _, arg := range args {
zkPath := fixZkPath(arg)
acls, stat, err := zconn.ACL(zkPath)
if stat == nil {
err = fmt.Errorf("no such node")
}
if err != nil {
hasError = true
log.Printf("stat: cannot access %v: %v", zkPath, err)
continue
}
fmt.Printf("Path: %s\n", zkPath)
fmt.Printf("Created: %s\n", stat.CTime().Format(timeFmtMicro))
fmt.Printf("Modified: %s\n", stat.MTime().Format(timeFmtMicro))
fmt.Printf("Size: %v\n", stat.DataLength())
fmt.Printf("Children: %v\n", stat.NumChildren())
fmt.Printf("Version: %v\n", stat.Version())
fmt.Printf("Ephemeral: %v\n", stat.EphemeralOwner())
fmt.Printf("ACL:\n")
for _, acl := range acls {
fmt.Printf(" %v:%v %v\n", acl.Scheme, acl.Id, fmtAcl(acl))
}
}
if hasError {
os.Exit(1)
}
}
var charPermMap map[string]uint32
var permCharMap map[uint32]string
func init() {
charPermMap = map[string]uint32 {
"r": zookeeper.PERM_READ,
"w": zookeeper.PERM_WRITE,
"d": zookeeper.PERM_DELETE,
"c": zookeeper.PERM_CREATE,
"a": zookeeper.PERM_ADMIN,
}
permCharMap = make(map[uint32]string)
for c, p := range charPermMap {
permCharMap[p] = c
}
}
func fmtAcl(acl zookeeper.ACL) string {
s := ""
for _, perm := range []uint32{zookeeper.PERM_READ, zookeeper.PERM_WRITE, zookeeper.PERM_DELETE, zookeeper.PERM_CREATE, zookeeper.PERM_ADMIN} {
if acl.Perms & perm != 0{
s += permCharMap[perm]
} else {
s += "-"
}
}
return s
}
func cmdChmod(args []string) {
if len(args) < 2 {
log.Fatal("chmod: no permission specified")
}
mode := args[0]
if mode[0] != 'n' {
log.Fatal("chmod: invalid mode")
}
addPerms := false
if mode[1] == '+' {
addPerms = true
} else if mode[1] != '-' {
log.Fatal("chmod: invalid mode")
}
var permMask uint32
for _, c := range mode[2:] {
permMask |= charPermMap[string(c)]
}
hasError := false
for _, arg := range args[1:] {
zkPath := fixZkPath(arg)
aclv, _, err := zconn.ACL(zkPath)
if err != nil {
hasError = true
log.Printf("chmod: cannot set access %v: %v", zkPath, err)
continue
}
if addPerms {
aclv[0].Perms |= permMask
} else {
aclv[0].Perms &= ^permMask
}
err = zconn.SetACL(zkPath, aclv, -1)
if err != nil {
hasError = true
log.Printf("chmod: cannot set access %v: %v", zkPath, err)
continue
}
}
if hasError {
os.Exit(1)
}
}
func cmdCp(args []string) {
if len(args) < 2 {
log.Fatalf("cp: need to specify source and destination paths")
} else if len(args) == 2 {
fileCp(args[0], args[1])
} else {
multiFileCp(args)
}
}
func getPathData(filePath string) (string, error) {
if isZkFile(filePath) {
data, _, err := zconn.Get(filePath)
return data, err
} else {
var err error
file, err := os.Open(filePath)
if err == nil {
data, err := ioutil.ReadAll(file)
if err == nil {
return string(data), err
}
}
return "", err
}
panic("unreachable")
}
func setPathData(filePath, data string) error {
if isZkFile(filePath) {
_, err := zconn.Set(filePath, data, -1)
if err != nil && err.(*zookeeper.Error).Code == zookeeper.ZNONODE {
_, err = zk.CreateRecursive(zconn, filePath, data, 0, zookeeper.WorldACL(zookeeper.PERM_ALL))
}
return err
} else {
return ioutil.WriteFile(filePath, []byte(data), 0666)
}
panic("unreachable")
}
func fileCp(srcPath, dstPath string) {
dstIsDir := dstPath[len(dstPath)-1] == '/'
srcPath = fixZkPath(srcPath)
dstPath = fixZkPath(dstPath)
if !isZkFile(srcPath) && !isZkFile(dstPath) {
log.Fatal("cp: neither src nor dst is a /zk file: exitting")
}
data, err := getPathData(srcPath)
if err != nil {
log.Fatalf("cp: cannot read %v: %v", srcPath, err)
}
// If we are copying to a local directory - say '.', make the filename
// the same as the source.
if !isZkFile(dstPath) {
fileInfo, err := os.Stat(dstPath)
if err != nil {
if err.(*os.PathError).Err != syscall.ENOENT {
log.Fatalf("cp: cannot stat %v: %v", dstPath, err)
}
} else if fileInfo.IsDir() {
dstPath = path.Join(dstPath, path.Base(srcPath))
}
} else if dstIsDir {
// If we are copying into zk, interpret trailing slash as treating the
// dstPath as a directory.
dstPath = path.Join(dstPath, path.Base(srcPath))
}
if err := setPathData(dstPath, data); err != nil {
log.Fatalf("cp: cannot write %v: %v", dstPath, err)
}
}
func multiFileCp(args []string) {
dstPath := args[len(args)-1]
if dstPath[len(dstPath)-1] != '/' {
// In multifile context, dstPath must be a directory.
dstPath += "/"
}
for _, srcPath := range args[:len(args)-1] {
fileCp(srcPath, dstPath)
}
}
// Store a zk tree in a zip archive. This won't be immediately useful to
// zip tools since even "directories" can contain data.
func cmdZip(args []string) {
if len(args) < 2 {
log.Fatalf("zip: need to specify source and destination paths")
}
dstPath := args[len(args)-1]
args = args[:len(args)-1]
if !strings.HasSuffix(dstPath, ".zip") {
log.Fatalf("zip: need to specify destination .zip path: %v", dstPath)
}
zipFile, err := os.Create(dstPath)
if err != nil {
log.Fatalf("zip: error %v", err)
}
pathList := make([]string, 0, 256)
for _, arg := range args {
zkPath := fixZkPath(arg)
children, err := zk.ChildrenRecursive(zconn, zkPath)
if err != nil {
log.Fatalf("zip: error %v", err)
}
for _, child := range children {
pathList = append(pathList, path.Join(zkPath, child))
}
}
zipWriter := zip.NewWriter(zipFile)
for _, path := range pathList {
data, stat, err := zconn.Get(path)
if err != nil {
log.Fatal("zip: get failed: %v", err)
}
// Skip ephemerals - not sure why you would archive them.
if stat.EphemeralOwner() > 0 {
continue
}
fi := &zip.FileHeader{Name: path, Method: zip.Deflate}
fi.SetModTime(stat.MTime())
f, err := zipWriter.CreateHeader(fi)
if err != nil {
log.Fatal("zip: create failed: %v", err)
}
_, err = f.Write([]byte(data))
if err != nil {
log.Fatal("zip: create failed: %v", err)
}
}
err = zipWriter.Close()
if err != nil {
log.Fatal("zip: close failed: %v", err)
}
zipFile.Close()
}
func cmdUnzip(args []string) {
if len(args) != 2 {
log.Fatalf("zip: need to specify source and destination paths")
}
srcPath := args[0]
dstPath := args[1]
if !strings.HasSuffix(srcPath, ".zip") {
log.Fatalf("zip: need to specify src .zip path: %v", srcPath)
}
zipReader, err := zip.OpenReader(srcPath)
if err != nil {
log.Fatalf("zip: error %v", err)
}
defer zipReader.Close()
for _, zf := range zipReader.File {
rc, err := zf.Open()
if err != nil {
log.Fatalf("unzip: error %v", err)
}
data, err := ioutil.ReadAll(rc)
if err != nil {
log.Fatal("unzip: failed reading archive: %v", err)
}
zkPath := zf.Name
if dstPath != "/" {
zkPath = path.Join(dstPath, zkPath)
}
_, err = zk.CreateRecursive(zconn, zkPath, string(data), 0, zookeeper.WorldACL(zookeeper.PERM_ALL))
if err != nil && err.(*zookeeper.Error).Code != zookeeper.ZNODEEXISTS {
log.Fatal("unzip: zk create failed: %v", err)
}
_, err = zconn.Set(zkPath, string(data), -1)
if err != nil {
log.Fatal("unzip: zk set failed: %v", err)
}
rc.Close()
}
}

86
go/cmd/zkctl/zkctl.go Normal file
Просмотреть файл

@ -0,0 +1,86 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package main
import (
"bufio"
"flag"
"fmt"
"log"
"os"
"strings"
"code.google.com/p/vitess.x/go/zk/zkctl"
"code.google.com/p/vitess/go/relog"
)
var usage = `
Commands:
init | start | shutdown | teardown
`
var zkCfg = flag.String("zk.cfg", "6@<hostname>:3801:3802:3803",
"zkid@server1:leaderPort1:electionPort1:clientPort1,...)")
var myId = flag.Uint("zk.myid", 0,
"which server do you want to be? only needed when running multiple instance on one box, otherwise myid is implied by hostname")
var force = flag.Bool("force", false, "force action, no promptin")
var logLevel = flag.String("log.level", "WARNING", "set log level")
var stdin *bufio.Reader
func init() {
flag.Usage = func() {
fmt.Fprintf(os.Stderr, "Usage of %s:\n", os.Args[0])
flag.PrintDefaults()
fmt.Fprintf(os.Stderr, usage)
}
stdin = bufio.NewReader(os.Stdin)
}
func confirm(prompt string) bool {
if *force {
return true
}
fmt.Fprintf(os.Stderr, prompt+" [NO/yes] ")
line, _ := stdin.ReadString('\n')
return strings.ToLower(strings.TrimSpace(line)) == "yes"
}
func main() {
flag.Parse()
args := flag.Args()
if len(args) == 0 {
flag.Usage()
os.Exit(1)
}
logger := relog.New(os.Stderr, "zkctl ",
log.Ldate|log.Lmicroseconds|log.Lshortfile,
relog.LogNameToLogLevel(*logLevel))
relog.SetLogger(logger)
zkConfig := zkctl.MakeZkConfigFromString(*zkCfg, *myId)
zkd := zkctl.NewZkd(zkConfig)
action := flag.Arg(0)
var err error
switch action {
case "init":
err = zkd.Init()
case "shutdown":
err = zkd.Shutdown()
case "start":
err = zkd.Start()
case "teardown":
err = zkd.Teardown()
default:
log.Fatalf("invalid action: %v", action)
}
if err != nil {
log.Fatalf("failed %v: %v", action, err)
}
}

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

@ -0,0 +1,337 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package mysqlctl
/*
the binlogreader is intended to "tail -f" a binlog, but be smart enough
to stop tailing it when mysql is done writing to that binlog. The stop
condition is if EOF is reached *and* the next file has appeared.
*/
import (
"bufio"
"code.google.com/p/vitess/go/relog"
"encoding/binary"
"errors"
"flag"
"fmt"
"io"
"net/http"
"os"
"path"
"strconv"
"strings"
"time"
)
const (
BINLOG_HEADER_SIZE = 4 // copied from mysqlbinlog.cc for mysql 5.0.33
EVENT_HEADER_SIZE = 19 // 4.0 and above, can be larger in 5.x
DEFAULT_BLOCK_SIZE = 16 * 1024
)
var binlogBlockSize int64
var maxWaitTimeout float64
var logWaitTimeout float64
func init() {
flag.Int64Var(&binlogBlockSize, "binlog-block-size", DEFAULT_BLOCK_SIZE,
"block size for binlog chunks")
flag.Float64Var(&maxWaitTimeout, "binlog-max-wait", 3600.0,
"maximum time to wait for more data before closing (seconds)")
flag.Float64Var(&logWaitTimeout, "binlog-wait", 5.0,
"time to wait for more data (seconds)")
}
type stats struct {
Reads int64
Bytes int64
Sleeps int64
StartTime time.Time
}
type request struct {
config *Mycnf
startPosition int64
file *os.File
nextFilename string
stats
}
type BinlogReader struct {
binLogPrefix string
}
func (blr *BinlogReader) binLogPathForId(fileId int) string {
return fmt.Sprintf("%v.%06d", blr.binLogPrefix, fileId)
}
func NewBinlogReader(binLogPrefix string) *BinlogReader {
return &BinlogReader{binLogPrefix: binLogPrefix}
}
/*
based on http://forge.mysql.com/wiki/MySQL_Internals_Binary_Log
+=====================================+
| event | timestamp 0 : 4 |
| header +----------------------------+
| | type_code 4 : 1 |
| +----------------------------+
| | server_id 5 : 4 |
| +----------------------------+
| | event_length 9 : 4 |
| +----------------------------+
| | next_position 13 : 4 |
| +----------------------------+
| | flags 17 : 2 |
+=====================================+
| event | fixed part 19 : y |
| data +----------------------------+
| | variable part |
+=====================================+
*/
func readFirstEventSize(binlog io.ReadSeeker) uint32 {
pos, _ := binlog.Seek(0, 1)
defer binlog.Seek(pos, 0)
if _, err := binlog.Seek(BINLOG_HEADER_SIZE+9, 0); err != nil {
panic("failed binlog seek: " + err.Error())
}
var eventLength uint32
if err := binary.Read(binlog, binary.LittleEndian, &eventLength); err != nil {
panic("failed binlog read: " + err.Error())
}
return eventLength
}
func (blr *BinlogReader) serve(filename string, startPosition int64, writer http.ResponseWriter) {
flusher := writer.(http.Flusher)
stats := stats{StartTime: time.Now()}
binlogFile, nextLog := blr.open(filename)
defer binlogFile.Close()
positionWaitStart := make(map[int64]time.Time)
if startPosition > 0 {
// the start position can be greater than the file length
// in which case, we just keep rotating files until we find it
for {
size, err := binlogFile.Seek(0, 2)
if err != nil {
relog.Error("BinlogReader.serve seek err: %v", err)
return
}
if startPosition > size {
startPosition -= size
// swap to next file
binlogFile.Close()
binlogFile, nextLog = blr.open(nextLog)
// normally we chomp subsequent headers, so we have to
// add this back into the position
//startPosition += BINLOG_HEADER_SIZE
} else {
break
}
}
// inject the header again to fool mysqlbinlog
// FIXME(msolomon) experimentally determine the header size.
// 5.1.50 is 106, 5.0.24 is 98
firstEventSize := readFirstEventSize(binlogFile)
prefixSize := int64(BINLOG_HEADER_SIZE + firstEventSize)
writer.Header().Set("Vt-Binlog-Offset", strconv.FormatInt(prefixSize, 10))
relog.Info("BinlogReader.serve inject header + first event: %v", prefixSize)
position, err := binlogFile.Seek(0, 0)
if err == nil {
_, err = io.CopyN(writer, binlogFile, prefixSize)
//relog.Info("BinlogReader %x copy @ %v:%v,%v", stats.StartTime, binlogFile.Name(), position, written)
}
if err != nil {
relog.Error("BinlogReader.serve err: %v", err)
return
}
position, err = binlogFile.Seek(startPosition, 0)
relog.Info("BinlogReader %x seek to startPosition %v @ %v:%v", stats.StartTime, startPosition, binlogFile.Name(), position)
} else {
writer.Header().Set("Vt-Binlog-Offset", "0")
}
// FIXME(msolomon) register stats on http handler
for {
//position, _ := binlogFile.Seek(0, 1)
written, err := io.CopyN(writer, binlogFile, binlogBlockSize)
//relog.Info("BinlogReader %x copy @ %v:%v,%v", stats.StartTime, binlogFile.Name(), position, written)
if err != nil && err != io.EOF {
relog.Error("BinlogReader.serve err: %v", err)
return
}
stats.Reads++
stats.Bytes += written
if written != binlogBlockSize {
if _, statErr := os.Stat(nextLog); statErr == nil {
relog.Info("BinlogReader swap log file: %v", nextLog)
// swap to next log file
binlogFile.Close()
binlogFile, nextLog = blr.open(nextLog)
positionWaitStart = make(map[int64]time.Time)
binlogFile.Seek(BINLOG_HEADER_SIZE, 0)
} else {
flusher.Flush()
position, _ := binlogFile.Seek(0, 1)
relog.Info("BinlogReader %x wait for more data: %v:%v", stats.StartTime, binlogFile.Name(), position)
// wait for more data
time.Sleep(time.Duration(logWaitTimeout * 1e9))
stats.Sleeps++
now := time.Now()
if lastSlept, ok := positionWaitStart[position]; ok {
if (now.Sub(lastSlept)) > time.Duration(maxWaitTimeout*1e9) {
relog.Error("MAX_WAIT_TIMEOUT exceeded, closing connection")
return
}
} else {
positionWaitStart[position] = now
}
}
}
}
}
func (blr *BinlogReader) HandleBinlogRequest(rw http.ResponseWriter, req *http.Request) {
defer func() {
if err := recover(); err != nil {
// nothing to do, but note it here and soldier on
relog.Error("HandleBinlogRequest failed: %v", err)
}
}()
// FIXME(msolomon) some sort of security, no?
relog.Info("serve %v", req.URL.Path)
// path is something like /vt/vt-xxxxxx-bin-log:position
pieces := strings.SplitN(path.Base(req.URL.Path), ":", 2)
pos, _ := strconv.ParseInt(pieces[1], 10, 64)
blr.serve(pieces[0], pos, rw)
}
// return open log file and the name of the next log path to watch
func (blr *BinlogReader) open(name string) (*os.File, string) {
ext := path.Ext(name)
fileId, err := strconv.Atoi(ext[1:])
if err != nil {
panic(errors.New("bad binlog name: " + name))
}
logPath := blr.binLogPathForId(fileId)
if !strings.HasSuffix(logPath, name) {
panic(errors.New("binlog name mismatch: " + logPath + " vs " + name))
}
file, err := os.Open(logPath)
if err != nil {
panic(err)
}
nextLog := blr.binLogPathForId(fileId + 1)
return file, nextLog
}
func (blr *BinlogReader) ServeData(filename string, startPosition int64, writer io.Writer) {
stats := stats{StartTime: time.Now()}
binlogFile, nextLog := blr.open(filename)
defer binlogFile.Close()
positionWaitStart := make(map[int64]time.Time)
//var offsetString string
bufWriter := bufio.NewWriterSize(writer, 16*1024)
if startPosition > 0 {
// the start position can be greater than the file length
// in which case, we just keep rotating files until we find it
for {
size, err := binlogFile.Seek(0, 2)
if err != nil {
relog.Error("BinlogReader.serve seek err: %v", err)
return
}
if startPosition > size {
startPosition -= size
// swap to next file
binlogFile.Close()
binlogFile, nextLog = blr.open(nextLog)
// normally we chomp subsequent headers, so we have to
// add this back into the position
//startPosition += BINLOG_HEADER_SIZE
} else {
break
}
}
// inject the header again to fool mysqlbinlog
// FIXME(msolomon) experimentally determine the header size.
// 5.1.50 is 106, 5.0.24 is 98
firstEventSize := readFirstEventSize(binlogFile)
prefixSize := int64(BINLOG_HEADER_SIZE + firstEventSize)
relog.Info("BinlogReader.serve inject header + first event: %v", prefixSize)
//offsetString = fmt.Sprintf("Vt-Binlog-Offset: %v\n", strconv.FormatInt(prefixSize, 10))
position, err := binlogFile.Seek(0, 0)
if err == nil {
_, err = io.CopyN(writer, binlogFile, prefixSize)
//relog.Info("Sending prefix, BinlogReader copy @ %v:%v,%v", binlogFile.Name(), position, written)
}
if err != nil {
relog.Error("BinlogReader.serve err: %v", err)
return
}
position, err = binlogFile.Seek(startPosition, 0)
relog.Info("BinlogReader %x seek to startPosition %v @ %v:%v", stats.StartTime, startPosition, binlogFile.Name(), position)
}
for {
//position, _ := binlogFile.Seek(0, 1)
written, err := io.CopyN(writer, binlogFile, binlogBlockSize)
if err != nil && err != io.EOF {
relog.Error("BinlogReader.serve err: %v", err)
return
}
//relog.Info("BinlogReader copy @ %v:%v,%v", binlogFile.Name(), position, written)
stats.Reads++
stats.Bytes += written
if written != binlogBlockSize {
if _, statErr := os.Stat(nextLog); statErr == nil {
relog.Info("BinlogReader swap log file: %v", nextLog)
// swap to next log file
binlogFile.Close()
binlogFile, nextLog = blr.open(nextLog)
positionWaitStart = make(map[int64]time.Time)
binlogFile.Seek(BINLOG_HEADER_SIZE, 0)
} else {
bufWriter.Flush()
position, _ := binlogFile.Seek(0, 1)
//relog.Info("BinlogReader %x wait for more data: %v:%v", stats.StartTime, binlogFile.Name(), position)
// wait for more data
time.Sleep(time.Duration(logWaitTimeout * 1e9))
stats.Sleeps++
now := time.Now()
if lastSlept, ok := positionWaitStart[position]; ok {
if (now.Sub(lastSlept)) > time.Duration(maxWaitTimeout*1e9) {
relog.Error("MAX_WAIT_TIMEOUT exceeded, closing connection")
return
}
} else {
positionWaitStart[position] = now
}
}
}
}
}

35
go/vt/mysqlctl/clone.go Normal file
Просмотреть файл

@ -0,0 +1,35 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package mysqlctl
import (
"errors"
"net/rpc"
)
// the overall states the src/dst hosts can be in. eventually, some of these
// operations/states might be interleaved
/*
const (
RUNNING = iota,
STOPPED,
COMPRESSING,
COMPRESSED,
TRANSFERRING,
DECOMPRESSING,
DECOMPRESSED,
COPYING,
PRUNING,
REPLICATING,
READY,
HW_FAILURE
)
*/
var UnimplementedError = errors.New("unimplemented")
func connectToTabletServer(addr string) (client *rpc.Client, err error) {
client, err = rpc.DialHTTP("tcp", addr)
return
}

119
go/vt/mysqlctl/fileutil.go Normal file
Просмотреть файл

@ -0,0 +1,119 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package mysqlctl
import (
"bufio"
"compress/gzip"
"crypto/md5"
"encoding/hex"
"io"
"io/ioutil"
"os"
"path"
)
/* compress a single file with gzip, leaving the src file intact.
FIXME(msolomon) not sure how well Go will schedule cpu intensive tasks
might be better if this forked off workers.
*/
func compressFile(srcPath, dstPath string) error {
srcFile, err := os.OpenFile(srcPath, os.O_RDONLY, 0)
if err != nil {
return err
}
defer srcFile.Close()
src := bufio.NewReaderSize(srcFile, 2*1024*1024)
dir, filePrefix := path.Split(dstPath)
dstFile, err := ioutil.TempFile(dir, filePrefix)
if err != nil {
return err
}
defer dstFile.Close()
dst := bufio.NewWriterSize(dstFile, 2*1024*1024)
compressor := gzip.NewWriter(dst)
defer compressor.Close()
_, err = io.Copy(compressor, src)
if err != nil {
return err
}
// close dst manually to flush all buffers to disk
compressor.Close()
dst.Flush()
dstFile.Close()
// atomically move completed compressed file
return os.Rename(dstFile.Name(), dstPath)
}
/* uncompress a single file with gzip, leaving the src file intact.
FIXME(msolomon) not sure how well Go will schedule cpu intensive tasks
might be better if this forked off workers.
*/
func uncompressFile(srcPath, dstPath string) error {
srcFile, err := os.OpenFile(srcPath, os.O_RDONLY, 0)
if err != nil {
return err
}
defer srcFile.Close()
src := bufio.NewReaderSize(srcFile, 2*1024*1024)
if err != nil {
return err
}
decompressor, err := gzip.NewReader(src)
if err != nil {
return err
}
defer decompressor.Close()
dir, filePrefix := path.Split(dstPath)
dstFile, err := ioutil.TempFile(dir, filePrefix)
if err != nil {
return err
}
defer dstFile.Close()
dst := bufio.NewWriterSize(dstFile, 2*1024*1024)
_, err = io.Copy(dst, decompressor)
if err != nil {
return err
}
// close dst manually to flush all buffers to disk
dst.Flush()
dstFile.Close()
// atomically move uncompressed file
if err := os.Chmod(dstFile.Name(), 0664); err != nil {
return err
}
return os.Rename(dstFile.Name(), dstPath)
}
func md5File(filename string) (string, error) {
file, err := os.OpenFile(filename, os.O_RDONLY, 0)
if err != nil {
return "", err
}
defer file.Close()
src := bufio.NewReaderSize(file, 2*1024*1024)
hasher := md5.New()
_, err = io.Copy(hasher, src)
if err != nil {
return "", err
}
return hex.EncodeToString(hasher.Sum(nil)), nil
}

221
go/vt/mysqlctl/mycnf.go Normal file
Просмотреть файл

@ -0,0 +1,221 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
/*
Generate my.cnf files from templates.
*/
package mysqlctl
import (
"bufio"
"bytes"
"fmt"
"io"
"io/ioutil"
"os"
"path"
"strconv"
"strings"
"text/template"
)
type VtReplParams struct {
TabletHost string
TabletPort int
StartKey string
EndKey string
}
type Mycnf struct {
ServerId uint
TabletDir string
DataDir string
MycnfPath string
InnodbDataHomeDir string
InnodbLogGroupHomeDir string
DatabaseName string // for replication
SocketPath string
MysqlPort int
VtHost string
VtPort int
StartKey string
EndKey string
}
var innodbDataSubdir = "innodb/data"
var innodbLogSubdir = "innodb/log"
/* uid is a unique id for a particular tablet - it must be unique within the
tabletservers deployed within a keyspace, lest there be collisions on disk.
mysqldPort needs to be unique per instance per machine (shocking) but choosing
this sensibly has nothing to do with the config, so I'll punt.
*/
func NewMycnf(uid uint, mysqlPort int, keyspace string, vtRepl VtReplParams) *Mycnf {
cnf := new(Mycnf)
cnf.ServerId = uid
cnf.MysqlPort = mysqlPort
cnf.TabletDir = fmt.Sprintf("/vt/vt_%010d", uid)
cnf.DataDir = path.Join(cnf.TabletDir, "data")
cnf.MycnfPath = path.Join(cnf.TabletDir, "my.cnf")
cnf.InnodbDataHomeDir = path.Join(cnf.TabletDir, innodbDataSubdir)
cnf.InnodbLogGroupHomeDir = path.Join(cnf.TabletDir, innodbLogSubdir)
cnf.SocketPath = path.Join(cnf.TabletDir, "mysql.sock")
// this might be empty if you aren't assigned to a keyspace
cnf.DatabaseName = keyspace
cnf.VtHost = vtRepl.TabletHost
cnf.VtPort = vtRepl.TabletPort
cnf.StartKey = vtRepl.StartKey
cnf.EndKey = vtRepl.EndKey
return cnf
}
func (cnf *Mycnf) DirectoryList() []string {
return []string{
cnf.DataDir,
cnf.InnodbDataHomeDir,
cnf.InnodbLogGroupHomeDir,
cnf.relayLogDir(),
cnf.binLogDir(),
}
}
func (cnf *Mycnf) ErrorLogPath() string {
return path.Join(cnf.TabletDir, "error.log")
}
func (cnf *Mycnf) SlowLogPath() string {
return path.Join(cnf.TabletDir, "slow-query.log")
}
func (cnf *Mycnf) relayLogDir() string {
return path.Join(cnf.TabletDir, "relay-logs")
}
func (cnf *Mycnf) RelayLogPath() string {
return path.Join(cnf.relayLogDir(),
fmt.Sprintf("vt-%010d-relay-bin", cnf.ServerId))
}
func (cnf *Mycnf) RelayLogIndexPath() string {
return cnf.RelayLogPath() + ".index"
}
func (cnf *Mycnf) RelayLogInfoPath() string {
return path.Join(cnf.TabletDir, "relay-logs", "relay.info")
}
func (cnf *Mycnf) binLogDir() string {
return path.Join(cnf.TabletDir, "bin-logs")
}
func (cnf *Mycnf) BinLogPath() string {
return path.Join(cnf.binLogDir(),
fmt.Sprintf("vt-%010d-bin", cnf.ServerId))
}
func (cnf *Mycnf) BinLogPathForId(fileid int) string {
return path.Join(cnf.binLogDir(),
fmt.Sprintf("vt-%010d-bin.%06d", cnf.ServerId, fileid))
}
func (cnf *Mycnf) BinLogIndexPath() string {
return cnf.BinLogPath() + ".index"
}
func (cnf *Mycnf) MasterInfoPath() string {
return path.Join(cnf.TabletDir, "master.info")
}
func (cnf *Mycnf) PidFile() string {
return path.Join(cnf.TabletDir, "mysql.pid")
}
/*
Join cnf files cnfPaths and subsitute in the right values.
*/
func MakeMycnf(cnfPaths []string, mycnf *Mycnf, header string) (string, error) {
myTemplateSource := new(bytes.Buffer)
for _, line := range strings.Split(header, "\n") {
fmt.Fprintf(myTemplateSource, "## %v\n", strings.TrimSpace(line))
}
myTemplateSource.WriteString("[mysqld]\n")
for _, path := range cnfPaths {
data, dataErr := ioutil.ReadFile(path)
if dataErr != nil {
return "", dataErr
}
myTemplateSource.WriteString("## " + path + "\n")
myTemplateSource.Write(data)
}
myTemplate, err := template.New("").Parse(myTemplateSource.String())
if err != nil {
return "", err
}
mycnfData := new(bytes.Buffer)
err = myTemplate.Execute(mycnfData, mycnf)
if err != nil {
return "", err
}
return mycnfData.String(), nil
}
/* Create a config for this instance. Search cnfPath for the appropriate
cnf template files.
*/
func MakeMycnfForMysqld(mysqld *Mysqld, cnfPath, header string) (string, error) {
// FIXME(msolomon) determine config list from mysqld struct
cnfs := []string{"default", "master", "replica"}
paths := make([]string, len(cnfs))
for i, name := range cnfs {
paths[i] = fmt.Sprintf("%v/%v.cnf", cnfPath, name)
}
return MakeMycnf(paths, mysqld.config, header)
}
func ReadMycnf(cnfPath string) (*Mycnf, error) {
f, err := os.Open(cnfPath)
if err != nil {
return nil, err
}
defer f.Close()
buf := bufio.NewReader(f)
mycnf := &Mycnf{SocketPath: "/var/lib/mysql/mysql.sock",
MycnfPath: cnfPath,
// FIXME(msolomon) remove this whole method, just asking for trouble
VtHost: "localhost",
VtPort: 6612,
}
for {
line, _, err := buf.ReadLine()
if err == io.EOF {
break
}
line = bytes.TrimSpace(line)
if bytes.HasPrefix(line, []byte("server-id")) {
serverId, err := strconv.Atoi(string(bytes.TrimSpace(bytes.Split(line, []byte("="))[1])))
if err != nil {
return nil, err
}
mycnf.ServerId = uint(serverId)
} else if bytes.HasPrefix(line, []byte("port")) {
port, err := strconv.Atoi(string(bytes.TrimSpace(bytes.Split(line, []byte("="))[1])))
if err != nil {
return nil, err
}
mycnf.MysqlPort = port
} else if bytes.HasPrefix(line, []byte("innodb_log_group_home_dir")) {
mycnf.InnodbLogGroupHomeDir = string(bytes.TrimSpace(bytes.Split(line, []byte("="))[1]))
} else if bytes.HasPrefix(line, []byte("innodb_data_home_dir")) {
mycnf.InnodbDataHomeDir = string(bytes.TrimSpace(bytes.Split(line, []byte("="))[1]))
} else if bytes.HasPrefix(line, []byte("socket")) {
mycnf.SocketPath = string(bytes.TrimSpace(bytes.Split(line, []byte("="))[1]))
}
}
return mycnf, nil
}

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

@ -0,0 +1,28 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package mysqlctl
import (
"os"
"testing"
)
func TestMycnf(t *testing.T) {
var vtRepl VtReplParams
vtRepl.TabletHost = "localhost"
vtRepl.TabletPort = 6702
vtRepl.StartKey = ""
vtRepl.EndKey = ""
tablet0 := NewMysqld(NewMycnf(0, 6802, "", vtRepl), nil, nil)
cnfTemplatePath := os.ExpandEnv("$VTROOT/src/code.google.com/p/vitess.x/config/mycnf")
// FIXME(msolomon) make a path that has a chance of succeeding elsewhere
data, err := MakeMycnfForMysqld(tablet0, cnfTemplatePath, "test header")
if err != nil {
t.Errorf("err: %v", err)
} else {
t.Logf("data: %v", data)
}
}

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

@ -0,0 +1,101 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package mysqlctl
/* quick hack to diff two string by running diff externally. probably linux only. */
import (
"code.google.com/p/vitess/go/relog"
"io"
"os"
)
/*
-d, --database=name List entries for just this database (local log only).
-D, --disable-log-bin
Disable binary log. This is useful, if you enabled
--to-last-log and are sending the output to the same
MySQL server. This way you could avoid an endless loop.
You would also like to use it when restoring after a
crash to avoid duplication of the statements you already
have. NOTE: you will need a SUPER privilege to use this
option.
-F, --force-if-open Force if binlog was not closed properly.
-f, --force-read Force reading unknown binlog events.
-h, --host=name Get the binlog from server.
-l, --local-load=name
Prepare local temporary files for LOAD DATA INFILE in the
specified directory.
-o, --offset=# Skip the first N entries.
-p, --password[=name]
Password to connect to remote server.
-P, --port=# Port number to use for connection or 0 for default to, in
order of preference, my.cnf, $MYSQL_TCP_PORT,
/etc/services, built-in default (3306).
--protocol=name The protocol to use for connection (tcp, socket, pipe,
memory).
-R, --read-from-remote-server
Read binary logs from a MySQL server.
-r, --result-file=name
Direct output to a given file.
--server-id=# Extract only binlog entries created by the server having
the given id.
--set-charset=name Add 'SET NAMES character_set' to the output.
-S, --socket=name The socket file to use for connection.
-j, --start-position=#
Start reading the binlog at position N. Applies to the
first binlog passed on the command line.
--stop-position=# Stop reading the binlog at position N. Applies to the
last binlog passed on the command line.
-t, --to-last-log Requires -R. Will not stop at the end of the requested
binlog but rather continue printing until the end of the
last binlog of the MySQL server. If you send the output
to the same MySQL server, that may lead to an endless
loop.
-u, --user=name Connect to the remote server as username.
*/
type BinlogImport struct {
User string
Password string
Host string
Port uint
StartPosition uint
}
// return a Reader from which the decoded binlog can be read
func DecodeMysqlBinlog(binlog *os.File) (io.Reader, error) {
dir := os.ExpandEnv("$VT_MYSQL_BIN")
name := "vt_mysqlbinlog"
arg := []string{"vt_mysqlbinlog", "-"}
dataRdFile, dataWrFile, pipeErr := os.Pipe()
if pipeErr != nil {
return nil, pipeErr
}
// let the caller close the read file
defer dataWrFile.Close()
fds := []*os.File{
binlog,
dataWrFile,
os.Stderr,
}
attrs := &os.ProcAttr{Dir: dir, Files: fds}
process, err := os.StartProcess(name, arg, attrs)
if err != nil {
return nil, err
}
go func() {
// just make sure we don't spawn zombies
waitMsg, err := process.Wait()
relog.Error("vt_mysqlbinlog exited: %v err: %v", waitMsg, err)
}()
return dataRdFile, nil
}

230
go/vt/mysqlctl/mysqld.go Normal file
Просмотреть файл

@ -0,0 +1,230 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
/*
Commands for controlling an external mysql process.
Some commands are issued as exec'd tools, some are handled by connecting via
the mysql protocol.
*/
package mysqlctl
import (
"errors"
"fmt"
"io/ioutil"
"os"
"os/exec"
"strings"
"syscall"
"time"
"code.google.com/p/vitess/go/mysql"
"code.google.com/p/vitess/go/relog"
)
const (
MysqlWaitTime = 20 // number of seconds to wait
)
type CreateConnection func() (*mysql.Connection, error)
type Mysqld struct {
config *Mycnf
createConnection CreateConnection
}
func NewMysqld(config *Mycnf, dbaconfig map[string]interface{}) *Mysqld {
createSuperConnection := func() (*mysql.Connection, error) {
return mysql.Connect(dbaconfig)
}
return &Mysqld{config, createSuperConnection}
}
func Start(mt *Mysqld) error {
relog.Info("mysqlctl.Start")
dir := os.ExpandEnv("$VTROOT/dist/vt-mysql")
name := dir + "/bin/mysqld_safe"
arg := []string{
"--defaults-file=" + mt.config.MycnfPath}
env := []string{
os.ExpandEnv("LD_LIBRARY_PATH=$VTROOT/dist/vt-mysql/lib/mysql"),
}
cmd := exec.Command(name, arg...)
cmd.Dir = dir
cmd.Env = env
relog.Info("Start %v", cmd)
_, err := cmd.StderrPipe()
if err != nil {
return nil
}
err = cmd.Start()
if err != nil {
return nil
}
// wait so we don't get a bunch of defunct processes
go cmd.Wait()
// give it some time to succeed - usually by the time the socket emerges
// we are in good shape
for i := 0; i < MysqlWaitTime; i++ {
time.Sleep(1e9)
_, statErr := os.Stat(mt.config.SocketPath)
if statErr == nil {
return nil
} else if statErr.(*os.PathError).Err != syscall.ENOENT {
return statErr
}
}
return errors.New(name + ": deadline exceeded waiting for " + mt.config.SocketPath)
}
/* waitForMysqld: should the function block until mysqld has stopped?
This can actually take a *long* time if the buffer cache needs to be fully
flushed - on the order of 20-30 minutes.
*/
func Shutdown(mt *Mysqld, waitForMysqld bool) error {
relog.Info("mysqlctl.Shutdown")
// possibly mysql is already shutdown, check for a few files first
_, socketPathErr := os.Stat(mt.config.SocketPath)
_, pidPathErr := os.Stat(mt.config.PidFile())
if socketPathErr != nil && pidPathErr != nil {
relog.Warning("assuming shutdown - no socket, no pid file")
return nil
}
dir := os.ExpandEnv("$VTROOT/dist/vt-mysql")
name := dir + "/bin/mysqladmin"
arg := []string{
"-u", "vt_dba", "-S", mt.config.SocketPath,
"shutdown"}
env := []string{
os.ExpandEnv("LD_LIBRARY_PATH=$VTROOT/dist/vt-mysql/lib/mysql"),
}
_, err := execCmd(name, arg, env, dir)
if err != nil {
return err
}
// wait for mysqld to really stop. use the sock file as a proxy for that since
// we can't call wait() in a process we didn't start.
if waitForMysqld {
for i := 0; i < MysqlWaitTime; i++ {
_, statErr := os.Stat(mt.config.SocketPath)
// NOTE: dreaded PathError :(
if statErr != nil && statErr.(*os.PathError).Err == syscall.ENOENT {
return nil
}
time.Sleep(1e9)
}
return errors.New("gave up waiting for mysqld to stop")
}
return nil
}
/* exec and wait for a return code. look for name in $PATH. */
func execCmd(name string, args, env []string, dir string) (cmd *exec.Cmd, err error) {
cmdPath, _ := exec.LookPath(name)
relog.Info("execCmd: %v %v %v", name, cmdPath, args)
cmd = exec.Command(cmdPath, args...)
cmd.Env = env
cmd.Dir = dir
output, err := cmd.CombinedOutput()
if err != nil {
err = errors.New(name + ": " + string(output))
}
return cmd, err
}
func Init(mt *Mysqld) error {
relog.Info("mysqlctl.Init")
for _, path := range mt.config.DirectoryList() {
if err := os.MkdirAll(path, 0775); err != nil {
relog.Error("%s", err.Error())
return err
}
// FIXME(msolomon) validate permissions?
}
cnfTemplatePath := os.ExpandEnv("$VTROOT/src/code.google.com/p/vitess.x/config/mycnf")
configData, err := MakeMycnfForMysqld(mt, cnfTemplatePath, "tablet uid?")
if err == nil {
err = ioutil.WriteFile(mt.config.MycnfPath, []byte(configData), 0664)
}
if err != nil {
relog.Error("failed creating %v: %v", mt.config.MycnfPath, err)
return err
}
dbTbzPath := os.ExpandEnv("$VTROOT/src/code.google.com/p/vitess.x/data/bootstrap/mysql-db-dir.tbz")
relog.Info("decompress bootstrap db %v", dbTbzPath)
args := []string{"-xj", "-C", mt.config.DataDir, "-f", dbTbzPath}
_, tarErr := execCmd("tar", args, []string{}, "")
if tarErr != nil {
relog.Error("failed unpacking %v: %v", dbTbzPath, tarErr)
return tarErr
}
if err = Start(mt); err != nil {
relog.Error("failed starting, check %v", mt.config.ErrorLogPath())
return err
}
schemaPath := os.ExpandEnv("$VTROOT/src/code.google.com/p/vitess.x/data/bootstrap/_vt_schema.sql")
schema, err := ioutil.ReadFile(schemaPath)
if err != nil {
return err
}
sqlCmds := make([]string, 0, 10)
relog.Info("initial schema: %v", string(schema))
for _, cmd := range strings.Split(string(schema), ";") {
cmd = strings.TrimSpace(cmd)
if cmd == "" {
continue
}
sqlCmds = append(sqlCmds, cmd)
}
return mt.executeSuperQueryList(sqlCmds)
}
func Teardown(mt *Mysqld, force bool) error {
relog.Info("mysqlctl.Teardown")
if err := Shutdown(mt, true); err != nil {
if !force {
relog.Error("failed mysqld shutdown: %v", err.Error())
return err
} else {
relog.Warning("failed mysqld shutdown: %v", err.Error())
}
}
var removalErr error
for _, dir := range mt.config.DirectoryList() {
relog.Info("remove data dir %v", dir)
if err := os.RemoveAll(dir); err != nil {
relog.Error("failed removing %v: %v", dir, err.Error())
removalErr = err
}
}
return removalErr
}
func Reinit(mt *Mysqld) error {
if err := Teardown(mt, false); err != nil {
return err
}
return Init(mt)
}
func (mysqld *Mysqld) Addr() string {
host, err := os.Hostname()
if err != nil {
panic(err)
}
return fmt.Sprintf("%v:%v", host, mysqld.config.MysqlPort)
}

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

@ -0,0 +1,42 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package mysqlctl
import "testing"
func TestStartShutdown(t *testing.T) {
tablet0 := NewMysqld(NewMycnf(0, ""), nil)
tablet1 := NewMysqld(NewMycnf(1, ""), nil)
var err error
err = Init(tablet0)
if err != nil {
t.Fatalf("Init(0) err: %v", err)
}
err = Init(tablet1)
if err != nil {
t.Fatalf("Init(1) err: %v", err)
}
err = Shutdown(tablet0, true)
if err != nil {
t.Fatalf("Shutdown() err: %v", err)
}
err = Start(tablet0)
if err != nil {
t.Fatalf("Start() err: %v", err)
}
err = Teardown(tablet0)
if err != nil {
t.Fatalf("Teardown(0) err: %v", err)
}
err = Teardown(tablet1)
if err != nil {
t.Fatalf("Teardown(1) err: %v", err)
}
}

102
go/vt/mysqlctl/reparent.go Normal file
Просмотреть файл

@ -0,0 +1,102 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package mysqlctl
import (
"fmt"
"time"
"code.google.com/p/vitess/go/relog"
)
// if the master is still alive, then we need to demote it gracefully
// make it read-only, flush the writes and get the position
func (mysqld *Mysqld) DemoteMaster() (*ReplicationPosition, error) {
// label as TYPE_REPLICA
mysqld.SetReadOnly(true)
cmds := []string{
"FLUSH TABLES WITH READ LOCK",
"UNLOCK TABLES",
}
if err := mysqld.executeSuperQueryList(cmds); err != nil {
return nil, err
}
return mysqld.MasterStatus()
}
/*
replicationState: info slaves need to reparent themselves
waitPosition: slaves can wait for this position when restarting replication
timePromoted: this timestamp (unix nanoseconds) is inserted into _vt.replication_test to verify the replication config
*/
func (mysqld *Mysqld) PromoteSlave() (replicationState *ReplicationState, waitPosition *ReplicationPosition, timePromoted int64, err error) {
cmds := []string{
"STOP SLAVE",
"RESET MASTER",
"RESET SLAVE",
}
if err = mysqld.executeSuperQueryList(cmds); err != nil {
return
}
replicationPosition, err := mysqld.MasterStatus()
if err != nil {
return
}
replicationState = NewReplicationState(mysqld.Addr())
replicationState.ReplicationPosition = *replicationPosition
timePromoted = time.Now().UnixNano()
// write a row to verify that replication is functioning
cmd := fmt.Sprintf("INSERT INTO _vt.replication_test (time_created_ns) VALUES (%v)", timePromoted)
if err = mysqld.executeSuperQuery(cmd); err != nil {
return
}
// this is the wait-point for checking replication
waitPosition, err = mysqld.MasterStatus()
if err != nil {
return
}
err = mysqld.SetReadOnly(false)
return
}
func (mysqld *Mysqld) RestartSlave(replicationState *ReplicationState, waitPosition *ReplicationPosition, timeCheck int64) error {
relog.Info("Restart Slave")
cmds := []string{
"STOP SLAVE",
"RESET SLAVE",
}
cmds = append(cmds, StartReplicationCommands(replicationState)...)
if err := mysqld.executeSuperQueryList(cmds); err != nil {
return err
}
if err := mysqld.WaitForSlaveStart(SlaveStartDeadline); err != nil {
return err
}
cmd := fmt.Sprintf("SELECT MASTER_POS_WAIT('%v', %v)",
waitPosition.MasterLogFile, waitPosition.MasterLogPosition)
if err := mysqld.executeSuperQuery(cmd); err != nil {
return err
}
return mysqld.CheckReplication(timeCheck)
}
// Check for the magic row inserted under controlled reparenting.
func (mysqld *Mysqld) CheckReplication(timeCheck int64) error {
relog.Info("Check Slave")
checkQuery := fmt.Sprintf("SELECT * FROM _vt.replication_test WHERE time_created_ns = %v",
timeCheck)
rows, err := mysqld.fetchSuperQuery(checkQuery)
if err != nil {
return err
}
if len(rows) != 1 {
return fmt.Errorf("replication failed - unexpected row count %v", len(rows))
}
return nil
}

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

@ -0,0 +1,783 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
/*
Handle creating replicas and setting up the replication streams.
*/
package mysqlctl
import (
"bytes"
"code.google.com/p/vitess/go/relog"
"errors"
"fmt"
"io"
"io/ioutil"
"net"
"net/http"
"net/http/httputil"
"net/url"
"os"
"path"
"strconv"
"strings"
"text/template"
"time"
)
const (
SlaveStartDeadline = 30
)
type ReplicationPosition struct {
MasterLogFile string
MasterLogPosition uint
}
func (rp ReplicationPosition) MapKey() string {
return fmt.Sprintf("%v:%d", rp.MasterLogFile, rp.MasterLogPosition)
}
type ReplicationState struct {
// ReplicationPosition is not anonymous because the default json encoder has begun to fail here.
ReplicationPosition ReplicationPosition
MasterHost string
MasterPort int
MasterUser string
MasterPassword string
MasterConnectRetry int
}
// FIXME(msolomon) wrong place for default credentials
func NewReplicationState(masterAddr string) *ReplicationState {
addrPieces := strings.Split(masterAddr, ":")
port, err := strconv.Atoi(addrPieces[1])
if err != nil {
panic(err)
}
return &ReplicationState{MasterUser: "vt_repl", MasterPassword: "", MasterConnectRetry: 10,
MasterHost: addrPieces[0], MasterPort: port}
}
type ReplicaSource struct {
Addr string // this is the address of the tabletserver, not mysql
DbName string
FileList []string
HashList []string // md5 sum of the compressed file
*ReplicationState
}
func NewReplicaSource(addr, mysqlAddr string) *ReplicaSource {
return &ReplicaSource{
Addr: addr,
FileList: make([]string, 0, 256),
HashList: make([]string, 0, 256),
ReplicationState: NewReplicationState(mysqlAddr)}
}
var changeMasterCmd = `CHANGE MASTER TO
MASTER_HOST = '{{.MasterHost}}',
MASTER_PORT = {{.MasterPort}},
MASTER_USER = '{{.MasterUser}}',
MASTER_PASSWORD = '{{.MasterPassword}}',
MASTER_LOG_FILE = '{{.ReplicationPosition.MasterLogFile}}',
MASTER_LOG_POS = {{.ReplicationPosition.MasterLogPosition}},
MASTER_CONNECT_RETRY = {{.MasterConnectRetry}}`
func StartReplicationCommands(replState *ReplicationState) []string {
return []string{
"RESET SLAVE",
mustFillStringTemplate(changeMasterCmd, replState),
"START SLAVE"}
}
// Read replication state from local files.
func ReadReplicationState(mysqld *Mysqld) (*ReplicationState, error) {
relayInfo, err := ioutil.ReadFile(mysqld.config.RelayLogInfoPath())
if err != nil {
return nil, err
}
// FIXME(msolomon) not sure i'll need this data
masterInfo, err := ioutil.ReadFile(mysqld.config.MasterInfoPath())
if err != nil {
return nil, err
}
relayParts := strings.Split(string(relayInfo), " ")
masterParts := strings.Split(string(masterInfo), " ")
// FIXME(msolomon) does the file supply port?
addr := fmt.Sprintf("%v:%v", masterParts[3], 3306)
replState := NewReplicationState(addr)
replState.ReplicationPosition.MasterLogFile = relayParts[2]
temp, _ := strconv.ParseUint(relayParts[3], 10, 0)
replState.ReplicationPosition.MasterLogPosition = uint(temp)
replState.MasterUser = masterParts[4]
replState.MasterPassword = masterParts[5]
return replState, nil
}
func mustFillStringTemplate(tmpl string, vars interface{}) string {
myTemplate := template.Must(template.New("").Parse(tmpl))
data := new(bytes.Buffer)
if err := myTemplate.Execute(data, vars); err != nil {
panic(err)
}
return data.String()
}
func (mysqld *Mysqld) ValidateReplicaSource() error {
rows, err := mysqld.fetchSuperQuery("SHOW PROCESSLIST")
if err != nil {
return err
}
if len(rows) > 8 {
return errors.New("too many processes")
}
slaveStatus, err := mysqld.slaveStatus()
if err != nil {
if err != ERR_NOT_SLAVE {
return err
}
} else {
lagSeconds, _ := strconv.Atoi(slaveStatus["seconds_behind_master"])
if lagSeconds > 5 {
return errors.New("lag_seconds exceed maximum tolerance")
}
}
// FIXME(msolomon) check free space based on an estimate of the current
// size of the db files.
// Also, check that we aren't already cloning/compressing or acting as a
// source. Mysqld being down isn't enough, presumably that will be
// restarted as soon as the snapshot is taken.
return nil
}
func (mysqld *Mysqld) ValidateReplicaTarget() error {
rows, err := mysqld.fetchSuperQuery("SHOW PROCESSLIST")
if err != nil {
return err
}
if len(rows) > 4 {
return errors.New("too many active db processes")
}
rows, err = mysqld.fetchSuperQuery("SHOW DATABASES")
if err != nil {
return err
}
for _, row := range rows {
if strings.HasPrefix(row[0].(string), "vt_") {
dbName := row[0].(string)
tableRows, err := mysqld.fetchSuperQuery("SHOW TABLES FROM " + dbName)
if err != nil {
relog.Error("failed checking tables %v", err)
} else if len(tableRows) == 0 {
// no tables == empty db, all is well
continue
}
return errors.New("found active db named " + dbName)
}
}
return nil
}
func (mysqld *Mysqld) ValidateSplitReplicaTarget() error {
rows, err := mysqld.fetchSuperQuery("SHOW PROCESSLIST")
if err != nil {
return err
}
if len(rows) > 4 {
return errors.New("too many active db processes")
}
rows, err = mysqld.fetchSuperQuery("SHOW DATABASES")
if err != nil {
return err
}
// NOTE: we expect that database was already created during tablet
// assignment.
return nil
}
/*
This piece runs on the machine acting as the source in the create replica
action.
get master/slave status
shutdown mysql
compress /vt/vt_[0-9a-f]+/data/vt_.+
compute md5() sums
place in /vt/clone_src where they will be served by http server (not rpc)
restart mysql
*/
func (mysqld *Mysqld) CreateReplicaSource(dbName, replicaSourcePath, sourceAddr string, allowHierarchicalReplication bool) (_replicaSource *ReplicaSource, err error) {
if dbName == "" {
err = errors.New("no database name provided")
return
}
if err = mysqld.ValidateReplicaSource(); err != nil {
return
}
// FIXME(msolomon) bleh, must match patterns in mycnf - probably belongs
// in there as derived paths.
cloneSourcePath := path.Join(replicaSourcePath, "data", dbName)
cloneInnodbDataSourcePath := path.Join(replicaSourcePath, innodbDataSubdir)
cloneInnodbLogSourcePath := path.Join(replicaSourcePath, innodbLogSubdir)
// clean out and start fresh
for _, _path := range []string{cloneSourcePath, cloneInnodbDataSourcePath, cloneInnodbLogSourcePath} {
if err = os.RemoveAll(_path); err != nil {
return
}
if err = os.MkdirAll(_path, 0775); err != nil {
return
}
}
dbDataDir := path.Join(mysqld.config.DataDir, dbName)
dbFiles, dirErr := ioutil.ReadDir(dbDataDir)
if dirErr != nil {
return nil, dirErr
}
innodbDataFiles, dirErr := ioutil.ReadDir(mysqld.config.InnodbDataHomeDir)
if dirErr != nil {
return nil, dirErr
}
innodbLogFiles, dirErr := ioutil.ReadDir(mysqld.config.InnodbLogGroupHomeDir)
if dirErr != nil {
return nil, dirErr
}
// if the source is a slave use the master replication position,
// unless we are allowing hierachical replicas.
masterAddr := ""
replicationPosition, statusErr := mysqld.SlaveStatus()
if statusErr != nil {
if statusErr != ERR_NOT_SLAVE {
// this is a real error
return nil, statusErr
}
// we are really a master, so we need that position
replicationPosition, statusErr = mysqld.MasterStatus()
if statusErr != nil {
return nil, statusErr
}
masterAddr = mysqld.Addr()
} else {
// we are a slave, check our replication strategy
if allowHierarchicalReplication {
masterAddr = mysqld.Addr()
} else {
masterAddr, err = mysqld.GetMasterAddr()
if err != nil {
return
}
}
}
replicaSource := NewReplicaSource(sourceAddr, masterAddr)
replicaSource.Addr = sourceAddr
replicaSource.DbName = dbName
replicaSource.ReplicationPosition = *replicationPosition
// save initial state so we can restore on Start()
slaveStartRequired := false
if slaveStatus, slaveErr := mysqld.slaveStatus(); slaveErr == nil {
slaveStartRequired = (slaveStatus["Slave_IO_Running"] == "Yes" && slaveStatus["Slave_SQL_Running"] == "Yes")
}
readOnly := true
if readOnly, err = mysqld.IsReadOnly(); err != nil {
return
}
if err = Shutdown(mysqld, true); err != nil {
return
}
// FIXME(msolomon) should mysqld just restart on any failure?
compressFiles := func(filenames []os.FileInfo, srcDir, dstDir string) error {
for _, fileInfo := range filenames {
if !fileInfo.IsDir() {
srcPath := path.Join(srcDir, fileInfo.Name())
dstPath := path.Join(dstDir, fileInfo.Name()+".gz")
if compressErr := compressFile(srcPath, dstPath); err != nil {
return compressErr
}
hash, hashErr := md5File(dstPath)
if hashErr != nil {
return hashErr
}
replicaSource.FileList = append(replicaSource.FileList, dstPath)
replicaSource.HashList = append(replicaSource.HashList, hash)
relog.Info("%v:%v ready", dstPath, hash)
}
}
return nil
}
if err = compressFiles(dbFiles, dbDataDir, cloneSourcePath); err != nil {
return
}
if err = compressFiles(innodbDataFiles, mysqld.config.InnodbDataHomeDir, cloneInnodbDataSourcePath); err != nil {
return
}
if err = compressFiles(innodbLogFiles, mysqld.config.InnodbLogGroupHomeDir, cloneInnodbLogSourcePath); err != nil {
return
}
if err = Start(mysqld); err != nil {
return
}
// restore original mysqld state that we saved above
if slaveStartRequired {
if err = mysqld.StartSlave(); err != nil {
return
}
// this should be quick, but we might as well just wait
if err = mysqld.WaitForSlaveStart(5); err != nil {
return
}
}
if err = mysqld.SetReadOnly(readOnly); err != nil {
return
}
// ok, copy over the pointer on success
_replicaSource = replicaSource
return
}
/*
This piece runs on the presumably empty machine acting as the target in the
create replica action.
validate target (self)
shutdown_mysql()
create temp data directory /vt/target/vt_<keyspace>
copy compressed data files via HTTP
verify md5sum of compressed files
uncompress into /vt/vt_<target-uid>/data/vt_<keyspace>
start_mysql()
clean up compressed files
*/
func (mysqld *Mysqld) CreateReplicaTarget(replicaSource *ReplicaSource, tempStoragePath string) (err error) {
if replicaSource == nil {
return errors.New("nil replicaSource")
}
if err = mysqld.ValidateReplicaTarget(); err != nil {
return
}
if err = Shutdown(mysqld, true); err != nil {
return
}
replicaDbPath := path.Join(mysqld.config.DataDir, replicaSource.DbName)
cleanDirs := []string{tempStoragePath, replicaDbPath, mysqld.config.InnodbDataHomeDir, mysqld.config.InnodbLogGroupHomeDir}
// clean out and start fresh
// FIXME(msolomon) this might be changed to allow partial recovery
for _, dir := range cleanDirs {
if err = os.RemoveAll(dir); err != nil {
return
}
if err = os.MkdirAll(dir, 0775); err != nil {
return
}
}
httpConn, connErr := net.Dial("tcp", replicaSource.Addr)
if connErr != nil {
return connErr
}
defer httpConn.Close()
fileClient := httputil.NewClientConn(httpConn, nil)
defer fileClient.Close()
// FIXME(msolomon) parallelize
// FIXME(msolomon) pull out simple URL fetch?
// FIXME(msolomon) automatically retry a file transfer at least once
// FIXME(msolomon) deadlines?
for i, srcPath := range replicaSource.FileList {
srcHash := replicaSource.HashList[i]
urlstr := "http://" + replicaSource.Addr + srcPath
urlobj, parseErr := url.Parse(urlstr)
if parseErr != nil {
return errors.New("failed to create url " + urlstr)
}
req := &http.Request{Method: "GET",
Host: replicaSource.Addr,
URL: urlobj}
err = fileClient.Write(req)
if err != nil {
return errors.New("failed requesting " + urlstr)
}
var response *http.Response
response, err = fileClient.Read(req)
if err != nil {
return errors.New("failed fetching " + urlstr)
}
if response.StatusCode != 200 {
return errors.New("failed fetching " + urlstr + ": " + response.Status)
}
relativePath := strings.SplitN(srcPath, "/", 5)[4]
gzFilename := path.Join(tempStoragePath, relativePath)
filename := path.Join(mysqld.config.TabletDir, relativePath)
// trim .gz
filename = filename[:len(filename)-3]
dir, _ := path.Split(gzFilename)
if dirErr := os.MkdirAll(dir, 0775); dirErr != nil {
return dirErr
}
// FIXME(msolomon) buffer output?
file, fileErr := os.OpenFile(gzFilename, os.O_CREATE|os.O_WRONLY, 0664)
if fileErr != nil {
return fileErr
}
defer file.Close()
_, err = io.Copy(file, response.Body)
if err != nil {
return
}
file.Close()
hash, hashErr := md5File(gzFilename)
if hashErr != nil {
return hashErr
}
if srcHash != hash {
return errors.New("hash mismatch for " + gzFilename + ", " + srcHash + " != " + hash)
}
if err = uncompressFile(gzFilename, filename); err != nil {
return
}
if err = os.Remove(gzFilename); err != nil {
// don't stop the process for this error
relog.Error("failed to remove %v", gzFilename)
}
relog.Info("%v ready", filename)
}
if err = Start(mysqld); err != nil {
return
}
cmdList := StartReplicationCommands(replicaSource.ReplicationState)
relog.Info("StartReplicationCommands %#v", cmdList)
if err = mysqld.executeSuperQueryList(cmdList); err != nil {
return
}
err = mysqld.WaitForSlaveStart(SlaveStartDeadline)
return
}
func (mysqld *Mysqld) WaitForSlaveStart(slaveStartDeadline int) (err error) {
var rowMap map[string]string
for slaveWait := 0; slaveWait < slaveStartDeadline; slaveWait++ {
rowMap, err = mysqld.slaveStatus()
if err != nil {
return
}
if rowMap["Slave_IO_Running"] == "Yes" && rowMap["Slave_SQL_Running"] == "Yes" {
return nil
}
time.Sleep(1e9)
}
errorKeys := []string{"Last_Error", "Last_IO_Error", "Last_SQL_Error"}
errs := make([]string, 0, len(errorKeys))
for _, key := range errorKeys {
if rowMap[key] != "" {
errs = append(errs, key+": "+rowMap[key])
}
}
if len(errs) != 0 {
return errors.New(strings.Join(errs, ", "))
}
return nil
}
func (mysqld *Mysqld) StartSlave() error {
return mysqld.executeSuperQuery("SLAVE START")
}
func (mysqld *Mysqld) StopSlave() error {
return mysqld.executeSuperQuery("SLAVE STOP")
}
func (mysqld *Mysqld) GetMasterAddr() (string, error) {
slaveStatus, err := mysqld.slaveStatus()
if err != nil {
return "", err
}
masterAddr := slaveStatus["Master_Host"] + ":" + slaveStatus["Master_Port"]
return masterAddr, nil
}
func (mysqld *Mysqld) IsReadOnly() (bool, error) {
rows, err := mysqld.fetchSuperQuery("SHOW VARIABLES LIKE 'read_only'")
if err != nil {
return true, err
}
if len(rows) != 1 {
return true, errors.New("no read_only variable in mysql")
}
if rows[0][1].(string) == "ON" {
return true, nil
}
return false, nil
}
func (mysqld *Mysqld) SetReadOnly(on bool) error {
query := "SET GLOBAL read_only = "
if on {
query += "ON"
} else {
query += "OFF"
}
return mysqld.executeSuperQuery(query)
}
var ERR_NOT_SLAVE = errors.New("no slave status")
func (mysqld *Mysqld) slaveStatus() (map[string]string, error) {
rows, err := mysqld.fetchSuperQuery("SHOW SLAVE STATUS")
if err != nil {
return nil, err
}
if len(rows) != 1 {
return nil, ERR_NOT_SLAVE
}
rowMap := make(map[string]string)
for i, column := range rows[0] {
if column == nil {
rowMap[showSlaveStatusColumnNames[i]] = ""
} else {
rowMap[showSlaveStatusColumnNames[i]] = column.(string)
}
}
return rowMap, nil
}
func (mysqld *Mysqld) SlaveStatus() (*ReplicationPosition, error) {
fields, err := mysqld.slaveStatus()
if err != nil {
return nil, err
}
pos := new(ReplicationPosition)
pos.MasterLogFile = fields["Relay_Master_Log_File"]
temp, _ := strconv.ParseUint(fields["Exec_Master_Log_Pos"], 10, 0)
pos.MasterLogPosition = uint(temp)
return pos, nil
}
/*
mysql> show master status\G
**************************** 1. row ***************************
File: vt-000001c6-bin.000003
Position: 106
Binlog_Do_DB:
Binlog_Ignore_DB:
*/
func (mysqld *Mysqld) MasterStatus() (position *ReplicationPosition, err error) {
rows, err := mysqld.fetchSuperQuery("SHOW MASTER STATUS")
if err != nil {
return
}
if len(rows) != 1 {
err = errors.New("unexpected result for show master status")
return
}
position = &ReplicationPosition{}
position.MasterLogFile = rows[0][0].(string)
temp, err := strconv.ParseUint(rows[0][1].(string), 10, 0)
position.MasterLogPosition = uint(temp)
return
}
func (mysqld *Mysqld) WaitForSlave(maxLag int) (err error) {
// FIXME(msolomon) verify that slave started based on show slave status;
var rowMap map[string]string
for {
rowMap, err = mysqld.slaveStatus()
if err != nil {
return
}
if rowMap["Seconds_Behind_Master"] == "NULL" {
break
} else {
lag, err := strconv.Atoi(rowMap["Seconds_Behind_Master"])
if err != nil {
break
}
if lag < maxLag {
return nil
}
}
time.Sleep(1e9)
}
errorKeys := []string{"Last_Error", "Last_IO_Error", "Last_SQL_Error"}
errs := make([]string, 0, len(errorKeys))
for _, key := range errorKeys {
if rowMap[key] != "" {
errs = append(errs, key+": "+rowMap[key])
}
}
if len(errs) != 0 {
return errors.New(strings.Join(errs, ", "))
}
return errors.New("replication stopped, it will never catch up")
}
/*
Slave_IO_State: Waiting for master to send event
Master_Host: voltron
Master_User: vt_repl
Master_Port: 6600
Connect_Retry: 10
Master_Log_File: vt-00000001-bin.000002
Read_Master_Log_Pos: 106
Relay_Log_File: vt-00000002-relay-bin.000003
Relay_Log_Pos: 257
Relay_Master_Log_File: vt-00000001-bin.000002
Slave_IO_Running: Yes
Slave_SQL_Running: Yes
Replicate_Do_DB:
Replicate_Ignore_DB:
Replicate_Do_Table:
Replicate_Ignore_Table:
Replicate_Wild_Do_Table:
Replicate_Wild_Ignore_Table:
Last_Errno: 0
Last_Error:
Skip_Counter: 0
Exec_Master_Log_Pos: 106
Relay_Log_Space: 569
Until_Condition: None
Until_Log_File:
Until_Log_Pos: 0
Master_SSL_Allowed: No
Master_SSL_CA_File:
Master_SSL_CA_Path:
Master_SSL_Cert:
Master_SSL_Cipher:
Master_SSL_Key:
Seconds_Behind_Master: 0
Master_SSL_Verify_Server_Cert: No
Last_IO_Errno: 0
Last_IO_Error:
Last_SQL_Errno: 0
Last_SQL_Error:
*/
var showSlaveStatusColumnNames = []string{
"Slave_IO_State",
"Master_Host",
"Master_User",
"Master_Port",
"Connect_Retry",
"Master_Log_File",
"Read_Master_Log_Pos",
"Relay_Log_File",
"Relay_Log_Pos",
"Relay_Master_Log_File",
"Slave_IO_Running",
"Slave_SQL_Running",
"Replicate_Do_DB",
"Replicate_Ignore_DB",
"Replicate_Do_Table",
"Replicate_Ignore_Table",
"Replicate_Wild_Do_Table",
"Replicate_Wild_Ignore_Table",
"Last_Errno",
"Last_Error",
"Skip_Counter",
"Exec_Master_Log_Pos",
"Relay_Log_Space",
"Until_Condition",
"Until_Log_File",
"Until_Log_Pos",
"Master_SSL_Allowed",
"Master_SSL_CA_File",
"Master_SSL_CA_Path",
"Master_SSL_Cert",
"Master_SSL_Cipher",
"Master_SSL_Key",
"Seconds_Behind_Master",
"Master_SSL_Verify_Server_Cert",
"Last_IO_Errno",
"Last_IO_Error",
"Last_SQL_Errno",
"Last_SQL_Error",
}
func (mysqld *Mysqld) executeSuperQuery(query string) error {
return mysqld.executeSuperQueryList([]string{query})
}
// FIXME(msolomon) should there be a query lock so we only
// run one admin action at a time?
func (mysqld *Mysqld) fetchSuperQuery(query string) ([][]interface{}, error) {
conn, connErr := mysqld.createConnection()
if connErr != nil {
return nil, connErr
}
defer conn.Close()
relog.Info("fetch %v", query)
qr, err := conn.ExecuteFetch([]byte(query), 10000, false)
if err != nil {
return nil, err
}
return qr.Rows, nil
}
func (mysqld *Mysqld) executeSuperQueryList(queryList []string) error {
conn, connErr := mysqld.createConnection()
if connErr != nil {
return connErr
}
defer conn.Close()
for _, query := range queryList {
relog.Info("exec %v", query)
if _, err := conn.ExecuteFetch([]byte(query), 10000, false); err != nil {
return err
}
}
return nil
}
func (mysqld *Mysqld) ConfigureKeyRange(startKey string, endKey string) error {
replicationCmds := []string{
"SET GLOBAL vt_enable_binlog_splitter_rbr = 1",
"SET GLOBAL vt_shard_key_range_start = \"0x" + startKey + "\"",
"SET GLOBAL vt_shard_key_range_end = \"0x" + endKey + "\""}
if err := mysqld.executeSuperQueryList(replicationCmds); err != nil {
return err
}
return nil
}
func (mysqld *Mysqld) ResetKeyRange() error {
replicationCmds := []string{
"SET GLOBAL vt_shard_key_range_start = \"\"",
"SET GLOBAL vt_shard_key_range_end = \"\""}
if err := mysqld.executeSuperQueryList(replicationCmds); err != nil {
return err
}
return nil
}

472
go/vt/mysqlctl/split.go Normal file
Просмотреть файл

@ -0,0 +1,472 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package mysqlctl
// FIXME(msolomon) this actions were copy/pasted from replication.go because
// they were conceptually quite similar. They should be reconciled at some
// point.
/*
Given a single shard, split into 2 subshards, each addressing some subset of the total key ranges.
T is the tablet server controlling M
R is the entity_id key range that T handles
M is the master mysql db
S is the stemcell mysql slave, which takes no normal traffic (does this have a tablet server?)
M', M" are new master db's, each of which will have some subset of the key range of M
S', S" are new stemcell db's, each of which will have some number of slaves
T', T" are the corresponding tablet servers for M'/M"
Assume masters take a significant amount of read traffic (unlike EMD).
Resharding may be implemented as a subprocess from the tablet server that communicates back over a netchan. This will make it easier to patch without taking down the tablet server.
Acquire machine resources (M'/M", S'/S", ...)
2*M + 2*S + min((N+X), 2*min # of replicas) + (2 * Lag)
N is replica count local to M
X is replicas outside of M's datacenter
Laggards are optional (but probably good)
The global minimum for replicas per shard is ~3 for durability and the ability to clone while you are online serving queries.
Install/init tablet server processes T'/T"
Install/init mysql on M'/M"
SET GLOBAL read_only = 1;
does this allow replication to proceed?
what about commands issued by SUPER?
Arrange replication layout amongst new instances
If there are latency/geographic considerations, this is where they manifest themselves. In general, the stemcells will be the source of the replication streams. Each geographic area should have a stemcell which acts as the parent for all other slaves in that area. The local stemcell should slave from the master's stemcell. It should look like a shrub more than a tree.
Alternatively, this layout can be used for an initial copy of the table dumps. After the initial data load, the replication streams can be set up. This might be faster, but is likely to be more complex to manage.
Apply baseline schema
turn off indexes to increase throughput? can't do this on InnoDB
Stop replication on stemcell S
Record replication position on S for M' and M"
Given two key ranges, R' and R" set the replication key range on M' and M"
this requires modifications to mysql replication which I have made in the past to be redone
This should be fixable to row-based replication as well.
For each table on S, export subranges to M' and M":
SELECT * FROM table WHERE R'.start <= id AND id < R'.end
SELECT * FROM table WHERE R".start <= id AND id < R".end
Feed dump query streams in M' and M" respectively
use some sort of SELECT INTO..., LOAD FROM... to optimize?
use some sort of COMMIT buffering to optimize?
disable AUTOCOMMIT
SET UNIQUE_CHECKS=0; do some stuff; SET UNIQUE_CHECKS=1;
use the tablet server to compress or do data-only instead of sending full SQL commands
will single replication threads handle the inserts fast enough downstream of S' and S"?
Once the bulk export is complete, restart replication on S.
Once the bulk import is complete, rebuild tables? (might not be necessary since data is sequential)
Reparent M' and M" to S
set the key range that replication will accept
Start splitting replication on M' and M"
Wait for M'/M" to catch up to S (implying caught up to M)
Wait for S'x and S"x hosts (mysql instances slaved from the new stemcells) to catch up to M'/M".
S'Lag and S"Lag (24 hour lag hosts) will not be 24 hrs behind for 23+ hrs
Writes can now be shunted from M to M'/M"
writes are likely to be warm from replication
reads will be cold since there is no traffic going to the T'/T" - the row cache is empty
row cache could be warmed, but the invalidation is tricky if you are allowing writes
8GB of cache will take 120 seconds to transfer, even if you can nearly max out the 1Gb port to an adjacent machine
if shards are small, this might not be a big deal
Start failing writes on T, report that T split to smart clients.
SET GLOBAL read_only = 1 on M to prevent ghost writes.
Set T to refuse new connections (read or write)
Disconnect replication on M'/M" from S.
SET GLOBAL read_only = 0 on M'/M" to allow new writes.
Update table wrangler and reassign R'/R" to T'/T".
T disconnects reading clients and shutsdown mysql.
How aggressively can we do this? The faster the better.
Garbage collect the hosts.
leave the 24 lag for 1 day
*/
import (
"encoding/base64"
"errors"
"io"
"net"
"net/http"
"net/http/httputil"
"net/url"
"os"
"path"
"strconv"
"strings"
"code.google.com/p/vitess/go/relog"
)
type SplitReplicaSource struct {
*ReplicaSource
StartKey string
EndKey string
Schema string
}
func NewSplitReplicaSource(addr, mysqlAddr string) *SplitReplicaSource {
return &SplitReplicaSource{ReplicaSource: NewReplicaSource(addr, mysqlAddr)}
}
// FIXME(msolomon) use query format/bind vars
var selectIntoOutfile = `SELECT * INTO OUTFILE :tableoutputpath
CHARACTER SET binary
FIELDS TERMINATED BY ',' OPTIONALLY ENCLOSED BY '"' ESCAPED BY '\\'
LINES TERMINATED BY '\n'
FROM {{.TableName}} WHERE {{.KeyspaceIdColumnName}} >= :startkey AND
{{.KeyspaceIdColumnName}} < :endkey`
var loadDataInfile = `LOAD DATA INFILE '{{.TableInputPath}}' INTO TABLE {{.TableName}}
CHARACTER SET binary
FIELDS TERMINATED BY ',' OPTIONALLY ENCLOSED BY '"' ESCAPED BY '\\'
LINES TERMINATED BY '\n'`
func b64ForFilename(s string) string {
return strings.Replace(base64.URLEncoding.EncodeToString([]byte(s)), "=", "", -1)
}
/*
copied from replication.
create a series of raw dump files the contain rows to be reinserted
dbName - mysql db name
keyName - name of the mysql column that is the leading edge of all primary keys
startKey, endKey - the row range to prepare
replicaSourcePath - where to copy the output data
sourceAddr - the ip addr of the machine running the export
*/
func (mysqld *Mysqld) CreateSplitReplicaSource(dbName, keyName, startKey, endKey, replicaSourcePath, sourceAddr string) (_replicaSource *SplitReplicaSource, err error) {
if dbName == "" {
err = errors.New("no database name provided")
return
}
// same logic applies here
relog.Info("ValidateReplicaSource")
if err = mysqld.ValidateReplicaSource(); err != nil {
return
}
// FIXME(msolomon) bleh, must match patterns in mycnf - probably belongs
// in there as derived paths.
cloneSourcePath := path.Join(replicaSourcePath, "data", dbName+"-"+b64ForFilename(startKey)+","+b64ForFilename(endKey))
// clean out and start fresh
for _, _path := range []string{cloneSourcePath} {
if err = os.RemoveAll(_path); err != nil {
return
}
if err = os.MkdirAll(_path, 0775); err != nil {
return
}
}
// get a list of tables to process
rows, fetchErr := mysqld.fetchSuperQuery("SHOW TABLES")
if fetchErr != nil {
return nil, fetchErr
}
if len(rows) == 0 {
return nil, errors.New("empty table list")
}
tableNames := make([]string, len(rows))
for i, row := range rows {
tableNames[i] = row[0].(string)
}
relog.Info("Fetch Tables: %#v %#v", rows, tableNames)
/*
mysql> show master status\G
**************************** 1. row ***************************
File: vt-000001c6-bin.000003
Position: 106
Binlog_Do_DB:
Binlog_Ignore_DB:
*/
// FIXME(msolomon) handle both master and slave situtation
rows, fetchErr = mysqld.fetchSuperQuery("SHOW MASTER STATUS")
if fetchErr != nil {
return nil, fetchErr
}
if len(rows) != 1 {
return nil, errors.New("unexpected result for show master status")
}
relog.Info("Save Master Status")
replicaSource := NewSplitReplicaSource(sourceAddr, mysqld.Addr())
replicaSource.DbName = dbName
replicaSource.ReplicationPosition.MasterLogFile = rows[0][0].(string)
temp, _ := strconv.ParseUint(rows[0][1].(string), 10, 0)
replicaSource.ReplicationPosition.MasterLogPosition = uint(temp)
// save initial state so we can restore on Start()
slaveStartRequired := false
if slaveStatus, slaveErr := mysqld.slaveStatus(); slaveErr == nil {
slaveStartRequired = (slaveStatus["Slave_IO_Running"] == "Yes" && slaveStatus["Slave_SQL_Running"] == "Yes")
}
readOnly := true
if readOnly, err = mysqld.IsReadOnly(); err != nil {
return
}
relog.Info("Set Read Only")
if !readOnly {
mysqld.SetReadOnly(true)
}
relog.Info("Stop Slave")
if err = mysqld.StopSlave(); err != nil {
return
}
relog.Info("Flush tables")
if err = mysqld.executeSuperQuery("FLUSH TABLES WITH READ LOCK"); err != nil {
return
}
// export each table to a CSV-like file, compress the results
tableFiles := make([]string, len(tableNames))
// FIXME(msolomon) parallelize
for i, tableName := range tableNames {
relog.Info("Dump table %v...", tableName)
filename := path.Join(cloneSourcePath, tableName+".csv")
tableFiles[i] = filename
queryParams := map[string]string{
"TableName": tableName,
"KeyspaceIdColumnName": keyName,
}
// FIXME(sougou/msolomon): no bindparams for the new mysql module
/*bindParams := map[string]interface{}{
"tableoutputpath": filename,
"startkey": startKey,
"endkey": endKey,
}*/
query := mustFillStringTemplate(selectIntoOutfile, queryParams)
relog.Info(" %v", query)
if err = mysqld.executeSuperQuery(query); err != nil {
// FIXME(msolomon) on abort, should everything go back the way it was?
// alternatively, we could just leave it and wait for the wrangler to
// notice and start cleaning up
return
}
}
// FIXME(msolomon) should mysqld just restart on any failure?
compressFiles := func(filenames []string) error {
for _, srcPath := range filenames {
dstPath := srcPath + ".gz"
if err := compressFile(srcPath, dstPath); err != nil {
return err
}
// prune files to free up disk space, if it errors, we'll figure out
// later
os.Remove(srcPath)
hash, hashErr := md5File(dstPath)
if hashErr != nil {
return hashErr
}
replicaSource.FileList = append(replicaSource.FileList, dstPath)
replicaSource.HashList = append(replicaSource.HashList, hash)
relog.Info("%v:%v ready", dstPath, hash)
}
return nil
}
// FIXME(msolomon) at some point, you could pipeline requests for speed
if err = compressFiles(tableFiles); err != nil {
return
}
if err = mysqld.executeSuperQuery("UNLOCK TABLES"); err != nil {
return
}
// restore original mysqld state that we saved above
if slaveStartRequired {
if err = mysqld.StartSlave(); err != nil {
return
}
// this should be quick, but we might as well just wait
if err = mysqld.WaitForSlaveStart(5); err != nil {
return
}
}
if err = mysqld.SetReadOnly(readOnly); err != nil {
return
}
// ok, copy over the pointer on success
_replicaSource = replicaSource
relog.Info("mysqld replicaSource %#v", replicaSource)
return
}
/*
This piece runs on the presumably empty machine acting as the target in the
create replica action.
validate target (self)
shutdown_mysql()
create temp data directory /vt/target/vt_<keyspace>
copy compressed data files via HTTP
verify md5sum of compressed files
uncompress into /vt/vt_<target-uid>/data/vt_<keyspace>
start_mysql()
clean up compressed files
*/
func (mysqld *Mysqld) CreateSplitReplicaTarget(replicaSource *SplitReplicaSource, tempStoragePath string) (err error) {
if err = mysqld.ValidateSplitReplicaTarget(); err != nil {
return
}
cleanDirs := []string{tempStoragePath}
// clean out and start fresh
// FIXME(msolomon) this might be changed to allow partial recovery
for _, dir := range cleanDirs {
if err = os.RemoveAll(dir); err != nil {
return
}
if err = os.MkdirAll(dir, 0775); err != nil {
return
}
}
if err = mysqld.SetReadOnly(true); err != nil {
return
}
// we could conditionally create the database, but this helps us
// verify that other parts of the process are working
createDbCmds := []string{
// "CREATE DATABASE " + replicaSource.DbName + " IF NOT EXISTS",
"USE " + replicaSource.DbName}
for _, cmd := range strings.Split(replicaSource.Schema, ";") {
cmd = strings.TrimSpace(cmd)
if cmd == "" {
continue
}
createDbCmds = append(createDbCmds, cmd)
}
// FIXME(msolomon) make sure this works with multiple tables
if err = mysqld.executeSuperQueryList(createDbCmds); err != nil {
return
}
httpConn, connErr := net.Dial("tcp", replicaSource.Addr)
if connErr != nil {
return connErr
}
defer httpConn.Close()
fileClient := httputil.NewClientConn(httpConn, nil)
defer fileClient.Close()
// FIXME(msolomon) parallelize
// FIXME(msolomon) pull out simple URL fetch?
// FIXME(msolomon) automatically retry a file transfer at least once
// FIXME(msolomon) deadlines?
// FIXME(msolomon) work into replication.go
for i, srcPath := range replicaSource.FileList {
srcHash := replicaSource.HashList[i]
urlstr := "http://" + replicaSource.Addr + srcPath
urlobj, parseErr := url.Parse(urlstr)
if parseErr != nil {
return errors.New("failed to create url " + urlstr)
}
req := &http.Request{Method: "GET",
Host: replicaSource.Addr,
URL: urlobj}
err = fileClient.Write(req)
if err != nil {
return errors.New("failed requesting " + urlstr)
}
var response *http.Response
response, err = fileClient.Read(req)
if err != nil {
return errors.New("failed fetching " + urlstr)
}
if response.StatusCode != 200 {
return errors.New("failed fetching " + urlstr + ": " + response.Status)
}
relativePath := strings.SplitN(srcPath, "/", 5)[4]
gzFilename := path.Join(tempStoragePath, relativePath)
// trim .gz
filename := gzFilename[:len(gzFilename)-3]
dir, _ := path.Split(gzFilename)
if dirErr := os.MkdirAll(dir, 0775); dirErr != nil {
return dirErr
}
// FIXME(msolomon) buffer output?
file, fileErr := os.OpenFile(gzFilename, os.O_CREATE|os.O_WRONLY, 0664)
if fileErr != nil {
return fileErr
}
defer file.Close()
_, err = io.Copy(file, response.Body)
if err != nil {
return
}
file.Close()
hash, hashErr := md5File(gzFilename)
if hashErr != nil {
return hashErr
}
if srcHash != hash {
return errors.New("hash mismatch for " + gzFilename + ", " + srcHash + " != " + hash)
}
if err = uncompressFile(gzFilename, filename); err != nil {
return
}
if err = os.Remove(gzFilename); err != nil {
// don't stop the process for this error
relog.Info("failed to remove %v", gzFilename)
}
tableName := strings.Replace(path.Base(filename), ".csv", "", -1)
queryParams := map[string]string{
"TableInputPath": filename,
"TableName": replicaSource.DbName + "." + tableName,
}
query := mustFillStringTemplate(loadDataInfile, queryParams)
if err = mysqld.executeSuperQuery(query); err != nil {
// FIXME(msolomon) on abort, we should just tear down
// alternatively, we could just leave it and wait for the wrangler to
// notice and start cleaning up
return
}
relog.Info("%v ready", filename)
}
// FIXME(msolomon) start *split* replication, you need the new start/end
// keys
cmdList := StartSplitReplicationCommands(replicaSource.ReplicationState, replicaSource.StartKey, replicaSource.EndKey)
relog.Info("StartSplitReplicationCommands %#v", cmdList)
if err = mysqld.executeSuperQueryList(cmdList); err != nil {
return
}
err = mysqld.WaitForSlaveStart(SlaveStartDeadline)
if err != nil {
return
}
// ok, now that replication is under way, wait for us to be caught up
if err = mysqld.WaitForSlave(5); err != nil {
return
}
// don't set readonly until the rest of the system is ready
return
}
func StartSplitReplicationCommands(replState *ReplicationState, startKey string, endKey string) []string {
return []string{
"SET GLOBAL vt_enable_binlog_splitter_rbr = 1",
"SET GLOBAL vt_shard_key_range_start = \"" + startKey + "\"",
"SET GLOBAL vt_shard_key_range_end = \"" + endKey + "\"",
"RESET SLAVE",
mustFillStringTemplate(changeMasterCmd, replState),
"START SLAVE"}
}

15
go/vt/rpc/empty.go Normal file
Просмотреть файл

@ -0,0 +1,15 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package rpc
// RPC structs shared between many components
type UnusedRequest string
var NilRequest = new(UnusedRequest)
type UnusedResponse string
var NilResponse = new(UnusedResponse)

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

@ -0,0 +1,50 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
/*
Actions modify the state of a tablet, shard or keyspace.
*/
package tabletmanager
import (
"encoding/json"
)
const (
TABLET_ACTION_PING = "Ping"
TABLET_ACTION_SET_RDONLY = "SetReadOnly"
TABLET_ACTION_SET_RDWR = "SetReadWrite"
TABLET_ACTION_CHANGE_TYPE = "ChangeType"
TABLET_ACTION_DEMOTE_MASTER = "DemoteMaster"
TABLET_ACTION_PROMOTE_SLAVE = "PromoteSlave"
TABLET_ACTION_RESTART_SLAVE = "RestartSlave"
TABLET_ACTION_SCRAP = "Scrap"
// Shard actions - involve all tablets in a shard
SHARD_ACTION_REPARENT = "ReparentShard"
)
type ActionNode struct {
Action string
ActionGuid string
Error string
Args map[string]string
path string // path in zookeeper representing this action
}
func ActionNodeFromJson(data, path string) (*ActionNode, error) {
node := &ActionNode{Args: make(map[string]string)}
err := json.Unmarshal([]byte(data), node)
if err != nil {
return nil, err
}
node.path = path
return node, nil
}
func ActionNodeToJson(n *ActionNode) string {
return toJson(n)
}

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

@ -0,0 +1,367 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package tabletmanager
import (
"encoding/json"
"fmt"
"path"
"code.google.com/p/vitess.x/go/vt/mysqlctl"
"code.google.com/p/vitess.x/go/zk"
"code.google.com/p/vitess/go/relog"
"launchpad.net/gozk/zookeeper"
)
/*
The actor applies individual commands to execute an action read
from a node in zookeeper. Anything that modifies the state of the
table should be applied by this code.
The actor signals completion by removing the action node from zookeeper.
Errors are written to the action node and must (currently) be resolved
by hand using zk tools.
*/
type TabletActorError string
func (e TabletActorError) Error() string {
return string(e)
}
type TabletActor struct {
mysqld *mysqlctl.Mysqld
zconn zk.Conn
zkTabletPath string
zkVtRoot string
}
func NewTabletActor(mysqld *mysqlctl.Mysqld, zconn zk.Conn) *TabletActor {
return &TabletActor{mysqld, zconn, "", ""}
}
// FIXME(msolomon) protect against unforeseen panics and classify errors as "fatal" or
// resolvable. For instance, if your zk connection fails, better to just fail. If data
// is corrupt, you can't fix it gracefully.
func (ta *TabletActor) HandleAction(actionPath, action, actionGuid string) error {
data, _, zkErr := ta.zconn.Get(actionPath)
if zkErr != nil {
relog.Error("HandleAction failed: %v", zkErr)
return zkErr
}
actionNode, err := ActionNodeFromJson(data, actionPath)
if err != nil {
relog.Error("HandleAction failed unmarshaling %v: %v", actionPath, err)
return err
}
ta.zkTabletPath = TabletPathFromActionPath(actionPath)
ta.zkVtRoot = VtRootFromTabletPath(ta.zkTabletPath)
relog.Info("HandleAction: %v %v", actionPath, data)
// validate actions, but don't write this back into zk
if actionNode.Action != action || actionNode.ActionGuid != actionGuid {
relog.Error("HandleAction validation failed %v: (%v,%v) (%v,%v)",
actionPath, actionNode.Action, action, actionNode.ActionGuid, actionGuid)
return TabletActorError("invalid action initiation: " + action + " " + actionGuid)
}
actionErr := ta.dispatchAction(actionNode)
if actionErr != nil {
// on failure, set an error field on the node - let other tools deal
// with it.
actionNode.Error = actionErr.Error()
} else {
actionNode.Error = ""
}
newData := ActionNodeToJson(actionNode)
if newData != data {
_, zkErr = ta.zconn.Set(actionPath, newData, -1)
if zkErr != nil {
relog.Error("HandleAction failed writing: %v", zkErr)
return zkErr
}
}
if actionErr != nil {
return actionErr
} else {
// remove from zk on success
zkErr = ta.zconn.Delete(actionPath, -1)
if zkErr != nil {
relog.Error("HandleAction failed deleting: %v", zkErr)
return zkErr
}
}
return nil
}
func (ta *TabletActor) dispatchAction(actionNode *ActionNode) (err error) {
defer func() {
if x := recover(); x != nil {
err = x.(error)
}
}()
switch actionNode.Action {
case TABLET_ACTION_PING:
// Just an end-to-end verification that we got the message.
err = nil
case TABLET_ACTION_SET_RDONLY:
err = ta.setReadOnly(true)
case TABLET_ACTION_SET_RDWR:
err = ta.setReadOnly(false)
case TABLET_ACTION_CHANGE_TYPE:
err = ta.changeType(actionNode.Args)
case TABLET_ACTION_DEMOTE_MASTER:
err = ta.demoteMaster()
case TABLET_ACTION_PROMOTE_SLAVE:
err = ta.promoteSlave(actionNode.Args)
case TABLET_ACTION_RESTART_SLAVE:
err = ta.restartSlave(actionNode.Args)
case TABLET_ACTION_SCRAP:
err = ta.scrap()
default:
err = TabletActorError("invalid action: " + actionNode.Action)
}
return
}
func (ta *TabletActor) setReadOnly(rdonly bool) error {
err := ta.mysqld.SetReadOnly(rdonly)
if err != nil {
return err
}
tablet, err := ReadTablet(ta.zconn, ta.zkTabletPath)
if err != nil {
return err
}
if rdonly {
tablet.State = STATE_READ_ONLY
} else {
tablet.State = STATE_READ_WRITE
}
return UpdateTablet(ta.zconn, ta.zkTabletPath, tablet)
}
func (ta *TabletActor) changeType(args map[string]string) error {
dbType, ok := args["DbType"]
if !ok {
return fmt.Errorf("missing DbType in args")
}
return ChangeType(ta.zconn, ta.zkTabletPath, TabletType(dbType))
}
func (ta *TabletActor) demoteMaster() error {
_, err := ta.mysqld.DemoteMaster()
if err != nil {
return err
}
tablet, err := ReadTablet(ta.zconn, ta.zkTabletPath)
if err != nil {
return err
}
tablet.State = STATE_READ_ONLY
// NOTE(msolomon) there is no serving graph update - the master tablet will
// be replaced. Even though writes may fail, reads will succeed. It will be
// less noisy to simply leave the entry until well promote the master.
return UpdateTablet(ta.zconn, ta.zkTabletPath, tablet)
}
type RestartSlaveData struct {
ReplicationState *mysqlctl.ReplicationState
WaitPosition *mysqlctl.ReplicationPosition
TimePromoted int64 // used to verify replication - a row will be inserted with this timestamp
Parent TabletAlias
Force bool
}
func (ta *TabletActor) promoteSlave(args map[string]string) error {
zkShardActionPath, ok := args["ShardActionPath"]
if !ok {
return fmt.Errorf("missing ShardActionPath in args")
}
tablet, err := ReadTablet(ta.zconn, ta.zkTabletPath)
if err != nil {
return err
}
zkRestartSlaveDataPath := path.Join(zkShardActionPath, "restart_slave_data.json")
// The presence of this node indicates that the promote action succeeded.
stat, err := ta.zconn.Exists(zkRestartSlaveDataPath)
if stat != nil {
err = fmt.Errorf("restart_slave_data.json already exists - suspicious")
}
if err != nil {
return err
}
// No slave data, perform the action.
alias := TabletAlias{tablet.Tablet.Cell, tablet.Tablet.Uid}
rsd := &RestartSlaveData{Parent: alias, Force: (tablet.Parent.Uid == NO_TABLET)}
rsd.ReplicationState, rsd.WaitPosition, rsd.TimePromoted, err = ta.mysqld.PromoteSlave()
if err != nil {
return err
}
// This data is valuable - commit it to zk first.
_, err = ta.zconn.Create(zkRestartSlaveDataPath, toJson(rsd), 0, zookeeper.WorldACL(zookeeper.PERM_ALL))
if err != nil {
return err
}
// Remove tablet from the replication graph if this is not already the master.
if tablet.Parent.Uid != NO_TABLET {
oldReplicationPath := tablet.ReplicationPath()
err = ta.zconn.Delete(oldReplicationPath, -1)
if err != nil && err.(*zookeeper.Error).Code != zookeeper.ZNONODE {
return err
}
}
// Update tablet regardless - trend towards consistency.
tablet.State = STATE_READ_WRITE
tablet.Type = TYPE_MASTER
tablet.Parent.Cell = ""
tablet.Parent.Uid = NO_TABLET
err = UpdateTablet(ta.zconn, ta.zkTabletPath, tablet)
if err != nil {
return err
}
// NOTE(msolomon) A serving graph update is required, but in order for the
// shard to be consistent the master must be scrapped first. That is
// externally coordinated by the wrangler reparent action.
// Insert the new tablet location in the replication graph now that
// we've updated the tablet.
newReplicationPath := tablet.ReplicationPath()
_, err = ta.zconn.Create(newReplicationPath, "", 0, zookeeper.WorldACL(zookeeper.PERM_ALL))
if err != nil && err.(*zookeeper.Error).Code != zookeeper.ZNODEEXISTS {
return err
}
return nil
}
func (ta *TabletActor) restartSlave(args map[string]string) error {
zkShardActionPath, ok := args["ShardActionPath"]
if !ok {
return fmt.Errorf("missing ShardActionPath in args")
}
tablet, err := ReadTablet(ta.zconn, ta.zkTabletPath)
if err != nil {
return err
}
zkRestartSlaveDataPath := path.Join(zkShardActionPath, "restart_slave_data.json")
data, _, err := ta.zconn.Get(zkRestartSlaveDataPath)
if err != nil {
return err
}
rsd := new(RestartSlaveData)
err = json.Unmarshal([]byte(data), rsd)
if err != nil {
return err
}
// If this check fails, we seem reparented. The only part that could have failed
// is the insert in the replication graph. Do NOT try to reparent
// again. That will either wedge replication to corrupt data.
if tablet.Parent != rsd.Parent {
relog.Debug("restart with new parent")
// Remove tablet from the replication graph.
oldReplicationPath := tablet.ReplicationPath()
err = ta.zconn.Delete(oldReplicationPath, -1)
if err != nil && err.(*zookeeper.Error).Code != zookeeper.ZNONODE {
return err
}
err = ta.mysqld.RestartSlave(rsd.ReplicationState, rsd.WaitPosition, rsd.TimePromoted)
if err != nil {
return err
}
// Once this action completes, update authoritive tablet node first.
tablet.Parent = rsd.Parent
err = UpdateTablet(ta.zconn, ta.zkTabletPath, tablet)
if err != nil {
return err
}
} else if rsd.Force {
err = ta.mysqld.RestartSlave(rsd.ReplicationState, rsd.WaitPosition, rsd.TimePromoted)
if err != nil {
return err
}
}
// Insert the new tablet location in the replication graph now that
// we've updated the tablet.
newReplicationPath := tablet.ReplicationPath()
_, err = ta.zconn.Create(newReplicationPath, "", 0, zookeeper.WorldACL(zookeeper.PERM_ALL))
if err != nil && err.(*zookeeper.Error).Code != zookeeper.ZNODEEXISTS {
return err
}
return nil
}
func (ta *TabletActor) scrap() error {
return Scrap(ta.zconn, ta.zkTabletPath, false)
}
// Make this external, since in needs to be forced from time to time.
func Scrap(zconn zk.Conn, zkTabletPath string, force bool) error {
tablet, err := ReadTablet(zconn, zkTabletPath)
if err != nil {
return err
}
tablet.Type = TYPE_SCRAP
err = UpdateTablet(zconn, zkTabletPath, tablet)
if err != nil {
return err
}
err = zconn.Delete(tablet.ReplicationPath(), -1)
if err != nil {
switch err.(*zookeeper.Error).Code {
case zookeeper.ZNONODE:
relog.Debug("no replicationpath: %v", tablet.ReplicationPath())
return nil
case zookeeper.ZNOTEMPTY:
// If you are forcing the scrapping of a master, you can't update the
// replication graph yet, since other nodes are still under the impression
// they are slaved to this tablet.
// If the node was not empty, we can't do anything about it - the replication
// graph needs to be fixed by reparenting. If the action was forced, assume
// the user knows best and squelch the error.
if tablet.Parent.Uid == NO_TABLET && force {
return nil
}
default:
return err
}
}
return nil
}
// Make this external, since in needs to be forced from time to time.
func ChangeType(zconn zk.Conn, zkTabletPath string, newType TabletType) error {
tablet, err := ReadTablet(zconn, zkTabletPath)
if err != nil {
return err
}
if !IsTrivialTypeChange(tablet.Type, newType) {
return fmt.Errorf("cannot change tablet type %v -> %v", tablet.Type, newType)
}
tablet.Type = newType
return UpdateTablet(zconn, zkTabletPath, tablet)
}

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

@ -0,0 +1,358 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
/*
The agent listens on a zk node for new actions to perform.
It passes them off to a separate action process. Even though some
actions could be completed inline very quickly, the external process
makes it easy to track and interrupt complex actions that may wedge
due to external circumstances.
*/
package tabletmanager
import (
"encoding/json"
"errors"
"flag"
"fmt"
"net"
"os"
"os/exec"
"sort"
"strconv"
"sync"
"time"
"code.google.com/p/vitess.x/go/vt/naming"
"code.google.com/p/vitess.x/go/zk"
"code.google.com/p/vitess/go/relog"
"launchpad.net/gozk/zookeeper"
)
type ActionAgent struct {
zconn zk.Conn
zkTabletPath string // FIXME(msolomon) use tabletInfo
zkActionPath string
vtActionBinPath string // path to vt_action binary
MycnfPath string // path to my.cnf file
mutex sync.Mutex
_tablet *TabletInfo // must be accessed with lock - TabletInfo objects are not synchronized.
}
// bindAddr: the address for the query service advertised by this agent
func NewActionAgent(zconn zk.Conn, zkTabletPath string) *ActionAgent {
actionPath := TabletActionPath(zkTabletPath)
return &ActionAgent{zconn: zconn, zkTabletPath: zkTabletPath, zkActionPath: actionPath}
}
func (agent *ActionAgent) readTablet() error {
// Reread in case there were changes
tablet, err := ReadTablet(agent.zconn, agent.zkTabletPath)
if err != nil {
return err
}
agent.mutex.Lock()
agent._tablet = tablet
agent.mutex.Unlock()
return nil
}
func (agent *ActionAgent) Tablet() *TabletInfo {
agent.mutex.Lock()
tablet := agent._tablet
agent.mutex.Unlock()
return tablet
}
// FIXME(msolomon) need a real path discovery mechanism, a config file
// or more command line args.
func (agent *ActionAgent) resolvePaths() error {
vtActionBinPaths := []string{os.ExpandEnv("$VTROOT/src/code.google.com/p/vitess.x/go/cmd/vtaction/vtaction"),
"/usr/local/bin/vtaction"}
for _, path := range vtActionBinPaths {
if _, err := os.Stat(path); err == nil {
agent.vtActionBinPath = path
break
}
}
if agent.vtActionBinPath == "" {
return errors.New("no vtaction binary found")
}
mycnfPaths := []string{fmt.Sprintf("/vt/vt_%010d/my.cnf", agent.Tablet().Uid),
"/var/lib/mysql/my.cnf", "/etc/my.cnf"}
for _, path := range mycnfPaths {
if _, err := os.Stat(path); err == nil {
agent.MycnfPath = path
break
}
}
if agent.MycnfPath == "" {
return errors.New("no my.cnf found")
}
return nil
}
func (agent *ActionAgent) dispatchAction(actionPath string) {
relog.Info("action dispatch %v", actionPath)
data, _, err := agent.zconn.Get(actionPath)
if err != nil {
relog.Error("action dispatch failed: %v", err)
return
}
actionNode, err := ActionNodeFromJson(data, actionPath)
if err != nil {
relog.Error("action decode failed: %v %v", actionPath, err)
return
}
cmd := []string{
agent.vtActionBinPath,
"-action", actionNode.Action,
"-action-node", actionPath,
"-action-guid", actionNode.ActionGuid,
"-mycnf-path", agent.MycnfPath,
"-logfile", flag.Lookup("logfile").Value.String(),
}
relog.Info("action launch %v", cmd)
vtActionCmd := exec.Command(cmd[0], cmd[1:]...)
stdOut, vtActionErr := vtActionCmd.CombinedOutput()
if vtActionErr != nil {
relog.Error("action failed: %v %v\n%s", actionPath, vtActionErr, stdOut)
return
}
relog.Info("action completed %v %s", actionPath, stdOut)
// Actions should have side effects on the tablet, so reload the data.
if err := agent.readTablet(); err != nil {
relog.Warning("failed rereading tablet after action: %v %v", actionPath, err)
}
}
func (agent *ActionAgent) handleActionQueue() (<-chan zookeeper.Event, error) {
// This read may seem a bit pedantic, but it makes it easier for the system
// to trend towards consistency if an action fails or somehow the action
// queue gets mangled by an errant process.
children, _, watch, err := agent.zconn.ChildrenW(agent.zkActionPath)
if err != nil {
return watch, err
}
if len(children) > 0 {
sort.Strings(children)
for _, child := range children {
actionPath := agent.zkActionPath + "/" + child
agent.dispatchAction(actionPath)
}
}
return watch, nil
}
func (agent *ActionAgent) verifyZkPaths() error {
tablet := agent.Tablet()
if tablet == nil {
panic(fmt.Errorf("agent._tablet is nil"))
}
zkReplicationPath := tablet.ReplicationPath()
_, err := agent.zconn.Create(zkReplicationPath, "", 0, zookeeper.WorldACL(zookeeper.PERM_ALL))
if err != nil && err.(*zookeeper.Error).Code != zookeeper.ZNODEEXISTS {
return err
}
// Ensure that the action node is there.
_, err = agent.zconn.Create(agent.zkActionPath, "", 0, zookeeper.WorldACL(zookeeper.PERM_ALL))
if err != nil && err.(*zookeeper.Error).Code != zookeeper.ZNODEEXISTS {
return err
}
return nil
}
func (agent *ActionAgent) verifyZkServingAddrs() error {
if !agent.Tablet().IsServingType() {
return nil
}
// Load the shard and see if we are supposed to be serving. We might be a serving type,
// but we might be in a transitional state. Only once the shard info is updated do we
// put ourselves in the client serving graph.
shardInfo, err := ReadShard(agent.zconn, agent.Tablet().ShardPath())
if err != nil {
return err
}
if !shardInfo.Contains(agent.Tablet().Tablet) {
return nil
}
// Check to see our address is registered in the right place.
zkPathName := naming.ZkPathForVtName(agent.Tablet().Tablet.Cell, agent.Tablet().Keyspace,
agent.Tablet().Shard, string(agent.Tablet().Type))
f := func(oldValue string, oldStat *zookeeper.Stat) (string, error) {
return agent.updateEndpoints(oldValue, oldStat)
}
err = agent.zconn.RetryChange(zkPathName, 0, zookeeper.WorldACL(zookeeper.PERM_ALL), f)
if err == skipUpdateErr {
err = nil
relog.Warning("skipped serving graph update")
}
return err
}
var skipUpdateErr = fmt.Errorf("skip update")
// A function conforming to the RetryChange protocl. If the data returned
// is identical, no update is performed.
func (agent *ActionAgent) updateEndpoints(oldValue string, oldStat *zookeeper.Stat) (newValue string, err error) {
if oldStat == nil {
// The incoming object doesn't exist - we haven't been placed in the serving
// graph yet, so don't update. Assume the next process that rebuilds the graph
// will get the updated tablet location.
return "", skipUpdateErr
}
addrs := naming.NewAddrs()
if oldValue != "" {
err = json.Unmarshal([]byte(oldValue), addrs)
if err != nil {
return
}
foundTablet := false
for _, entry := range addrs.Entries {
if entry.Uid == agent.Tablet().Uid {
foundTablet = true
vtAddr := fmt.Sprintf("%v:%v", entry.Host, entry.NamedPortMap["_vtocc"])
mysqlAddr := fmt.Sprintf("%v:%v", entry.Host, entry.NamedPortMap["_mysql"])
if vtAddr != agent.Tablet().Addr || mysqlAddr != agent.Tablet().MysqlAddr {
// update needed
host, port := splitHostPort(agent.Tablet().Addr)
entry.Host = host
entry.NamedPortMap["_vtocc"] = port
host, port = splitHostPort(agent.Tablet().MysqlAddr)
entry.NamedPortMap["_mysql"] = port
}
break
}
}
if !foundTablet {
addrs.Entries = append(addrs.Entries, *vtnsAddrForTablet(agent.Tablet().Tablet))
}
} else {
addrs.Entries = append(addrs.Entries, *vtnsAddrForTablet(agent.Tablet().Tablet))
}
return toJson(addrs), nil
}
func splitHostPort(addr string) (string, int) {
host, port, err := net.SplitHostPort(addr)
if err != nil {
panic(err)
}
p, err := strconv.ParseInt(port, 10, 16)
if err != nil {
panic(err)
}
return host, int(p)
}
// Resolve an address where the host has been left blank, like ":3306"
func resolveAddr(addr string) string {
host, port := splitHostPort(addr)
if host == "" {
hostname, err := os.Hostname()
if err != nil {
panic(err)
}
host = hostname
}
return fmt.Sprintf("%v:%v", host, port)
}
func vtnsAddrForTablet(tablet *Tablet) *naming.VtnsAddr {
host, port := splitHostPort(tablet.Addr)
entry := naming.NewAddr(tablet.Uid, host, 0)
entry.NamedPortMap["_vtocc"] = port
host, port = splitHostPort(tablet.MysqlAddr)
entry.NamedPortMap["_mysql"] = port
return entry
}
func (agent *ActionAgent) Start(bindAddr, mysqlAddr string) {
var err error
if err = agent.readTablet(); err != nil {
panic(err)
}
if err = agent.resolvePaths(); err != nil {
panic(err)
}
// Update bind addr for mysql and query service in the tablet node.
f := func(oldValue string, oldStat *zookeeper.Stat) (string, error) {
if oldValue == "" {
return "", fmt.Errorf("no data for tablet addr update: %v", agent.zkTabletPath)
}
tablet := tabletFromJson(oldValue)
tablet.Addr = resolveAddr(bindAddr)
tablet.MysqlAddr = resolveAddr(mysqlAddr)
return toJson(tablet), nil
}
err = agent.zconn.RetryChange(agent.Tablet().Path(), 0, zookeeper.WorldACL(zookeeper.PERM_ALL), f)
if err != nil {
panic(err)
}
// Reread in case there were changes
if err := agent.readTablet(); err != nil {
panic(err)
}
if err := zk.CreatePidNode(agent.zconn, agent.Tablet().PidPath()); err != nil {
panic(err)
}
if err = agent.verifyZkPaths(); err != nil {
panic(err)
}
if err = agent.verifyZkServingAddrs(); err != nil {
panic(err)
}
go agent.actionEventLoop()
}
func (agent *ActionAgent) actionEventLoop() {
for {
// Process any pending actions when we startup, before we start listening
// for events.
watch, err := agent.handleActionQueue()
if err != nil {
relog.Warning("action queue failed: %v", err)
time.Sleep(5 * time.Second)
continue
}
event := <-watch
if !event.Ok() {
// NOTE(msolomon) The zk meta conn will reconnect automatically, or
// error out. At this point, there isn't much to do.
relog.Warning("zookeeper not OK: %v", event)
time.Sleep(5 * time.Second)
} else if event.Type == zookeeper.EVENT_CHILD {
agent.handleActionQueue()
}
}
}

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

@ -0,0 +1,163 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
/*
Actions modify the state of a tablet, shard or keyspace.
*/
package tabletmanager
import (
"fmt"
"os"
"os/user"
"time"
"code.google.com/p/vitess.x/go/zk"
"code.google.com/p/vitess/go/relog"
"launchpad.net/gozk/zookeeper"
)
/*
The actor applies individual commands to execute an action read from a node
in zookeeper.
The actor signals completion by removing the action node from zookeeper.
Errors are written to the action node and must (currently) be resolved by
hand using zk tools.
*/
type InitiatorError string
func (e InitiatorError) Error() string {
return string(e)
}
type ActionInitiator struct {
zconn zk.Conn
}
func NewActionInitiator(zconn zk.Conn) *ActionInitiator {
return &ActionInitiator{zconn}
}
func actionGuid() string {
now := time.Now().Unix()
username := "unknown"
if u, err := user.Current(); err == nil {
username = u.Username
}
hostname := "unknown"
if h, err := os.Hostname(); err == nil {
hostname = h
}
return fmt.Sprintf("%v-%v-%v", now, username, hostname)
}
// FIXME(msolomon) do we care if the action is queued?
func (ai *ActionInitiator) writeTabletAction(zkTabletPath string, node *ActionNode) (actionPath string, err error) {
node.ActionGuid = actionGuid()
data := ActionNodeToJson(node)
actionPath = TabletActionPath(zkTabletPath)
// Action paths end in a trailing slash to that when we create
// sequential nodes, they are created as children, not siblings.
return ai.zconn.Create(actionPath+"/", data, zookeeper.SEQUENCE, zookeeper.WorldACL(zookeeper.PERM_ALL))
}
// FIXME(msolomon) do we care if the action is queued?
func (ai *ActionInitiator) writeShardAction(zkShardPath string, node *ActionNode) (actionPath string, err error) {
MustBeShardPath(zkShardPath)
node.ActionGuid = actionGuid()
data := ActionNodeToJson(node)
actionPath = ShardActionPath(zkShardPath)
// Action paths end in a trailing slash to that when we create
// sequential nodes, they are created as children, not siblings.
return ai.zconn.Create(actionPath+"/", data, zookeeper.SEQUENCE, zookeeper.WorldACL(zookeeper.PERM_ALL))
}
func (ai *ActionInitiator) Ping(zkTabletPath string) (actionPath string, err error) {
return ai.writeTabletAction(zkTabletPath, &ActionNode{Action: TABLET_ACTION_PING})
}
func (ai *ActionInitiator) ChangeType(zkTabletPath string, dbType TabletType) (actionPath string, err error) {
args := map[string]string{"DbType": string(dbType)}
return ai.writeTabletAction(zkTabletPath, &ActionNode{Action: TABLET_ACTION_CHANGE_TYPE, Args: args})
}
func (ai *ActionInitiator) SetReadOnly(zkTabletPath string) (actionPath string, err error) {
return ai.writeTabletAction(zkTabletPath, &ActionNode{Action: TABLET_ACTION_SET_RDONLY})
}
func (ai *ActionInitiator) SetReadWrite(zkTabletPath string) (actionPath string, err error) {
return ai.writeTabletAction(zkTabletPath, &ActionNode{Action: TABLET_ACTION_SET_RDWR})
}
func (ai *ActionInitiator) DemoteMaster(zkTabletPath string) (actionPath string, err error) {
return ai.writeTabletAction(zkTabletPath, &ActionNode{Action: TABLET_ACTION_DEMOTE_MASTER})
}
func (ai *ActionInitiator) PromoteSlave(zkTabletPath, zkShardActionPath string) (actionPath string, err error) {
args := map[string]string{"ShardActionPath": zkShardActionPath}
return ai.writeTabletAction(zkTabletPath, &ActionNode{Action: TABLET_ACTION_PROMOTE_SLAVE, Args: args})
}
func (ai *ActionInitiator) RestartSlave(zkTabletPath, zkShardActionPath string) (actionPath string, err error) {
args := map[string]string{"ShardActionPath": zkShardActionPath}
return ai.writeTabletAction(zkTabletPath, &ActionNode{Action: TABLET_ACTION_RESTART_SLAVE, Args: args})
}
func (ai *ActionInitiator) Scrap(zkTabletPath string) (actionPath string, err error) {
return ai.writeTabletAction(zkTabletPath, &ActionNode{Action: TABLET_ACTION_SCRAP})
}
func (ai *ActionInitiator) ReparentShard(zkShardPath, zkTabletPath string) (actionPath string, err error) {
MustBeTabletPath(zkTabletPath)
node := &ActionNode{Action: SHARD_ACTION_REPARENT}
node.Args = map[string]string{"tabletPath": zkTabletPath}
return ai.writeShardAction(zkShardPath, node)
}
func (ai *ActionInitiator) WaitForCompletion(actionPath string, waitTime time.Duration) error {
// If there is no duration specified, block for a sufficiently long time.
if waitTime <= 0 {
waitTime = 24 * time.Hour
}
timer := time.NewTimer(waitTime)
defer timer.Stop()
for {
data, _, watch, err := ai.zconn.GetW(actionPath)
if err != nil {
zkErr := err.(*zookeeper.Error)
if zkErr.Code == zookeeper.ZNONODE {
return nil
}
return fmt.Errorf("action err: %v %v", actionPath, err)
} else {
actionNode, dataErr := ActionNodeFromJson(data, actionPath)
if dataErr != nil {
return fmt.Errorf("action data error: %v %v %#v", actionPath, dataErr, data)
} else if actionNode.Error != "" {
return fmt.Errorf("action failed: %v %v", actionPath, actionNode.Error)
}
}
select {
case actionEvent := <-watch:
switch actionEvent.Type {
case zookeeper.EVENT_CHANGED:
// reload the node and try again
continue
case zookeeper.EVENT_DELETED:
return nil
default:
// FIXME(msolomon) handle zk disconnect
relog.Warning("unexpected zk event: %v", actionEvent)
}
case <-timer.C:
return fmt.Errorf("action err: %v deadline exceeded %v", actionPath, waitTime)
}
}
panic("unreachable")
}

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

@ -0,0 +1,17 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package tabletmanager
import (
"encoding/json"
)
func toJson(x interface{}) string {
data, err := json.MarshalIndent(x, "", " ")
if err != nil {
panic(err)
}
return string(data)
}

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

@ -0,0 +1,35 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package tabletmanager
import (
"code.google.com/p/vitess.x/go/zk"
// "launchpad.net/gozk/zookeeper"
)
/*
These functions deal with keeping data in the shard graph up to date.
The shard graph is the client-side view of the cluster and is derived from canonical
data sources in zk.
Some of this could be implemented by watching zk nodes, but there are enough cases where
automatic updating is undersirable. Instead, these functions are called where appropriate.
*/
// A given tablet should only appear in once in the serving graph.
// A given tablet can change address, but if it changes db type, so all db typenodes need to be scanned
// and updated as appropriate. That is handled by UpdateServingGraphForShard.
// If no entry is found for this tablet an error is returned.
func UpdateServingGraphForTablet(zconn zk.Conn, tablet *Tablet) error {
return nil
}
// Recompute all nodes in the serving graph for a list of tablets in the shard.
// This will overwrite existing data.
func UpdateServingGraphForShard(zconn zk.Conn, tablet *[]Tablet) error {
return nil
}

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

@ -0,0 +1,55 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
/*
The TabletServer RPC service handles commands from the wrangler.
*/
package tabletmanager
import (
"fmt"
"code.google.com/p/vitess.x/go/vt/mysqlctl"
"code.google.com/p/vitess.x/go/vt/rpc"
"code.google.com/p/vitess/go/relog"
)
type TabletManager struct {
addr string
tablet *Tablet
mysqld *mysqlctl.Mysqld
}
func NewTabletManager(addr string, tablet *Tablet, mysqld *mysqlctl.Mysqld) *TabletManager {
return &TabletManager{addr, tablet, mysqld}
}
// fatten up an error so it has more information when debugging
func (tm *TabletManager) wrapErr(err error) error {
if err == nil {
return err
}
relog.Error("%v", err)
return fmt.Errorf("%v (%v)", err, tm.addr)
}
// Return slave position in terms of the master logs.
func (tm *TabletManager) SlavePosition(_ *rpc.UnusedRequest, reply *mysqlctl.ReplicationPosition) (err error) {
relog.Debug("SlavePosition")
position, err := tm.mysqld.SlaveStatus()
if err == nil {
*reply = *position
}
return tm.wrapErr(err)
}
func (tm *TabletManager) MasterPosition(_ *rpc.UnusedRequest, reply *mysqlctl.ReplicationPosition) (err error) {
relog.Debug("MasterPosition")
position, err := tm.mysqld.MasterStatus()
if err == nil {
*reply = *position
}
relog.Debug("MasterPosition %#v %v", reply, err)
return tm.wrapErr(err)
}

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

@ -0,0 +1,279 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package tabletmanager
import (
"encoding/json"
"fmt"
"path"
"strings"
"code.google.com/p/vitess.x/go/vt/naming"
"code.google.com/p/vitess.x/go/zk"
"code.google.com/p/vitess/go/relog"
"launchpad.net/gozk/zookeeper"
)
/*
Functions for dealing with shard representations in zookeeper.
*/
/*
A pure data struct for information serialized into json and stored in zookeeper
*/
type Shard struct {
// FIXME(msolomon) More will be required here, but for now I don't know the best way
// to handle having ad-hoc db types beyond replica etc.
// This node is used to present clients with a controlled view of the shard unaware
// of every management action.
MasterAlias TabletAlias // There can be only at most one master, but there may be none. (0)
// Uids by type
ReplicaAliases []TabletAlias
RdonlyAliases []TabletAlias
}
func (shard *Shard) Contains(tablet *Tablet) bool {
alias := TabletAlias{tablet.Cell, tablet.Uid}
switch tablet.Type {
case TYPE_MASTER:
return shard.MasterAlias == alias
case TYPE_REPLICA:
for _, replicaAlias := range shard.ReplicaAliases {
if replicaAlias == alias {
return true
}
}
case TYPE_RDONLY:
for _, rdonlyAlias := range shard.RdonlyAliases {
if rdonlyAlias == alias {
return true
}
}
}
return false
}
func (shard *Shard) Json() string {
return toJson(shard)
}
func newShard() *Shard {
return &Shard{ReplicaAliases: make([]TabletAlias, 0, 16),
RdonlyAliases: make([]TabletAlias, 0, 16)}
}
func zkShardFromJson(data string) (*Shard, error) {
shard := newShard()
err := json.Unmarshal([]byte(data), shard)
if err != nil {
return nil, fmt.Errorf("bad shard data %v", err)
}
return shard, nil
}
/*
A meta struct that contains paths to give the zk data more context and convenience
This is the main way we interact with a shard.
*/
type ShardInfo struct {
zkVtRoot string // root path in zk for all vt nodes
keyspace string
shardName string
*Shard
}
func (si *ShardInfo) Json() string {
return si.Shard.Json()
}
func (si *ShardInfo) ShardPath() string {
return ShardPath(si.zkVtRoot, si.keyspace, si.shardName)
}
func (si *ShardInfo) TabletPath(alias TabletAlias) string {
zkRoot := fmt.Sprintf("/zk/%v/vt", alias.Cell)
return TabletPath(zkRoot, alias.Uid)
}
func (si *ShardInfo) MasterTabletPath() (string, error) {
if si.Shard.MasterAlias.Uid == NO_TABLET {
return "", fmt.Errorf("no master tablet for shard %v", si.ShardPath())
}
return si.TabletPath(si.Shard.MasterAlias), nil
}
func (si *ShardInfo) Rebuild(shardTablets []*TabletInfo) {
tmp := newShard()
for _, ti := range shardTablets {
tablet := ti.Tablet
cell := ti.Cell()
alias := TabletAlias{cell, tablet.Uid}
switch tablet.Type {
case TYPE_MASTER:
tmp.MasterAlias = alias
case TYPE_REPLICA:
tmp.ReplicaAliases = append(tmp.ReplicaAliases, alias)
case TYPE_RDONLY:
tmp.RdonlyAliases = append(tmp.RdonlyAliases, alias)
}
}
si.Shard = tmp
}
// shardData: JSON blob
// force: skip error on empty JSON data
func newShardInfo(zkShardPath, shardData string) (shardInfo *ShardInfo, err error) {
if shardData == "" {
return nil, fmt.Errorf("empty shard data: %v", zkShardPath)
}
zkVtRoot := VtRootFromShardPath(zkShardPath)
pathParts := strings.Split(zkShardPath, "/")
keyspace := pathParts[len(pathParts)-3]
shardName := pathParts[len(pathParts)-1]
var shard *Shard
if shardData != "" {
shard, err = zkShardFromJson(shardData)
if err != nil {
return nil, err
}
}
return &ShardInfo{zkVtRoot, keyspace, shardName, shard}, nil
}
func ReadShard(zconn zk.Conn, zkShardPath string) (*ShardInfo, error) {
MustBeShardPath(zkShardPath)
data, _, err := zconn.Get(zkShardPath)
if err != nil {
return nil, err
}
shardInfo, err := newShardInfo(zkShardPath, data)
if err != nil {
return nil, err
}
return shardInfo, nil
}
func UpdateShard(zconn zk.Conn, si *ShardInfo) error {
_, err := zconn.Set(si.ShardPath(), si.Json(), -1)
return err
}
func FindAllTabletAliasesInShard(zconn zk.Conn, si *ShardInfo) ([]TabletAlias, error) {
children, err := zk.ChildrenRecursive(zconn, si.ShardPath())
if err != nil {
return nil, err
}
aliases := make([]TabletAlias, 0, len(children))
for _, child := range children {
alias := path.Base(child)
if strings.Contains(alias, "action") {
continue
}
zkTabletReplicationPath := path.Join(si.ShardPath(), child)
cell, uid, err := parseTabletReplicationPath(zkTabletReplicationPath)
if err != nil {
continue
}
aliases = append(aliases, TabletAlias{cell, uid})
}
return aliases, nil
}
/*
Update shard file with new master, replicas, etc.
/vt/keyspaces/<keyspace>/shards/<shard uid>
Write to zkns files?
Re-read from zk to make sure we are using the side effects of all actions.
*/
func RebuildShard(zconn zk.Conn, zkShardPath string) error {
// NOTE(msolomon) nasty hack - pass non-empty string to bypass data check
shardInfo, err := newShardInfo(zkShardPath, "{}")
if err != nil {
return err
}
aliases, err := FindAllTabletAliasesInShard(zconn, shardInfo)
if err != nil {
return err
}
tablets := make([]*TabletInfo, 0, len(aliases))
for _, alias := range aliases {
tablet, err := ReadTablet(zconn, shardInfo.TabletPath(alias))
if err != nil {
return err
}
tablets = append(tablets, tablet)
}
shardInfo.Rebuild(tablets)
if err = UpdateShard(zconn, shardInfo); err != nil {
return err
}
// Get all existing db types so they can be removed if nothing had been editted.
// This applies to all cells, which can't be determined until you walk through all the tablets.
existingDbTypePaths := make(map[string]bool)
// Update addresses in the serving graph
pathAddrsMap := make(map[string]*naming.VtnsAddrs)
for _, tablet := range tablets {
zkSgShardPath := naming.ZkPathForVtShard(tablet.Tablet.Cell, tablet.Tablet.Keyspace, tablet.Shard)
children, _, err := zconn.Children(zkSgShardPath)
if err != nil {
if err.(*zookeeper.Error).Code != zookeeper.ZNONODE {
relog.Warning("unable to list existing db types: %v", err)
}
} else {
for _, child := range children {
existingDbTypePaths[path.Join(zkSgShardPath, child)] = true
}
}
zkPath := naming.ZkPathForVtName(tablet.Tablet.Cell, tablet.Keyspace, tablet.Shard, string(tablet.Type))
addrs, ok := pathAddrsMap[zkPath]
if !ok {
addrs = naming.NewAddrs()
pathAddrsMap[zkPath] = addrs
}
entry := vtnsAddrForTablet(tablet.Tablet)
addrs.Entries = append(addrs.Entries, *entry)
}
for zkPath, addrs := range pathAddrsMap {
data := toJson(addrs)
_, err = zk.CreateRecursive(zconn, zkPath, data, 0, zookeeper.WorldACL(zookeeper.PERM_ALL))
if err != nil {
if err.(*zookeeper.Error).Code == zookeeper.ZNODEEXISTS {
// Node already exists - just stomp away. Multiple writers shouldn't be here.
// We use RetryChange here because it won't update the node unnecessarily.
f := func(oldValue string, oldStat *zookeeper.Stat) (string, error) {
return data, nil
}
err = zconn.RetryChange(zkPath, 0, zookeeper.WorldACL(zookeeper.PERM_ALL), f)
}
}
if err != nil {
return fmt.Errorf("writing endpoints failed: %v", err)
}
}
// Delete any pre-existing paths that were not updated by this process.
for zkDbTypePath, _ := range existingDbTypePaths {
if _, ok := pathAddrsMap[zkDbTypePath]; !ok {
relog.Info("removing stale db type from serving graph: %v", zkDbTypePath)
if err := zconn.Delete(zkDbTypePath, -1); err != nil {
relog.Warning("unable to remove stale db type from serving graph: %v", err)
}
}
}
return nil
}

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

@ -0,0 +1,314 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package tabletmanager
import (
"encoding/json"
"fmt"
"path"
"code.google.com/p/vitess.x/go/zk"
"launchpad.net/gozk/zookeeper"
)
type TabletType string
const (
// idle
TYPE_IDLE = TabletType("idle")
// primary copy of data
TYPE_MASTER = TabletType("master")
// a slaved copy of the data ready to be promoted to master
TYPE_REPLICA = TabletType("replica")
// a slaved copy of the data for olap load patterns.
// too many aliases for olap - need to pick one
TYPE_RDONLY = TabletType("rdonly")
TYPE_BATCH = TabletType("batch")
// a slaved copy of the data ready, but not serving query traffic
// could be a potential master.
TYPE_SPARE = TabletType("spare")
// a slaved copy of the data ready, but not serving query traffic
// implies something abnormal about the setup - don't consider it
// a potential master.
TYPE_EXPERIMENTAL = TabletType("experimental")
// a slaved copy of the data intentionally lagged for pseudo backup
TYPE_LAG = TabletType("lag")
// a slaved copy of the data, but offline to queries other than backup
// replication sql thread may be stopped
TYPE_BACKUP = TabletType("backup")
// a machine with data that needs to be wiped
TYPE_SCRAP = TabletType("scrap")
)
// Can this db type be trivially reassigned without changes to the replication grpah?
func IsTrivialTypeChange(oldTabletType, newTabletType TabletType) bool {
switch oldTabletType {
case TYPE_REPLICA, TYPE_RDONLY, TYPE_BATCH, TYPE_SPARE, TYPE_BACKUP, TYPE_EXPERIMENTAL:
switch newTabletType {
case TYPE_REPLICA, TYPE_RDONLY, TYPE_BATCH, TYPE_SPARE, TYPE_BACKUP, TYPE_EXPERIMENTAL:
return true
}
case TYPE_SCRAP:
if newTabletType == TYPE_IDLE {
return true
}
}
return false
}
const (
// According to docs, the tablet uid / (mysql server id) is uint32.
// However, zero appears to be a sufficiently degenerate value to use
// as a marker for not having a parent server id.
// http://dev.mysql.com/doc/refman/5.1/en/replication-options.html
NO_TABLET = 0
)
type TabletState string
const (
// The normal state for a master
STATE_READ_WRITE = TabletState("ReadWrite")
// The normal state for a slave, or temporarily a master. Not to be confused with type, which implies a workload.
STATE_READ_ONLY = TabletState("ReadOnly")
)
/* Tablets are really globally unique, but crawling every cell to find out where
it lives is time consuming and expensive. This is only needed during complex operations.
Tablet cell assignments don't change that often.
*/
type TabletAlias struct {
Cell string
Uid uint
}
func (ta *TabletAlias) String() string {
return fmtAlias(ta.Cell, ta.Uid)
}
func fmtAlias(cell string, uid uint) string {
return fmt.Sprintf("%v-%v", cell, tabletUidStr(uid))
}
/*
A pure data struct for information serialized into json and stored in zookeeper
*/
type Tablet struct {
Cell string // the zk cell this tablet is assigned to (doesn't change)
Uid uint // the server id for this instance
Parent TabletAlias // the globally unique alias for our replication parent - zero if this is the global master
Addr string // host:port for queryserver
MysqlAddr string // host:port for the mysql instance
Keyspace string
Shard string
Type TabletType
State TabletState
}
func (tablet *Tablet) IsServingType() bool {
switch tablet.Type {
case TYPE_MASTER, TYPE_REPLICA, TYPE_RDONLY, TYPE_BATCH:
return true
}
return false
}
func (tablet *Tablet) String() string {
return fmt.Sprintf("Tablet{%v}", tablet.Uid)
}
func (tablet *Tablet) Json() string {
return toJson(tablet)
}
func (tablet *Tablet) Hostname() string {
host, _ := splitHostPort(tablet.Addr)
return host
}
type TabletInfo struct {
zkVtRoot string // zk path to vt subtree - /zk/test/vt for instance
version int // zk node version - used to prevent stomping concurrent writes
*Tablet
}
func (ti *TabletInfo) Path() string {
return TabletPath(ti.zkVtRoot, ti.Uid)
}
func (ti *TabletInfo) PidPath() string {
return path.Join(TabletPath(ti.zkVtRoot, ti.Uid), "pid")
}
func (ti *TabletInfo) ShardPath() string {
return ShardPath(ti.zkVtRoot, ti.Keyspace, ti.Shard)
}
// FIXME(msolomon) may not be necessary - we have cell aliased on the tablet info itself
func (ti *TabletInfo) Cell() string {
return zk.ZkCellFromZkPath(ti.zkVtRoot)
}
// This is the path that indicates the tablet's position in the shard replication graph.
// This is too complicated for zk_path, so it's on this struct.
func (ti *TabletInfo) ReplicationPath() string {
return TabletReplicationPath(ti.zkVtRoot, ti.Tablet)
}
func TabletReplicationPath(zkVtRoot string, tablet *Tablet) string {
zkPath := ShardPath(zkVtRoot, tablet.Keyspace, tablet.Shard)
cell := zk.ZkCellFromZkPath(zkVtRoot)
if cell == "local" || cell == "global" {
panic(fmt.Errorf("invalid cell name for replication path: %v", cell))
}
if tablet.Parent.Uid == NO_TABLET {
zkPath = path.Join(zkPath, fmtAlias(tablet.Cell, tablet.Uid))
} else {
// FIXME(msolomon) assumes one level of replication hierarchy
zkPath = path.Join(zkPath, fmtAlias(tablet.Parent.Cell, tablet.Parent.Uid),
fmtAlias(tablet.Cell, tablet.Uid))
}
return zkPath
}
func NewTablet(cell string, uid uint, parent TabletAlias, vtAddr, mysqlAddr, keyspace, shardId string, tabletType TabletType) *Tablet {
state := STATE_READ_ONLY
if tabletType == TYPE_MASTER {
state = STATE_READ_WRITE
if parent.Uid != NO_TABLET {
panic(fmt.Errorf("master cannot have parent: %v", parent.Uid))
}
}
return &Tablet{cell, uid, parent, vtAddr, mysqlAddr, keyspace, shardId, tabletType, state}
}
func tabletFromJson(data string) *Tablet {
t := &Tablet{}
err := json.Unmarshal([]byte(data), t)
if err != nil {
panic(err)
}
return t
}
func ReadTablet(zconn zk.Conn, zkTabletPath string) (*TabletInfo, error) {
MustBeTabletPath(zkTabletPath)
data, stat, err := zconn.Get(zkTabletPath)
if err != nil {
return nil, err
}
tablet := tabletFromJson(data)
zkVtRoot := VtRootFromTabletPath(zkTabletPath)
return &TabletInfo{zkVtRoot, stat.Version(), tablet}, nil
}
func UpdateTablet(zconn zk.Conn, zkTabletPath string, tablet *TabletInfo) error {
MustBeTabletPath(zkTabletPath)
version := -1
if tablet.version != 0 {
version = tablet.version
}
// FIXME(msolomon) update replication path?
_, err := zconn.Set(zkTabletPath, tablet.Json(), version)
return err
}
func Validate(zconn zk.Conn, zkTabletPath string, zkTabletReplicationPath string) error {
MustBeTabletPath(zkTabletPath)
tablet, err := ReadTablet(zconn, zkTabletPath)
if err != nil {
return err
}
zkPaths := []string{
TabletActionPath(zkTabletPath),
ShardActionPath(tablet.ShardPath()),
}
if tablet.Type != TYPE_SCRAP {
if zkTabletReplicationPath != "" && zkTabletReplicationPath != tablet.ReplicationPath() {
return fmt.Errorf("replication path mismatch, tablet expects %v but found %v",
tablet.ReplicationPath(), zkTabletReplicationPath)
}
// Unless we are scrapped, check with are in the replication graph
zkPaths = append(zkPaths, tablet.ReplicationPath())
} else {
// Scrap nodes should not appear in the replication graph unless an action is running.
_, _, err := zconn.Get(tablet.ReplicationPath())
if !(err != nil && err.(*zookeeper.Error).Code == zookeeper.ZNONODE) {
return fmt.Errorf("unexpected replication path found for scrap tablet: %v",
tablet.ReplicationPath())
}
}
for _, zkPath := range zkPaths {
_, _, err := zconn.Get(zkPath)
if err != nil {
return err
}
}
return nil
}
func CreateTablet(zconn zk.Conn, zkTabletPath string, tablet *Tablet) error {
MustBeTabletPath(zkTabletPath)
// Create /vt/tablets/<uid>
_, err := zk.CreateRecursive(zconn, zkTabletPath, tablet.Json(), 0, zookeeper.WorldACL(zookeeper.PERM_ALL))
if err != nil {
return err
}
// FIXME(msolomon) pull out into a list of required path and make the agent guarantee existence.
// Create /vt/tablets/<uid>/action
_, err = zconn.Create(TabletActionPath(zkTabletPath), "", 0, zookeeper.WorldACL(zookeeper.PERM_ALL))
if err != nil {
return err
}
zkVtRootPath := VtRootFromTabletPath(zkTabletPath)
shardPath := ShardPath(zkVtRootPath, tablet.Keyspace, tablet.Shard)
// Create /vt/keyspaces/<keyspace>/shards/<shard id>
_, err = zk.CreateRecursive(zconn, shardPath, newShard().Json(), 0, zookeeper.WorldACL(zookeeper.PERM_ALL))
if err != nil && err.(*zookeeper.Error).Code == zookeeper.ZNODEEXISTS {
// If the node exists, bully for us
err = nil
}
shardActionPath := ShardActionPath(shardPath)
// Create /vt/keyspaces/<keyspace>/shards/<shard id>/action
_, err = zk.CreateRecursive(zconn, shardActionPath, "", 0, zookeeper.WorldACL(zookeeper.PERM_ALL))
if err != nil && err.(*zookeeper.Error).Code == zookeeper.ZNODEEXISTS {
// If the node exists, bully for us
err = nil
}
_, err = zk.CreateRecursive(zconn, TabletReplicationPath(zkVtRootPath, tablet), "", 0, zookeeper.WorldACL(zookeeper.PERM_ALL))
if err != nil && err.(*zookeeper.Error).Code == zookeeper.ZNODEEXISTS {
// If the node exists, bully for us
err = nil
}
return err
}

385
go/vt/tabletmanager/test.py Executable file
Просмотреть файл

@ -0,0 +1,385 @@
#!/usr/bin/python
import json
from optparse import OptionParser
import os
import shlex
import signal
import socket
from subprocess import check_call, Popen, CalledProcessError, PIPE
import threading
import sys
import time
import MySQLdb
class TestError(Exception):
pass
class Break(Exception):
pass
def pause(prompt):
if options.debug:
raw_input(prompt)
pid_map = {}
def _add_proc(proc):
pid_map[proc.pid] = proc
with open('.test-pids', 'a') as f:
print >> f, proc.pid, os.path.basename(proc.args[0])
vtxtop = os.environ['VTXTOP']
hostname = socket.gethostname()
def run(cmd, trap_output=False, **kargs):
args = shlex.split(cmd)
if trap_output:
kargs['stdout'] = PIPE
kargs['stderr'] = PIPE
if options.verbose:
print "run:", cmd, ', '.join('%s=%s' % x for x in kargs.iteritems())
proc = Popen(args, **kargs)
proc.args = args
stdout, stderr = proc.communicate()
if proc.returncode:
raise TestError('cmd fail:', args, stdout, stderr)
return stdout, stderr
def run_fail(cmd, **kargs):
args = shlex.split(cmd)
kargs['stdout'] = PIPE
kargs['stderr'] = PIPE
if options.verbose:
print "run: (expect fail)", cmd, ', '.join('%s=%s' % x for x in kargs.iteritems())
proc = Popen(args, **kargs)
proc.args = args
stdout, stderr = proc.communicate()
if proc.returncode == 0:
raise TestError('expected fail:', args, stdout, stderr)
return stdout, stderr
# run a daemon - kill when this script exits
def run_bg(cmd, **kargs):
if options.verbose:
print "run:", cmd, ', '.join('%s=%s' % x for x in kargs.iteritems())
args = shlex.split(cmd)
proc = Popen(args=args, **kargs)
proc.args = args
_add_proc(proc)
if options.verbose:
def x():
rc = proc.wait()
if rc not in (0, -9):
sys.stderr.write("proc failed: %s %s\n" % (proc.returncode, proc.args))
t = threading.Thread(target=x)
t.daemon = True
t.start()
return proc
def check_db_var(uid, name, value):
conn = MySQLdb.Connect(user='vt_dba',
unix_socket='/vt/vt_%010d/mysql.sock' % uid)
cursor = conn.cursor()
cursor.execute("show variables like '%s'" % name)
row = cursor.fetchone()
if row != (name, value):
raise TestError('variable not set correctly', name, row)
conn.close()
def check_db_read_only(uid):
return check_db_var(uid, 'read_only', 'ON')
def check_db_read_write(uid):
return check_db_var(uid, 'read_only', 'OFF')
def wait_db_read_only(uid):
for x in xrange(3):
try:
check_db_read_only(uid)
return
except TestError, e:
print >> sys.stderr, 'WARNING: ', e
time.sleep(1.0)
raise e
def wait_procs(proc_list, raise_on_error=True):
for proc in proc_list:
proc.wait()
for proc in proc_list:
if proc.returncode and raise_on_error:
raise CalledProcessError(proc.returncode, proc.args)
def setup():
setup_procs = []
# compile all the tools
run('go build', cwd=vtxtop+'/go/cmd/mysqlctl')
run('go build', cwd=vtxtop+'/go/cmd/vtaction')
run('go build', cwd=vtxtop+'/go/cmd/vttablet')
run('go build', cwd=vtxtop+'/go/cmd/vtctl')
run('go build', cwd=vtxtop+'/go/cmd/zkctl')
run('go build', cwd=vtxtop+'/go/cmd/zk')
# start mysql instance external to the test
setup_procs.append(run_bg(vtxtop+'/go/cmd/mysqlctl/mysqlctl -tablet-uid 62344 -port 6700 -mysql-port 3700 init'))
setup_procs.append(run_bg(vtxtop+'/go/cmd/mysqlctl/mysqlctl -tablet-uid 62044 -port 6701 -mysql-port 3701 init'))
setup_procs.append(run_bg(vtxtop+'/go/cmd/mysqlctl/mysqlctl -tablet-uid 41983 -port 6702 -mysql-port 3702 init'))
setup_procs.append(run_bg(vtxtop+'/go/cmd/mysqlctl/mysqlctl -tablet-uid 31981 -port 6703 -mysql-port 3703 init'))
setup_procs.append(run_bg(vtxtop+'/go/cmd/zkctl/zkctl -zk.cfg 1@'+hostname+':3801:3802:3803 init'))
wait_procs(setup_procs)
with open('.test-zk-client-conf.json', 'w') as f:
zk_cell_mapping = {'test_nj': 'localhost:3803',
'test_ny': 'localhost:3803',
'test_ca': 'localhost:3803',
'global': 'localhost:3803',}
json.dump(zk_cell_mapping, f)
os.putenv('ZK_CLIENT_CONFIG', '.test-zk-client-conf.json')
run(vtxtop+'/go/cmd/zk/zk touch -p /zk/test_nj/vt')
run(vtxtop+'/go/cmd/zk/zk touch -p /zk/test_ny/vt')
run(vtxtop+'/go/cmd/zk/zk touch -p /zk/test_ca/vt')
def teardown():
if options.skip_teardown:
return
teardown_procs = []
for x in (62344, 62044, 41983, 31981):
teardown_procs.append(run_bg(vtxtop+'/go/cmd/mysqlctl/mysqlctl -tablet-uid %u -force teardown' % x))
teardown_procs.append(run_bg(vtxtop+'/go/cmd/zkctl/zkctl -zk.cfg 1@'+hostname+':3801:3802:3803 teardown'))
wait_procs(teardown_procs, raise_on_error=False)
for proc in pid_map.values():
if proc.pid and proc.returncode is None:
proc.kill()
with open('.test-pids') as f:
for line in f:
try:
parts = line.strip().split()
pid = int(parts[0])
proc_name = parts[1]
proc = pid_map.get(pid)
if not proc or (proc and proc.pid and proc.returncode is None):
os.kill(pid, signal.SIGTERM)
except OSError, e:
if options.verbose:
print >> sys.stderr, e
for path in ('.test-pids', '.test-zk-client-conf.json'):
try:
os.remove(path)
except OSError, e:
if options.verbose:
print >> sys.stderr, e, path
def run_test_sanity():
# Start up a master mysql and vttablet
run(vtxtop+'/go/cmd/vtctl/vtctl -force InitTablet /zk/test_nj/vt/tablets/0000062344 localhost 3700 6700 test_keyspace 0 master ""')
run(vtxtop+'/go/cmd/vtctl/vtctl RebuildShard /zk/global/vt/keyspaces/test_keyspace/shards/0')
run(vtxtop+'/go/cmd/vtctl/vtctl Validate /zk/test_nj/vt')
agent_62344 = run_bg(vtxtop+'/go/cmd/vttablet/vttablet -port 6700 -tablet-path /zk/test_nj/vt/tablets/0000062344 -logfile /vt/vt_0000062344/vttablet.log')
run(vtxtop+'/go/cmd/vtctl/vtctl Ping /zk/test_nj/vt/tablets/0000062344')
# Quickly check basic actions.
run(vtxtop+'/go/cmd/vtctl/vtctl SetReadOnly /zk/test_nj/vt/tablets/0000062344')
wait_db_read_only(62344)
run(vtxtop+'/go/cmd/vtctl/vtctl SetReadWrite /zk/test_nj/vt/tablets/0000062344')
check_db_read_write(62344)
run(vtxtop+'/go/cmd/vtctl/vtctl DemoteMaster /zk/test_nj/vt/tablets/0000062344')
wait_db_read_only(62344)
run(vtxtop+'/go/cmd/vtctl/vtctl Validate /zk/test_nj/vt')
agent_62344.kill()
def _wipe_zk():
run(vtxtop+'/go/cmd/zk/zk rm -rf /zk/test_nj/vt')
run(vtxtop+'/go/cmd/zk/zk rm -rf /zk/test_ny/vt')
#run(vtxtop+'/go/cmd/zk/zk rm -rf /zk/test_ca/vt')
run(vtxtop+'/go/cmd/zk/zk rm -rf /zk/global/vt')
def _check_zk(ping_tablets=False):
if ping_tablets:
run(vtxtop+'/go/cmd/vtctl/vtctl -ping-tablets Validate /zk/test_nj/vt')
run(vtxtop+'/go/cmd/vtctl/vtctl -ping-tablets Validate /zk/test_ny/vt')
else:
run(vtxtop+'/go/cmd/vtctl/vtctl Validate /zk/test_nj/vt')
run(vtxtop+'/go/cmd/vtctl/vtctl Validate /zk/test_ny/vt')
#run(vtxtop+'/go/cmd/vtctl/vtctl Validate /zk/test_ca/vt')
def _check_db_addr(db_addr, expected_addr):
# Run in the background to capture output.
proc = run_bg(vtxtop+'/go/cmd/vtctl/vtctl -zk.local-cell=test_nj Resolve ' + db_addr, stdout=PIPE)
stdout = proc.communicate()[0].strip()
if stdout != expected_addr:
raise TestError('wrong zk address', db_addr, stdout, expected_addr)
def run_test_reparent_graceful():
_wipe_zk()
# Start up a master mysql and vttablet
run(vtxtop+'/go/cmd/vtctl/vtctl -force InitTablet /zk/test_nj/vt/tablets/0000062344 localhost 3700 6700 test_keyspace 0 master ""')
agent_62344 = run_bg(vtxtop+'/go/cmd/vttablet/vttablet -port 6700 -tablet-path /zk/test_nj/vt/tablets/0000062344 -logfile /vt/vt_0000062344/vttablet.log')
# Create a few slaves for testing reparenting.
run(vtxtop+'/go/cmd/vtctl/vtctl -force InitTablet /zk/test_nj/vt/tablets/0000062044 localhost 3701 6701 test_keyspace 0 replica /zk/global/vt/keyspaces/test_keyspace/shards/0/test_nj-62344')
agent_62044 = run_bg(vtxtop+'/go/cmd/vttablet/vttablet -port 6701 -tablet-path /zk/test_nj/vt/tablets/0000062044 -logfile /vt/vt_0000062044/vttablet.log')
run(vtxtop+'/go/cmd/vtctl/vtctl -force InitTablet /zk/test_nj/vt/tablets/0000041983 localhost 3702 6702 test_keyspace 0 replica /zk/global/vt/keyspaces/test_keyspace/shards/0/test_nj-62344')
agent_41983 = run_bg(vtxtop+'/go/cmd/vttablet/vttablet -port 6702 -tablet-path /zk/test_nj/vt/tablets/0000041983 -logfile /vt/vt_0000041983/vttablet.log')
run(vtxtop+'/go/cmd/vtctl/vtctl -force InitTablet /zk/test_ny/vt/tablets/0000031981 localhost 3703 6703 test_keyspace 0 replica /zk/global/vt/keyspaces/test_keyspace/shards/0/test_nj-62344')
agent_31983 = run_bg(vtxtop+'/go/cmd/vttablet/vttablet -port 6703 -tablet-path /zk/test_ny/vt/tablets/0000031981 -logfile /vt/vt_0000031981/vttablet.log')
# Recompute the shard layout node - until you do that, it might not be valid.
run(vtxtop+'/go/cmd/vtctl/vtctl RebuildShard /zk/global/vt/keyspaces/test_keyspace/shards/0')
_check_zk(ping_tablets=True)
# Force the slaves to reparent assuming that all the datasets are identical.
pause("force ReparentShard?")
run(vtxtop+'/go/cmd/vtctl/vtctl -force ReparentShard /zk/global/vt/keyspaces/test_keyspace/shards/0 /zk/test_nj/vt/tablets/0000062344')
_check_zk()
expected_addr = hostname + ':6700'
_check_db_addr('test_keyspace.0.master:_vtocc', expected_addr)
# Convert a replica to a spare. That should leave only one node serving traffic,
# but still needs to appear in the replication graph.
run(vtxtop+'/go/cmd/vtctl/vtctl ChangeType /zk/test_nj/vt/tablets/0000041983 spare')
_check_zk()
expected_addr = hostname + ':6701'
_check_db_addr('test_keyspace.0.replica:_vtocc', expected_addr)
# Perform a graceful reparent operation.
pause("graceful ReparentShard?")
run(vtxtop+'/go/cmd/vtctl/vtctl ReparentShard /zk/global/vt/keyspaces/test_keyspace/shards/0 /zk/test_nj/vt/tablets/0000062044')
_check_zk()
expected_addr = hostname + ':6701'
_check_db_addr('test_keyspace.0.master:_vtocc', expected_addr)
agent_62344.kill()
agent_62044.kill()
agent_41983.kill()
agent_31983.kill()
# Test address correction.
agent_62044 = run_bg(vtxtop+'/go/cmd/vttablet/vttablet -port 6773 -tablet-path /zk/test_nj/vt/tablets/0000062044 -logfile /vt/vt_0000062044/vttablet.log')
# Wait a moment for address to reregister.
time.sleep(1.0)
expected_addr = hostname + ':6773'
_check_db_addr('test_keyspace.0.master:_vtocc', expected_addr)
agent_62044.kill()
def run_test_reparent_down_master():
_wipe_zk()
# Start up a master mysql and vttablet
run(vtxtop+'/go/cmd/vtctl/vtctl -force InitTablet /zk/test_nj/vt/tablets/0000062344 localhost 3700 6700 test_keyspace 0 master ""')
agent_62344 = run_bg(vtxtop+'/go/cmd/vttablet/vttablet -port 6700 -tablet-path /zk/test_nj/vt/tablets/0000062344 -logfile /vt/vt_0000062344/vttablet.log')
# Create a few slaves for testing reparenting.
run(vtxtop+'/go/cmd/vtctl/vtctl -force InitTablet /zk/test_nj/vt/tablets/0000062044 localhost 3701 6701 test_keyspace 0 replica /zk/global/vt/keyspaces/test_keyspace/shards/0/test_nj-62344')
agent_62044 = run_bg(vtxtop+'/go/cmd/vttablet/vttablet -port 6701 -tablet-path /zk/test_nj/vt/tablets/0000062044 -logfile /vt/vt_0000062044/vttablet.log')
run(vtxtop+'/go/cmd/vtctl/vtctl -force InitTablet /zk/test_nj/vt/tablets/0000041983 localhost 3702 6702 test_keyspace 0 replica /zk/global/vt/keyspaces/test_keyspace/shards/0/test_nj-62344')
agent_41983 = run_bg(vtxtop+'/go/cmd/vttablet/vttablet -port 6702 -tablet-path /zk/test_nj/vt/tablets/0000041983 -logfile /vt/vt_0000041983/vttablet.log')
run(vtxtop+'/go/cmd/vtctl/vtctl -force InitTablet /zk/test_ny/vt/tablets/0000031981 localhost 3703 6703 test_keyspace 0 replica /zk/global/vt/keyspaces/test_keyspace/shards/0/test_nj-62344')
agent_31983 = run_bg(vtxtop+'/go/cmd/vttablet/vttablet -port 6703 -tablet-path /zk/test_ny/vt/tablets/0000031981 -logfile /vt/vt_0000031981/vttablet.log')
# Recompute the shard layout node - until you do that, it might not be valid.
run(vtxtop+'/go/cmd/vtctl/vtctl RebuildShard /zk/global/vt/keyspaces/test_keyspace/shards/0')
_check_zk()
# Force the slaves to reparent assuming that all the datasets are identical.
run(vtxtop+'/go/cmd/vtctl/vtctl -force ReparentShard /zk/global/vt/keyspaces/test_keyspace/shards/0 /zk/test_nj/vt/tablets/0000062344')
_check_zk()
# Make the master agent unavailable.
agent_62344.kill()
expected_addr = hostname + ':6700'
_check_db_addr('test_keyspace.0.master:_vtocc', expected_addr)
# Perform a reparent operation - this will hang for some amount of time.
run_fail(vtxtop+'/go/cmd/vtctl/vtctl -wait-time 5s ReparentShard /zk/global/vt/keyspaces/test_keyspace/shards/0 /zk/test_nj/vt/tablets/0000062044')
# Should timeout and fail
run_fail(vtxtop+'/go/cmd/vtctl/vtctl -wait-time 5s ScrapTablet /zk/test_nj/vt/tablets/0000062344')
# Force the scrap action in zk even though tablet is not accessible.
run(vtxtop+'/go/cmd/vtctl/vtctl -force ScrapTablet /zk/test_nj/vt/tablets/0000062344')
run(vtxtop+'/go/cmd/vtctl/vtctl -force ChangeType /zk/test_nj/vt/tablets/0000062344 idle')
idle_tablets, _ = run(vtxtop+'/go/cmd/vtctl/vtctl ListIdle /zk/test_nj/vt', trap_output=True)
if '0000062344' not in idle_tablets:
raise TestError('idle tablet not found', idle_tablets)
# Remove pending locks (make this the force option to ReparentShard?)
run(vtxtop+'/go/cmd/vtctl/vtctl -force PurgeActions /zk/global/vt/keyspaces/test_keyspace/shards/0/action')
# Scrapping a tablet shouldn't take it out of the servering graph.
expected_addr = hostname + ':6700'
_check_db_addr('test_keyspace.0.master:_vtocc', expected_addr)
# Re-run reparent operation, this shoud now proceed unimpeded.
run(vtxtop+'/go/cmd/vtctl/vtctl ReparentShard /zk/global/vt/keyspaces/test_keyspace/shards/0 /zk/test_nj/vt/tablets/0000062044')
_check_zk()
expected_addr = hostname + ':6701'
_check_db_addr('test_keyspace.0.master:_vtocc', expected_addr)
agent_62044.kill()
agent_41983.kill()
agent_31983.kill()
def run_all():
run_test_sanity()
run_test_sanity() # run twice to check behavior with existing znode data
run_test_reparent_graceful()
run_test_reparent_down_master()
options = None
def main():
global options
parser = OptionParser()
parser.add_option('-v', '--verbose', action='store_true')
parser.add_option('--debug', action='store_true')
parser.add_option('--skip-teardown', action='store_true')
(options, args) = parser.parse_args()
if not args:
args = ['run_all']
try:
if args[0] != 'teardown':
setup()
if args[0] != 'setup':
for arg in args:
globals()[arg]()
print "GREAT SUCCESS"
except KeyboardInterrupt:
pass
except Break:
options.skip_teardown = True
finally:
teardown()
if __name__ == '__main__':
main()

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

@ -0,0 +1,140 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package tabletmanager
import (
"fmt"
"path"
"strconv"
"strings"
"code.google.com/p/vitess.x/go/zk"
)
/* Functions for extracting and deriving zk paths. */
func VtRootFromTabletPath(zkTabletPath string) string {
defer func() {
if e := recover(); e != nil {
panic(fmt.Errorf("invalid tablet path: %v", e))
}
}()
pathParts := strings.Split(zkTabletPath, "/")
if pathParts[len(pathParts)-2] != "tablets" {
panic(fmt.Errorf("invalid tablet path: %v", zkTabletPath))
}
return strings.Join(pathParts[:len(pathParts)-2], "/")
}
func VtRootFromShardPath(zkShardPath string) string {
defer func() {
if e := recover(); e != nil {
panic(fmt.Errorf("invalid shard path: %v %v", zkShardPath, e))
}
}()
pathParts := strings.Split(zkShardPath, "/")
if pathParts[len(pathParts)-2] != "shards" || pathParts[len(pathParts)-4] != "keyspaces" {
panic(fmt.Errorf("invalid shard path: %v", zkShardPath))
}
if pathParts[2] != "global" {
panic(fmt.Errorf("invalid shard path - not global: %v", zkShardPath))
}
return strings.Join(pathParts[:len(pathParts)-4], "/")
}
// /vt/tablets/<tablet uid>
func MustBeTabletPath(zkTabletPath string) {
VtRootFromTabletPath(zkTabletPath)
}
// This is the path that indicates the authoritive table node.
func TabletPath(zkVtRoot string, tabletUid uint) string {
tabletPath := path.Join(zkVtRoot, "tablets", tabletUidStr(tabletUid))
MustBeTabletPath(tabletPath)
return tabletPath
}
func TabletActionPath(zkTabletPath string) string {
MustBeTabletPath(zkTabletPath)
return path.Join(zkTabletPath, "action")
}
// /vt/keyspaces/<keyspace>/shards/<shard uid>
func MustBeShardPath(zkShardPath string) {
VtRootFromShardPath(zkShardPath)
}
// zkVtRoot: /zk/XX/vt
func ShardPath(zkVtRoot, keyspace, shard string) string {
shardPath := path.Join("/zk/global/vt", "keyspaces", keyspace, "shards", shard)
MustBeShardPath(shardPath)
return shardPath
}
// zkShardPath: /zk/global/vt/keyspaces/XX/shards/YY
func ShardActionPath(zkShardPath string) string {
MustBeShardPath(zkShardPath)
return path.Join(zkShardPath, "action")
}
// FIXME(msolomon) this is really weak
// Tablet aliases are the nodes that point into /vt/tablets/<uid> from the keyspace
// Note that these are *global*
func IsTabletReplicationPath(zkReplicationPath string) bool {
_, _, err := parseTabletReplicationPath(zkReplicationPath)
return err == nil
}
func parseTabletReplicationPath(zkReplicationPath string) (cell string, uid uint, err error) {
cell = zk.ZkCellFromZkPath(zkReplicationPath)
if cell != "global" {
cell = ""
err = fmt.Errorf("invalid cell, expected global: %v", zkReplicationPath)
return
}
nameParts := strings.Split(path.Base(zkReplicationPath), "-")
if len(nameParts) != 2 {
err = fmt.Errorf("invalid path %v", zkReplicationPath)
return
}
cell = nameParts[0]
_uid, err := strconv.ParseUint(nameParts[1], 10, 0)
if err != nil {
err = fmt.Errorf("invalid uid %v: %v", zkReplicationPath, err)
}
uid = uint(_uid)
return
}
func ParseTabletReplicationPath(zkReplicationPath string) (cell string, uid uint) {
cell, uid, err := parseTabletReplicationPath(zkReplicationPath)
if err != nil {
panic(err)
}
return
}
func tabletUidStr(uid uint) string {
return fmt.Sprintf("%010d", uid)
}
func TabletPathForAlias(alias TabletAlias) string {
return fmt.Sprintf("/zk/%v/vt/tablets/%v", alias.Cell, tabletUidStr(alias.Uid))
}
// zkActionPath is /zk/test/vt/tablets/<uid>/action/0000000001
func TabletPathFromActionPath(zkActionPath string) string {
zkPathParts := strings.Split(zkActionPath, "/")
if zkPathParts[len(zkPathParts)-2] != "action" {
panic(fmt.Errorf("invalid action path: %v", zkActionPath))
}
tabletPath := strings.Join(zkPathParts[:len(zkPathParts)-2], "/")
MustBeTabletPath(tabletPath)
return tabletPath
}

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

@ -0,0 +1,134 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package tabletmanager
import (
"testing"
)
func testExpectedShardPanic(t *testing.T, path string) {
defer func() {
recover()
}()
MustBeShardPath(path)
t.Errorf("expected shard panic: %v", path)
}
func testExpectedTabletPanic(t *testing.T, path string) {
defer func() {
recover()
}()
MustBeTabletPath(path)
t.Errorf("expected tablet panic: %v", path)
}
func TestInvalidShard(t *testing.T) {
testExpectedShardPanic(t, "/vt/keyspaces/test/shards/0/123456789")
}
func TestEmptyShard(t *testing.T) {
testExpectedShardPanic(t, "")
}
func TestValidShard(t *testing.T) {
defer func() {
if x := recover(); x != nil {
t.Error(x)
}
}()
MustBeShardPath("/zk/global/vt/keyspaces/test/shards/0")
}
func TestValidShardWithTrailingSlash(t *testing.T) {
// we have to be strict - otherwise things are a mess
testExpectedShardPanic(t, "/zk/global/vt/keyspaces/test/shards/0/")
}
func TestValidTablet(t *testing.T) {
defer func() {
if x := recover(); x != nil {
t.Error(x)
}
}()
MustBeTabletPath("/vt/tablets/0")
}
func TestValidTabletWithTrailingSlash(t *testing.T) {
testExpectedTabletPanic(t, "/vt/tablets/0/")
}
func TestRealPath(t *testing.T) {
defer func() {
if x := recover(); x != nil {
t.Error(x)
}
}()
MustBeTabletPath("/zk/test/vt/tablets/0000062344")
}
func TestShardInfo(t *testing.T) {
defer func() {
if x := recover(); x != nil {
t.Error(x)
}
}()
zkPath := "/zk/global/vt/keyspaces/test_keyspace/shards/shard0"
// intentionally degenerate data to bypass empty data check
si, err := newShardInfo(zkPath, "{}")
if err != nil {
t.Error("newShardErr: %v", err)
}
if si.zkVtRoot != "/zk/global/vt" {
t.Errorf("bad zkVtRoot: %v", si.zkVtRoot)
}
if si.keyspace != "test_keyspace" {
t.Errorf("bad keyspace: %v", si.keyspace)
}
if si.shardName != "shard0" {
t.Errorf("bad shard: %v", si.shardName)
}
if zkPath != si.ShardPath() {
t.Errorf("bad ShardPath: %v", si.ShardPath())
}
}
func TestVtRootFromShardPath(t *testing.T) {
defer func() {
if x := recover(); x != nil {
t.Error(x)
}
}()
path := VtRootFromShardPath("/zk/global/vt/keyspaces/test_keyspace/shards/shard0")
expectedPath := "/zk/global/vt"
if path != expectedPath {
t.Errorf("%v not expected path %v", path, expectedPath)
}
}
func TestVtRootFromTabletPath(t *testing.T) {
defer func() {
if x := recover(); x != nil {
t.Error(x)
}
}()
path := VtRootFromTabletPath("/zk/test/vt/tablets/0000062344")
expectedPath := "/zk/test/vt"
if path != expectedPath {
t.Errorf("%v not expected path %v", path, expectedPath)
}
}
func TestTabletPathFromActionPath(t *testing.T) {
defer func() {
if x := recover(); x != nil {
t.Error(x)
}
}()
path := TabletPathFromActionPath("/zk/test/vt/tablets/0000062344/action/0000000001")
expectedPath := "/zk/test/vt/tablets/0000062344"
if path != expectedPath {
t.Errorf("%v not expected path %v", path, expectedPath)
}
}

443
go/vt/wrangler/reparent.go Normal file
Просмотреть файл

@ -0,0 +1,443 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package zkwrangler
/*
Assume a graph of mysql nodes.
Replace node N with X.
Connect to N and record file/position from "show master status"
On N: (Demote Master)
SET GLOBAL READ_ONLY = 1;
FLUSH TABLES WITH READ LOCK;
UNLOCK TABLES;
While this is read-only, all the replicas should sync to the same point.
For all slaves of N:
show slave status
relay_master_log_file
exec_master_log_pos
Map file:pos to list of slaves that are in sync
There should be only one group (ideally). If not, manually resolve, or pick
the largest group.
Select X from N - X is the new root node. Might not be a "master" in terms of
voltron, but it will be the data source for the rest of the nodes.
On X: (Promote Slave)
STOP SLAVE;
RESET MASTER;
RESET SLAVE;
SHOW MASTER STATUS;
replication file,position
INSERT INTO _vt.replication_test (time_created_ns) VALUES (<time>);
SHOW MASTER STATUS;
wait file,position
SET GLOBAL READ_ONLY=0;
For all slaves in majority N:
if slave != X (Restart Slave)
STOP SLAVE;
RESET SLAVE;
CHANGE MASTER TO X;
START SLAVE;
SELECT MASTER_POS_WAIT(file, pos, deadline)
SELECT time_created FROM _vt.replication_test WHERE time_created_ns = <time>;
if no connection to N is available, ???
*/
import (
"fmt"
"net/rpc"
"sort"
"strings"
"time"
"code.google.com/p/vitess.x/go/vt/mysqlctl"
vtrpc "code.google.com/p/vitess.x/go/vt/rpc"
tm "code.google.com/p/vitess.x/go/vt/tabletmanager"
"code.google.com/p/vitess.x/go/zk"
"code.google.com/p/vitess/go/relog"
)
const (
SLAVE_STATUS_DEADLINE = 10e9
)
type Wrangler struct {
zconn zk.Conn
ai *tm.ActionInitiator
}
func NewWrangler(zconn zk.Conn, ai *tm.ActionInitiator) *Wrangler {
return &Wrangler{zconn, ai}
}
func (wr *Wrangler) readTablet(zkTabletPath string) (*tm.TabletInfo, error) {
return tm.ReadTablet(wr.zconn, zkTabletPath)
}
/*
Create the reparenting action and launch a goroutine to coordinate the procedure.
The actionNode can be watched for updates.
TODO(msolomon): We could supply a channel of updates to cut out zk round trips.
force: true if we are trying to skip sanity checks - mostly for test setups
*/
func (wr *Wrangler) ReparentShard(zkShardPath, zkTabletPath string, force bool) (actionPath string, err error) {
tm.MustBeShardPath(zkShardPath)
tm.MustBeTabletPath(zkTabletPath)
shardInfo, err := tm.ReadShard(wr.zconn, zkShardPath)
if err != nil {
return
}
currentMasterTabletPath, err := shardInfo.MasterTabletPath()
if err != nil {
return
}
if currentMasterTabletPath == zkTabletPath && !force {
return "", fmt.Errorf("master-elect tablet %v is already master - specify -force to override", zkTabletPath)
}
tablet, err := wr.readTablet(zkTabletPath)
if err != nil {
return
}
actionPath, err = wr.ai.ReparentShard(zkShardPath, zkTabletPath)
if err != nil {
return
}
// Make sure two of these don't get scheduled at the same time.
ok, err := zk.ObtainQueueLock(wr.zconn, actionPath, false)
if err != nil {
return
}
if !ok {
// just clean up for now, in the future we may want to try harder, or wait
wr.zconn.Delete(actionPath, -1)
panic(fmt.Errorf("failed to obtain action lock: %v", actionPath))
}
go wr.reparentShardHandler(shardInfo, tablet, actionPath)
return
}
/*
shardInfo for the shard we want to reparent.
masterElectTablet is the shart we want to promote when the time comes.
zkShardActionPath - zk path to the node representing this action.
*/
func (wr *Wrangler) reparentShardHandler(shardInfo *tm.ShardInfo, masterElectTablet *tm.TabletInfo, zkShardActionPath string) {
relog.Debug("reparentShard starting %#v %v", masterElectTablet, zkShardActionPath)
reparentErr := wr.reparentShard(shardInfo, masterElectTablet, zkShardActionPath)
relog.Debug("reparentShard finished %v", reparentErr)
var err error
if reparentErr == nil {
err = zk.DeleteRecursive(wr.zconn, zkShardActionPath, -1)
} else {
data, stat, err := wr.zconn.Get(zkShardActionPath)
if err == nil {
var actionNode *tm.ActionNode
actionNode, err = tm.ActionNodeFromJson(data, zkShardActionPath)
if err == nil {
actionNode.Error = reparentErr.Error()
data = tm.ActionNodeToJson(actionNode)
_, err = wr.zconn.Set(zkShardActionPath, data, stat.Version())
}
}
}
if err != nil {
relog.Error("action node update failed: %v", err)
if reparentErr != nil {
relog.Fatal("reparent failed: %v", reparentErr)
}
}
}
func (wr *Wrangler) reparentShard(shardInfo *tm.ShardInfo, masterElectTablet *tm.TabletInfo, zkShardActionPath string) error {
// Get shard's master tablet.
zkMasterTabletPath, err := shardInfo.MasterTabletPath()
if err != nil {
return err
}
masterTablet, err := wr.readTablet(zkMasterTabletPath)
if err != nil {
return err
}
// Validate a bunch of assumptions we make about the replication graph.
if masterTablet.Parent.Uid != tm.NO_TABLET {
return fmt.Errorf("masterTablet has ParentUid: %v", masterTablet.Parent.Uid)
}
// FIXME(msolomon) this assumes no hierarchical replication, which is currently the case.
tabletAliases, err := tm.FindAllTabletAliasesInShard(wr.zconn, shardInfo)
relog.Debug("shardUids: %v", tabletAliases)
if err != nil {
return err
}
slaveTabletMap := make(map[uint]*tm.TabletInfo)
// FIXME(msolomon) this assumes that the replica nodes must all be in a good
// state when the reparent happens. the better thing to guarantee is that
// *enough* replica nodes are in a good state. In fact, "enough" is probably a function
// of each datacenter. It's complicated.
// FIXME(msolomon) handle multiple datacenters
for _, alias := range tabletAliases {
if alias.Uid == masterTablet.Uid {
// skip master
continue
}
tablet, err := wr.readTablet(shardInfo.TabletPath(alias))
if err != nil {
return fmt.Errorf("tablet unavailable: %v", err)
}
if tablet.Parent.Uid != masterTablet.Uid {
return fmt.Errorf("tablet not slaved correctly, expected %v, found %v", masterTablet.Uid, tablet.Parent.Uid)
}
slaveTabletMap[alias.Uid] = tablet
}
relog.Debug("read all slave tablets")
var masterPosition *mysqlctl.ReplicationPosition
// If the masterTablet type doesn't match, we can assume that it's been
// removed by other operations. For instance, a DBA or health-check process
// setting it's type to SCRAP.
if masterTablet.Type == tm.TYPE_MASTER && masterTablet.Uid != masterElectTablet.Uid {
relog.Debug("demote master %v", zkMasterTabletPath)
actionPath, err := wr.ai.DemoteMaster(zkMasterTabletPath)
if err == nil {
err = wr.ai.WaitForCompletion(actionPath, 1*time.Minute)
}
if err != nil {
// FIXME(msolomon) This suggests that the master is dead and we need to take steps.
return err
}
masterPosition, err = getMasterPosition(masterTablet.Tablet)
if err != nil {
// FIXME(msolomon) handle the case where the master is failed, not demoted.
return err
}
}
if masterTablet.Uid != masterElectTablet.Uid {
relog.Debug("check slaves %v", zkMasterTabletPath)
err = checkSlaveConsistency(slaveTabletMap, masterPosition)
if err != nil {
return err
}
}
zkMasterElectPath := masterElectTablet.Path()
relog.Debug("promote slave %v", zkMasterElectPath)
actionPath, err := wr.ai.PromoteSlave(zkMasterElectPath, zkShardActionPath)
if err == nil {
err = wr.ai.WaitForCompletion(actionPath, 1*time.Minute)
}
if err != nil {
// FIXME(msolomon) This suggests that the master-elect is dead.
// We need to classify certain errors as temporary and retry.
return err
}
// Once the slave is promoted, remove it from our map
if masterTablet.Uid != masterElectTablet.Uid {
delete(slaveTabletMap, masterElectTablet.Uid)
}
// FIXME(msolomon) actions could block forever - set a reasonable deadline
restartSlaveErrors := make([]error, 0, len(slaveTabletMap))
// FIXME(msolomon) could be done in parallel
for _, slaveTablet := range slaveTabletMap {
relog.Debug("restart slave %v", slaveTablet.Path())
actionPath, err := wr.ai.RestartSlave(slaveTablet.Path(), zkShardActionPath)
if err == nil {
err = wr.ai.WaitForCompletion(actionPath, 1*time.Minute)
}
if err != nil {
// FIXME(msolomon) Don't bail early, just mark this phase as failed. We might
// decide to proceed if enough of these succeed.
// FIXME(msolomon) This is a somewhat delicate retry - have to figure out
// why it failed on the tablet end. This could lead to a nasty case of having
// to recompute where to start replication. Practically speaking, that chance
// is pretty low.
relog.Warning("restart slave failed: %v", err)
restartSlaveErrors = append(restartSlaveErrors, err)
}
}
if masterTablet.Uid != masterElectTablet.Uid {
relog.Debug("scrap demoted master %v", zkMasterTabletPath)
// If there is a master, scrap it for now.
// We could reintroduce it and reparent it and use it as new replica.
if masterPosition != nil {
scrapActionPath, scrapErr := wr.ai.Scrap(zkMasterTabletPath)
if scrapErr != nil {
relog.Warning("initiating scrap failed: %v", scrapErr)
} else {
err = wr.ai.WaitForCompletion(scrapActionPath, 1*time.Minute)
if err != nil {
relog.Warning("waiting for scrap failed: %v", err)
}
}
} else {
relog.Debug("forcing scrap: %v", zkMasterTabletPath)
// The master is dead, so we have to take action ourselves since we presume
// the actor will not get the initiation for some time.
if scrapErr := tm.Scrap(wr.zconn, zkMasterTabletPath, false); scrapErr != nil {
relog.Warning("forcing scrap failed: %v", scrapErr)
}
}
}
relog.Debug("update shard")
err = tm.RebuildShard(wr.zconn, shardInfo.ShardPath())
if err != nil {
return err
}
if len(restartSlaveErrors) > 0 {
msgs := make([]string, len(restartSlaveErrors))
for i, e := range restartSlaveErrors {
msgs[i] = e.Error()
}
// This is more of a warning at this point.
// FIXME(msolomon) classify errors
err = fmt.Errorf("restart slaves failed (%v): %v", len(msgs), strings.Join(msgs, ", "))
}
return err
}
func getMasterPosition(tablet *tm.Tablet) (*mysqlctl.ReplicationPosition, error) {
timer := time.NewTimer(SLAVE_STATUS_DEADLINE)
defer timer.Stop()
callChan := make(chan *rpc.Call, 1)
var client *rpc.Client
go func() {
var clientErr error
client, clientErr := rpc.DialHTTP("tcp", tablet.Addr)
if clientErr != nil {
callChan <- &rpc.Call{Error: fmt.Errorf("dial failed: %v", clientErr)}
} else {
client.Go("TabletManager.MasterPosition", vtrpc.NilRequest, new(mysqlctl.ReplicationPosition), callChan)
}
}()
var call *rpc.Call
select {
case <-timer.C:
case call = <-callChan:
}
if client != nil {
client.Close()
}
if call == nil {
return nil, fmt.Errorf("TabletManager.MasterPosition deadline exceeded %v", tablet.Addr)
}
if call.Error != nil {
return nil, call.Error
}
return call.Reply.(*mysqlctl.ReplicationPosition), nil
}
type rpcContext struct {
tablet *tm.TabletInfo
client *rpc.Client
position *mysqlctl.ReplicationPosition
err error
}
/* Check all the tablets to see if we can proceed with reparenting.
masterPosition is supplied from the demoted master if we are doing this gracefully.
*/
func checkSlaveConsistency(tabletMap map[uint]*tm.TabletInfo, masterPosition *mysqlctl.ReplicationPosition) error {
relog.Debug("checkSlaveConsistency %#v %#v", tabletMap, masterPosition)
timer := time.NewTimer(SLAVE_STATUS_DEADLINE)
defer timer.Stop()
// FIXME(msolomon) Something still feels clumsy here and I can't put my finger on it.
calls := make(chan *rpcContext, len(tabletMap))
for _, tablet := range tabletMap {
go func() {
ctx := &rpcContext{tablet: tablet}
ctx.client, ctx.err = rpc.DialHTTP("tcp", tablet.Addr)
if ctx.err == nil {
ctx.position = new(mysqlctl.ReplicationPosition)
ctx.err = ctx.client.Call("TabletManager.SlavePosition", vtrpc.NilRequest, ctx.position)
ctx.client.Close()
}
calls <- ctx
}()
}
replies := make([]*rpcContext, 0, len(tabletMap))
// wait for responses
for i := 0; i < len(tabletMap); i++ {
select {
case <-timer.C:
break
case call := <-calls:
replies = append(replies, call)
}
}
replyErrorCount := len(tabletMap) - len(replies)
// map positions to tablets
positionMap := make(map[string][]uint)
for _, ctx := range replies {
if ctx.err != nil {
replyErrorCount++
} else {
mapKey := ctx.position.MapKey()
if _, ok := positionMap[mapKey]; !ok {
positionMap[mapKey] = make([]uint, 0, 32)
}
positionMap[mapKey] = append(positionMap[mapKey], ctx.tablet.Uid)
}
}
if len(positionMap) == 1 && replyErrorCount == 0 {
// great, everyone agrees
// demotedMasterReplicationState is nil if demotion failed
if masterPosition != nil {
demotedMapKey := masterPosition.MapKey()
if _, ok := positionMap[demotedMapKey]; !ok {
for slaveMapKey, _ := range positionMap {
return fmt.Errorf("slave position doesn't match demoted master: %v != %v", demotedMapKey,
slaveMapKey)
}
}
}
} else {
// FIXME(msolomon) in the event of a crash, do you pick replica that is
// furthest along or do you promote the majority? data loss vs availability
// sounds like you pick the latest group and reclone.
items := make([]string, 0, 32)
for slaveMapKey, uids := range positionMap {
items = append(items, fmt.Sprintf("%v (%v)", slaveMapKey, uids))
}
sort.Strings(items)
// FIXME(msolomon) add instructions how to do so.
return fmt.Errorf("inconsistent slaves, mark some offline (%v) %v %#v", replyErrorCount, strings.Join(items, ", "), positionMap)
}
return nil
}

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

@ -0,0 +1,51 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package zkwrangler
import (
"log"
"testing"
"launchpad.net/gozk/zookeeper"
)
var zconn *zookeeper.Conn
func init() {
// This test requires considerable setup so short circuit during sanity checks.
if testing.Short() {
return
}
_zconn, session, err := zookeeper.Dial("localhost:2181", 1e9)
if err != nil {
log.Fatal("zk connect failed: %v", err.Error())
}
event := <-session
if event.State != zookeeper.STATE_CONNECTED {
log.Fatal("zk connect failed: %v", event.State)
}
zconn = _zconn
}
func TestGetMasterPosition(t *testing.T) {
if testing.Short() {
t.Logf("skipping")
return
}
wr := NewWrangler(zconn, nil)
tablet, err := wr.readTablet("/zk/test/vt/tablets/0000062344")
if err != nil {
t.Error(err)
}
replicationPosition, err := getMasterPosition(tablet)
if err != nil {
t.Error(err)
}
if replicationPosition.MapKey() == ":0" {
t.Errorf("empty replicationPosition")
}
t.Logf("replicationPosition: %#v %v", replicationPosition, replicationPosition.MapKey())
}

115
go/zk/config.go Normal file
Просмотреть файл

@ -0,0 +1,115 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package zk
import (
"encoding/json"
"flag"
"fmt"
"log"
"os"
"strings"
// "launchpad.net/gozk/zookeeper"
)
var zkConfigPaths = []string{"/etc/zookeeper/zk_client.json"}
var localCell = flag.String("zk.local-cell", "",
"closest zk cell used for /zk/local paths")
var localAddrs = flag.String("zk.local-addrs", "",
"list of zookeeper servers (host:port, ...)")
var globalAddrs = flag.String("zk.global-addrs", "",
"list of global zookeeper servers (host:port, ...)")
// Read the cell from -zk.local-cell, or the environment ZK_CLIENT_LOCAL_CELL
// or guess the cell by the hostname. This is either the first two characters
// or the character before a dash '-'.
func guessLocalCell() string {
if *localCell != "" {
return *localCell
}
envCell := os.Getenv("ZK_CLIENT_LOCAL_CELL")
if envCell != "" {
return envCell
}
hostname, err := os.Hostname()
if err != nil {
panic(err)
}
shortHostname := strings.Split(hostname, ".")[0]
hostParts := strings.Split(shortHostname, "-")
if len(hostParts) > 1 {
return hostParts[0]
}
return shortHostname[:2]
}
func ZkCellFromZkPath(zkPath string) string {
defer func() {
if x := recover(); x != nil {
panic(fmt.Errorf("no cell name in path: %v", zkPath))
}
}()
pathParts := strings.Split(zkPath, "/")
cell := pathParts[2]
if strings.Contains(cell, "-") {
panic(fmt.Errorf("invalid cell name %v", cell))
}
return cell
}
func getConfigPaths() []string {
zkConfigPath := os.Getenv("ZK_CLIENT_CONFIG")
if zkConfigPath != "" {
return []string{zkConfigPath}
}
return zkConfigPaths
}
func ZkPathToZkAddr(zkPath string) string {
if !strings.HasPrefix(zkPath, "/zk") {
panic(fmt.Errorf("invalid zk path: %v", zkPath))
}
var cellAddrMap map[string]string
for _, configPath := range getConfigPaths() {
file, err := os.Open(configPath)
if err != nil {
log.Printf("error reading config file: %v: %v", configPath, err)
continue
}
err = json.NewDecoder(file).Decode(&cellAddrMap)
file.Close()
if err != nil {
log.Printf("error decoding config file %v: %v", configPath, err)
continue
}
break
}
cell := ZkCellFromZkPath(zkPath)
if cell == "local" {
cell = guessLocalCell()
} else if cell == "global" {
if *globalAddrs != "" {
return *globalAddrs
} else if _, ok := cellAddrMap[cell]; !ok {
// if there is no "global" cell, look for a dc-specific
// address for the global cell
cell = guessLocalCell() + "-global"
}
}
addr := cellAddrMap[cell]
if addr != "" {
return addr
}
panic(fmt.Errorf("no addr found for zk cell: %#v", cell))
}

50
go/zk/config_test.go Normal file
Просмотреть файл

@ -0,0 +1,50 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package zk
import (
"encoding/json"
"fmt"
"os"
"testing"
"time"
)
func TestZkPathToZkAddr(t *testing.T) {
configPath := fmt.Sprintf("./.zk-test-conf-%v", time.Now().UnixNano())
defer func() {
os.Remove(configPath)
}()
if err := os.Setenv("ZK_CLIENT_CONFIG", configPath); err != nil {
t.Errorf("setenv failed: %v", err)
}
hostname, err := os.Hostname()
if err != nil {
t.Errorf("hostname failed: %v")
}
fakeCell := hostname[:2]
fakeAddr := "localhost:2181"
configMap := map[string]string{fakeCell: fakeAddr}
file, err := os.Create(configPath)
if err != nil {
t.Errorf("create failed: %v")
}
err = json.NewEncoder(file).Encode(configMap)
if err != nil {
t.Errorf("encode failed: %v")
}
file.Close()
for _, path := range []string{"/zk/" + fakeCell, "/zk/" + fakeCell + "/", "/zk/local", "/zk/local/"} {
zkAddr := ZkPathToZkAddr(path)
if zkAddr != fakeAddr {
t.Errorf("addr mismatch for path %v %v != %v", path, zkAddr, fakeAddr)
}
}
}

103
go/zk/conn_cache.go Normal file
Просмотреть файл

@ -0,0 +1,103 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package zk
import (
"fmt"
"log"
"sync"
"time"
"launchpad.net/gozk/zookeeper"
)
/* When you need to talk to multiple zk cells, you need a simple
abstraction so you aren't caching clients all over the place.
ConnCache guarantees that you have at most one zookeeper connection per cell.
*/
type cachedConn struct {
mutex sync.Mutex // used to notify if multiple goroutine simultaneously want a connection
zconn *zookeeper.Conn
}
type ConnCache struct {
mutex sync.Mutex
zconnCellMap map[string]*cachedConn // map cell name to connection
connectTimeout time.Duration
}
func (cc *ConnCache) ConnForPath(zkPath string) (*zookeeper.Conn, error) {
zcell := ZkCellFromZkPath(zkPath)
cc.mutex.Lock()
conn, ok := cc.zconnCellMap[zcell]
if !ok {
conn = &cachedConn{}
cc.zconnCellMap[zcell] = conn
}
cc.mutex.Unlock()
// We only want one goroutine at a time trying to connect here, so keep the
// lock during the zk dial process.
conn.mutex.Lock()
defer conn.mutex.Unlock()
if conn.zconn != nil {
return conn.zconn, nil
}
zconn, session, err := zookeeper.Dial(ZkPathToZkAddr(zkPath), cc.connectTimeout)
if err == nil {
// Wait for connection.
// FIXME(msolomon) the deadlines seems to be a bit fuzzy, need to double check
// and potentially do a high-level select here.
event := <-session
if event.State != zookeeper.STATE_CONNECTED {
err = fmt.Errorf("zk connect failed: %v", event.State)
}
if err == nil {
conn.zconn = zconn
go cc.handleSessionEvents(zcell, zconn, session)
} else {
zconn.Close()
}
}
return conn.zconn, err
}
func (cc *ConnCache) handleSessionEvents(cell string, conn *zookeeper.Conn, session <-chan zookeeper.Event) {
for event := range session {
if !event.Ok() {
conn.Close()
cc.mutex.Lock()
delete(cc.zconnCellMap, cell)
cc.mutex.Unlock()
log.Printf("zk conn cache: session for cell %v ended: %v", cell, event)
}
}
}
func (cc *ConnCache) Close() error {
cc.mutex.Lock()
defer cc.mutex.Unlock()
for _, conn := range cc.zconnCellMap {
conn.mutex.Lock()
if conn.zconn != nil {
conn.zconn.Close()
conn.zconn = nil
}
conn.mutex.Unlock()
}
cc.zconnCellMap = nil
return nil
}
func NewConnCache(connectTimeout time.Duration) *ConnCache {
return &ConnCache{
zconnCellMap: make(map[string]*cachedConn),
connectTimeout: connectTimeout}
}

145
go/zk/global.go Normal file
Просмотреть файл

@ -0,0 +1,145 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
/* Emulate a "global" namespace across n zk quorums. */
package zk
import (
"fmt"
"sort"
"time"
"launchpad.net/gozk/zookeeper"
)
const (
DEFAULT_MAX_RETRIES = 3
)
type GlobalZookeeperError string
func (e GlobalZookeeperError) Error() string {
return string(e)
}
type GlobalConn struct {
serverAddrs []string
zconns []*zookeeper.Conn
maxRetries int
}
func Dial(serverAddrs []string, recvTimeout time.Duration) (*GlobalConn, <-chan zookeeper.Event, error) {
zconns := make([]*zookeeper.Conn, len(serverAddrs))
zchans := make([]<-chan zookeeper.Event, len(serverAddrs))
for i, addr := range serverAddrs {
conn, eventChan, err := zookeeper.Dial(addr, recvTimeout)
if err != nil {
// teardown
for j := 0; j < i; j++ {
zconns[j].Close()
}
return nil, nil, err
}
zconns[i] = conn
zchans[i] = eventChan
}
eventChan := make(chan zookeeper.Event, 1)
go func() {
var e zookeeper.Event
for _, c := range zchans {
e = <-c
}
eventChan <- e
close(eventChan)
}()
return &GlobalConn{serverAddrs, zconns, DEFAULT_MAX_RETRIES}, eventChan, nil
}
func (gzc *GlobalConn) Close() (err error) {
for _, zc := range gzc.zconns {
if zcErr := zc.Close(); zcErr != nil {
err = zcErr
}
}
return
}
func (gzc *GlobalConn) Create(path, value string, flags int, aclv []zookeeper.ACL) (pathCreated string, err error) {
createdPaths := make([]string, len(gzc.zconns))
errs := make([]error, len(gzc.zconns))
for i, zconn := range gzc.zconns {
createdPaths[i], errs[i] = zconn.Create(path, value, flags, aclv)
if errs[i] != nil {
return "", fmt.Errorf("global create error: %v %v", createdPaths, errs[i])
}
if createdPaths[0] != createdPaths[i] {
return "", fmt.Errorf("inconsistent global create: %v", createdPaths)
}
}
return createdPaths[0], errs[0]
}
func (gzc *GlobalConn) Get(path string) (data string, stat *zookeeper.Stat, err error) {
datas := make([]string, len(gzc.zconns))
for i, zconn := range gzc.zconns {
data, stat, err = zconn.Get(path)
if err != nil {
return "", nil, fmt.Errorf("global get error: %v", err)
}
datas[i] = data
if datas[0] != data {
return "", nil, fmt.Errorf("inconsistent global get: %v", err)
}
}
return
}
func (gzc *GlobalConn) Children(path string) (children []string, stat *zookeeper.Stat, err error) {
childrens := make([][]string, len(gzc.zconns))
for i, zconn := range gzc.zconns {
children, stat, err = zconn.Children(path)
sort.Strings(children)
if err != nil {
return nil, nil, fmt.Errorf("global children error: %v", err)
}
childrens[i] = children
if !eqSlice(childrens[0], children) {
return nil, nil, fmt.Errorf("inconsistent global children: %v", err)
}
}
return
}
func eqSlice(a, b []string) bool {
if len(a) != len(b) {
return false
}
for i, x := range a {
if b[i] != x {
return false
}
}
return true
}
func (gzc *GlobalConn) Set(path, value string, version int) (stat *zookeeper.Stat, err error) {
for _, zconn := range gzc.zconns {
stat, err = zconn.Set(path, value, version)
if err != nil {
return nil, fmt.Errorf("inconsistent global set: %v", err)
}
}
return stat, nil
}
func (gzc *GlobalConn) Delete(path string, version int) (err error) {
for _, zconn := range gzc.zconns {
err = zconn.Delete(path, version)
if err != nil {
return fmt.Errorf("inconsistent global delete: %v", err)
}
}
return
}

165
go/zk/metaconn.go Normal file
Просмотреть файл

@ -0,0 +1,165 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package zk
import (
"strings"
"time"
"launchpad.net/gozk/zookeeper"
)
type Conn interface {
Get(path string) (data string, stat *zookeeper.Stat, err error)
GetW(path string) (data string, stat *zookeeper.Stat, watch <-chan zookeeper.Event, err error)
Children(path string) (children []string, stat *zookeeper.Stat, err error)
ChildrenW(path string) (children []string, stat *zookeeper.Stat, watch <-chan zookeeper.Event, err error)
Exists(path string) (stat *zookeeper.Stat, err error)
ExistsW(path string) (stat *zookeeper.Stat, watch <-chan zookeeper.Event, err error)
Create(path, value string, flags int, aclv []zookeeper.ACL) (pathCreated string, err error)
Set(path, value string, version int) (stat *zookeeper.Stat, err error)
Delete(path string, version int) (err error)
Close() error
RetryChange(path string, flags int, acl []zookeeper.ACL, changeFunc zookeeper.ChangeFunc) error
ACL(path string) ([]zookeeper.ACL, *zookeeper.Stat, error)
SetACL(path string, aclv []zookeeper.ACL, version int) error
}
/* Smooth API to talk to any zk path in the global system. Emulates
"/zk/local" paths by guessing and substituting the correct cell for
your current environment. */
type MetaConn struct {
connCache *ConnCache
}
func resolveZkPath(path string) string {
cell := ZkCellFromZkPath(path)
if cell != "local" {
return path
}
parts := strings.Split(path, "/")
parts[2] = guessLocalCell()
return strings.Join(parts, "/")
}
func (conn *MetaConn) Get(path string) (data string, stat *zookeeper.Stat, err error) {
zconn, err := conn.connCache.ConnForPath(path)
if err != nil {
return
}
return zconn.Get(resolveZkPath(path))
}
func (conn *MetaConn) GetW(path string) (data string, stat *zookeeper.Stat, watch <-chan zookeeper.Event, err error) {
zconn, err := conn.connCache.ConnForPath(path)
if err != nil {
return
}
return zconn.GetW(resolveZkPath(path))
}
func (conn *MetaConn) Children(path string) (children []string, stat *zookeeper.Stat, err error) {
zconn, err := conn.connCache.ConnForPath(path)
if err != nil {
return
}
return zconn.Children(resolveZkPath(path))
}
func (conn *MetaConn) ChildrenW(path string) (children []string, stat *zookeeper.Stat, watch <-chan zookeeper.Event, err error) {
zconn, err := conn.connCache.ConnForPath(path)
if err != nil {
return
}
return zconn.ChildrenW(resolveZkPath(path))
}
func (conn *MetaConn) Exists(path string) (stat *zookeeper.Stat, err error) {
zconn, err := conn.connCache.ConnForPath(path)
if err != nil {
return
}
return zconn.Exists(resolveZkPath(path))
}
func (conn *MetaConn) ExistsW(path string) (stat *zookeeper.Stat, watch <-chan zookeeper.Event, err error) {
zconn, err := conn.connCache.ConnForPath(path)
if err != nil {
return
}
return zconn.ExistsW(resolveZkPath(path))
}
func (conn *MetaConn) Create(path, value string, flags int, aclv []zookeeper.ACL) (pathCreated string, err error) {
zconn, err := conn.connCache.ConnForPath(path)
if err != nil {
return
}
return zconn.Create(resolveZkPath(path), value, flags, aclv)
}
func (conn *MetaConn) Set(path, value string, version int) (stat *zookeeper.Stat, err error) {
zconn, err := conn.connCache.ConnForPath(path)
if err != nil {
return
}
return zconn.Set(resolveZkPath(path), value, version)
}
func (conn *MetaConn) Delete(path string, version int) (err error) {
zconn, err := conn.connCache.ConnForPath(path)
if err != nil {
return
}
return zconn.Delete(resolveZkPath(path), version)
}
func (conn *MetaConn) Close() error {
return conn.connCache.Close()
}
func (conn *MetaConn) RetryChange(path string, flags int, acl []zookeeper.ACL, changeFunc zookeeper.ChangeFunc) error {
zconn, err := conn.connCache.ConnForPath(path)
if err != nil {
return err
}
return zconn.RetryChange(resolveZkPath(path), flags, acl, changeFunc)
}
func (conn *MetaConn) ACL(path string) ([]zookeeper.ACL, *zookeeper.Stat, error) {
zconn, err := conn.connCache.ConnForPath(path)
if err != nil {
return nil, nil, err
}
return zconn.ACL(path)
}
func (conn *MetaConn) SetACL(path string, aclv []zookeeper.ACL, version int) error {
zconn, err := conn.connCache.ConnForPath(path)
if err != nil {
return err
}
return zconn.SetACL(path, aclv, version)
}
/*
NOTE(msolomon) not a good idea
func (conn *MetaConn) GetSession(path string) <-chan zookeeper.Event {
return conn.connCache.SessionForPath(path)
}
*/
func NewMetaConn(connectTimeout time.Duration) *MetaConn {
return &MetaConn{NewConnCache(connectTimeout)}
}

178
go/zk/zkctl/zkconf.go Normal file
Просмотреть файл

@ -0,0 +1,178 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
/*
Generate zoo.conf files from templates.
*/
package zkctl
import (
"bytes"
"fmt"
"io/ioutil"
"os"
"path"
"strconv"
"strings"
"text/template"
)
type zkServerAddr struct {
ServerId uint
Hostname string
LeaderPort int
ElectionPort int
ClientPort int
}
type ZkConfig struct {
ServerId uint
ClientPort int
Servers []zkServerAddr
Global bool
}
/* ServerId is a unique id for a server - must be 1-255
*/
func NewZkConfig() *ZkConfig {
return &ZkConfig{
ClientPort: 2181,
Servers: make([]zkServerAddr, 0, 16),
}
}
func (cnf *ZkConfig) DataDir() string {
if cnf.Global {
return fmt.Sprintf("/vt/zk_global_%03d", cnf.ServerId)
}
return fmt.Sprintf("/vt/zk_%03d", cnf.ServerId)
}
func (cnf *ZkConfig) DirectoryList() []string {
return []string{
cnf.DataDir(),
cnf.LogDir(),
}
}
func (cnf *ZkConfig) LogDir() string {
return path.Join(cnf.DataDir(), "logs")
}
func (cnf *ZkConfig) ConfigFile() string {
return path.Join(cnf.DataDir(), "zoo.cfg")
}
func (cnf *ZkConfig) PidFile() string {
return path.Join(cnf.DataDir(), "zk.pid")
}
func (cnf *ZkConfig) MyidFile() string {
return path.Join(cnf.DataDir(), "myid")
}
func (cnf *ZkConfig) WriteMyid() error {
return ioutil.WriteFile(cnf.MyidFile(), []byte(fmt.Sprintf("%v", cnf.ServerId)), 0664)
}
/*
Join cnf files cnfPaths and subsitute in the right values.
*/
func MakeZooCfg(cnfPaths []string, cnf *ZkConfig, header string) (string, error) {
myTemplateSource := new(bytes.Buffer)
for _, line := range strings.Split(header, "\n") {
fmt.Fprintf(myTemplateSource, "## %v\n", strings.TrimSpace(line))
}
for _, path := range cnfPaths {
data, dataErr := ioutil.ReadFile(path)
if dataErr != nil {
return "", dataErr
}
myTemplateSource.WriteString("## " + path + "\n")
myTemplateSource.Write(data)
}
myTemplate, err := template.New("foo").Parse(myTemplateSource.String())
if err != nil {
return "", err
}
cnfData := new(bytes.Buffer)
err = myTemplate.Execute(cnfData, cnf)
if err != nil {
return "", err
}
return cnfData.String(), nil
}
const GUESS_MYID = 0
func MakeZooCfgForString(cmdLine, cnfPath, header string) (string, error) {
cnfs := []string{"zoo"}
paths := make([]string, len(cnfs))
for i, name := range cnfs {
paths[i] = fmt.Sprintf("%v/%v.cfg", cnfPath, name)
}
zkConfig := MakeZkConfigFromString(cmdLine, GUESS_MYID)
return MakeZooCfg(paths, zkConfig, header)
}
/*
Create a config for this instance. Search cnfPath for the appropriate
cnf template files.
<server_id>@<hostname>:<leader_port>:<election_port>:<client_port>
If server_id > 1000, then we assume this is a global quorum.
server_id's must be 1-255, global id's are 1001-1255 mod 1000.
*/
func MakeZkConfigFromString(cmdLine string, myId uint) *ZkConfig {
zkConfig := NewZkConfig()
for _, zki := range strings.Split(cmdLine, ",") {
zkiParts := strings.SplitN(zki, "@", 2)
if len(zkiParts) != 2 {
panic("bad command line format for zk config")
}
zkId := zkiParts[0]
zkAddrParts := strings.Split(zkiParts[1], ":")
serverId, _ := strconv.ParseUint(zkId, 10, 0)
if serverId > 1000 {
serverId = serverId % 1000
zkConfig.Global = true
}
zkServer := zkServerAddr{ServerId: uint(serverId), ClientPort: 2181,
LeaderPort: 2888, ElectionPort: 3888}
switch len(zkAddrParts) {
case 4:
zkServer.ClientPort, _ = strconv.Atoi(zkAddrParts[3])
fallthrough
case 3:
zkServer.ElectionPort, _ = strconv.Atoi(zkAddrParts[2])
fallthrough
case 2:
zkServer.LeaderPort, _ = strconv.Atoi(zkAddrParts[1])
fallthrough
case 1:
zkServer.Hostname = zkAddrParts[0]
// if !strings.Contains(zkServer.Hostname, ".") {
// panic(fmt.Errorf("expected fully qualified hostname: %v", zkServer.Hostname))
// }
default:
panic(fmt.Errorf("bad command line format for zk config"))
}
zkConfig.Servers = append(zkConfig.Servers, zkServer)
}
hostname, _ := os.Hostname()
for _, zkServer := range zkConfig.Servers {
if (myId > 0 && myId == zkServer.ServerId) || (myId == 0 && zkServer.Hostname == hostname) {
zkConfig.ServerId = zkServer.ServerId
zkConfig.ClientPort = zkServer.ClientPort
break
}
}
if zkConfig.ServerId == 0 {
panic(fmt.Errorf("no zk server found for host %v in config %v", hostname, cmdLine))
}
return zkConfig
}

285
go/zk/zkctl/zkctl.go Normal file
Просмотреть файл

@ -0,0 +1,285 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
/*
Commands for controlling an external zookeeper process.
*/
package zkctl
import (
"bytes"
"errors"
"fmt"
"io/ioutil"
"net"
"os"
"os/exec"
"path"
"strconv"
"syscall"
"time"
"code.google.com/p/vitess/go/relog"
"launchpad.net/gozk/zookeeper"
)
const (
StartWaitTime = 20 // number of seconds to wait
)
func init() {
zookeeper.SetLogLevel(zookeeper.LOG_WARN)
}
type Zkd struct {
config *ZkConfig
// createConnection CreateConnection
}
// createConnection: closure that returns a new connection or an error
func NewZkd(config *ZkConfig) *Zkd {
return &Zkd{config}
}
func (zkd *Zkd) LocalClientAddr() string {
return fmt.Sprintf("localhost:%v", zkd.config.ClientPort)
}
/*
ZOO_LOG_DIR=""
ZOO_CFG="/.../zoo.cfg"
ZOOMAIN="org.apache.zookeeper.server.quorum.QuorumPeerMain"
java -DZOO_LOG_DIR=${ZOO_LOG_DIR} -cp $CLASSPATH $ZOOMAIN $YT_ZK_CFG
*/
func (zkd *Zkd) Start() error {
relog.Info("zkctl.Start")
// NOTE(msolomon) use a script here so we can detach and continue to run
// if the wrangler process dies. this pretty much the same as mysqld_safe.
dirs := []string{
os.ExpandEnv("$VTROOT/dist/vt/bin"),
os.ExpandEnv("$VTROOT/src/code.google.com/p/vitess.x/go/zk/zkctl"),
}
var name string
var dir string
for _, dir = range dirs {
name = path.Join(dir, "zksrv.sh")
if _, statErr := os.Stat(name); statErr == nil {
break
}
}
arg := []string{
zkd.config.LogDir(),
zkd.config.ConfigFile(),
zkd.config.PidFile(),
}
cmd := exec.Command(name, arg...)
cmd.Env = os.Environ()
cmd.Dir = dir
if err := cmd.Start(); err != nil {
return err
}
// give it some time to succeed - usually by the time the socket emerges
// we are in good shape
for i := 0; i < StartWaitTime; i++ {
conn, err := net.Dial("tcp", fmt.Sprintf(":%v", zkd.config.ClientPort))
if err != nil {
time.Sleep(1e9)
continue
} else {
conn.Close()
}
}
// wait so we don't get a bunch of defunct processes
go cmd.Wait()
return nil
}
func (zkd *Zkd) Shutdown() error {
relog.Info("zkctl.Shutdown")
pidData, err := ioutil.ReadFile(zkd.config.PidFile())
if err != nil {
return err
}
pid, err := strconv.Atoi(string(bytes.TrimSpace(pidData)))
if err != nil {
return err
}
err = syscall.Kill(pid, 9)
if err != nil && err != syscall.ESRCH {
return err
}
return nil
}
func (zkd *Zkd) makeCfg() (string, error) {
cnfTemplatePaths := []string{os.ExpandEnv("$VTROOT/src/code.google.com/p/vitess.x/config/zkcfg/zoo.cfg")}
return MakeZooCfg(cnfTemplatePaths, zkd.config,
"# generated by vt")
}
func (zkd *Zkd) ConfigChanged() (bool, error) {
oldCfg, err := ioutil.ReadFile(zkd.config.ConfigFile())
if err != nil {
if pathErr, ok := err.(*os.PathError); ok && pathErr.Err == syscall.ENOENT {
oldCfg = []byte{}
err = nil
} else {
return false, err
}
}
currentCfg, err := zkd.makeCfg()
if err != nil {
return false, err
}
return string(oldCfg) != currentCfg, nil
}
func (zkd *Zkd) Init() error {
if zkd.Inited() {
return fmt.Errorf("zk already inited")
}
return zkd.init(false)
}
func (zkd *Zkd) Reinit(preserveData bool) (err error) {
if preserveData {
err = zkd.Shutdown()
} else {
err = zkd.Teardown()
}
if err == nil {
err = zkd.init(preserveData)
}
return
}
func (zkd *Zkd) init(preserveData bool) error {
relog.Info("zkd.Init")
for _, path := range zkd.config.DirectoryList() {
if err := os.MkdirAll(path, 0775); err != nil {
relog.Error(err.Error())
return err
}
// FIXME(msolomon) validate permissions?
}
configData, err := zkd.makeCfg()
if err == nil {
err = ioutil.WriteFile(zkd.config.ConfigFile(), []byte(configData), 0664)
}
if err != nil {
relog.Error("failed creating %v: %v", zkd.config.ConfigFile(), err)
return err
}
err = zkd.config.WriteMyid()
if err != nil {
relog.Error("failed creating %v: %v", zkd.config.MyidFile(), err)
return err
}
if err = zkd.Start(); err != nil {
relog.Error("failed starting, check %v", zkd.config.LogDir())
return err
}
zkAddr := fmt.Sprintf("localhost:%v", zkd.config.ClientPort)
zk, session, err := zookeeper.Dial(zkAddr, StartWaitTime*time.Second)
if err != nil {
return err
}
event := <-session
if event.State != zookeeper.STATE_CONNECTED {
return err
}
defer zk.Close()
if !preserveData {
_, err = zk.Create("/zk", "", 0, zookeeper.WorldACL(zookeeper.PERM_ALL))
if err != nil {
return err
}
if zkd.config.Global {
_, err = zk.Create("/zk/global", "", 0, zookeeper.WorldACL(zookeeper.PERM_ALL))
if err != nil {
return err
}
}
}
return nil
}
func (zkd *Zkd) Teardown() error {
relog.Info("zkctl.Teardown")
if err := zkd.Shutdown(); err != nil {
relog.Error("failed zookeeper shutdown: %v", err.Error())
return err
}
var removalErr error
for _, dir := range zkd.config.DirectoryList() {
relog.Debug("remove data dir %v", dir)
if err := os.RemoveAll(dir); err != nil {
relog.Error("failed removing %v: %v", dir, err.Error())
removalErr = err
}
}
return removalErr
}
func (zkd *Zkd) CheckProcess() error {
pidFile := zkd.config.PidFile()
data, err := ioutil.ReadFile(pidFile)
if err != nil {
return err
}
pid, err := strconv.Atoi(string(data))
// found a pid - if the process is burned, fast-fail
// otherwise, try to connect and fail slowly
if err == nil {
_, err := os.FindProcess(pid)
if err != nil {
return err
}
}
zkAddr := fmt.Sprintf("localhost:%v", zkd.config.ClientPort)
zk, session, err := zookeeper.Dial(zkAddr, StartWaitTime*time.Second)
if err != nil {
return err
}
defer zk.Close()
timer := time.NewTimer(StartWaitTime * 1e9)
defer timer.Stop()
select {
case event := <-session:
if event.State != zookeeper.STATE_CONNECTED {
return err
}
case <-timer.C:
return errors.New("zk deadline exceeded connecting to " + zkAddr)
}
_, _, err = zk.Get("/zk")
return err
}
func (zkd *Zkd) Inited() bool {
myidFile := zkd.config.MyidFile()
_, statErr := os.Stat(myidFile)
if statErr == nil {
return true
} else if statErr.(*os.PathError).Err != syscall.ENOENT {
panic("can't access file " + myidFile + ": " + statErr.Error())
}
return false
}

52
go/zk/zkctl/zkctl_test.go Normal file
Просмотреть файл

@ -0,0 +1,52 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package zkctl
import "testing"
func TestLifeCycle(t *testing.T) {
testLifeCycle(t, "255@voltron:2889:3889:2182")
}
func TestLifeCycleGlobal(t *testing.T) {
testLifeCycle(t, "1255@voltron:2890:3890:2183")
}
func testLifeCycle(t *testing.T, config string) {
zkConf := MakeZkConfigFromString(config)
zkd := NewZkd(zkConf)
var err error
err = zkd.Init()
if err != nil {
t.Fatalf("Init() err: %v", err)
}
err = zkd.Shutdown()
if err != nil {
t.Fatalf("Shutdown() err: %v", err)
}
err = zkd.Start()
if err != nil {
t.Fatalf("Start() err: %v", err)
}
err = zkd.Teardown()
if err != nil {
t.Fatalf("Teardown() err: %v", err)
}
}
func testInit(t *testing.T) {
zkConf := MakeZkConfigFromString("255@voltron:2889:3889:2182")
zkd := NewZkd(zkConf)
var err error
err = zkd.Init()
if err != nil {
t.Fatalf("Init() err: %v", err)
}
}

63
go/zk/zkctl/zksrv.sh Executable file
Просмотреть файл

@ -0,0 +1,63 @@
#!/bin/bash
# Copyright 2012, Google Inc. All rights reserved.
# Use of this source code is governed by a BSD-style license that can
# be found in the LICENSE file.
# simple wrapper for starting up zookeeper so it detaches from the parent
# process and ingnores signals
logdir="$1"
config="$2"
pidfile="$3"
zkroot="$VTROOT/dist/vt-zookeeper-3.3.2"
classpath="$VTROOT/dist/vt-zookeeper-3.3.2/lib/zookeeper-3.3.2-fatjar.jar:$VTTOP/dist/vt-zookeeper-3.3.2/lib/"
mkdir -p "$logdir"
touch "$logdir/zksrv.log"
log() {
now=`/bin/date`
echo "$now $*" >> "$logdir/zksrv.log"
return 0
}
for java in /usr/local/bin/java /usr/bin/java; do
if [ -x "$java" ]; then
break
fi
done
if [ ! -x "$java" ]; then
log "ERROR no java binary found"
exit 1
fi
if [ "$VTDEV" ]; then
# use less memory
java="$java -client -Xincgc -Xms1m -Xmx32m"
else
# enable hotspot
java="$java -server"
fi
cmd="$java -DZOO_LOG_DIR=$logdir -cp $classpath org.apache.zookeeper.server.quorum.QuorumPeerMain $config"
start=`/bin/date +%s`
log "INFO starting $cmd"
$cmd < /dev/null &> /dev/null &
pid=$!
log "INFO pid: $pid pidfile: $pidfile"
if [ "$pidfile" ]; then
if [ -f "$pidfile" ]; then
rm "$pidfile"
fi
echo "$pid" > "$pidfile"
fi
wait $pid
log "INFO exit status $pid: $exit_status"

165
go/zk/zkns/zkns.go Normal file
Просмотреть файл

@ -0,0 +1,165 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package zkns
import (
"encoding/json"
"fmt"
"math/rand"
"net"
"sort"
"strings"
"code.google.com/p/vitess.x/go/zk"
)
type ZknsAddr struct {
// These fields came from a Python app originally that used a different
// naming convention.
Host string `json:"host"`
Port int `json:"port"`
NamedPortMap map[string]int `json:"named_port_map"`
version int // zk version to allow non-stomping writes
}
func NewAddr(host string, port int) *ZknsAddr {
return &ZknsAddr{Host: host, Port: port, NamedPortMap: make(map[string]int)}
}
type ZknsAddrs struct {
Entries []ZknsAddr
version int // zk version to allow non-stomping writes
}
func NewAddrs() *ZknsAddrs {
return &ZknsAddrs{Entries: make([]ZknsAddr, 0, 8), version: -1}
}
func toJson(x interface{}) string {
data, err := json.MarshalIndent(x, "", " ")
if err != nil {
panic(err)
}
return string(data)
}
func addrFromJson(data string) *ZknsAddr {
addr := &ZknsAddr{}
if err := json.Unmarshal([]byte(data), addr); err != nil {
panic(err)
}
return addr
}
func ReadAddrs(zconn zk.Conn, zkPath string) (*ZknsAddrs, error) {
data, stat, err := zconn.Get(zkPath)
if err != nil {
return nil, err
}
addrs := new(ZknsAddrs)
err = json.Unmarshal([]byte(data), addrs)
if err != nil {
return nil, err
}
addrs.version = stat.Version()
return addrs, nil
}
// byPriorityWeight sorts records by ascending priority and weight.
type byPriorityWeight []*net.SRV
func (s byPriorityWeight) Len() int { return len(s) }
func (s byPriorityWeight) Swap(i, j int) { s[i], s[j] = s[j], s[i] }
func (s byPriorityWeight) Less(i, j int) bool {
return s[i].Priority < s[j].Priority ||
(s[i].Priority == s[j].Priority && s[i].Weight < s[j].Weight)
}
// shuffleByWeight shuffles SRV records by weight using the algorithm
// described in RFC 2782.
func (addrs byPriorityWeight) shuffleByWeight() {
sum := 0
for _, addr := range addrs {
sum += int(addr.Weight)
}
for sum > 0 && len(addrs) > 1 {
s := 0
n := rand.Intn(sum + 1)
for i := range addrs {
s += int(addrs[i].Weight)
if s >= n {
if i > 0 {
t := addrs[i]
copy(addrs[1:i+1], addrs[0:i])
addrs[0] = t
}
break
}
}
sum -= int(addrs[0].Weight)
addrs = addrs[1:]
}
}
// sort reorders SRV records as specified in RFC 2782.
func (addrs byPriorityWeight) sort() {
sort.Sort(addrs)
i := 0
for j := 1; j < len(addrs); j++ {
if addrs[i].Priority != addrs[j].Priority {
addrs[i:j].shuffleByWeight()
i = j
}
}
addrs[i:].shuffleByWeight()
}
// sort reorders SRV records as specified in RFC 2782.
func Sort(srvs []*net.SRV) {
byPriorityWeight(srvs).sort()
}
/*
zkPath is normally just the path to a file in zk. It can also reference a named
port like this:
/zk/cell/zkns/path:_named_port
*/
func LookupName(zconn zk.Conn, zkPath string) ([]*net.SRV, error) {
zkPathParts := strings.Split(zkPath, ":")
namedPort := ""
if len(zkPathParts) == 2 {
zkPath = zkPathParts[0]
namedPort = zkPathParts[1]
}
addrs, err := ReadAddrs(zconn, zkPath)
if err != nil {
return nil, fmt.Errorf("LookupName failed: %v %v", zkPath, err)
}
srvs := make([]*net.SRV, 0, len(addrs.Entries))
hasError := false
for _, addr := range addrs.Entries {
srv := &net.SRV{Target: addr.Host}
if namedPort == "" {
srv.Port = uint16(addr.Port)
} else {
srv.Port = uint16(addr.NamedPortMap[namedPort])
}
if srv.Port == 0 {
hasError = true
} else {
srvs = append(srvs, srv)
}
}
Sort(srvs)
if hasError && len(srvs) == 0 {
return nil, fmt.Errorf("LookupName failed: %v no valid endpoints found", zkPath)
}
return srvs, nil
}

218
go/zk/zkutil.go Normal file
Просмотреть файл

@ -0,0 +1,218 @@
// Copyright 2012, Google Inc. All rights reserved.
// Use of this source code is governed by a BSD-style
// license that can be found in the LICENSE file.
package zk
import (
"fmt"
"log"
"os"
"path"
"sort"
"strings"
"sync"
"time"
"launchpad.net/gozk/zookeeper"
)
/*
Create a path and any pieces required, think mkdir -p.
*/
func CreateRecursive(zconn Conn, zkPath, value string, flags int, aclv []zookeeper.ACL) (pathCreated string, err error) {
parts := strings.Split(zkPath, "/")
if parts[1] != "zk" {
return "", fmt.Errorf("non zk path: %v", zkPath)
}
if len(parts) > 2 {
tmpPath := "/zk"
for _, p := range parts[2 : len(parts)-1] {
tmpPath = path.Join(tmpPath, p)
_, err = zconn.Create(tmpPath, "", flags, aclv)
if err != nil && err.(*zookeeper.Error).Code != zookeeper.ZNODEEXISTS {
return "", err
}
}
}
return zconn.Create(zkPath, value, flags, aclv)
}
func CreateOrUpdate(zconn Conn, zkPath, value string, flags int, aclv []zookeeper.ACL, recursive bool) (pathCreated string, err error) {
if recursive {
pathCreated, err = CreateRecursive(zconn, zkPath, value, 0, zookeeper.WorldACL(zookeeper.PERM_ALL))
} else {
pathCreated, err = zconn.Create(zkPath, value, 0, zookeeper.WorldACL(zookeeper.PERM_ALL))
}
if err != nil && err.(*zookeeper.Error).Code == zookeeper.ZNODEEXISTS {
pathCreated = ""
_, err = zconn.Set(zkPath, value, -1)
}
return
}
type pathItem struct {
path string
err error
}
func ChildrenRecursive(zconn Conn, zkPath string) ([]string, error) {
var err error
mutex := sync.Mutex{}
wg := sync.WaitGroup{}
pathList := make([]string, 0, 32)
children, _, err := zconn.Children(zkPath)
if err != nil {
return nil, err
}
for _, child := range children {
mutex.Lock()
pathList = append(pathList, child)
mutex.Unlock()
childPath := path.Join(zkPath, child)
childCopy := child
wg.Add(1)
go func() {
rChildren, zkErr := ChildrenRecursive(zconn, childPath)
if zkErr != nil {
mutex.Lock()
err = zkErr
mutex.Unlock()
} else {
mutex.Lock()
for _, rChild := range rChildren {
pathList = append(pathList, path.Join(childCopy, rChild))
}
mutex.Unlock()
}
wg.Done()
}()
}
wg.Wait()
mutex.Lock()
defer mutex.Unlock()
if err != nil {
return nil, err
}
return pathList, nil
}
func DeleteRecursive(zconn Conn, zkPath string, version int) error {
// version: -1 delete any version of the node at path - only applies to the top node
err := zconn.Delete(zkPath, version)
if err == nil {
return nil
}
if err.(*zookeeper.Error).Code != zookeeper.ZNOTEMPTY {
return err
}
// Remove the ability for other nodes to get created while we are trying to delete.
// Otherwise, you can enter a race condition, or get starved out from deleting.
err = zconn.SetACL(zkPath, zookeeper.WorldACL(zookeeper.PERM_ADMIN|zookeeper.PERM_DELETE|zookeeper.PERM_READ), version)
if err != nil {
return err
}
children, _, err := zconn.Children(zkPath)
if err != nil {
return err
}
for _, child := range children {
err := DeleteRecursive(zconn, path.Join(zkPath, child), -1)
if err != nil {
return fmt.Errorf("recursive delete failed: %v", err)
}
}
err = zconn.Delete(zkPath, version)
if err != nil && err.(*zookeeper.Error).Code != zookeeper.ZNOTEMPTY {
err = fmt.Errorf("nodes getting recreated underneath delete (app race condition): %v", zkPath)
}
return err
}
/*
The lexically lowest node is the lock holder - verify that this
path holds the lock. Call this queue-lock because the semantics are
a hybrid. Normal zookeeper locks make assumptions about sequential
numbering that don't hold when the data in a lock is modified.
*/
func ObtainQueueLock(zconn Conn, zkPath string, wait bool) (bool, error) {
if wait {
panic("unimplemented")
}
queueNode := path.Dir(zkPath)
lockNode := path.Base(zkPath)
children, _, err := zconn.Children(queueNode)
if err != nil {
return false, err
}
sort.Strings(children)
if len(children) > 0 {
return children[0] == lockNode, nil
}
return false, fmt.Errorf("empty queue node: %v", queueNode)
}
func CreatePidNode(zconn Conn, zkPath string) error {
hostname, err := os.Hostname()
if err != nil {
return fmt.Errorf("failed creating pid node %v: %v", zkPath, err )
}
data := fmt.Sprintf("host:%v\npid:%v\n", hostname, os.Getpid())
// On the first try, assume the cluster is up and running, that will
// help hunt down any config issues present at startup
_, err = zconn.Create(zkPath, data, zookeeper.EPHEMERAL, zookeeper.WorldACL(zookeeper.PERM_ALL))
if err != nil {
if err.(*zookeeper.Error).Code == zookeeper.ZNODEEXISTS {
err = zconn.Delete(zkPath, -1)
}
if err != nil {
return fmt.Errorf("failed deleting pid node: %v: %v", zkPath, err)
}
_, err = zconn.Create(zkPath, data, zookeeper.EPHEMERAL, zookeeper.WorldACL(zookeeper.PERM_ALL))
if err != nil {
return fmt.Errorf("failed creating pid node: %v: %v", zkPath, err)
}
}
go func() {
for {
_, _, watch, err := zconn.GetW(zkPath)
if err != nil {
log.Printf("WARNING: failed reading pid node: %v: %v", zkPath, err)
} else {
event := <-watch
log.Printf("INFO: pid node event %v: %v", zkPath, event)
if event.Ok() {
if event.Type == zookeeper.EVENT_DELETED {
// Another process took over (most likely), but no sense in starting
// a data race. Just stop watching.
log.Printf("INFO: pid watcher stopped, pid node deleted %v: %v", zkPath)
return
}
continue
}
}
_, err = zconn.Create(zkPath, data, zookeeper.EPHEMERAL, zookeeper.WorldACL(zookeeper.PERM_ALL))
if err != nil {
log.Printf("WARNING: failed recreating pid node: %v: %v", zkPath, err)
time.Sleep(30*time.Second)
} else {
log.Printf("INFO: recreated pid node: %v", zkPath)
}
}
}()
return nil
}