Merge pull request #14672 from tjungblu/etcd-14638

Expect exit code enhancement
This commit is contained in:
Marek Siarkowicz 2022-11-14 12:34:43 +01:00 committed by GitHub
commit 4cdcb91fac
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
23 changed files with 453 additions and 279 deletions

View File

@ -19,6 +19,7 @@ package expect
import (
"bufio"
"context"
"errors"
"fmt"
"io"
"os"
@ -33,6 +34,10 @@ import (
const DEBUG_LINES_TAIL = 40
var (
ErrProcessRunning = fmt.Errorf("process is still running")
)
type ExpectProcess struct {
cfg expectConfig
@ -40,11 +45,12 @@ type ExpectProcess struct {
fpty *os.File
wg sync.WaitGroup
mu sync.Mutex // protects lines and err
lines []string
count int // increment whenever new line gets added
cur int // current read position
err error
mu sync.Mutex // protects lines, count, cur, exitErr and exitCode
lines []string
count int // increment whenever new line gets added
cur int // current read position
exitErr error // process exit error
exitCode int
}
// NewExpect creates a new process for expect testing.
@ -69,8 +75,9 @@ func NewExpectWithEnv(name string, args []string, env []string, serverProcessCon
return nil, err
}
ep.wg.Add(1)
ep.wg.Add(2)
go ep.read()
go ep.waitSaveExitErr()
return ep, nil
}
@ -95,46 +102,83 @@ func (ep *ExpectProcess) Pid() int {
func (ep *ExpectProcess) read() {
defer ep.wg.Done()
printDebugLines := os.Getenv("EXPECT_DEBUG") != ""
defer func(fpty *os.File) {
err := fpty.Close()
if err != nil {
// we deliberately only log the error here, closing the PTY should mostly be (expected) broken pipes
fmt.Printf("error while closing fpty: %v", err)
}
}(ep.fpty)
r := bufio.NewReader(ep.fpty)
for {
l, err := r.ReadString('\n')
ep.mu.Lock()
if l != "" {
if printDebugLines {
fmt.Printf("%s (%s) (%d): %s", ep.cmd.Path, ep.cfg.name, ep.cmd.Process.Pid, l)
}
ep.lines = append(ep.lines, l)
ep.count++
}
err := ep.tryReadNextLine(r)
if err != nil {
ep.err = err
ep.mu.Unlock()
break
}
ep.mu.Unlock()
}
}
func (ep *ExpectProcess) tryReadNextLine(r *bufio.Reader) error {
printDebugLines := os.Getenv("EXPECT_DEBUG") != ""
l, err := r.ReadString('\n')
ep.mu.Lock()
defer ep.mu.Unlock()
if l != "" {
if printDebugLines {
fmt.Printf("%s (%s) (%d): %s", ep.cmd.Path, ep.cfg.name, ep.cmd.Process.Pid, l)
}
ep.lines = append(ep.lines, l)
ep.count++
}
// we're checking the error here at the bottom to ensure any leftover reads are still taken into account
return err
}
func (ep *ExpectProcess) waitSaveExitErr() {
defer ep.wg.Done()
err := ep.waitProcess()
ep.mu.Lock()
defer ep.mu.Unlock()
if err != nil {
ep.exitErr = err
}
}
// ExpectFunc returns the first line satisfying the function f.
func (ep *ExpectProcess) ExpectFunc(ctx context.Context, f func(string) bool) (string, error) {
i := 0
for {
ep.mu.Lock()
for i < len(ep.lines) {
line := ep.lines[i]
i++
if f(line) {
ep.mu.Unlock()
return line, nil
line, errsFound := func() (string, bool) {
ep.mu.Lock()
defer ep.mu.Unlock()
// check if this expect has been already closed
if ep.cmd == nil {
return "", true
}
for i < len(ep.lines) {
line := ep.lines[i]
i++
if f(line) {
return line, false
}
}
return "", ep.exitErr != nil
}()
if line != "" {
return line, nil
}
if ep.err != nil {
ep.mu.Unlock()
if errsFound {
break
}
ep.mu.Unlock()
select {
case <-ctx.Done():
@ -143,16 +187,18 @@ func (ep *ExpectProcess) ExpectFunc(ctx context.Context, f func(string) bool) (s
// continue loop
}
}
ep.mu.Lock()
defer ep.mu.Unlock()
lastLinesIndex := len(ep.lines) - DEBUG_LINES_TAIL
if lastLinesIndex < 0 {
lastLinesIndex = 0
}
lastLines := strings.Join(ep.lines[lastLinesIndex:], "")
ep.mu.Unlock()
return "", fmt.Errorf("match not found."+
" Set EXPECT_DEBUG for more info Err: %v, last lines:\n%s",
ep.err, lastLines)
return "", fmt.Errorf("match not found. "+
" Set EXPECT_DEBUG for more info Errs: [%v], last lines:\n%s",
ep.exitErr, lastLines)
}
// ExpectWithContext returns the first line containing the given string.
@ -174,47 +220,85 @@ func (ep *ExpectProcess) LineCount() int {
return ep.count
}
// Stop kills the expect process and waits for it to exit.
func (ep *ExpectProcess) Stop() error { return ep.close(true) }
// ExitCode returns the exit code of this process.
// If the process is still running, it returns exit code 0 and ErrProcessRunning.
func (ep *ExpectProcess) ExitCode() (int, error) {
ep.mu.Lock()
defer ep.mu.Unlock()
if ep.cmd == nil {
return ep.exitCode, nil
}
return 0, ErrProcessRunning
}
// ExitError returns the exit error of this process (if any).
// If the process is still running, it returns ErrProcessRunning instead.
func (ep *ExpectProcess) ExitError() error {
ep.mu.Lock()
defer ep.mu.Unlock()
if ep.cmd == nil {
return ep.exitErr
}
return ErrProcessRunning
}
// Stop signals the process to terminate via SIGTERM
func (ep *ExpectProcess) Stop() error {
err := ep.Signal(syscall.SIGTERM)
if err != nil && strings.Contains(err.Error(), "os: process already finished") {
return nil
}
return err
}
// Signal sends a signal to the expect process
func (ep *ExpectProcess) Signal(sig os.Signal) error {
ep.mu.Lock()
defer ep.mu.Unlock()
if ep.cmd == nil {
return errors.New("expect process already closed")
}
return ep.cmd.Process.Signal(sig)
}
func (ep *ExpectProcess) Wait() error {
_, err := ep.cmd.Process.Wait()
return err
func (ep *ExpectProcess) waitProcess() error {
state, err := ep.cmd.Process.Wait()
if err != nil {
return err
}
ep.mu.Lock()
defer ep.mu.Unlock()
ep.exitCode = state.ExitCode()
if !state.Success() {
return fmt.Errorf("unexpected exit code [%d] after running [%s]", ep.exitCode, ep.cmd.String())
}
return nil
}
// Close waits for the expect process to exit.
// Close currently does not return error if process exited with !=0 status.
// TODO: Close should expose underlying process failure by default.
func (ep *ExpectProcess) Close() error { return ep.close(false) }
// Wait waits for the process to finish.
func (ep *ExpectProcess) Wait() {
ep.wg.Wait()
}
func (ep *ExpectProcess) close(kill bool) error {
if ep.cmd == nil {
return ep.err
}
if kill {
ep.Signal(syscall.SIGTERM)
}
err := ep.cmd.Wait()
ep.fpty.Close()
// Close waits for the expect process to exit and return its error.
func (ep *ExpectProcess) Close() error {
ep.wg.Wait()
if err != nil {
if !kill && strings.Contains(err.Error(), "exit status") {
// non-zero exit code
err = nil
} else if kill && strings.Contains(err.Error(), "signal:") {
err = nil
}
}
ep.mu.Lock()
defer ep.mu.Unlock()
// this signals to other funcs that the process has finished
ep.cmd = nil
return err
return ep.exitErr
}
func (ep *ExpectProcess) Send(command string) error {
@ -222,15 +306,6 @@ func (ep *ExpectProcess) Send(command string) error {
return err
}
func (ep *ExpectProcess) ProcessError() error {
if strings.Contains(ep.err.Error(), "input/output error") {
// TODO: The expect library should not return
// `/dev/ptmx: input/output error` when process just exits.
return nil
}
return ep.err
}
func (ep *ExpectProcess) Lines() []string {
ep.mu.Lock()
defer ep.mu.Unlock()

View File

@ -19,9 +19,11 @@ package expect
import (
"context"
"os"
"strings"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
@ -65,9 +67,57 @@ func TestExpectFuncTimeout(t *testing.T) {
require.ErrorAs(t, err, &context.DeadlineExceeded)
if err = ep.Stop(); err != nil {
if err := ep.Stop(); err != nil {
t.Fatal(err)
}
err = ep.Close()
require.ErrorContains(t, err, "unexpected exit code [-1] after running [/usr/bin/tail -f /dev/null]")
require.Equal(t, -1, ep.exitCode)
}
func TestExpectFuncExitFailure(t *testing.T) {
// tail -x should not exist and return a non-zero exit code
ep, err := NewExpect("tail", "-x")
if err != nil {
t.Fatal(err)
}
ctx, cancel := context.WithTimeout(context.Background(), 500*time.Millisecond)
defer cancel()
_, err = ep.ExpectFunc(ctx, func(s string) bool {
return strings.Contains(s, "something entirely unexpected")
})
require.ErrorContains(t, err, "unexpected exit code [1] after running [/usr/bin/tail -x]")
require.Equal(t, 1, ep.exitCode)
}
func TestExpectFuncExitFailureStop(t *testing.T) {
// tail -x should not exist and return a non-zero exit code
ep, err := NewExpect("tail", "-x")
if err != nil {
t.Fatal(err)
}
ctx, cancel := context.WithTimeout(context.Background(), 500*time.Millisecond)
defer cancel()
_, err = ep.ExpectFunc(ctx, func(s string) bool {
return strings.Contains(s, "something entirely unexpected")
})
require.ErrorContains(t, err, "unexpected exit code [1] after running [/usr/bin/tail -x]")
exitCode, err := ep.ExitCode()
require.Equal(t, 0, exitCode)
require.Equal(t, err, ErrProcessRunning)
if err := ep.Stop(); err != nil {
t.Fatal(err)
}
err = ep.Close()
require.ErrorContains(t, err, "unexpected exit code [1] after running [/usr/bin/tail -x]")
exitCode, err = ep.ExitCode()
require.Equal(t, 1, exitCode)
require.NoError(t, err)
}
func TestEcho(t *testing.T) {
@ -138,10 +188,8 @@ func TestSignal(t *testing.T) {
donec := make(chan struct{})
go func() {
defer close(donec)
werr := "signal: interrupt"
if cerr := ep.Close(); cerr == nil || cerr.Error() != werr {
t.Errorf("got error %v, wanted error %s", cerr, werr)
}
err = ep.Close()
assert.ErrorContains(t, err, "unexpected exit code [-1] after running [/usr/bin/sleep 100]")
}()
select {
case <-time.After(5 * time.Second):

View File

@ -28,30 +28,30 @@ import (
)
type txnReq struct {
compare []string
ifSucess []string
ifFail []string
results []string
compare []string
ifSuccess []string
ifFail []string
results []string
}
func TestTxnSucc(t *testing.T) {
testRunner.BeforeTest(t)
reqs := []txnReq{
{
compare: []string{`value("key1") != "value2"`, `value("key2") != "value1"`},
ifSucess: []string{"get key1", "get key2"},
results: []string{"SUCCESS", "key1", "value1", "key2", "value2"},
compare: []string{`value("key1") != "value2"`, `value("key2") != "value1"`},
ifSuccess: []string{"get key1", "get key2"},
results: []string{"SUCCESS", "key1", "value1", "key2", "value2"},
},
{
compare: []string{`version("key1") = "1"`, `version("key2") = "1"`},
ifSucess: []string{"get key1", "get key2", `put "key \"with\" space" "value \x23"`},
ifFail: []string{`put key1 "fail"`, `put key2 "fail"`},
results: []string{"SUCCESS", "key1", "value1", "key2", "value2", "OK"},
compare: []string{`version("key1") = "1"`, `version("key2") = "1"`},
ifSuccess: []string{"get key1", "get key2", `put "key \"with\" space" "value \x23"`},
ifFail: []string{`put key1 "fail"`, `put key2 "fail"`},
results: []string{"SUCCESS", "key1", "value1", "key2", "value2", "OK"},
},
{
compare: []string{`version("key \"with\" space") = "1"`},
ifSucess: []string{`get "key \"with\" space"`},
results: []string{"SUCCESS", `key "with" space`, "value \x23"},
compare: []string{`version("key \"with\" space") = "1"`},
ifSuccess: []string{`get "key \"with\" space"`},
results: []string{"SUCCESS", `key "with" space`, "value \x23"},
},
}
for _, cfg := range clusterTestCases() {
@ -69,7 +69,7 @@ func TestTxnSucc(t *testing.T) {
t.Fatalf("could not create key:%s, value:%s", "key2", "value2")
}
for _, req := range reqs {
resp, err := cc.Txn(ctx, req.compare, req.ifSucess, req.ifFail, config.TxnOptions{
resp, err := cc.Txn(ctx, req.compare, req.ifSuccess, req.ifFail, config.TxnOptions{
Interactive: true,
})
if err != nil {
@ -86,16 +86,16 @@ func TestTxnFail(t *testing.T) {
testRunner.BeforeTest(t)
reqs := []txnReq{
{
compare: []string{`version("key") < "0"`},
ifSucess: []string{`put key "success"`},
ifFail: []string{`put key "fail"`},
results: []string{"FAILURE", "OK"},
compare: []string{`version("key") < "0"`},
ifSuccess: []string{`put key "success"`},
ifFail: []string{`put key "fail"`},
results: []string{"FAILURE", "OK"},
},
{
compare: []string{`value("key1") != "value1"`},
ifSucess: []string{`put key1 "success"`},
ifFail: []string{`put key1 "fail"`},
results: []string{"FAILURE", "OK"},
compare: []string{`value("key1") != "value1"`},
ifSuccess: []string{`put key1 "success"`},
ifFail: []string{`put key1 "fail"`},
results: []string{"FAILURE", "OK"},
},
}
for _, cfg := range clusterTestCases() {
@ -110,7 +110,7 @@ func TestTxnFail(t *testing.T) {
t.Fatalf("could not create key:%s, value:%s", "key1", "value1")
}
for _, req := range reqs {
resp, err := cc.Txn(ctx, req.compare, req.ifSucess, req.ifFail, config.TxnOptions{
resp, err := cc.Txn(ctx, req.compare, req.ifSuccess, req.ifFail, config.TxnOptions{
Interactive: true,
})
if err != nil {

View File

@ -21,6 +21,7 @@ import (
"testing"
"time"
"github.com/stretchr/testify/require"
clientv3 "go.etcd.io/etcd/client/v3"
"go.etcd.io/etcd/tests/v3/framework/e2e"
)
@ -118,9 +119,8 @@ func authDisableTest(cx ctlCtx) {
// test-user doesn't have the permission, it must fail
cx.user, cx.pass = "test-user", "pass"
if err := ctlV3PutFailPerm(cx, "hoo", "bar"); err != nil {
cx.t.Fatal(err)
}
err := ctlV3PutFailPerm(cx, "hoo", "bar")
require.ErrorContains(cx.t, err, "permission denied")
cx.user, cx.pass = "root", "root"
if err := ctlV3AuthDisable(cx); err != nil {
@ -241,9 +241,9 @@ func authCredWriteKeyTest(cx ctlCtx) {
// try invalid user
cx.user, cx.pass = "a", "b"
if err := ctlV3PutFailAuth(cx, "foo", "bar"); err != nil {
cx.t.Fatal(err)
}
err := ctlV3PutFailAuth(cx, "foo", "bar")
require.ErrorContains(cx.t, err, "authentication failed")
// confirm put failed
cx.user, cx.pass = "test-user", "pass"
if err := ctlV3Get(cx, []string{"foo"}, []kv{{"foo", "bar"}}...); err != nil {
@ -262,9 +262,9 @@ func authCredWriteKeyTest(cx ctlCtx) {
// try bad password
cx.user, cx.pass = "test-user", "badpass"
if err := ctlV3PutFailAuth(cx, "foo", "baz"); err != nil {
cx.t.Fatal(err)
}
err = ctlV3PutFailAuth(cx, "foo", "baz")
require.ErrorContains(cx.t, err, "authentication failed")
// confirm put failed
cx.user, cx.pass = "test-user", "pass"
if err := ctlV3Get(cx, []string{"foo"}, []kv{{"foo", "bar2"}}...); err != nil {
@ -286,9 +286,8 @@ func authRoleUpdateTest(cx ctlCtx) {
// try put to not granted key
cx.user, cx.pass = "test-user", "pass"
if err := ctlV3PutFailPerm(cx, "hoo", "bar"); err != nil {
cx.t.Fatal(err)
}
err := ctlV3PutFailPerm(cx, "hoo", "bar")
require.ErrorContains(cx.t, err, "permission denied")
// grant a new key
cx.user, cx.pass = "root", "root"
@ -314,9 +313,8 @@ func authRoleUpdateTest(cx ctlCtx) {
// try put to the revoked key
cx.user, cx.pass = "test-user", "pass"
if err := ctlV3PutFailPerm(cx, "hoo", "bar"); err != nil {
cx.t.Fatal(err)
}
err = ctlV3PutFailPerm(cx, "hoo", "bar")
require.ErrorContains(cx.t, err, "permission denied")
// confirm a key still granted can be accessed
if err := ctlV3Get(cx, []string{"foo"}, []kv{{"foo", "bar"}}...); err != nil {
@ -355,9 +353,8 @@ func authUserDeleteDuringOpsTest(cx ctlCtx) {
// check the user is deleted
cx.user, cx.pass = "test-user", "pass"
if err := ctlV3PutFailAuth(cx, "foo", "baz"); err != nil {
cx.t.Fatal(err)
}
err = ctlV3PutFailAuth(cx, "foo", "baz")
require.ErrorContains(cx.t, err, "authentication failed")
}
func authRoleRevokeDuringOpsTest(cx ctlCtx) {
@ -415,9 +412,8 @@ func authRoleRevokeDuringOpsTest(cx ctlCtx) {
// check the role is revoked and permission is lost from the user
cx.user, cx.pass = "test-user", "pass"
if err := ctlV3PutFailPerm(cx, "foo", "baz"); err != nil {
cx.t.Fatal(err)
}
err = ctlV3PutFailPerm(cx, "foo", "baz")
require.ErrorContains(cx.t, err, "permission denied")
// try a key that can be accessed from the remaining role
cx.user, cx.pass = "test-user", "pass"
@ -492,45 +488,45 @@ func authTestTxn(cx ctlCtx) {
cx.user, cx.pass = "test-user", "pass"
rqs := txnRequests{
compare: []string{`version("c2") = "1"`},
ifSucess: []string{"get s2"},
ifFail: []string{"get f2"},
results: []string{"SUCCESS", "s2", "v"},
compare: []string{`version("c2") = "1"`},
ifSuccess: []string{"get s2"},
ifFail: []string{"get f2"},
results: []string{"SUCCESS", "s2", "v"},
}
if err := ctlV3Txn(cx, rqs); err != nil {
if err := ctlV3Txn(cx, rqs, false); err != nil {
cx.t.Fatal(err)
}
// a key of compare case isn't granted
rqs = txnRequests{
compare: []string{`version("c1") = "1"`},
ifSucess: []string{"get s2"},
ifFail: []string{"get f2"},
results: []string{"Error: etcdserver: permission denied"},
compare: []string{`version("c1") = "1"`},
ifSuccess: []string{"get s2"},
ifFail: []string{"get f2"},
results: []string{"Error: etcdserver: permission denied"},
}
if err := ctlV3Txn(cx, rqs); err != nil {
if err := ctlV3Txn(cx, rqs, true); err != nil {
cx.t.Fatal(err)
}
// a key of success case isn't granted
rqs = txnRequests{
compare: []string{`version("c2") = "1"`},
ifSucess: []string{"get s1"},
ifFail: []string{"get f2"},
results: []string{"Error: etcdserver: permission denied"},
compare: []string{`version("c2") = "1"`},
ifSuccess: []string{"get s1"},
ifFail: []string{"get f2"},
results: []string{"Error: etcdserver: permission denied"},
}
if err := ctlV3Txn(cx, rqs); err != nil {
if err := ctlV3Txn(cx, rqs, true); err != nil {
cx.t.Fatal(err)
}
// a key of failure case isn't granted
rqs = txnRequests{
compare: []string{`version("c2") = "1"`},
ifSucess: []string{"get s2"},
ifFail: []string{"get f1"},
results: []string{"Error: etcdserver: permission denied"},
compare: []string{`version("c2") = "1"`},
ifSuccess: []string{"get s2"},
ifFail: []string{"get f1"},
results: []string{"Error: etcdserver: permission denied"},
}
if err := ctlV3Txn(cx, rqs); err != nil {
if err := ctlV3Txn(cx, rqs, true); err != nil {
cx.t.Fatal(err)
}
}
@ -559,9 +555,8 @@ func authTestPrefixPerm(cx ctlCtx) {
}
}
if err := ctlV3PutFailPerm(cx, clientv3.GetPrefixRangeEnd(prefix), "baz"); err != nil {
cx.t.Fatal(err)
}
err := ctlV3PutFailPerm(cx, clientv3.GetPrefixRangeEnd(prefix), "baz")
require.ErrorContains(cx.t, err, "permission denied")
// grant the entire keys to test-user
cx.user, cx.pass = "root", "root"
@ -679,11 +674,10 @@ func authTestCertCN(cx ctlCtx) {
cx.t.Error(err)
}
// try a non granted key
// try a non-granted key
cx.user, cx.pass = "", ""
if err := ctlV3PutFailPerm(cx, "baz", "bar"); err != nil {
cx.t.Error(err)
}
err := ctlV3PutFailPerm(cx, "baz", "bar")
require.ErrorContains(cx.t, err, "permission denied")
}
func authTestRevokeWithDelete(cx ctlCtx) {
@ -766,9 +760,8 @@ func authTestFromKeyPerm(cx ctlCtx) {
}
// try a non granted key
if err := ctlV3PutFailPerm(cx, "x", "baz"); err != nil {
cx.t.Fatal(err)
}
err := ctlV3PutFailPerm(cx, "x", "baz")
require.ErrorContains(cx.t, err, "permission denied")
// revoke the open ended permission
cx.user, cx.pass = "root", "root"
@ -780,9 +773,8 @@ func authTestFromKeyPerm(cx ctlCtx) {
cx.user, cx.pass = "test-user", "pass"
for i := 0; i < 10; i++ {
key := fmt.Sprintf("z%d", i)
if err := ctlV3PutFailPerm(cx, key, "val"); err != nil {
cx.t.Fatal(err)
}
err := ctlV3PutFailPerm(cx, key, "val")
require.ErrorContains(cx.t, err, "permission denied")
}
// grant the entire keys
@ -810,9 +802,8 @@ func authTestFromKeyPerm(cx ctlCtx) {
cx.user, cx.pass = "test-user", "pass"
for i := 0; i < 10; i++ {
key := fmt.Sprintf("z%d", i)
if err := ctlV3PutFailPerm(cx, key, "val"); err != nil {
cx.t.Fatal(err)
}
err := ctlV3PutFailPerm(cx, key, "val")
require.ErrorContains(cx.t, err, "permission denied")
}
}
@ -848,9 +839,8 @@ func authLeaseTestTimeToLiveExpired(cx ctlCtx) {
authSetupTestUser(cx)
ttl := 3
if err := leaseTestTimeToLiveExpire(cx, ttl); err != nil {
cx.t.Fatalf("leaseTestTimeToLiveExpire: error (%v)", err)
}
err := leaseTestTimeToLiveExpire(cx, ttl)
require.NoError(cx.t, err)
}
func leaseTestTimeToLiveExpire(cx ctlCtx, ttl int) error {
@ -984,14 +974,13 @@ func authTestWatch(cx ctlCtx) {
var err error
if tt.want {
err = ctlV3Watch(cx, tt.args, tt.wkv...)
} else {
err = ctlV3WatchFailPerm(cx, tt.args)
}
if err != nil {
if cx.dialTimeout > 0 && !isGRPCTimedout(err) {
if err != nil && cx.dialTimeout > 0 && !isGRPCTimedout(err) {
cx.t.Errorf("watchTest #%d: ctlV3Watch error (%v)", i, err)
}
} else {
err = ctlV3WatchFailPerm(cx, tt.args)
// this will not have any meaningful error output, but the process fails due to the cancellation
require.ErrorContains(cx.t, err, "unexpected exit code")
}
<-donec
@ -1025,9 +1014,8 @@ func authTestRoleGet(cx ctlCtx) {
expected = []string{
"Error: etcdserver: permission denied",
}
if err := e2e.SpawnWithExpects(append(cx.PrefixArgs(), "role", "get", "root"), cx.envMap, expected...); err != nil {
cx.t.Fatal(err)
}
err := e2e.SpawnWithExpects(append(cx.PrefixArgs(), "role", "get", "root"), cx.envMap, expected...)
require.ErrorContains(cx.t, err, "permission denied")
}
func authTestUserGet(cx ctlCtx) {
@ -1056,9 +1044,8 @@ func authTestUserGet(cx ctlCtx) {
expected = []string{
"Error: etcdserver: permission denied",
}
if err := e2e.SpawnWithExpects(append(cx.PrefixArgs(), "user", "get", "root"), cx.envMap, expected...); err != nil {
cx.t.Fatal(err)
}
err := e2e.SpawnWithExpects(append(cx.PrefixArgs(), "user", "get", "root"), cx.envMap, expected...)
require.ErrorContains(cx.t, err, "permission denied")
}
func authTestRoleList(cx ctlCtx) {
@ -1207,16 +1194,14 @@ func certCNAndUsername(cx ctlCtx, noPassword bool) {
cx.t.Error(err)
}
// try a non granted key for both of them
// try a non-granted key for both of them
cx.user, cx.pass = "", ""
if err := ctlV3PutFailPerm(cx, "baz", "bar"); err != nil {
cx.t.Error(err)
}
err := ctlV3PutFailPerm(cx, "baz", "bar")
require.ErrorContains(cx.t, err, "permission denied")
cx.user, cx.pass = "test-user", "pass"
if err := ctlV3PutFailPerm(cx, "baz", "bar"); err != nil {
cx.t.Error(err)
}
err = ctlV3PutFailPerm(cx, "baz", "bar")
require.ErrorContains(cx.t, err, "permission denied")
}
func authTestCertCNAndUsername(cx ctlCtx) {

View File

@ -21,6 +21,7 @@ import (
"testing"
"time"
"github.com/stretchr/testify/require"
"go.etcd.io/etcd/pkg/v3/expect"
"go.etcd.io/etcd/tests/v3/framework/e2e"
)
@ -32,7 +33,7 @@ func TestCtlV3Elect(t *testing.T) {
func testElect(cx ctlCtx) {
name := "a"
holder, ch, err := ctlV3Elect(cx, name, "p1")
holder, ch, err := ctlV3Elect(cx, name, "p1", false)
if err != nil {
cx.t.Fatal(err)
}
@ -48,7 +49,7 @@ func testElect(cx ctlCtx) {
}
// blocked process that won't win the election
blocked, ch, err := ctlV3Elect(cx, name, "p2")
blocked, ch, err := ctlV3Elect(cx, name, "p2", true)
if err != nil {
cx.t.Fatal(err)
}
@ -59,7 +60,7 @@ func testElect(cx ctlCtx) {
}
// overlap with a blocker that will win the election
blockAcquire, ch, err := ctlV3Elect(cx, name, "p2")
blockAcquire, ch, err := ctlV3Elect(cx, name, "p2", false)
if err != nil {
cx.t.Fatal(err)
}
@ -74,8 +75,10 @@ func testElect(cx ctlCtx) {
if err = blocked.Signal(os.Interrupt); err != nil {
cx.t.Fatal(err)
}
if err = e2e.CloseWithTimeout(blocked, time.Second); err != nil {
cx.t.Fatal(err)
err = e2e.CloseWithTimeout(blocked, time.Second)
if err != nil {
// due to being blocked, this can potentially get killed and thus exit non-zero sometimes
require.ErrorContains(cx.t, err, "unexpected exit code")
}
// kill the holder with clean shutdown
@ -98,7 +101,7 @@ func testElect(cx ctlCtx) {
}
// ctlV3Elect creates a elect process with a channel listening for when it wins the election.
func ctlV3Elect(cx ctlCtx, name, proposal string) (*expect.ExpectProcess, <-chan string, error) {
func ctlV3Elect(cx ctlCtx, name, proposal string, expectFailure bool) (*expect.ExpectProcess, <-chan string, error) {
cmdArgs := append(cx.PrefixArgs(), "elect", name, proposal)
proc, err := e2e.SpawnCmd(cmdArgs, cx.envMap)
outc := make(chan string, 1)
@ -109,7 +112,9 @@ func ctlV3Elect(cx ctlCtx, name, proposal string) (*expect.ExpectProcess, <-chan
go func() {
s, xerr := proc.ExpectFunc(context.TODO(), func(string) bool { return true })
if xerr != nil {
cx.t.Errorf("expect failed (%v)", xerr)
if !expectFailure {
cx.t.Errorf("expect failed (%v)", xerr)
}
}
outc <- s
}()

View File

@ -15,10 +15,12 @@
package e2e
import (
"context"
"fmt"
"strings"
"testing"
"time"
"github.com/stretchr/testify/require"
"go.etcd.io/etcd/tests/v3/framework/e2e"
)
@ -50,9 +52,9 @@ func TestCtlV3GetRevokedCRL(t *testing.T) {
func testGetRevokedCRL(cx ctlCtx) {
// test reject
if err := ctlV3Put(cx, "k", "v", ""); err == nil || !strings.Contains(err.Error(), "Error:") {
cx.t.Fatalf("expected reset connection on put, got %v", err)
}
err := ctlV3Put(cx, "k", "v", "")
require.ErrorContains(cx.t, err, "context deadline exceeded")
// test accept
cx.epc.Cfg.IsClientCRL = false
if err := ctlV3Put(cx, "k", "v", ""); err != nil {
@ -216,9 +218,13 @@ func getKeysOnlyTest(cx ctlCtx) {
if err := e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, "key"); err != nil {
cx.t.Fatal(err)
}
if err := e2e.SpawnWithExpects(cmdArgs, cx.envMap, "val"); err == nil {
cx.t.Fatalf("got value but passed --keys-only")
}
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel()
lines, err := e2e.SpawnWithExpectLines(ctx, cmdArgs, cx.envMap, "key")
require.NoError(cx.t, err)
require.NotContains(cx.t, lines, "val", "got value but passed --keys-only")
}
func getCountOnlyTest(cx ctlCtx) {
@ -250,13 +256,14 @@ func getCountOnlyTest(cx ctlCtx) {
if err := e2e.SpawnWithExpects(cmdArgs, cx.envMap, "\"Count\" : 3"); err != nil {
cx.t.Fatal(err)
}
expected := []string{
"\"Count\" : 3",
}
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel()
cmdArgs = append(cx.PrefixArgs(), []string{"get", "--count-only", "key3", "--prefix", "--write-out=fields"}...)
if err := e2e.SpawnWithExpects(cmdArgs, cx.envMap, expected...); err == nil {
cx.t.Fatal(err)
}
lines, err := e2e.SpawnWithExpectLines(ctx, cmdArgs, cx.envMap, "\"Count\"")
require.NoError(cx.t, err)
require.NotContains(cx.t, lines, "\"Count\" : 3")
}
func delTest(cx ctlCtx) {

View File

@ -22,6 +22,7 @@ import (
"testing"
"time"
"github.com/stretchr/testify/require"
"go.etcd.io/etcd/pkg/v3/expect"
"go.etcd.io/etcd/tests/v3/framework/e2e"
)
@ -79,8 +80,10 @@ func testLock(cx ctlCtx) {
if err = blocked.Signal(os.Interrupt); err != nil {
cx.t.Fatal(err)
}
if err = e2e.CloseWithTimeout(blocked, time.Second); err != nil {
cx.t.Fatal(err)
err = e2e.CloseWithTimeout(blocked, time.Second)
if err != nil {
// due to being blocked, this can potentially get killed and thus exit non-zero sometimes
require.ErrorContains(cx.t, err, "unexpected exit code")
}
// kill the holder with clean shutdown
@ -113,9 +116,8 @@ func testLockWithCmd(cx ctlCtx) {
code := 3
awkCmd := []string{"awk", fmt.Sprintf("BEGIN{exit %d}", code)}
expect := fmt.Sprintf("Error: exit status %d", code)
if err := ctlV3LockWithCmd(cx, awkCmd, expect); err != nil {
cx.t.Fatal(err)
}
err := ctlV3LockWithCmd(cx, awkCmd, expect)
require.ErrorContains(cx.t, err, expect)
}
// ctlV3Lock creates a lock process with a channel listening for when it acquires the lock.
@ -130,7 +132,7 @@ func ctlV3Lock(cx ctlCtx, name string) (*expect.ExpectProcess, <-chan string, er
go func() {
s, xerr := proc.ExpectFunc(context.TODO(), func(string) bool { return true })
if xerr != nil {
cx.t.Errorf("expect failed (%v)", xerr)
require.ErrorContains(cx.t, xerr, "Error: context canceled")
}
outc <- s
}()
@ -142,5 +144,7 @@ func ctlV3LockWithCmd(cx ctlCtx, execCmd []string, as ...string) error {
// use command as lock name
cmdArgs := append(cx.PrefixArgs(), "lock", execCmd[0])
cmdArgs = append(cmdArgs, execCmd...)
return e2e.SpawnWithExpects(cmdArgs, cx.envMap, as...)
ctx, cancel := context.WithTimeout(context.Background(), 5*time.Second)
defer cancel()
return e2e.SpawnWithExpectsContext(ctx, cmdArgs, cx.envMap, as...)
}

View File

@ -21,6 +21,7 @@ import (
"testing"
"time"
"github.com/stretchr/testify/require"
"go.etcd.io/etcd/client/pkg/v3/transport"
"go.etcd.io/etcd/client/pkg/v3/types"
"go.etcd.io/etcd/client/v3"
@ -110,27 +111,34 @@ func testCtlV3MoveLeader(t *testing.T, cfg e2e.EtcdProcessClusterConfig, envVars
}
tests := []struct {
eps []string
expect string
eps []string
expect string
expectErr bool
}{
{ // request to non-leader
[]string{cx.epc.EndpointsV3()[(leadIdx+1)%3]},
"no leader endpoint given at ",
true,
},
{ // request to leader
[]string{cx.epc.EndpointsV3()[leadIdx]},
fmt.Sprintf("Leadership transferred from %s to %s", types.ID(leaderID), types.ID(transferee)),
false,
},
{ // request to all endpoints
cx.epc.EndpointsV3(),
fmt.Sprintf("Leadership transferred"),
false,
},
}
for i, tc := range tests {
prefix := cx.prefixArgs(tc.eps)
cmdArgs := append(prefix, "move-leader", types.ID(transferee).String())
if err := e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, tc.expect); err != nil {
t.Fatalf("#%d: %v", i, err)
err := e2e.SpawnWithExpectWithEnv(cmdArgs, cx.envMap, tc.expect)
if tc.expectErr {
require.ErrorContains(t, err, tc.expect)
} else {
require.Nilf(t, err, "#%d: %v", i, err)
}
}
}

View File

@ -25,6 +25,7 @@ import (
"testing"
"time"
"github.com/stretchr/testify/require"
"go.etcd.io/etcd/etcdutl/v3/snapshot"
"go.etcd.io/etcd/pkg/v3/expect"
"go.etcd.io/etcd/tests/v3/framework/e2e"
@ -90,10 +91,7 @@ func snapshotCorruptTest(cx ctlCtx) {
fpath),
cx.envMap,
"expected sha256")
if serr != nil {
cx.t.Fatal(serr)
}
require.ErrorContains(cx.t, serr, "Error: expected sha256")
}
// This test ensures that the snapshot status does not modify the snapshot file

View File

@ -248,9 +248,8 @@ func runCtlTest(t *testing.T, testFunc func(ctlCtx), testOfflineFunc func(ctlCtx
cx.envMap = make(map[string]string)
}
if cx.epc != nil {
if errC := cx.epc.Close(); errC != nil {
t.Fatalf("error closing etcd processes (%v)", errC)
}
cx.epc.Stop()
cx.epc.Close()
}
}()
@ -270,6 +269,7 @@ func runCtlTest(t *testing.T, testFunc func(ctlCtx), testOfflineFunc func(ctlCtx
}
t.Log("closing test cluster...")
assert.NoError(t, cx.epc.Stop())
assert.NoError(t, cx.epc.Close())
cx.epc = nil
t.Log("closed test cluster...")

View File

@ -19,13 +19,13 @@ import (
)
type txnRequests struct {
compare []string
ifSucess []string
ifFail []string
results []string
compare []string
ifSuccess []string
ifFail []string
results []string
}
func ctlV3Txn(cx ctlCtx, rqs txnRequests) error {
func ctlV3Txn(cx ctlCtx, rqs txnRequests, expectedExitErr bool) error {
// TODO: support non-interactive mode
cmdArgs := append(cx.PrefixArgs(), "txn")
if cx.interactive {
@ -52,7 +52,7 @@ func ctlV3Txn(cx ctlCtx, rqs txnRequests) error {
if err != nil {
return err
}
for _, req := range rqs.ifSucess {
for _, req := range rqs.ifSuccess {
if err = proc.Send(req + "\r"); err != nil {
return err
}
@ -80,5 +80,11 @@ func ctlV3Txn(cx ctlCtx, rqs txnRequests) error {
return err
}
}
return proc.Close()
err = proc.Close()
if expectedExitErr {
return nil
}
return err
}

View File

@ -21,6 +21,7 @@ import (
"strings"
"testing"
"github.com/stretchr/testify/require"
"go.etcd.io/etcd/pkg/v3/expect"
"go.etcd.io/etcd/tests/v3/framework/e2e"
)
@ -57,6 +58,7 @@ func TestEtcdMultiPeer(t *testing.T) {
for i := range procs {
if procs[i] != nil {
procs[i].Stop()
procs[i].Close()
}
}
}()
@ -128,6 +130,7 @@ func TestEtcdPeerCNAuth(t *testing.T) {
for i := range procs {
if procs[i] != nil {
procs[i].Stop()
procs[i].Close()
}
}
}()
@ -206,6 +209,7 @@ func TestEtcdPeerNameAuth(t *testing.T) {
for i := range procs {
if procs[i] != nil {
procs[i].Stop()
procs[i].Close()
}
os.RemoveAll(tmpdirs[i])
}
@ -287,9 +291,7 @@ func TestGrpcproxyAndCommonName(t *testing.T) {
}
err := e2e.SpawnWithExpect(argsWithNonEmptyCN, "cert has non empty Common Name")
if err != nil {
t.Errorf("Unexpected error: %s", err)
}
require.ErrorContains(t, err, "cert has non empty Common Name")
p, err := e2e.SpawnCmd(argsWithEmptyCN, nil)
defer func() {

View File

@ -37,7 +37,10 @@ func TestGateway(t *testing.T) {
eps := strings.Join(ec.EndpointsV3(), ",")
p := startGateway(t, eps)
defer p.Stop()
defer func() {
p.Stop()
p.Close()
}()
err = e2e.SpawnWithExpect([]string{e2e.BinPath.Etcdctl, "--endpoints=" + defaultGatewayEndpoint, "put", "foo", "bar"}, "OK\r\n")
if err != nil {

View File

@ -28,6 +28,8 @@ func TestInitDaemonNotifyWithoutQuorum(t *testing.T) {
t.Fatalf("Failed to initilize the etcd cluster: %v", err)
}
defer epc.Close()
// Remove two members, so that only one etcd will get started
epc.Procs = epc.Procs[:1]
@ -40,6 +42,4 @@ func TestInitDaemonNotifyWithoutQuorum(t *testing.T) {
e2e.AssertProcessLogs(t, epc.Procs[0], "startEtcd: timed out waiting for the ready notification")
// Expect log message indicating systemd notify message has been sent
e2e.AssertProcessLogs(t, epc.Procs[0], "notifying init daemon")
epc.Close()
}

View File

@ -24,6 +24,7 @@ import (
"github.com/coreos/go-semver/semver"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"go.etcd.io/etcd/api/v3/version"
"go.etcd.io/etcd/client/pkg/v3/fileutil"
"go.etcd.io/etcd/server/v3/storage/backend"
@ -155,7 +156,11 @@ func TestEtctlutlMigrate(t *testing.T) {
}
err = e2e.SpawnWithExpect(args, tc.expectLogsSubString)
if err != nil {
t.Fatal(err)
if tc.expectLogsSubString != "" {
require.ErrorContains(t, err, tc.expectLogsSubString)
} else {
t.Fatal(err)
}
}
t.Log("etcdutl migrate...")

View File

@ -21,6 +21,7 @@ import (
"fmt"
"testing"
"github.com/stretchr/testify/require"
"go.etcd.io/etcd/api/v3/version"
"go.etcd.io/etcd/tests/v3/framework/e2e"
)
@ -52,24 +53,16 @@ func cipherSuiteTestValid(cx ctlCtx) {
MetricsURLScheme: cx.cfg.MetricsURLScheme,
Ciphers: "ECDHE-RSA-AES128-GCM-SHA256", // TLS_ECDHE_RSA_WITH_AES_128_GCM_SHA256
}); err != nil {
cx.t.Fatalf("failed get with curl (%v)", err)
require.ErrorContains(cx.t, err, fmt.Sprintf(`etcd_server_version{server_version="%s"} 1`, version.Version))
}
}
func cipherSuiteTestMismatch(cx ctlCtx) {
var err error
for _, exp := range []string{"alert handshake failure", "failed setting cipher list"} {
err = e2e.CURLGet(cx.epc, e2e.CURLReq{
Endpoint: "/metrics",
Expected: exp,
MetricsURLScheme: cx.cfg.MetricsURLScheme,
Ciphers: "ECDHE-RSA-DES-CBC3-SHA", // TLS_ECDHE_RSA_WITH_3DES_EDE_CBC_SHA
})
if err == nil {
break
}
}
if err != nil {
cx.t.Fatalf("failed get with curl (%v)", err)
}
err := e2e.CURLGet(cx.epc, e2e.CURLReq{
Endpoint: "/metrics",
Expected: "failed setting cipher list",
MetricsURLScheme: cx.cfg.MetricsURLScheme,
Ciphers: "ECDHE-RSA-DES-CBC3-SHA", // TLS_ECDHE_RSA_WITH_3DES_EDE_CBC_SHA
})
require.ErrorContains(cx.t, err, "curl: (59) failed setting cipher list")
}

View File

@ -25,6 +25,7 @@ import (
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
pb "go.etcd.io/etcd/api/v3/etcdserverpb"
"go.etcd.io/etcd/client/pkg/v3/testutil"
"go.etcd.io/etcd/tests/v3/framework/e2e"
@ -212,7 +213,7 @@ func submitRangeAfterConcurrentWatch(cx ctlCtx, expectedValue string) {
cx.t.Log("Submitting range request...")
if err := e2e.CURLPost(cx.epc, e2e.CURLReq{Endpoint: "/v3/kv/range", Value: string(rangeData), Expected: expectedValue, Timeout: 5}); err != nil {
cx.t.Fatalf("testV3CurlMaxStream get failed, error: %v", err)
require.ErrorContains(cx.t, err, expectedValue)
}
cx.t.Log("range request done")
}

View File

@ -24,6 +24,7 @@ import (
"strconv"
"testing"
"github.com/stretchr/testify/require"
"go.etcd.io/etcd/api/v3/authpb"
pb "go.etcd.io/etcd/api/v3/etcdserverpb"
"go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
@ -141,9 +142,8 @@ func testV3CurlWatch(cx ctlCtx) {
cx.t.Fatalf("failed testV3CurlWatch put with curl using prefix (%s) (%v)", p, err)
}
// expects "bar", timeout after 2 seconds since stream waits forever
if err = e2e.CURLPost(cx.epc, e2e.CURLReq{Endpoint: path.Join(p, "/watch"), Value: wstr, Expected: `"YmFy"`, Timeout: 2}); err != nil {
cx.t.Fatalf("failed testV3CurlWatch watch with curl using prefix (%s) (%v)", p, err)
}
err = e2e.CURLPost(cx.epc, e2e.CURLReq{Endpoint: path.Join(p, "/watch"), Value: wstr, Expected: `"YmFy"`, Timeout: 2})
require.ErrorContains(cx.t, err, "unexpected exit code")
}
func testV3CurlTxn(cx ctlCtx) {

View File

@ -16,6 +16,7 @@ package e2e
import (
"context"
"errors"
"fmt"
"net/url"
"path"
@ -29,7 +30,6 @@ import (
"go.uber.org/zap/zaptest"
"go.etcd.io/etcd/api/v3/etcdserverpb"
"go.etcd.io/etcd/api/v3/v3rpc/rpctypes"
"go.etcd.io/etcd/server/v3/etcdserver"
"go.etcd.io/etcd/tests/v3/framework/config"
)
@ -664,18 +664,23 @@ func (epc *EtcdProcessCluster) CloseProc(ctx context.Context, finder func(EtcdPr
return fmt.Errorf("failed to find member ID: %w", err)
}
memberRemoved := false
for i := 0; i < 10; i++ {
_, err = memberCtl.MemberRemove(ctx, memberID)
if err != nil && strings.Contains(err.Error(), rpctypes.ErrGRPCUnhealthy.Error()) {
time.Sleep(500 * time.Millisecond)
continue
_, err := memberCtl.MemberRemove(ctx, memberID)
if err != nil && strings.Contains(err.Error(), "member not found") {
memberRemoved = true
break
}
break
time.Sleep(500 * time.Millisecond)
}
if err != nil {
return fmt.Errorf("failed to remove member: %w", err)
if !memberRemoved {
return errors.New("failed to remove member after 10 tries")
}
epc.lg.Info("successfully removed member", zap.String("acurl", proc.Config().Acurl))
// Then stop process
return proc.Close()
}

View File

@ -15,9 +15,11 @@
package e2e
import (
"context"
"fmt"
"math/rand"
"strings"
"time"
)
type CURLReq struct {
@ -38,6 +40,15 @@ type CURLReq struct {
Ciphers string
}
func (r CURLReq) timeoutDuration() time.Duration {
if r.Timeout != 0 {
return time.Duration(r.Timeout) * time.Second
}
// assume a sane default to finish a curl request
return 5 * time.Second
}
// CURLPrefixArgs builds the beginning of a curl command for a given key
// addressed to a random URL in the given cluster.
func CURLPrefixArgs(cfg *EtcdProcessClusterConfig, member EtcdProcess, method string, req CURLReq) []string {
@ -94,13 +105,20 @@ func CURLPrefixArgs(cfg *EtcdProcessClusterConfig, member EtcdProcess, method st
}
func CURLPost(clus *EtcdProcessCluster, req CURLReq) error {
return SpawnWithExpect(CURLPrefixArgs(clus.Cfg, clus.Procs[rand.Intn(clus.Cfg.ClusterSize)], "POST", req), req.Expected)
ctx, cancel := context.WithTimeout(context.Background(), req.timeoutDuration())
defer cancel()
return SpawnWithExpectsContext(ctx, CURLPrefixArgs(clus.Cfg, clus.Procs[rand.Intn(clus.Cfg.ClusterSize)], "POST", req), nil, req.Expected)
}
func CURLPut(clus *EtcdProcessCluster, req CURLReq) error {
return SpawnWithExpect(CURLPrefixArgs(clus.Cfg, clus.Procs[rand.Intn(clus.Cfg.ClusterSize)], "PUT", req), req.Expected)
ctx, cancel := context.WithTimeout(context.Background(), req.timeoutDuration())
defer cancel()
return SpawnWithExpectsContext(ctx, CURLPrefixArgs(clus.Cfg, clus.Procs[rand.Intn(clus.Cfg.ClusterSize)], "PUT", req), nil, req.Expected)
}
func CURLGet(clus *EtcdProcessCluster, req CURLReq) error {
return SpawnWithExpect(CURLPrefixArgs(clus.Cfg, clus.Procs[rand.Intn(clus.Cfg.ClusterSize)], "GET", req), req.Expected)
ctx, cancel := context.WithTimeout(context.Background(), req.timeoutDuration())
defer cancel()
return SpawnWithExpectsContext(ctx, CURLPrefixArgs(clus.Cfg, clus.Procs[rand.Intn(clus.Cfg.ClusterSize)], "GET", req), nil, req.Expected)
}

View File

@ -19,6 +19,7 @@ import (
"fmt"
"net/url"
"os"
"strings"
"syscall"
"testing"
"time"
@ -134,11 +135,18 @@ func (ep *EtcdServerProcess) Stop() (err error) {
if ep == nil || ep.proc == nil {
return nil
}
defer func() {
ep.proc = nil
}()
err = ep.proc.Stop()
ep.proc = nil
if err != nil {
return err
}
err = ep.proc.Close()
if err != nil && !strings.Contains(err.Error(), "unexpected exit code") {
return err
}
<-ep.donec
ep.donec = make(chan struct{})
if ep.cfg.Purl.Scheme == "unix" || ep.cfg.Purl.Scheme == "unixs" {
@ -183,11 +191,7 @@ func (ep *EtcdServerProcess) Kill() error {
}
func (ep *EtcdServerProcess) Wait() error {
err := ep.proc.Wait()
if err != nil {
ep.cfg.lg.Error("failed to wait for server exit", zap.String("name", ep.cfg.Name))
return err
}
ep.proc.Wait()
ep.cfg.lg.Info("server exited", zap.String("name", ep.cfg.Name))
ep.proc = nil
return nil

View File

@ -50,7 +50,11 @@ func SpawnWithExpectWithEnv(args []string, envVars map[string]string, expected s
}
func SpawnWithExpects(args []string, envVars map[string]string, xs ...string) error {
_, err := SpawnWithExpectLines(context.TODO(), args, envVars, xs...)
return SpawnWithExpectsContext(context.TODO(), args, envVars, xs...)
}
func SpawnWithExpectsContext(ctx context.Context, args []string, envVars map[string]string, xs ...string) error {
_, err := SpawnWithExpectLines(ctx, args, envVars, xs...)
return err
}
@ -74,26 +78,29 @@ func SpawnWithExpectLines(ctx context.Context, args []string, envVars map[string
lines = append(lines, l)
}
perr := proc.Close()
if perr != nil {
return lines, fmt.Errorf("err: %w, with output lines %v", perr, proc.Lines())
}
l := proc.LineCount()
if len(xs) == 0 && l != noOutputLineCount { // expect no output
return nil, fmt.Errorf("unexpected output from %v (got lines %q, line count %d) %v. Try EXPECT_DEBUG=TRUE", args, lines, l, l != noOutputLineCount)
}
return lines, perr
return lines, nil
}
func RunUtilCompletion(args []string, envVars map[string]string) ([]string, error) {
proc, err := SpawnCmd(args, envVars)
if err != nil {
return nil, fmt.Errorf("failed to spawn command: %w", err)
return nil, fmt.Errorf("failed to spawn command %v with error: %w", args, err)
}
defer proc.Stop()
perr := proc.Wait()
// make sure that all the outputs are received
proc.Close()
if perr != nil {
return nil, fmt.Errorf("unexpected error from command %v: %w", args, perr)
proc.Wait()
err = proc.Close()
if err != nil {
return nil, fmt.Errorf("failed to close command %v with error: %w", args, err)
}
return proc.Lines(), nil
}

View File

@ -69,7 +69,7 @@ func (f killFailpoint) Trigger(ctx context.Context, clus *e2e.EtcdProcessCluster
return err
}
err = member.Wait()
if err != nil {
if err != nil && !strings.Contains(err.Error(), "unexpected exit code") {
return err
}
err = member.Start(ctx)
@ -103,7 +103,7 @@ func (f goFailpoint) Trigger(ctx context.Context, clus *e2e.EtcdProcessCluster)
}
}
err = member.Wait()
if err != nil {
if err != nil && !strings.Contains(err.Error(), "unexpected exit code") {
return err
}
err = member.Start(ctx)