From ba3d1c226ccc58e5ff6ec1d296f295d4a1a37863 Mon Sep 17 00:00:00 2001 From: Arindam Nayak Date: Thu, 24 Oct 2019 13:14:16 +0530 Subject: [PATCH 01/15] Added cluster setup using go processes Signed-off-by: Arindam Nayak --- dev.env | 11 + go/test/endtoend/cluster/cluster_process.go | 290 ++++++++++++++++++ go/test/endtoend/cluster/etcd_process.go | 158 ++++++++++ go/test/endtoend/cluster/mysqlctl_process.go | 85 +++++ go/test/endtoend/cluster/vtctl_process.go | 79 +++++ .../endtoend/cluster/vtctlclient_process.go | 100 ++++++ go/test/endtoend/cluster/vtctld_process.go | 175 +++++++++++ go/test/endtoend/cluster/vtgate_process.go | 187 +++++++++++ go/test/endtoend/cluster/vttablet_process.go | 201 ++++++++++++ .../endtoend/clustertest/add_keyspace_test.go | 85 +++++ go/test/endtoend/clustertest/etcd_test.go | 29 ++ go/test/endtoend/clustertest/main_test.go | 114 +++++++ go/test/endtoend/clustertest/vtcltd_test.go | 28 ++ go/test/endtoend/clustertest/vtgate_test.go | 52 ++++ go/test/endtoend/clustertest/vttablet_test.go | 41 +++ go/test/endtoend/vtgate/aggr_test.go | 57 ++++ go/test/endtoend/vtgate/lookup_test.go | 270 ++++++++++++++++ go/test/endtoend/vtgate/main_test.go | 203 ++++++++++++ go/test/endtoend/vtgate/sequence/seq_test.go | 174 +++++++++++ 19 files changed, 2339 insertions(+) create mode 100644 go/test/endtoend/cluster/cluster_process.go create mode 100644 go/test/endtoend/cluster/etcd_process.go create mode 100644 go/test/endtoend/cluster/mysqlctl_process.go create mode 100644 go/test/endtoend/cluster/vtctl_process.go create mode 100644 go/test/endtoend/cluster/vtctlclient_process.go create mode 100644 go/test/endtoend/cluster/vtctld_process.go create mode 100644 go/test/endtoend/cluster/vtgate_process.go create mode 100644 go/test/endtoend/cluster/vttablet_process.go create mode 100644 go/test/endtoend/clustertest/add_keyspace_test.go create mode 100644 go/test/endtoend/clustertest/etcd_test.go create mode 100644 go/test/endtoend/clustertest/main_test.go create mode 100644 go/test/endtoend/clustertest/vtcltd_test.go create mode 100644 go/test/endtoend/clustertest/vtgate_test.go create mode 100644 go/test/endtoend/clustertest/vttablet_test.go create mode 100644 go/test/endtoend/vtgate/aggr_test.go create mode 100644 go/test/endtoend/vtgate/lookup_test.go create mode 100644 go/test/endtoend/vtgate/main_test.go create mode 100644 go/test/endtoend/vtgate/sequence/seq_test.go diff --git a/dev.env b/dev.env index 4619c9f9efa..b7d65b9533b 100644 --- a/dev.env +++ b/dev.env @@ -73,6 +73,17 @@ PATH=$(prepend_path "$PATH" "$VTROOT/dist/chromedriver") PATH=$(prepend_path "$PATH" "$VTROOT/dist/node/bin") export PATH +# Etcd path. +case $(uname) in + Linux) etcd_platform=linux;; + Darwin) etcd_platform=darwin;; +esac + +ETCD_VERSION=$(cat "${VTROOT}/dist/etcd/.installed_version") +ETCD_BINDIR="${VTROOT}/dist/etcd/etcd-${ETCD_VERSION}-${etcd_platform}-amd64/" +PATH=$(prepend_path "$PATH" "$ETCD_BINDIR") +export PATH + # GOROOT sanity go_bin=$(which go) go_env=$(go env | grep GOROOT | cut -f 2 -d\") diff --git a/go/test/endtoend/cluster/cluster_process.go b/go/test/endtoend/cluster/cluster_process.go new file mode 100644 index 00000000000..54892d3be0b --- /dev/null +++ b/go/test/endtoend/cluster/cluster_process.go @@ -0,0 +1,290 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package cluster + +import ( + "fmt" + "math/rand" + + "vitess.io/vitess/go/vt/log" +) + +// DefaultCell : If no cell name is passed, then use following +const DefaultCell = "zone1" + +// LocalProcessCluster Testcases need to use this to iniate a cluster +type LocalProcessCluster struct { + Keyspaces []Keyspace + Cell string + BaseTabletUID int + Hostname string + TopoPort int + + VtgateMySQLPort int + VtctldHTTPPort int + + // standalone executable + VtctlclientProcess VtctlClientProcess + VtctlProcess VtctlProcess + + // background executable processes + topoProcess EtcdProcess + vtctldProcess VtctldProcess + VtgateProcess VtgateProcess + + nextPortForProcess int +} + +// Keyspace : Cluster accepts keyspace to launch it +type Keyspace struct { + Name string + SchemaSQL string + VSchema string + Shards []Shard +} + +// Shard with associated vttablets +type Shard struct { + Name string + Vttablets []Vttablet +} + +// Vttablet stores the properties needed to start a vttablet process +type Vttablet struct { + Type string + TabletUID int + HTTPPort int + GrpcPort int + MySQLPort int + + // background executable processes + mysqlctlProcess MysqlctlProcess + vttabletProcess VttabletProcess +} + +// StartTopo starts topology server +func (cluster *LocalProcessCluster) StartTopo() (err error) { + if cluster.Cell == "" { + cluster.Cell = DefaultCell + } + cluster.TopoPort = cluster.GetAndReservePort() + cluster.topoProcess = *EtcdProcessInstance(cluster.TopoPort, cluster.Hostname) + log.Info(fmt.Sprintf("Starting etcd server on port : %d", cluster.TopoPort)) + if err = cluster.topoProcess.Setup(); err != nil { + log.Error(err.Error()) + return + } + + log.Info("Creating topo dirs") + if err = cluster.topoProcess.ManageTopoDir("mkdir", "/vitess/global"); err != nil { + log.Error(err.Error()) + return + } + + if err = cluster.topoProcess.ManageTopoDir("mkdir", "/vitess/"+cluster.Cell); err != nil { + log.Error(err.Error()) + return + } + + log.Info("Adding cell info") + cluster.VtctlProcess = *VtctlProcessInstance(cluster.topoProcess.Port, cluster.Hostname) + if err = cluster.VtctlProcess.AddCellInfo(cluster.Cell); err != nil { + log.Error(err) + return + } + + cluster.vtctldProcess = *VtctldProcessInstance(cluster.GetAndReservePort(), cluster.GetAndReservePort(), cluster.topoProcess.Port, cluster.Hostname) + log.Info(fmt.Sprintf("Starting vtctld server on port : %d", cluster.vtctldProcess.Port)) + cluster.VtctldHTTPPort = cluster.vtctldProcess.Port + if err = cluster.vtctldProcess.Setup(cluster.Cell); err != nil { + log.Error(err.Error()) + return + } + + cluster.VtctlclientProcess = *VtctlClientProcessInstance("localhost", cluster.vtctldProcess.GrpcPort) + return +} + +// StartUnshardedKeyspace starts unshared keyspace with shard name as "0" +func (cluster *LocalProcessCluster) StartUnshardedKeyspace(keyspace Keyspace, replicaCount int, rdonly bool) error { + return cluster.StartKeyspace(keyspace, []string{"0"}, replicaCount, rdonly) +} + +// StartKeyspace starts required number of shard and the corresponding tablets +// keyspace : struct containing keyspace name, Sqlschema to apply, VSchema to apply +// shardName : list of shard names +// replicaCount: total number of replicas excluding master and rdonly +// rdonly: whether readonly tablets needed +func (cluster *LocalProcessCluster) StartKeyspace(keyspace Keyspace, shardNames []string, replicaCount int, rdonly bool) (err error) { + totalTabletsRequired := replicaCount + 1 // + 1 is for master + if rdonly { + totalTabletsRequired = totalTabletsRequired + 1 // + 1 for rdonly + } + shards := make([]Shard, 0) + log.Info("Starting keyspace : " + keyspace.Name) + _ = cluster.VtctlProcess.CreateKeyspace(keyspace.Name) + for _, shardName := range shardNames { + shard := &Shard{ + Name: shardName, + } + log.Info("Starting shard : " + shardName) + for i := 0; i < totalTabletsRequired; i++ { + // instantiate vttable object with reserved ports + tablet := &Vttablet{ + TabletUID: cluster.GetAndReserveTabletUID(), + HTTPPort: cluster.GetAndReservePort(), + GrpcPort: cluster.GetAndReservePort(), + MySQLPort: cluster.GetAndReservePort(), + } + if i == 0 { // Make the first one as master + tablet.Type = "master" + } else if i == totalTabletsRequired-1 && rdonly { // Make the last one as rdonly if rdonly flag is passed + tablet.Type = "rdonly" + } + // Start Mysqlctl process + log.Info(fmt.Sprintf("Starting mysqlctl for table uid %d, mysql port %d", tablet.TabletUID, tablet.MySQLPort)) + tablet.mysqlctlProcess = *MysqlCtlProcessInstance(tablet.TabletUID, tablet.MySQLPort) + if err = tablet.mysqlctlProcess.Start(); err != nil { + log.Error(err.Error()) + return + } + + // start vttablet process + tablet.vttabletProcess = *VttabletProcessInstance(tablet.HTTPPort, + tablet.GrpcPort, + tablet.TabletUID, + cluster.Cell, + shardName, + cluster.Hostname, + keyspace.Name, + cluster.vtctldProcess.Port, + tablet.Type, + cluster.topoProcess.Port, + cluster.Hostname) + log.Info(fmt.Sprintf("Starting vttablet for tablet uid %d, grpc port %d", tablet.TabletUID, tablet.GrpcPort)) + + if err = tablet.vttabletProcess.Setup(); err != nil { + log.Error(err.Error()) + return + } + + shard.Vttablets = append(shard.Vttablets, *tablet) + } + + // Make first tablet as master + if err = cluster.VtctlclientProcess.InitShardMaster(keyspace.Name, shardName, cluster.Cell, shard.Vttablets[0].TabletUID); err != nil { + log.Error(err.Error()) + return + } + + shards = append(shards, *shard) + } + keyspace.Shards = shards + cluster.Keyspaces = append(cluster.Keyspaces, keyspace) + + // Apply Schema SQL + if err = cluster.VtctlclientProcess.ApplySchema(keyspace.Name, keyspace.SchemaSQL); err != nil { + log.Error(err.Error()) + return + } + + //Apply VSchema + if err = cluster.VtctlclientProcess.ApplyVSchema(keyspace.Name, keyspace.VSchema); err != nil { + log.Error(err.Error()) + return + } + + log.Info("Done creating keyspace : " + keyspace.Name) + return +} + +// StartVtgate starts vtgate +func (cluster *LocalProcessCluster) StartVtgate() (err error) { + vtgateHTTPPort := cluster.GetAndReservePort() + vtgateGrpcPort := cluster.GetAndReservePort() + cluster.VtgateMySQLPort = cluster.GetAndReservePort() + log.Info(fmt.Sprintf("Starting vtgate on port %d", vtgateHTTPPort)) + cluster.VtgateProcess = *VtgateProcessInstance( + vtgateHTTPPort, + vtgateGrpcPort, + cluster.VtgateMySQLPort, + cluster.Cell, + cluster.Cell, + cluster.Hostname, "MASTER,REPLICA", + cluster.topoProcess.Port, + cluster.Hostname) + + log.Info(fmt.Sprintf("Vtgate started, connect to mysql using : mysql -h 127.0.0.1 -P %d", cluster.VtgateMySQLPort)) + return cluster.VtgateProcess.Setup() +} + +// Teardown brings down the cluster by invoking teardown for individual processes +func (cluster *LocalProcessCluster) Teardown() (err error) { + if err = cluster.VtgateProcess.TearDown(); err != nil { + log.Error(err.Error()) + return + } + + for _, keyspace := range cluster.Keyspaces { + for _, shard := range keyspace.Shards { + for _, tablet := range shard.Vttablets { + if err = tablet.mysqlctlProcess.Stop(); err != nil { + log.Error(err.Error()) + return + } + + if err = tablet.vttabletProcess.TearDown(); err != nil { + log.Error(err.Error()) + return + } + } + } + } + + if err = cluster.vtctldProcess.TearDown(); err != nil { + log.Error(err.Error()) + return + } + + if err = cluster.topoProcess.TearDown(cluster.Cell); err != nil { + log.Error(err.Error()) + return + } + return err +} + +// GetAndReservePort gives port for required process +func (cluster *LocalProcessCluster) GetAndReservePort() int { + if cluster.nextPortForProcess == 0 { + cluster.nextPortForProcess = getRandomNumber(20000, 15000) + } + cluster.nextPortForProcess = cluster.nextPortForProcess + 1 + return cluster.nextPortForProcess +} + +// GetAndReserveTabletUID gives tablet uid +func (cluster *LocalProcessCluster) GetAndReserveTabletUID() int { + if cluster.BaseTabletUID == 0 { + cluster.BaseTabletUID = getRandomNumber(100, 0) + } + cluster.BaseTabletUID = cluster.BaseTabletUID + 1 + return cluster.BaseTabletUID +} + +func getRandomNumber(maxNumber int32, baseNumber int) int { + return int(rand.Int31n(maxNumber)) + baseNumber +} diff --git a/go/test/endtoend/cluster/etcd_process.go b/go/test/endtoend/cluster/etcd_process.go new file mode 100644 index 00000000000..b4b95c53d18 --- /dev/null +++ b/go/test/endtoend/cluster/etcd_process.go @@ -0,0 +1,158 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package cluster + +import ( + "fmt" + "net/http" + "os" + "os/exec" + "path" + "strings" + "syscall" + "time" + + "vitess.io/vitess/go/vt/log" +) + +// EtcdProcess is a generic handle for a running Etcd . +// It can be spawned manually +type EtcdProcess struct { + Name string + Binary string + DataDirectory string + ListenClientURL string + AdvertiseClientURL string + Port int + Host string + VerifyURL string + + proc *exec.Cmd + exit chan error +} + +// Setup spawns a new etcd service and initializes it with the defaults. +// The service is kept running in the background until TearDown() is called. +func (etcd *EtcdProcess) Setup() (err error) { + etcd.proc = exec.Command( + etcd.Binary, + "--data-dir", etcd.DataDirectory, + "--listen-client-urls", etcd.ListenClientURL, + "--advertise-client-urls", etcd.AdvertiseClientURL, + ) + + etcd.proc.Stderr = os.Stderr + etcd.proc.Stdout = os.Stdout + + etcd.proc.Env = append(etcd.proc.Env, os.Environ()...) + + log.Infof("%v %v", strings.Join(etcd.proc.Args, " ")) + println("Starting etcd with args " + strings.Join(etcd.proc.Args, " ")) + err = etcd.proc.Start() + if err != nil { + return + } + + etcd.exit = make(chan error) + go func() { + etcd.exit <- etcd.proc.Wait() + }() + + timeout := time.Now().Add(60 * time.Second) + for time.Now().Before(timeout) { + if etcd.IsHealthy() { + return + } + select { + case err := <-etcd.exit: + return fmt.Errorf("process '%s' exited prematurely (err: %s)", etcd.Name, err) + default: + time.Sleep(300 * time.Millisecond) + } + } + + return fmt.Errorf("process '%s' timed out after 60s (err: %s)", etcd.Name, <-etcd.exit) +} + +// TearDown shutdowns the running mysqld service +func (etcd *EtcdProcess) TearDown(Cell string) error { + if etcd.proc == nil || etcd.exit == nil { + return nil + } + + etcd.removeTopoDirectories(Cell) + + // Attempt graceful shutdown with SIGTERM first + etcd.proc.Process.Signal(syscall.SIGTERM) + os.RemoveAll(path.Join(os.Getenv("VTDATAROOT"), "etcd")) + select { + case err := <-etcd.exit: + etcd.proc = nil + return err + + case <-time.After(10 * time.Second): + etcd.proc.Process.Kill() + etcd.proc = nil + return <-etcd.exit + } + +} + +// IsHealthy function checks if etcd server is up and running +func (etcd *EtcdProcess) IsHealthy() bool { + resp, err := http.Get(etcd.VerifyURL) + if err != nil { + return false + } + if resp.StatusCode == 200 { + return true + } + return false +} + +func (etcd *EtcdProcess) removeTopoDirectories(Cell string) { + _ = etcd.ManageTopoDir("rmdir", "/vitess/global") + _ = etcd.ManageTopoDir("rmdir", "/vitess/"+Cell) +} + +// ManageTopoDir creates global and zone in etcd2 +func (etcd *EtcdProcess) ManageTopoDir(command string, directory string) error { + tmpProcess := exec.Command( + "etcdctl", + "--endpoints", etcd.ListenClientURL, + command, directory, + ) + return tmpProcess.Run() +} + +// EtcdProcessInstance returns a EtcdProcess handle for a etcd sevice, +// configured with the given Config. +// The process must be manually started by calling setup() +func EtcdProcessInstance(port int, hostname string) *EtcdProcess { + etcd := &EtcdProcess{ + Name: "etcd", + Binary: "etcd", + Port: port, + Host: hostname, + } + + etcd.AdvertiseClientURL = fmt.Sprintf("http://%s:%d", etcd.Host, etcd.Port) + etcd.ListenClientURL = fmt.Sprintf("http://%s:%d", etcd.Host, etcd.Port) + etcd.DataDirectory = path.Join(os.Getenv("VTDATAROOT"), "etcd") + etcd.VerifyURL = fmt.Sprintf("http://%s:%d/v2/keys", etcd.Host, etcd.Port) + return etcd +} diff --git a/go/test/endtoend/cluster/mysqlctl_process.go b/go/test/endtoend/cluster/mysqlctl_process.go new file mode 100644 index 00000000000..084d21c0837 --- /dev/null +++ b/go/test/endtoend/cluster/mysqlctl_process.go @@ -0,0 +1,85 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package cluster + +import ( + "fmt" + "os" + "os/exec" + "path" +) + +// MysqlctlProcess is a generic handle for a running mysqlctl command . +// It can be spawned manually +type MysqlctlProcess struct { + Name string + Binary string + LogDirectory string + TabletUID int + MySQLPort int + InitDBFile string +} + +// InitDb executes mysqlctl command to add cell info +func (mysqlctl *MysqlctlProcess) InitDb() (err error) { + tmpProcess := exec.Command( + mysqlctl.Binary, + "-log_dir", mysqlctl.LogDirectory, + "-tablet_uid", fmt.Sprintf("%d", mysqlctl.TabletUID), + "-mysql_port", fmt.Sprintf("%d", mysqlctl.MySQLPort), + "init", + "-init_db_sql_file", mysqlctl.InitDBFile, + ) + return tmpProcess.Run() +} + +// Start executes mysqlctl command to start mysql instance +func (mysqlctl *MysqlctlProcess) Start() (err error) { + tmpProcess := exec.Command( + mysqlctl.Binary, + "-log_dir", mysqlctl.LogDirectory, + "-tablet_uid", fmt.Sprintf("%d", mysqlctl.TabletUID), + "-mysql_port", fmt.Sprintf("%d", mysqlctl.MySQLPort), + "init", + "-init_db_sql_file", mysqlctl.InitDBFile, + ) + return tmpProcess.Run() +} + +// Stop executes mysqlctl command to stop mysql instance +func (mysqlctl *MysqlctlProcess) Stop() (err error) { + tmpProcess := exec.Command( + mysqlctl.Binary, + "-tablet_uid", fmt.Sprintf("%d", mysqlctl.TabletUID), + "shutdown", + ) + return tmpProcess.Run() +} + +// MysqlCtlProcessInstance returns a Mysqlctl handle for mysqlctl process +// configured with the given Config. +func MysqlCtlProcessInstance(TabletUID int, MySQLPort int) *MysqlctlProcess { + mysqlctl := &MysqlctlProcess{ + Name: "mysqlctl", + Binary: "mysqlctl", + LogDirectory: path.Join(os.Getenv("VTDATAROOT"), "/tmp"), + InitDBFile: path.Join(os.Getenv("VTROOT"), "/config/init_db.sql"), + } + mysqlctl.MySQLPort = MySQLPort + mysqlctl.TabletUID = TabletUID + return mysqlctl +} diff --git a/go/test/endtoend/cluster/vtctl_process.go b/go/test/endtoend/cluster/vtctl_process.go new file mode 100644 index 00000000000..74511b5c70f --- /dev/null +++ b/go/test/endtoend/cluster/vtctl_process.go @@ -0,0 +1,79 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package cluster + +import ( + "fmt" + "os/exec" + "strings" + + "vitess.io/vitess/go/vt/log" +) + +// VtctlProcess is a generic handle for a running vtctl command . +// It can be spawned manually +type VtctlProcess struct { + Name string + Binary string + TopoImplementation string + TopoGlobalAddress string + TopoGlobalRoot string + TopoServerAddress string +} + +// AddCellInfo executes vtctl command to add cell info +func (vtctl *VtctlProcess) AddCellInfo(Cell string) (err error) { + tmpProcess := exec.Command( + vtctl.Binary, + "-topo_implementation", vtctl.TopoImplementation, + "-topo_global_server_address", vtctl.TopoGlobalAddress, + "-topo_global_root", vtctl.TopoGlobalRoot, + "AddCellInfo", + "-root", "/vitess/"+Cell, + "-server_address", vtctl.TopoServerAddress, + Cell, + ) + return tmpProcess.Run() +} + +// CreateKeyspace executes vtctl command to create keyspace +func (vtctl *VtctlProcess) CreateKeyspace(keyspace string) (err error) { + tmpProcess := exec.Command( + vtctl.Binary, + "-topo_implementation", vtctl.TopoImplementation, + "-topo_global_server_address", vtctl.TopoGlobalAddress, + "-topo_global_root", vtctl.TopoGlobalRoot, + "CreateKeyspace", keyspace, + ) + log.Info(fmt.Sprintf("Starting CreateKeyspace with arguments %v", strings.Join(tmpProcess.Args, " "))) + return tmpProcess.Run() +} + +// VtctlProcessInstance returns a VtctlProcess handle for vtctl process +// configured with the given Config. +// The process must be manually started by calling setup() +func VtctlProcessInstance(topoPort int, hostname string) *VtctlProcess { + vtctl := &VtctlProcess{ + Name: "vtctl", + Binary: "vtctl", + TopoImplementation: "etcd2", + TopoGlobalAddress: fmt.Sprintf("%s:%d", hostname, topoPort), + TopoGlobalRoot: "/vitess/global", + TopoServerAddress: fmt.Sprintf("%s:%d", hostname, topoPort), + } + return vtctl +} diff --git a/go/test/endtoend/cluster/vtctlclient_process.go b/go/test/endtoend/cluster/vtctlclient_process.go new file mode 100644 index 00000000000..30184d74e90 --- /dev/null +++ b/go/test/endtoend/cluster/vtctlclient_process.go @@ -0,0 +1,100 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package cluster + +import ( + "fmt" + "os" + "os/exec" + "path" + "strings" + + "vitess.io/vitess/go/vt/log" +) + +// VtctlClientProcess is a generic handle for a running vtctlclient command . +// It can be spawned manually +type VtctlClientProcess struct { + Name string + Binary string + Server string + TempDirectory string + ZoneName string +} + +// InitShardMaster executes vtctlclient command to make one of tablet as master +func (vtctlclient *VtctlClientProcess) InitShardMaster(Keyspace string, Shard string, Cell string, TabletUID int) (err error) { + return vtctlclient.ExecuteCommand( + "InitShardMaster", + "-force", + fmt.Sprintf("%s/%s", Keyspace, Shard), + fmt.Sprintf("%s-%d", Cell, TabletUID)) +} + +// ApplySchema applies SQL schema to the keyspace +func (vtctlclient *VtctlClientProcess) ApplySchema(Keyspace string, SQL string) (err error) { + return vtctlclient.ExecuteCommand( + "ApplySchema", + "-sql", SQL, + Keyspace) +} + +// ApplyVSchema applies vitess schema (JSON format) to the keyspace +func (vtctlclient *VtctlClientProcess) ApplyVSchema(Keyspace string, JSON string) (err error) { + return vtctlclient.ExecuteCommand( + "ApplyVSchema", + "-vschema", JSON, + Keyspace, + ) +} + +// ExecuteCommand executes any vtctlclient command +func (vtctlclient *VtctlClientProcess) ExecuteCommand(args ...string) (err error) { + args = append([]string{"-server", vtctlclient.Server}, args...) + tmpProcess := exec.Command( + vtctlclient.Binary, + args..., + ) + println(fmt.Sprintf("Executing vtctlclient with arguments %v", strings.Join(tmpProcess.Args, " "))) + log.Info(fmt.Sprintf("Executing vtctlclient with arguments %v", strings.Join(tmpProcess.Args, " "))) + return tmpProcess.Run() +} + +// ExecuteCommandWithOutput executes any vtctlclient command and returns output +func (vtctlclient *VtctlClientProcess) ExecuteCommandWithOutput(args ...string) (result string, err error) { + args = append([]string{"-server", vtctlclient.Server}, args...) + tmpProcess := exec.Command( + vtctlclient.Binary, + args..., + ) + println(fmt.Sprintf("Executing vtctlclient with arguments %v", strings.Join(tmpProcess.Args, " "))) + log.Info(fmt.Sprintf("Executing vtctlclient with arguments %v", strings.Join(tmpProcess.Args, " "))) + resultByte, err := tmpProcess.CombinedOutput() + return string(resultByte), err +} + +// VtctlClientProcessInstance returns a VtctlProcess handle for vtctlclient process +// configured with the given Config. +func VtctlClientProcessInstance(Hostname string, GrpcPort int) *VtctlClientProcess { + vtctlclient := &VtctlClientProcess{ + Name: "vtctlclient", + Binary: "vtctlclient", + Server: fmt.Sprintf("%s:%d", Hostname, GrpcPort), + TempDirectory: path.Join(os.Getenv("VTDATAROOT"), "/tmp"), + } + return vtctlclient +} diff --git a/go/test/endtoend/cluster/vtctld_process.go b/go/test/endtoend/cluster/vtctld_process.go new file mode 100644 index 00000000000..f4f1c26c47b --- /dev/null +++ b/go/test/endtoend/cluster/vtctld_process.go @@ -0,0 +1,175 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package cluster + +import ( + "fmt" + "net/http" + "os" + "os/exec" + "path" + "strings" + "syscall" + "time" + + "vitess.io/vitess/go/vt/log" +) + +// VtctldProcess is a generic handle for a running vtctld . +// It can be spawned manually +type VtctldProcess struct { + Name string + Binary string + CommonArg VtctlProcess + WebDir string + WebDir2 string + ServiceMap string + BackupStorageImplementation string + FileBackupStorageRoot string + LogDir string + Port int + GrpcPort int + PidFile string + VerifyURL string + Directory string + + proc *exec.Cmd + exit chan error +} + +// Setup starts vtctld process with required arguements +func (vtctld *VtctldProcess) Setup(Cell string) (err error) { + err = os.Mkdir(path.Join(vtctld.Directory, "tmp"), 0700) + if err != nil { + return + } + err = os.Mkdir(path.Join(vtctld.Directory, "backups"), 0700) + if err != nil { + return + } + vtctld.proc = exec.Command( + vtctld.Binary, + "-enable_queries", + "-topo_implementation", vtctld.CommonArg.TopoImplementation, + "-topo_global_server_address", vtctld.CommonArg.TopoGlobalAddress, + "-topo_global_root", vtctld.CommonArg.TopoGlobalRoot, + "-cell", Cell, + "-web_dir", vtctld.WebDir, + "-web_dir2", vtctld.WebDir2, + "-workflow_manager_init", + "-workflow_manager_use_election", + "-service_map", vtctld.ServiceMap, + "-backup_storage_implementation", vtctld.BackupStorageImplementation, + "-file_backup_storage_root", vtctld.FileBackupStorageRoot, + "-log_dir", vtctld.LogDir, + "-port", fmt.Sprintf("%d", vtctld.Port), + "-grpc_port", fmt.Sprintf("%d", vtctld.GrpcPort), + "-pid_file", vtctld.PidFile, + ) + + vtctld.proc.Stderr = os.Stderr + vtctld.proc.Stdout = os.Stdout + + vtctld.proc.Env = append(vtctld.proc.Env, os.Environ()...) + + log.Infof("%v %v", strings.Join(vtctld.proc.Args, " ")) + + err = vtctld.proc.Start() + if err != nil { + return + } + + vtctld.exit = make(chan error) + go func() { + vtctld.exit <- vtctld.proc.Wait() + }() + + timeout := time.Now().Add(60 * time.Second) + for time.Now().Before(timeout) { + if vtctld.IsHealthy() { + return nil + } + select { + case err := <-vtctld.exit: + return fmt.Errorf("process '%s' exited prematurely (err: %s)", vtctld.Name, err) + default: + time.Sleep(300 * time.Millisecond) + } + } + + return fmt.Errorf("process '%s' timed out after 60s (err: %s)", vtctld.Name, <-vtctld.exit) +} + +// IsHealthy function checks if vtctld process is up and running +func (vtctld *VtctldProcess) IsHealthy() bool { + resp, err := http.Get(vtctld.VerifyURL) + if err != nil { + return false + } + if resp.StatusCode == 200 { + return true + } + return false +} + +// TearDown shutdowns the running vtctld service +func (vtctld *VtctldProcess) TearDown() error { + if vtctld.proc == nil || vtctld.exit == nil { + return nil + } + + os.RemoveAll(path.Join(vtctld.Directory, "tmp")) + os.RemoveAll(path.Join(vtctld.Directory, "backups")) + + // Attempt graceful shutdown with SIGTERM first + vtctld.proc.Process.Signal(syscall.SIGTERM) + + select { + case err := <-vtctld.exit: + vtctld.proc = nil + return err + + case <-time.After(10 * time.Second): + vtctld.proc.Process.Kill() + vtctld.proc = nil + return <-vtctld.exit + } +} + +// VtctldProcessInstance returns a VtctlProcess handle for vtctl process +// configured with the given Config. +// The process must be manually started by calling setup() +func VtctldProcessInstance(httpPort int, grpcPort int, topoPort int, hostname string) *VtctldProcess { + vtctl := VtctlProcessInstance(topoPort, hostname) + vtctld := &VtctldProcess{ + Name: "vtctld", + Binary: "vtctld", + CommonArg: *vtctl, + WebDir: path.Join(os.Getenv("VTROOT"), "/web/vtctld"), + WebDir2: path.Join(os.Getenv("VTROOT"), "/web/vtctld2/app"), + ServiceMap: "grpc-vtctl", + BackupStorageImplementation: "file", + FileBackupStorageRoot: path.Join(os.Getenv("VTDATAROOT"), "/backups"), + LogDir: path.Join(os.Getenv("VTDATAROOT"), "/tmp"), + Port: httpPort, + GrpcPort: grpcPort, + PidFile: path.Join(os.Getenv("VTDATAROOT"), "/tmp", "vtctld.pid"), + Directory: os.Getenv("VTDATAROOT"), + } + vtctld.VerifyURL = fmt.Sprintf("http://localhost:%d", vtctld.Port) + return vtctld +} diff --git a/go/test/endtoend/cluster/vtgate_process.go b/go/test/endtoend/cluster/vtgate_process.go new file mode 100644 index 00000000000..44fca93abda --- /dev/null +++ b/go/test/endtoend/cluster/vtgate_process.go @@ -0,0 +1,187 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package cluster + +import ( + "encoding/json" + "fmt" + "io/ioutil" + "net/http" + "os" + "os/exec" + "path" + "strings" + "syscall" + "time" + + "vitess.io/vitess/go/vt/log" +) + +// VtgateProcess is a generic handle for a running vtgate . +// It can be spawned manually +type VtgateProcess struct { + Name string + Binary string + CommonArg VtctlProcess + LogDir string + FileToLogQueries string + Port int + GrpcPort int + MySQLServerPort int + MySQLServerSocketPath string + Cell string + CellsToWatch string + TabletTypesToWait string + GatewayImplementation string + ServiceMap string + PidFile string + MySQLAuthServerImpl string + Directory string + VerifyURL string + + proc *exec.Cmd + exit chan error +} + +// Setup starts Vtgate process with required arguements +func (vtgate *VtgateProcess) Setup() (err error) { + + vtgate.proc = exec.Command( + vtgate.Binary, + "-topo_implementation", vtgate.CommonArg.TopoImplementation, + "-topo_global_server_address", vtgate.CommonArg.TopoGlobalAddress, + "-topo_global_root", vtgate.CommonArg.TopoGlobalRoot, + "-log_dir", vtgate.LogDir, + "-log_queries_to_file", vtgate.FileToLogQueries, + "-port", fmt.Sprintf("%d", vtgate.Port), + "-grpc_port", fmt.Sprintf("%d", vtgate.GrpcPort), + "-mysql_server_port", fmt.Sprintf("%d", vtgate.MySQLServerPort), + "-mysql_server_socket_path", vtgate.MySQLServerSocketPath, + "-cell", vtgate.Cell, + "-cells_to_watch", vtgate.CellsToWatch, + "-tablet_types_to_wait", vtgate.TabletTypesToWait, + "-gateway_implementation", vtgate.GatewayImplementation, + "-service_map", vtgate.ServiceMap, + "-mysql_auth_server_impl", vtgate.MySQLAuthServerImpl, + "-pid_file", vtgate.PidFile, + ) + + vtgate.proc.Stderr = os.Stderr + vtgate.proc.Stdout = os.Stdout + + vtgate.proc.Env = append(vtgate.proc.Env, os.Environ()...) + + log.Infof("%v %v", strings.Join(vtgate.proc.Args, " ")) + + err = vtgate.proc.Start() + if err != nil { + return + } + + vtgate.exit = make(chan error) + go func() { + vtgate.exit <- vtgate.proc.Wait() + }() + + timeout := time.Now().Add(60 * time.Second) + for time.Now().Before(timeout) { + if vtgate.WaitForStatus() { + return nil + } + select { + case err := <-vtgate.exit: + return fmt.Errorf("process '%s' exited prematurely (err: %s)", vtgate.Name, err) + default: + time.Sleep(300 * time.Millisecond) + } + } + + return fmt.Errorf("process '%s' timed out after 60s (err: %s)", vtgate.Name, <-vtgate.exit) +} + +// WaitForStatus function checks if vtgate process is up and running +func (vtgate *VtgateProcess) WaitForStatus() bool { + resp, err := http.Get(vtgate.VerifyURL) + if err != nil { + return false + } + if resp.StatusCode == 200 { + resultMap := make(map[string]interface{}) + respByte, _ := ioutil.ReadAll(resp.Body) + err := json.Unmarshal(respByte, &resultMap) + if err != nil { + panic(err) + } + //for key, value := range resultMap { + // println("VTGate API Response: Key = " + key + ", value = " + fmt.Sprintf("%v", value)) + //} + //println(string(respByte)) + //return resultMap["TabletStateName"] == "NOT_SERVING" + return true + } + return false +} + +// TearDown shuts down the running vtgate service +func (vtgate *VtgateProcess) TearDown() error { + if vtgate.proc == nil || vtgate.exit == nil { + return nil + } + // Attempt graceful shutdown with SIGTERM first + vtgate.proc.Process.Signal(syscall.SIGTERM) + + select { + case err := <-vtgate.exit: + vtgate.proc = nil + return err + + case <-time.After(10 * time.Second): + vtgate.proc.Process.Kill() + vtgate.proc = nil + return <-vtgate.exit + } +} + +// VtgateProcessInstance returns a Vtgate handle for vtgate process +// configured with the given Config. +// The process must be manually started by calling setup() +func VtgateProcessInstance(Port int, GrpcPort int, MySQLServerPort int, Cell string, CellsToWatch string, Hostname string, TabletTypesToWait string, topoPort int, hostname string) *VtgateProcess { + vtctl := VtctlProcessInstance(topoPort, hostname) + vtgate := &VtgateProcess{ + Name: "vtgate", + Binary: "vtgate", + FileToLogQueries: path.Join(os.Getenv("VTDATAROOT"), "/tmp/vtgate_querylog.txt"), + Directory: os.Getenv("VTDATAROOT"), + ServiceMap: "grpc-vtgateservice", + LogDir: path.Join(os.Getenv("VTDATAROOT"), "/tmp"), + Port: Port, + GrpcPort: GrpcPort, + MySQLServerPort: MySQLServerPort, + MySQLServerSocketPath: "/tmp/mysql.sock", + Cell: Cell, + CellsToWatch: CellsToWatch, + TabletTypesToWait: TabletTypesToWait, + GatewayImplementation: "discoverygateway", + CommonArg: *vtctl, + PidFile: path.Join(os.Getenv("VTDATAROOT"), "/tmp/vtgate.pid"), + MySQLAuthServerImpl: "none", + } + + vtgate.VerifyURL = fmt.Sprintf("http://%s:%d/debug/vars", Hostname, Port) + + return vtgate +} diff --git a/go/test/endtoend/cluster/vttablet_process.go b/go/test/endtoend/cluster/vttablet_process.go new file mode 100644 index 00000000000..65626c96559 --- /dev/null +++ b/go/test/endtoend/cluster/vttablet_process.go @@ -0,0 +1,201 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package cluster + +import ( + "encoding/json" + "fmt" + "io/ioutil" + "net/http" + "os" + "os/exec" + "path" + "strings" + "syscall" + "time" + + "vitess.io/vitess/go/vt/log" +) + +// VttabletProcess is a generic handle for a running vttablet . +// It can be spawned manually +type VttabletProcess struct { + Name string + Binary string + FileToLogQueries string + TabletUID int + TabletPath string + Cell string + Port int + GrpcPort int + PidFile string + Shard string + CommonArg VtctlProcess + LogDir string + TabletHostname string + Keyspace string + TabletType string + HealthCheckInterval int + BackupStorageImplementation string + FileBackupStorageRoot string + ServiceMap string + VtctldAddress string + Directory string + VerifyURL string + + proc *exec.Cmd + exit chan error +} + +// Setup starts vtctld process with required arguements +func (vttablet *VttabletProcess) Setup() (err error) { + + vttablet.proc = exec.Command( + vttablet.Binary, + "-topo_implementation", vttablet.CommonArg.TopoImplementation, + "-topo_global_server_address", vttablet.CommonArg.TopoGlobalAddress, + "-topo_global_root", vttablet.CommonArg.TopoGlobalRoot, + "-log_queries_to_file", vttablet.FileToLogQueries, + "-tablet-path", vttablet.TabletPath, + "-port", fmt.Sprintf("%d", vttablet.Port), + "-grpc_port", fmt.Sprintf("%d", vttablet.GrpcPort), + "-pid_file", vttablet.PidFile, + "-init_shard", vttablet.Shard, + "-log_dir", vttablet.LogDir, + "-tablet_hostname", vttablet.TabletHostname, + "-init_keyspace", vttablet.Keyspace, + "-init_tablet_type", vttablet.TabletType, + "-health_check_interval", fmt.Sprintf("%ds", vttablet.HealthCheckInterval), + "-enable_semi_sync", + "-enable_replication_reporter", + "-backup_storage_implementation", vttablet.BackupStorageImplementation, + "-file_backup_storage_root", vttablet.FileBackupStorageRoot, + "-restore_from_backup", + "-service_map", vttablet.ServiceMap, + "-vtctld_addr", vttablet.VtctldAddress, + ) + + vttablet.proc.Stderr = os.Stderr + vttablet.proc.Stdout = os.Stdout + + vttablet.proc.Env = append(vttablet.proc.Env, os.Environ()...) + + log.Infof("%v %v", strings.Join(vttablet.proc.Args, " ")) + + err = vttablet.proc.Start() + if err != nil { + return + } + + vttablet.exit = make(chan error) + go func() { + vttablet.exit <- vttablet.proc.Wait() + }() + + timeout := time.Now().Add(60 * time.Second) + for time.Now().Before(timeout) { + if vttablet.WaitForStatus("NOT_SERVING") { + return nil + } + select { + case err := <-vttablet.exit: + return fmt.Errorf("process '%s' exited prematurely (err: %s)", vttablet.Name, err) + default: + time.Sleep(300 * time.Millisecond) + } + } + + return fmt.Errorf("process '%s' timed out after 60s (err: %s)", vttablet.Name, <-vttablet.exit) +} + +// WaitForStatus function checks if vttablet process is up and running +func (vttablet *VttabletProcess) WaitForStatus(status string) bool { + resp, err := http.Get(vttablet.VerifyURL) + if err != nil { + return false + } + if resp.StatusCode == 200 { + resultMap := make(map[string]interface{}) + respByte, _ := ioutil.ReadAll(resp.Body) + err := json.Unmarshal(respByte, &resultMap) + if err != nil { + panic(err) + } + return resultMap["TabletStateName"] == status + } + return false +} + +// TearDown shuts down the running vttablet service +func (vttablet *VttabletProcess) TearDown() error { + if vttablet.proc == nil { + fmt.Printf("No process found for vttablet %d", vttablet.TabletUID) + } + if vttablet.proc == nil || vttablet.exit == nil { + return nil + } + // Attempt graceful shutdown with SIGTERM first + vttablet.proc.Process.Signal(syscall.SIGTERM) + + os.RemoveAll(vttablet.Directory) + + select { + case err := <-vttablet.exit: + vttablet.proc = nil + return err + + case <-time.After(10 * time.Second): + vttablet.proc.Process.Kill() + vttablet.proc = nil + return <-vttablet.exit + } +} + +// VttabletProcessInstance returns a VttabletProcess handle for vttablet process +// configured with the given Config. +// The process must be manually started by calling setup() +func VttabletProcessInstance(Port int, GrpcPort int, TabletUID int, Cell string, Shard string, Hostname string, Keyspace string, VtctldPort int, TabletType string, topoPort int, hostname string) *VttabletProcess { + vtctl := VtctlProcessInstance(topoPort, hostname) + vttablet := &VttabletProcess{ + Name: "vttablet", + Binary: "vttablet", + FileToLogQueries: path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("/tmp/vt_%010d/vttable.pid", TabletUID)), + Directory: path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("/vt_%010d", TabletUID)), + TabletPath: fmt.Sprintf("%s-%010d", Cell, TabletUID), + ServiceMap: "grpc-queryservice,grpc-tabletmanager,grpc-updatestream", + LogDir: path.Join(os.Getenv("VTDATAROOT"), "/tmp"), + Shard: Shard, + TabletHostname: Hostname, + Keyspace: Keyspace, + TabletType: "replica", + CommonArg: *vtctl, + HealthCheckInterval: 5, + BackupStorageImplementation: "file", + FileBackupStorageRoot: path.Join(os.Getenv("VTDATAROOT"), "/backups"), + Port: Port, + GrpcPort: GrpcPort, + PidFile: path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("/vt_%010d/vttable.pid", TabletUID)), + VtctldAddress: fmt.Sprintf("http://%s:%d", Hostname, VtctldPort), + } + + if TabletType == "rdonly" { + vttablet.TabletType = TabletType + } + vttablet.VerifyURL = fmt.Sprintf("http://%s:%d/debug/vars", Hostname, Port) + + return vttablet +} diff --git a/go/test/endtoend/clustertest/add_keyspace_test.go b/go/test/endtoend/clustertest/add_keyspace_test.go new file mode 100644 index 00000000000..e305866b752 --- /dev/null +++ b/go/test/endtoend/clustertest/add_keyspace_test.go @@ -0,0 +1,85 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + +This adds sharded keyspace dynamically in this test only and test sql insert, select +*/ + +package clustertest + +import ( + "context" + "fmt" + "testing" + + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/test/endtoend/cluster" +) + +var ( + testKeyspace = &cluster.Keyspace{ + Name: "kstest", + SchemaSQL: `create table vt_user ( +id bigint, +name varchar(64), +primary key (id) +) Engine=InnoDB`, + VSchema: `{ + "sharded": true, + "vindexes": { + "hash_index": { + "type": "hash" + } + }, + "tables": { + "vt_user": { + "column_vindexes": [ + { + "column": "id", + "name": "hash_index" + } + ] + } + } +}`, + } +) + +func TestAddKeyspace(t *testing.T) { + if err := clusterInstance.StartKeyspace(*testKeyspace, []string{"-80", "80-"}, 1, true); err != nil { + println(err.Error()) + t.Fatal(err) + } + // Restart vtgate process + _ = clusterInstance.VtgateProcess.TearDown() + _ = clusterInstance.VtgateProcess.Setup() + + ctx := context.Background() + vtParams := mysql.ConnParams{ + Host: clusterInstance.Hostname, + Port: clusterInstance.VtgateMySQLPort, + } + conn, err := mysql.Connect(ctx, &vtParams) + if err != nil { + t.Fatal(err) + } + defer conn.Close() + + exec(t, conn, "insert into vt_user(id, name) values(1,'name1')") + + qr := exec(t, conn, "select id, name from vt_user") + if got, want := fmt.Sprintf("%v", qr.Rows), `[[INT64(1) VARCHAR("name1")]]`; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } +} diff --git a/go/test/endtoend/clustertest/etcd_test.go b/go/test/endtoend/clustertest/etcd_test.go new file mode 100644 index 00000000000..cb0138b0d5e --- /dev/null +++ b/go/test/endtoend/clustertest/etcd_test.go @@ -0,0 +1,29 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package clustertest + +import ( + "fmt" + "testing" +) + +func TestEtcdServer(t *testing.T) { + etcdURL := fmt.Sprintf("http://%s:%d/v2/keys", clusterInstance.Hostname, clusterInstance.TopoPort) + testURL(t, etcdURL, "generic etcd url") + testURL(t, etcdURL+"/vitess/global", "vitess global key") + testURL(t, etcdURL+"/vitess/zone1", "vitess zone1 key") +} diff --git a/go/test/endtoend/clustertest/main_test.go b/go/test/endtoend/clustertest/main_test.go new file mode 100644 index 00000000000..91eb4bde771 --- /dev/null +++ b/go/test/endtoend/clustertest/main_test.go @@ -0,0 +1,114 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package clustertest + +import ( + "flag" + "net/http" + "os" + "testing" + + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/test/endtoend/cluster" +) + +var ( + clusterInstance *cluster.LocalProcessCluster + vtParams mysql.ConnParams + keyspaceName = "commerce" + cell = "zone1" + sqlSchema = `create table product( + sku varbinary(128), + description varbinary(128), + price bigint, + primary key(sku) + ) ENGINE=InnoDB; + create table customer( + id bigint not null auto_increment, + email varchar(128), + primary key(id) + ) ENGINE=InnoDB; + create table corder( + order_id bigint not null auto_increment, + customer_id bigint, + sku varbinary(128), + price bigint, + primary key(order_id) + ) ENGINE=InnoDB;` + + vSchema = `{ + "tables": { + "product": {}, + "customer": {}, + "corder": {} + } + }` +) + +func TestMain(m *testing.M) { + flag.Parse() + + exitCode := func() int { + clusterInstance = &cluster.LocalProcessCluster{Cell: cell, Hostname: "localhost"} + defer clusterInstance.Teardown() + + // Start topo server + err := clusterInstance.StartTopo() + if err != nil { + return 1 + } + + // Start keyspace + keyspace := &cluster.Keyspace{ + Name: keyspaceName, + SchemaSQL: sqlSchema, + VSchema: vSchema, + } + err = clusterInstance.StartUnshardedKeyspace(*keyspace, 1, true) + if err != nil { + return 1 + } + + // Start vtgate + err = clusterInstance.StartVtgate() + if err != nil { + return 1 + } + vtParams = mysql.ConnParams{ + Host: clusterInstance.Hostname, + Port: clusterInstance.VtgateMySQLPort, + } + return m.Run() + }() + os.Exit(exitCode) +} + +func testURL(t *testing.T, url string, testCaseName string) { + statusCode := getStatusForURL(url) + if got, want := statusCode, 200; got != want { + t.Errorf("select:\n%v want\n%v for %s", got, want, testCaseName) + } +} + +// getStatusForUrl returns the status code for the URL +func getStatusForURL(url string) int { + resp, _ := http.Get(url) + if resp != nil { + return resp.StatusCode + } + return 0 +} diff --git a/go/test/endtoend/clustertest/vtcltd_test.go b/go/test/endtoend/clustertest/vtcltd_test.go new file mode 100644 index 00000000000..4704fd7f99a --- /dev/null +++ b/go/test/endtoend/clustertest/vtcltd_test.go @@ -0,0 +1,28 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + +*/ + +package clustertest + +import ( + "fmt" + "testing" +) + +func TestVtctldProcess(t *testing.T) { + url := fmt.Sprintf("http://localhost:%d/api/keyspaces/", clusterInstance.VtctldHTTPPort) + testURL(t, url, "keyspace url") +} diff --git a/go/test/endtoend/clustertest/vtgate_test.go b/go/test/endtoend/clustertest/vtgate_test.go new file mode 100644 index 00000000000..67773139cc0 --- /dev/null +++ b/go/test/endtoend/clustertest/vtgate_test.go @@ -0,0 +1,52 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + +This tests select/insert using the unshared keyspace added in main_test +*/ +package clustertest + +import ( + "context" + "fmt" + "testing" + + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/sqltypes" +) + +func TestVtgateProcess(t *testing.T) { + ctx := context.Background() + conn, err := mysql.Connect(ctx, &vtParams) + if err != nil { + t.Fatal(err) + } + defer conn.Close() + + exec(t, conn, "insert into customer(id, email) values(1,'email1')") + + qr := exec(t, conn, "select id, email from customer") + if got, want := fmt.Sprintf("%v", qr.Rows), `[[INT64(1) VARCHAR("email1")]]`; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } +} + +func exec(t *testing.T, conn *mysql.Conn, query string) *sqltypes.Result { + t.Helper() + qr, err := conn.ExecuteFetch(query, 1000, true) + if err != nil { + t.Fatal(err) + } + return qr +} diff --git a/go/test/endtoend/clustertest/vttablet_test.go b/go/test/endtoend/clustertest/vttablet_test.go new file mode 100644 index 00000000000..30e7beeca15 --- /dev/null +++ b/go/test/endtoend/clustertest/vttablet_test.go @@ -0,0 +1,41 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. + +*/ + +package clustertest + +import ( + "encoding/json" + "fmt" + "io/ioutil" + "net/http" + "testing" +) + +func TestVttabletProcess(t *testing.T) { + firstTabletPort := clusterInstance.Keyspaces[0].Shards[0].Vttablets[0].HTTPPort + testURL(t, fmt.Sprintf("http://localhost:%d/debug/vars/", firstTabletPort), "tablet debug var url") + resp, _ := http.Get(fmt.Sprintf("http://localhost:%d/debug/vars", firstTabletPort)) + resultMap := make(map[string]interface{}) + respByte, _ := ioutil.ReadAll(resp.Body) + err := json.Unmarshal(respByte, &resultMap) + if err != nil { + panic(err) + } + if got, want := resultMap["TabletKeyspace"], "commerce"; got != want { + t.Errorf("select:\n%v want\n%v for %s", got, want, "Keyspace of tablet should match") + } +} diff --git a/go/test/endtoend/vtgate/aggr_test.go b/go/test/endtoend/vtgate/aggr_test.go new file mode 100644 index 00000000000..e23e4a8970e --- /dev/null +++ b/go/test/endtoend/vtgate/aggr_test.go @@ -0,0 +1,57 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package vtgate + +import ( + "context" + "fmt" + "testing" + + "vitess.io/vitess/go/mysql" +) + +func TestAggregateTypes(t *testing.T) { + ctx := context.Background() + conn, err := mysql.Connect(ctx, &vtParams) + if err != nil { + t.Fatal(err) + } + defer conn.Close() + + exec(t, conn, "insert into aggr_test(id, val1, val2) values(1,'a',1), (2,'A',1), (3,'b',1), (4,'c',3), (5,'c',4)") + exec(t, conn, "insert into aggr_test(id, val1, val2) values(6,'d',null), (7,'e',null), (8,'E',1)") + + qr := exec(t, conn, "select val1, count(distinct val2), count(*) from aggr_test group by val1") + if got, want := fmt.Sprintf("%v", qr.Rows), `[[VARCHAR("a") INT64(1) INT64(2)] [VARCHAR("b") INT64(1) INT64(1)] [VARCHAR("c") INT64(2) INT64(2)] [VARCHAR("d") INT64(0) INT64(1)] [VARCHAR("e") INT64(1) INT64(2)]]`; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + + qr = exec(t, conn, "select val1, sum(distinct val2), sum(val2) from aggr_test group by val1") + if got, want := fmt.Sprintf("%v", qr.Rows), `[[VARCHAR("a") DECIMAL(1) DECIMAL(2)] [VARCHAR("b") DECIMAL(1) DECIMAL(1)] [VARCHAR("c") DECIMAL(7) DECIMAL(7)] [VARCHAR("d") NULL NULL] [VARCHAR("e") DECIMAL(1) DECIMAL(1)]]`; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + + qr = exec(t, conn, "select val1, count(distinct val2) k, count(*) from aggr_test group by val1 order by k desc, val1") + if got, want := fmt.Sprintf("%v", qr.Rows), `[[VARCHAR("c") INT64(2) INT64(2)] [VARCHAR("a") INT64(1) INT64(2)] [VARCHAR("b") INT64(1) INT64(1)] [VARCHAR("e") INT64(1) INT64(2)] [VARCHAR("d") INT64(0) INT64(1)]]`; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + + qr = exec(t, conn, "select val1, count(distinct val2) k, count(*) from aggr_test group by val1 order by k desc, val1 limit 4") + if got, want := fmt.Sprintf("%v", qr.Rows), `[[VARCHAR("c") INT64(2) INT64(2)] [VARCHAR("a") INT64(1) INT64(2)] [VARCHAR("b") INT64(1) INT64(1)] [VARCHAR("e") INT64(1) INT64(2)]]`; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } +} diff --git a/go/test/endtoend/vtgate/lookup_test.go b/go/test/endtoend/vtgate/lookup_test.go new file mode 100644 index 00000000000..126b2593609 --- /dev/null +++ b/go/test/endtoend/vtgate/lookup_test.go @@ -0,0 +1,270 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package vtgate + +import ( + "context" + "fmt" + "strings" + "testing" + + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/sqltypes" +) + +func TestConsistentLookup(t *testing.T) { + ctx := context.Background() + conn, err := mysql.Connect(ctx, &vtParams) + if err != nil { + t.Fatal(err) + } + defer conn.Close() + // conn2 is for queries that target shards. + conn2, err := mysql.Connect(ctx, &vtParams) + if err != nil { + t.Fatal(err) + } + defer conn2.Close() + + // Simple insert. + exec(t, conn, "begin") + exec(t, conn, "insert into t1(id1, id2) values(1, 4)") + exec(t, conn, "commit") + qr := exec(t, conn, "select * from t1") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(1) INT64(4)]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + qr = exec(t, conn, "select * from t1_id2_idx") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(4) VARBINARY(\"\\x16k@\\xb4J\\xbaK\\xd6\")]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + + // Inserting again should fail. + exec(t, conn, "begin") + _, err = conn.ExecuteFetch("insert into t1(id1, id2) values(1, 4)", 1000, false) + exec(t, conn, "rollback") + want := "duplicate entry" + if err == nil || !strings.Contains(err.Error(), want) { + t.Errorf("second insert: %v, must contain %s", err, want) + } + + // Simple delete. + exec(t, conn, "begin") + exec(t, conn, "delete from t1 where id1=1") + exec(t, conn, "commit") + qr = exec(t, conn, "select * from t1") + if got, want := fmt.Sprintf("%v", qr.Rows), "[]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + qr = exec(t, conn, "select * from t1_id2_idx") + if got, want := fmt.Sprintf("%v", qr.Rows), "[]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + + // Autocommit insert. + exec(t, conn, "insert into t1(id1, id2) values(1, 4)") + qr = exec(t, conn, "select * from t1") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(1) INT64(4)]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + qr = exec(t, conn, "select id2 from t1_id2_idx") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(4)]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + // Autocommit delete. + exec(t, conn, "delete from t1 where id1=1") + + // Dangling row pointing to existing keyspace id. + exec(t, conn, "insert into t1(id1, id2) values(1, 4)") + // Delete the main row only. + exec(t, conn2, "use `ks:-80`") + exec(t, conn2, "delete from t1 where id1=1") + // Verify the lookup row is still there. + qr = exec(t, conn, "select id2 from t1_id2_idx") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(4)]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + // Insert should still succeed. + exec(t, conn, "begin") + exec(t, conn, "insert into t1(id1, id2) values(1, 4)") + exec(t, conn, "commit") + qr = exec(t, conn, "select * from t1") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(1) INT64(4)]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + // Lookup row should be unchanged. + qr = exec(t, conn, "select * from t1_id2_idx") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(4) VARBINARY(\"\\x16k@\\xb4J\\xbaK\\xd6\")]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + + // Dangling row not pointing to existing keyspace id. + exec(t, conn2, "use `ks:-80`") + exec(t, conn2, "delete from t1 where id1=1") + // Update the lookup row with bogus keyspace id. + exec(t, conn, "update t1_id2_idx set keyspace_id='aaa' where id2=4") + qr = exec(t, conn, "select * from t1_id2_idx") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(4) VARBINARY(\"aaa\")]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + // Insert should still succeed. + exec(t, conn, "begin") + exec(t, conn, "insert into t1(id1, id2) values(1, 4)") + exec(t, conn, "commit") + qr = exec(t, conn, "select * from t1") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(1) INT64(4)]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + // lookup row must be updated. + qr = exec(t, conn, "select * from t1_id2_idx") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(4) VARBINARY(\"\\x16k@\\xb4J\\xbaK\\xd6\")]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + + // Update, but don't change anything. This should not deadlock. + exec(t, conn, "begin") + exec(t, conn, "update t1 set id2=4 where id1=1") + exec(t, conn, "commit") + qr = exec(t, conn, "select * from t1") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(1) INT64(4)]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + qr = exec(t, conn, "select * from t1_id2_idx") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(4) VARBINARY(\"\\x16k@\\xb4J\\xbaK\\xd6\")]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + + // Update, and change the lookup value. This should change main and lookup rows. + exec(t, conn, "begin") + exec(t, conn, "update t1 set id2=5 where id1=1") + exec(t, conn, "commit") + qr = exec(t, conn, "select * from t1") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(1) INT64(5)]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + qr = exec(t, conn, "select * from t1_id2_idx") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(5) VARBINARY(\"\\x16k@\\xb4J\\xbaK\\xd6\")]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + exec(t, conn, "delete from t1 where id1=1") +} + +func TestConsistentLookupMultiInsert(t *testing.T) { + ctx := context.Background() + conn, err := mysql.Connect(ctx, &vtParams) + if err != nil { + t.Fatal(err) + } + defer conn.Close() + // conn2 is for queries that target shards. + conn2, err := mysql.Connect(ctx, &vtParams) + if err != nil { + t.Fatal(err) + } + defer conn2.Close() + + exec(t, conn, "begin") + exec(t, conn, "insert into t1(id1, id2) values(1,4), (2,5)") + exec(t, conn, "commit") + qr := exec(t, conn, "select * from t1") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(1) INT64(4)] [INT64(2) INT64(5)]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + qr = exec(t, conn, "select count(*) from t1_id2_idx") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(2)]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + + // Delete one row but leave its lookup dangling. + exec(t, conn2, "use `ks:-80`") + exec(t, conn2, "delete from t1 where id1=1") + // Insert a bogus lookup row. + exec(t, conn, "insert into t1_id2_idx(id2, keyspace_id) values(6, 'aaa')") + // Insert 3 rows: + // first row will insert without changing lookup. + // second will insert and change lookup. + // third will be a fresh insert for main and lookup. + exec(t, conn, "begin") + exec(t, conn, "insert into t1(id1, id2) values(1,2), (3,6), (4,7)") + exec(t, conn, "commit") + qr = exec(t, conn, "select id1, id2 from t1 order by id1") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(1) INT64(2)] [INT64(2) INT64(5)] [INT64(3) INT64(6)] [INT64(4) INT64(7)]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + qr = exec(t, conn, "select * from t1_id2_idx where id2=6") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(6) VARBINARY(\"N\\xb1\\x90É¢\\xfa\\x16\\x9c\")]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + qr = exec(t, conn, "select count(*) from t1_id2_idx") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(5)]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + exec(t, conn, "delete from t1 where id1=1") + exec(t, conn, "delete from t1 where id1=2") + exec(t, conn, "delete from t1 where id1=3") + exec(t, conn, "delete from t1 where id1=4") + exec(t, conn, "delete from t1_id2_idx where id2=4") +} + +func TestHashLookupMultiInsertIgnore(t *testing.T) { + ctx := context.Background() + conn, err := mysql.Connect(ctx, &vtParams) + if err != nil { + t.Fatal(err) + } + defer conn.Close() + // conn2 is for queries that target shards. + conn2, err := mysql.Connect(ctx, &vtParams) + if err != nil { + t.Fatal(err) + } + defer conn2.Close() + + // DB should start out clean + qr := exec(t, conn, "select count(*) from t2_id4_idx") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(0)]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + qr = exec(t, conn, "select count(*) from t2") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(0)]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + + // Try inserting a bunch of ids at once + exec(t, conn, "begin") + exec(t, conn, "insert ignore into t2(id3, id4) values(50,60), (30,40), (10,20)") + exec(t, conn, "commit") + + // Verify + qr = exec(t, conn, "select id3, id4 from t2 order by id3") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(10) INT64(20)] [INT64(30) INT64(40)] [INT64(50) INT64(60)]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + qr = exec(t, conn, "select id3, id4 from t2_id4_idx order by id3") + if got, want := fmt.Sprintf("%v", qr.Rows), "[[INT64(10) INT64(20)] [INT64(30) INT64(40)] [INT64(50) INT64(60)]]"; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } +} + +func exec(t *testing.T, conn *mysql.Conn, query string) *sqltypes.Result { + t.Helper() + qr, err := conn.ExecuteFetch(query, 1000, true) + if err != nil { + t.Fatal(err) + } + return qr +} diff --git a/go/test/endtoend/vtgate/main_test.go b/go/test/endtoend/vtgate/main_test.go new file mode 100644 index 00000000000..9cd2a9a88ec --- /dev/null +++ b/go/test/endtoend/vtgate/main_test.go @@ -0,0 +1,203 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package vtgate + +import ( + "flag" + "os" + "testing" + + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/test/endtoend/cluster" +) + +var ( + clusterInstance *cluster.LocalProcessCluster + vtParams mysql.ConnParams + KeyspaceName = "ks" + Cell = "test" + SchemaSQL = `create table t1( + id1 bigint, + id2 bigint, + primary key(id1) +) Engine=InnoDB; + +create table t1_id2_idx( + id2 bigint, + keyspace_id varbinary(10), + primary key(id2) +) Engine=InnoDB; + +create table vstream_test( + id bigint, + val bigint, + primary key(id) +) Engine=InnoDB; + +create table aggr_test( + id bigint, + val1 varchar(16), + val2 bigint, + primary key(id) +) Engine=InnoDB; + +create table t2( + id3 bigint, + id4 bigint, + primary key(id3) +) Engine=InnoDB; + +create table t2_id4_idx( + id bigint not null auto_increment, + id4 bigint, + id3 bigint, + primary key(id), + key idx_id4(id4) +) Engine=InnoDB; +` + + VSchema = ` + { + "sharded": true, + "vindexes": { + "hash": { + "type": "hash" + }, + "t1_id2_vdx": { + "type": "consistent_lookup_unique", + "params": { + "table": "t1_id2_idx", + "from": "id2", + "to": "keyspace_id" + }, + "owner": "t1" + }, + "t2_id4_idx": { + "type": "lookup_hash", + "params": { + "table": "t2_id4_idx", + "from": "id4", + "to": "id3", + "autocommit": "true" + }, + "owner": "t2" + } + }, + "tables": { + "t1": { + "column_vindexes": [ + { + "column": "id1", + "name": "hash" + }, + { + "column": "id2", + "name": "t1_id2_vdx" + } + ] + }, + "t1_id2_idx": { + "column_vindexes": [ + { + "column": "id2", + "name": "hash" + } + ] + }, + "t2": { + "column_vindexes": [ + { + "column": "id3", + "name": "hash" + }, + { + "column": "id4", + "name": "t2_id4_idx" + } + ] + }, + "t2_id4_idx": { + "column_vindexes": [ + { + "column": "id4", + "name": "hash" + } + ] + }, + "vstream_test": { + "column_vindexes": [ + { + "column": "id", + "name": "hash" + } + ] + }, + "aggr_test": { + "column_vindexes": [ + { + "column": "id", + "name": "hash" + } + ], + "columns": [ + { + "name": "val1", + "type": "VARCHAR" + } + ] + } + } +}` +) + +func TestMain(m *testing.M) { + flag.Parse() + + exitCode := func() int { + clusterInstance = &cluster.LocalProcessCluster{Cell: Cell, Hostname: "localhost"} + defer clusterInstance.Teardown() + + // Start topo server + err := clusterInstance.StartTopo() + if err != nil { + return 1 + } + + // Start keyspace + keyspace := &cluster.Keyspace{ + Name: KeyspaceName, + SchemaSQL: SchemaSQL, + VSchema: VSchema, + } + err = clusterInstance.StartKeyspace(*keyspace, []string{"-80", "80-"}, 1, true) + if err != nil { + return 1 + } + + // Start vtgate + err = clusterInstance.StartVtgate() + if err != nil { + return 1 + } + vtParams = mysql.ConnParams{ + Host: clusterInstance.Hostname, + Port: clusterInstance.VtgateMySQLPort, + } + return m.Run() + }() + os.Exit(exitCode) +} diff --git a/go/test/endtoend/vtgate/sequence/seq_test.go b/go/test/endtoend/vtgate/sequence/seq_test.go new file mode 100644 index 00000000000..396bfb0ee11 --- /dev/null +++ b/go/test/endtoend/vtgate/sequence/seq_test.go @@ -0,0 +1,174 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package sequence + +import ( + "context" + "flag" + "fmt" + "os" + "strings" + "testing" + + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/test/endtoend/cluster" +) + +var ( + clusterInstance *cluster.LocalProcessCluster + keyspaceName = "ks" + cell = "zone1" + hostname = "localhost" + sqlSchema = ` + create table sequence_test( + id bigint, + val varchar(16), + primary key(id) + )Engine=InnoDB; + + create table sequence_test_seq ( + id int default 0, + next_id bigint default null, + cache bigint default null, + primary key(id) + ) comment 'vitess_sequence' Engine=InnoDB; + ` + + vSchema = ` + { + "sharded":false, + "vindexes": { + "hash_index": { + "type": "hash" + } + }, + "tables": { + "sequence_test":{ + "auto_increment":{ + "column" : "id", + "sequence" : "sequence_test_seq" + }, + "column_vindexes": [ + { + "column": "id", + "name": "hash_index" + } + ] + }, + "sequence_test_seq": { + "type": "sequence" + } + } + } + ` +) + +func TestMain(m *testing.M) { + flag.Parse() + + exitCode := func() int { + clusterInstance = &cluster.LocalProcessCluster{Cell: cell, Hostname: hostname} + defer clusterInstance.Teardown() + + // Start topo server + if err := clusterInstance.StartTopo(); err != nil { + return 1 + } + + // Start keyspace + keyspace := &cluster.Keyspace{ + Name: keyspaceName, + SchemaSQL: sqlSchema, + VSchema: vSchema, + } + if err := clusterInstance.StartUnshardedKeyspace(*keyspace, 1, false); err != nil { + return 1 + } + + // Start vtgate + if err := clusterInstance.StartVtgate(); err != nil { + return 1 + } + + return m.Run() + }() + os.Exit(exitCode) +} + +func exec(t *testing.T, conn *mysql.Conn, query string) *sqltypes.Result { + t.Helper() + qr, err := conn.ExecuteFetch(query, 1000, true) + if err != nil { + t.Fatal(err) + } + return qr +} + +func TestSeq(t *testing.T) { + ctx := context.Background() + vtParams := mysql.ConnParams{ + Host: "localhost", + Port: clusterInstance.VtgateMySQLPort, + } + conn, err := mysql.Connect(ctx, &vtParams) + if err != nil { + t.Fatal(err) + } + defer conn.Close() + + //Initialize seq table + exec(t, conn, "insert into sequence_test_seq(id, next_id, cache) values(0,1,10)") + + //Insert 4 values in the main table + exec(t, conn, "insert into sequence_test(val) values('a'), ('b') ,('c'), ('d')") + + // Test select calls to main table and verify expected id. + qr := exec(t, conn, "select id, val from sequence_test where id=4") + if got, want := fmt.Sprintf("%v", qr.Rows), `[[INT64(4) VARCHAR("d")]]`; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + + // Test next available seq id from cache + qr = exec(t, conn, "select next 1 values from sequence_test_seq") + if got, want := fmt.Sprintf("%v", qr.Rows), `[[INT64(5)]]`; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + + //Test next_id from seq table which should be the increased by cache value(id+cache) + qr = exec(t, conn, "select next_id from sequence_test_seq") + if got, want := fmt.Sprintf("%v", qr.Rows), `[[INT64(11)]]`; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + + // Test insert with no auto-inc + exec(t, conn, "insert into sequence_test(id, val) values(6, 'f')") + qr = exec(t, conn, "select * from sequence_test") + if got, want := fmt.Sprintf("%v", qr.Rows), `[[INT64(1) VARCHAR("a")] [INT64(2) VARCHAR("b")] [INT64(3) VARCHAR("c")] [INT64(4) VARCHAR("d")] [INT64(6) VARCHAR("f")]]`; got != want { + t.Errorf("select:\n%v want\n%v", got, want) + } + + //Next insert will fail as we have corrupted the sequence + exec(t, conn, "begin") + _, err = conn.ExecuteFetch("insert into sequence_test(val) values('g')", 1000, false) + exec(t, conn, "rollback") + want := "Duplicate entry" + if err == nil || !strings.Contains(err.Error(), want) { + t.Errorf("wrong insert: %v, must contain %s", err, want) + } + +} From ee6a9399e9fdaab54ddcb7bc98b8628ac865d17c Mon Sep 17 00:00:00 2001 From: Arindam Nayak Date: Thu, 24 Oct 2019 15:22:19 +0530 Subject: [PATCH 02/15] added etcdctl as link Signed-off-by: Arindam Nayak --- bootstrap.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/bootstrap.sh b/bootstrap.sh index 6c9ef84988f..c93d89a2257 100755 --- a/bootstrap.sh +++ b/bootstrap.sh @@ -214,6 +214,7 @@ function install_etcd() { fi rm "$file" ln -snf "$dist/etcd-${version}-${platform}-amd64/etcd" "$VTROOT/bin/etcd" + ln -snf "$dist/etcd-${version}-${platform}-amd64/etcdctl" "$VTROOT/bin/etcdctl" } install_dep "etcd" "v3.3.10" "$VTROOT/dist/etcd" install_etcd From fa27ea88a0afceef7ab96d9b10834acc57c28e12 Mon Sep 17 00:00:00 2001 From: Arindam Nayak Date: Thu, 24 Oct 2019 15:46:39 +0530 Subject: [PATCH 03/15] revert bootstrap.sh change Signed-off-by: Arindam Nayak --- bootstrap.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/bootstrap.sh b/bootstrap.sh index eda8c7ddedd..3e3846eed19 100755 --- a/bootstrap.sh +++ b/bootstrap.sh @@ -226,7 +226,6 @@ function install_etcd() { fi rm "$file" ln -snf "$dist/etcd-${version}-${platform}-${target}/etcd" "$VTROOT/bin/etcd" - ln -snf "$dist/etcd-${version}-${platform}-${target}/etcdctl" "$VTROOT/bin/etcdctl" } install_dep "etcd" "v3.3.10" "$VTROOT/dist/etcd" install_etcd From f5b3737f44278417eeb764cec5131f20d15c03d2 Mon Sep 17 00:00:00 2001 From: Arindam Nayak Date: Thu, 24 Oct 2019 17:35:27 +0530 Subject: [PATCH 04/15] Added alternative for etcdctl Signed-off-by: Arindam Nayak --- go/test/endtoend/cluster/etcd_process.go | 22 +++++++++++++++------- 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/go/test/endtoend/cluster/etcd_process.go b/go/test/endtoend/cluster/etcd_process.go index b4b95c53d18..b5c287946f9 100644 --- a/go/test/endtoend/cluster/etcd_process.go +++ b/go/test/endtoend/cluster/etcd_process.go @@ -130,13 +130,21 @@ func (etcd *EtcdProcess) removeTopoDirectories(Cell string) { } // ManageTopoDir creates global and zone in etcd2 -func (etcd *EtcdProcess) ManageTopoDir(command string, directory string) error { - tmpProcess := exec.Command( - "etcdctl", - "--endpoints", etcd.ListenClientURL, - command, directory, - ) - return tmpProcess.Run() +func (etcd *EtcdProcess) ManageTopoDir(command string, directory string) (err error) { + url := etcd.VerifyURL + directory + payload := strings.NewReader(`{"dir":"true"}`) + if command == "mkdir" { + req, _ := http.NewRequest("PUT", url, payload) + req.Header.Add("content-type", "application/json") + _, err = http.DefaultClient.Do(req) + return err + } else if command == "rmdir" { + req, _ := http.NewRequest("DELETE", url+"?dir=true", payload) + _, err = http.DefaultClient.Do(req) + return err + } else { + return nil + } } // EtcdProcessInstance returns a EtcdProcess handle for a etcd sevice, From 0bd72fe6533c120fc2ee180bd6a9be18708edfa2 Mon Sep 17 00:00:00 2001 From: Arindam Nayak Date: Thu, 24 Oct 2019 17:35:27 +0530 Subject: [PATCH 05/15] Added alternative for etcdctl Signed-off-by: Arindam Nayak --- go/test/endtoend/cluster/etcd_process.go | 22 +++++++++++++++------- 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/go/test/endtoend/cluster/etcd_process.go b/go/test/endtoend/cluster/etcd_process.go index b4b95c53d18..b5c287946f9 100644 --- a/go/test/endtoend/cluster/etcd_process.go +++ b/go/test/endtoend/cluster/etcd_process.go @@ -130,13 +130,21 @@ func (etcd *EtcdProcess) removeTopoDirectories(Cell string) { } // ManageTopoDir creates global and zone in etcd2 -func (etcd *EtcdProcess) ManageTopoDir(command string, directory string) error { - tmpProcess := exec.Command( - "etcdctl", - "--endpoints", etcd.ListenClientURL, - command, directory, - ) - return tmpProcess.Run() +func (etcd *EtcdProcess) ManageTopoDir(command string, directory string) (err error) { + url := etcd.VerifyURL + directory + payload := strings.NewReader(`{"dir":"true"}`) + if command == "mkdir" { + req, _ := http.NewRequest("PUT", url, payload) + req.Header.Add("content-type", "application/json") + _, err = http.DefaultClient.Do(req) + return err + } else if command == "rmdir" { + req, _ := http.NewRequest("DELETE", url+"?dir=true", payload) + _, err = http.DefaultClient.Do(req) + return err + } else { + return nil + } } // EtcdProcessInstance returns a EtcdProcess handle for a etcd sevice, From ef7139d821457fde8cfd3cd8b0faf8f96b727c14 Mon Sep 17 00:00:00 2001 From: Arindam Nayak Date: Fri, 25 Oct 2019 17:25:09 +0530 Subject: [PATCH 06/15] externalize etcd peer port and tmp directory Signed-off-by: Arindam Nayak --- go/test/endtoend/cluster/cluster_process.go | 23 ++++++++------ go/test/endtoend/cluster/etcd_process.go | 28 ++++++++++------- go/test/endtoend/cluster/mysqlctl_process.go | 8 ++--- .../endtoend/cluster/vtctlclient_process.go | 8 ++--- go/test/endtoend/cluster/vtctld_process.go | 27 +++++++++-------- go/test/endtoend/cluster/vtgate_process.go | 22 +++++++------- go/test/endtoend/cluster/vttablet_process.go | 30 +++++++++---------- tools/e2e_test_runner.sh | 23 ++++++++++---- 8 files changed, 96 insertions(+), 73 deletions(-) diff --git a/go/test/endtoend/cluster/cluster_process.go b/go/test/endtoend/cluster/cluster_process.go index 54892d3be0b..cfc49232086 100644 --- a/go/test/endtoend/cluster/cluster_process.go +++ b/go/test/endtoend/cluster/cluster_process.go @@ -19,6 +19,8 @@ package cluster import ( "fmt" "math/rand" + "os" + "path" "vitess.io/vitess/go/vt/log" ) @@ -33,6 +35,7 @@ type LocalProcessCluster struct { BaseTabletUID int Hostname string TopoPort int + TmpDirectory string VtgateMySQLPort int VtctldHTTPPort int @@ -82,7 +85,8 @@ func (cluster *LocalProcessCluster) StartTopo() (err error) { cluster.Cell = DefaultCell } cluster.TopoPort = cluster.GetAndReservePort() - cluster.topoProcess = *EtcdProcessInstance(cluster.TopoPort, cluster.Hostname) + cluster.TmpDirectory = path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("/tmp_%d", cluster.GetAndReservePort())) + cluster.topoProcess = *EtcdProcessInstance(cluster.TopoPort, cluster.GetAndReservePort(), cluster.Hostname, "global") log.Info(fmt.Sprintf("Starting etcd server on port : %d", cluster.TopoPort)) if err = cluster.topoProcess.Setup(); err != nil { log.Error(err.Error()) @@ -107,7 +111,7 @@ func (cluster *LocalProcessCluster) StartTopo() (err error) { return } - cluster.vtctldProcess = *VtctldProcessInstance(cluster.GetAndReservePort(), cluster.GetAndReservePort(), cluster.topoProcess.Port, cluster.Hostname) + cluster.vtctldProcess = *VtctldProcessInstance(cluster.GetAndReservePort(), cluster.GetAndReservePort(), cluster.topoProcess.Port, cluster.Hostname, cluster.TmpDirectory) log.Info(fmt.Sprintf("Starting vtctld server on port : %d", cluster.vtctldProcess.Port)) cluster.VtctldHTTPPort = cluster.vtctldProcess.Port if err = cluster.vtctldProcess.Setup(cluster.Cell); err != nil { @@ -115,7 +119,7 @@ func (cluster *LocalProcessCluster) StartTopo() (err error) { return } - cluster.VtctlclientProcess = *VtctlClientProcessInstance("localhost", cluster.vtctldProcess.GrpcPort) + cluster.VtctlclientProcess = *VtctlClientProcessInstance("localhost", cluster.vtctldProcess.GrpcPort, cluster.TmpDirectory) return } @@ -157,7 +161,7 @@ func (cluster *LocalProcessCluster) StartKeyspace(keyspace Keyspace, shardNames } // Start Mysqlctl process log.Info(fmt.Sprintf("Starting mysqlctl for table uid %d, mysql port %d", tablet.TabletUID, tablet.MySQLPort)) - tablet.mysqlctlProcess = *MysqlCtlProcessInstance(tablet.TabletUID, tablet.MySQLPort) + tablet.mysqlctlProcess = *MysqlCtlProcessInstance(tablet.TabletUID, tablet.MySQLPort, cluster.TmpDirectory) if err = tablet.mysqlctlProcess.Start(); err != nil { log.Error(err.Error()) return @@ -169,12 +173,12 @@ func (cluster *LocalProcessCluster) StartKeyspace(keyspace Keyspace, shardNames tablet.TabletUID, cluster.Cell, shardName, - cluster.Hostname, keyspace.Name, cluster.vtctldProcess.Port, tablet.Type, cluster.topoProcess.Port, - cluster.Hostname) + cluster.Hostname, + cluster.TmpDirectory) log.Info(fmt.Sprintf("Starting vttablet for tablet uid %d, grpc port %d", tablet.TabletUID, tablet.GrpcPort)) if err = tablet.vttabletProcess.Setup(); err != nil { @@ -224,9 +228,10 @@ func (cluster *LocalProcessCluster) StartVtgate() (err error) { cluster.VtgateMySQLPort, cluster.Cell, cluster.Cell, - cluster.Hostname, "MASTER,REPLICA", + "MASTER,REPLICA", cluster.topoProcess.Port, - cluster.Hostname) + cluster.Hostname, + cluster.TmpDirectory) log.Info(fmt.Sprintf("Vtgate started, connect to mysql using : mysql -h 127.0.0.1 -P %d", cluster.VtgateMySQLPort)) return cluster.VtgateProcess.Setup() @@ -279,7 +284,7 @@ func (cluster *LocalProcessCluster) GetAndReservePort() int { // GetAndReserveTabletUID gives tablet uid func (cluster *LocalProcessCluster) GetAndReserveTabletUID() int { if cluster.BaseTabletUID == 0 { - cluster.BaseTabletUID = getRandomNumber(100, 0) + cluster.BaseTabletUID = getRandomNumber(10000, 0) } cluster.BaseTabletUID = cluster.BaseTabletUID + 1 return cluster.BaseTabletUID diff --git a/go/test/endtoend/cluster/etcd_process.go b/go/test/endtoend/cluster/etcd_process.go index b5c287946f9..6367112c33e 100644 --- a/go/test/endtoend/cluster/etcd_process.go +++ b/go/test/endtoend/cluster/etcd_process.go @@ -38,8 +38,10 @@ type EtcdProcess struct { ListenClientURL string AdvertiseClientURL string Port int + PeerPort int Host string VerifyURL string + PeerURL string proc *exec.Cmd exit chan error @@ -50,9 +52,13 @@ type EtcdProcess struct { func (etcd *EtcdProcess) Setup() (err error) { etcd.proc = exec.Command( etcd.Binary, + "--name", etcd.Name, "--data-dir", etcd.DataDirectory, "--listen-client-urls", etcd.ListenClientURL, "--advertise-client-urls", etcd.AdvertiseClientURL, + "--initial-advertise-peer-urls", etcd.PeerURL, + "--listen-peer-urls", etcd.PeerURL, + "--initial-cluster", fmt.Sprintf("%s=%s", etcd.Name, etcd.PeerURL), ) etcd.proc.Stderr = os.Stderr @@ -79,13 +85,13 @@ func (etcd *EtcdProcess) Setup() (err error) { } select { case err := <-etcd.exit: - return fmt.Errorf("process '%s' exited prematurely (err: %s)", etcd.Name, err) + return fmt.Errorf("process '%s' exited prematurely (err: %s)", etcd.Binary, err) default: time.Sleep(300 * time.Millisecond) } } - return fmt.Errorf("process '%s' timed out after 60s (err: %s)", etcd.Name, <-etcd.exit) + return fmt.Errorf("process '%s' timed out after 60s (err: %s)", etcd.Binary, <-etcd.exit) } // TearDown shutdowns the running mysqld service @@ -97,8 +103,8 @@ func (etcd *EtcdProcess) TearDown(Cell string) error { etcd.removeTopoDirectories(Cell) // Attempt graceful shutdown with SIGTERM first - etcd.proc.Process.Signal(syscall.SIGTERM) - os.RemoveAll(path.Join(os.Getenv("VTDATAROOT"), "etcd")) + _ = etcd.proc.Process.Signal(syscall.SIGTERM) + _ = os.RemoveAll(etcd.DataDirectory) select { case err := <-etcd.exit: etcd.proc = nil @@ -150,17 +156,19 @@ func (etcd *EtcdProcess) ManageTopoDir(command string, directory string) (err er // EtcdProcessInstance returns a EtcdProcess handle for a etcd sevice, // configured with the given Config. // The process must be manually started by calling setup() -func EtcdProcessInstance(port int, hostname string) *EtcdProcess { +func EtcdProcessInstance(port int, peerPort int, hostname string, name string) *EtcdProcess { etcd := &EtcdProcess{ - Name: "etcd", - Binary: "etcd", - Port: port, - Host: hostname, + Name: name, + Binary: "etcd", + Port: port, + Host: hostname, + PeerPort: peerPort, } etcd.AdvertiseClientURL = fmt.Sprintf("http://%s:%d", etcd.Host, etcd.Port) etcd.ListenClientURL = fmt.Sprintf("http://%s:%d", etcd.Host, etcd.Port) - etcd.DataDirectory = path.Join(os.Getenv("VTDATAROOT"), "etcd") + etcd.DataDirectory = path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("%s_%d", "etcd", port)) etcd.VerifyURL = fmt.Sprintf("http://%s:%d/v2/keys", etcd.Host, etcd.Port) + etcd.PeerURL = fmt.Sprintf("http://%s:%d", hostname, peerPort) return etcd } diff --git a/go/test/endtoend/cluster/mysqlctl_process.go b/go/test/endtoend/cluster/mysqlctl_process.go index 084d21c0837..14efd76774c 100644 --- a/go/test/endtoend/cluster/mysqlctl_process.go +++ b/go/test/endtoend/cluster/mysqlctl_process.go @@ -72,14 +72,14 @@ func (mysqlctl *MysqlctlProcess) Stop() (err error) { // MysqlCtlProcessInstance returns a Mysqlctl handle for mysqlctl process // configured with the given Config. -func MysqlCtlProcessInstance(TabletUID int, MySQLPort int) *MysqlctlProcess { +func MysqlCtlProcessInstance(tabletUID int, mySQLPort int, tmpDirectory string) *MysqlctlProcess { mysqlctl := &MysqlctlProcess{ Name: "mysqlctl", Binary: "mysqlctl", - LogDirectory: path.Join(os.Getenv("VTDATAROOT"), "/tmp"), + LogDirectory: tmpDirectory, InitDBFile: path.Join(os.Getenv("VTROOT"), "/config/init_db.sql"), } - mysqlctl.MySQLPort = MySQLPort - mysqlctl.TabletUID = TabletUID + mysqlctl.MySQLPort = mySQLPort + mysqlctl.TabletUID = tabletUID return mysqlctl } diff --git a/go/test/endtoend/cluster/vtctlclient_process.go b/go/test/endtoend/cluster/vtctlclient_process.go index 30184d74e90..982f0fd6a70 100644 --- a/go/test/endtoend/cluster/vtctlclient_process.go +++ b/go/test/endtoend/cluster/vtctlclient_process.go @@ -18,9 +18,7 @@ package cluster import ( "fmt" - "os" "os/exec" - "path" "strings" "vitess.io/vitess/go/vt/log" @@ -89,12 +87,12 @@ func (vtctlclient *VtctlClientProcess) ExecuteCommandWithOutput(args ...string) // VtctlClientProcessInstance returns a VtctlProcess handle for vtctlclient process // configured with the given Config. -func VtctlClientProcessInstance(Hostname string, GrpcPort int) *VtctlClientProcess { +func VtctlClientProcessInstance(hostname string, grpcPort int, tmpDirectory string) *VtctlClientProcess { vtctlclient := &VtctlClientProcess{ Name: "vtctlclient", Binary: "vtctlclient", - Server: fmt.Sprintf("%s:%d", Hostname, GrpcPort), - TempDirectory: path.Join(os.Getenv("VTDATAROOT"), "/tmp"), + Server: fmt.Sprintf("%s:%d", hostname, grpcPort), + TempDirectory: tmpDirectory, } return vtctlclient } diff --git a/go/test/endtoend/cluster/vtctld_process.go b/go/test/endtoend/cluster/vtctld_process.go index f4f1c26c47b..6909aa4cf36 100644 --- a/go/test/endtoend/cluster/vtctld_process.go +++ b/go/test/endtoend/cluster/vtctld_process.go @@ -53,14 +53,8 @@ type VtctldProcess struct { // Setup starts vtctld process with required arguements func (vtctld *VtctldProcess) Setup(Cell string) (err error) { - err = os.Mkdir(path.Join(vtctld.Directory, "tmp"), 0700) - if err != nil { - return - } - err = os.Mkdir(path.Join(vtctld.Directory, "backups"), 0700) - if err != nil { - return - } + _ = createDirectory(vtctld.LogDir, 0700) + _ = createDirectory(path.Join(vtctld.Directory, "backups"), 0700) vtctld.proc = exec.Command( vtctld.Binary, "-enable_queries", @@ -114,6 +108,13 @@ func (vtctld *VtctldProcess) Setup(Cell string) (err error) { return fmt.Errorf("process '%s' timed out after 60s (err: %s)", vtctld.Name, <-vtctld.exit) } +func createDirectory(dirName string, mode os.FileMode) error { + if _, err := os.Stat(dirName); os.IsNotExist(err) { + return os.Mkdir(dirName, mode) + } + return nil +} + // IsHealthy function checks if vtctld process is up and running func (vtctld *VtctldProcess) IsHealthy() bool { resp, err := http.Get(vtctld.VerifyURL) @@ -132,8 +133,8 @@ func (vtctld *VtctldProcess) TearDown() error { return nil } - os.RemoveAll(path.Join(vtctld.Directory, "tmp")) - os.RemoveAll(path.Join(vtctld.Directory, "backups")) + os.RemoveAll(vtctld.LogDir) + //os.RemoveAll(path.Join(vtctld.Directory, "backups")) // Attempt graceful shutdown with SIGTERM first vtctld.proc.Process.Signal(syscall.SIGTERM) @@ -153,7 +154,7 @@ func (vtctld *VtctldProcess) TearDown() error { // VtctldProcessInstance returns a VtctlProcess handle for vtctl process // configured with the given Config. // The process must be manually started by calling setup() -func VtctldProcessInstance(httpPort int, grpcPort int, topoPort int, hostname string) *VtctldProcess { +func VtctldProcessInstance(httpPort int, grpcPort int, topoPort int, hostname string, tmpDirectory string) *VtctldProcess { vtctl := VtctlProcessInstance(topoPort, hostname) vtctld := &VtctldProcess{ Name: "vtctld", @@ -164,10 +165,10 @@ func VtctldProcessInstance(httpPort int, grpcPort int, topoPort int, hostname st ServiceMap: "grpc-vtctl", BackupStorageImplementation: "file", FileBackupStorageRoot: path.Join(os.Getenv("VTDATAROOT"), "/backups"), - LogDir: path.Join(os.Getenv("VTDATAROOT"), "/tmp"), + LogDir: tmpDirectory, Port: httpPort, GrpcPort: grpcPort, - PidFile: path.Join(os.Getenv("VTDATAROOT"), "/tmp", "vtctld.pid"), + PidFile: path.Join(tmpDirectory, "vtctld.pid"), Directory: os.Getenv("VTDATAROOT"), } vtctld.VerifyURL = fmt.Sprintf("http://localhost:%d", vtctld.Port) diff --git a/go/test/endtoend/cluster/vtgate_process.go b/go/test/endtoend/cluster/vtgate_process.go index 44fca93abda..0eb7b0d5571 100644 --- a/go/test/endtoend/cluster/vtgate_process.go +++ b/go/test/endtoend/cluster/vtgate_process.go @@ -159,29 +159,29 @@ func (vtgate *VtgateProcess) TearDown() error { // VtgateProcessInstance returns a Vtgate handle for vtgate process // configured with the given Config. // The process must be manually started by calling setup() -func VtgateProcessInstance(Port int, GrpcPort int, MySQLServerPort int, Cell string, CellsToWatch string, Hostname string, TabletTypesToWait string, topoPort int, hostname string) *VtgateProcess { +func VtgateProcessInstance(port int, grpcPort int, mySQLServerPort int, cell string, cellsToWatch string, tabletTypesToWait string, topoPort int, hostname string, tmpDirectory string) *VtgateProcess { vtctl := VtctlProcessInstance(topoPort, hostname) vtgate := &VtgateProcess{ Name: "vtgate", Binary: "vtgate", - FileToLogQueries: path.Join(os.Getenv("VTDATAROOT"), "/tmp/vtgate_querylog.txt"), + FileToLogQueries: path.Join(tmpDirectory, "/vtgate_querylog.txt"), Directory: os.Getenv("VTDATAROOT"), ServiceMap: "grpc-vtgateservice", - LogDir: path.Join(os.Getenv("VTDATAROOT"), "/tmp"), - Port: Port, - GrpcPort: GrpcPort, - MySQLServerPort: MySQLServerPort, + LogDir: tmpDirectory, + Port: port, + GrpcPort: grpcPort, + MySQLServerPort: mySQLServerPort, MySQLServerSocketPath: "/tmp/mysql.sock", - Cell: Cell, - CellsToWatch: CellsToWatch, - TabletTypesToWait: TabletTypesToWait, + Cell: cell, + CellsToWatch: cellsToWatch, + TabletTypesToWait: tabletTypesToWait, GatewayImplementation: "discoverygateway", CommonArg: *vtctl, - PidFile: path.Join(os.Getenv("VTDATAROOT"), "/tmp/vtgate.pid"), + PidFile: path.Join(tmpDirectory, "/vtgate.pid"), MySQLAuthServerImpl: "none", } - vtgate.VerifyURL = fmt.Sprintf("http://%s:%d/debug/vars", Hostname, Port) + vtgate.VerifyURL = fmt.Sprintf("http://%s:%d/debug/vars", hostname, port) return vtgate } diff --git a/go/test/endtoend/cluster/vttablet_process.go b/go/test/endtoend/cluster/vttablet_process.go index 65626c96559..1a05b0e621c 100644 --- a/go/test/endtoend/cluster/vttablet_process.go +++ b/go/test/endtoend/cluster/vttablet_process.go @@ -168,34 +168,34 @@ func (vttablet *VttabletProcess) TearDown() error { // VttabletProcessInstance returns a VttabletProcess handle for vttablet process // configured with the given Config. // The process must be manually started by calling setup() -func VttabletProcessInstance(Port int, GrpcPort int, TabletUID int, Cell string, Shard string, Hostname string, Keyspace string, VtctldPort int, TabletType string, topoPort int, hostname string) *VttabletProcess { +func VttabletProcessInstance(port int, grpcPort int, tabletUID int, cell string, shard string, keyspace string, vtctldPort int, tabletType string, topoPort int, hostname string, tmpDirectory string) *VttabletProcess { vtctl := VtctlProcessInstance(topoPort, hostname) vttablet := &VttabletProcess{ Name: "vttablet", Binary: "vttablet", - FileToLogQueries: path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("/tmp/vt_%010d/vttable.pid", TabletUID)), - Directory: path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("/vt_%010d", TabletUID)), - TabletPath: fmt.Sprintf("%s-%010d", Cell, TabletUID), + FileToLogQueries: path.Join(tmpDirectory, fmt.Sprintf("/vt_%010d/vttable.pid", tabletUID)), + Directory: path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("/vt_%010d", tabletUID)), + TabletPath: fmt.Sprintf("%s-%010d", cell, tabletUID), ServiceMap: "grpc-queryservice,grpc-tabletmanager,grpc-updatestream", - LogDir: path.Join(os.Getenv("VTDATAROOT"), "/tmp"), - Shard: Shard, - TabletHostname: Hostname, - Keyspace: Keyspace, + LogDir: tmpDirectory, + Shard: shard, + TabletHostname: hostname, + Keyspace: keyspace, TabletType: "replica", CommonArg: *vtctl, HealthCheckInterval: 5, BackupStorageImplementation: "file", FileBackupStorageRoot: path.Join(os.Getenv("VTDATAROOT"), "/backups"), - Port: Port, - GrpcPort: GrpcPort, - PidFile: path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("/vt_%010d/vttable.pid", TabletUID)), - VtctldAddress: fmt.Sprintf("http://%s:%d", Hostname, VtctldPort), + Port: port, + GrpcPort: grpcPort, + PidFile: path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("/vt_%010d/vttable.pid", tabletUID)), + VtctldAddress: fmt.Sprintf("http://%s:%d", hostname, vtctldPort), } - if TabletType == "rdonly" { - vttablet.TabletType = TabletType + if tabletType == "rdonly" { + vttablet.TabletType = tabletType } - vttablet.VerifyURL = fmt.Sprintf("http://%s:%d/debug/vars", Hostname, Port) + vttablet.VerifyURL = fmt.Sprintf("http://%s:%d/debug/vars", hostname, port) return vttablet } diff --git a/tools/e2e_test_runner.sh b/tools/e2e_test_runner.sh index 4f7822d18ef..2ad3666e433 100755 --- a/tools/e2e_test_runner.sh +++ b/tools/e2e_test_runner.sh @@ -1,13 +1,13 @@ #!/bin/bash # Copyright 2019 The Vitess Authors. -# +# # Licensed under the Apache License, Version 2.0 (the "License"); # you may not use this file except in compliance with the License. # You may obtain a copy of the License at -# +# # http://www.apache.org/licenses/LICENSE-2.0 -# +# # Unless required by applicable law or agreed to in writing, software # distributed under the License is distributed on an "AS IS" BASIS, # WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. @@ -38,11 +38,12 @@ fi packages_with_tests=$(go list -f '{{if len .TestGoFiles}}{{.ImportPath}} {{join .TestGoFiles " "}}{{end}}' ./go/.../endtoend/... | sort) # Flaky tests have the suffix "_flaky_test.go". -all_except_flaky_tests=$(echo "$packages_with_tests" | grep -vE ".+ .+_flaky_test\.go" | cut -d" " -f1) -flaky_tests=$(echo "$packages_with_tests" | grep -E ".+ .+_flaky_test\.go" | cut -d" " -f1) +all_except_flaky_and_cluster_tests=$(echo "$packages_with_tests" | grep -vE ".+ .+_flaky_test\.go" | grep -vE "go/test/endtoend" | cut -d" " -f1) +flaky_tests=$(echo "$packages_with_tests" | grep -E ".+ .+_flaky_test\.go" | grep -vE "go/test/endtoend" | cut -d" " -f1) +cluster_tests=$(echo "$packages_with_tests" | grep -E "go/test/endtoend" | cut -d" " -f1) # Run non-flaky tests. -echo "$all_except_flaky_tests" | xargs go test $VT_GO_PARALLEL +echo "$all_except_flaky_and_cluster_tests" | xargs go test $VT_GO_PARALLEL if [ $? -ne 0 ]; then echo "ERROR: Go unit tests failed. See above for errors." echo @@ -51,6 +52,16 @@ if [ $? -ne 0 ]; then exit 1 fi +# Run cluster test sequentially +echo "$cluster_tests" | xargs go test -v -p=1 +if [ $? -ne 0 ]; then + echo "ERROR: Go cluster tests failed. See above for errors." + echo + echo "This should NOT happen. Did you introduce a flaky unit test?" + echo "If so, please rename it to the suffix _flaky_test.go." + exit 1 +fi + # Run flaky tests sequentially. Retry when necessary. for pkg in $flaky_tests; do max_attempts=3 From f3969f8835ff6896ba7ac4c4c2bb3213591af1be Mon Sep 17 00:00:00 2001 From: Arindam Nayak Date: Fri, 25 Oct 2019 19:33:55 +0530 Subject: [PATCH 07/15] separated cluster test to shard 2 Signed-off-by: Arindam Nayak --- test/config.json | 12 ++++++++++++ tools/e2e_test_cluster.sh | 34 ++++++++++++++++++++++++++++++++++ tools/e2e_test_runner.sh | 11 ----------- 3 files changed, 46 insertions(+), 11 deletions(-) create mode 100644 tools/e2e_test_cluster.sh diff --git a/test/config.json b/test/config.json index c379855a537..37cf5bb04f0 100644 --- a/test/config.json +++ b/test/config.json @@ -411,6 +411,18 @@ "RetryMax": 0, "Tags": [] }, + "cluster_endtoend": { + "File": "", + "Args": [], + "Command": [ + "make", + "tools/e2e_test_cluster.sh" + ], + "Manual": false, + "Shard": 2, + "RetryMax": 0, + "Tags": [] + }, "e2e_race": { "File": "", "Args": [], diff --git a/tools/e2e_test_cluster.sh b/tools/e2e_test_cluster.sh new file mode 100644 index 00000000000..414d20d44e2 --- /dev/null +++ b/tools/e2e_test_cluster.sh @@ -0,0 +1,34 @@ +#!/bin/bash + +# Copyright 2019 The Vitess Authors. +# +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# These test uses excutables and launch them as process +# After that all tests run, here we are testing those + +# All Go packages with test files. +# Output per line: * +packages_with_tests=$(go list -f '{{if len .TestGoFiles}}{{.ImportPath}} {{join .TestGoFiles " "}}{{end}}' ./go/.../endtoend/... | sort) + +cluster_tests=$(echo "$packages_with_tests" | grep -E "go/test/endtoend" | cut -d" " -f1) + +# Run cluster test sequentially +echo "$cluster_tests" | xargs go test -v -p=1 +if [ $? -ne 0 ]; then + echo "ERROR: Go cluster tests failed. See above for errors." + echo + echo "This should NOT happen. Did you introduce a flaky unit test?" + echo "If so, please rename it to the suffix _flaky_test.go." + exit 1 +fi diff --git a/tools/e2e_test_runner.sh b/tools/e2e_test_runner.sh index 2ad3666e433..e2b9da256ad 100755 --- a/tools/e2e_test_runner.sh +++ b/tools/e2e_test_runner.sh @@ -40,7 +40,6 @@ packages_with_tests=$(go list -f '{{if len .TestGoFiles}}{{.ImportPath}} {{join # Flaky tests have the suffix "_flaky_test.go". all_except_flaky_and_cluster_tests=$(echo "$packages_with_tests" | grep -vE ".+ .+_flaky_test\.go" | grep -vE "go/test/endtoend" | cut -d" " -f1) flaky_tests=$(echo "$packages_with_tests" | grep -E ".+ .+_flaky_test\.go" | grep -vE "go/test/endtoend" | cut -d" " -f1) -cluster_tests=$(echo "$packages_with_tests" | grep -E "go/test/endtoend" | cut -d" " -f1) # Run non-flaky tests. echo "$all_except_flaky_and_cluster_tests" | xargs go test $VT_GO_PARALLEL @@ -52,16 +51,6 @@ if [ $? -ne 0 ]; then exit 1 fi -# Run cluster test sequentially -echo "$cluster_tests" | xargs go test -v -p=1 -if [ $? -ne 0 ]; then - echo "ERROR: Go cluster tests failed. See above for errors." - echo - echo "This should NOT happen. Did you introduce a flaky unit test?" - echo "If so, please rename it to the suffix _flaky_test.go." - exit 1 -fi - # Run flaky tests sequentially. Retry when necessary. for pkg in $flaky_tests; do max_attempts=3 From c7adf4cfcbb090dbf820db3e2385311617294728 Mon Sep 17 00:00:00 2001 From: Ajeet Jain Date: Wed, 30 Oct 2019 10:20:22 +0530 Subject: [PATCH 08/15] Transaction mode and svchema e2e GO test cases * transaction mode and vschema test cases Signed-off-by: Ajeet jain * review comments implemented Signed-off-by: Ajeet jain --- go/test/endtoend/cluster/cluster_process.go | 34 ++- go/test/endtoend/cluster/vtgate_process.go | 20 +- go/test/endtoend/cluster/vttablet_process.go | 32 +-- .../vtgate/transaction/trxn_mode_test.go | 228 ++++++++++++++++++ .../endtoend/vtgate/vschema/vschema_test.go | 169 +++++++++++++ 5 files changed, 456 insertions(+), 27 deletions(-) create mode 100644 go/test/endtoend/vtgate/transaction/trxn_mode_test.go create mode 100644 go/test/endtoend/vtgate/vschema/vschema_test.go diff --git a/go/test/endtoend/cluster/cluster_process.go b/go/test/endtoend/cluster/cluster_process.go index 54892d3be0b..d66930328bc 100644 --- a/go/test/endtoend/cluster/cluster_process.go +++ b/go/test/endtoend/cluster/cluster_process.go @@ -35,6 +35,7 @@ type LocalProcessCluster struct { TopoPort int VtgateMySQLPort int + VtgateGrpcPort int VtctldHTTPPort int // standalone executable @@ -47,6 +48,12 @@ type LocalProcessCluster struct { VtgateProcess VtgateProcess nextPortForProcess int + + //Extra arguments for vtTablet + VtTabletExtraArgs []string + + //Extra arguments for vtGate + VtGateExtraArgs []string } // Keyspace : Cluster accepts keyspace to launch it @@ -174,7 +181,7 @@ func (cluster *LocalProcessCluster) StartKeyspace(keyspace Keyspace, shardNames cluster.vtctldProcess.Port, tablet.Type, cluster.topoProcess.Port, - cluster.Hostname) + cluster.VtTabletExtraArgs) log.Info(fmt.Sprintf("Starting vttablet for tablet uid %d, grpc port %d", tablet.TabletUID, tablet.GrpcPort)) if err = tablet.vttabletProcess.Setup(); err != nil { @@ -203,9 +210,11 @@ func (cluster *LocalProcessCluster) StartKeyspace(keyspace Keyspace, shardNames } //Apply VSchema - if err = cluster.VtctlclientProcess.ApplyVSchema(keyspace.Name, keyspace.VSchema); err != nil { - log.Error(err.Error()) - return + if keyspace.VSchema != "" { + if err = cluster.VtctlclientProcess.ApplyVSchema(keyspace.Name, keyspace.VSchema); err != nil { + log.Error(err.Error()) + return + } } log.Info("Done creating keyspace : " + keyspace.Name) @@ -226,12 +235,27 @@ func (cluster *LocalProcessCluster) StartVtgate() (err error) { cluster.Cell, cluster.Hostname, "MASTER,REPLICA", cluster.topoProcess.Port, - cluster.Hostname) + cluster.VtGateExtraArgs) log.Info(fmt.Sprintf("Vtgate started, connect to mysql using : mysql -h 127.0.0.1 -P %d", cluster.VtgateMySQLPort)) return cluster.VtgateProcess.Setup() } +// ReStartVtgate starts vtgate with updated configs +func (cluster *LocalProcessCluster) ReStartVtgate() (err error) { + err = cluster.VtgateProcess.TearDown() + if err != nil { + log.Error(err.Error()) + return + } + err = cluster.StartVtgate() + if err != nil { + log.Error(err.Error()) + return + } + return err +} + // Teardown brings down the cluster by invoking teardown for individual processes func (cluster *LocalProcessCluster) Teardown() (err error) { if err = cluster.VtgateProcess.TearDown(); err != nil { diff --git a/go/test/endtoend/cluster/vtgate_process.go b/go/test/endtoend/cluster/vtgate_process.go index 44fca93abda..9979baa4787 100644 --- a/go/test/endtoend/cluster/vtgate_process.go +++ b/go/test/endtoend/cluster/vtgate_process.go @@ -52,6 +52,8 @@ type VtgateProcess struct { MySQLAuthServerImpl string Directory string VerifyURL string + //Extra Args to be set before starting the vtgate process + ExtraArgs []string proc *exec.Cmd exit chan error @@ -79,6 +81,7 @@ func (vtgate *VtgateProcess) Setup() (err error) { "-mysql_auth_server_impl", vtgate.MySQLAuthServerImpl, "-pid_file", vtgate.PidFile, ) + vtgate.proc.Args = append(vtgate.proc.Args, vtgate.ExtraArgs...) vtgate.proc.Stderr = os.Stderr vtgate.proc.Stdout = os.Stdout @@ -159,7 +162,7 @@ func (vtgate *VtgateProcess) TearDown() error { // VtgateProcessInstance returns a Vtgate handle for vtgate process // configured with the given Config. // The process must be manually started by calling setup() -func VtgateProcessInstance(Port int, GrpcPort int, MySQLServerPort int, Cell string, CellsToWatch string, Hostname string, TabletTypesToWait string, topoPort int, hostname string) *VtgateProcess { +func VtgateProcessInstance(port int, grpcPort int, mySQLServerPort int, cell string, cellsToWatch string, hostname string, tabletTypesToWait string, topoPort int, extraArgs []string) *VtgateProcess { vtctl := VtctlProcessInstance(topoPort, hostname) vtgate := &VtgateProcess{ Name: "vtgate", @@ -168,20 +171,21 @@ func VtgateProcessInstance(Port int, GrpcPort int, MySQLServerPort int, Cell str Directory: os.Getenv("VTDATAROOT"), ServiceMap: "grpc-vtgateservice", LogDir: path.Join(os.Getenv("VTDATAROOT"), "/tmp"), - Port: Port, - GrpcPort: GrpcPort, - MySQLServerPort: MySQLServerPort, + Port: port, + GrpcPort: grpcPort, + MySQLServerPort: mySQLServerPort, MySQLServerSocketPath: "/tmp/mysql.sock", - Cell: Cell, - CellsToWatch: CellsToWatch, - TabletTypesToWait: TabletTypesToWait, + Cell: cell, + CellsToWatch: cellsToWatch, + TabletTypesToWait: tabletTypesToWait, GatewayImplementation: "discoverygateway", CommonArg: *vtctl, PidFile: path.Join(os.Getenv("VTDATAROOT"), "/tmp/vtgate.pid"), MySQLAuthServerImpl: "none", + ExtraArgs: extraArgs, } - vtgate.VerifyURL = fmt.Sprintf("http://%s:%d/debug/vars", Hostname, Port) + vtgate.VerifyURL = fmt.Sprintf("http://%s:%d/debug/vars", hostname, port) return vtgate } diff --git a/go/test/endtoend/cluster/vttablet_process.go b/go/test/endtoend/cluster/vttablet_process.go index 65626c96559..c795c1048a0 100644 --- a/go/test/endtoend/cluster/vttablet_process.go +++ b/go/test/endtoend/cluster/vttablet_process.go @@ -56,6 +56,8 @@ type VttabletProcess struct { VtctldAddress string Directory string VerifyURL string + //Extra Args to be set before starting the vttablet process + ExtraArgs []string proc *exec.Cmd exit chan error @@ -88,6 +90,7 @@ func (vttablet *VttabletProcess) Setup() (err error) { "-service_map", vttablet.ServiceMap, "-vtctld_addr", vttablet.VtctldAddress, ) + vttablet.proc.Args = append(vttablet.proc.Args, vttablet.ExtraArgs...) vttablet.proc.Stderr = os.Stderr vttablet.proc.Stdout = os.Stdout @@ -168,34 +171,35 @@ func (vttablet *VttabletProcess) TearDown() error { // VttabletProcessInstance returns a VttabletProcess handle for vttablet process // configured with the given Config. // The process must be manually started by calling setup() -func VttabletProcessInstance(Port int, GrpcPort int, TabletUID int, Cell string, Shard string, Hostname string, Keyspace string, VtctldPort int, TabletType string, topoPort int, hostname string) *VttabletProcess { +func VttabletProcessInstance(port int, grpcPort int, tabletUID int, cell string, shard string, hostname string, keyspace string, vtctldPort int, tabletType string, topoPort int, extraArgs []string) *VttabletProcess { vtctl := VtctlProcessInstance(topoPort, hostname) vttablet := &VttabletProcess{ Name: "vttablet", Binary: "vttablet", - FileToLogQueries: path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("/tmp/vt_%010d/vttable.pid", TabletUID)), - Directory: path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("/vt_%010d", TabletUID)), - TabletPath: fmt.Sprintf("%s-%010d", Cell, TabletUID), + FileToLogQueries: path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("/tmp/vt_%010d/vttable.pid", tabletUID)), + Directory: path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("/vt_%010d", tabletUID)), + TabletPath: fmt.Sprintf("%s-%010d", cell, tabletUID), ServiceMap: "grpc-queryservice,grpc-tabletmanager,grpc-updatestream", LogDir: path.Join(os.Getenv("VTDATAROOT"), "/tmp"), - Shard: Shard, - TabletHostname: Hostname, - Keyspace: Keyspace, + Shard: shard, + TabletHostname: hostname, + Keyspace: keyspace, TabletType: "replica", CommonArg: *vtctl, HealthCheckInterval: 5, BackupStorageImplementation: "file", FileBackupStorageRoot: path.Join(os.Getenv("VTDATAROOT"), "/backups"), - Port: Port, - GrpcPort: GrpcPort, - PidFile: path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("/vt_%010d/vttable.pid", TabletUID)), - VtctldAddress: fmt.Sprintf("http://%s:%d", Hostname, VtctldPort), + Port: port, + GrpcPort: grpcPort, + PidFile: path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("/vt_%010d/vttable.pid", tabletUID)), + VtctldAddress: fmt.Sprintf("http://%s:%d", hostname, vtctldPort), + ExtraArgs: extraArgs, } - if TabletType == "rdonly" { - vttablet.TabletType = TabletType + if tabletType == "rdonly" { + vttablet.TabletType = tabletType } - vttablet.VerifyURL = fmt.Sprintf("http://%s:%d/debug/vars", Hostname, Port) + vttablet.VerifyURL = fmt.Sprintf("http://%s:%d/debug/vars", hostname, port) return vttablet } diff --git a/go/test/endtoend/vtgate/transaction/trxn_mode_test.go b/go/test/endtoend/vtgate/transaction/trxn_mode_test.go new file mode 100644 index 00000000000..4e7d64da9c8 --- /dev/null +++ b/go/test/endtoend/vtgate/transaction/trxn_mode_test.go @@ -0,0 +1,228 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package transaction + +import ( + "context" + "flag" + "fmt" + "os" + "strings" + "testing" + + "github.com/stretchr/testify/assert" + + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/test/endtoend/cluster" +) + +var ( + clusterInstance *cluster.LocalProcessCluster + vtParams mysql.ConnParams + keyspaceName = "ks" + cell = "zone1" + hostname = "localhost" + sqlSchema = ` + create table twopc_user ( + user_id bigint, + name varchar(128), + primary key (user_id) + ) Engine=InnoDB; + + create table twopc_lookup ( + name varchar(128), + id bigint, + primary key (id) + ) Engine=InnoDB;` + + vSchema = ` + { + "sharded":true, + "vindexes": { + "hash_index": { + "type": "hash" + }, + "twopc_lookup_vdx": { + "type": "lookup_hash_unique", + "params": { + "table": "twopc_lookup", + "from": "name", + "to": "id", + "autocommit": "true" + }, + "owner": "twopc_user" + } + }, + "tables": { + "twopc_user":{ + "column_vindexes": [ + { + "column": "user_id", + "name": "hash_index" + }, + { + "column": "name", + "name": "twopc_lookup_vdx" + } + ] + }, + "twopc_lookup": { + "column_vindexes": [ + { + "column": "id", + "name": "hash_index" + } + ] + } + } + } + ` +) + +func TestMain(m *testing.M) { + flag.Parse() + + exitcode, err := func() (int, error) { + clusterInstance = &cluster.LocalProcessCluster{Cell: cell, Hostname: hostname} + defer clusterInstance.Teardown() + + // Reserve vtGate port in order to pass it to vtTablet + clusterInstance.VtgateGrpcPort = clusterInstance.GetAndReservePort() + // Set extra tablet args for twopc + clusterInstance.VtTabletExtraArgs = []string{ + "-twopc_enable", + "-twopc_coordinator_address", fmt.Sprintf("localhost:%d", clusterInstance.VtgateGrpcPort), + "-twopc_abandon_age", "3600", + } + + // Start topo server + if err := clusterInstance.StartTopo(); err != nil { + return 1, err + } + + // Start keyspace + keyspace := &cluster.Keyspace{ + Name: keyspaceName, + SchemaSQL: sqlSchema, + VSchema: vSchema, + } + if err := clusterInstance.StartKeyspace(*keyspace, []string{"-80", "80-"}, 1, false); err != nil { + return 1, err + } + + // Starting Vtgate in SINGLE transaction mode + clusterInstance.VtGateExtraArgs = []string{"-transaction_mode", "SINGLE"} + if err := clusterInstance.StartVtgate(); err != nil { + return 1, err + } + vtParams = mysql.ConnParams{ + Host: clusterInstance.Hostname, + Port: clusterInstance.VtgateMySQLPort, + } + + return m.Run(), nil + }() + if err != nil { + fmt.Printf("%v\n", err) + os.Exit(1) + } else { + os.Exit(exitcode) + } +} + +func exec(t *testing.T, conn *mysql.Conn, query string) *sqltypes.Result { + t.Helper() + qr, err := conn.ExecuteFetch(query, 1000, true) + if err != nil { + t.Fatal(err) + } + return qr +} + +// TestTransactionModes tests trasactions using twopc mode +func TestTransactionModes(t *testing.T) { + ctx := context.Background() + conn, err := mysql.Connect(ctx, &vtParams) + if err != nil { + t.Fatal(err) + } + defer conn.Close() + + // Insert targeted to multiple tables should fail as Transaction mode is SINGLE + exec(t, conn, "begin") + exec(t, conn, "insert into twopc_user(user_id, name) values(1,'john')") + _, err = conn.ExecuteFetch("insert into twopc_user(user_id, name) values(6,'vick')", 1000, false) + exec(t, conn, "rollback") + want := "multi-db transaction attempted" + if err == nil || !strings.Contains(err.Error(), want) { + t.Errorf("multi-db insert: %v, must contain %s", err, want) + } + + // Enable TWOPC transaction mode + clusterInstance.VtGateExtraArgs = []string{"-transaction_mode", "TWOPC"} + // Restart VtGate + if err = clusterInstance.ReStartVtgate(); err != nil { + t.Errorf("Fail to re-start vtgate with new config: %v", err) + } + + // Make a new mysql connection to vtGate + vtParams = mysql.ConnParams{ + Host: clusterInstance.Hostname, + Port: clusterInstance.VtgateMySQLPort, + } + conn2, err := mysql.Connect(ctx, &vtParams) + if err != nil { + t.Fatal(err) + } + defer conn2.Close() + + // Insert targeted to multiple db should PASS with TWOPC trx mode + exec(t, conn2, "begin") + exec(t, conn2, "insert into twopc_user(user_id, name) values(3,'mark')") + exec(t, conn2, "insert into twopc_user(user_id, name) values(4,'doug')") + exec(t, conn2, "insert into twopc_lookup(name, id) values('Tim',7)") + exec(t, conn2, "commit") + + // Verify the values are present + qr := exec(t, conn2, "select user_id from twopc_user where name='mark'") + got := fmt.Sprintf("%v", qr.Rows) + want = `[[INT64(3)]]` + assert.Equal(t, want, got) + + qr = exec(t, conn2, "select name from twopc_lookup where id=3") + got = fmt.Sprintf("%v", qr.Rows) + want = `[[VARCHAR("mark")]]` + assert.Equal(t, want, got) + + // DELETE from multiple tables using TWOPC transaction mode + exec(t, conn2, "begin") + exec(t, conn2, "delete from twopc_user where user_id = 3") + exec(t, conn2, "delete from twopc_lookup where id = 3") + exec(t, conn2, "commit") + + // VERIFY that values are deleted + qr = exec(t, conn2, "select user_id from twopc_user where user_id=3") + got = fmt.Sprintf("%v", qr.Rows) + want = `[]` + assert.Equal(t, want, got) + + qr = exec(t, conn2, "select name from twopc_lookup where id=3") + got = fmt.Sprintf("%v", qr.Rows) + want = `[]` + assert.Equal(t, want, got) +} diff --git a/go/test/endtoend/vtgate/vschema/vschema_test.go b/go/test/endtoend/vtgate/vschema/vschema_test.go new file mode 100644 index 00000000000..1b0a984ede1 --- /dev/null +++ b/go/test/endtoend/vtgate/vschema/vschema_test.go @@ -0,0 +1,169 @@ +/* +Copyright 2019 The Vitess Authors. + +Licensed under the Apache License, Version 2.0 (the "License"); +you may not use this file except in compliance with the License. +You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + +Unless required by applicable law or agreed to in writing, software +distributed under the License is distributed on an "AS IS" BASIS, +WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +See the License for the specific language governing permissions and +limitations under the License. +*/ + +package vschema + +import ( + "context" + "flag" + "fmt" + "os" + "testing" + + "github.com/stretchr/testify/assert" + "vitess.io/vitess/go/mysql" + "vitess.io/vitess/go/sqltypes" + "vitess.io/vitess/go/test/endtoend/cluster" +) + +var ( + clusterInstance *cluster.LocalProcessCluster + vtParams mysql.ConnParams + hostname = "localhost" + keyspaceName = "ks" + cell = "zone1" + sqlSchema = ` + create table vt_user ( + id bigint, + name varchar(64), + primary key (id) + ) Engine=InnoDB; + + create table main ( + id bigint, + val varchar(128), + primary key(id) + ) Engine=InnoDB; +` +) + +func TestMain(m *testing.M) { + flag.Parse() + + exitcode, err := func() (int, error) { + clusterInstance = &cluster.LocalProcessCluster{Cell: cell, Hostname: hostname} + defer clusterInstance.Teardown() + + // Start topo server + if err := clusterInstance.StartTopo(); err != nil { + return 1, err + } + + // List of users authorized to execute vschema ddl operations + clusterInstance.VtGateExtraArgs = []string{"-vschema_ddl_authorized_users=%"} + + // Start keyspace + keyspace := &cluster.Keyspace{ + Name: keyspaceName, + SchemaSQL: sqlSchema, + } + if err := clusterInstance.StartUnshardedKeyspace(*keyspace, 1, false); err != nil { + return 1, err + } + + // Start vtgate + if err := clusterInstance.StartVtgate(); err != nil { + return 1, err + } + vtParams = mysql.ConnParams{ + Host: clusterInstance.Hostname, + Port: clusterInstance.VtgateMySQLPort, + } + return m.Run(), nil + }() + if err != nil { + fmt.Printf("%v\n", err) + os.Exit(1) + } else { + os.Exit(exitcode) + } + +} + +func TestVSchema(t *testing.T) { + ctx := context.Background() + conn, err := mysql.Connect(ctx, &vtParams) + if err != nil { + t.Fatal(err) + } + defer conn.Close() + + // Test the empty database with no vschema + exec(t, conn, "insert into vt_user (id,name) values(1,'test1'), (2,'test2'), (3,'test3'), (4,'test4')") + + qr := exec(t, conn, "select id, name from vt_user order by id") + got := fmt.Sprintf("%v", qr.Rows) + want := `[[INT64(1) VARCHAR("test1")] [INT64(2) VARCHAR("test2")] [INT64(3) VARCHAR("test3")] [INT64(4) VARCHAR("test4")]]` + assert.Equal(t, want, got) + + qr = exec(t, conn, "delete from vt_user") + got = fmt.Sprintf("%v", qr.Rows) + want = `[]` + assert.Equal(t, want, got) + + // Test empty vschema + qr = exec(t, conn, "SHOW VSCHEMA TABLES") + got = fmt.Sprintf("%v", qr.Rows) + want = `[[VARCHAR("dual")]]` + assert.Equal(t, want, got) + + // Use the DDL to create an unsharded vschema and test again + + // Create VSchema and do a Select to force update VSCHEMA + exec(t, conn, "begin") + exec(t, conn, "ALTER VSCHEMA ADD TABLE vt_user") + exec(t, conn, "select * from vt_user") + exec(t, conn, "commit") + + exec(t, conn, "begin") + exec(t, conn, "ALTER VSCHEMA ADD TABLE main") + exec(t, conn, "select * from main") + exec(t, conn, "commit") + + // Test Showing Tables + qr = exec(t, conn, "SHOW VSCHEMA TABLES") + got = fmt.Sprintf("%v", qr.Rows) + want = `[[VARCHAR("dual")] [VARCHAR("main")] [VARCHAR("vt_user")]]` + assert.Equal(t, want, got) + + // Test Showing Vindexes + qr = exec(t, conn, "SHOW VSCHEMA VINDEXES") + got = fmt.Sprintf("%v", qr.Rows) + want = `[]` + assert.Equal(t, want, got) + + // Test DML operations + exec(t, conn, "insert into vt_user (id,name) values(1,'test1'), (2,'test2'), (3,'test3'), (4,'test4')") + qr = exec(t, conn, "select id, name from vt_user order by id") + got = fmt.Sprintf("%v", qr.Rows) + want = `[[INT64(1) VARCHAR("test1")] [INT64(2) VARCHAR("test2")] [INT64(3) VARCHAR("test3")] [INT64(4) VARCHAR("test4")]]` + assert.Equal(t, want, got) + + qr = exec(t, conn, "delete from vt_user") + got = fmt.Sprintf("%v", qr.Rows) + want = `[]` + assert.Equal(t, want, got) + +} + +func exec(t *testing.T, conn *mysql.Conn, query string) *sqltypes.Result { + t.Helper() + qr, err := conn.ExecuteFetch(query, 1000, true) + if err != nil { + t.Fatal(err) + } + return qr +} From 5ac032e49b0196f1b2ddecf985fb51cc90eaf585 Mon Sep 17 00:00:00 2001 From: Arindam Nayak Date: Fri, 25 Oct 2019 22:20:57 +0530 Subject: [PATCH 09/15] exclude cluster test from e2e_race Signed-off-by: Arindam Nayak --- Makefile | 3 +++ test/config.json | 2 +- tools/e2e_test_cluster.sh | 1 + tools/e2e_test_race.sh | 1 + 4 files changed, 6 insertions(+), 1 deletion(-) mode change 100644 => 100755 tools/e2e_test_cluster.sh diff --git a/Makefile b/Makefile index e20940f2b2a..63e5bf459bc 100644 --- a/Makefile +++ b/Makefile @@ -104,6 +104,9 @@ unit_test_race: build e2e_test_race: build tools/e2e_test_race.sh +e2e_test_cluster: build + tools/e2e_test_cluster.sh + .ONESHELL: SHELL = /bin/bash diff --git a/test/config.json b/test/config.json index 37cf5bb04f0..6be1cee2696 100644 --- a/test/config.json +++ b/test/config.json @@ -416,7 +416,7 @@ "Args": [], "Command": [ "make", - "tools/e2e_test_cluster.sh" + "e2e_test_cluster" ], "Manual": false, "Shard": 2, diff --git a/tools/e2e_test_cluster.sh b/tools/e2e_test_cluster.sh old mode 100644 new mode 100755 index 414d20d44e2..d9ca94f6557 --- a/tools/e2e_test_cluster.sh +++ b/tools/e2e_test_cluster.sh @@ -24,6 +24,7 @@ packages_with_tests=$(go list -f '{{if len .TestGoFiles}}{{.ImportPath}} {{join cluster_tests=$(echo "$packages_with_tests" | grep -E "go/test/endtoend" | cut -d" " -f1) # Run cluster test sequentially +echo "running cluster tests $cluster_tests" echo "$cluster_tests" | xargs go test -v -p=1 if [ $? -ne 0 ]; then echo "ERROR: Go cluster tests failed. See above for errors." diff --git a/tools/e2e_test_race.sh b/tools/e2e_test_race.sh index a6e01567ba7..d1cf9765036 100755 --- a/tools/e2e_test_race.sh +++ b/tools/e2e_test_race.sh @@ -34,6 +34,7 @@ export GO111MODULE=on # All endtoend Go packages with test files. # Output per line: * packages_with_tests=$(go list -f '{{if len .TestGoFiles}}{{.ImportPath}} {{join .TestGoFiles " "}}{{end}}' ./go/.../endtoend/... | sort) +packages_with_tests=$(echo "$packages_with_tests" | grep -vE "go/test/endtoend" | cut -d" " -f1) # endtoend tests should be in a directory called endtoend all_e2e_tests=$(echo "$packages_with_tests" | cut -d" " -f1) From c2627e77b3c1903d457b34dc7f6a3c690616425b Mon Sep 17 00:00:00 2001 From: Arindam Nayak Date: Wed, 30 Oct 2019 17:59:20 +0530 Subject: [PATCH 10/15] updated health check for vtgate and vtctld Signed-off-by: Arindam Nayak --- go/test/endtoend/cluster/vtctld_process.go | 2 +- go/test/endtoend/cluster/vtgate_process.go | 18 ++++++++++++------ 2 files changed, 13 insertions(+), 7 deletions(-) diff --git a/go/test/endtoend/cluster/vtctld_process.go b/go/test/endtoend/cluster/vtctld_process.go index 6909aa4cf36..c82a04002b1 100644 --- a/go/test/endtoend/cluster/vtctld_process.go +++ b/go/test/endtoend/cluster/vtctld_process.go @@ -171,6 +171,6 @@ func VtctldProcessInstance(httpPort int, grpcPort int, topoPort int, hostname st PidFile: path.Join(tmpDirectory, "vtctld.pid"), Directory: os.Getenv("VTDATAROOT"), } - vtctld.VerifyURL = fmt.Sprintf("http://localhost:%d", vtctld.Port) + vtctld.VerifyURL = fmt.Sprintf("http://%s:%d/debug/vars", hostname, vtctld.Port) return vtctld } diff --git a/go/test/endtoend/cluster/vtgate_process.go b/go/test/endtoend/cluster/vtgate_process.go index 0eb7b0d5571..785954d6093 100644 --- a/go/test/endtoend/cluster/vtgate_process.go +++ b/go/test/endtoend/cluster/vtgate_process.go @@ -24,6 +24,7 @@ import ( "os" "os/exec" "path" + "reflect" "strings" "syscall" "time" @@ -126,12 +127,17 @@ func (vtgate *VtgateProcess) WaitForStatus() bool { if err != nil { panic(err) } - //for key, value := range resultMap { - // println("VTGate API Response: Key = " + key + ", value = " + fmt.Sprintf("%v", value)) - //} - //println(string(respByte)) - //return resultMap["TabletStateName"] == "NOT_SERVING" - return true + object := reflect.ValueOf(resultMap["HealthcheckConnections"]) + masterConnectionExist := false + if object.Kind() == reflect.Map { + for _, key := range object.MapKeys() { + + if strings.Contains(key.String(),"master") { + masterConnectionExist = true + } + } + } + return masterConnectionExist } return false } From 976242f80ce641c2612ff97d1e10319d7c2c2963 Mon Sep 17 00:00:00 2001 From: Arindam Nayak Date: Wed, 6 Nov 2019 22:39:04 +0530 Subject: [PATCH 11/15] Signed-off-by: Arindam Nayak fix the test error --- dev.env | 22 ++++++------- go/test/endtoend/cluster/cluster_process.go | 33 ++++++++++++++----- go/test/endtoend/cluster/etcd_process.go | 8 +++-- go/test/endtoend/cluster/mysqlctl_process.go | 2 +- go/test/endtoend/cluster/vtctld_process.go | 3 -- go/test/endtoend/cluster/vtgate_process.go | 2 +- go/test/endtoend/cluster/vttablet_process.go | 6 ++-- go/test/endtoend/clustertest/main_test.go | 2 +- go/test/endtoend/vtgate/main_test.go | 2 +- go/test/endtoend/vtgate/sequence/seq_test.go | 2 +- .../vtgate/transaction/trxn_mode_test.go | 2 +- .../endtoend/vtgate/vschema/vschema_test.go | 2 +- 12 files changed, 51 insertions(+), 35 deletions(-) diff --git a/dev.env b/dev.env index b7d65b9533b..784b9b026ec 100644 --- a/dev.env +++ b/dev.env @@ -73,17 +73,6 @@ PATH=$(prepend_path "$PATH" "$VTROOT/dist/chromedriver") PATH=$(prepend_path "$PATH" "$VTROOT/dist/node/bin") export PATH -# Etcd path. -case $(uname) in - Linux) etcd_platform=linux;; - Darwin) etcd_platform=darwin;; -esac - -ETCD_VERSION=$(cat "${VTROOT}/dist/etcd/.installed_version") -ETCD_BINDIR="${VTROOT}/dist/etcd/etcd-${ETCD_VERSION}-${etcd_platform}-amd64/" -PATH=$(prepend_path "$PATH" "$ETCD_BINDIR") -export PATH - # GOROOT sanity go_bin=$(which go) go_env=$(go env | grep GOROOT | cut -f 2 -d\") @@ -113,3 +102,14 @@ export PKG_CONFIG_PATH alias gt='cd $GOTOP' alias pt='cd $PYTOP' alias vt='cd $VTTOP' + +# Etcd path. +case $(uname) in + Linux) etcd_platform=linux;; + Darwin) etcd_platform=darwin;; +esac + +ETCD_VERSION=$(cat "${VTROOT}/dist/etcd/.installed_version") +ETCD_BINDIR="${VTROOT}/dist/etcd/etcd-${ETCD_VERSION}-${etcd_platform}-amd64/" +PATH=$(prepend_path "$PATH" "$ETCD_BINDIR") +export PATH \ No newline at end of file diff --git a/go/test/endtoend/cluster/cluster_process.go b/go/test/endtoend/cluster/cluster_process.go index e9e9a61733d..21566c18a13 100644 --- a/go/test/endtoend/cluster/cluster_process.go +++ b/go/test/endtoend/cluster/cluster_process.go @@ -17,6 +17,7 @@ limitations under the License. package cluster import ( + "flag" "fmt" "math/rand" "os" @@ -28,14 +29,20 @@ import ( // DefaultCell : If no cell name is passed, then use following const DefaultCell = "zone1" +var ( + keepData = flag.Bool("keep-data", false, "don't delete the per-test VTDATAROOT subfolders") +) + // LocalProcessCluster Testcases need to use this to iniate a cluster type LocalProcessCluster struct { - Keyspaces []Keyspace - Cell string - BaseTabletUID int - Hostname string - TopoPort int - TmpDirectory string + Keyspaces []Keyspace + Cell string + BaseTabletUID int + Hostname string + TopoPort int + TmpDirectory string + OriginalVTDATAROOT string + CurrentVTDATAROOT string VtgateMySQLPort int VtgateGrpcPort int @@ -238,7 +245,7 @@ func (cluster *LocalProcessCluster) StartVtgate() (err error) { cluster.VtgateMySQLPort, cluster.Cell, cluster.Cell, - cluster.Hostname, + cluster.Hostname, "MASTER,REPLICA", cluster.topoProcess.Port, cluster.TmpDirectory, @@ -248,6 +255,16 @@ func (cluster *LocalProcessCluster) StartVtgate() (err error) { return cluster.VtgateProcess.Setup() } +// NewCluster instantiates a new cluster +func NewCluster(cell string, hostname string) *LocalProcessCluster { + cluster := &LocalProcessCluster{Cell: cell, Hostname: hostname} + cluster.OriginalVTDATAROOT = os.Getenv("VTDATAROOT") + cluster.CurrentVTDATAROOT = path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("vtroot_%d", cluster.GetAndReservePort())) + _ = createDirectory(cluster.CurrentVTDATAROOT, 0700) + _ = os.Setenv("VTDATAROOT", cluster.CurrentVTDATAROOT) + return cluster +} + // ReStartVtgate starts vtgate with updated configs func (cluster *LocalProcessCluster) ReStartVtgate() (err error) { err = cluster.VtgateProcess.TearDown() @@ -291,7 +308,7 @@ func (cluster *LocalProcessCluster) Teardown() (err error) { return } - if err = cluster.topoProcess.TearDown(cluster.Cell); err != nil { + if err = cluster.topoProcess.TearDown(cluster.Cell, cluster.OriginalVTDATAROOT, cluster.CurrentVTDATAROOT, *keepData); err != nil { log.Error(err.Error()) return } diff --git a/go/test/endtoend/cluster/etcd_process.go b/go/test/endtoend/cluster/etcd_process.go index 6367112c33e..284a849d3ca 100644 --- a/go/test/endtoend/cluster/etcd_process.go +++ b/go/test/endtoend/cluster/etcd_process.go @@ -95,7 +95,7 @@ func (etcd *EtcdProcess) Setup() (err error) { } // TearDown shutdowns the running mysqld service -func (etcd *EtcdProcess) TearDown(Cell string) error { +func (etcd *EtcdProcess) TearDown(Cell string, originalVtRoot string, currentRoot string, keepdata bool) error { if etcd.proc == nil || etcd.exit == nil { return nil } @@ -104,7 +104,11 @@ func (etcd *EtcdProcess) TearDown(Cell string) error { // Attempt graceful shutdown with SIGTERM first _ = etcd.proc.Process.Signal(syscall.SIGTERM) - _ = os.RemoveAll(etcd.DataDirectory) + if !*keepData { + _ = os.RemoveAll(etcd.DataDirectory) + _ = os.RemoveAll(currentRoot) + } + _ = os.Setenv("VTDATAROOT", originalVtRoot) select { case err := <-etcd.exit: etcd.proc = nil diff --git a/go/test/endtoend/cluster/mysqlctl_process.go b/go/test/endtoend/cluster/mysqlctl_process.go index 14efd76774c..baec38d391d 100644 --- a/go/test/endtoend/cluster/mysqlctl_process.go +++ b/go/test/endtoend/cluster/mysqlctl_process.go @@ -67,7 +67,7 @@ func (mysqlctl *MysqlctlProcess) Stop() (err error) { "-tablet_uid", fmt.Sprintf("%d", mysqlctl.TabletUID), "shutdown", ) - return tmpProcess.Run() + return tmpProcess.Start() } // MysqlCtlProcessInstance returns a Mysqlctl handle for mysqlctl process diff --git a/go/test/endtoend/cluster/vtctld_process.go b/go/test/endtoend/cluster/vtctld_process.go index c82a04002b1..d8666b40912 100644 --- a/go/test/endtoend/cluster/vtctld_process.go +++ b/go/test/endtoend/cluster/vtctld_process.go @@ -133,9 +133,6 @@ func (vtctld *VtctldProcess) TearDown() error { return nil } - os.RemoveAll(vtctld.LogDir) - //os.RemoveAll(path.Join(vtctld.Directory, "backups")) - // Attempt graceful shutdown with SIGTERM first vtctld.proc.Process.Signal(syscall.SIGTERM) diff --git a/go/test/endtoend/cluster/vtgate_process.go b/go/test/endtoend/cluster/vtgate_process.go index 243d8eeba19..e2b771735ad 100644 --- a/go/test/endtoend/cluster/vtgate_process.go +++ b/go/test/endtoend/cluster/vtgate_process.go @@ -180,7 +180,7 @@ func VtgateProcessInstance(port int, grpcPort int, mySQLServerPort int, cell str Port: port, GrpcPort: grpcPort, MySQLServerPort: mySQLServerPort, - MySQLServerSocketPath: "/tmp/mysql.sock", + MySQLServerSocketPath: path.Join(tmpDirectory, "mysql.sock"), Cell: cell, CellsToWatch: cellsToWatch, TabletTypesToWait: tabletTypesToWait, diff --git a/go/test/endtoend/cluster/vttablet_process.go b/go/test/endtoend/cluster/vttablet_process.go index f8b23b60f24..5e0251f1119 100644 --- a/go/test/endtoend/cluster/vttablet_process.go +++ b/go/test/endtoend/cluster/vttablet_process.go @@ -154,8 +154,6 @@ func (vttablet *VttabletProcess) TearDown() error { // Attempt graceful shutdown with SIGTERM first vttablet.proc.Process.Signal(syscall.SIGTERM) - os.RemoveAll(vttablet.Directory) - select { case err := <-vttablet.exit: vttablet.proc = nil @@ -176,7 +174,7 @@ func VttabletProcessInstance(port int, grpcPort int, tabletUID int, cell string, vttablet := &VttabletProcess{ Name: "vttablet", Binary: "vttablet", - FileToLogQueries: path.Join(tmpDirectory, fmt.Sprintf("/vt_%010d/vttable.pid", tabletUID)), + FileToLogQueries: path.Join(tmpDirectory, fmt.Sprintf("/vt_%010d/querylog.txt", tabletUID)), Directory: path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("/vt_%010d", tabletUID)), TabletPath: fmt.Sprintf("%s-%010d", cell, tabletUID), ServiceMap: "grpc-queryservice,grpc-tabletmanager,grpc-updatestream", @@ -191,7 +189,7 @@ func VttabletProcessInstance(port int, grpcPort int, tabletUID int, cell string, FileBackupStorageRoot: path.Join(os.Getenv("VTDATAROOT"), "/backups"), Port: port, GrpcPort: grpcPort, - PidFile: path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("/vt_%010d/vttable.pid", tabletUID)), + PidFile: path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("/vt_%010d/vttablet.pid", tabletUID)), VtctldAddress: fmt.Sprintf("http://%s:%d", hostname, vtctldPort), ExtraArgs: extraArgs, } diff --git a/go/test/endtoend/clustertest/main_test.go b/go/test/endtoend/clustertest/main_test.go index 91eb4bde771..11c48433ac3 100644 --- a/go/test/endtoend/clustertest/main_test.go +++ b/go/test/endtoend/clustertest/main_test.go @@ -63,7 +63,7 @@ func TestMain(m *testing.M) { flag.Parse() exitCode := func() int { - clusterInstance = &cluster.LocalProcessCluster{Cell: cell, Hostname: "localhost"} + clusterInstance = cluster.NewCluster(cell, "localhost") defer clusterInstance.Teardown() // Start topo server diff --git a/go/test/endtoend/vtgate/main_test.go b/go/test/endtoend/vtgate/main_test.go index 9cd2a9a88ec..d707b0d511d 100644 --- a/go/test/endtoend/vtgate/main_test.go +++ b/go/test/endtoend/vtgate/main_test.go @@ -168,7 +168,7 @@ func TestMain(m *testing.M) { flag.Parse() exitCode := func() int { - clusterInstance = &cluster.LocalProcessCluster{Cell: Cell, Hostname: "localhost"} + clusterInstance = cluster.NewCluster(Cell, "localhost") defer clusterInstance.Teardown() // Start topo server diff --git a/go/test/endtoend/vtgate/sequence/seq_test.go b/go/test/endtoend/vtgate/sequence/seq_test.go index 396bfb0ee11..72ce77bea02 100644 --- a/go/test/endtoend/vtgate/sequence/seq_test.go +++ b/go/test/endtoend/vtgate/sequence/seq_test.go @@ -82,7 +82,7 @@ func TestMain(m *testing.M) { flag.Parse() exitCode := func() int { - clusterInstance = &cluster.LocalProcessCluster{Cell: cell, Hostname: hostname} + clusterInstance = cluster.NewCluster(cell, hostname) defer clusterInstance.Teardown() // Start topo server diff --git a/go/test/endtoend/vtgate/transaction/trxn_mode_test.go b/go/test/endtoend/vtgate/transaction/trxn_mode_test.go index 4e7d64da9c8..bf20e10dd9b 100644 --- a/go/test/endtoend/vtgate/transaction/trxn_mode_test.go +++ b/go/test/endtoend/vtgate/transaction/trxn_mode_test.go @@ -98,7 +98,7 @@ func TestMain(m *testing.M) { flag.Parse() exitcode, err := func() (int, error) { - clusterInstance = &cluster.LocalProcessCluster{Cell: cell, Hostname: hostname} + clusterInstance = cluster.NewCluster(cell, hostname) defer clusterInstance.Teardown() // Reserve vtGate port in order to pass it to vtTablet diff --git a/go/test/endtoend/vtgate/vschema/vschema_test.go b/go/test/endtoend/vtgate/vschema/vschema_test.go index 1b0a984ede1..bcab68351dc 100644 --- a/go/test/endtoend/vtgate/vschema/vschema_test.go +++ b/go/test/endtoend/vtgate/vschema/vschema_test.go @@ -54,7 +54,7 @@ func TestMain(m *testing.M) { flag.Parse() exitcode, err := func() (int, error) { - clusterInstance = &cluster.LocalProcessCluster{Cell: cell, Hostname: hostname} + clusterInstance = cluster.NewCluster(cell, hostname) defer clusterInstance.Teardown() // Start topo server From c88c678592493f0711def57cabb4a53488788ee3 Mon Sep 17 00:00:00 2001 From: Arindam Nayak Date: Thu, 7 Nov 2019 18:08:23 +0530 Subject: [PATCH 12/15] fix tablet teardown Signed-off-by: Arindam Nayak --- go/test/endtoend/cluster/vttablet_process.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/go/test/endtoend/cluster/vttablet_process.go b/go/test/endtoend/cluster/vttablet_process.go index 5e0251f1119..f3ced054d00 100644 --- a/go/test/endtoend/cluster/vttablet_process.go +++ b/go/test/endtoend/cluster/vttablet_process.go @@ -155,9 +155,9 @@ func (vttablet *VttabletProcess) TearDown() error { vttablet.proc.Process.Signal(syscall.SIGTERM) select { - case err := <-vttablet.exit: + case <-vttablet.exit: vttablet.proc = nil - return err + return nil case <-time.After(10 * time.Second): vttablet.proc.Process.Kill() From b7672bd0d7451e53c03dfb0dab663ac677b5bb79 Mon Sep 17 00:00:00 2001 From: Morgan Tocker Date: Thu, 7 Nov 2019 12:20:39 -0700 Subject: [PATCH 13/15] Add GitHub action for building new test Signed-off-by: Morgan Tocker --- .github/workflows/e2e-test-cluster.yml | 41 ++++++++++++++++++++++++++ 1 file changed, 41 insertions(+) create mode 100644 .github/workflows/e2e-test-cluster.yml diff --git a/.github/workflows/e2e-test-cluster.yml b/.github/workflows/e2e-test-cluster.yml new file mode 100644 index 00000000000..969008305ef --- /dev/null +++ b/.github/workflows/e2e-test-cluster.yml @@ -0,0 +1,41 @@ +name: e2e Test Cluster +on: [push, pull_request] +jobs: + + build: + name: Build + runs-on: ubuntu-latest + steps: + + - name: Set up Go + uses: actions/setup-go@v1 + with: + go-version: 1.13 + + - name: Check out code + uses: actions/checkout@v1 + + - name: Get dependencies + run: | + sudo apt-get install -y mysql-server mysql-client make unzip g++ etcd curl git wget + sudo service mysql stop + sudo service etcd stop + sudo ln -s /etc/apparmor.d/usr.sbin.mysqld /etc/apparmor.d/disable/ + sudo apparmor_parser -R /etc/apparmor.d/usr.sbin.mysqld + go mod download + + - name: Run bootstrap.sh + run: | + echo "Copying new bootstrap over location of legacy one." + cp .github/bootstrap.sh . + ./bootstrap.sh + + - name: Build + run: | + GOBIN=$PWD/bin make build + + - name: Run e2e test cluster + run: | + export PATH=$PWD/bin:$PATH + source ./dev.env + VTDATAROOT=/tmp/vtdataroot VTTOP=$PWD VTROOT=$PWD tools/e2e_test_cluster.sh From 24fabc99117978fc6e5d11a6a432d20aaaa536e5 Mon Sep 17 00:00:00 2001 From: Arindam Nayak Date: Fri, 8 Nov 2019 11:40:01 +0530 Subject: [PATCH 14/15] moved cluster test to not to execute in travis, as it will be managed by github workflow Signed-off-by: Arindam Nayak --- test/config.json | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/test/config.json b/test/config.json index 3cd04db2ce1..603e772de11 100644 --- a/test/config.json +++ b/test/config.json @@ -430,7 +430,7 @@ "e2e_test_cluster" ], "Manual": false, - "Shard": 2, + "Shard": 5, "RetryMax": 0, "Tags": [] }, From bb51e19675bffb2585878730e49006da0eee9ec1 Mon Sep 17 00:00:00 2001 From: Arindam Nayak Date: Mon, 11 Nov 2019 11:59:50 +0530 Subject: [PATCH 15/15] added rand.seed when cluster instance is built Signed-off-by: Arindam Nayak --- go/test/endtoend/cluster/cluster_process.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/go/test/endtoend/cluster/cluster_process.go b/go/test/endtoend/cluster/cluster_process.go index 21566c18a13..780917918ce 100644 --- a/go/test/endtoend/cluster/cluster_process.go +++ b/go/test/endtoend/cluster/cluster_process.go @@ -22,6 +22,7 @@ import ( "math/rand" "os" "path" + "time" "vitess.io/vitess/go/vt/log" ) @@ -262,6 +263,7 @@ func NewCluster(cell string, hostname string) *LocalProcessCluster { cluster.CurrentVTDATAROOT = path.Join(os.Getenv("VTDATAROOT"), fmt.Sprintf("vtroot_%d", cluster.GetAndReservePort())) _ = createDirectory(cluster.CurrentVTDATAROOT, 0700) _ = os.Setenv("VTDATAROOT", cluster.CurrentVTDATAROOT) + rand.Seed(time.Now().UTC().UnixNano()) return cluster }