mirror of
https://github.com/etcd-io/etcd.git
synced 2024-09-27 06:25:44 +00:00
Merge pull request #7812 from fanminshi/refactor_runner
etcd-runner: fix runner and minor refactoring.
This commit is contained in:
commit
60b9adc267
@ -20,32 +20,34 @@ import (
|
|||||||
"fmt"
|
"fmt"
|
||||||
|
|
||||||
"github.com/coreos/etcd/clientv3/concurrency"
|
"github.com/coreos/etcd/clientv3/concurrency"
|
||||||
|
|
||||||
"github.com/spf13/cobra"
|
"github.com/spf13/cobra"
|
||||||
)
|
)
|
||||||
|
|
||||||
// NewElectionCommand returns the cobra command for "election runner".
|
// NewElectionCommand returns the cobra command for "election runner".
|
||||||
func NewElectionCommand() *cobra.Command {
|
func NewElectionCommand() *cobra.Command {
|
||||||
cmd := &cobra.Command{
|
cmd := &cobra.Command{
|
||||||
Use: "election",
|
Use: "election [election name (defaults to 'elector')]",
|
||||||
Short: "Performs election operation",
|
Short: "Performs election operation",
|
||||||
Run: runElectionFunc,
|
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")
|
cmd.Flags().IntVar(&totalClientConnections, "total-client-connections", 10, "total number of client connections")
|
||||||
return cmd
|
return cmd
|
||||||
}
|
}
|
||||||
|
|
||||||
func runElectionFunc(cmd *cobra.Command, args []string) {
|
func runElectionFunc(cmd *cobra.Command, args []string) {
|
||||||
if len(args) > 0 {
|
election := "elector"
|
||||||
ExitWithError(ExitBadArgs, errors.New("election does not take any argument"))
|
if len(args) == 1 {
|
||||||
|
election = args[0]
|
||||||
|
}
|
||||||
|
if len(args) > 1 {
|
||||||
|
ExitWithError(ExitBadArgs, errors.New("election takes at most one argument"))
|
||||||
}
|
}
|
||||||
|
|
||||||
rcs := make([]roundClient, totalClientConnections)
|
rcs := make([]roundClient, totalClientConnections)
|
||||||
validatec, releasec := make(chan struct{}, len(rcs)), make(chan struct{}, len(rcs))
|
validatec := make(chan struct{}, len(rcs))
|
||||||
for range rcs {
|
// nextc closes when election is ready for next round.
|
||||||
releasec <- struct{}{}
|
nextc := make(chan struct{})
|
||||||
}
|
|
||||||
|
|
||||||
eps := endpointsFromFlag(cmd)
|
eps := endpointsFromFlag(cmd)
|
||||||
dialTimeout := dialTimeoutFromCmd(cmd)
|
dialTimeout := dialTimeoutFromCmd(cmd)
|
||||||
|
|
||||||
@ -53,6 +55,10 @@ func runElectionFunc(cmd *cobra.Command, args []string) {
|
|||||||
v := fmt.Sprintf("%d", i)
|
v := fmt.Sprintf("%d", i)
|
||||||
observedLeader := ""
|
observedLeader := ""
|
||||||
validateWaiters := 0
|
validateWaiters := 0
|
||||||
|
var rcNextc chan struct{}
|
||||||
|
setRcNextc := func() {
|
||||||
|
rcNextc = nextc
|
||||||
|
}
|
||||||
|
|
||||||
rcs[i].c = newClient(eps, dialTimeout)
|
rcs[i].c = newClient(eps, dialTimeout)
|
||||||
var (
|
var (
|
||||||
@ -65,18 +71,22 @@ func runElectionFunc(cmd *cobra.Command, args []string) {
|
|||||||
break
|
break
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
e := concurrency.NewElection(s, "electors")
|
|
||||||
|
|
||||||
rcs[i].acquire = func() error {
|
e := concurrency.NewElection(s, election)
|
||||||
<-releasec
|
rcs[i].acquire = func() (err error) {
|
||||||
ctx, cancel := context.WithCancel(context.Background())
|
ctx, cancel := context.WithCancel(context.Background())
|
||||||
|
donec := make(chan struct{})
|
||||||
go func() {
|
go func() {
|
||||||
if ol, ok := <-e.Observe(ctx); ok {
|
defer close(donec)
|
||||||
observedLeader = string(ol.Kvs[0].Value)
|
for ctx.Err() == nil {
|
||||||
if observedLeader != v {
|
if ol, ok := <-e.Observe(ctx); ok {
|
||||||
cancel()
|
observedLeader = string(ol.Kvs[0].Value)
|
||||||
|
break
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
if observedLeader != v {
|
||||||
|
cancel()
|
||||||
|
}
|
||||||
}()
|
}()
|
||||||
err = e.Campaign(ctx, v)
|
err = e.Campaign(ctx, v)
|
||||||
if err == nil {
|
if err == nil {
|
||||||
@ -85,18 +95,24 @@ func runElectionFunc(cmd *cobra.Command, args []string) {
|
|||||||
if observedLeader == v {
|
if observedLeader == v {
|
||||||
validateWaiters = len(rcs)
|
validateWaiters = len(rcs)
|
||||||
}
|
}
|
||||||
|
cancel()
|
||||||
|
<-donec
|
||||||
select {
|
select {
|
||||||
case <-ctx.Done():
|
case <-ctx.Done():
|
||||||
return nil
|
return nil
|
||||||
default:
|
default:
|
||||||
cancel()
|
|
||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
rcs[i].validate = func() error {
|
rcs[i].validate = func() error {
|
||||||
if l, err := e.Leader(context.TODO()); err == nil && string(l.Kvs[0].Value) != observedLeader {
|
l, err := e.Leader(context.TODO())
|
||||||
return fmt.Errorf("expected leader %q, got %q", observedLeader, l)
|
if err == nil && string(l.Kvs[0].Value) != observedLeader {
|
||||||
|
return fmt.Errorf("expected leader %q, got %q", observedLeader, l.Kvs[0].Value)
|
||||||
}
|
}
|
||||||
|
if err != nil {
|
||||||
|
return err
|
||||||
|
}
|
||||||
|
setRcNextc()
|
||||||
validatec <- struct{}{}
|
validatec <- struct{}{}
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
@ -113,14 +129,15 @@ func runElectionFunc(cmd *cobra.Command, args []string) {
|
|||||||
return err
|
return err
|
||||||
}
|
}
|
||||||
if observedLeader == v {
|
if observedLeader == v {
|
||||||
for range rcs {
|
close(nextc)
|
||||||
releasec <- struct{}{}
|
nextc = make(chan struct{})
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
<-rcNextc
|
||||||
observedLeader = ""
|
observedLeader = ""
|
||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
// each client creates 1 key from Campaign() and delete it from Resign()
|
||||||
doRounds(rcs, rounds)
|
// a round involves in 2*len(rcs) requests.
|
||||||
|
doRounds(rcs, rounds, 2*len(rcs))
|
||||||
}
|
}
|
||||||
|
@ -15,6 +15,7 @@
|
|||||||
package command
|
package command
|
||||||
|
|
||||||
import (
|
import (
|
||||||
|
"context"
|
||||||
"fmt"
|
"fmt"
|
||||||
"log"
|
"log"
|
||||||
"sync"
|
"sync"
|
||||||
@ -23,25 +24,18 @@ import (
|
|||||||
"github.com/coreos/etcd/clientv3"
|
"github.com/coreos/etcd/clientv3"
|
||||||
|
|
||||||
"github.com/spf13/cobra"
|
"github.com/spf13/cobra"
|
||||||
|
"golang.org/x/time/rate"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
// shared flags
|
||||||
var (
|
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
|
||||||
totalClientConnections int // total number of client connections to be made with server
|
endpoints []string
|
||||||
noOfPrefixes int // total number of prefixes which will be watched upon
|
dialTimeout time.Duration
|
||||||
watchPerPrefix int // number of watchers per prefix
|
rounds int // total number of rounds to run; set to <= 0 to run forever.
|
||||||
reqRate int // put request per second
|
reqRate int // maximum number of requests 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 {
|
type roundClient struct {
|
||||||
c *clientv3.Client
|
c *clientv3.Client
|
||||||
progress int
|
progress int
|
||||||
@ -61,16 +55,21 @@ func newClient(eps []string, timeout time.Duration) *clientv3.Client {
|
|||||||
return c
|
return c
|
||||||
}
|
}
|
||||||
|
|
||||||
func doRounds(rcs []roundClient, rounds int) {
|
func doRounds(rcs []roundClient, rounds int, requests int) {
|
||||||
var mu sync.Mutex
|
var mu sync.Mutex
|
||||||
var wg sync.WaitGroup
|
var wg sync.WaitGroup
|
||||||
|
|
||||||
wg.Add(len(rcs))
|
wg.Add(len(rcs))
|
||||||
finished := make(chan struct{})
|
finished := make(chan struct{})
|
||||||
|
limiter := rate.NewLimiter(rate.Limit(reqRate), reqRate)
|
||||||
for i := range rcs {
|
for i := range rcs {
|
||||||
go func(rc *roundClient) {
|
go func(rc *roundClient) {
|
||||||
defer wg.Done()
|
defer wg.Done()
|
||||||
for rc.progress < rounds {
|
for rc.progress < rounds || rounds <= 0 {
|
||||||
|
if err := limiter.WaitN(context.Background(), requests/len(rcs)); err != nil {
|
||||||
|
log.Panicf("rate limiter error %v", err)
|
||||||
|
}
|
||||||
|
|
||||||
for rc.acquire() != nil { /* spin */
|
for rc.acquire() != nil { /* spin */
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -85,7 +84,7 @@ func doRounds(rcs []roundClient, rounds int) {
|
|||||||
finished <- struct{}{}
|
finished <- struct{}{}
|
||||||
|
|
||||||
mu.Lock()
|
mu.Lock()
|
||||||
for rc.release() != nil {
|
for rc.release() != nil { /* spin */
|
||||||
mu.Unlock()
|
mu.Unlock()
|
||||||
mu.Lock()
|
mu.Lock()
|
||||||
}
|
}
|
||||||
@ -95,7 +94,7 @@ func doRounds(rcs []roundClient, rounds int) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
start := time.Now()
|
start := time.Now()
|
||||||
for i := 1; i < len(rcs)*rounds+1; i++ {
|
for i := 1; i < len(rcs)*rounds+1 || rounds <= 0; i++ {
|
||||||
select {
|
select {
|
||||||
case <-finished:
|
case <-finished:
|
||||||
if i%100 == 0 {
|
if i%100 == 0 {
|
||||||
|
@ -14,7 +14,7 @@
|
|||||||
|
|
||||||
// copied from https://github.com/rkt/rkt/blob/master/rkt/help.go
|
// copied from https://github.com/rkt/rkt/blob/master/rkt/help.go
|
||||||
|
|
||||||
package main
|
package command
|
||||||
|
|
||||||
import (
|
import (
|
||||||
"bytes"
|
"bytes"
|
@ -22,11 +22,16 @@ import (
|
|||||||
"time"
|
"time"
|
||||||
|
|
||||||
"github.com/coreos/etcd/clientv3"
|
"github.com/coreos/etcd/clientv3"
|
||||||
|
|
||||||
"github.com/spf13/cobra"
|
"github.com/spf13/cobra"
|
||||||
"google.golang.org/grpc"
|
"google.golang.org/grpc"
|
||||||
"google.golang.org/grpc/codes"
|
"google.golang.org/grpc/codes"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
var (
|
||||||
|
leaseTTL int64
|
||||||
|
)
|
||||||
|
|
||||||
// NewLeaseRenewerCommand returns the cobra command for "lease-renewer runner".
|
// NewLeaseRenewerCommand returns the cobra command for "lease-renewer runner".
|
||||||
func NewLeaseRenewerCommand() *cobra.Command {
|
func NewLeaseRenewerCommand() *cobra.Command {
|
||||||
cmd := &cobra.Command{
|
cmd := &cobra.Command{
|
||||||
@ -34,6 +39,7 @@ func NewLeaseRenewerCommand() *cobra.Command {
|
|||||||
Short: "Performs lease renew operation",
|
Short: "Performs lease renew operation",
|
||||||
Run: runLeaseRenewerFunc,
|
Run: runLeaseRenewerFunc,
|
||||||
}
|
}
|
||||||
|
cmd.Flags().Int64Var(&leaseTTL, "ttl", 5, "lease's ttl")
|
||||||
return cmd
|
return cmd
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -53,7 +59,7 @@ func runLeaseRenewerFunc(cmd *cobra.Command, args []string) {
|
|||||||
err error
|
err error
|
||||||
)
|
)
|
||||||
for {
|
for {
|
||||||
l, err = c.Lease.Grant(ctx, 5)
|
l, err = c.Lease.Grant(ctx, leaseTTL)
|
||||||
if err == nil {
|
if err == nil {
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
@ -65,14 +71,14 @@ func runLeaseRenewerFunc(cmd *cobra.Command, args []string) {
|
|||||||
err = lk.Err
|
err = lk.Err
|
||||||
if grpc.Code(err) == codes.NotFound {
|
if grpc.Code(err) == codes.NotFound {
|
||||||
if time.Since(expire) < 0 {
|
if time.Since(expire) < 0 {
|
||||||
log.Printf("bad renew! exceeded: %v", time.Since(expire))
|
log.Fatalf("bad renew! exceeded: %v", time.Since(expire))
|
||||||
for {
|
for {
|
||||||
lk = c.Lease.KeepAliveOnce(ctx, l.ID)
|
lk = c.Lease.KeepAliveOnce(ctx, l.ID)
|
||||||
fmt.Println(lk)
|
fmt.Println(lk)
|
||||||
time.Sleep(time.Second)
|
time.Sleep(time.Second)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
log.Printf("lost lease %d, expire: %v\n", l.ID, expire)
|
log.Fatalf("lost lease %d, expire: %v\n", l.ID, expire)
|
||||||
break
|
break
|
||||||
}
|
}
|
||||||
if err != nil {
|
if err != nil {
|
||||||
|
@ -20,24 +20,29 @@ import (
|
|||||||
"fmt"
|
"fmt"
|
||||||
|
|
||||||
"github.com/coreos/etcd/clientv3/concurrency"
|
"github.com/coreos/etcd/clientv3/concurrency"
|
||||||
|
|
||||||
"github.com/spf13/cobra"
|
"github.com/spf13/cobra"
|
||||||
)
|
)
|
||||||
|
|
||||||
// NewLockRacerCommand returns the cobra command for "lock-racer runner".
|
// NewLockRacerCommand returns the cobra command for "lock-racer runner".
|
||||||
func NewLockRacerCommand() *cobra.Command {
|
func NewLockRacerCommand() *cobra.Command {
|
||||||
cmd := &cobra.Command{
|
cmd := &cobra.Command{
|
||||||
Use: "lock-racer",
|
Use: "lock-racer [name of lock (defaults to 'racers')]",
|
||||||
Short: "Performs lock race operation",
|
Short: "Performs lock race operation",
|
||||||
Run: runRacerFunc,
|
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")
|
cmd.Flags().IntVar(&totalClientConnections, "total-client-connections", 10, "total number of client connections")
|
||||||
return cmd
|
return cmd
|
||||||
}
|
}
|
||||||
|
|
||||||
func runRacerFunc(cmd *cobra.Command, args []string) {
|
func runRacerFunc(cmd *cobra.Command, args []string) {
|
||||||
if len(args) > 0 {
|
racers := "racers"
|
||||||
ExitWithError(ExitBadArgs, errors.New("lock-racer does not take any argument"))
|
if len(args) == 1 {
|
||||||
|
racers = args[0]
|
||||||
|
}
|
||||||
|
|
||||||
|
if len(args) > 1 {
|
||||||
|
ExitWithError(ExitBadArgs, errors.New("lock-racer takes at most one argument"))
|
||||||
}
|
}
|
||||||
|
|
||||||
rcs := make([]roundClient, totalClientConnections)
|
rcs := make([]roundClient, totalClientConnections)
|
||||||
@ -61,7 +66,7 @@ func runRacerFunc(cmd *cobra.Command, args []string) {
|
|||||||
break
|
break
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
m := concurrency.NewMutex(s, "racers")
|
m := concurrency.NewMutex(s, racers)
|
||||||
rcs[i].acquire = func() error { return m.Lock(ctx) }
|
rcs[i].acquire = func() error { return m.Lock(ctx) }
|
||||||
rcs[i].validate = func() error {
|
rcs[i].validate = func() error {
|
||||||
if cnt++; cnt != 1 {
|
if cnt++; cnt != 1 {
|
||||||
@ -77,5 +82,7 @@ func runRacerFunc(cmd *cobra.Command, args []string) {
|
|||||||
return nil
|
return nil
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
doRounds(rcs, rounds)
|
// each client creates 1 key from NewMutex() and delete it from Unlock()
|
||||||
|
// a round involves in 2*len(rcs) requests.
|
||||||
|
doRounds(rcs, rounds, 2*len(rcs))
|
||||||
}
|
}
|
||||||
|
69
tools/functional-tester/etcd-runner/command/root.go
Normal file
69
tools/functional-tester/etcd-runner/command/root.go
Normal file
@ -0,0 +1,69 @@
|
|||||||
|
// Copyright 2017 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 (
|
||||||
|
"log"
|
||||||
|
"math/rand"
|
||||||
|
"time"
|
||||||
|
|
||||||
|
"github.com/spf13/cobra"
|
||||||
|
)
|
||||||
|
|
||||||
|
const (
|
||||||
|
cliName = "etcd-runner"
|
||||||
|
cliDescription = "Stress tests using clientv3 functionality.."
|
||||||
|
|
||||||
|
defaultDialTimeout = 2 * time.Second
|
||||||
|
)
|
||||||
|
|
||||||
|
var (
|
||||||
|
rootCmd = &cobra.Command{
|
||||||
|
Use: cliName,
|
||||||
|
Short: cliDescription,
|
||||||
|
SuggestFor: []string{"etcd-runner"},
|
||||||
|
}
|
||||||
|
)
|
||||||
|
|
||||||
|
func init() {
|
||||||
|
cobra.EnablePrefixMatching = true
|
||||||
|
|
||||||
|
rand.Seed(time.Now().UnixNano())
|
||||||
|
|
||||||
|
log.SetFlags(log.Lmicroseconds)
|
||||||
|
|
||||||
|
rootCmd.PersistentFlags().StringSliceVar(&endpoints, "endpoints", []string{"127.0.0.1:2379"}, "gRPC endpoints")
|
||||||
|
rootCmd.PersistentFlags().DurationVar(&dialTimeout, "dial-timeout", defaultDialTimeout, "dial timeout for client connections")
|
||||||
|
rootCmd.PersistentFlags().IntVar(&reqRate, "req-rate", 30, "maximum number of requests per second")
|
||||||
|
rootCmd.PersistentFlags().IntVar(&rounds, "rounds", 100, "number of rounds to run; 0 to run forever")
|
||||||
|
|
||||||
|
rootCmd.AddCommand(
|
||||||
|
NewElectionCommand(),
|
||||||
|
NewLeaseRenewerCommand(),
|
||||||
|
NewLockRacerCommand(),
|
||||||
|
NewWatchCommand(),
|
||||||
|
)
|
||||||
|
}
|
||||||
|
|
||||||
|
func Start() {
|
||||||
|
rootCmd.SetUsageFunc(usageFunc)
|
||||||
|
|
||||||
|
// Make help just show the usage
|
||||||
|
rootCmd.SetHelpTemplate(`{{.UsageString}}`)
|
||||||
|
|
||||||
|
if err := rootCmd.Execute(); err != nil {
|
||||||
|
ExitWithError(ExitError, err)
|
||||||
|
}
|
||||||
|
}
|
@ -24,10 +24,19 @@ 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"
|
"github.com/spf13/cobra"
|
||||||
"golang.org/x/time/rate"
|
"golang.org/x/time/rate"
|
||||||
)
|
)
|
||||||
|
|
||||||
|
var (
|
||||||
|
runningTime time.Duration // time for which operation should be performed
|
||||||
|
noOfPrefixes int // total number of prefixes which will be watched upon
|
||||||
|
watchPerPrefix int // number of watchers per prefix
|
||||||
|
watchPrefix string // prefix append to keys in watcher
|
||||||
|
totalKeys int // total number of keys for operation
|
||||||
|
)
|
||||||
|
|
||||||
// NewWatchCommand returns the cobra command for "watcher runner".
|
// NewWatchCommand returns the cobra command for "watcher runner".
|
||||||
func NewWatchCommand() *cobra.Command {
|
func NewWatchCommand() *cobra.Command {
|
||||||
cmd := &cobra.Command{
|
cmd := &cobra.Command{
|
||||||
@ -35,12 +44,12 @@ func NewWatchCommand() *cobra.Command {
|
|||||||
Short: "Performs watch operation",
|
Short: "Performs watch operation",
|
||||||
Run: runWatcherFunc,
|
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().DurationVar(&runningTime, "running-time", 60, "number of seconds to run")
|
||||||
|
cmd.Flags().StringVar(&watchPrefix, "prefix", "", "the prefix to append on all keys")
|
||||||
cmd.Flags().IntVar(&noOfPrefixes, "total-prefixes", 10, "total no of prefixes to use")
|
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(&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")
|
cmd.Flags().IntVar(&totalKeys, "total-keys", 1000, "total number of keys to watch")
|
||||||
|
|
||||||
return cmd
|
return cmd
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -50,7 +59,7 @@ func runWatcherFunc(cmd *cobra.Command, args []string) {
|
|||||||
}
|
}
|
||||||
|
|
||||||
ctx := context.Background()
|
ctx := context.Background()
|
||||||
for round := 0; round < rounds; round++ {
|
for round := 0; round < rounds || rounds <= 0; round++ {
|
||||||
fmt.Println("round", round)
|
fmt.Println("round", round)
|
||||||
performWatchOnPrefixes(ctx, cmd, round)
|
performWatchOnPrefixes(ctx, cmd, round)
|
||||||
}
|
}
|
||||||
@ -94,7 +103,7 @@ func performWatchOnPrefixes(ctx context.Context, cmd *cobra.Command, round int)
|
|||||||
if err = limiter.Wait(ctxt); err != nil {
|
if err = limiter.Wait(ctxt); err != nil {
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
if err = putKeyAtMostOnce(ctxt, client, roundPrefix+"-"+prefix+"-"+key); err != nil {
|
if err = putKeyAtMostOnce(ctxt, client, watchPrefix+"-"+roundPrefix+"-"+prefix+"-"+key); err != nil {
|
||||||
log.Fatalf("failed to put key: %v", err)
|
log.Fatalf("failed to put key: %v", err)
|
||||||
return
|
return
|
||||||
}
|
}
|
||||||
@ -112,15 +121,15 @@ func performWatchOnPrefixes(ctx context.Context, cmd *cobra.Command, round int)
|
|||||||
rc := newClient(eps, dialTimeout)
|
rc := newClient(eps, dialTimeout)
|
||||||
rcs = append(rcs, rc)
|
rcs = append(rcs, rc)
|
||||||
|
|
||||||
watchPrefix := roundPrefix + "-" + prefix
|
wprefix := watchPrefix + "-" + roundPrefix + "-" + prefix
|
||||||
|
|
||||||
wc := rc.Watch(ctxc, watchPrefix, clientv3.WithPrefix(), clientv3.WithRev(revision))
|
wc := rc.Watch(ctxc, wprefix, clientv3.WithPrefix(), clientv3.WithRev(revision))
|
||||||
wcs = append(wcs, wc)
|
wcs = append(wcs, wc)
|
||||||
|
|
||||||
wg.Add(1)
|
wg.Add(1)
|
||||||
go func() {
|
go func() {
|
||||||
defer wg.Done()
|
defer wg.Done()
|
||||||
checkWatchResponse(wc, watchPrefix, keys)
|
checkWatchResponse(wc, wprefix, keys)
|
||||||
}()
|
}()
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -139,7 +148,7 @@ func performWatchOnPrefixes(ctx context.Context, cmd *cobra.Command, round int)
|
|||||||
rc.Close()
|
rc.Close()
|
||||||
}
|
}
|
||||||
|
|
||||||
if err = deletePrefix(ctx, client, roundPrefix); err != nil {
|
if err = deletePrefix(ctx, client, watchPrefix); err != nil {
|
||||||
log.Fatalf("failed to clean up keys after test: %v", err)
|
log.Fatalf("failed to clean up keys after test: %v", err)
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -148,7 +157,7 @@ func checkWatchResponse(wc clientv3.WatchChan, prefix string, keys []string) {
|
|||||||
for n := 0; n < len(keys); {
|
for n := 0; n < len(keys); {
|
||||||
wr, more := <-wc
|
wr, more := <-wc
|
||||||
if !more {
|
if !more {
|
||||||
log.Fatalf("expect more keys (received %d/%d) for %s", len(keys), n, prefix)
|
log.Fatalf("expect more keys (received %d/%d) for %s", n, len(keys), prefix)
|
||||||
}
|
}
|
||||||
for _, event := range wr.Events {
|
for _, event := range wr.Events {
|
||||||
expectedKey := prefix + "-" + keys[n]
|
expectedKey := prefix + "-" + keys[n]
|
||||||
|
@ -15,61 +15,8 @@
|
|||||||
// etcd-runner is a command line application that performs tests on etcd.
|
// etcd-runner is a command line application that performs tests on etcd.
|
||||||
package main
|
package main
|
||||||
|
|
||||||
import (
|
import "github.com/coreos/etcd/tools/functional-tester/etcd-runner/command"
|
||||||
"log"
|
|
||||||
"time"
|
|
||||||
|
|
||||||
"github.com/coreos/etcd/tools/functional-tester/etcd-runner/command"
|
|
||||||
"github.com/spf13/cobra"
|
|
||||||
)
|
|
||||||
|
|
||||||
const (
|
|
||||||
cliName = "etcd-runner"
|
|
||||||
cliDescription = "Stress tests using clientv3 functionality.."
|
|
||||||
|
|
||||||
defaultDialTimeout = 2 * time.Second
|
|
||||||
)
|
|
||||||
|
|
||||||
var (
|
|
||||||
globalFlags = command.GlobalFlags{}
|
|
||||||
)
|
|
||||||
|
|
||||||
var (
|
|
||||||
rootCmd = &cobra.Command{
|
|
||||||
Use: cliName,
|
|
||||||
Short: cliDescription,
|
|
||||||
SuggestFor: []string{"etcd-runner"},
|
|
||||||
}
|
|
||||||
)
|
|
||||||
|
|
||||||
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(),
|
|
||||||
)
|
|
||||||
}
|
|
||||||
|
|
||||||
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() {
|
||||||
Start()
|
command.Start()
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user