Upgrading memorytopo to full topo.Impl.

And using it in all tests that can use it.
Removing other fake zk connections in favor of launching a ZK server.
Refactoring zkcustomrule to use new zk2 library.
This commit is contained in:
Alain Jobart 2016-12-28 08:00:17 -08:00
Родитель 3a87e4c36c
Коммит b3cea990e4
94 изменённых файлов: 1536 добавлений и 1819 удалений

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

@ -25,7 +25,7 @@ import (
"github.com/youtube/vitess/go/vt/servenv"
"github.com/youtube/vitess/go/vt/tabletserver"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/vtctld"
"github.com/youtube/vitess/go/vt/vtgate"
@ -80,9 +80,8 @@ func main() {
flag.Set("enable_realtime_stats", "true")
flag.Set("log_dir", "$VTDATAROOT/tmp")
// Create topo server. We use a 'zk2' implementation, based on
// a memory map per cell.
ts = zk2topo.NewFakeServer(tpb.Cells...)
// Create topo server. We use a 'memorytopo' implementation.
ts = memorytopo.NewServer(tpb.Cells...)
servenv.Init()
tabletserver.Init()
@ -117,7 +116,7 @@ func main() {
// vtctld configuration and init
vtctld.InitVtctld(ts)
vtctld.HandleExplorer("zk2", vtctld.NewBackendExplorer(ts.Impl))
vtctld.HandleExplorer("memorytopo", vtctld.NewBackendExplorer(ts.Impl))
servenv.OnTerm(func() {
// FIXME(alainjobart): stop vtgate

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

@ -8,7 +8,7 @@ import (
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
querypb "github.com/youtube/vitess/go/vt/proto/query"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
@ -16,7 +16,7 @@ import (
func TestFindAllKeyspaceShards(t *testing.T) {
ctx := context.Background()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
// no keyspace / shards
ks, err := findAllKeyspaceShards(ctx, ts, "cell1")

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

@ -23,7 +23,7 @@ import (
"github.com/youtube/vitess/go/vt/tabletserver/querytypes"
"github.com/youtube/vitess/go/vt/tabletserver/tabletconn"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
binlogdatapb "github.com/youtube/vitess/go/vt/proto/binlogdata"
querypb "github.com/youtube/vitess/go/vt/proto/query"
@ -352,7 +352,7 @@ var mockedThrottlerSettings = &sqltypes.Result{
}
func TestBinlogPlayerMapHorizontalSplit(t *testing.T) {
ts := zk2topo.NewFakeServer("cell1")
ts := memorytopo.NewServer("cell1")
ctx := context.Background()
// create the keyspace, a full set of covering shards,
@ -538,7 +538,7 @@ func TestBinlogPlayerMapHorizontalSplit(t *testing.T) {
}
func TestBinlogPlayerMapHorizontalSplitStopStartUntil(t *testing.T) {
ts := zk2topo.NewFakeServer("cell1")
ts := memorytopo.NewServer("cell1")
ctx := context.Background()
// create the keyspace, a full set of covering shards,
@ -732,7 +732,7 @@ func TestBinlogPlayerMapHorizontalSplitStopStartUntil(t *testing.T) {
}
func TestBinlogPlayerMapVerticalSplit(t *testing.T) {
ts := zk2topo.NewFakeServer("cell1")
ts := memorytopo.NewServer("cell1")
ctx := context.Background()
// create the keyspaces, with one shard each

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

@ -14,6 +14,8 @@ import (
"testing"
"time"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/sqltypes"
"github.com/youtube/vitess/go/vt/binlog/binlogplayer"
"github.com/youtube/vitess/go/vt/health"
@ -21,8 +23,7 @@ import (
"github.com/youtube/vitess/go/vt/tabletserver"
"github.com/youtube/vitess/go/vt/tabletserver/tabletservermock"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
@ -115,7 +116,7 @@ func (fhc *fakeHealthCheck) HTMLName() template.HTML {
}
func createTestAgent(ctx context.Context, t *testing.T, preStart func(*ActionAgent)) (*ActionAgent, chan<- *binlogplayer.VtClientMock) {
ts := zk2topo.NewFakeServer("cell1")
ts := memorytopo.NewServer("cell1")
if err := ts.CreateKeyspace(ctx, "test_keyspace", &topodatapb.Keyspace{}); err != nil {
t.Fatalf("CreateKeyspace failed: %v", err)

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

@ -8,13 +8,14 @@ import (
"fmt"
"testing"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/history"
"github.com/youtube/vitess/go/vt/dbconfigs"
"github.com/youtube/vitess/go/vt/mysqlctl"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"golang.org/x/net/context"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
@ -25,7 +26,7 @@ import (
func TestInitTablet(t *testing.T) {
ctx := context.Background()
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
tabletAlias := &topodatapb.TabletAlias{
Cell: "cell1",
Uid: 1,

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

@ -11,49 +11,56 @@ import (
"time"
log "github.com/golang/glog"
zookeeper "github.com/samuel/go-zookeeper/zk"
"github.com/samuel/go-zookeeper/zk"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/servenv"
"github.com/youtube/vitess/go/vt/tabletserver"
"github.com/youtube/vitess/go/zk"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
)
var (
// Actual ZkCustomRule object in charge of rule updates
zkCustomRule = NewZkCustomRule(zk.NewMetaConn())
// Commandline flag to specify rule path in zookeeper
zkRulePath = flag.String("zkcustomrules", "", "zookeeper based custom rule path")
// Commandline flag to specify rule server and path.
zkRuleServer = flag.String("zkcustomrules_address", "", "zookeeper server to get rules from")
zkRulePath = flag.String("zkcustomrules_path", "", "path in the zookeeper server to get rules from")
)
// InvalidQueryRulesVersion is used to mark invalid query rules
const InvalidQueryRulesVersion int64 = -1
// invalidQueryRulesVersion is used to mark invalid query rules
const invalidQueryRulesVersion int64 = -1
// ZkCustomRuleSource is zookeeper based custom rule source name
const ZkCustomRuleSource string = "ZK_CUSTOM_RULE"
const zkCustomRuleSource string = "ZK_CUSTOM_RULE"
// ZkCustomRule is Zookeeper backed implementation of CustomRuleManager
type ZkCustomRule struct {
// Zookeeper connection. Set at construction time.
zconn zk2topo.Conn
// Path of the rules files. Set at construction time.
path string
// mu protects all the following fields.
mu sync.Mutex
path string
zconn zk.Conn
watch <-chan zookeeper.Event // Zookeeper watch for listenning data change notifications
watch <-chan zk.Event // Zookeeper watch for listenning data change notifications
currentRuleSet *tabletserver.QueryRules
currentRuleSetVersion int64 // implemented with Zookeeper transaction id
finish chan int
currentRuleSetVersion int64 // implemented with Zookeeper modification version
done chan struct{}
}
// NewZkCustomRule Creates new ZkCustomRule structure
func NewZkCustomRule(zkconn zk.Conn) *ZkCustomRule {
func NewZkCustomRule(server, path string) *ZkCustomRule {
return &ZkCustomRule{
zconn: zkconn,
zconn: zk2topo.Connect(server),
path: path,
currentRuleSet: tabletserver.NewQueryRules(),
currentRuleSetVersion: InvalidQueryRulesVersion,
finish: make(chan int, 1)}
currentRuleSetVersion: invalidQueryRulesVersion,
done: make(chan struct{}),
}
}
// Open Registers Zookeeper watch, gets inital QueryRules and starts polling routine
func (zkcr *ZkCustomRule) Open(qsc tabletserver.Controller, rulePath string) (err error) {
zkcr.path = rulePath
// Start registers Zookeeper watch, gets inital QueryRules and starts
// polling routine.
func (zkcr *ZkCustomRule) Start(qsc tabletserver.Controller) (err error) {
err = zkcr.refreshWatch()
if err != nil {
return err
@ -69,7 +76,8 @@ func (zkcr *ZkCustomRule) Open(qsc tabletserver.Controller, rulePath string) (er
// refreshWatch gets a new watch channel for ZkCustomRule, it is called when
// the old watch channel is closed on errors
func (zkcr *ZkCustomRule) refreshWatch() error {
_, _, watch, err := zkcr.zconn.GetW(zkcr.path)
ctx := context.Background()
_, _, watch, err := zkcr.zconn.GetW(ctx, zkcr.path)
if err != nil {
log.Warningf("Fail to get a valid watch from ZK service: %v", err)
return err
@ -81,7 +89,8 @@ func (zkcr *ZkCustomRule) refreshWatch() error {
// refreshData gets query rules from Zookeeper and refresh internal QueryRules cache
// this function will also call TabletServer.SetQueryRules to propagate rule changes to query service
func (zkcr *ZkCustomRule) refreshData(qsc tabletserver.Controller, nodeRemoval bool) error {
data, stat, err := zkcr.zconn.Get(zkcr.path)
ctx := context.Background()
data, stat, err := zkcr.zconn.Get(ctx, zkcr.path)
if err != nil {
log.Warningf("Error encountered when trying to get data and watch from Zk: %v", err)
return err
@ -101,35 +110,35 @@ func (zkcr *ZkCustomRule) refreshData(qsc tabletserver.Controller, nodeRemoval b
zkcr.currentRuleSetVersion = stat.Mzxid
if !reflect.DeepEqual(zkcr.currentRuleSet, qrs) {
zkcr.currentRuleSet = qrs.Copy()
qsc.SetQueryRules(ZkCustomRuleSource, qrs.Copy())
qsc.SetQueryRules(zkCustomRuleSource, qrs.Copy())
log.Infof("Custom rule version %v fetched from Zookeeper and applied to vttablet", zkcr.currentRuleSetVersion)
}
return nil
}
const sleepDuringZkFailure time.Duration = 30
const sleepDuringZkFailure time.Duration = 30 * time.Second
// poll polls the Zookeeper watch channel for data changes and refresh watch channel if watch channel is closed
// by Zookeeper Go library on error conditions such as connection reset
func (zkcr *ZkCustomRule) poll(qsc tabletserver.Controller) {
for {
select {
case <-zkcr.finish:
case <-zkcr.done:
return
case event := <-zkcr.watch:
switch event.Type {
case zookeeper.EventNodeCreated, zookeeper.EventNodeDataChanged, zookeeper.EventNodeDeleted:
err := zkcr.refreshData(qsc, event.Type == zookeeper.EventNodeDeleted) // refresh rules
case zk.EventNodeCreated, zk.EventNodeDataChanged, zk.EventNodeDeleted:
err := zkcr.refreshData(qsc, event.Type == zk.EventNodeDeleted) // refresh rules
if err != nil {
// Sleep to avoid busy waiting during connection re-establishment
<-time.After(time.Second * sleepDuringZkFailure)
<-time.After(sleepDuringZkFailure)
}
case zookeeper.EventSession:
case zk.EventSession:
err := zkcr.refreshWatch() // need to to get a new watch
if err != nil {
// Sleep to avoid busy waiting during connection re-establishment
<-time.After(time.Second * sleepDuringZkFailure)
<-time.After(sleepDuringZkFailure)
}
zkcr.refreshData(qsc, false)
}
@ -137,28 +146,32 @@ func (zkcr *ZkCustomRule) poll(qsc tabletserver.Controller) {
}
}
// Close signals an termination to polling go routine and closes Zookeeper connection object
func (zkcr *ZkCustomRule) Close() {
// Stop signals a termination to polling go routine and closes
// Zookeeper connection object.
func (zkcr *ZkCustomRule) Stop() {
close(zkcr.done)
zkcr.zconn.Close()
zkcr.finish <- 1
}
// GetRules retrives cached rules
// GetRules retrives cached rules.
func (zkcr *ZkCustomRule) GetRules() (qrs *tabletserver.QueryRules, version int64, err error) {
zkcr.mu.Lock()
defer zkcr.mu.Unlock()
return zkcr.currentRuleSet.Copy(), zkcr.currentRuleSetVersion, nil
}
// ActivateZkCustomRules activates zookeeper dynamic custom rule mechanism
func ActivateZkCustomRules(qsc tabletserver.Controller) {
if *zkRulePath != "" {
qsc.RegisterQueryRuleSource(ZkCustomRuleSource)
zkCustomRule.Open(qsc, *zkRulePath)
// activateZkCustomRules activates zookeeper dynamic custom rule mechanism.
func activateZkCustomRules(qsc tabletserver.Controller) {
if *zkRuleServer != "" && *zkRulePath != "" {
qsc.RegisterQueryRuleSource(zkCustomRuleSource)
zkCustomRule := NewZkCustomRule(*zkRuleServer, *zkRulePath)
zkCustomRule.Start(qsc)
servenv.OnTerm(zkCustomRule.Stop)
}
}
func init() {
tabletserver.RegisterFunctions = append(tabletserver.RegisterFunctions, ActivateZkCustomRules)
servenv.OnTerm(zkCustomRule.Close)
tabletserver.RegisterFunctions = append(tabletserver.RegisterFunctions, activateZkCustomRules)
}

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

@ -5,58 +5,66 @@
package zkcustomrule
import (
"context"
"reflect"
"testing"
"time"
zookeeper "github.com/samuel/go-zookeeper/zk"
"github.com/samuel/go-zookeeper/zk"
"github.com/youtube/vitess/go/vt/tabletserver"
"github.com/youtube/vitess/go/vt/tabletserver/tabletservermock"
"github.com/youtube/vitess/go/zk"
"github.com/youtube/vitess/go/zk/fakezk"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/zk/zkctl"
)
var customRule1 = `[
{
"Name": "r1",
"Description": "disallow bindvar 'asdfg'",
"BindVarConds":[{
"Name": "asdfg",
"OnAbsent": false,
"Operator": ""
}]
}
]`
var customRule1 = `
[
{
"Name": "r1",
"Description": "disallow bindvar 'asdfg'",
"BindVarConds":[{
"Name": "asdfg",
"OnAbsent": false,
"Operator": ""
}]
}
]`
var customRule2 = `[
{
"Name": "r2",
"Description": "disallow insert on table test",
"TableNames" : ["test"],
"Query" : "(insert)|(INSERT)"
}
]`
var conn zk.Conn
func setUpFakeZk(t *testing.T) {
conn = fakezk.NewConn()
conn.Create("/zk", nil, 0, zookeeper.WorldACL(zookeeper.PermAll))
conn.Create("/zk/fake", nil, 0, zookeeper.WorldACL(zookeeper.PermAll))
conn.Create("/zk/fake/customrules", nil, 0, zookeeper.WorldACL(zookeeper.PermAll))
conn.Create("/zk/fake/customrules/testrules", []byte("customrule1"), 0, zookeeper.WorldACL(zookeeper.PermAll))
conn.Set("/zk/fake/customrules/testrules", []byte(customRule1), -1)
}
var customRule2 = `
[
{
"Name": "r2",
"Description": "disallow insert on table test",
"TableNames" : ["test"],
"Query" : "(insert)|(INSERT)"
}
]`
func TestZkCustomRule(t *testing.T) {
// Start a real single ZK daemon, and close it after all tests are done.
zkd, serverAddr := zkctl.StartLocalZk(2)
defer zkd.Teardown()
// Create fake file.
serverPath := "/zk/fake/customrules/testrules"
ctx := context.Background()
conn := zk2topo.Connect(serverAddr)
defer conn.Close()
if _, err := zk2topo.CreateRecursive(ctx, conn, serverPath, []byte(customRule1), 0, zk.WorldACL(zk2topo.PermFile), 3); err != nil {
t.Fatalf("CreateRecursive failed: %v", err)
}
// Start a mock tabletserver.
tqsc := tabletservermock.NewController()
setUpFakeZk(t)
zkcr := NewZkCustomRule(conn)
err := zkcr.Open(tqsc, "/zk/fake/customrules/testrules")
// Setup the ZkCustomRule
zkcr := NewZkCustomRule(serverAddr, serverPath)
err := zkcr.Start(tqsc)
if err != nil {
t.Fatalf("Cannot open zookeeper custom rule service, err=%v", err)
t.Fatalf("Cannot start zookeeper custom rule service: %v", err)
}
defer zkcr.Stop()
var qrs *tabletserver.QueryRules
// Test if we can successfully fetch the original rule (test GetRules)
@ -70,7 +78,7 @@ func TestZkCustomRule(t *testing.T) {
}
// Test updating rules
conn.Set("/zk/fake/customrules/testrules", []byte(customRule2), -1)
conn.Set(ctx, serverPath, []byte(customRule2), -1)
<-time.After(time.Second) //Wait for the polling thread to respond
qrs, _, err = zkcr.GetRules()
if err != nil {
@ -86,7 +94,7 @@ func TestZkCustomRule(t *testing.T) {
}
// Test rule path removal
conn.Delete("/zk/fake/customrules/testrules", -1)
conn.Delete(ctx, serverPath, -1)
<-time.After(time.Second)
qrs, _, err = zkcr.GetRules()
if err != nil {
@ -97,8 +105,7 @@ func TestZkCustomRule(t *testing.T) {
}
// Test rule path revival
conn.Create("/zk/fake/customrules/testrules", []byte("customrule2"), 0, zookeeper.WorldACL(zookeeper.PermAll))
conn.Set("/zk/fake/customrules/testrules", []byte(customRule2), -1)
conn.Create(ctx, serverPath, []byte("customrule2"), 0, zk.WorldACL(zk2topo.PermFile))
<-time.After(time.Second) //Wait for the polling thread to respond
qrs, _, err = zkcr.GetRules()
if err != nil {
@ -108,6 +115,4 @@ func TestZkCustomRule(t *testing.T) {
if qr == nil {
t.Fatalf("Expect custom rule r2 to be found, but got nothing, qrs=%v", qrs)
}
zkcr.Close()
}

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

@ -19,7 +19,7 @@ import (
"github.com/youtube/vitess/go/vt/tabletserver/grpcqueryservice"
"github.com/youtube/vitess/go/vt/tabletserver/queryservice/fakes"
"github.com/youtube/vitess/go/vt/throttler"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler"
"github.com/youtube/vitess/go/vt/wrangler/testlib"
@ -94,7 +94,7 @@ type replica struct {
func newReplica(lagUpdateInterval, degrationInterval, degrationDuration time.Duration) *replica {
t := &testing.T{}
ts := zk2topo.NewFakeServer("cell1")
ts := memorytopo.NewServer("cell1")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
db := fakesqldb.Register()
fakeTablet := testlib.NewFakeTablet(t, wr, "cell1", 0,

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

@ -14,7 +14,7 @@ const (
// plug-ins to be used with Vitess.
//
// Zookeeper is a good example of an implementation, as defined in
// go/vt/zktopo.
// go/vt/topo/zk2topo.
//
// This API is very generic, and key/value store oriented.
// We use regular paths for object names, and we can list all

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

@ -3,9 +3,9 @@ package etcd2topo
import (
"path"
"github.com/coreos/etcd/clientv3"
"golang.org/x/net/context"
"github.com/coreos/etcd/clientv3"
"github.com/youtube/vitess/go/vt/topo"
)

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

@ -9,9 +9,10 @@ import (
"path"
"github.com/golang/protobuf/proto"
"github.com/youtube/vitess/go/vt/topo"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)

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

@ -14,8 +14,9 @@ import (
"github.com/coreos/etcd/clientv3"
"github.com/coreos/etcd/mvcc/mvccpb"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/vt/topo"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
)
var (

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

@ -9,9 +9,10 @@ import (
"path"
"github.com/golang/protobuf/proto"
"github.com/youtube/vitess/go/vt/topo"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)

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

@ -26,9 +26,9 @@ import (
"strings"
"sync"
"github.com/coreos/etcd/clientv3"
"golang.org/x/net/context"
"github.com/coreos/etcd/clientv3"
"github.com/youtube/vitess/go/vt/topo"
)

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

@ -14,7 +14,6 @@ import (
"testing"
"time"
log "github.com/golang/glog"
"github.com/golang/protobuf/proto"
"golang.org/x/net/context"
@ -119,7 +118,7 @@ func TestEtcdTopo(t *testing.T) {
}
data, err := proto.Marshal(ci)
if err != nil {
log.Fatalf("cannot proto.Marshal CellInfo: %v", err)
t.Fatalf("cannot proto.Marshal CellInfo: %v", err)
}
nodePath := path.Join(s.global.root, cellsPath, cell, topo.CellInfoFile)
if _, err := s.global.cli.Put(ctx, nodePath, string(data)); err != nil {

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

@ -8,12 +8,13 @@ import (
"fmt"
"path"
"github.com/golang/protobuf/proto"
"golang.org/x/net/context"
"github.com/golang/protobuf/proto"
"github.com/youtube/vitess/go/vt/topo"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
vschemapb "github.com/youtube/vitess/go/vt/proto/vschema"
"github.com/youtube/vitess/go/vt/topo"
)
// GetSrvKeyspaceNames implements topo.Server.

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

@ -9,9 +9,10 @@ import (
"sort"
"github.com/golang/protobuf/proto"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/topoproto"
"golang.org/x/net/context"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)

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

@ -25,5 +25,3 @@ func VersionFromInt(version int64) topo.Version {
}
return EtcdVersion(version)
}
var _ topo.Version = (EtcdVersion)(0) // compile-time interface check

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

@ -4,11 +4,12 @@ import (
"fmt"
"path"
"github.com/golang/protobuf/proto"
"golang.org/x/net/context"
"github.com/golang/protobuf/proto"
vschemapb "github.com/youtube/vitess/go/vt/proto/vschema"
"github.com/youtube/vitess/go/vt/topo"
vschemapb "github.com/youtube/vitess/go/vt/proto/vschema"
)
/*

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

@ -4,10 +4,10 @@ import (
"fmt"
"path"
"golang.org/x/net/context"
"github.com/coreos/etcd/clientv3"
"github.com/coreos/etcd/mvcc/mvccpb"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
)

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

@ -5,34 +5,19 @@
package helpers
import (
"os"
"testing"
log "github.com/golang/glog"
zookeeper "github.com/samuel/go-zookeeper/zk"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/testfiles"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/zktopo"
"github.com/youtube/vitess/go/zk"
"github.com/youtube/vitess/go/zk/fakezk"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
func createSetup(ctx context.Context, t *testing.T) (topo.Impl, topo.Impl) {
fromConn := fakezk.NewConn()
fromTS := zktopo.NewServer(fromConn)
toConn := fakezk.NewConn()
toTS := zktopo.NewServer(toConn)
for _, zkPath := range []string{"/zk/test_cell/vt", "/zk/global/vt"} {
if _, err := zk.CreateRecursive(fromConn, zkPath, nil, 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
t.Fatalf("cannot init fromTS: %v", err)
}
}
fromTS := memorytopo.New("test_cell")
toTS := memorytopo.New("test_cell")
// create a keyspace and a couple tablets
if err := fromTS.CreateKeyspace(ctx, "test_keyspace", &topodatapb.Keyspace{}); err != nil {
@ -84,13 +69,6 @@ func createSetup(ctx context.Context, t *testing.T) (topo.Impl, topo.Impl) {
t.Fatalf("cannot create slave tablet: %v", err)
}
os.Setenv("ZK_CLIENT_CONFIG", testfiles.Locate("topo_helpers_test_zk_client.json"))
cells, err := fromTS.GetKnownCells(ctx)
if err != nil {
t.Fatalf("fromTS.GetKnownCells: %v", err)
}
log.Infof("Cells: %v", cells)
return fromTS, toTS
}

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

@ -5,45 +5,16 @@
package helpers
import (
"fmt"
"testing"
zookeeper "github.com/samuel/go-zookeeper/zk"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/topo/test"
"github.com/youtube/vitess/go/vt/zktopo"
"github.com/youtube/vitess/go/zk"
"github.com/youtube/vitess/go/zk/fakezk"
)
type fakeServer struct {
topo.Impl
localCells []string
}
func (s fakeServer) GetKnownCells(ctx context.Context) ([]string, error) {
return s.localCells, nil
}
func newFakeTeeServer(t *testing.T) topo.Impl {
cells := []string{"test", topo.GlobalCell} // global has to be last
zconn1 := fakezk.NewConn()
zconn2 := fakezk.NewConn()
for _, cell := range cells {
if _, err := zk.CreateRecursive(zconn1, fmt.Sprintf("/zk/%v/vt", cell), nil, 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
t.Fatalf("cannot init ZooKeeper: %v", err)
}
if _, err := zk.CreateRecursive(zconn2, fmt.Sprintf("/zk/%v/vt", cell), nil, 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
t.Fatalf("cannot init ZooKeeper: %v", err)
}
}
s1 := fakeServer{Impl: zktopo.NewServer(zconn1), localCells: cells[:len(cells)-1]}
s2 := fakeServer{Impl: zktopo.NewServer(zconn2), localCells: cells[:len(cells)-1]}
s1 := memorytopo.New("test")
s2 := memorytopo.New("test")
return NewTee(s1, s2, false)
}

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

@ -0,0 +1,34 @@
package memorytopo
import (
"fmt"
"sort"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
)
// ListDir is part of the topo.Backend interface.
func (mt *MemoryTopo) ListDir(ctx context.Context, cell, dirPath string) ([]string, error) {
mt.mu.Lock()
defer mt.mu.Unlock()
// Get the node to list.
n := mt.nodeByPath(cell, dirPath)
if n == nil {
return nil, topo.ErrNoNode
}
// Check it's a directory.
if !n.isDirectory() {
return nil, fmt.Errorf("node %v in cell %v is not a directory", dirPath, cell)
}
var result []string
for n := range n.children {
result = append(result, n)
}
sort.Strings(result)
return result, nil
}

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

@ -0,0 +1,111 @@
// Copyright 2016, 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 memorytopo
import (
"path"
log "github.com/golang/glog"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
)
// NewMasterParticipation is part of the topo.Server interface
func (mt *MemoryTopo) NewMasterParticipation(name, id string) (topo.MasterParticipation, error) {
mt.mu.Lock()
defer mt.mu.Unlock()
// Make sure the global path exists.
electionPath := path.Join(electionsPath, name)
if n := mt.getOrCreatePath(topo.GlobalCell, electionPath); n == nil {
return nil, topo.ErrNoNode
}
return &mtMasterParticipation{
mt: mt,
name: name,
id: id,
stop: make(chan struct{}),
done: make(chan struct{}),
}, nil
}
// mtMasterParticipation implements topo.MasterParticipation.
//
// We use a directory (in global election path, with the name) with
// ephemeral files in it, that contains the id. The oldest revision
// wins the election.
type mtMasterParticipation struct {
// s is our parent etcd topo Server
mt *MemoryTopo
// name is the name of this MasterParticipation
name string
// id is the process's current id.
id string
// stop is a channel closed when Stop is called.
stop chan struct{}
// done is a channel closed when we're done processing the Stop
done chan struct{}
}
// WaitForMastership is part of the topo.MasterParticipation interface.
func (mp *mtMasterParticipation) WaitForMastership() (context.Context, error) {
electionPath := path.Join(electionsPath, mp.name)
lockPath := ""
// We use a cancelable context here. If stop is closed,
// we just cancel that context.
lockCtx, lockCancel := context.WithCancel(context.Background())
go func() {
select {
case <-mp.stop:
if lockPath != "" {
if err := mp.mt.unlock(context.Background(), electionPath, lockPath); err != nil {
log.Errorf("failed to delete lockPath %v for election %v: %v", lockPath, mp.name, err)
}
}
lockCancel()
close(mp.done)
}
}()
// Try to get the mastership, by getting a lock.
var err error
lockPath, err = mp.mt.lock(lockCtx, electionPath, mp.id)
if err != nil {
// It can be that we were interrupted.
return nil, err
}
// We got the lock. Return the lockContext. If Stop() is called,
// it will cancel the lockCtx, and cancel the returned context.
return lockCtx, nil
}
// Stop is part of the topo.MasterParticipation interface
func (mp *mtMasterParticipation) Stop() {
close(mp.stop)
<-mp.done
}
// GetCurrentMasterID is part of the topo.MasterParticipation interface
func (mp *mtMasterParticipation) GetCurrentMasterID() (string, error) {
electionPath := path.Join(electionsPath, mp.name)
mp.mt.mu.Lock()
defer mp.mt.mu.Unlock()
n := mp.mt.nodeByPath(topo.GlobalCell, electionPath)
if n == nil {
return "", nil
}
return n.lockContents, nil
}

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

@ -0,0 +1,156 @@
package memorytopo
import (
"fmt"
"path"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
)
// Create is part of topo.Backend interface.
func (mt *MemoryTopo) Create(ctx context.Context, cell, filePath string, contents []byte) (topo.Version, error) {
if contents == nil {
contents = []byte{}
}
mt.mu.Lock()
defer mt.mu.Unlock()
// Get the parent dir.
dir, file := path.Split(filePath)
p := mt.getOrCreatePath(cell, dir)
if p == nil {
return nil, fmt.Errorf("trying to create file %v in cell %v in a path that contains files", filePath, cell)
}
// Check the file doesn't already exist.
if _, ok := p.children[file]; ok {
return nil, topo.ErrNodeExists
}
// Create the file.
n := mt.newFile(file, contents, p)
p.children[file] = n
return NodeVersion(n.version), nil
}
// Update is part of topo.Backend interface.
func (mt *MemoryTopo) Update(ctx context.Context, cell, filePath string, contents []byte, version topo.Version) (topo.Version, error) {
if contents == nil {
contents = []byte{}
}
mt.mu.Lock()
defer mt.mu.Unlock()
// Get the parent dir, we'll need it in case of creation.
dir, file := path.Split(filePath)
p := mt.nodeByPath(cell, dir)
if p == nil {
// Parent doesn't exist, let's create it if we need to.
if version != nil {
return nil, topo.ErrNoNode
}
p = mt.getOrCreatePath(cell, dir)
if p == nil {
return nil, fmt.Errorf("trying to create file %v in cell %v in a path that contains files", filePath, cell)
}
}
// Get the existing file.
n, ok := p.children[file]
if !ok {
// File doesn't exist, see if we need to create it.
if version != nil {
return nil, topo.ErrNoNode
}
n = mt.newFile(file, contents, p)
p.children[file] = n
return NodeVersion(n.version), nil
}
// Check if it's a directory.
if n.isDirectory() {
return nil, fmt.Errorf("Update(%v,%v) failed: it's a directory", cell, filePath)
}
// Check the version.
if version != nil && n.version != uint64(version.(NodeVersion)) {
return nil, topo.ErrBadVersion
}
// Now we can update.
n.version = mt.getNextVersion()
n.contents = contents
// Call the watches
for _, w := range n.watches {
w <- &topo.WatchData{
Contents: n.contents,
Version: NodeVersion(n.version),
}
}
return NodeVersion(n.version), nil
}
// Get is part of topo.Backend interface.
func (mt *MemoryTopo) Get(ctx context.Context, cell, filePath string) ([]byte, topo.Version, error) {
mt.mu.Lock()
defer mt.mu.Unlock()
// Get the node.
n := mt.nodeByPath(cell, filePath)
if n == nil {
return nil, nil, topo.ErrNoNode
}
if n.contents == nil {
// it's a directory
return nil, nil, fmt.Errorf("cannot Get() directory %v in cell %v", filePath, cell)
}
return n.contents, NodeVersion(n.version), nil
}
// Delete is part of topo.Backend interface.
func (mt *MemoryTopo) Delete(ctx context.Context, cell, filePath string, version topo.Version) error {
mt.mu.Lock()
defer mt.mu.Unlock()
// Get the parent dir.
dir, file := path.Split(filePath)
p := mt.nodeByPath(cell, dir)
if p == nil {
return topo.ErrNoNode
}
// Get the existing file.
n, ok := p.children[file]
if !ok {
return topo.ErrNoNode
}
// Check if it's a directory.
if n.isDirectory() {
return fmt.Errorf("Delete(%v,%v) failed: it's a directory", cell, filePath)
}
// Check the version.
if version != nil && n.version != uint64(version.(NodeVersion)) {
return topo.ErrBadVersion
}
// Now we can delete.
mt.recursiveDelete(n)
// Call the watches
for _, w := range n.watches {
w <- &topo.WatchData{
Err: topo.ErrNoNode,
}
close(w)
}
return nil
}

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

@ -0,0 +1,75 @@
package memorytopo
import (
"fmt"
"path"
"github.com/golang/protobuf/proto"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
// CreateKeyspace implements topo.Impl.CreateKeyspace
func (mt *MemoryTopo) CreateKeyspace(ctx context.Context, keyspace string, value *topodatapb.Keyspace) error {
data, err := proto.Marshal(value)
if err != nil {
return err
}
keyspacePath := path.Join(keyspacesPath, keyspace, topo.KeyspaceFile)
_, err = mt.Create(ctx, topo.GlobalCell, keyspacePath, data)
return err
}
// UpdateKeyspace implements topo.Impl.UpdateKeyspace
func (mt *MemoryTopo) UpdateKeyspace(ctx context.Context, keyspace string, value *topodatapb.Keyspace, existingVersion int64) (int64, error) {
data, err := proto.Marshal(value)
if err != nil {
return -1, err
}
keyspacePath := path.Join(keyspacesPath, keyspace, topo.KeyspaceFile)
version, err := mt.Update(ctx, topo.GlobalCell, keyspacePath, data, VersionFromInt(existingVersion))
if err != nil {
return -1, err
}
return int64(version.(NodeVersion)), nil
}
// GetKeyspace implements topo.Impl.GetKeyspace
func (mt *MemoryTopo) GetKeyspace(ctx context.Context, keyspace string) (*topodatapb.Keyspace, int64, error) {
keyspacePath := path.Join(keyspacesPath, keyspace, topo.KeyspaceFile)
data, version, err := mt.Get(ctx, topo.GlobalCell, keyspacePath)
if err != nil {
return nil, 0, err
}
k := &topodatapb.Keyspace{}
if err = proto.Unmarshal(data, k); err != nil {
return nil, 0, fmt.Errorf("bad keyspace data %v", err)
}
return k, int64(version.(NodeVersion)), nil
}
// GetKeyspaces implements topo.Impl.GetKeyspaces
func (mt *MemoryTopo) GetKeyspaces(ctx context.Context) ([]string, error) {
children, err := mt.ListDir(ctx, topo.GlobalCell, keyspacesPath)
switch err {
case nil:
return children, nil
case topo.ErrNoNode:
return nil, nil
default:
return nil, err
}
}
// DeleteKeyspace implements topo.Impl.DeleteKeyspace
func (mt *MemoryTopo) DeleteKeyspace(ctx context.Context, keyspace string) error {
keyspacePath := path.Join(keyspacesPath, keyspace, topo.KeyspaceFile)
return mt.Delete(ctx, topo.GlobalCell, keyspacePath, nil)
}

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

@ -0,0 +1,100 @@
package memorytopo
import (
"fmt"
"path"
log "github.com/golang/glog"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
)
// convertError converts a context error into a topo error.
func convertError(err error) error {
switch err {
case context.Canceled:
return topo.ErrInterrupted
case context.DeadlineExceeded:
return topo.ErrTimeout
}
return err
}
func (mt *MemoryTopo) lock(ctx context.Context, nodePath string, contents string) (string, error) {
for {
mt.mu.Lock()
n := mt.nodeByPath(topo.GlobalCell, nodePath)
if n == nil {
mt.mu.Unlock()
return "", topo.ErrNoNode
}
if l := n.lock; l != nil {
// Someone else has the lock. Just wait for it.
mt.mu.Unlock()
select {
case <-l:
// Node was unlocked, try again to grab it.
continue
case <-ctx.Done():
// Done waiting
return "", convertError(ctx.Err())
}
}
// Noone has the lock, grab it.
n.lock = make(chan struct{})
n.lockContents = contents
mt.mu.Unlock()
return nodePath, nil
}
}
func (mt *MemoryTopo) unlock(ctx context.Context, nodePath, actionPath string) error {
if nodePath != actionPath {
return fmt.Errorf("invalid actionPath %v was expecting %v", actionPath, nodePath)
}
mt.mu.Lock()
defer mt.mu.Unlock()
n := mt.nodeByPath(topo.GlobalCell, nodePath)
if n == nil {
return topo.ErrNoNode
}
if n.lock == nil {
return fmt.Errorf("node %v is not locked", nodePath)
}
close(n.lock)
n.lock = nil
n.lockContents = ""
return nil
}
// LockKeyspaceForAction implements topo.Server.
func (mt *MemoryTopo) LockKeyspaceForAction(ctx context.Context, keyspace, contents string) (string, error) {
keyspacePath := path.Join(keyspacesPath, keyspace)
return mt.lock(ctx, keyspacePath, contents)
}
// UnlockKeyspaceForAction implements topo.Server.
func (mt *MemoryTopo) UnlockKeyspaceForAction(ctx context.Context, keyspace, actionPath, results string) error {
keyspacePath := path.Join(keyspacesPath, keyspace)
log.Infof("results of %v: %v", actionPath, results)
return mt.unlock(ctx, keyspacePath, actionPath)
}
// LockShardForAction implements topo.Server.
func (mt *MemoryTopo) LockShardForAction(ctx context.Context, keyspace, shard, contents string) (string, error) {
shardPath := path.Join(keyspacesPath, keyspace, shardsPath, shard)
return mt.lock(ctx, shardPath, contents)
}
// UnlockShardForAction implements topo.Server.
func (mt *MemoryTopo) UnlockShardForAction(ctx context.Context, keyspace, shard, actionPath, results string) error {
shardPath := path.Join(keyspacesPath, keyspace, shardsPath, shard)
log.Infof("results of %v: %v", actionPath, results)
return mt.unlock(ctx, shardPath, actionPath)
}

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

@ -1,41 +1,45 @@
// Package memorytopo contains an implementation of the topo.Backend
// API based on an in-process memory map.
//
// At the moment, it is actually a topo.Impl implementation as well,
// based on faketopo. As we convert more and more code to the new
// file-based topo.Backend APIs, this will grow. Eventually, the topo.Impl
// interface will be retired.
// It also contains the plumbing to make it a topo.Impl as well.
// Eventually we will ove the difference to go/vt/topo.
package memorytopo
import (
"fmt"
"path"
"sort"
"strings"
"sync"
log "github.com/golang/glog"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/test/faketopo"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
const (
// Path components
keyspacesPath = "keyspaces"
shardsPath = "shards"
tabletsPath = "tablets"
electionsPath = "elections"
)
var (
nextWatchIndex = 0
)
// MemoryTopo is a memory-based implementation of topo.Backend.
// It takes a file-system like approach, with directories at each level
// MemoryTopo is a memory-based implementation of topo.Backend. It
// takes a file-system like approach, with directories at each level
// being an actual directory node. This is meant to be closer to
// file-system like servers, like ZooKeeper or Chubby. The fake etcd
// version is closer to a node-based fake.
// file-system like servers, like ZooKeeper or Chubby. etcd or Consul
// implementations would be closer to a node-based implementation.
type MemoryTopo struct {
faketopo.FakeTopo
// mu protects the following fields.
mu sync.Mutex
// cells is the toplevel node that has one child per cell.
cells *node
// cells is the toplevel map that has one entry per cell.
cells map[string]*node
// generation is used to generate unique incrementing version
// numbers. We want a global counter so when creating a file,
// then deleting it, then re-creating it, we don't restart the
@ -50,32 +54,57 @@ type node struct {
version uint64
contents []byte
children map[string]*node
// parent is a pointer to the parent node.
// It is set to nil in toplevel and cell node.
parent *node
// watches is a map of all watches for this node.
watches map[int]chan *topo.WatchData
// lock is nil when the node is not locked.
// otherwise it has a channel that is closed by unlock.
lock chan struct{}
// lockContents is the contents of the locks.
// For regular locks, it has the contents that was passed in.
// For master election, it has the id of the election leader.
lockContents string
}
func (n *node) isDirectory() bool {
return n.children != nil
}
// NodeVersion is the local topo.Version implementation
type NodeVersion uint64
// New returns a new MemoryTopo for all the cells. It will create one
// cell for each parameter passed in. It will log.Fatal out in case
// of a problem.
func New(cells ...string) *MemoryTopo {
mt := &MemoryTopo{
cells: make(map[string]*node),
}
mt.cells[topo.GlobalCell] = mt.newDirectory(topo.GlobalCell, nil)
func (v NodeVersion) String() string {
return fmt.Sprintf("%v", uint64(v))
ctx := context.Background()
ts := topo.Server{Impl: mt}
for _, cell := range cells {
if err := ts.CreateCellInfo(ctx, cell, &topodatapb.CellInfo{
Root: "/",
}); err != nil {
log.Fatalf("ts.CreateCellInfo(%v) failed: %v", cell, err)
}
mt.cells[cell] = mt.newDirectory(cell, nil)
}
return mt
}
// NewMemoryTopo returns a new MemoryTopo for all the cells.
func NewMemoryTopo(cells []string) *MemoryTopo {
result := &MemoryTopo{}
result.cells = result.newDirectory("", nil)
for _, cell := range cells {
result.cells.children[cell] = result.newDirectory(cell, nil)
}
return result
// NewServer returns a topo.Server based on a MemoryTopo.
func NewServer(cells ...string) topo.Server {
return topo.Server{Impl: New(cells...)}
}
// Close is part of the topo.Impl interface.
func (mt *MemoryTopo) Close() {
}
func (mt *MemoryTopo) getNextVersion() uint64 {
@ -103,9 +132,12 @@ func (mt *MemoryTopo) newDirectory(name string, parent *node) *node {
}
func (mt *MemoryTopo) nodeByPath(cell, filePath string) *node {
n, ok := mt.cells[cell]
if !ok {
return nil
}
parts := strings.Split(filePath, "/")
parts[0] = cell
n := mt.cells
for _, part := range parts {
if part == "" {
// Skip empty parts, usually happens at the end.
@ -126,9 +158,12 @@ func (mt *MemoryTopo) nodeByPath(cell, filePath string) *node {
}
func (mt *MemoryTopo) getOrCreatePath(cell, filePath string) *node {
n, ok := mt.cells[cell]
if !ok {
return nil
}
parts := strings.Split(filePath, "/")
parts[0] = cell
n := mt.cells
for _, part := range parts {
if part == "" {
// Skip empty parts, usually happens at the end.
@ -161,224 +196,17 @@ func (mt *MemoryTopo) recursiveDelete(n *node) {
}
}
// ListDir is part of the topo.Backend interface.
func (mt *MemoryTopo) ListDir(ctx context.Context, cell, dirPath string) ([]string, error) {
mt.mu.Lock()
defer mt.mu.Unlock()
// Get the node to list.
n := mt.nodeByPath(cell, dirPath)
if n == nil {
return nil, topo.ErrNoNode
}
// Check it's a directory.
if !n.isDirectory() {
return nil, fmt.Errorf("node %v in cell %v is not a directory", dirPath, cell)
}
var result []string
for n := range n.children {
result = append(result, n)
}
sort.Strings(result)
return result, nil
}
// Create is part of topo.Backend interface.
func (mt *MemoryTopo) Create(ctx context.Context, cell, filePath string, contents []byte) (topo.Version, error) {
if contents == nil {
contents = []byte{}
}
mt.mu.Lock()
defer mt.mu.Unlock()
// Get the parent dir.
dir, file := path.Split(filePath)
p := mt.getOrCreatePath(cell, dir)
if p == nil {
return nil, fmt.Errorf("trying to create file %v in cell %v in a path that contains files", filePath, cell)
}
// Check the file doesn't already exist.
if _, ok := p.children[file]; ok {
return nil, topo.ErrNodeExists
}
// Create the file.
n := mt.newFile(file, contents, p)
p.children[file] = n
return NodeVersion(n.version), nil
}
// Update is part of topo.Backend interface.
func (mt *MemoryTopo) Update(ctx context.Context, cell, filePath string, contents []byte, version topo.Version) (topo.Version, error) {
if contents == nil {
contents = []byte{}
}
mt.mu.Lock()
defer mt.mu.Unlock()
// Get the parent dir, we'll need it in case of creation.
dir, file := path.Split(filePath)
p := mt.nodeByPath(cell, dir)
if p == nil {
return nil, topo.ErrNoNode
}
// Get the existing file.
n, ok := p.children[file]
if !ok {
// File doesn't exist, see if we need to create it.
if version != nil {
return nil, topo.ErrNoNode
}
n = mt.newFile(file, contents, p)
p.children[file] = n
return NodeVersion(n.version), nil
}
// Check if it's a directory.
if n.isDirectory() {
return nil, fmt.Errorf("Update(%v,%v) failed: it's a directory", cell, filePath)
}
// Check the version.
if version != nil && n.version != uint64(version.(NodeVersion)) {
return nil, topo.ErrBadVersion
}
// Now we can update.
n.version = mt.getNextVersion()
n.contents = contents
// Call the watches
for _, w := range n.watches {
w <- &topo.WatchData{
Contents: n.contents,
Version: NodeVersion(n.version),
}
}
return NodeVersion(n.version), nil
}
// Get is part of topo.Backend interface.
func (mt *MemoryTopo) Get(ctx context.Context, cell, filePath string) ([]byte, topo.Version, error) {
mt.mu.Lock()
defer mt.mu.Unlock()
// Get the node.
n := mt.nodeByPath(cell, filePath)
if n == nil {
return nil, nil, topo.ErrNoNode
}
if n.contents == nil {
// it's a directory
return nil, nil, fmt.Errorf("cannot Get() directory %v in cell %v", filePath, cell)
}
return n.contents, NodeVersion(n.version), nil
}
// Delete is part of topo.Backend interface.
func (mt *MemoryTopo) Delete(ctx context.Context, cell, filePath string, version topo.Version) error {
mt.mu.Lock()
defer mt.mu.Unlock()
// Get the parent dir.
dir, file := path.Split(filePath)
p := mt.nodeByPath(cell, dir)
if p == nil {
return topo.ErrNoNode
}
// Get the existing file.
n, ok := p.children[file]
if !ok {
return topo.ErrNoNode
}
// Check if it's a directory.
if n.isDirectory() {
return fmt.Errorf("Delete(%v,%v) failed: it's a directory", cell, filePath)
}
// Check the version.
if version != nil && n.version != uint64(version.(NodeVersion)) {
return topo.ErrBadVersion
}
// Now we can delete.
mt.recursiveDelete(n)
// Call the watches
for _, w := range n.watches {
w <- &topo.WatchData{
Err: topo.ErrNoNode,
}
close(w)
}
return nil
}
// Watch is part of the topo.Backend interface.
func (mt *MemoryTopo) Watch(ctx context.Context, cell string, filePath string) (*topo.WatchData, <-chan *topo.WatchData, topo.CancelFunc) {
mt.mu.Lock()
defer mt.mu.Unlock()
n := mt.nodeByPath(cell, filePath)
if n == nil {
return &topo.WatchData{Err: topo.ErrNoNode}, nil, nil
}
if n.contents == nil {
// it's a directory
return &topo.WatchData{Err: fmt.Errorf("cannot watch directory %v in cell %v", filePath, cell)}, nil, nil
}
current := &topo.WatchData{
Contents: n.contents,
Version: NodeVersion(n.version),
}
notifications := make(chan *topo.WatchData, 100)
watchIndex := nextWatchIndex
nextWatchIndex++
n.watches[watchIndex] = notifications
cancel := func() {
// This function can be called at any point, so we first need
// to make sure the watch is still valid.
mt.mu.Lock()
defer mt.mu.Unlock()
n := mt.nodeByPath(cell, filePath)
if n == nil {
return
}
if w, ok := n.watches[watchIndex]; ok {
delete(n.watches, watchIndex)
w <- &topo.WatchData{Err: topo.ErrInterrupted}
close(w)
}
}
return current, notifications, cancel
}
// GetKnownCells is part of the topo.Server interface.
func (mt *MemoryTopo) GetKnownCells(ctx context.Context) ([]string, error) {
mt.mu.Lock()
defer mt.mu.Unlock()
var result []string
for c := range mt.cells.children {
for c := range mt.cells {
if c != topo.GlobalCell {
result = append(result, c)
}
}
sort.Strings(result)
return result, nil
}
var _ topo.Impl = (*MemoryTopo)(nil) // compile-time interface check

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

@ -0,0 +1,91 @@
package memorytopo
import (
"fmt"
"path"
"github.com/golang/protobuf/proto"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
// UpdateShardReplicationFields implements topo.Impl.UpdateShardReplicationFields
func (mt *MemoryTopo) UpdateShardReplicationFields(ctx context.Context, cell, keyspace, shard string, update func(*topodatapb.ShardReplication) error) error {
p := path.Join(keyspacesPath, keyspace, shardsPath, shard, topo.ShardReplicationFile)
for {
data, version, err := mt.Get(ctx, cell, p)
sr := &topodatapb.ShardReplication{}
switch err {
case topo.ErrNoNode:
// Empty node, version is nil
case nil:
// Use any data we got.
if err = proto.Unmarshal(data, sr); err != nil {
return fmt.Errorf("bad ShardReplication data %v", err)
}
default:
return err
}
err = update(sr)
switch err {
case topo.ErrNoUpdateNeeded:
return nil
case nil:
// keep going
default:
return err
}
// marshall and save
data, err = proto.Marshal(sr)
if err != nil {
return err
}
if version == nil {
// We have to create, and we catch ErrNodeExists.
_, err = mt.Create(ctx, cell, p, data)
if err != topo.ErrNodeExists {
return err
}
} else {
// We have to update, and we catch ErrBadVersion.
_, err = mt.Update(ctx, cell, p, data, version)
if err != topo.ErrBadVersion {
return err
}
}
}
}
// GetShardReplication implements topo.Impl.GetShardReplication
func (mt *MemoryTopo) GetShardReplication(ctx context.Context, cell, keyspace, shard string) (*topo.ShardReplicationInfo, error) {
p := path.Join(keyspacesPath, keyspace, shardsPath, shard, topo.ShardReplicationFile)
data, _, err := mt.Get(ctx, cell, p)
if err != nil {
return nil, err
}
sr := &topodatapb.ShardReplication{}
if err = proto.Unmarshal(data, sr); err != nil {
return nil, fmt.Errorf("bad ShardReplication data %v", err)
}
return topo.NewShardReplicationInfo(sr, cell, keyspace, shard), nil
}
// DeleteShardReplication implements topo.Impl.DeleteShardReplication
func (mt *MemoryTopo) DeleteShardReplication(ctx context.Context, cell, keyspace, shard string) error {
p := path.Join(keyspacesPath, keyspace, shardsPath, shard, topo.ShardReplicationFile)
return mt.Delete(ctx, cell, p, nil)
}
// DeleteKeyspaceReplication implements topo.Impl.DeleteKeyspaceReplication
func (mt *MemoryTopo) DeleteKeyspaceReplication(ctx context.Context, cell, keyspace string) error {
p := path.Join(keyspacesPath, keyspace)
return mt.Delete(ctx, cell, p, nil)
}

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

@ -0,0 +1,15 @@
package memorytopo
import (
"testing"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/test"
)
func TestMemoryTopo(t *testing.T) {
// Run the TopoServerTestSuite tests.
test.TopoServerTestSuite(t, func() topo.Impl {
return New("test")
})
}

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

@ -0,0 +1,83 @@
package memorytopo
import (
"fmt"
"path"
"github.com/golang/protobuf/proto"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
vschemapb "github.com/youtube/vitess/go/vt/proto/vschema"
)
// GetSrvKeyspaceNames implements topo.Impl.GetSrvKeyspaceNames
func (mt *MemoryTopo) GetSrvKeyspaceNames(ctx context.Context, cell string) ([]string, error) {
children, err := mt.ListDir(ctx, cell, keyspacesPath)
switch err {
case nil:
return children, nil
case topo.ErrNoNode:
return nil, nil
default:
return nil, err
}
}
// UpdateSrvKeyspace implements topo.Impl.UpdateSrvKeyspace
func (mt *MemoryTopo) UpdateSrvKeyspace(ctx context.Context, cell, keyspace string, srvKeyspace *topodatapb.SrvKeyspace) error {
nodePath := path.Join(keyspacesPath, keyspace, topo.SrvKeyspaceFile)
data, err := proto.Marshal(srvKeyspace)
if err != nil {
return err
}
_, err = mt.Update(ctx, cell, nodePath, data, nil)
return err
}
// DeleteSrvKeyspace implements topo.Impl.DeleteSrvKeyspace
func (mt *MemoryTopo) DeleteSrvKeyspace(ctx context.Context, cell, keyspace string) error {
nodePath := path.Join(keyspacesPath, keyspace, topo.SrvKeyspaceFile)
return mt.Delete(ctx, cell, nodePath, nil)
}
// GetSrvKeyspace implements topo.Impl.GetSrvKeyspace
func (mt *MemoryTopo) GetSrvKeyspace(ctx context.Context, cell, keyspace string) (*topodatapb.SrvKeyspace, error) {
nodePath := path.Join(keyspacesPath, keyspace, topo.SrvKeyspaceFile)
data, _, err := mt.Get(ctx, cell, nodePath)
if err != nil {
return nil, err
}
srvKeyspace := &topodatapb.SrvKeyspace{}
if err := proto.Unmarshal(data, srvKeyspace); err != nil {
return nil, fmt.Errorf("SrvKeyspace unmarshal failed: %v %v", data, err)
}
return srvKeyspace, nil
}
// UpdateSrvVSchema implements topo.Impl.UpdateSrvVSchema
func (mt *MemoryTopo) UpdateSrvVSchema(ctx context.Context, cell string, srvVSchema *vschemapb.SrvVSchema) error {
nodePath := topo.SrvVSchemaFile
data, err := proto.Marshal(srvVSchema)
if err != nil {
return err
}
_, err = mt.Update(ctx, cell, nodePath, data, nil)
return err
}
// GetSrvVSchema implements topo.Impl.GetSrvVSchema
func (mt *MemoryTopo) GetSrvVSchema(ctx context.Context, cell string) (*vschemapb.SrvVSchema, error) {
nodePath := topo.SrvVSchemaFile
data, _, err := mt.Get(ctx, cell, nodePath)
if err != nil {
return nil, err
}
srvVSchema := &vschemapb.SrvVSchema{}
if err := proto.Unmarshal(data, srvVSchema); err != nil {
return nil, fmt.Errorf("SrvVSchema unmarshal failed: %v %v", data, err)
}
return srvVSchema, nil
}

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

@ -0,0 +1,84 @@
package memorytopo
import (
"fmt"
"path"
"github.com/golang/protobuf/proto"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
// CreateShard implements topo.Impl.CreateShard
func (mt *MemoryTopo) CreateShard(ctx context.Context, keyspace, shard string, value *topodatapb.Shard) error {
data, err := proto.Marshal(value)
if err != nil {
return err
}
shardPath := path.Join(keyspacesPath, keyspace, shardsPath, shard, topo.ShardFile)
_, err = mt.Create(ctx, topo.GlobalCell, shardPath, data)
return err
}
// UpdateShard implements topo.Impl.UpdateShard
func (mt *MemoryTopo) UpdateShard(ctx context.Context, keyspace, shard string, value *topodatapb.Shard, existingVersion int64) (int64, error) {
data, err := proto.Marshal(value)
if err != nil {
return -1, err
}
shardPath := path.Join(keyspacesPath, keyspace, shardsPath, shard, topo.ShardFile)
version, err := mt.Update(ctx, topo.GlobalCell, shardPath, data, VersionFromInt(existingVersion))
if err != nil {
return -1, err
}
return int64(version.(NodeVersion)), nil
}
// ValidateShard implements topo.Impl.ValidateShard
func (mt *MemoryTopo) ValidateShard(ctx context.Context, keyspace, shard string) error {
return nil
}
// GetShard implements topo.Impl.GetShard
func (mt *MemoryTopo) GetShard(ctx context.Context, keyspace, shard string) (*topodatapb.Shard, int64, error) {
shardPath := path.Join(keyspacesPath, keyspace, shardsPath, shard, topo.ShardFile)
data, version, err := mt.Get(ctx, topo.GlobalCell, shardPath)
if err != nil {
return nil, 0, err
}
sh := &topodatapb.Shard{}
if err = proto.Unmarshal(data, sh); err != nil {
return nil, 0, fmt.Errorf("bad shard data: %v", err)
}
return sh, int64(version.(NodeVersion)), nil
}
// GetShardNames implements topo.Impl.GetShardNames
func (mt *MemoryTopo) GetShardNames(ctx context.Context, keyspace string) ([]string, error) {
shardsPath := path.Join(keyspacesPath, keyspace, shardsPath)
children, err := mt.ListDir(ctx, topo.GlobalCell, shardsPath)
if err == topo.ErrNoNode {
// The directory doesn't exist, let's see if the keyspace
// is here or not.
_, _, kerr := mt.GetKeyspace(ctx, keyspace)
if kerr == nil {
// Keyspace is here, means no shards.
return nil, nil
}
return nil, err
}
return children, err
}
// DeleteShard implements topo.Impl.DeleteShard
func (mt *MemoryTopo) DeleteShard(ctx context.Context, keyspace, shard string) error {
shardPath := path.Join(keyspacesPath, keyspace, shardsPath, shard, topo.ShardFile)
return mt.Delete(ctx, topo.GlobalCell, shardPath, nil)
}

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

@ -0,0 +1,96 @@
package memorytopo
import (
"path"
"sort"
"github.com/golang/protobuf/proto"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/topoproto"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
// tabletPathForAlias converts a tablet alias to the node path.
func tabletPathForAlias(alias *topodatapb.TabletAlias) string {
return path.Join(tabletsPath, topoproto.TabletAliasString(alias), topo.TabletFile)
}
// CreateTablet implements topo.Impl.CreateTablet
func (mt *MemoryTopo) CreateTablet(ctx context.Context, tablet *topodatapb.Tablet) error {
data, err := proto.Marshal(tablet)
if err != nil {
return err
}
nodePath := tabletPathForAlias(tablet.Alias)
_, err = mt.Create(ctx, tablet.Alias.Cell, nodePath, data)
return err
}
// UpdateTablet implements topo.Impl.UpdateTablet
func (mt *MemoryTopo) UpdateTablet(ctx context.Context, tablet *topodatapb.Tablet, existingVersion int64) (int64, error) {
data, err := proto.Marshal(tablet)
if err != nil {
return 0, err
}
nodePath := tabletPathForAlias(tablet.Alias)
version, err := mt.Update(ctx, tablet.Alias.Cell, nodePath, data, VersionFromInt(existingVersion))
if err != nil {
return 0, err
}
return int64(version.(NodeVersion)), nil
}
// DeleteTablet implements topo.Impl.DeleteTablet
func (mt *MemoryTopo) DeleteTablet(ctx context.Context, alias *topodatapb.TabletAlias) error {
nodePath := tabletPathForAlias(alias)
return mt.Delete(ctx, alias.Cell, nodePath, nil)
}
// GetTablet implements topo.Impl.GetTablet
func (mt *MemoryTopo) GetTablet(ctx context.Context, alias *topodatapb.TabletAlias) (*topodatapb.Tablet, int64, error) {
nodePath := tabletPathForAlias(alias)
data, version, err := mt.Get(ctx, alias.Cell, nodePath)
if err != nil {
return nil, 0, err
}
tablet := &topodatapb.Tablet{}
if err := proto.Unmarshal(data, tablet); err != nil {
return nil, 0, err
}
return tablet, int64(version.(NodeVersion)), nil
}
// GetTabletsByCell implements topo.Impl.GetTabletsByCell
func (mt *MemoryTopo) GetTabletsByCell(ctx context.Context, cell string) ([]*topodatapb.TabletAlias, error) {
// Check if the cell exists first. We need to return ErrNoNode if not.
mt.mu.Lock()
if _, ok := mt.cells[cell]; !ok {
mt.mu.Unlock()
return nil, topo.ErrNoNode
}
mt.mu.Unlock()
children, err := mt.ListDir(ctx, cell, tabletsPath)
if err != nil {
if err == topo.ErrNoNode {
return nil, nil
}
return nil, err
}
sort.Strings(children)
result := make([]*topodatapb.TabletAlias, len(children))
for i, child := range children {
result[i], err = topoproto.ParseTabletAlias(child)
if err != nil {
return nil, err
}
}
return result, nil
}

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

@ -0,0 +1,24 @@
package memorytopo
import (
"fmt"
"github.com/youtube/vitess/go/vt/topo"
)
// NodeVersion is the local topo.Version implementation
type NodeVersion uint64
func (v NodeVersion) String() string {
return fmt.Sprintf("%v", uint64(v))
}
// VersionFromInt is used by old-style functions to create a proper
// Version: if version is -1, returns nil. Otherwise returns the
// NodeVersion object.
func VersionFromInt(version int64) topo.Version {
if version == -1 {
return nil
}
return NodeVersion(version)
}

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

@ -0,0 +1,49 @@
package memorytopo
import (
"fmt"
"path"
"github.com/golang/protobuf/proto"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
vschemapb "github.com/youtube/vitess/go/vt/proto/vschema"
)
/*
This file contains the vschema management code for etcdtopo.Server
*/
// SaveVSchema saves the JSON vschema into the topo.
func (s *MemoryTopo) SaveVSchema(ctx context.Context, keyspace string, vschema *vschemapb.Keyspace) error {
p := path.Join(keyspacesPath, keyspace, topo.VSchemaFile)
data, err := proto.Marshal(vschema)
if err != nil {
return err
}
if len(data) == 0 {
// No vschema, remove it. So we can remove the keyspace.
err = s.Delete(ctx, topo.GlobalCell, p, nil)
} else {
_, err = s.Update(ctx, topo.GlobalCell, p, data, nil)
}
return err
}
// GetVSchema fetches the vschema from the topo.
func (s *MemoryTopo) GetVSchema(ctx context.Context, keyspace string) (*vschemapb.Keyspace, error) {
p := path.Join(keyspacesPath, keyspace, topo.VSchemaFile)
data, _, err := s.Get(ctx, topo.GlobalCell, p)
if err != nil {
return nil, err
}
var vs vschemapb.Keyspace
err = proto.Unmarshal(data, &vs)
if err != nil {
return nil, fmt.Errorf("bad vschema data (%v): %q", err, data)
}
return &vs, nil
}

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

@ -0,0 +1,52 @@
package memorytopo
import (
"fmt"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
)
// Watch is part of the topo.Backend interface.
func (mt *MemoryTopo) Watch(ctx context.Context, cell string, filePath string) (*topo.WatchData, <-chan *topo.WatchData, topo.CancelFunc) {
mt.mu.Lock()
defer mt.mu.Unlock()
n := mt.nodeByPath(cell, filePath)
if n == nil {
return &topo.WatchData{Err: topo.ErrNoNode}, nil, nil
}
if n.contents == nil {
// it's a directory
return &topo.WatchData{Err: fmt.Errorf("cannot watch directory %v in cell %v", filePath, cell)}, nil, nil
}
current := &topo.WatchData{
Contents: n.contents,
Version: NodeVersion(n.version),
}
notifications := make(chan *topo.WatchData, 100)
watchIndex := nextWatchIndex
nextWatchIndex++
n.watches[watchIndex] = notifications
cancel := func() {
// This function can be called at any point, so we first need
// to make sure the watch is still valid.
mt.mu.Lock()
defer mt.mu.Unlock()
n := mt.nodeByPath(cell, filePath)
if n == nil {
return
}
if w, ok := n.watches[watchIndex]; ok {
delete(n.watches, watchIndex)
w <- &topo.WatchData{Err: topo.ErrInterrupted}
close(w)
}
}
return current, notifications, cancel
}

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

@ -64,7 +64,7 @@ var (
// Impl is the interface used to talk to a persistent
// backend storage server and locking service.
//
// Zookeeper is a good example of this, and zktopo contains the
// Zookeeper is a good example of this, and go/vt/topo/zk2topo contains the
// implementation for this using zookeeper.
//
// Inside Google, we use Chubby.

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

@ -1,8 +1,3 @@
// Package test contains utilities to test topo.Impl
// implementations. If you are testing your implementation, you will
// want to call all the check methods in your test methods. For an
// example, look at the tests in
// github.com/youtube/vitess/go/vt/zktopo.
package test
import (

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

@ -1,31 +0,0 @@
package test
import (
"testing"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
)
// This file applies the relevant tests to memorytopo. It is not in
// the right place, should be situated along memorytopo. But
// memorytopo only implements the Backend parts of the topology, so
// until all the underlying topo implementations are converted to use
// only topo.Backend, this has to be here, so we can specify the
// subset of tests that work for Backend.
func TestMemoryTopo(t *testing.T) {
factory := func() topo.Impl {
return memorytopo.NewMemoryTopo([]string{topo.GlobalCell, "test"})
}
t.Log("=== checkDirectory")
ts := factory()
checkDirectory(t, ts)
ts.Close()
t.Log("=== checkFile")
ts = factory()
checkFile(t, ts)
ts.Close()
}

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

@ -1,7 +1,8 @@
// Package test contains utilities to test topo.Impl
// implementations. If you are testing your implementation, you will
// want to call TopoServerTestSuite in your test method. For an
// example, look at the tests in github.com/youtube/vitess/go/vt/zktopo.
// example, look at the tests in
// github.com/youtube/vitess/go/vt/topo/memorytopo.
package test
import (

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

@ -9,21 +9,19 @@ import (
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
// This file contains tests for the replication.go file. Note we use a
// zk2topo, because memorytopo doesn't support all topo server
// methods yet.
// This file contains tests for the replication.go file.
func TestFixShardReplication(t *testing.T) {
cell := "cell1"
keyspace := "ks1"
shard := "shard1"
ctx := context.Background()
ts := zk2topo.NewFakeServer(cell)
ts := topo.Server{Impl: memorytopo.New(cell)}
// Create a tablet.
alias := &topodatapb.TabletAlias{

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

@ -41,7 +41,7 @@ func TestWatchSrvKeyspaceNoNode(t *testing.T) {
cell := "cell1"
keyspace := "ks1"
ctx := context.Background()
mt := memorytopo.NewMemoryTopo([]string{topo.GlobalCell, cell})
mt := memorytopo.New(cell)
ts := topo.Server{Impl: mt}
// No SrvKeyspace -> ErrNoNode
@ -56,7 +56,7 @@ func TestWatchSrvKeyspace(t *testing.T) {
cell := "cell1"
keyspace := "ks1"
ctx := context.Background()
mt := memorytopo.NewMemoryTopo([]string{topo.GlobalCell, cell})
mt := memorytopo.New(cell)
ts := topo.Server{Impl: mt}
// Create initial value
@ -178,7 +178,7 @@ func TestWatchSrvKeyspaceCancel(t *testing.T) {
cell := "cell1"
keyspace := "ks1"
ctx := context.Background()
mt := memorytopo.NewMemoryTopo([]string{topo.GlobalCell, cell})
mt := memorytopo.New(cell)
ts := topo.Server{Impl: mt}
// No SrvKeyspace -> ErrNoNode

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

@ -7,14 +7,12 @@ import (
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
// This file contains tests for the tablet.go file. Note we use a
// zk2topo, because memorytopo doesn't support all topo server
// methods yet.
// This file contains tests for the tablet.go file.
// TestCreateTablet tests all the logic in the topo.CreateTablet method.
func TestCreateTablet(t *testing.T) {
@ -22,7 +20,7 @@ func TestCreateTablet(t *testing.T) {
keyspace := "ks1"
shard := "shard1"
ctx := context.Background()
ts := zk2topo.NewFakeServer(cell)
ts := topo.Server{Impl: memorytopo.New(cell)}
// Create a tablet.
alias := &topodatapb.TabletAlias{

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

@ -5,8 +5,9 @@
package zk2topo
import (
"github.com/youtube/vitess/go/vt/topo"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
)
// This file contains the cell management methods of zktopo.Server.

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

@ -1,399 +0,0 @@
// Copyright 2013, 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 zk2topo
import (
"bytes"
"fmt"
"strings"
"sync"
"time"
"github.com/samuel/go-zookeeper/zk"
"golang.org/x/net/context"
)
// fakeConn is a pretty complete mock implementation of a Zookeper
// Conn object. All operations work as expected with the exceptions of
// zk.Conn.ACL and zk.Conn.SetACL. zk.Conn.SetACL will succeed, but it
// is a noop (and the ACLs won't be respected). zk.Conn.ACL will
// panic. It is OK to access the connection from multiple goroutines,
// but the locking is very naive (every operation locks the whole
// connection).
// fakeConn implements the Conn interface, backed by a map.
type fakeConn struct {
mu sync.Mutex
root *node
zxid int64
existWatches map[string][]chan zk.Event
}
// ConnectFake returns a fake connection.
// It ignores addr.
func ConnectFake(addr string) Conn {
return newFakeConn()
}
// newFakeConn returns a fake Conn implementation. Data is stored in
// memory, and there's a global connection lock for concurrent access.
func newFakeConn() *fakeConn {
return &fakeConn{
root: &node{
name: "/",
children: make(map[string]*node),
},
existWatches: make(map[string][]chan zk.Event)}
}
func (conn *fakeConn) Get(ctx context.Context, zkPath string) (data []byte, stat *zk.Stat, err error) {
conn.mu.Lock()
defer conn.mu.Unlock()
node, _, rest, err := conn.getNode(zkPath, "get")
if err != nil {
return nil, nil, err
}
if len(rest) != 0 {
return nil, nil, zk.ErrNoNode
}
return node.content, node.statCopy(), nil
}
func (conn *fakeConn) GetW(ctx context.Context, zkPath string) (data []byte, stat *zk.Stat, watch <-chan zk.Event, err error) {
conn.mu.Lock()
defer conn.mu.Unlock()
node, _, rest, err := conn.getNode(zkPath, "getw")
if err != nil {
return nil, nil, nil, err
}
if len(rest) != 0 {
return nil, nil, nil, zk.ErrNoNode
}
c := make(chan zk.Event, 1)
node.changeWatches = append(node.changeWatches, c)
return node.content, node.statCopy(), c, nil
}
func (conn *fakeConn) Children(ctx context.Context, zkPath string) (children []string, stat *zk.Stat, err error) {
conn.mu.Lock()
defer conn.mu.Unlock()
node, _, rest, err := conn.getNode(zkPath, "children")
if err != nil {
return nil, nil, err
}
if len(rest) != 0 {
return nil, nil, zk.ErrNoNode
}
for name := range node.children {
children = append(children, name)
}
return children, node.statCopy(), nil
}
func (conn *fakeConn) ChildrenW(ctx context.Context, zkPath string) (children []string, stat *zk.Stat, watch <-chan zk.Event, err error) {
conn.mu.Lock()
defer conn.mu.Unlock()
node, _, rest, err := conn.getNode(zkPath, "childrenw")
if err != nil {
return nil, nil, nil, err
}
if len(rest) != 0 {
return nil, nil, nil, zk.ErrNoNode
}
c := make(chan zk.Event, 1)
node.childrenWatches = append(node.childrenWatches, c)
for name := range node.children {
children = append(children, name)
}
return children, node.statCopy(), c, nil
}
func (conn *fakeConn) Exists(ctx context.Context, zkPath string) (exists bool, stat *zk.Stat, err error) {
// FIXME(szopa): if the path is bad, Op will be "get."
_, stat, err = conn.Get(ctx, zkPath)
if err == zk.ErrNoNode {
return false, nil, nil
}
return true, stat, err
}
func (conn *fakeConn) ExistsW(ctx context.Context, zkPath string) (exists bool, stat *zk.Stat, watch <-chan zk.Event, err error) {
conn.mu.Lock()
defer conn.mu.Unlock()
node, _, rest, err := conn.getNode(zkPath, "existsw")
if err != nil {
return false, nil, nil, err
}
c := make(chan zk.Event, 1)
if len(rest) != 0 {
watches, ok := conn.existWatches[zkPath]
if !ok {
watches = make([]chan zk.Event, 0)
conn.existWatches[zkPath] = watches
}
conn.existWatches[zkPath] = append(watches, c)
return false, nil, c, nil
}
node.existWatches = append(node.existWatches, c)
return true, node.statCopy(), c, nil
}
func (conn *fakeConn) Create(ctx context.Context, zkPath string, value []byte, flags int32, aclv []zk.ACL) (zkPathCreated string, err error) {
conn.mu.Lock()
defer conn.mu.Unlock()
parent, _, rest, err := conn.getNode(zkPath, "create")
if err != nil {
return "", err
}
if len(rest) == 0 {
return "", zk.ErrNodeExists
}
if len(rest) > 1 {
return "", zk.ErrNoNode
}
zxid := conn.getZxid()
name := rest[0]
if (flags & zk.FlagSequence) != 0 {
sequence := parent.nextSequence()
name += sequence
zkPath = zkPath + sequence
}
parent.children[name] = &node{
acl: aclv,
children: make(map[string]*node),
existWatches: make([]chan zk.Event, 0),
name: name,
content: value,
stat: zk.Stat{
Mtime: ZkTime(time.Now()),
Ctime: ZkTime(time.Now()),
Czxid: zxid,
Mzxid: zxid,
},
}
event := zk.Event{
Type: zk.EventNodeCreated,
Path: zkPath,
State: zk.StateConnected,
}
if watches, ok := conn.existWatches[zkPath]; ok {
delete(conn.existWatches, zkPath)
for _, watch := range watches {
watch <- event
}
}
childrenEvent := zk.Event{
Type: zk.EventNodeChildrenChanged,
Path: zkPath,
State: zk.StateConnected,
}
for _, watch := range parent.childrenWatches {
watch <- childrenEvent
close(watch)
}
parent.childrenWatches = nil
parent.stat.Cversion++
return zkPath, nil
}
func (conn *fakeConn) Set(ctx context.Context, zkPath string, value []byte, version int32) (stat *zk.Stat, err error) {
conn.mu.Lock()
defer conn.mu.Unlock()
node, _, rest, err := conn.getNode(zkPath, "set")
if err != nil {
return nil, err
}
if len(rest) != 0 {
return nil, zk.ErrNoNode
}
if version != -1 && node.stat.Version != version {
return nil, zk.ErrBadVersion
}
node.content = value
node.stat.Version++
for _, watch := range node.changeWatches {
watch <- zk.Event{
Type: zk.EventNodeDataChanged,
Path: zkPath,
State: zk.StateConnected,
}
}
node.changeWatches = nil
return node.statCopy(), nil
}
func (conn *fakeConn) Delete(ctx context.Context, zkPath string, version int32) (err error) {
conn.mu.Lock()
defer conn.mu.Unlock()
node, parent, rest, err := conn.getNode(zkPath, "delete")
if err != nil {
return err
}
if len(rest) > 0 {
return zk.ErrNoNode
}
if len(node.children) > 0 {
return zk.ErrNotEmpty
}
if version != -1 && node.stat.Version != version {
return zk.ErrBadVersion
}
delete(parent.children, node.name)
event := zk.Event{
Type: zk.EventNodeDeleted,
Path: zkPath,
State: zk.StateConnected,
}
for _, watch := range node.existWatches {
watch <- event
}
for _, watch := range node.changeWatches {
watch <- event
}
node.existWatches = nil
node.changeWatches = nil
childrenEvent := zk.Event{
Type: zk.EventNodeChildrenChanged,
Path: zkPath,
State: zk.StateConnected,
}
for _, watch := range parent.childrenWatches {
watch <- childrenEvent
}
return nil
}
func (conn *fakeConn) GetACL(ctx context.Context, zkPath string) (acl []zk.ACL, stat *zk.Stat, err error) {
panic("not implemented")
}
func (conn *fakeConn) SetACL(ctx context.Context, zkPath string, aclv []zk.ACL, version int32) (err error) {
return nil
}
func (conn *fakeConn) Close() error {
conn.mu.Lock()
defer conn.mu.Unlock()
for _, watches := range conn.existWatches {
for _, c := range watches {
close(c)
}
}
conn.root.closeAllWatches()
return nil
}
func (conn *fakeConn) getNode(zkPath string, op string) (node *node, parent *node, rest []string, err error) {
// We have two edge cases that contradict each-other in the logic below:
// - zkPath=/
// - zkPath=/path/to/locks/
// To make this simpler, we hard-code '/'.
if zkPath == "/" {
return conn.root, nil, nil, nil
}
// Make sure the path starts with '/'.
parts := strings.Split(zkPath, "/")
if parts[0] != "" {
return nil, nil, nil, zk.ErrInvalidPath
}
elements := parts[1:]
parent = nil
current := conn.root
for i, el := range elements {
candidateParent := current
candidate, ok := current.children[el]
if !ok {
return current, parent, elements[i:], nil
}
current, parent = candidate, candidateParent
}
return current, parent, []string{}, nil
}
type node struct {
name string
content []byte
stat zk.Stat
acl []zk.ACL
children map[string]*node
sequence int
existWatches []chan zk.Event
changeWatches []chan zk.Event
childrenWatches []chan zk.Event
}
func (n *node) statCopy() *zk.Stat {
result := n.stat
result.NumChildren = int32(len(n.children))
return &result
}
func (n *node) closeAllWatches() {
for _, c := range n.existWatches {
close(c)
}
for _, c := range n.changeWatches {
close(c)
}
for _, c := range n.childrenWatches {
close(c)
}
for _, child := range n.children {
child.closeAllWatches()
}
}
func (n *node) nextSequence() string {
n.sequence++
return fmt.Sprintf("%010d", n.sequence)
}
func (n *node) fprintRecursive(level int, buf *bytes.Buffer) {
start := strings.Repeat(" ", level)
fmt.Fprintf(buf, "%v-%v:\n", start, n.name)
if len(n.content) > 0 {
fmt.Fprintf(buf, "%v content: %q\n\n", start, n.content)
}
if len(n.children) > 0 {
for _, child := range n.children {
child.fprintRecursive(level+1, buf)
}
}
}
func (conn *fakeConn) String() string {
conn.mu.Lock()
defer conn.mu.Unlock()
b := new(bytes.Buffer)
conn.root.fprintRecursive(0, b)
return b.String()
}
func (conn *fakeConn) getZxid() int64 {
conn.zxid++
return conn.zxid
}

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

@ -71,7 +71,7 @@ func (zs *Server) lockForAction(ctx context.Context, locksDir, contents string)
return "", errToReturn
}
log.Warningf("------ Most likely blocking locks: %v\n%v", childPath, data)
log.Warningf("------ Most likely blocking lock: %v\n%v", childPath, string(data))
return "", errToReturn
}

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

@ -6,12 +6,10 @@ package zk2topo
import (
"fmt"
"log"
"path"
"sync"
"github.com/golang/protobuf/proto"
"github.com/samuel/go-zookeeper/zk"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
@ -39,51 +37,15 @@ type instance struct {
// Server is the zookeeper topo.Impl implementation.
type Server struct {
// connect is the function used to create a Conn. Set at creation.
connect func(addr string) Conn
// mu protects the following fields.
mu sync.Mutex
// instances is a map of cell name to instance.
instances map[string]*instance
}
// NewFakeServer returns a topo.Server based on FakeConn. The global cell
// data is in the '/root' sub-directory, to make sure the root path is
// properly used. It will create one cell for each parameter passed in.
// It will log.Fatal out in case of a problem.
func NewFakeServer(cells ...string) topo.Server {
ctx := context.Background()
conn := newFakeConn()
if _, err := conn.Create(ctx, "/root", nil, 0, zk.WorldACL(PermDirectory)); err != nil {
log.Fatalf("Create(/root) failed: %v", err)
}
zs := &Server{
connect: ConnectFake,
instances: map[string]*instance{
topo.GlobalCell: {
root: "/root",
conn: conn,
},
},
}
ts := topo.Server{Impl: zs}
for _, cell := range cells {
// Each cell will have its own fake connection, with
// its own fake map, so we just use the root there.
if err := ts.CreateCellInfo(ctx, cell, &topodatapb.CellInfo{
Root: "/",
}); err != nil {
log.Fatalf("ts.CreateCellInfo(%v) failed: %v", cell, err)
}
}
return ts
}
// NewServer returns a Server connecting to real Zookeeper processes.
func NewServer(serverAddr, root string) *Server {
return &Server{
connect: Connect,
instances: map[string]*instance{
topo.GlobalCell: {
root: root,
@ -128,7 +90,7 @@ func (zs *Server) connForCell(ctx context.Context, cell string) (Conn, string, e
}
ins = &instance{
root: ci.Root,
conn: zs.connect(ci.ServerAddress),
conn: Connect(ci.ServerAddress),
}
zs.instances[cell] = ins
return ins.conn, ins.root, nil

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

@ -1,15 +1,76 @@
package zk2topo
import (
"fmt"
"path"
"testing"
"github.com/golang/protobuf/proto"
"github.com/samuel/go-zookeeper/zk"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/test"
"github.com/youtube/vitess/go/zk/zkctl"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
func TestZk2Topo(t *testing.T) {
// Start a real single ZK daemon, and close it after all tests are done.
zkd, serverAddr := zkctl.StartLocalZk(1)
defer zkd.Teardown()
// This function will create a toplevel directory for a new test.
testIndex := 0
newServer := func(cells ...string) *Server {
// Each test will use its own sub-directories.
testRoot := fmt.Sprintf("/test-%v", testIndex)
testIndex++
ctx := context.Background()
c := Connect(serverAddr)
if _, err := c.Create(ctx, testRoot, nil, 0, zk.WorldACL(PermDirectory)); err != nil {
t.Fatalf("Create(%v) failed: %v", testRoot, err)
}
globalRoot := path.Join(testRoot, "global")
if _, err := c.Create(ctx, globalRoot, nil, 0, zk.WorldACL(PermDirectory)); err != nil {
t.Fatalf("Create(%v) failed: %v", globalRoot, err)
}
cellsDir := path.Join(globalRoot, cellsPath)
if _, err := c.Create(ctx, cellsDir, nil, 0, zk.WorldACL(PermDirectory)); err != nil {
t.Fatalf("Create(%v) failed: %v", cellsDir, err)
}
for _, cell := range cells {
cellRoot := path.Join(testRoot, cell)
if _, err := c.Create(ctx, cellRoot, nil, 0, zk.WorldACL(PermDirectory)); err != nil {
t.Fatalf("Create(%v) failed: %v", cellRoot, err)
}
// Create the CellInfo for the cell.
ci := &topodatapb.CellInfo{
ServerAddress: serverAddr,
Root: cellRoot,
}
data, err := proto.Marshal(ci)
if err != nil {
t.Fatalf("cannot proto.Marshal CellInfo: %v", err)
}
cellDir := path.Join(cellsDir, cell)
if _, err := c.Create(ctx, cellDir, nil, 0, zk.WorldACL(PermDirectory)); err != nil {
t.Fatalf("Create(%v) failed: %v", cellDir, err)
}
nodePath := path.Join(cellDir, topo.CellInfoFile)
if _, err := c.Create(ctx, nodePath, data, 0, zk.WorldACL(PermFile)); err != nil {
t.Fatalf("Create(%v) failed: %v", nodePath, err)
}
}
return NewServer(serverAddr, globalRoot)
}
test.TopoServerTestSuite(t, func() topo.Impl {
ts := NewFakeServer("test")
return ts.Impl
return newServer("test")
})
}

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

@ -11,8 +11,9 @@ import (
"github.com/golang/protobuf/proto"
"golang.org/x/net/context"
vschemapb "github.com/youtube/vitess/go/vt/proto/vschema"
"github.com/youtube/vitess/go/vt/topo"
vschemapb "github.com/youtube/vitess/go/vt/proto/vschema"
)
// This file contains the vschema management code for zktopo.Server.

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

@ -7,7 +7,7 @@ import (
"github.com/golang/protobuf/proto"
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
vschemapb "github.com/youtube/vitess/go/vt/proto/vschema"
@ -20,7 +20,7 @@ func TestRebuildVSchema(t *testing.T) {
// Set up topology.
cells := []string{"cell1", "cell2"}
ts := zk2topo.NewFakeServer(cells...)
ts := memorytopo.NewServer(cells...)
// Rebuild with no keyspace / no vschema
if err := RebuildVSchema(ctx, logger, ts, cells); err != nil {

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

@ -14,7 +14,7 @@ import (
"golang.org/x/net/context"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
)
// TestCreateShard tests a few cases for topo.CreateShard
@ -22,7 +22,7 @@ func TestCreateShard(t *testing.T) {
ctx := context.Background()
// Set up topology.
ts := zk2topo.NewFakeServer("test_cell")
ts := memorytopo.NewServer("test_cell")
keyspace := "test_keyspace"
shard := "0"
@ -49,7 +49,7 @@ func TestCreateShardCustomSharding(t *testing.T) {
ctx := context.Background()
// Set up topology.
ts := zk2topo.NewFakeServer("test_cell")
ts := memorytopo.NewServer("test_cell")
// create keyspace
keyspace := "test_keyspace"
@ -91,7 +91,7 @@ func TestGetOrCreateShard(t *testing.T) {
ctx := context.Background()
// Set up topology.
ts := zk2topo.NewFakeServer("test_cell")
ts := memorytopo.NewServer("test_cell")
// and do massive parallel GetOrCreateShard
keyspace := "test_keyspace"

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

@ -447,8 +447,8 @@ func listTabletsByShard(ctx context.Context, wr *wrangler.Wrangler, keyspace, sh
return dumpTablets(ctx, wr, tabletAliases)
}
func dumpAllTablets(ctx context.Context, wr *wrangler.Wrangler, zkVtPath string) error {
tablets, err := topotools.GetAllTablets(ctx, wr.TopoServer(), zkVtPath)
func dumpAllTablets(ctx context.Context, wr *wrangler.Wrangler, cell string) error {
tablets, err := topotools.GetAllTablets(ctx, wr.TopoServer(), cell)
if err != nil {
return err
}

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

@ -18,17 +18,18 @@ import (
"testing"
"time"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/vtctl/vtctlclient"
"golang.org/x/net/context"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
// import the gRPC client implementation for tablet manager
_ "github.com/youtube/vitess/go/vt/tabletmanager/grpctmclient"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
func init() {
@ -39,7 +40,7 @@ func init() {
// CreateTopoServer returns the test topo server properly configured
func CreateTopoServer(t *testing.T) topo.Server {
return zk2topo.NewFakeServer("cell1", "cell2")
return memorytopo.NewServer("cell1")
}
// TestSuite runs the test suite on the given topo server and client

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

@ -11,7 +11,7 @@ import (
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/wrangler"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
@ -26,7 +26,7 @@ func compactJSON(in []byte) string {
func TestAPI(t *testing.T) {
ctx := context.Background()
cells := []string{"cell1", "cell2"}
ts := zk2topo.NewFakeServer(cells...)
ts := memorytopo.NewServer(cells...)
actionRepo := NewActionRepository(ts)
server := httptest.NewServer(nil)
defer server.Close()

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

@ -36,12 +36,12 @@ func NewBackendExplorer(backend topo.Impl) *BackendExplorer {
}
// HandlePath is part of the Explorer interface
func (ex *BackendExplorer) HandlePath(zkPath string, r *http.Request) *explorer.Result {
func (ex *BackendExplorer) HandlePath(nodePath string, r *http.Request) *explorer.Result {
ctx := context.Background()
result := &explorer.Result{}
// Handle toplevel display: global, then one line per cell.
if zkPath == "/" {
if nodePath == "/" {
cells, err := ex.backend.GetKnownCells(ctx)
if err != nil {
result.Error = err.Error()
@ -53,13 +53,13 @@ func (ex *BackendExplorer) HandlePath(zkPath string, r *http.Request) *explorer.
}
// Now find the cell.
parts := strings.Split(zkPath, "/")
parts := strings.Split(nodePath, "/")
if parts[0] != "" || len(parts) < 2 {
result.Error = "Invalid path: " + zkPath
result.Error = "Invalid path: " + nodePath
return result
}
cell := parts[1]
relativePath := zkPath[len(cell)+1:]
relativePath := nodePath[len(cell)+1:]
// Get the file contents, if any.
data, _, err := ex.backend.Get(ctx, cell, relativePath)

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

@ -4,15 +4,17 @@ import (
"net/http"
"testing"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
func TestHandleExplorerRedirect(t *testing.T) {
ctx := context.Background()
ts := zk2topo.NewFakeServer("cell1")
ts := memorytopo.NewServer("cell1")
if err := ts.CreateTablet(ctx, &topodatapb.Tablet{
Alias: &topodatapb.TabletAlias{
Cell: "cell1",

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

@ -14,7 +14,7 @@ import (
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/tabletserver/grpcqueryservice"
"github.com/youtube/vitess/go/vt/tabletserver/queryservice/fakes"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler"
"github.com/youtube/vitess/go/vt/wrangler/testlib"
@ -30,7 +30,7 @@ func TestRealtimeStatsWithQueryService(t *testing.T) {
keyspace := "ks"
shard := "-80"
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
if err := ts.CreateKeyspace(context.Background(), keyspace, &topodatapb.Keyspace{

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

@ -12,7 +12,7 @@ import (
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/tabletserver/grpcqueryservice"
"github.com/youtube/vitess/go/vt/tabletserver/queryservice/fakes"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler"
"github.com/youtube/vitess/go/vt/wrangler/testlib"
@ -73,7 +73,7 @@ func (s *streamHealthTabletServer) BroadcastHealth(terTimestamp int64, stats *qu
func TestTabletData(t *testing.T) {
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
if err := ts.CreateKeyspace(context.Background(), "ks", &topodatapb.Keyspace{

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

@ -17,7 +17,7 @@ import (
"github.com/youtube/vitess/go/vt/tabletserver/tabletconn"
"github.com/youtube/vitess/go/vt/tabletserver/tabletconntest"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/vtgate/gateway"
querypb "github.com/youtube/vitess/go/vt/proto/query"
@ -41,7 +41,7 @@ func CreateFakeServers(t *testing.T) (*tabletconntest.FakeQueryService, topo.Ser
}
// The topo server has a single SrvKeyspace
ts := zk2topo.NewFakeServer(cell)
ts := memorytopo.NewServer(cell)
if err := ts.UpdateSrvKeyspace(context.Background(), cell, tabletconntest.TestTarget.Keyspace, &topodatapb.SrvKeyspace{
Partitions: []*topodatapb.SrvKeyspace_KeyspacePartition{
{

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

@ -14,6 +14,8 @@ import (
"testing"
"time"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/sqltypes"
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
"github.com/youtube/vitess/go/vt/mysqlctl/tmutils"
@ -21,10 +23,9 @@ import (
"github.com/youtube/vitess/go/vt/tabletserver/grpcqueryservice"
"github.com/youtube/vitess/go/vt/tabletserver/queryservice/fakes"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler/testlib"
"golang.org/x/net/context"
querypb "github.com/youtube/vitess/go/vt/proto/query"
tabletmanagerdatapb "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
@ -63,7 +64,7 @@ type legacySplitCloneTestCase struct {
func (tc *legacySplitCloneTestCase) setUp(v3 bool) {
*useV3ReshardingMode = v3
db := fakesqldb.Register()
tc.ts = zk2topo.NewFakeServer("cell1", "cell2")
tc.ts = memorytopo.NewServer("cell1", "cell2")
ctx := context.Background()
tc.wi = NewInstance(tc.ts, "cell1", time.Second)

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

@ -18,7 +18,7 @@ import (
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/tabletserver/tabletconn"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/wrangler"
tabletmanagerdatapb "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
@ -180,7 +180,7 @@ func TestNewRestartableResultReader(t *testing.T) {
defer flag.Set("tablet_protocol", protocol)
// Create dependencies e.g. a "singleTabletProvider" instance.
ts := zk2topo.NewFakeServer("cell1")
ts := memorytopo.NewServer("cell1")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
alias := &topodatapb.TabletAlias{
Cell: "cell1",

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

@ -14,6 +14,8 @@ import (
"testing"
"time"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/sqltypes"
"github.com/youtube/vitess/go/vt/concurrency"
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
@ -22,11 +24,10 @@ import (
"github.com/youtube/vitess/go/vt/tabletserver/grpcqueryservice"
"github.com/youtube/vitess/go/vt/tabletserver/queryservice/fakes"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/topo/topoproto"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler/testlib"
"golang.org/x/net/context"
querypb "github.com/youtube/vitess/go/vt/proto/query"
tabletmanagerdatapb "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
@ -88,7 +89,7 @@ func (tc *splitCloneTestCase) setUp(v3 bool) {
func (tc *splitCloneTestCase) setUpWithConcurreny(v3 bool, concurrency, writeQueryMaxRows, rowsCount int) {
*useV3ReshardingMode = v3
db := fakesqldb.Register()
tc.ts = zk2topo.NewFakeServer("cell1", "cell2")
tc.ts = memorytopo.NewServer("cell1", "cell2")
ctx := context.Background()
tc.wi = NewInstance(tc.ts, "cell1", time.Second)

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

@ -10,16 +10,17 @@ import (
"testing"
"time"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/sqltypes"
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/mysqlctl/tmutils"
"github.com/youtube/vitess/go/vt/tabletserver/grpcqueryservice"
"github.com/youtube/vitess/go/vt/tabletserver/queryservice/fakes"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler"
"github.com/youtube/vitess/go/vt/wrangler/testlib"
"golang.org/x/net/context"
querypb "github.com/youtube/vitess/go/vt/proto/query"
tabletmanagerdatapb "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
@ -159,7 +160,7 @@ func (sq *sourceTabletServer) StreamExecute(ctx context.Context, target *querypb
func testSplitDiff(t *testing.T, v3 bool) {
*useV3ReshardingMode = v3
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
ctx := context.Background()
wi := NewInstance(ts, "cell1", time.Second)

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

@ -8,15 +8,16 @@ import (
"testing"
"time"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
"github.com/youtube/vitess/go/vt/mysqlctl/tmutils"
"github.com/youtube/vitess/go/vt/tabletserver/grpcqueryservice"
"github.com/youtube/vitess/go/vt/tabletserver/queryservice/fakes"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/topo/topoproto"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler/testlib"
"golang.org/x/net/context"
tabletmanagerdatapb "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
@ -50,7 +51,7 @@ func createVerticalSplitCloneDestinationFakeDb(t *testing.T, name string, insert
// has not changed in the meantime.
func TestVerticalSplitClone(t *testing.T) {
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
ctx := context.Background()
wi := NewInstance(ts, "cell1", time.Second)

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

@ -10,16 +10,17 @@ import (
"testing"
"time"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/sqltypes"
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/mysqlctl/tmutils"
"github.com/youtube/vitess/go/vt/tabletserver/grpcqueryservice"
"github.com/youtube/vitess/go/vt/tabletserver/queryservice/fakes"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler"
"github.com/youtube/vitess/go/vt/wrangler/testlib"
"golang.org/x/net/context"
querypb "github.com/youtube/vitess/go/vt/proto/query"
tabletmanagerdatapb "github.com/youtube/vitess/go/vt/proto/tabletmanagerdata"
@ -81,7 +82,7 @@ func (sq *verticalDiffTabletServer) StreamExecute(ctx context.Context, target *q
func TestVerticalSplitDiff(t *testing.T) {
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
ctx := context.Background()
wi := NewInstance(ts, "cell1", time.Second)

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

@ -24,9 +24,8 @@ import (
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/logutil"
vtrpcpb "github.com/youtube/vitess/go/vt/proto/vtrpc"
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/vterrors"
"github.com/youtube/vitess/go/vt/worker"
"github.com/youtube/vitess/go/vt/worker/vtworkerclient"
@ -34,6 +33,8 @@ import (
// Import the gRPC client implementation for tablet manager because the real
// vtworker implementation requires it.
_ "github.com/youtube/vitess/go/vt/tabletmanager/grpctmclient"
vtrpcpb "github.com/youtube/vitess/go/vt/proto/vtrpc"
)
func init() {
@ -44,7 +45,7 @@ func init() {
// CreateWorkerInstance returns a properly configured vtworker instance.
func CreateWorkerInstance(t *testing.T) *worker.Instance {
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
return worker.NewInstance(ts, "cell1", 1*time.Second)
}

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

@ -10,13 +10,12 @@ import (
"testing"
"time"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
)
func TestLongPolling(t *testing.T) {
ts := memorytopo.NewMemoryTopo([]string{"cell1"})
m := NewManager(topo.Server{Impl: ts})
ts := memorytopo.NewServer("cell1")
m := NewManager(ts)
// Register the manager to a web handler, start a web server.
m.HandleHTTPLongPolling("/workflow")

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

@ -8,7 +8,6 @@ import (
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
workflowpb "github.com/youtube/vitess/go/vt/proto/workflow"
@ -45,7 +44,7 @@ func startManager(t *testing.T, m *Manager) (*sync.WaitGroup, context.CancelFunc
// TestManagerSimpleRun starts and stops a job within a Manager.
func TestManagerSimpleRun(t *testing.T) {
ts := topo.Server{Impl: memorytopo.NewMemoryTopo([]string{"cell1"})}
ts := memorytopo.NewServer("cell1")
m := NewManager(ts)
// Run the manager in the background.
@ -74,7 +73,7 @@ func TestManagerSimpleRun(t *testing.T) {
// TestManagerRestart starts a job within a manager, stops the
// manager, restarts a manager, and stops the job.
func TestManagerRestart(t *testing.T) {
ts := topo.Server{Impl: memorytopo.NewMemoryTopo([]string{"cell1"})}
ts := memorytopo.NewServer("cell1")
m := NewManager(ts)
// Run the manager in the background.

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

@ -8,20 +8,18 @@ import (
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
// This file contains tests for the keyspace.go file. Note we use a
// zk2topo, because memorytopo doesn't support all topo server
// methods yet.
// This file contains tests for the keyspace.go file.
func TestKeyspace(t *testing.T) {
cell := "cell1"
keyspace := "ks1"
ctx := context.Background()
ts := zk2topo.NewFakeServer(cell)
ts := memorytopo.NewServer(cell)
// Create a Keyspace
if err := ts.CreateKeyspace(ctx, keyspace, &topodatapb.Keyspace{}); err != nil {

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

@ -8,21 +8,19 @@ import (
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
// This file contains tests for the shard.go file. Note we use a
// zk2topo, because memorytopo doesn't support all topo server
// methods yet.
// This file contains tests for the shard.go file.
func TestShard(t *testing.T) {
cell := "cell1"
keyspace := "ks1"
shard := "sh1"
ctx := context.Background()
ts := zk2topo.NewFakeServer(cell)
ts := memorytopo.NewServer(cell)
// Create a Keyspace / Shard
if err := ts.CreateKeyspace(ctx, keyspace, &topodatapb.Keyspace{}); err != nil {

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

@ -9,13 +9,12 @@ import (
"time"
"github.com/gorilla/websocket"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
)
func TestWebSocket(t *testing.T) {
ts := memorytopo.NewMemoryTopo([]string{"cell1"})
m := NewManager(topo.Server{Impl: ts})
ts := memorytopo.NewServer("cell1")
m := NewManager(ts)
// Register the manager to a web handler, start a web server.
m.HandleHTTPWebSocket("/workflow")

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

@ -14,7 +14,7 @@ import (
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/mysqlctl/tmutils"
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler"
@ -29,7 +29,7 @@ import (
func TestApplySchema_AllowLongUnavailability(t *testing.T) {
cell := "cell1"
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer(cell)
ts := memorytopo.NewServer(cell)
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
vp := NewVtctlPipe(t, ts)
defer vp.Close()

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

@ -11,6 +11,8 @@ import (
"path"
"testing"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/sqltypes"
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/mysqlctl"
@ -18,11 +20,10 @@ import (
"github.com/youtube/vitess/go/vt/mysqlctl/filebackupstorage"
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/topo/topoproto"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler"
"golang.org/x/net/context"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
@ -31,7 +32,7 @@ func TestBackupRestore(t *testing.T) {
// Initialize our environment
ctx := context.Background()
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
vp := NewVtctlPipe(t, ts)
defer vp.Close()

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

@ -13,8 +13,8 @@ import (
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/mysqlctl/tmutils"
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/topo/topoproto"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler"
@ -32,7 +32,7 @@ func TestCopySchemaShard_UseShardAsSource(t *testing.T) {
func copySchema(t *testing.T, useShardAsSource bool) {
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
vp := NewVtctlPipe(t, ts)
defer vp.Close()

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

@ -10,21 +10,22 @@ import (
"testing"
"time"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/topo/topoproto"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler"
"golang.org/x/net/context"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
func TestEmergencyReparentShard(t *testing.T) {
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
vp := NewVtctlPipe(t, ts)
defer vp.Close()
@ -147,7 +148,7 @@ func TestEmergencyReparentShard(t *testing.T) {
func TestEmergencyReparentShardMasterElectNotBest(t *testing.T) {
ctx := context.Background()
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
// Create a master, a couple good slaves

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

@ -10,16 +10,17 @@ import (
"testing"
"time"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/sqltypes"
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/topo/topoproto"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler"
"golang.org/x/net/context"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
@ -29,7 +30,7 @@ import (
func TestInitMasterShard(t *testing.T) {
ctx := context.Background()
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
vp := NewVtctlPipe(t, ts)
defer vp.Close()
@ -131,7 +132,7 @@ func TestInitMasterShard(t *testing.T) {
func TestInitMasterShardChecks(t *testing.T) {
ctx := context.Background()
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
master := NewFakeTablet(t, wr, "cell1", 0, topodatapb.TabletType_MASTER, db)
@ -169,7 +170,7 @@ func TestInitMasterShardChecks(t *testing.T) {
func TestInitMasterShardOneSlaveFails(t *testing.T) {
ctx := context.Background()
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
// Create a master, a couple slaves

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

@ -8,14 +8,15 @@ import (
"reflect"
"testing"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/sqltypes"
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler"
"golang.org/x/net/context"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
@ -23,7 +24,7 @@ import (
func TestMigrateServedFrom(t *testing.T) {
ctx := context.Background()
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
vp := NewVtctlPipe(t, ts)
defer vp.Close()

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

@ -9,15 +9,16 @@ import (
"strings"
"testing"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/sqltypes"
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler"
"golang.org/x/net/context"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
@ -50,7 +51,7 @@ func TestMigrateServedTypes(t *testing.T) {
flag.Set("wait_for_drain_sleep_replica", "0s")
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
vp := NewVtctlPipe(t, ts)
defer vp.Close()

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

@ -8,14 +8,15 @@ import (
"strings"
"testing"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/sqltypes"
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler"
"golang.org/x/net/context"
querypb "github.com/youtube/vitess/go/vt/proto/query"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
@ -25,7 +26,7 @@ func TestPermissions(t *testing.T) {
// Initialize our environment
ctx := context.Background()
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
vp := NewVtctlPipe(t, ts)
defer vp.Close()

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

@ -13,8 +13,8 @@ import (
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/tabletserver/tabletservermock"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/topo/topoproto"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler"
@ -23,7 +23,7 @@ import (
func TestPlannedReparentShard(t *testing.T) {
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
vp := NewVtctlPipe(t, ts)
defer vp.Close()
@ -149,7 +149,7 @@ func TestPlannedReparentShard(t *testing.T) {
func TestPlannedReparentNoMaster(t *testing.T) {
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
vp := NewVtctlPipe(t, ts)
defer vp.Close()

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

@ -17,8 +17,8 @@ import (
"github.com/youtube/vitess/go/vt/tabletmanager"
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/topo/topoproto"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topotools"
"github.com/youtube/vitess/go/vt/topotools/events"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
@ -32,7 +32,7 @@ func TestTabletExternallyReparented(t *testing.T) {
ctx := context.Background()
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
vp := NewVtctlPipe(t, ts)
defer vp.Close()
@ -161,7 +161,7 @@ func TestTabletExternallyReparentedWithDifferentMysqlPort(t *testing.T) {
ctx := context.Background()
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1")
ts := memorytopo.NewServer("cell1")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
// Create an old master, a new master, two good slaves, one bad slave
@ -210,7 +210,7 @@ func TestTabletExternallyReparentedContinueOnUnexpectedMaster(t *testing.T) {
ctx := context.Background()
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1")
ts := memorytopo.NewServer("cell1")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
// Create an old master, a new master, two good slaves, one bad slave
@ -253,7 +253,7 @@ func TestTabletExternallyReparentedFailedOldMaster(t *testing.T) {
ctx := context.Background()
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
// Create an old master, a new master, and a good slave.

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

@ -8,15 +8,16 @@ import (
"fmt"
"testing"
"golang.org/x/net/context"
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/mysqlctl/replication"
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/topo/topoproto"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler"
"golang.org/x/net/context"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
@ -24,7 +25,7 @@ import (
func TestShardReplicationStatuses(t *testing.T) {
ctx := context.Background()
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
// create shard and tablets
@ -91,7 +92,7 @@ func TestShardReplicationStatuses(t *testing.T) {
func TestReparentTablet(t *testing.T) {
ctx := context.Background()
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
// create shard and tablets

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

@ -9,7 +9,7 @@ import (
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler"
@ -19,7 +19,7 @@ import (
func TestDeleteShardCleanup(t *testing.T) {
ctx := context.Background()
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
vp := NewVtctlPipe(t, ts)
defer vp.Close()

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

@ -14,7 +14,7 @@ import (
"github.com/youtube/vitess/go/vt/throttler"
"github.com/youtube/vitess/go/vt/throttler/grpcthrottlerserver"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
// The test uses the gRPC throttler client and server implementations.
_ "github.com/youtube/vitess/go/vt/throttler/grpcthrottlerclient"
@ -34,7 +34,7 @@ func TestVtctlThrottlerCommands(t *testing.T) {
addr := fmt.Sprintf("localhost:%v", listener.Addr().(*net.TCPAddr).Port)
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
vp := NewVtctlPipe(t, ts)
defer vp.Close()

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

@ -13,7 +13,7 @@ import (
"github.com/youtube/vitess/go/vt/logutil"
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler"
@ -50,7 +50,7 @@ func TestVersion(t *testing.T) {
// Initialize our environment
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
vp := NewVtctlPipe(t, ts)
defer vp.Close()

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

@ -16,7 +16,7 @@ import (
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/tabletserver/grpcqueryservice"
"github.com/youtube/vitess/go/vt/tabletserver/queryservice/fakes"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler"
@ -60,7 +60,7 @@ func testWaitForDrain(t *testing.T, desc, cells string, drain drainDirective, ex
flag.Set("vtctl_healthcheck_timeout", "1s")
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
vp := NewVtctlPipe(t, ts)
defer vp.Close()

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

@ -18,7 +18,7 @@ import (
"github.com/youtube/vitess/go/vt/tabletmanager/tmclient"
"github.com/youtube/vitess/go/vt/tabletserver"
"github.com/youtube/vitess/go/vt/tabletserver/grpcqueryservice"
"github.com/youtube/vitess/go/vt/topo/zk2topo"
"github.com/youtube/vitess/go/vt/topo/memorytopo"
"github.com/youtube/vitess/go/vt/vttest/fakesqldb"
"github.com/youtube/vitess/go/vt/wrangler"
@ -79,7 +79,7 @@ func TestWaitForFilteredReplication_unhealthy(t *testing.T) {
func waitForFilteredReplication(t *testing.T, expectedErr string, initialStats *querypb.RealtimeStats, broadcastStatsFunc func() *querypb.RealtimeStats) {
db := fakesqldb.Register()
ts := zk2topo.NewFakeServer("cell1", "cell2")
ts := memorytopo.NewServer("cell1", "cell2")
wr := wrangler.New(logutil.NewConsoleLogger(), ts, tmclient.NewTabletManagerClient())
vp := NewVtctlPipe(t, ts)
defer vp.Close()

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

@ -20,6 +20,13 @@ type Server struct {
zconn zk.Conn
}
// newServer creates a Server.
func newServer() *Server {
return &Server{
zconn: zk.NewMetaConn(),
}
}
// Close is part of topo.Server interface.
func (zkts *Server) Close() {
zkts.zconn.Close()
@ -30,18 +37,6 @@ func (zkts *Server) GetZConn() zk.Conn {
return zkts.zconn
}
// NewServer can be used to create a custom Server, for tests for instance.
func NewServer(zconn zk.Conn) topo.Impl {
return &Server{zconn: zconn}
}
func init() {
topo.RegisterFactory("zookeeper", func(serverAddr, root string) (topo.Impl, error) {
zconn := zk.NewMetaConn()
return NewServer(zconn), nil
})
}
//
// These helper methods are for ZK specific things
//
@ -115,3 +110,9 @@ func (zkts *Server) PruneActionLogs(zkActionLogPath string, keepCount int) (prun
}
return prunedCount, nil
}
func init() {
topo.RegisterFactory("zookeeper", func(serverAddr, root string) (topo.Impl, error) {
return newServer(), nil
})
}

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

@ -1,7 +1,10 @@
package zktopo
import (
"encoding/json"
"fmt"
"io/ioutil"
"os"
"path"
"testing"
@ -11,103 +14,138 @@ import (
"github.com/youtube/vitess/go/vt/topo"
"github.com/youtube/vitess/go/vt/topo/test"
"github.com/youtube/vitess/go/zk"
"github.com/youtube/vitess/go/zk/fakezk"
"github.com/youtube/vitess/go/zk/zkctl"
topodatapb "github.com/youtube/vitess/go/vt/proto/topodata"
)
// TestServer is a proxy for a real implementation of topo.Server that
// provides hooks for testing.
type TestServer struct {
topo.Impl
localCells []string
}
// newTestServer returns a new TestServer (with the required paths created)
func newTestServer(t *testing.T, cells []string) topo.Impl {
zconn := fakezk.NewConn()
// create the toplevel zk paths
if _, err := zk.CreateRecursive(zconn, "/zk/global/vt", nil, 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
t.Fatalf("cannot init ZooKeeper: %v", err)
}
for _, cell := range cells {
if _, err := zk.CreateRecursive(zconn, fmt.Sprintf("/zk/%v/vt", cell), nil, 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
t.Fatalf("cannot init ZooKeeper: %v", err)
}
}
return &TestServer{Impl: NewServer(zconn), localCells: cells}
}
// GetKnownCells is part of topo.Server interface
func (s *TestServer) GetKnownCells(ctx context.Context) ([]string, error) {
return s.localCells, nil
}
// Run the topology test suite on zktopo.
func TestZkTopo(t *testing.T) {
// Start a real single ZK daemon, and close it after all tests are done.
zkd, serverAddr := zkctl.StartLocalZk(3)
defer zkd.Teardown()
// Create a ZK_CLIENT_CONFIG file to use.
fd, err := ioutil.TempFile("", "zkconf")
if err != nil {
t.Fatalf("ioutil.TempFile failed: %v", err)
}
configPath := fd.Name()
fd.Close()
defer os.Remove(configPath)
if err := os.Setenv("ZK_CLIENT_CONFIG", configPath); err != nil {
t.Fatalf("setenv failed: %v", err)
}
// This function will wipe all data before creating new directories.
createServer := func(cells ...string) *Server {
// Create the config map, all pointing to our server.
configMap := map[string]string{"global": serverAddr}
for _, cell := range cells {
configMap[cell] = serverAddr
}
fd, err := os.OpenFile(configPath, os.O_RDWR|os.O_TRUNC, 0666)
if err != nil {
t.Fatalf("OpenFile failed: %v", err)
}
err = json.NewEncoder(fd).Encode(configMap)
if err != nil {
t.Fatalf("json.Encode failed: %v", err)
}
fd.Close()
conncache := zk.NewConnCache()
defer conncache.Close()
zconn, err := conncache.ConnForPath("/zk/global/vt")
if err != nil {
t.Fatalf("ConnForPath failed: %v", err)
}
// Wipe the old directories, create new ones.
if err := zk.DeleteRecursive(zconn, "/zk", -1); err != nil && err != zookeeper.ErrNoNode {
t.Fatalf("zk.DeleteRecursive failed: %v", err)
}
if _, err := zk.CreateRecursive(zconn, "/zk/global/vt", nil, 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
t.Fatalf("CreateRecursive(/zk/global/vt) failed: %v", err)
}
for _, cell := range cells {
p := fmt.Sprintf("/zk/%v/vt", cell)
if _, err := zk.CreateRecursive(zconn, p, nil, 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
t.Fatalf("CreateRecursive(%v) failed: %v", p, err)
}
}
return newServer()
}
test.TopoServerTestSuite(t, func() topo.Impl {
return newTestServer(t, []string{"test"})
return createServer("test")
})
impl := createServer("test")
testPurgeActions(t, impl)
impl.Close()
impl = createServer("test")
testPruneActionLogs(t, impl)
impl.Close()
}
// TestPurgeActions is a ZK specific unit test
func TestPurgeActions(t *testing.T) {
// testPurgeActions is a ZK specific unit test
func testPurgeActions(t *testing.T, impl *Server) {
t.Log("=== testPurgeActions")
ctx := context.Background()
ts := newTestServer(t, []string{"test"})
defer ts.Close()
ts := topo.Server{Impl: impl}
if err := ts.CreateKeyspace(ctx, "test_keyspace", &topodatapb.Keyspace{}); err != nil {
t.Fatalf("CreateKeyspace: %v", err)
}
actionPath := path.Join(GlobalKeyspacesPath, "test_keyspace", "action")
zkts := ts.(*TestServer).Impl.(*Server)
if _, err := zk.CreateRecursive(zkts.GetZConn(), actionPath+"/topurge", []byte("purgeme"), 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
if _, err := zk.CreateRecursive(impl.GetZConn(), actionPath+"/topurge", []byte("purgeme"), 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
t.Fatalf("CreateRecursive(topurge): %v", err)
}
if _, err := zk.CreateRecursive(zkts.GetZConn(), actionPath+"/tokeep", []byte("keepme"), 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
if _, err := zk.CreateRecursive(impl.GetZConn(), actionPath+"/tokeep", []byte("keepme"), 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
t.Fatalf("CreateRecursive(tokeep): %v", err)
}
if err := zkts.PurgeActions(actionPath, func(data []byte) bool {
if err := impl.PurgeActions(actionPath, func(data []byte) bool {
return string(data) == "purgeme"
}); err != nil {
t.Fatalf("PurgeActions(tokeep): %v", err)
}
actions, _, err := zkts.GetZConn().Children(actionPath)
actions, _, err := impl.GetZConn().Children(actionPath)
if err != nil || len(actions) != 1 || actions[0] != "tokeep" {
t.Errorf("PurgeActions kept the wrong things: %v %v", err, actions)
}
}
// TestPruneActionLogs is a ZK specific unit test
func TestPruneActionLogs(t *testing.T) {
// testPruneActionLogs is a ZK specific unit test
func testPruneActionLogs(t *testing.T, impl *Server) {
t.Log("=== testPruneActionLogs")
ctx := context.Background()
ts := newTestServer(t, []string{"test"})
defer ts.Close()
ts := topo.Server{Impl: impl}
if err := ts.CreateKeyspace(ctx, "test_keyspace", &topodatapb.Keyspace{}); err != nil {
t.Fatalf("CreateKeyspace: %v", err)
}
actionLogPath := path.Join(GlobalKeyspacesPath, "test_keyspace", "actionlog")
zkts := ts.(*TestServer).Impl.(*Server)
if _, err := zk.CreateRecursive(zkts.GetZConn(), actionLogPath+"/0", []byte("first"), 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
if _, err := zk.CreateRecursive(impl.GetZConn(), actionLogPath+"/0", []byte("first"), 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
t.Fatalf("CreateRecursive(stale): %v", err)
}
if _, err := zk.CreateRecursive(zkts.GetZConn(), actionLogPath+"/1", []byte("second"), 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
if _, err := zk.CreateRecursive(impl.GetZConn(), actionLogPath+"/1", []byte("second"), 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
t.Fatalf("CreateRecursive(fresh): %v", err)
}
if count, err := zkts.PruneActionLogs(actionLogPath, 1); err != nil || count != 1 {
if count, err := impl.PruneActionLogs(actionLogPath, 1); err != nil || count != 1 {
t.Fatalf("PruneActionLogs: %v %v", err, count)
}
actionLogs, _, err := zkts.GetZConn().Children(actionLogPath)
actionLogs, _, err := impl.GetZConn().Children(actionLogPath)
if err != nil || len(actionLogs) != 1 || actionLogs[0] != "1" {
t.Errorf("PruneActionLogs kept the wrong things: %v %v", err, actionLogs)
}

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

@ -85,7 +85,7 @@ func (cc *ConnCache) ConnForPath(zkPath string) (cn Conn, err error) {
zkAddr, err := ZkPathToZkAddr(zkPath)
if err != nil {
return nil, zookeeper.ErrInvalidPath
return nil, zookeeper.ErrNoNode
}
cc.setState(zcell, connecting)

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

@ -1,424 +0,0 @@
// Copyright 2013, 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 fakezk is a pretty complete mock implementation of a
// Zookeper connection (see go/zk/zk.Conn). All operations
// work as expected with the exceptions of zk.Conn.ACL and
// zk.Conn.SetACL. zk.Conn.SetACL will succeed, but it is a noop (and
// the ACLs won't be respected). zk.Conn.ACL will panic. It is OK to
// access the connection from multiple goroutines, but the locking is
// very naive (every operation locks the whole connection).
package fakezk
import (
"bytes"
"encoding/json"
"fmt"
"io/ioutil"
"strings"
"sync"
"time"
zookeeper "github.com/samuel/go-zookeeper/zk"
"github.com/youtube/vitess/go/zk"
)
type zconn struct {
mu sync.Mutex
root *node
zxid int64
existWatches map[string][]chan zookeeper.Event
}
func (conn *zconn) getZxid() int64 {
conn.zxid++
return conn.zxid
}
// NewConn returns a fake zk.Conn implementation. Data is stored in
// memory, and there's a global connection lock for concurrent access.
func NewConn() zk.Conn {
return &zconn{
root: &node{
name: "/",
children: make(map[string]*node),
},
existWatches: make(map[string][]chan zookeeper.Event)}
}
// NewConnFromFile returns a fake zk.Conn implementation, that is seeded
// with the json data extracted from the input file.
func NewConnFromFile(filename string) zk.Conn {
result := &zconn{
root: &node{
name: "/",
children: make(map[string]*node),
},
existWatches: make(map[string][]chan zookeeper.Event)}
data, err := ioutil.ReadFile(filename)
if err != nil {
panic(fmt.Errorf("NewConnFromFile failed to read file %v: %v", filename, err))
}
values := make(map[string]interface{})
if err := json.Unmarshal(data, &values); err != nil {
panic(fmt.Errorf("NewConnFromFile failed to json.Unmarshal file %v: %v", filename, err))
}
for k, v := range values {
jv, err := json.Marshal(v)
if err != nil {
panic(fmt.Errorf("NewConnFromFile failed to json.Marshal value %v: %v", k, err))
}
// CreateRecursive will work for a leaf node where the parent
// doesn't exist, but not for a node in the middle of a tree
// that already exists. So have to use 'Set' as a backup.
if _, err := zk.CreateRecursive(result, k, jv, 0, nil); err != nil {
if err == zookeeper.ErrNodeExists {
_, err = result.Set(k, jv, -1)
}
if err != nil {
panic(fmt.Errorf("NewConnFromFile failed to zk.CreateRecursive value %v: %v", k, err))
}
}
}
return result
}
func (conn *zconn) Get(zkPath string) (data []byte, stat *zookeeper.Stat, err error) {
conn.mu.Lock()
defer conn.mu.Unlock()
node, _, rest, err := conn.getNode(zkPath, "get")
if err != nil {
return nil, nil, err
}
if len(rest) != 0 {
return nil, nil, zookeeper.ErrNoNode
}
return node.content, node.statCopy(), nil
}
func (conn *zconn) GetW(zkPath string) (data []byte, stat *zookeeper.Stat, watch <-chan zookeeper.Event, err error) {
conn.mu.Lock()
defer conn.mu.Unlock()
node, _, rest, err := conn.getNode(zkPath, "getw")
if err != nil {
return nil, nil, nil, err
}
if len(rest) != 0 {
return nil, nil, nil, zookeeper.ErrNoNode
}
c := make(chan zookeeper.Event, 1)
node.changeWatches = append(node.changeWatches, c)
return node.content, node.statCopy(), c, nil
}
func (conn *zconn) Children(zkPath string) (children []string, stat *zookeeper.Stat, err error) {
conn.mu.Lock()
defer conn.mu.Unlock()
node, _, rest, err := conn.getNode(zkPath, "children")
if err != nil {
return nil, nil, err
}
if len(rest) != 0 {
return nil, nil, zookeeper.ErrNoNode
}
for name := range node.children {
children = append(children, name)
}
return children, node.statCopy(), nil
}
func (conn *zconn) ChildrenW(zkPath string) (children []string, stat *zookeeper.Stat, watch <-chan zookeeper.Event, err error) {
conn.mu.Lock()
defer conn.mu.Unlock()
node, _, rest, err := conn.getNode(zkPath, "childrenw")
if err != nil {
return nil, nil, nil, err
}
if len(rest) != 0 {
return nil, nil, nil, zookeeper.ErrNoNode
}
c := make(chan zookeeper.Event, 1)
node.childrenWatches = append(node.childrenWatches, c)
for name := range node.children {
children = append(children, name)
}
return children, node.statCopy(), c, nil
}
func (conn *zconn) Exists(zkPath string) (stat *zookeeper.Stat, err error) {
// FIXME(szopa): if the path is bad, Op will be "get."
_, stat, err = conn.Get(zkPath)
if err == zookeeper.ErrNoNode {
return nil, nil
}
return stat, err
}
func (conn *zconn) ExistsW(zkPath string) (stat *zookeeper.Stat, watch <-chan zookeeper.Event, err error) {
conn.mu.Lock()
defer conn.mu.Unlock()
c := make(chan zookeeper.Event, 1)
node, _, rest, err := conn.getNode(zkPath, "existsw")
if err != nil {
return nil, nil, err
}
if len(rest) != 0 {
watches, ok := conn.existWatches[zkPath]
if !ok {
watches = make([]chan zookeeper.Event, 0)
conn.existWatches[zkPath] = watches
}
conn.existWatches[zkPath] = append(watches, c)
return nil, c, nil
}
node.existWatches = append(node.existWatches, c)
return node.statCopy(), c, nil
}
func (conn *zconn) Create(zkPath string, value []byte, flags int, aclv []zookeeper.ACL) (zkPathCreated string, err error) {
conn.mu.Lock()
defer conn.mu.Unlock()
parent, _, rest, err := conn.getNode(zkPath, "create")
if err != nil {
return "", err
}
if len(rest) == 0 {
return "", zookeeper.ErrNodeExists
}
if len(rest) > 1 {
return "", zookeeper.ErrNoNode
}
zxid := conn.getZxid()
name := rest[0]
if (flags & zookeeper.FlagSequence) != 0 {
sequence := parent.nextSequence()
name += sequence
zkPath = zkPath + sequence
}
parent.children[name] = &node{
acl: aclv,
children: make(map[string]*node),
existWatches: make([]chan zookeeper.Event, 0),
name: name,
content: value,
stat: zookeeper.Stat{
Mtime: zk.ZkTime(time.Now()),
Ctime: zk.ZkTime(time.Now()),
Czxid: zxid,
Mzxid: zxid,
},
}
event := zookeeper.Event{
Type: zookeeper.EventNodeCreated,
Path: zkPath,
State: zookeeper.StateConnected,
}
if watches, ok := conn.existWatches[zkPath]; ok {
delete(conn.existWatches, zkPath)
for _, watch := range watches {
watch <- event
}
}
childrenEvent := zookeeper.Event{
Type: zookeeper.EventNodeChildrenChanged,
Path: zkPath,
State: zookeeper.StateConnected,
}
for _, watch := range parent.childrenWatches {
watch <- childrenEvent
close(watch)
}
parent.childrenWatches = nil
parent.stat.Cversion++
return zkPath, nil
}
func (conn *zconn) Set(zkPath string, value []byte, version int32) (stat *zookeeper.Stat, err error) {
conn.mu.Lock()
defer conn.mu.Unlock()
node, _, rest, err := conn.getNode(zkPath, "set")
if err != nil {
return nil, err
}
if len(rest) != 0 {
return nil, zookeeper.ErrNoNode
}
if version != -1 && node.stat.Version != version {
return nil, zookeeper.ErrBadVersion
}
node.content = value
node.stat.Version++
for _, watch := range node.changeWatches {
watch <- zookeeper.Event{
Type: zookeeper.EventNodeDataChanged,
Path: zkPath,
State: zookeeper.StateConnected,
}
}
node.changeWatches = nil
return node.statCopy(), nil
}
func (conn *zconn) Delete(zkPath string, version int32) (err error) {
conn.mu.Lock()
defer conn.mu.Unlock()
node, parent, rest, err := conn.getNode(zkPath, "delete")
if err != nil {
return err
}
if len(rest) > 0 {
return zookeeper.ErrNoNode
}
if len(node.children) > 0 {
return zookeeper.ErrNotEmpty
}
if version != -1 && node.stat.Version != version {
return zookeeper.ErrBadVersion
}
delete(parent.children, node.name)
event := zookeeper.Event{
Type: zookeeper.EventNodeDeleted,
Path: zkPath,
State: zookeeper.StateConnected,
}
for _, watch := range node.existWatches {
watch <- event
}
for _, watch := range node.changeWatches {
watch <- event
}
node.existWatches = nil
node.changeWatches = nil
childrenEvent := zookeeper.Event{
Type: zookeeper.EventNodeChildrenChanged,
Path: zkPath,
State: zookeeper.StateConnected,
}
for _, watch := range parent.childrenWatches {
watch <- childrenEvent
}
return nil
}
func (conn *zconn) Close() error {
conn.mu.Lock()
defer conn.mu.Unlock()
for _, watches := range conn.existWatches {
for _, c := range watches {
close(c)
}
}
conn.root.closeAllWatches()
return nil
}
func (conn *zconn) ACL(zkPath string) (acl []zookeeper.ACL, stat *zookeeper.Stat, err error) {
panic("not implemented")
}
func (conn *zconn) SetACL(zkPath string, aclv []zookeeper.ACL, version int32) (err error) {
return nil
}
func (conn *zconn) getNode(zkPath string, op string) (node *node, parent *node, rest []string, err error) {
// FIXME(szopa): Make sure the path starts with /.
parts := strings.Split(zkPath, "/")
if parts[0] != "" {
return nil, nil, nil, zookeeper.ErrInvalidPath
}
elements := parts[1:]
parent = nil
current := conn.root
for i, el := range elements {
candidateParent := current
candidate, ok := current.children[el]
if !ok {
return current, parent, elements[i:], nil
}
current, parent = candidate, candidateParent
}
return current, parent, []string{}, nil
}
type node struct {
name string
content []byte
stat zookeeper.Stat
acl []zookeeper.ACL
children map[string]*node
sequence int
existWatches []chan zookeeper.Event
changeWatches []chan zookeeper.Event
childrenWatches []chan zookeeper.Event
}
func (n *node) statCopy() *zookeeper.Stat {
result := n.stat
result.NumChildren = int32(len(n.children))
return &result
}
func (n *node) closeAllWatches() {
for _, c := range n.existWatches {
close(c)
}
for _, c := range n.changeWatches {
close(c)
}
for _, c := range n.childrenWatches {
close(c)
}
for _, child := range n.children {
child.closeAllWatches()
}
}
func (n *node) nextSequence() string {
n.sequence++
return fmt.Sprintf("%010d", n.sequence)
}
func (n *node) fprintRecursive(level int, buf *bytes.Buffer) {
start := strings.Repeat(" ", level)
fmt.Fprintf(buf, "%v-%v:\n", start, n.name)
if len(n.content) > 0 {
fmt.Fprintf(buf, "%v content: %q\n\n", start, n.content)
}
if len(n.children) > 0 {
for _, child := range n.children {
child.fprintRecursive(level+1, buf)
}
}
}
func (conn *zconn) String() string {
conn.mu.Lock()
defer conn.mu.Unlock()
b := new(bytes.Buffer)
conn.root.fprintRecursive(0, b)
return b.String()
}

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

@ -1,267 +0,0 @@
// Copyright 2013, 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 fakezk
import (
"sort"
"strings"
"testing"
"time"
zookeeper "github.com/samuel/go-zookeeper/zk"
"github.com/youtube/vitess/go/testfiles"
"github.com/youtube/vitess/go/zk"
)
func TestBasic(t *testing.T) {
conn := NewConn()
defer conn.Close()
// Make sure Conn implements the interface.
var _ zk.Conn = conn
if _, err := conn.Create("/zk", nil, 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
t.Fatalf("conn.Create: %v", err)
}
if _, err := conn.Create("/zk/foo", []byte("foo"), 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
t.Fatalf("conn.Create: %v", err)
}
data, _, err := conn.Get("/zk/foo")
if err != nil {
t.Fatalf("conn.Get: %v", err)
}
if string(data) != "foo" {
t.Errorf("got %q, wanted %q", data, "foo")
}
if _, err := conn.Set("/zk/foo", []byte("bar"), -1); err != nil {
t.Fatalf("conn.Set: %v", err)
}
data, _, err = conn.Get("/zk/foo")
if err != nil {
t.Fatalf("conn.Get: %v", err)
}
if string(data) != "bar" {
t.Errorf("got %q, wanted %q", data, "bar")
}
// Try Set with the wrong version.
if _, err := conn.Set("/zk/foo", []byte("bar"), 0); err == nil {
t.Error("conn.Set with a wrong version: expected error")
}
// Try Get with a node that doesn't exist.
if _, _, err := conn.Get("/zk/rabarbar"); err == nil {
t.Error("conn.Get with a node that doesn't exist: expected error")
}
// Try Set with a node that doesn't exist.
if _, err := conn.Set("/zk/barbarbar", []byte("bar"), -1); err == nil {
t.Error("conn.Get with a node that doesn't exist: expected error")
}
// Try Create with a node that exists.
if _, err := conn.Create("/zk/foo", []byte("foo"), 0, zookeeper.WorldACL(zookeeper.PermAll)); err == nil {
t.Errorf("conn.Create with a node that exists: expected error")
}
// Try Create with a node whose parents don't exist.
if _, err := conn.Create("/a/b/c", []byte("foo"), 0, zookeeper.WorldACL(zookeeper.PermAll)); err == nil {
t.Errorf("conn.Create with a node whose parents don't exist: expected error")
}
if err := conn.Delete("/zk/foo", -1); err != nil {
t.Errorf("conn.Delete: %v", err)
}
stat, err := conn.Exists("/zk/foo")
if err != nil {
t.Errorf("conn.Exists: %v", err)
}
if stat != nil {
t.Errorf("/zk/foo should be deleted, got: %v", stat)
}
}
func TestChildren(t *testing.T) {
conn := NewConn()
defer conn.Close()
nodes := []string{"/zk", "/zk/foo", "/zk/bar"}
wantChildren := []string{"bar", "foo"}
for _, path := range nodes {
if _, err := conn.Create(path, nil, 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
t.Fatalf("conn.Create: %v", err)
}
}
children, _, err := conn.Children("/zk")
if err != nil {
t.Fatalf(`conn.Children("/zk"): %v`, err)
}
sort.Strings(children)
if length := len(children); length != 2 {
t.Errorf("children: got %v, wanted %v", children, wantChildren)
}
for i, path := range children {
if wantChildren[i] != path {
t.Errorf("children: got %v, wanted %v", children, wantChildren)
break
}
}
}
func TestWatches(t *testing.T) {
conn := NewConn()
defer conn.Close()
// Creating sends an event to ExistsW.
stat, watch, err := conn.ExistsW("/zk")
if err != nil {
t.Errorf("conn.ExistsW: %v", err)
}
if stat != nil {
t.Errorf("stat is not nil: %v", stat)
}
if _, err := conn.Create("/zk", nil, 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
t.Fatalf("conn.Create: %v", err)
}
fireWatch(t, watch)
// Creating a child sends an event to ChildrenW.
_, _, watch, err = conn.ChildrenW("/zk")
if err != nil {
t.Errorf(`conn.ChildrenW("/zk"): %v`, err)
}
if _, err := conn.Create("/zk/foo", nil, 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
t.Fatalf("conn.Create: %v", err)
}
fireWatch(t, watch)
// Updating sends an event to GetW.
_, _, watch, err = conn.GetW("/zk")
if err != nil {
t.Errorf(`conn.GetW("/zk"): %v`, err)
}
if _, err := conn.Set("/zk", []byte("foo"), -1); err != nil {
t.Errorf("conn.Set /zk: %v", err)
}
fireWatch(t, watch)
// Deleting sends an event to ExistsW and to ChildrenW of the
// parent.
_, watch, err = conn.ExistsW("/zk/foo")
if err != nil {
t.Errorf("conn.ExistsW: %v", err)
}
_, _, parentWatch, err := conn.ChildrenW("/zk")
if err != nil {
t.Errorf(`conn.ChildrenW("/zk"): %v`, err)
}
if err := conn.Delete("/zk/foo", -1); err != nil {
t.Errorf("conn.Delete: %v", err)
}
fireWatch(t, watch)
fireWatch(t, parentWatch)
}
func fireWatch(t *testing.T, watch <-chan zookeeper.Event) zookeeper.Event {
timer := time.NewTimer(50 * time.Millisecond)
select {
case event := <-watch:
// TODO(szopa): Figure out what's the exact type of
// event.
return event
case <-timer.C:
t.Errorf("watch didn't get event")
}
panic("unreachable")
}
func TestSequence(t *testing.T) {
conn := NewConn()
defer conn.Close()
if _, err := conn.Create("/zk", nil, 0, zookeeper.WorldACL(zookeeper.PermAll)); err != nil {
t.Fatalf("conn.Create: %v", err)
}
newPath, err := conn.Create("/zk/", nil, zookeeper.FlagSequence, zookeeper.WorldACL(zookeeper.PermAll))
if err != nil {
t.Errorf("conn.Create: %v", err)
}
if wanted := "/zk/0000000001"; newPath != wanted {
t.Errorf("new path: got %q, wanted %q", newPath, wanted)
}
newPath, err = conn.Create("/zk/", nil, zookeeper.FlagSequence, zookeeper.WorldACL(zookeeper.PermAll))
if err != nil {
t.Errorf("conn.Create: %v", err)
}
if wanted := "/zk/0000000002"; newPath != wanted {
t.Errorf("new path: got %q, wanted %q", newPath, wanted)
}
if err := conn.Delete("/zk/0000000002", -1); err != nil {
t.Fatalf("conn.Delete: %v", err)
}
newPath, err = conn.Create("/zk/", nil, zookeeper.FlagSequence, zookeeper.WorldACL(zookeeper.PermAll))
if err != nil {
t.Errorf("conn.Create: %v", err)
}
if wanted := "/zk/0000000003"; newPath != wanted {
t.Errorf("new path: got %q, wanted %q", newPath, wanted)
}
newPath, err = conn.Create("/zk/action_", nil, zookeeper.FlagSequence, zookeeper.WorldACL(zookeeper.PermAll))
if err != nil {
t.Errorf("conn.Create: %v", err)
}
if wanted := "/zk/action_0000000004"; newPath != wanted {
t.Errorf("new path: got %q, wanted %q", newPath, wanted)
}
}
func TestFromFile(t *testing.T) {
conn := NewConnFromFile(testfiles.Locate("fakezk_test_config.json"))
keyspaces, _, err := conn.Children("/zk/testing/vt/ns")
if err != nil {
t.Errorf("conn.Children: %v", err)
}
if len(keyspaces) != 1 || keyspaces[0] != "test_keyspace" {
t.Errorf("conn.Children returned bad value: %v", keyspaces)
}
data, _, err := conn.Get("/zk/testing/vt/ns/test_keyspace")
if err != nil {
t.Errorf("conn.Get(/zk/testing/vt/ns/test_keyspace): %v", err)
}
if !strings.Contains(string(data), "ShardReferences") {
t.Errorf("conn.Get(/zk/testing/vt/ns/test_keyspace) returned bad value: %v", data)
}
data, _, err = conn.Get("/zk/testing/vt/ns/test_keyspace/0/master")
if err != nil {
t.Errorf("conn.Get(/zk/testing/vt/ns/test_keyspace/0/master): %v", err)
}
if !strings.Contains(string(data), "NamedPortMap") {
t.Errorf("conn.Get(/zk/testing/vt/ns/test_keyspace/0/master) returned bad value: %v", data)
}
}

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

@ -0,0 +1,41 @@
package zkctl
import (
"fmt"
"os"
"strconv"
log "github.com/golang/glog"
"github.com/youtube/vitess/go/netutil"
)
// StartLocalZk is a helper method to create a local ZK process. Used
// in tests, mostly. It will log.Fatal out if there is an error. Each
// call should use different serverID, so tests don't interfere with
// eachother.
func StartLocalZk(id int) (*Zkd, string) {
// Get the starting port.
env := os.Getenv("VTPORTSTART")
if env == "" {
env = "6700"
}
portStart, err := strconv.Atoi(env)
if err != nil {
log.Fatalf("cannot parse VTPORTSTART: %v", err)
}
port := portStart + (id-1)*3
// Build the config parameters.
hostname := netutil.FullyQualifiedHostnameOrPanic()
zkCfg := fmt.Sprintf("%v@%v:%v:%v:%v", id, hostname, port, port+1, port+2)
zkConfig := MakeZkConfigFromString(zkCfg, uint32(id))
zkd := NewZkd(zkConfig)
// Init & start zk.
if err := zkd.Init(); err != nil {
log.Fatalf("zkd.Init(%d, %d) failed: %v", id, port, err)
}
return zkd, fmt.Sprintf("%v:%v", hostname, port+2)
}

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

@ -5,24 +5,10 @@
package zkctl
import (
"fmt"
"os"
"path"
"testing"
)
func getUUID(t *testing.T) string {
f, err := os.Open("/dev/urandom")
if err != nil {
t.Fatalf("os.Open(/dev/urandom): %v", err)
}
b := make([]byte, 16)
f.Read(b)
f.Close()
return fmt.Sprintf("%x-%x-%x-%x-%x", b[0:4], b[4:6], b[6:8], b[8:10], b[10:])
}
// these test depend on starting and stopping ZK instances,
// This test depend on starting and stopping a ZK instance,
// but may leave files/processes behind if they don't succeed,
// so some manual cleanup may be required.
@ -31,33 +17,10 @@ func TestLifeCycle(t *testing.T) {
t.Skip("skipping integration test in short mode.")
}
testLifeCycle(t, "255@voltron:2888:3888:2181", 255)
}
config := "255@voltron:2888:3888:2181"
myID := 255
func TestLifeCycleGlobal(t *testing.T) {
if testing.Short() {
t.Skip("skipping integration test in short mode.")
}
testLifeCycle(t, "1255@voltron:2890:3890:2183", 1255)
}
func testLifeCycle(t *testing.T, config string, myID uint32) {
currentVtDataRoot := os.Getenv("VTDATAROOT")
vtDataRoot := path.Join(os.TempDir(), fmt.Sprintf("VTDATAROOT_%v", getUUID(t)))
if err := os.Setenv("VTDATAROOT", vtDataRoot); err != nil {
t.Fatalf("cannot set VTDATAROOT: %v", err)
}
defer os.Setenv("VTDATAROOT", currentVtDataRoot)
if err := os.Mkdir(vtDataRoot, 0755); err != nil {
t.Fatalf("cannot create VTDATAROOT directory")
}
defer func() {
if err := os.RemoveAll(vtDataRoot); err != nil {
t.Errorf("cannot remove test VTDATAROOT directory: %v", err)
}
}()
zkConf := MakeZkConfigFromString(config, myID)
zkConf := MakeZkConfigFromString(config, uint32(myID))
zkd := NewZkd(zkConf)
if err := zkd.Init(); err != nil {
t.Fatalf("Init() err: %v", err)