Sfoglia il codice sorgente

Merge pull request #7087 from sinsharat/make_etcd-runner_command_compliant

etcd-runner: make command compliant
Anthony Romano 9 anni fa
parent
commit
fd72ecfe92

+ 25 - 4
tools/functional-tester/etcd-runner/election.go → tools/functional-tester/etcd-runner/command/election_command.go

@@ -12,28 +12,49 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-package main
+package command
 
 import (
 	"context"
+	"errors"
 	"fmt"
 
 	"github.com/coreos/etcd/clientv3/concurrency"
+	"github.com/spf13/cobra"
 )
 
-func runElection(getClient getClientFunc, rounds int) {
-	rcs := make([]roundClient, 15)
+// NewElectionCommand returns the cobra command for "election runner".
+func NewElectionCommand() *cobra.Command {
+	cmd := &cobra.Command{
+		Use:   "election",
+		Short: "Performs election operation",
+		Run:   runElectionFunc,
+	}
+	cmd.Flags().IntVar(&rounds, "rounds", 100, "number of rounds to run")
+	cmd.Flags().IntVar(&totalClientConnections, "total-client-connections", 10, "total number of client connections")
+	return cmd
+}
+
+func runElectionFunc(cmd *cobra.Command, args []string) {
+	if len(args) > 0 {
+		ExitWithError(ExitBadArgs, errors.New("election does not take any argument"))
+	}
+
+	rcs := make([]roundClient, totalClientConnections)
 	validatec, releasec := make(chan struct{}, len(rcs)), make(chan struct{}, len(rcs))
 	for range rcs {
 		releasec <- struct{}{}
 	}
 
+	eps := endpointsFromFlag(cmd)
+	dialTimeout := dialTimeoutFromCmd(cmd)
+
 	for i := range rcs {
 		v := fmt.Sprintf("%d", i)
 		observedLeader := ""
 		validateWaiters := 0
 
-		rcs[i].c = getClient()
+		rcs[i].c = newClient(eps, dialTimeout)
 		var (
 			s   *concurrency.Session
 			err error

+ 42 - 0
tools/functional-tester/etcd-runner/command/error.go

@@ -0,0 +1,42 @@
+// Copyright 2015 The etcd 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 command
+
+import (
+	"fmt"
+	"os"
+
+	"github.com/coreos/etcd/client"
+)
+
+const (
+	// http://tldp.org/LDP/abs/html/exitcodes.html
+	ExitSuccess = iota
+	ExitError
+	ExitBadConnection
+	ExitInvalidInput // for txn, watch command
+	ExitBadFeature   // provided a valid flag with an unsupported value
+	ExitInterrupted
+	ExitIO
+	ExitBadArgs = 128
+)
+
+func ExitWithError(code int, err error) {
+	fmt.Fprintln(os.Stderr, "Error: ", err)
+	if cerr, ok := err.(*client.ClusterError); ok {
+		fmt.Fprintln(os.Stderr, cerr.Detail())
+	}
+	os.Exit(code)
+}

+ 130 - 0
tools/functional-tester/etcd-runner/command/global.go

@@ -0,0 +1,130 @@
+// Copyright 2016 The etcd 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 command
+
+import (
+	"fmt"
+	"log"
+	"sync"
+	"time"
+
+	"github.com/coreos/etcd/clientv3"
+
+	"github.com/spf13/cobra"
+)
+
+var (
+	rounds                 int           // total number of rounds the operation needs to be performed
+	totalClientConnections int           // total number of client connections to be made with server
+	noOfPrefixes           int           // total number of prefixes which will be watched upon
+	watchPerPrefix         int           // number of watchers per prefix
+	reqRate                int           // put request per second
+	totalKeys              int           // total number of keys for operation
+	runningTime            time.Duration // time for which operation should be performed
+)
+
+// GlobalFlags are flags that defined globally
+// and are inherited to all sub-commands.
+type GlobalFlags struct {
+	Endpoints   []string
+	DialTimeout time.Duration
+}
+
+type roundClient struct {
+	c        *clientv3.Client
+	progress int
+	acquire  func() error
+	validate func() error
+	release  func() error
+}
+
+func newClient(eps []string, timeout time.Duration) *clientv3.Client {
+	c, err := clientv3.New(clientv3.Config{
+		Endpoints:   eps,
+		DialTimeout: time.Duration(timeout) * time.Second,
+	})
+	if err != nil {
+		log.Fatal(err)
+	}
+	return c
+}
+
+func doRounds(rcs []roundClient, rounds int) {
+	var mu sync.Mutex
+	var wg sync.WaitGroup
+
+	wg.Add(len(rcs))
+	finished := make(chan struct{}, 0)
+	for i := range rcs {
+		go func(rc *roundClient) {
+			defer wg.Done()
+			for rc.progress < rounds {
+				for rc.acquire() != nil { /* spin */
+				}
+
+				mu.Lock()
+				if err := rc.validate(); err != nil {
+					log.Fatal(err)
+				}
+				mu.Unlock()
+
+				time.Sleep(10 * time.Millisecond)
+				rc.progress++
+				finished <- struct{}{}
+
+				mu.Lock()
+				for rc.release() != nil {
+					mu.Unlock()
+					mu.Lock()
+				}
+				mu.Unlock()
+			}
+		}(&rcs[i])
+	}
+
+	start := time.Now()
+	for i := 1; i < len(rcs)*rounds+1; i++ {
+		select {
+		case <-finished:
+			if i%100 == 0 {
+				fmt.Printf("finished %d, took %v\n", i, time.Since(start))
+				start = time.Now()
+			}
+		case <-time.After(time.Minute):
+			log.Panic("no progress after 1 minute!")
+		}
+	}
+	wg.Wait()
+
+	for _, rc := range rcs {
+		rc.c.Close()
+	}
+}
+
+func endpointsFromFlag(cmd *cobra.Command) []string {
+	endpoints, err := cmd.Flags().GetStringSlice("endpoints")
+	if err != nil {
+		ExitWithError(ExitError, err)
+	}
+	return endpoints
+}
+
+func dialTimeoutFromCmd(cmd *cobra.Command) time.Duration {
+	dialTimeout, err := cmd.Flags().GetDuration("dial-timeout")
+	if err != nil {
+		ExitWithError(ExitError, err)
+	}
+	return dialTimeout
+}

+ 21 - 3
tools/functional-tester/etcd-runner/lease_renewer.go → tools/functional-tester/etcd-runner/command/lease_renewer_command.go

@@ -12,21 +12,39 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-package main
+package command
 
 import (
 	"context"
+	"errors"
 	"fmt"
 	"log"
 	"time"
 
 	"github.com/coreos/etcd/clientv3"
+	"github.com/spf13/cobra"
 	"google.golang.org/grpc"
 	"google.golang.org/grpc/codes"
 )
 
-func runLeaseRenewer(getClient getClientFunc) {
-	c := getClient()
+// NewLeaseRenewerCommand returns the cobra command for "lease-renewer runner".
+func NewLeaseRenewerCommand() *cobra.Command {
+	cmd := &cobra.Command{
+		Use:   "lease-renewer",
+		Short: "Performs lease renew operation",
+		Run:   runLeaseRenewerFunc,
+	}
+	return cmd
+}
+
+func runLeaseRenewerFunc(cmd *cobra.Command, args []string) {
+	if len(args) > 0 {
+		ExitWithError(ExitBadArgs, errors.New("lease-renewer does not take any argument"))
+	}
+
+	eps := endpointsFromFlag(cmd)
+	dialTimeout := dialTimeoutFromCmd(cmd)
+	c := newClient(eps, dialTimeout)
 	ctx := context.Background()
 
 	for {

+ 29 - 5
tools/functional-tester/etcd-runner/lock_racer.go → tools/functional-tester/etcd-runner/command/lock_racer_command.go

@@ -12,25 +12,49 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-package main
+package command
 
 import (
 	"context"
+	"errors"
 	"fmt"
 
 	"github.com/coreos/etcd/clientv3/concurrency"
+	"github.com/spf13/cobra"
 )
 
-func runRacer(getClient getClientFunc, round int) {
-	rcs := make([]roundClient, 15)
+// NewLockRacerCommand returns the cobra command for "lock-racer runner".
+func NewLockRacerCommand() *cobra.Command {
+	cmd := &cobra.Command{
+		Use:   "lock-racer",
+		Short: "Performs lock race operation",
+		Run:   runRacerFunc,
+	}
+	cmd.Flags().IntVar(&rounds, "rounds", 100, "number of rounds to run")
+	cmd.Flags().IntVar(&totalClientConnections, "total-client-connections", 10, "total number of client connections")
+	return cmd
+}
+
+func runRacerFunc(cmd *cobra.Command, args []string) {
+	if len(args) > 0 {
+		ExitWithError(ExitBadArgs, errors.New("lock-racer does not take any argument"))
+	}
+
+	rcs := make([]roundClient, totalClientConnections)
 	ctx := context.Background()
 	cnt := 0
+
+	eps := endpointsFromFlag(cmd)
+	dialTimeout := dialTimeoutFromCmd(cmd)
+
 	for i := range rcs {
-		rcs[i].c = getClient()
 		var (
 			s   *concurrency.Session
 			err error
 		)
+
+		rcs[i].c = newClient(eps, dialTimeout)
+
 		for {
 			s, err = concurrency.NewSession(rcs[i].c)
 			if err == nil {
@@ -53,5 +77,5 @@ func runRacer(getClient getClientFunc, round int) {
 			return nil
 		}
 	}
-	doRounds(rcs, round)
+	doRounds(rcs, rounds)
 }

+ 35 - 15
tools/functional-tester/etcd-runner/watcher.go → tools/functional-tester/etcd-runner/command/watch_command.go

@@ -12,10 +12,11 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
-package main
+package command
 
 import (
 	"context"
+	"errors"
 	"fmt"
 	"log"
 	"sync"
@@ -23,29 +24,48 @@ import (
 
 	"github.com/coreos/etcd/clientv3"
 	"github.com/coreos/etcd/pkg/stringutil"
+	"github.com/spf13/cobra"
 	"golang.org/x/time/rate"
 )
 
-func runWatcher(getClient getClientFunc, limit int) {
+// NewWatchCommand returns the cobra command for "watcher runner".
+func NewWatchCommand() *cobra.Command {
+	cmd := &cobra.Command{
+		Use:   "watcher",
+		Short: "Performs watch operation",
+		Run:   runWatcherFunc,
+	}
+	cmd.Flags().IntVar(&rounds, "rounds", 100, "number of rounds to run")
+	cmd.Flags().DurationVar(&runningTime, "running-time", 60, "number of seconds to run")
+	cmd.Flags().IntVar(&noOfPrefixes, "total-prefixes", 10, "total no of prefixes to use")
+	cmd.Flags().IntVar(&watchPerPrefix, "watch-per-prefix", 10, "number of watchers per prefix")
+	cmd.Flags().IntVar(&reqRate, "req-rate", 30, "rate at which put request will be performed")
+	cmd.Flags().IntVar(&totalKeys, "total-keys", 1000, "total number of keys to watch")
+	return cmd
+}
+
+func runWatcherFunc(cmd *cobra.Command, args []string) {
+	if len(args) > 0 {
+		ExitWithError(ExitBadArgs, errors.New("watcher does not take any argument"))
+	}
+
 	ctx := context.Background()
-	for round := 0; round < limit; round++ {
+	for round := 0; round < rounds; round++ {
 		fmt.Println("round", round)
-		performWatchOnPrefixes(ctx, getClient, round)
+		performWatchOnPrefixes(ctx, cmd, round)
 	}
 }
 
-func performWatchOnPrefixes(ctx context.Context, getClient getClientFunc, round int) {
-	runningTime := 60 * time.Second // time for which operation should be performed
-	noOfPrefixes := 36              // total number of prefixes which will be watched upon
-	watchPerPrefix := 10            // number of watchers per prefix
-	reqRate := 30                   // put request per second
-	keyPrePrefix := 30              // max number of keyPrePrefixs for put operation
-
+func performWatchOnPrefixes(ctx context.Context, cmd *cobra.Command, round int) {
+	keyPerPrefix := totalKeys / noOfPrefixes
 	prefixes := stringutil.UniqueStrings(5, noOfPrefixes)
-	keys := stringutil.RandomStrings(10, keyPrePrefix)
+	keys := stringutil.RandomStrings(10, keyPerPrefix)
 
 	roundPrefix := fmt.Sprintf("%16x", round)
 
+	eps := endpointsFromFlag(cmd)
+	dialTimeout := dialTimeoutFromCmd(cmd)
+
 	var (
 		revision int64
 		wg       sync.WaitGroup
@@ -53,7 +73,7 @@ func performWatchOnPrefixes(ctx context.Context, getClient getClientFunc, round
 		err      error
 	)
 
-	client := getClient()
+	client := newClient(eps, dialTimeout)
 	defer client.Close()
 
 	gr, err = getKey(ctx, client, "non-existent")
@@ -62,7 +82,7 @@ func performWatchOnPrefixes(ctx context.Context, getClient getClientFunc, round
 	}
 	revision = gr.Header.Revision
 
-	ctxt, cancel := context.WithDeadline(ctx, time.Now().Add(runningTime))
+	ctxt, cancel := context.WithDeadline(ctx, time.Now().Add(runningTime*time.Second))
 	defer cancel()
 
 	// generate and put keys in cluster
@@ -89,7 +109,7 @@ func performWatchOnPrefixes(ctx context.Context, getClient getClientFunc, round
 
 	for _, prefix := range prefixes {
 		for j := 0; j < watchPerPrefix; j++ {
-			rc := getClient()
+			rc := newClient(eps, dialTimeout)
 			rcs = append(rcs, rc)
 
 			watchPrefix := roundPrefix + "-" + prefix

+ 174 - 0
tools/functional-tester/etcd-runner/help.go

@@ -0,0 +1,174 @@
+// Copyright 2015 The etcd 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.
+
+// copied from https://github.com/coreos/rkt/blob/master/rkt/help.go
+
+package main
+
+import (
+	"bytes"
+	"fmt"
+	"io"
+	"os"
+	"strings"
+	"text/tabwriter"
+	"text/template"
+
+	"github.com/coreos/etcd/version"
+	"github.com/spf13/cobra"
+	"github.com/spf13/pflag"
+)
+
+var (
+	commandUsageTemplate *template.Template
+	templFuncs           = template.FuncMap{
+		"descToLines": func(s string) []string {
+			// trim leading/trailing whitespace and split into slice of lines
+			return strings.Split(strings.Trim(s, "\n\t "), "\n")
+		},
+		"cmdName": func(cmd *cobra.Command, startCmd *cobra.Command) string {
+			parts := []string{cmd.Name()}
+			for cmd.HasParent() && cmd.Parent().Name() != startCmd.Name() {
+				cmd = cmd.Parent()
+				parts = append([]string{cmd.Name()}, parts...)
+			}
+			return strings.Join(parts, " ")
+		},
+	}
+)
+
+func init() {
+	commandUsage := `
+{{ $cmd := .Cmd }}\
+{{ $cmdname := cmdName .Cmd .Cmd.Root }}\
+NAME:
+{{ if not .Cmd.HasParent }}\
+{{printf "\t%s - %s" .Cmd.Name .Cmd.Short}}
+{{else}}\
+{{printf "\t%s - %s" $cmdname .Cmd.Short}}
+{{end}}\
+
+USAGE:
+{{printf "\t%s" .Cmd.UseLine}}
+{{ if not .Cmd.HasParent }}\
+
+VERSION:
+{{printf "\t%s" .Version}}
+{{end}}\
+{{if .Cmd.HasSubCommands}}\
+
+API VERSION:
+{{printf "\t%s" .APIVersion}}
+{{end}}\
+{{if .Cmd.HasSubCommands}}\
+
+
+COMMANDS:
+{{range .SubCommands}}\
+{{ $cmdname := cmdName . $cmd }}\
+{{ if .Runnable }}\
+{{printf "\t%s\t%s" $cmdname .Short}}
+{{end}}\
+{{end}}\
+{{end}}\
+{{ if .Cmd.Long }}\
+
+DESCRIPTION:
+{{range $line := descToLines .Cmd.Long}}{{printf "\t%s" $line}}
+{{end}}\
+{{end}}\
+{{if .Cmd.HasLocalFlags}}\
+
+OPTIONS:
+{{.LocalFlags}}\
+{{end}}\
+{{if .Cmd.HasInheritedFlags}}\
+
+GLOBAL OPTIONS:
+{{.GlobalFlags}}\
+{{end}}
+`[1:]
+
+	commandUsageTemplate = template.Must(template.New("command_usage").Funcs(templFuncs).Parse(strings.Replace(commandUsage, "\\\n", "", -1)))
+}
+
+func etcdFlagUsages(flagSet *pflag.FlagSet) string {
+	x := new(bytes.Buffer)
+
+	flagSet.VisitAll(func(flag *pflag.Flag) {
+		if len(flag.Deprecated) > 0 {
+			return
+		}
+		format := ""
+		if len(flag.Shorthand) > 0 {
+			format = "  -%s, --%s"
+		} else {
+			format = "   %s   --%s"
+		}
+		if len(flag.NoOptDefVal) > 0 {
+			format = format + "["
+		}
+		if flag.Value.Type() == "string" {
+			// put quotes on the value
+			format = format + "=%q"
+		} else {
+			format = format + "=%s"
+		}
+		if len(flag.NoOptDefVal) > 0 {
+			format = format + "]"
+		}
+		format = format + "\t%s\n"
+		shorthand := flag.Shorthand
+		fmt.Fprintf(x, format, shorthand, flag.Name, flag.DefValue, flag.Usage)
+	})
+
+	return x.String()
+}
+
+func getSubCommands(cmd *cobra.Command) []*cobra.Command {
+	var subCommands []*cobra.Command
+	for _, subCmd := range cmd.Commands() {
+		subCommands = append(subCommands, subCmd)
+		subCommands = append(subCommands, getSubCommands(subCmd)...)
+	}
+	return subCommands
+}
+
+func usageFunc(cmd *cobra.Command) error {
+	subCommands := getSubCommands(cmd)
+	tabOut := getTabOutWithWriter(os.Stdout)
+	commandUsageTemplate.Execute(tabOut, struct {
+		Cmd         *cobra.Command
+		LocalFlags  string
+		GlobalFlags string
+		SubCommands []*cobra.Command
+		Version     string
+		APIVersion  string
+	}{
+		cmd,
+		etcdFlagUsages(cmd.LocalFlags()),
+		etcdFlagUsages(cmd.InheritedFlags()),
+		subCommands,
+		version.Version,
+		version.APIVersion,
+	})
+	tabOut.Flush()
+	return nil
+}
+
+func getTabOutWithWriter(writer io.Writer) *tabwriter.Writer {
+	aTabOut := new(tabwriter.Writer)
+	aTabOut.Init(writer, 0, 8, 1, '\t', 0)
+	return aTabOut
+}

+ 39 - 96
tools/functional-tester/etcd-runner/main.go

@@ -12,121 +12,64 @@
 // See the License for the specific language governing permissions and
 // limitations under the License.
 
+// etcd-runner is a command line application that performs tests on etcd.
 package main
 
 import (
-	"flag"
-	"fmt"
 	"log"
-	"math/rand"
-	"os"
-	"strings"
-	"sync"
 	"time"
 
-	"github.com/coreos/etcd/clientv3"
+	"github.com/coreos/etcd/tools/functional-tester/etcd-runner/command"
+	"github.com/spf13/cobra"
 )
 
-func init() {
-	rand.Seed(time.Now().UTC().UnixNano())
-}
-
-func main() {
-	log.SetFlags(log.Lmicroseconds)
-
-	endpointStr := flag.String("endpoints", "localhost:2379", "endpoints of etcd cluster")
-	mode := flag.String("mode", "watcher", "test mode (election, lock-racer, lease-renewer, watcher)")
-	round := flag.Int("rounds", 100, "number of rounds to run")
-	clientTimeout := flag.Int("client-timeout", 60, "max timeout seconds for a client to get connection")
-	flag.Parse()
+const (
+	cliName        = "etcd-runner"
+	cliDescription = "Stress tests using clientv3 functionality.."
 
-	eps := strings.Split(*endpointStr, ",")
+	defaultDialTimeout = 2 * time.Second
+)
 
-	getClient := func() *clientv3.Client { return newClient(eps, *clientTimeout) }
+var (
+	globalFlags = command.GlobalFlags{}
+)
 
-	switch *mode {
-	case "election":
-		runElection(getClient, *round)
-	case "lock-racer":
-		runRacer(getClient, *round)
-	case "lease-renewer":
-		runLeaseRenewer(getClient)
-	case "watcher":
-		runWatcher(getClient, *round)
-	default:
-		fmt.Fprintf(os.Stderr, "unsupported mode %v\n", *mode)
+var (
+	rootCmd = &cobra.Command{
+		Use:        cliName,
+		Short:      cliDescription,
+		SuggestFor: []string{"etcd-runner"},
 	}
-}
-
-type getClientFunc func() *clientv3.Client
+)
 
-func newClient(eps []string, timeout int) *clientv3.Client {
-	c, err := clientv3.New(clientv3.Config{
-		Endpoints:   eps,
-		DialTimeout: time.Duration(timeout) * time.Second,
-	})
-	if err != nil {
-		log.Fatal(err)
-	}
-	return c
+func init() {
+	log.SetFlags(log.Lmicroseconds)
+	rootCmd.PersistentFlags().StringSliceVar(&globalFlags.Endpoints, "endpoints", []string{"127.0.0.1:2379"}, "gRPC endpoints")
+	rootCmd.PersistentFlags().DurationVar(&globalFlags.DialTimeout, "dial-timeout", defaultDialTimeout, "dial timeout for client connections")
+
+	rootCmd.AddCommand(
+		command.NewElectionCommand(),
+		command.NewLeaseRenewerCommand(),
+		command.NewLockRacerCommand(),
+		command.NewWatchCommand(),
+	)
 }
 
-type roundClient struct {
-	c        *clientv3.Client
-	progress int
-	acquire  func() error
-	validate func() error
-	release  func() error
+func init() {
+	cobra.EnablePrefixMatching = true
 }
 
-func doRounds(rcs []roundClient, rounds int) {
-	var mu sync.Mutex
-	var wg sync.WaitGroup
-
-	wg.Add(len(rcs))
-	finished := make(chan struct{}, 0)
-	for i := range rcs {
-		go func(rc *roundClient) {
-			defer wg.Done()
-			for rc.progress < rounds {
-				for rc.acquire() != nil { /* spin */
-				}
-
-				mu.Lock()
-				if err := rc.validate(); err != nil {
-					log.Fatal(err)
-				}
-				mu.Unlock()
+func Start() {
+	rootCmd.SetUsageFunc(usageFunc)
 
-				time.Sleep(10 * time.Millisecond)
-				rc.progress++
-				finished <- struct{}{}
+	// Make help just show the usage
+	rootCmd.SetHelpTemplate(`{{.UsageString}}`)
 
-				mu.Lock()
-				for rc.release() != nil {
-					mu.Unlock()
-					mu.Lock()
-				}
-				mu.Unlock()
-			}
-		}(&rcs[i])
+	if err := rootCmd.Execute(); err != nil {
+		command.ExitWithError(command.ExitError, err)
 	}
+}
 
-	start := time.Now()
-	for i := 1; i < len(rcs)*rounds+1; i++ {
-		select {
-		case <-finished:
-			if i%100 == 0 {
-				fmt.Printf("finished %d, took %v\n", i, time.Since(start))
-				start = time.Now()
-			}
-		case <-time.After(time.Minute):
-			log.Panic("no progress after 1 minute!")
-		}
-	}
-	wg.Wait()
-
-	for _, rc := range rcs {
-		rc.c.Close()
-	}
+func main() {
+	Start()
 }