diff --git a/.github/workflows/integrate.yaml b/.github/workflows/integrate.yaml index dfaeaf5dfb..791f0c97d0 100644 --- a/.github/workflows/integrate.yaml +++ b/.github/workflows/integrate.yaml @@ -200,7 +200,7 @@ jobs: # if: steps.test.outputs.exit_code != 0 if: always() run: | - find ${{env.working-directory}}/tests/tiup-playground/_tmp/data -type f | grep -E '*/ti.*/ti.*.log$' | xargs tar czvf ${{env.working-directory}}/playground.logs.tar.gz + find ${{env.working-directory}}/tests/tiup-playground/_tmp/home/data -type f | grep -E '*/ti.*/ti.*.log$' | xargs tar czvf ${{env.working-directory}}/playground.logs.tar.gz - name: Upload component log # if: steps.test.outputs.exit_code != 0 diff --git a/.gitignore b/.gitignore index 54e0206d35..58fe173bb1 100644 --- a/.gitignore +++ b/.gitignore @@ -30,3 +30,4 @@ bin/ /tests/tiup_mirrors/*.sha1 /tests/tiup_mirrors/ /logs +docker/secret/ diff --git a/components/playground/grafana.go b/components/playground/grafana.go index c15797250c..ae204917af 100644 --- a/components/playground/grafana.go +++ b/components/playground/grafana.go @@ -23,6 +23,7 @@ import ( "path/filepath" "regexp" "strings" + "sync" "github.com/pingcap/errors" "github.com/pingcap/tiup/pkg/environment" @@ -36,7 +37,9 @@ type grafana struct { port int version string - cmd *exec.Cmd + waitErr error + waitOnce sync.Once + cmd *exec.Cmd } func newGrafana(version string, host string) *grafana { @@ -149,6 +152,7 @@ func makeSureDir(fname string) error { var clusterName string = "playground" // dir should contains files untar the grafana. +// return not error iff the Cmd is started successfully. func (g *grafana) start(ctx context.Context, dir string, p8sURL string) (err error) { g.port, err = utils.GetFreePort(g.host, 3000) if err != nil { @@ -208,3 +212,11 @@ http_port = %d return g.cmd.Start() } + +func (g *grafana) wait() error { + g.waitOnce.Do(func() { + g.waitErr = g.cmd.Wait() + }) + + return g.waitErr +} diff --git a/components/playground/instance/instance.go b/components/playground/instance/instance.go index 8fd012fb5c..836eb0c26e 100644 --- a/components/playground/instance/instance.go +++ b/components/playground/instance/instance.go @@ -42,6 +42,7 @@ type instance struct { type Instance interface { Pid() int // Start the instance process. + // Will kill the process once the context is done. Start(ctx context.Context, version v0manifest.Version) error // Component Return the component name. Component() string @@ -52,7 +53,8 @@ type Instance interface { // StatusAddrs return the address to pull metrics. StatusAddrs() []string // Wait Should only call this if the instance is started successfully. - Wait(ctx context.Context) error + // The implementation should be safe to call Wait multi times. + Wait() error } func (inst *instance) StatusAddrs() (addrs []string) { diff --git a/components/playground/instance/process.go b/components/playground/instance/process.go index f8adc87da8..1f7db3e188 100644 --- a/components/playground/instance/process.go +++ b/components/playground/instance/process.go @@ -5,19 +5,19 @@ import ( "io" "os" "os/exec" + "sync" "time" "github.com/pingcap/errors" "github.com/pingcap/tiup/pkg/environment" tiupexec "github.com/pingcap/tiup/pkg/exec" "github.com/pingcap/tiup/pkg/repository/v0manifest" - "github.com/pingcap/tiup/pkg/utils" ) // Process represent process to be run by playground type Process interface { Start() error - Wait(ctx context.Context) error + Wait() error Pid() int Uptime() string SetOutputFile(fname string) error @@ -28,6 +28,9 @@ type Process interface { type process struct { cmd *exec.Cmd startTime time.Time + + waitOnce sync.Once + waitErr error } // Start the process @@ -38,8 +41,12 @@ func (p *process) Start() error { } // Wait implements Instance interface. -func (p *process) Wait(ctx context.Context) error { - return utils.WaitContext(ctx, p.cmd) +func (p *process) Wait() error { + p.waitOnce.Do(func() { + p.waitErr = p.cmd.Wait() + }) + + return p.waitErr } // Pid implements Instance interface. @@ -50,8 +57,9 @@ func (p *process) Pid() int { // Uptime implements Instance interface. func (p *process) Uptime() string { s := p.cmd.ProcessState - if s != nil && s.Exited() { - return "exited" + + if s != nil { + return s.String() } duration := time.Since(p.startTime) diff --git a/components/playground/main.go b/components/playground/main.go index 7ea0111396..135f291602 100644 --- a/components/playground/main.go +++ b/components/playground/main.go @@ -21,10 +21,13 @@ import ( "net/http" _ "net/http/pprof" "os" + "os/signal" "os/user" "path/filepath" "strconv" "strings" + "sync/atomic" + "syscall" "time" "github.com/fatih/color" @@ -101,7 +104,8 @@ Examples: $ tiup playground nightly --monitor=false # Start a local cluster and disable monitor system $ tiup playground --pd.config ~/config/pd.toml # Start a local cluster with specified configuration file, $ tiup playground --db.binpath /xx/tidb-server # Start a local cluster with component binary path`, - SilenceUsage: true, + SilenceUsage: true, + SilenceErrors: true, Args: func(cmd *cobra.Command, args []string) error { return nil }, @@ -126,7 +130,66 @@ Examples: } environment.SetGlobalEnv(env) - return p.bootCluster(env, opt) + var booted uint32 + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + go func() { + sc := make(chan os.Signal, 1) + signal.Notify(sc, + syscall.SIGHUP, + syscall.SIGINT, + syscall.SIGTERM, + syscall.SIGQUIT, + ) + + sig := (<-sc).(syscall.Signal) + atomic.StoreInt32(&p.curSig, int32(sig)) + fmt.Println("Playground receive signal: ", sig) + + // if bootCluster is not done we just cancel context to make it + // clean up and return ASAP and exit directly after timeout. + // Note now bootCluster can not learn the context is done and return quickly now + // like while it's downloading component. + if atomic.LoadUint32(&booted) == 0 { + cancel() + time.AfterFunc(time.Second, func() { + os.Exit(0) + }) + return + } + + go p.terminate(sig) + // If user try double ctrl+c, force quit + sig = (<-sc).(syscall.Signal) + atomic.StoreInt32(&p.curSig, int32(syscall.SIGKILL)) + if sig == syscall.SIGINT { + p.terminate(syscall.SIGKILL) + } + }() + + // TODO: we can set Pdeathsig of Cmd.SysProcAttr(linux only) in all the Cmd we started to kill + // all the process we start instead of let the orphaned child process adopted by init, + // this can make sure we kill all process event if + // playground is killed -9. + // ref: https://medium.com/@ganeshmaharaj/clean-exit-of-golangs-exec-command-897832ac3fa5 + bootErr := p.bootCluster(ctx, env, opt) + if bootErr != nil { + // always kill all process started and wait before quit. + atomic.StoreInt32(&p.curSig, int32(syscall.SIGKILL)) + p.terminate(syscall.SIGKILL) + _ = p.wait() + return errors.Annotate(bootErr, "Playground bootstrapping failed") + } + + atomic.StoreUint32(&booted, 1) + + waitErr := p.wait() + if waitErr != nil { + return errors.AddStack(waitErr) + } + + return nil }, } @@ -292,7 +355,7 @@ func newEtcdClient(endpoint string) (*clientv3.Client, error) { func main() { if err := execute(); err != nil { - fmt.Printf("Playground bootstrapping failed: %v\n", err) + fmt.Println(color.RedString("Error: %v", err)) os.Exit(1) } } diff --git a/components/playground/monitor.go b/components/playground/monitor.go index 415911960a..aa795b0986 100644 --- a/components/playground/monitor.go +++ b/components/playground/monitor.go @@ -21,6 +21,7 @@ import ( "os" "os/exec" "path/filepath" + "sync" "github.com/pingcap/errors" "github.com/pingcap/tiup/pkg/environment" @@ -69,20 +70,28 @@ type monitor struct { cmd *exec.Cmd sdFname string + + waitErr error + waitOnce sync.Once } -func newMonitor() *monitor { - return &monitor{} +func (m *monitor) wait() error { + m.waitOnce.Do(func() { + m.waitErr = m.cmd.Wait() + }) + + return m.waitErr } -func (m *monitor) startMonitor(ctx context.Context, version string, host, dir string) (int, *exec.Cmd, error) { +// the cmd is not started after return +func newMonitor(ctx context.Context, version string, host, dir string) (*monitor, error) { if err := os.MkdirAll(dir, 0755); err != nil { - return 0, nil, err + return nil, errors.AddStack(err) } port, err := utils.GetFreePort(host, 9090) if err != nil { - return 0, nil, err + return nil, errors.AddStack(err) } addr := fmt.Sprintf("%s:%d", host, port) @@ -114,10 +123,11 @@ scrape_configs: ` + m := new(monitor) m.sdFname = filepath.Join(dir, "targets.json") if err := ioutil.WriteFile(filepath.Join(dir, "prometheus.yml"), []byte(tmpl), os.ModePerm); err != nil { - return 0, nil, err + return nil, errors.AddStack(err) } args := []string{ @@ -130,11 +140,12 @@ scrape_configs: env := environment.GlobalEnv() cmd, err := tiupexec.PrepareCommand(ctx, "prometheus", v0manifest.Version(version), "", "", dir, dir, args, env) if err != nil { - return 0, nil, err + return nil, err } m.port = port m.cmd = cmd m.host = host - return port, cmd, nil + + return m, nil } diff --git a/components/playground/playground.go b/components/playground/playground.go index 9ea593d02e..fc8cabefd3 100755 --- a/components/playground/playground.go +++ b/components/playground/playground.go @@ -22,7 +22,6 @@ import ( "net/http" "os" "os/exec" - "os/signal" "path/filepath" "runtime" "strconv" @@ -35,7 +34,6 @@ import ( "github.com/cheynewallace/tabby" "github.com/fatih/color" "github.com/pingcap/errors" - "github.com/pingcap/failpoint" "github.com/pingcap/tiup/components/playground/instance" "github.com/pingcap/tiup/pkg/cluster/api" "github.com/pingcap/tiup/pkg/environment" @@ -46,27 +44,33 @@ import ( "golang.org/x/sync/errgroup" ) -// The deadline a process should quit after receives kill signal -const killDeadline = time.Second * 16 +// The duration process need to quit gracefully, or we kill the process. +const forceKillAfterDuration = time.Second * 10 // Playground represent the playground of a cluster. type Playground struct { - booted bool + booted bool + // the latest receive signal curSig int32 bootOptions *bootOptions port int - pds []*instance.PDInstance - tikvs []*instance.TiKVInstance - tidbs []*instance.TiDBInstance - tiflashs []*instance.TiFlashInstance - pumps []*instance.Pump - drainers []*instance.Drainer + pds []*instance.PDInstance + tikvs []*instance.TiKVInstance + tidbs []*instance.TiDBInstance + tiflashs []*instance.TiFlashInstance + pumps []*instance.Pump + drainers []*instance.Drainer + startedInstances []instance.Instance idAlloc map[string]int instanceWaiter errgroup.Group + // not nil iff we start the exec.Cmd successfully. + // we should and can safely call wait() to make sure the process quit + // before playground quit. monitor *monitor + grafana *grafana } // MonitorInfo represent the monitor @@ -398,14 +402,18 @@ func (p *Playground) sanitizeComponentConfig(cid string, cfg *instance.Config) e func (p *Playground) startInstance(ctx context.Context, inst instance.Instance) error { fmt.Printf("Start %s instance...\n", inst.Component()) - err := inst.Start(context.Background(), v0manifest.Version(p.bootOptions.version)) + err := inst.Start(ctx, v0manifest.Version(p.bootOptions.version)) + if err != nil { + return errors.AddStack(err) + } p.addWaitInstance(inst) - return errors.AddStack(err) + return nil } func (p *Playground) addWaitInstance(inst instance.Instance) { + p.startedInstances = append(p.startedInstances, inst) p.instanceWaiter.Go(func() error { - err := inst.Wait(context.TODO()) + err := inst.Wait() if err != nil && atomic.LoadInt32(&p.curSig) == 0 { fmt.Print(color.RedString("%s quit: %s\n", inst.Component(), err.Error())) if lines, _ := utils.TailN(inst.LogFile(), 10); len(lines) > 0 { @@ -414,6 +422,8 @@ func (p *Playground) addWaitInstance(inst instance.Instance) { } fmt.Print(color.YellowString("...\ncheck detail log from: %s\n", inst.LogFile())) } + } else { + fmt.Printf("%s quit\n", inst.Component()) } return err }) @@ -624,7 +634,7 @@ func (p *Playground) addInstance(componentID string, cfg instance.Config) (ins i return } -func (p *Playground) bootCluster(env *environment.Environment, options *bootOptions) error { +func (p *Playground) bootCluster(ctx context.Context, env *environment.Environment, options *bootOptions) error { for _, cfg := range []*instance.Config{&options.pd, &options.tidb, &options.tikv, &options.tiflash, &options.pump, &options.drainer} { path, err := getAbsolutePath(cfg.ConfigPath) if err != nil { @@ -666,9 +676,6 @@ func (p *Playground) bootCluster(env *environment.Environment, options *bootOpti } } - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - for i := 0; i < options.pd.Num; i++ { _, err := p.addInstance("pd", options.pd) if err != nil { @@ -708,14 +715,39 @@ func (p *Playground) bootCluster(env *environment.Environment, options *bootOpti fmt.Println("Playground Bootstrapping...") - var monitorCmd *exec.Cmd - var grafana *grafana var monitorInfo *MonitorInfo if options.monitor { var err error - if monitorCmd, monitorInfo, grafana, err = p.bootMonitor(ctx, env); err != nil { + + p.monitor, monitorInfo, err = p.bootMonitor(ctx, env) + if err != nil { + return errors.AddStack(err) + } + + p.instanceWaiter.Go(func() error { + err := p.monitor.wait() + if err != nil && atomic.LoadInt32(&p.curSig) == 0 { + fmt.Printf("Prometheus quit: %v\n", err) + } else { + fmt.Println("prometheus quit") + } + return err + }) + + p.grafana, err = p.bootGrafana(ctx, env, monitorInfo) + if err != nil { return errors.AddStack(err) } + + p.instanceWaiter.Go(func() error { + err := p.grafana.wait() + if err != nil && atomic.LoadInt32(&p.curSig) == 0 { + fmt.Printf("Grafana quit: %v\n", err) + } else { + fmt.Println("Grafana quit") + } + return err + }) } anyPumpReady := false @@ -804,7 +836,9 @@ func (p *Playground) bootCluster(env *environment.Environment, options *bootOpti if lastErr != nil { fmt.Println(color.RedString("TiFlash failed to start. %s", lastErr)) - } else { + } + + if len(succ) > 0 { fmt.Println(color.GreenString("CLUSTER START SUCCESSFULLY, Enjoy it ^-^")) for _, dbAddr := range succ { ss := strings.Split(dbAddr, ":") @@ -833,49 +867,6 @@ func (p *Playground) bootCluster(env *environment.Environment, options *bootOpti dumpDSN(p.tidbs) - failpoint.Inject("terminateEarly", func() error { - time.Sleep(20 * time.Second) - - fmt.Println("Early terminated via failpoint") - - extraCmds := []*exec.Cmd{} - if grafana != nil { - extraCmds = append(extraCmds, grafana.cmd) - } - if monitorCmd != nil { - extraCmds = append(extraCmds, monitorCmd) - } - p.terminate(syscall.SIGKILL, extraCmds...) - - return nil - }) - - go func() { - sc := make(chan os.Signal, 1) - signal.Notify(sc, - syscall.SIGHUP, - syscall.SIGINT, - syscall.SIGTERM, - syscall.SIGQUIT) - sig := (<-sc).(syscall.Signal) - atomic.StoreInt32(&p.curSig, int32(sig)) - extraCmds := []*exec.Cmd{} - if grafana != nil { - extraCmds = append(extraCmds, grafana.cmd) - } - if monitorCmd != nil { - extraCmds = append(extraCmds, monitorCmd) - } - go p.terminate(sig, extraCmds...) - - // If user try double ctrl+c, force quit - sig = (<-sc).(syscall.Signal) - atomic.StoreInt32(&p.curSig, int32(syscall.SIGKILL)) - if sig == syscall.SIGINT { - p.terminate(syscall.SIGKILL, extraCmds...) - } - }() - go func() { // fmt.Printf("serve at :%d\n", p.port) err := p.listenAndServeHTTP() @@ -886,58 +877,69 @@ func (p *Playground) bootCluster(env *environment.Environment, options *bootOpti logIfErr(p.renderSDFile()) - if grafana != nil { - p.instanceWaiter.Go(func() error { - err := grafana.cmd.Wait() - if err != nil && atomic.LoadInt32(&p.curSig) == 0 { - fmt.Printf("Grafana quit: %v\n", err) - } - return err - }) - fmt.Print(color.GreenString("To view the Grafana: http://%s:%d\n", grafana.host, grafana.port)) + if p.grafana != nil { + fmt.Print(color.GreenString("To view the Grafana: http://%s:%d\n", p.grafana.host, p.grafana.port)) } - // Wait all instance quit and return the first non-nil err. - err = p.instanceWaiter.Wait() - if err != nil && atomic.LoadInt32(&p.curSig) == 0 { - return err - } + return nil +} - if monitorCmd != nil { - if err := monitorCmd.Wait(); err != nil && atomic.LoadInt32(&p.curSig) == 0 { - fmt.Println("Monitor system wait failed", err) - } +// Wait all instance quit and return the first non-nil err. +// including p8s & grafana +func (p *Playground) wait() error { + err := p.instanceWaiter.Wait() + if err != nil && atomic.LoadInt32(&p.curSig) == 0 { + return errors.AddStack(err) } return nil } -func (p *Playground) terminate(sig syscall.Signal, extraCmds ...*exec.Cmd) { - _ = p.RWalkInstances(func(_ string, inst instance.Instance) error { - if sig != syscall.SIGINT { - _ = syscall.Kill(inst.Pid(), sig) - } +func (p *Playground) terminate(sig syscall.Signal) { + for _, inst := range p.startedInstances { if sig == syscall.SIGKILL { fmt.Printf("Force %s(%d) to quit...\n", inst.Component(), inst.Pid()) } else if atomic.LoadInt32(&p.curSig) == int32(sig) { // In case of double ctr+c fmt.Printf("Wait %s(%d) to quit...\n", inst.Component(), inst.Pid()) } - ctx, cancel := context.WithTimeout(context.Background(), killDeadline) - defer cancel() - if err := inst.Wait(ctx); err == utils.ErrorWaitTimeout { - _ = syscall.Kill(inst.Pid(), syscall.SIGKILL) + + if sig != syscall.SIGINT { + _ = syscall.Kill(inst.Pid(), sig) } - return nil - }) - for _, cmd := range extraCmds { + + inst := inst + timer := time.AfterFunc(forceKillAfterDuration, func() { + _ = syscall.Kill(inst.Pid(), syscall.SIGKILL) + }) + + _ = inst.Wait() + timer.Stop() + } + + if p.monitor != nil { if sig != syscall.SIGINT { - _ = syscall.Kill(cmd.Process.Pid, sig) + _ = syscall.Kill(p.monitor.cmd.Process.Pid, sig) } - ctx, cancel := context.WithTimeout(context.Background(), killDeadline) - defer cancel() - if err := utils.WaitContext(ctx, cmd); err == utils.ErrorWaitTimeout { - _ = syscall.Kill(cmd.Process.Pid, syscall.SIGKILL) + + timer := time.AfterFunc(forceKillAfterDuration, func() { + _ = syscall.Kill(p.monitor.cmd.Process.Pid, syscall.SIGKILL) + }) + + _ = p.monitor.wait() + timer.Stop() + } + + if p.grafana != nil { + if sig != syscall.SIGINT { + _ = syscall.Kill(p.grafana.cmd.Process.Pid, sig) } + + timer := time.AfterFunc(forceKillAfterDuration, func() { + _ = syscall.Kill(p.grafana.cmd.Process.Pid, syscall.SIGKILL) + }) + + _ = p.grafana.wait() + timer.Stop() } } @@ -964,65 +966,66 @@ func (p *Playground) renderSDFile() error { return nil } -func (p *Playground) bootMonitor(ctx context.Context, env *environment.Environment) (*exec.Cmd, *MonitorInfo, *grafana, error) { +// return not error iff the Cmd is started successfully. +// user must and can safely wait the Cmd +func (p *Playground) bootMonitor(ctx context.Context, env *environment.Environment) (*monitor, *MonitorInfo, error) { options := p.bootOptions monitorInfo := &MonitorInfo{} dataDir := os.Getenv(localdata.EnvNameInstanceDataDir) promDir := filepath.Join(dataDir, "prometheus") - monitor := newMonitor() - port, cmd, err := monitor.startMonitor(ctx, options.version, options.host, promDir) + monitor, err := newMonitor(ctx, options.version, options.host, promDir) if err != nil { - return nil, nil, nil, err + return nil, nil, err } - p.monitor = monitor monitorInfo.IP = options.host monitorInfo.BinaryPath = promDir - monitorInfo.Port = port + monitorInfo.Port = monitor.port - monitorCmd := cmd - go func() { - log, err := os.OpenFile(filepath.Join(promDir, "prom.log"), os.O_WRONLY|os.O_APPEND|os.O_CREATE, os.ModePerm) - if err != nil { - fmt.Println("Monitor system start failed", err) - return - } - defer log.Close() + // start the monitor cmd. + log, err := os.OpenFile(filepath.Join(promDir, "prom.log"), os.O_WRONLY|os.O_APPEND|os.O_CREATE, os.ModePerm) + if err != nil { + return nil, nil, errors.AddStack(err) + } + defer log.Close() - cmd.Stderr = log - cmd.Stdout = os.Stdout + monitor.cmd.Stderr = log + monitor.cmd.Stdout = os.Stdout - // fmt.Println("Start Prometheus instance...") - if err := cmd.Start(); err != nil { - fmt.Println("Monitor system start failed", err) - return - } - }() + if err := monitor.cmd.Start(); err != nil { + return nil, nil, errors.AddStack(err) + } + + return monitor, monitorInfo, nil +} +// return not error iff the Cmd is started successfully. +func (p *Playground) bootGrafana(ctx context.Context, env *environment.Environment, monitorInfo *MonitorInfo) (*grafana, error) { // set up grafana + options := p.bootOptions if err := installIfMissing(env.Profile(), "grafana", options.version); err != nil { - return nil, nil, nil, errors.AddStack(err) + return nil, errors.AddStack(err) } installPath, err := env.Profile().ComponentInstalledPath("grafana", v0manifest.Version(options.version)) if err != nil { - return nil, nil, nil, errors.AddStack(err) + return nil, errors.AddStack(err) } - dataDir = os.Getenv(localdata.EnvNameInstanceDataDir) + dataDir := os.Getenv(localdata.EnvNameInstanceDataDir) grafanaDir := filepath.Join(dataDir, "grafana") - cmd = exec.Command("cp", "-r", installPath, grafanaDir) + cmd := exec.Command("cp", "-r", installPath, grafanaDir) err = cmd.Run() if err != nil { - return nil, nil, nil, errors.AddStack(err) + return nil, errors.AddStack(err) } dashboardDir := filepath.Join(grafanaDir, "dashboards") err = os.MkdirAll(dashboardDir, 0755) if err != nil { - return nil, nil, nil, errors.AddStack(err) + return nil, errors.AddStack(err) } // mv {grafanaDir}/*.json {grafanaDir}/dashboards/ @@ -1039,22 +1042,22 @@ func (p *Playground) bootMonitor(ctx context.Context, env *environment.Environme return nil }) if err != nil { - return nil, nil, nil, errors.AddStack(err) + return nil, errors.AddStack(err) } err = replaceDatasource(dashboardDir, clusterName) if err != nil { - return nil, nil, nil, errors.AddStack(err) + return nil, errors.AddStack(err) } grafana := newGrafana(options.version, options.host) // fmt.Println("Start Grafana instance...") err = grafana.start(ctx, grafanaDir, fmt.Sprintf("http://%s:%d", monitorInfo.IP, monitorInfo.Port)) if err != nil { - return nil, nil, nil, errors.AddStack(err) + return nil, errors.AddStack(err) } - return monitorCmd, monitorInfo, grafana, nil + return grafana, nil } func logIfErr(err error) { diff --git a/pkg/utils/exec.go b/pkg/utils/exec.go deleted file mode 100644 index 3458b97544..0000000000 --- a/pkg/utils/exec.go +++ /dev/null @@ -1,48 +0,0 @@ -// Copyright 2020 PingCAP, Inc. -// -// 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, -// See the License for the specific language governing permissions and -// limitations under the License. - -package utils - -import ( - "context" - "errors" - "os/exec" -) - -// ErrorWaitTimeout is used to represent timeout of a command -// Example: -// _ = syscall.Kill(cmd.Process.Pid, syscall.SIGKILL) -// if err := WaitContext(context.WithTimeout(context.Background(), 3), cmd); err == ErrorWaitTimeout { -// // Do something -// } -var ErrorWaitTimeout = errors.New("wait command timeout") - -// WaitContext wrap cmd.Wait with context -func WaitContext(ctx context.Context, cmd *exec.Cmd) error { - // We use cmd.Process.Wait instead of cmd.Wait because cmd.Wait is not reenterable - c := make(chan error, 1) - go func() { - if cmd == nil || cmd.Process == nil { - c <- nil - } else { - _, err := cmd.Process.Wait() - c <- err - } - }() - select { - case <-ctx.Done(): - return ErrorWaitTimeout - case err := <-c: - return err - } -} diff --git a/tests/tiup-playground/test_playground.sh b/tests/tiup-playground/test_playground.sh old mode 100644 new mode 100755 index 416e4b2482..74340027bc --- a/tests/tiup-playground/test_playground.sh +++ b/tests/tiup-playground/test_playground.sh @@ -1,9 +1,10 @@ #!/usr/bin/env bash +set -eu + TEST_DIR=$(cd "$(dirname "$0")"; pwd) TMP_DIR=$TEST_DIR/_tmp -rm -rf $TMP_DIR/data # Profile home directory mkdir -p $TMP_DIR/home/bin/ @@ -11,8 +12,9 @@ export TIUP_HOME=$TMP_DIR/home curl https://tiup-mirrors.pingcap.com/root.json -o $TMP_DIR/home/bin/root.json # Prepare data directory -mkdir -p $TMP_DIR/data -export TIUP_INSTANCE_DATA_DIR=$TMP_DIR/data +rm -rf $TIUP_HOME/data +mkdir -p $TIUP_HOME/data +export TIUP_INSTANCE_DATA_DIR=$TIUP_HOME/data/test_play mkdir -p $TEST_DIR/cover @@ -25,6 +27,74 @@ function tiup-playground() { fi } -export GO_FAILPOINTS=github.com/pingcap/tiup/components/playground/terminateEarly=return +# usage: check_tidb_num 1 +# make sure the tidb number is 1 or other specified number +function check_tidb_num() { + mustbe=$1 + num=$(tiup-playground display | grep "tidb" | wc -l | sed 's/ //g') + if [ "$num" != "$mustbe" ]; then + echo "unexpected tidb instance number: $num" + tiup-playground display + fi +} + +function kill_all() { + killall -9 tidb-server || true + killall -9 tikv-server || true + killall -9 pd-server || true + killall -9 tiflash || true + killall -9 grafana-server || true + killall -9 tiup-playground || true + killall -9 prometheus || true +} + +outfile=/tmp/tiup-playground-test.out +tiup-playground v4.0.4 --tiflash 0 > $outfile 2>&1 & + + +trap "kill_all > /dev/null 2>&1" EXIT + +# wait start cluster successfully +timeout 300 grep -q "CLUSTER START SUCCESSFULLY" <(tail -f $outfile) +# playground dump this at the wd +sleep 3 +mv ./dsn $TIUP_INSTANCE_DATA_DIR/ +mv ./port $TIUP_INSTANCE_DATA_DIR/ + +tiup-playground display | grep -qv "exit" +tiup-playground scale-out --db 2 +sleep 5 + +# 1(init) + 2(scale-out) +check_tidb_num 3 + +# get pid of one tidb instance and scale-in +pid=`tiup-playground display | grep "tidb" | awk 'NR==1 {print $1}'` +tiup-playground scale-in --pid $pid + +sleep 5 +check_tidb_num 2 + +# get pid of one tidb instance and kill it +pid=`tiup-playground display | grep "tidb" | awk 'NR==1 {print $1}'` +kill -9 $pid +sleep 5 + +echo "*display after kill -9:" +tiup-playground display +tiup-playground display | grep "signal: killed" | wc -l | grep -q "1" + +# get pid of one tidb instance and kill it +pid=`tiup-playground display | grep "tidb" | grep -v "killed" | awk 'NR==1 {print $1}'` +kill $pid +sleep 5 +echo "*display after kill:" +tiup-playground display +tiup-playground display | grep "exit" | wc -l | grep -q "1" + +killall -2 tiup-playground.test || killall -2 tiup-playground +wait + +echo "\033[0;36m<<< Run all test success >>>\033[0m" + -tiup-playground v4.0.0