etcd-runner: make command compliant
This commit is contained in:
@ -12,28 +12,49 @@
|
|||||||
// See the License for the specific language governing permissions and
|
// See the License for the specific language governing permissions and
|
||||||
// limitations under the License.
|
// limitations under the License.
|
||||||
|
|
||||||
package main
|
package command
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
"errors"
|
||||||
"fmt"
|
"fmt"
|
||||||
|
|
||||||
"github.com/coreos/etcd/clientv3/concurrency"
|
"github.com/coreos/etcd/clientv3/concurrency"
|
||||||
|
"github.com/spf13/cobra"
|
||||||
)
|
)
|
||||||
|
|
||||||
func runElection(getClient getClientFunc, rounds int) {
|
// NewElectionCommand returns the cobra command for "election runner".
|
||||||
rcs := make([]roundClient, 15)
|
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))
|
validatec, releasec := make(chan struct{}, len(rcs)), make(chan struct{}, len(rcs))
|
||||||
for range rcs {
|
for range rcs {
|
||||||
releasec <- struct{}{}
|
releasec <- struct{}{}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
eps := endpointsFromFlag(cmd)
|
||||||
|
dialTimeout := dialTimeoutFromCmd(cmd)
|
||||||
|
|
||||||
for i := range rcs {
|
for i := range rcs {
|
||||||
v := fmt.Sprintf("%d", i)
|
v := fmt.Sprintf("%d", i)
|
||||||
observedLeader := ""
|
observedLeader := ""
|
||||||
validateWaiters := 0
|
validateWaiters := 0
|
||||||
|
|
||||||
rcs[i].c = getClient()
|
rcs[i].c = newClient(eps, dialTimeout)
|
||||||
var (
|
var (
|
||||||
s *concurrency.Session
|
s *concurrency.Session
|
||||||
err error
|
err error
|
42
tools/functional-tester/etcd-runner/command/error.go
Normal file
42
tools/functional-tester/etcd-runner/command/error.go
Normal file
@ -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)
|
||||||
|
}
|
125
tools/functional-tester/etcd-runner/command/global.go
Normal file
125
tools/functional-tester/etcd-runner/command/global.go
Normal file
@ -0,0 +1,125 @@
|
|||||||
|
// 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
|
||||||
|
totalClientConnections int
|
||||||
|
)
|
||||||
|
|
||||||
|
// 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
|
||||||
|
}
|
@ -12,21 +12,39 @@
|
|||||||
// See the License for the specific language governing permissions and
|
// See the License for the specific language governing permissions and
|
||||||
// limitations under the License.
|
// limitations under the License.
|
||||||
|
|
||||||
package main
|
package command
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
"errors"
|
||||||
"fmt"
|
"fmt"
|
||||||
"log"
|
"log"
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/coreos/etcd/clientv3"
|
"github.com/coreos/etcd/clientv3"
|
||||||
|
"github.com/spf13/cobra"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
"google.golang.org/grpc/codes"
|
"google.golang.org/grpc/codes"
|
||||||
)
|
)
|
||||||
|
|
||||||
func runLeaseRenewer(getClient getClientFunc) {
|
// NewLeaseRenewerCommand returns the cobra command for "lease-renewer runner".
|
||||||
c := getClient()
|
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()
|
ctx := context.Background()
|
||||||
|
|
||||||
for {
|
for {
|
@ -12,25 +12,49 @@
|
|||||||
// See the License for the specific language governing permissions and
|
// See the License for the specific language governing permissions and
|
||||||
// limitations under the License.
|
// limitations under the License.
|
||||||
|
|
||||||
package main
|
package command
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
"errors"
|
||||||
"fmt"
|
"fmt"
|
||||||
|
|
||||||
"github.com/coreos/etcd/clientv3/concurrency"
|
"github.com/coreos/etcd/clientv3/concurrency"
|
||||||
|
"github.com/spf13/cobra"
|
||||||
)
|
)
|
||||||
|
|
||||||
func runRacer(getClient getClientFunc, round int) {
|
// NewLockRacerCommand returns the cobra command for "lock-racer runner".
|
||||||
rcs := make([]roundClient, 15)
|
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()
|
ctx := context.Background()
|
||||||
cnt := 0
|
cnt := 0
|
||||||
|
|
||||||
|
eps := endpointsFromFlag(cmd)
|
||||||
|
dialTimeout := dialTimeoutFromCmd(cmd)
|
||||||
|
|
||||||
for i := range rcs {
|
for i := range rcs {
|
||||||
rcs[i].c = getClient()
|
|
||||||
var (
|
var (
|
||||||
s *concurrency.Session
|
s *concurrency.Session
|
||||||
err error
|
err error
|
||||||
)
|
)
|
||||||
|
|
||||||
|
rcs[i].c = newClient(eps, dialTimeout)
|
||||||
|
|
||||||
for {
|
for {
|
||||||
s, err = concurrency.NewSession(rcs[i].c)
|
s, err = concurrency.NewSession(rcs[i].c)
|
||||||
if err == nil {
|
if err == nil {
|
||||||
@ -53,5 +77,5 @@ func runRacer(getClient getClientFunc, round int) {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
doRounds(rcs, round)
|
doRounds(rcs, rounds)
|
||||||
}
|
}
|
@ -12,10 +12,11 @@
|
|||||||
// See the License for the specific language governing permissions and
|
// See the License for the specific language governing permissions and
|
||||||
// limitations under the License.
|
// limitations under the License.
|
||||||
|
|
||||||
package main
|
package command
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"context"
|
"context"
|
||||||
|
"errors"
|
||||||
"fmt"
|
"fmt"
|
||||||
"log"
|
"log"
|
||||||
"sync"
|
"sync"
|
||||||
@ -23,18 +24,34 @@ import (
|
|||||||
|
|
||||||
"github.com/coreos/etcd/clientv3"
|
"github.com/coreos/etcd/clientv3"
|
||||||
"github.com/coreos/etcd/pkg/stringutil"
|
"github.com/coreos/etcd/pkg/stringutil"
|
||||||
|
"github.com/spf13/cobra"
|
||||||
"golang.org/x/time/rate"
|
"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")
|
||||||
|
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()
|
ctx := context.Background()
|
||||||
for round := 0; round < limit; round++ {
|
for round := 0; round < rounds; round++ {
|
||||||
fmt.Println("round", round)
|
fmt.Println("round", round)
|
||||||
performWatchOnPrefixes(ctx, getClient, round)
|
performWatchOnPrefixes(ctx, cmd, round)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func performWatchOnPrefixes(ctx context.Context, getClient getClientFunc, round int) {
|
func performWatchOnPrefixes(ctx context.Context, cmd *cobra.Command, round int) {
|
||||||
runningTime := 60 * time.Second // time for which operation should be performed
|
runningTime := 60 * time.Second // time for which operation should be performed
|
||||||
noOfPrefixes := 36 // total number of prefixes which will be watched upon
|
noOfPrefixes := 36 // total number of prefixes which will be watched upon
|
||||||
watchPerPrefix := 10 // number of watchers per prefix
|
watchPerPrefix := 10 // number of watchers per prefix
|
||||||
@ -46,6 +63,9 @@ func performWatchOnPrefixes(ctx context.Context, getClient getClientFunc, round
|
|||||||
|
|
||||||
roundPrefix := fmt.Sprintf("%16x", round)
|
roundPrefix := fmt.Sprintf("%16x", round)
|
||||||
|
|
||||||
|
eps := endpointsFromFlag(cmd)
|
||||||
|
dialTimeout := dialTimeoutFromCmd(cmd)
|
||||||
|
|
||||||
var (
|
var (
|
||||||
revision int64
|
revision int64
|
||||||
wg sync.WaitGroup
|
wg sync.WaitGroup
|
||||||
@ -53,7 +73,7 @@ func performWatchOnPrefixes(ctx context.Context, getClient getClientFunc, round
|
|||||||
err error
|
err error
|
||||||
)
|
)
|
||||||
|
|
||||||
client := getClient()
|
client := newClient(eps, dialTimeout)
|
||||||
defer client.Close()
|
defer client.Close()
|
||||||
|
|
||||||
gr, err = getKey(ctx, client, "non-existent")
|
gr, err = getKey(ctx, client, "non-existent")
|
||||||
@ -89,7 +109,7 @@ func performWatchOnPrefixes(ctx context.Context, getClient getClientFunc, round
|
|||||||
|
|
||||||
for _, prefix := range prefixes {
|
for _, prefix := range prefixes {
|
||||||
for j := 0; j < watchPerPrefix; j++ {
|
for j := 0; j < watchPerPrefix; j++ {
|
||||||
rc := getClient()
|
rc := newClient(eps, dialTimeout)
|
||||||
rcs = append(rcs, rc)
|
rcs = append(rcs, rc)
|
||||||
|
|
||||||
watchPrefix := roundPrefix + "-" + prefix
|
watchPrefix := roundPrefix + "-" + prefix
|
174
tools/functional-tester/etcd-runner/help.go
Normal file
174
tools/functional-tester/etcd-runner/help.go
Normal file
@ -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
|
||||||
|
}
|
@ -12,121 +12,64 @@
|
|||||||
// See the License for the specific language governing permissions and
|
// See the License for the specific language governing permissions and
|
||||||
// limitations under the License.
|
// limitations under the License.
|
||||||
|
|
||||||
|
// etcd-runner is a command line application that performs tests on etcd.
|
||||||
package main
|
package main
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"flag"
|
|
||||||
"fmt"
|
|
||||||
"log"
|
"log"
|
||||||
"math/rand"
|
|
||||||
"os"
|
|
||||||
"strings"
|
|
||||||
"sync"
|
|
||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/coreos/etcd/clientv3"
|
"github.com/coreos/etcd/tools/functional-tester/etcd-runner/command"
|
||||||
|
"github.com/spf13/cobra"
|
||||||
|
)
|
||||||
|
|
||||||
|
const (
|
||||||
|
cliName = "etcdctl"
|
||||||
|
cliDescription = "A simple command line client for etcd3."
|
||||||
|
|
||||||
|
defaultDialTimeout = 2 * time.Second
|
||||||
|
)
|
||||||
|
|
||||||
|
var (
|
||||||
|
globalFlags = command.GlobalFlags{}
|
||||||
|
)
|
||||||
|
|
||||||
|
var (
|
||||||
|
rootCmd = &cobra.Command{
|
||||||
|
Use: cliName,
|
||||||
|
Short: cliDescription,
|
||||||
|
SuggestFor: []string{"etcdctl"},
|
||||||
|
}
|
||||||
)
|
)
|
||||||
|
|
||||||
func init() {
|
func init() {
|
||||||
rand.Seed(time.Now().UTC().UnixNano())
|
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(),
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
func init() {
|
||||||
|
cobra.EnablePrefixMatching = true
|
||||||
|
}
|
||||||
|
|
||||||
|
func Start() {
|
||||||
|
rootCmd.SetUsageFunc(usageFunc)
|
||||||
|
|
||||||
|
// Make help just show the usage
|
||||||
|
rootCmd.SetHelpTemplate(`{{.UsageString}}`)
|
||||||
|
|
||||||
|
if err := rootCmd.Execute(); err != nil {
|
||||||
|
command.ExitWithError(command.ExitError, err)
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
func main() {
|
func main() {
|
||||||
log.SetFlags(log.Lmicroseconds)
|
Start()
|
||||||
|
|
||||||
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()
|
|
||||||
|
|
||||||
eps := strings.Split(*endpointStr, ",")
|
|
||||||
|
|
||||||
getClient := func() *clientv3.Client { return newClient(eps, *clientTimeout) }
|
|
||||||
|
|
||||||
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)
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
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
|
|
||||||
}
|
|
||||||
|
|
||||||
type roundClient struct {
|
|
||||||
c *clientv3.Client
|
|
||||||
progress int
|
|
||||||
acquire func() error
|
|
||||||
validate func() error
|
|
||||||
release func() error
|
|
||||||
}
|
|
||||||
|
|
||||||
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()
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
Reference in New Issue
Block a user