Simplify the test/upgrade package (#2663)

* Top-level structure of test execution changed

* Use t.Parallel properly

* Put back passing Log for test purposes

* More changes

* Prevent panic in unit tests

* All tests passing

* Simplify TestSkipAtBackgroundVerification

* Reduce background_verif_test

* Print name of continual test instead of Continual Test XXX

* Simplify sending stop event

* All tests passing

* background_verification_test using t.Fatal

* Remove old stuff

* Remove old comments

* Remove comment

* Properly initialize logger from LogConfig

* Quite early

* Remove backwards compatible Log from Configuration

* Get rid of suiteExecution failed field

* Minor cleanup

* Bring back most of TestSkipAtBackgroundVerification

* Minor syntactic fix

* Remove unused postVerifyContinual field

* Execute OnWait at least once

* Remove redundant close

* Close closeCh after failed Setup phase

* Do not call OnWait at least once

* Put back error message verification

* Unit tests passing with GOMAXPROCS=2

* remove redundant Eventing continual test as this functionality is
  already tested by Serving continual test
* remove redundant ShouldNotBeSkippedTest - this was a normal passing
  test which is already tested elsewhere

* Add a note about GOMAXPROCS and logical CPUs for upgrade tests

* Use completeSuite where no failures are expected

* Create new zaptest.Logger for each sub-test

This will print output on test failure

* Deprecate LogConfig.Config as it is unused

* Rename Parallel/UpgradeDowngrade to Run/Steps

* minor change in comment

* Fix test
This commit is contained in:
Martin Gencur 2023-01-10 15:46:49 +01:00 committed by GitHub
parent 50da24e8cf
commit ecf3863941
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
12 changed files with 179 additions and 531 deletions

View File

@ -18,30 +18,18 @@ package upgrade_test
import (
"fmt"
"strings"
"testing"
"knative.dev/pkg/test/upgrade"
)
const (
bgMessages = 5
)
func TestSkipAtBackgroundVerification(t *testing.T) {
config, buf := newConfig(t)
skipMsg := "It is expected to be skipped"
doneCh := make(chan struct{})
beforeVerifyCh := make(chan struct{})
inVerifyCh := make(chan struct{})
expectedTexts := []string{
upgradeTestRunning,
"DEBUG\tFinished \"ShouldBeSkipped\"",
"DEBUG\tFinished \"ShouldNotBeSkipped\"",
upgradeTestSuccess,
"INFO\tSetup 1",
"INFO\tSetup 2",
"INFO\tVerify 2",
"WARN\t" + skipMsg,
}
s := upgrade.Suite{
@ -51,42 +39,14 @@ func TestSkipAtBackgroundVerification(t *testing.T) {
// Setup
func(c upgrade.Context) {
c.Log.Info("Setup 1")
go func() {
// Log messages before Verify phase.
for i := 0; i < bgMessages; i++ {
msg := fmt.Sprintf("BeforeVerify %d", i)
c.Log.Info(msg)
expectedTexts = append(expectedTexts, msg)
}
close(beforeVerifyCh)
<-inVerifyCh
// Log messages while Verify phase is in progress.
for i := 0; i < bgMessages; i++ {
msg := fmt.Sprintf("InVerify %d", i)
c.Log.Info(msg)
expectedTexts = append(expectedTexts, msg)
}
close(doneCh)
}()
},
// Verify
func(c upgrade.Context) {
<-beforeVerifyCh
close(inVerifyCh)
<-doneCh
c.Log.Warn(skipMsg)
c.T.Skip(skipMsg)
c.Log.Info("Verify 1")
},
),
upgrade.NewBackgroundVerification("ShouldNotBeSkipped",
func(c upgrade.Context) {
c.Log.Info("Setup 2")
},
func(c upgrade.Context) {
c.Log.Info("Verify 2")
},
),
},
},
}
@ -99,28 +59,12 @@ func TestSkipAtBackgroundVerification(t *testing.T) {
"INFO\tVerify 1",
}})
assert.textContains(out, texts{elms: expectedTexts})
verifyBackgroundLogs(t, out)
}
func verifyBackgroundLogs(t *testing.T, logs string) {
t.Helper()
for _, line := range strings.Split(logs, "\n") {
if (strings.Contains(line, "BeforeVerify") ||
strings.Contains(line, "InVerify")) &&
!strings.Contains(line, "⏳") {
t.Fatalf("Message was not logged by background logger: %q", line)
}
}
}
func TestFailAtBackgroundVerification(t *testing.T) {
doneCh := make(chan struct{})
beforeVerifyCh := make(chan struct{})
inVerifyCh := make(chan struct{})
const failingVerification = "FailAtVerification"
expectedTexts := []string{
upgradeTestRunning,
fmt.Sprintf("DEBUG\tFinished %q", failingVerification),
upgradeTestFailure,
"INFO\tSetup 1",
"INFO\tVerify 1",
@ -132,29 +76,9 @@ func TestFailAtBackgroundVerification(t *testing.T) {
// Setup
func(c upgrade.Context) {
c.Log.Info("Setup 1")
go func() {
// Log messages before Verify phase.
for i := 0; i < bgMessages; i++ {
msg := fmt.Sprintf("BeforeVerify %d", i)
c.Log.Info(msg)
expectedTexts = append(expectedTexts, msg)
}
close(beforeVerifyCh)
<-inVerifyCh
// Log messages while Verify phase is in progress.
for i := 0; i < bgMessages; i++ {
msg := fmt.Sprintf("InVerify %d", i)
c.Log.Info(msg)
expectedTexts = append(expectedTexts, msg)
}
close(doneCh)
}()
},
// Verify
func(c upgrade.Context) {
<-beforeVerifyCh
close(inVerifyCh)
<-doneCh
c.Log.Info("Verify 1")
c.T.Fatal(failureTestingMessage)
c.Log.Info("Verify 2")
@ -189,8 +113,7 @@ func TestFailAtBackgroundVerification(t *testing.T) {
assert.textContains(out, texts{elms: expectedTexts})
assert.textContains(testOutput, texts{
elms: []string{
fmt.Sprintf("--- FAIL: %s/VerifyContinualTests/%s", t.Name(), failingVerification),
fmt.Sprintf("--- FAIL: %s/Run/%s", t.Name(), failingVerification),
},
})
verifyBackgroundLogs(t, out)
}

View File

@ -26,20 +26,21 @@ import (
)
func TestSuiteExecuteWithFailures(t *testing.T) {
for i := 1; i <= 8; i++ {
for j := 1; j <= 2; j++ {
suite := completeSuite()
for i, st := range createSteps(suite) {
for j := range st.ops.ops {
fp := failurePoint{
step: i,
element: j,
step: i + 1,
element: j + 1,
}
testSuiteExecuteWithFailingStep(fp, t)
testSuiteExecuteWithFailingStep(suite, fp, t)
}
}
}
var allTestsFilter = func(_, _ string) (bool, error) { return true, nil }
func testSuiteExecuteWithFailingStep(fp failurePoint, t *testing.T) {
func testSuiteExecuteWithFailingStep(suite upgrade.Suite, fp failurePoint, t *testing.T) {
testName := fmt.Sprintf("FailAt-%d-%d", fp.step, fp.element)
t.Run(testName, func(t *testing.T) {
assert := assertions{tb: t}
@ -48,15 +49,15 @@ func testSuiteExecuteWithFailingStep(fp failurePoint, t *testing.T) {
c upgrade.Configuration
buf fmt.Stringer
)
suite := completeSuiteExample(fp)
txt := expectedTexts(suite, fp)
suiteWithFailures := enrichSuiteWithFailures(suite, fp)
txt := expectedTexts(suiteWithFailures, fp)
txt.append(upgradeTestRunning, upgradeTestFailure)
it := []testing.InternalTest{{
Name: testName,
F: func(t *testing.T) {
c, buf = newConfig(t)
suite.Execute(c)
suiteWithFailures.Execute(c)
},
}}
var ok bool

View File

@ -17,38 +17,39 @@ limitations under the License.
package upgrade
import (
"bytes"
"strings"
"testing"
"time"
"go.uber.org/zap/zapcore"
"go.uber.org/zap"
"go.uber.org/zap/zaptest"
)
// Execute the Suite of upgrade tests with a Configuration given.
// When the suite includes Continual tests the number of logical CPUs
// usable by the test process must be at least <NUMBER OF CONTINUAL TESTS> + 1.
// The -parallel test flag or GOMAXPROCS environment variable might be
// used to adjust the settings.
func (s *Suite) Execute(c Configuration) {
l, err := c.logger(c.T)
if err != nil {
c.T.Fatal("Failed to build logger:", err)
return
}
l := c.logger(c.T)
se := suiteExecution{
suite: enrichSuite(s),
configuration: c,
failed: false,
}
l.Info("🏃 Running upgrade test suite...")
se.execute()
if !se.failed {
if !c.T.Failed() {
l.Info("🥳🎉 Success! Upgrade suite completed without errors.")
} else {
l.Error("💣🤬💔️ Upgrade suite have failed!")
}
}
func (c Configuration) logger(t *testing.T) *zap.SugaredLogger {
return zaptest.NewLogger(t, zaptest.WrapOptions(c.Options...)).Sugar()
}
// NewOperation creates a new upgrade operation or test.
func NewOperation(name string, handler func(c Context)) Operation {
return &simpleOperation{name: name, handler: handler}
@ -56,14 +57,14 @@ func NewOperation(name string, handler func(c Context)) Operation {
// NewBackgroundVerification is convenience function to easily setup a
// background operation that will setup environment and then verify environment
// status after receiving a StopEvent.
// status after receiving a stop event.
func NewBackgroundVerification(name string, setup func(c Context), verify func(c Context)) BackgroundOperation {
return NewBackgroundOperation(name, setup, func(bc BackgroundContext) {
WaitForStopEvent(bc, WaitForStopEventConfiguration{
Name: name,
OnStop: func(event StopEvent) {
OnStop: func() {
verify(Context{
T: event.T,
T: bc.T,
Log: bc.Log,
})
},
@ -90,8 +91,9 @@ func NewBackgroundOperation(name string, setup func(c Context),
func WaitForStopEvent(bc BackgroundContext, w WaitForStopEventConfiguration) {
for {
select {
case stopEvent := <-bc.Stop:
handleStopEvent(stopEvent, bc, w)
case <-bc.Stop:
bc.Log.Debugf("%s received a stop event", w.Name)
w.OnStop()
return
default:
w.OnWait(bc, w)
@ -100,32 +102,6 @@ func WaitForStopEvent(bc BackgroundContext, w WaitForStopEventConfiguration) {
}
}
// Name returns a friendly human readable text.
func (s *StopEvent) Name() string {
return s.name
}
func handleStopEvent(
se StopEvent,
bc BackgroundContext,
wc WaitForStopEventConfiguration,
) {
bc.Log.Debugf("%s have received a stop event: %s", wc.Name, se.Name())
logFn := se.logger.Info
logFn(wrapLog(bc.logBuffer.Dump()))
defer func() {
defer close(se.Finished)
if se.T.Failed() {
logFn = se.logger.Error
}
logFn(wrapLog(bc.logBuffer.Dump()))
}()
wc.OnStop(se)
}
func enrichSuite(s *Suite) *enrichedSuite {
es := &enrichedSuite{
installations: s.Installations,
@ -139,7 +115,7 @@ func enrichSuite(s *Suite) *enrichedSuite {
for i, test := range s.Tests.Continual {
es.tests.continual[i] = stoppableOperation{
BackgroundOperation: test,
stop: make(chan StopEvent),
stop: make(chan struct{}),
}
}
return es
@ -168,64 +144,9 @@ func (s *simpleBackgroundOperation) Setup() func(c Context) {
// Handler will be executed in background while upgrade/downgrade is being
// executed. It can be used to constantly validate environment during that
// time and/or wait for StopEvent being sent. After StopEvent is received
// time and/or wait for a stop event being sent. After a stop event is received
// user should validate environment, clean up resources, and report found
// issues to testing.T forwarded in StepEvent.
// issues to testing.T forwarded in BackgroundContext.
func (s *simpleBackgroundOperation) Handler() func(bc BackgroundContext) {
return s.handler
}
func (b *threadSafeBuffer) Read(p []byte) (n int, err error) {
b.Mutex.Lock()
defer b.Mutex.Unlock()
return b.Buffer.Read(p)
}
func (b *threadSafeBuffer) Write(p []byte) (n int, err error) {
b.Mutex.Lock()
defer b.Mutex.Unlock()
return b.Buffer.Write(p)
}
// Dump returns the previous content of the buffer and creates a new
// empty buffer for future writes.
func (b *threadSafeBuffer) Dump() string {
b.Mutex.Lock()
defer b.Mutex.Unlock()
buf := b.Buffer.String()
b.Buffer = bytes.Buffer{}
return buf
}
// newInMemoryLoggerBuffer creates a logger that writes logs into a byte buffer.
// This byte buffer is returned and can be used to process the logs at later stage.
func newInMemoryLoggerBuffer(config Configuration) (*zap.Logger, *threadSafeBuffer) {
logConfig := config.logConfig()
buf := &threadSafeBuffer{}
core := zapcore.NewCore(
zapcore.NewConsoleEncoder(logConfig.Config.EncoderConfig),
zapcore.AddSync(buf),
logConfig.Config.Level)
var opts []zap.Option
if logConfig.Config.Development {
opts = append(opts, zap.Development())
}
if !logConfig.Config.DisableCaller {
opts = append(opts, zap.AddCaller())
}
stackLevel := zap.ErrorLevel
if logConfig.Config.Development {
stackLevel = zap.WarnLevel
}
if !logConfig.Config.DisableStacktrace {
opts = append(opts, zap.AddStacktrace(stackLevel))
}
return zap.New(core, opts...), buf
}
func wrapLog(log string) string {
s := strings.Join(strings.Split(log, "\n"), "\n⏳ ")
return "Rewind of background logs (prefix ⏳):\n" +
"⏳ " + strings.TrimSuffix(s, "⏳ ")
}

View File

@ -48,7 +48,7 @@ func TestExpectedTextsForEmptySuite(t *testing.T) {
func TestExpectedTextsForCompleteSuite(t *testing.T) {
assert := assertions{tb: t}
fp := notFailing
suite := completeSuiteExample(fp)
suite := completeSuiteExampleWithFailures(fp)
txt := expectedTexts(suite, fp)
expected := []string{
"1) 💿 Installing base installations. 2 are registered.",
@ -57,9 +57,8 @@ func TestExpectedTextsForCompleteSuite(t *testing.T) {
"2) ✅️️ Testing functionality before upgrade is performed. 2 tests are registered.",
`2.1) Testing with "Serving pre upgrade test".`,
`2.2) Testing with "Eventing pre upgrade test".`,
"3) 🔄 Starting continual tests. 2 tests are registered.",
"3) 🔄 Starting continual tests. 1 tests are registered.",
`3.1) Starting continual tests of "Serving continual test".`,
`3.2) Starting continual tests of "Eventing continual test".`,
"4) 📀 Upgrading with 2 registered operations.",
`4.1) Upgrading with "Serving HEAD".`,
`4.2) Upgrading with "Eventing HEAD".`,
@ -72,9 +71,6 @@ func TestExpectedTextsForCompleteSuite(t *testing.T) {
"7) ✅️️ Testing functionality after downgrade is performed. 2 tests are registered.",
`7.1) Testing with "Serving post downgrade test".`,
`7.2) Testing with "Eventing post downgrade test".`,
"8) ✋ Verifying 2 running continual tests.",
`8.1) Verifying "Serving continual test".`,
`8.2) Verifying "Eventing continual test".`,
}
assert.arraysEqual(txt.elms, expected)
}
@ -85,7 +81,7 @@ func TestExpectedTextsForFailingCompleteSuite(t *testing.T) {
step: 2,
element: 1,
}
suite := completeSuiteExample(fp)
suite := completeSuiteExampleWithFailures(fp)
txt := expectedTexts(suite, fp)
expected := []string{
"1) 💿 Installing base installations. 2 are registered.",
@ -110,22 +106,20 @@ func TestSuiteExecuteEmpty(t *testing.T) {
txt := expectedTexts(suite, fp)
txt.append(upgradeTestRunning, upgradeTestSuccess)
assert.textContains(output, txt)
}
func TestSuiteExecuteWithComplete(t *testing.T) {
assert := assertions{t}
c, buf := newConfig(t)
fp := notFailing
suite := completeSuiteExample(fp)
suite := completeSuite()
upgrade.DefaultOnWait = probeOnWaitFunc()
suite.Execute(c)
output := buf.String()
if c.T.Failed() {
return
}
expected := expectedTexts(suite, fp)
expected := expectedTexts(suite, notFailing)
expected.append(upgradeTestRunning, upgradeTestSuccess)
expected.append(
"Installing Serving stable 0.17.1",
@ -134,12 +128,9 @@ func TestSuiteExecuteWithComplete(t *testing.T) {
"Installing Eventing HEAD at 12f67cc",
"Installing Serving stable 0.17.1",
"Installing Eventing stable 0.17.2",
"Serving have received a stop event",
"Eventing continual test have received a stop event",
"Serving received a stop event",
"Running Serving continual test",
"Stopping and verify of Eventing continual test",
"Serving - probing functionality...",
"Eventing continual test - probing functionality...",
)
assert.textContains(output, expected)
}

View File

@ -1,110 +0,0 @@
/*
Copyright 2021 The Knative 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 upgrade
import (
"errors"
"fmt"
"net/url"
"strings"
"testing"
"go.uber.org/zap"
)
const testingTScheme = "testing-t"
var (
// ErrInvalidTestingOutputMethod when given invalid testing output method.
ErrInvalidTestingOutputMethod = errors.New("invalid testing output method")
// ErrNoTestingTRegisteredForTest when no testing.TB registered for test.
ErrNoTestingTRegisteredForTest = errors.New("no testing.TB registered for test")
testingTeeBees = make(map[string]testing.TB)
)
func (c Configuration) logger(tb testing.TB) (*zap.SugaredLogger, error) {
// TODO(mgencur): Remove when dependent repositories use LogConfig instead of Log.
// This is for backwards compatibility.
if c.Log != nil {
return c.Log.Sugar(), nil
}
testName := tb.Name()
testingTeeBees[testName] = tb
cfg := c.logConfig().Config
outs := []string{fmt.Sprintf("%s://Log/%s", testingTScheme, testName)}
if isStandardOutputOnly(cfg.OutputPaths) {
cfg.OutputPaths = outs
} else {
cfg.OutputPaths = append(cfg.OutputPaths, outs...)
}
errOuts := []string{fmt.Sprintf("%s://Error/%s", testingTScheme, testName)}
if isStandardOutputOnly(cfg.ErrorOutputPaths) {
cfg.ErrorOutputPaths = errOuts
} else {
cfg.ErrorOutputPaths = append(cfg.ErrorOutputPaths, errOuts...)
}
err := zap.RegisterSink(testingTScheme, func(url *url.URL) (zap.Sink, error) {
var ok bool
testName = strings.TrimPrefix(url.Path, "/")
tb, ok = testingTeeBees[testName]
if !ok {
return nil, fmt.Errorf("%w: %s", ErrNoTestingTRegisteredForTest, testName)
}
switch url.Host {
case "Log":
return sink{writer: tb.Log}, nil
case "Error":
return sink{writer: tb.Error}, nil
}
return nil, fmt.Errorf("%w: %s", ErrInvalidTestingOutputMethod, url.Host)
})
if err != nil && !strings.HasPrefix(err.Error(), "sink factory already registered") {
return nil, err
}
var logger *zap.Logger
logger, err = cfg.Build(c.logConfig().Options...)
if err != nil {
return nil, err
}
return logger.Sugar(), nil
}
type sink struct {
writer func(args ...interface{})
}
func (t sink) Write(bytes []byte) (n int, err error) {
msg := string(bytes)
t.writer(msg)
return len(bytes), nil
}
func (t sink) Sync() error {
return nil
}
func (t sink) Close() error {
return nil
}
// isStandardOutputOnly checks the output paths and returns true, if they
// contain only standard outputs.
func isStandardOutputOnly(paths []string) bool {
return len(paths) == 1 && (paths[0] == "stderr" || paths[0] == "stdout")
}

View File

@ -16,15 +16,9 @@ limitations under the License.
package upgrade
import (
"bytes"
"sync"
)
type suiteExecution struct {
suite *enrichedSuite
configuration Configuration
failed bool
}
type enrichedSuite struct {
@ -41,7 +35,7 @@ type enrichedTests struct {
type stoppableOperation struct {
BackgroundOperation
stop chan StopEvent
stop chan struct{}
}
type operationGroup struct {
@ -63,10 +57,3 @@ type simpleBackgroundOperation struct {
setup func(c Context)
handler func(bc BackgroundContext)
}
// threadSafeBuffer avoids race conditions on bytes.Buffer.
// See: https://stackoverflow.com/a/36226525/844449
type threadSafeBuffer struct {
bytes.Buffer
sync.Mutex
}

View File

@ -22,8 +22,8 @@ import (
const skippingOperationTemplate = `Skipping "%s" as previous operation have failed`
func (se *suiteExecution) installingBase(num int) {
se.processOperationGroup(operationGroup{
func (se *suiteExecution) installingBase(t *testing.T, num int) {
se.processOperationGroup(t, operationGroup{
num: num,
operations: se.suite.installations.Base,
groupName: "InstallingBase",
@ -33,8 +33,8 @@ func (se *suiteExecution) installingBase(num int) {
})
}
func (se *suiteExecution) preUpgradeTests(num int) {
se.processOperationGroup(operationGroup{
func (se *suiteExecution) preUpgradeTests(t *testing.T, num int) {
se.processOperationGroup(t, operationGroup{
num: num,
operations: se.suite.tests.preUpgrade,
groupName: "PreUpgradeTests",
@ -45,93 +45,43 @@ func (se *suiteExecution) preUpgradeTests(num int) {
})
}
func (se *suiteExecution) startContinualTests(num int) {
func (se *suiteExecution) runContinualTests(t *testing.T, num int, stopCh <-chan struct{}) {
l := se.configuration.logger(t)
operations := se.suite.tests.continual
groupTemplate := "%d) 🔄 Starting continual tests. " +
"%d tests are registered."
elementTemplate := `%d.%d) Starting continual tests of "%s".`
numOps := len(operations)
se.configuration.T.Run("ContinualTests", func(t *testing.T) {
l, err := se.configuration.logger(t)
if err != nil {
t.Fatal(err)
}
if numOps > 0 {
l.Infof(groupTemplate, num, numOps)
for i := range operations {
operation := operations[i]
l.Infof(elementTemplate, num, i+1, operation.Name())
if se.failed {
l.Debugf(skippingOperationTemplate, operation.Name())
return
}
if numOps > 0 {
l.Infof(groupTemplate, num, numOps)
for i := range operations {
operation := operations[i]
l.Debugf(elementTemplate, num, i+1, operation.Name())
t.Run(operation.Name(), func(t *testing.T) {
l := se.configuration.logger(t)
setup := operation.Setup()
logger, buffer := newInMemoryLoggerBuffer(se.configuration)
t.Run("Setup"+operation.Name(), func(t *testing.T) {
l, err = se.configuration.logger(t)
if err != nil {
t.Fatal(err)
}
setup(Context{T: t, Log: logger.Sugar()})
})
handler := operation.Handler()
go func() {
handler(BackgroundContext{
Log: logger.Sugar(),
Stop: operation.stop,
logBuffer: buffer,
})
}()
se.failed = se.failed || t.Failed()
if se.failed {
// need to dump logs here, because verify will not be executed.
l.Error(wrapLog(buffer.Dump()))
setup(Context{T: t, Log: l})
if t.Failed() {
return
}
}
} else {
l.Infof("%d) 🔄 No continual tests registered. Skipping.", num)
}
})
}
func (se *suiteExecution) verifyContinualTests(num int) {
testsCount := len(se.suite.tests.continual)
if testsCount > 0 {
se.configuration.T.Run("VerifyContinualTests", func(t *testing.T) {
l, err := se.configuration.logger(t)
if err != nil {
t.Fatal(err)
}
l.Infof("%d) ✋ Verifying %d running continual tests.", num, testsCount)
for i, operation := range se.suite.tests.continual {
t.Run(operation.Name(), func(t *testing.T) {
l, err = se.configuration.logger(t)
if err != nil {
t.Fatal(err)
}
l.Infof(`%d.%d) Verifying "%s".`, num, i+1, operation.Name())
finished := make(chan struct{})
operation.stop <- StopEvent{
T: t,
Finished: finished,
logger: l,
name: "Stop of " + operation.Name(),
}
<-finished
se.failed = se.failed || t.Failed()
l.Debugf(`Finished "%s"`, operation.Name())
t.Parallel()
handle := operation.Handler()
// Blocking operation.
handle(BackgroundContext{
T: t,
Log: l,
Stop: stopCh,
})
}
})
l.Debugf(`Finished "%s"`, operation.Name())
})
}
} else {
l.Infof("%d) 🔄 No continual tests registered. Skipping.", num)
}
}
func (se *suiteExecution) upgradeWith(num int) {
se.processOperationGroup(operationGroup{
func (se *suiteExecution) upgradeWith(t *testing.T, num int) {
se.processOperationGroup(t, operationGroup{
num: num,
operations: se.suite.installations.UpgradeWith,
groupName: "UpgradeWith",
@ -141,8 +91,8 @@ func (se *suiteExecution) upgradeWith(num int) {
})
}
func (se *suiteExecution) postUpgradeTests(num int) {
se.processOperationGroup(operationGroup{
func (se *suiteExecution) postUpgradeTests(t *testing.T, num int) {
se.processOperationGroup(t, operationGroup{
num: num,
operations: se.suite.tests.postUpgrade,
groupName: "PostUpgradeTests",
@ -153,8 +103,8 @@ func (se *suiteExecution) postUpgradeTests(num int) {
})
}
func (se *suiteExecution) downgradeWith(num int) {
se.processOperationGroup(operationGroup{
func (se *suiteExecution) downgradeWith(t *testing.T, num int) {
se.processOperationGroup(t, operationGroup{
num: num,
operations: se.suite.installations.DowngradeWith,
groupName: "DowngradeWith",
@ -164,8 +114,8 @@ func (se *suiteExecution) downgradeWith(num int) {
})
}
func (se *suiteExecution) postDowngradeTests(num int) {
se.processOperationGroup(operationGroup{
func (se *suiteExecution) postDowngradeTests(t *testing.T, num int) {
se.processOperationGroup(t, operationGroup{
num: num,
operations: se.suite.tests.postDowngrade,
groupName: "PostDowngradeTests",

View File

@ -69,11 +69,6 @@ var (
element: `%d.%d) Testing with "%s".`,
skipped: "%d) ✅️️ No post downgrade tests registered. Skipping.",
}),
verifyContinual: createMessages(formats{
starting: "%d) ✋ Verifying %d running continual tests.",
element: `%d.%d) Verifying "%s".`,
skipped: "",
}),
}
)
@ -111,7 +106,7 @@ func servingComponent() component {
bc.Log.Info("Running Serving continual test")
upgrade.WaitForStopEvent(bc, upgrade.WaitForStopEventConfiguration{
Name: "Serving",
OnStop: func(event upgrade.StopEvent) {
OnStop: func() {
bc.Log.Info("Stopping and verify of Serving continual test")
time.Sleep(shortWait)
},
@ -150,16 +145,6 @@ func eventingComponent() component {
c.Log.Info("Running Eventing post downgrade test")
time.Sleep(shortWait)
}),
continual: upgrade.NewBackgroundVerification("Eventing continual test",
func(c upgrade.Context) {
c.Log.Info("Setup of Eventing continual test")
time.Sleep(shortWait)
},
func(c upgrade.Context) {
c.Log.Info("Stopping and verify of Eventing continual test")
time.Sleep(shortWait)
},
),
},
}
}

View File

@ -20,30 +20,22 @@ import (
"testing"
)
func (se *suiteExecution) processOperationGroup(op operationGroup) {
se.configuration.T.Run(op.groupName, func(t *testing.T) {
l, err := se.configuration.logger(t)
if err != nil {
t.Fatal(err)
}
func (se *suiteExecution) processOperationGroup(t *testing.T, op operationGroup) {
t.Run(op.groupName, func(t *testing.T) {
l := se.configuration.logger(t)
if len(op.operations) > 0 {
l.Infof(op.groupTemplate, op.num, len(op.operations))
for i, operation := range op.operations {
l.Infof(op.elementTemplate, op.num, i+1, operation.Name())
if se.failed {
if t.Failed() {
l.Debugf(skippingOperationTemplate, operation.Name())
return
}
handler := operation.Handler()
t.Run(operation.Name(), func(t *testing.T) {
l, err = se.configuration.logger(t)
if err != nil {
t.Fatal(err)
}
handler(Context{T: t, Log: l})
})
se.failed = se.failed || t.Failed()
if se.failed {
if t.Failed() {
return
}
}
@ -55,38 +47,50 @@ func (se *suiteExecution) processOperationGroup(op operationGroup) {
func (se *suiteExecution) execute() {
idx := 1
operations := []func(num int){
stopCh := make(chan struct{})
t := se.configuration.T
operations := []func(t *testing.T, num int){
se.installingBase,
se.preUpgradeTests,
}
for _, operation := range operations {
operation(idx)
operation(t, idx)
idx++
if se.failed {
if t.Failed() {
return
}
}
se.startContinualTests(idx)
idx++
if se.failed {
return
}
defer func() {
se.verifyContinualTests(idx)
}()
t.Run("Run", func(t *testing.T) {
// Calls t.Parallel() after doing setup phase. The second part runs in parallel
// with Steps below.
se.runContinualTests(t, idx, stopCh)
operations = []func(num int){
se.upgradeWith,
se.postUpgradeTests,
se.downgradeWith,
se.postDowngradeTests,
}
for _, operation := range operations {
operation(idx)
idx++
if se.failed {
// At this point only the setup phase of continual tests was done. We want
// to quit early in the event of failures.
if t.Failed() {
close(stopCh)
return
}
}
operations = []func(t *testing.T, num int){
se.upgradeWith,
se.postUpgradeTests,
se.downgradeWith,
se.postDowngradeTests,
}
t.Run("Steps", func(t *testing.T) {
defer close(stopCh)
// The rest of this test group will run in parallel with individual continual tests.
t.Parallel()
for _, operation := range operations {
operation(t, idx)
idx++
if t.Failed() {
return
}
}
})
})
}

View File

@ -17,7 +17,6 @@ limitations under the License.
package upgrade_test
import (
"bytes"
"fmt"
"testing"
@ -32,25 +31,25 @@ const (
)
func newConfig(t *testing.T) (upgrade.Configuration, fmt.Stringer) {
var buf bytes.Buffer
buf := threadSafeBuffer{}
cfg := zap.NewDevelopmentConfig()
cfg.EncoderConfig.TimeKey = ""
cfg.EncoderConfig.CallerKey = ""
syncedBuf := zapcore.AddSync(&buf)
c := upgrade.Configuration{
T: t,
LogConfig: upgrade.LogConfig{
Config: cfg,
Options: []zap.Option{
zap.WrapCore(func(core zapcore.Core) zapcore.Core {
return zapcore.NewCore(
zapcore.NewConsoleEncoder(cfg.EncoderConfig),
zapcore.NewMultiWriteSyncer(syncedBuf), cfg.Level)
}),
zap.ErrorOutput(syncedBuf),
},
logConfig := upgrade.LogConfig{
Options: []zap.Option{
zap.WrapCore(func(core zapcore.Core) zapcore.Core {
return zapcore.NewCore(
zapcore.NewConsoleEncoder(cfg.EncoderConfig),
zapcore.NewMultiWriteSyncer(syncedBuf), cfg.Level)
}),
zap.ErrorOutput(syncedBuf),
},
}
c := upgrade.Configuration{
T: t,
LogConfig: logConfig,
}
return c, &buf
}
@ -98,12 +97,6 @@ func createSteps(s upgrade.Suite) []*step {
updateSuite: func(ops operations, s *upgrade.Suite) {
s.Tests.PostDowngrade = ops.asOperations()
},
}, {
messages: messageFormatters.verifyContinual,
ops: continualTestsGeneralized,
updateSuite: func(ops operations, s *upgrade.Suite) {
s.Tests.Continual = ops.asBackgroundOperation()
},
}}
}
@ -171,10 +164,10 @@ func (tt *texts) append(messages ...string) {
}
}
func completeSuiteExample(fp failurePoint) upgrade.Suite {
func completeSuite() upgrade.Suite {
serving := servingComponent()
eventing := eventingComponent()
suite := upgrade.Suite{
return upgrade.Suite{
Tests: upgrade.Tests{
PreUpgrade: []upgrade.Operation{
serving.tests.preUpgrade, eventing.tests.preUpgrade,
@ -186,7 +179,7 @@ func completeSuiteExample(fp failurePoint) upgrade.Suite {
serving.tests.postDowngrade, eventing.tests.postDowngrade,
},
Continual: []upgrade.BackgroundOperation{
serving.tests.continual, eventing.tests.continual,
serving.tests.continual,
},
},
Installations: upgrade.Installations{
@ -201,7 +194,10 @@ func completeSuiteExample(fp failurePoint) upgrade.Suite {
},
},
}
return enrichSuiteWithFailures(suite, fp)
}
func completeSuiteExampleWithFailures(fp failurePoint) upgrade.Suite {
return enrichSuiteWithFailures(completeSuite(), fp)
}
func emptySuiteExample() upgrade.Suite {
@ -254,8 +250,8 @@ func (o *operation) fail(setupFail bool) {
} else {
prev := o.bg
o.bg = upgrade.NewBackgroundOperation(testName, func(c upgrade.Context) {
handler := prev.Setup()
handler(c)
setup := prev.Setup()
setup(c)
if setupFail {
c.T.Error(failureTestingMessage)
c.Log.Error(failureTestingMessage)
@ -263,9 +259,9 @@ func (o *operation) fail(setupFail bool) {
}, func(bc upgrade.BackgroundContext) {
upgrade.WaitForStopEvent(bc, upgrade.WaitForStopEventConfiguration{
Name: testName,
OnStop: func(event upgrade.StopEvent) {
OnStop: func() {
if !setupFail {
event.T.Error(failureTestingMessage)
bc.T.Error(failureTestingMessage)
bc.Log.Error(failureTestingMessage)
}
},

View File

@ -16,7 +16,12 @@ limitations under the License.
package upgrade_test
import "knative.dev/pkg/test/upgrade"
import (
"bytes"
"sync"
"knative.dev/pkg/test/upgrade"
)
type failurePoint struct {
step int
@ -57,14 +62,13 @@ type messages struct {
}
type messageFormatterRepository struct {
baseInstall messages
preUpgrade messages
startContinual messages
upgrade messages
postUpgrade messages
downgrade messages
postDowngrade messages
verifyContinual messages
baseInstall messages
preUpgrade messages
startContinual messages
upgrade messages
postUpgrade messages
downgrade messages
postDowngrade messages
}
type component struct {
@ -83,3 +87,22 @@ type tests struct {
continual upgrade.BackgroundOperation
postDowngrade upgrade.Operation
}
// threadSafeBuffer avoids race conditions on bytes.Buffer.
// See: https://stackoverflow.com/a/36226525/844449
type threadSafeBuffer struct {
bytes.Buffer
sync.Mutex
}
func (b *threadSafeBuffer) Read(p []byte) (n int, err error) {
b.Mutex.Lock()
defer b.Mutex.Unlock()
return b.Buffer.Read(p)
}
func (b *threadSafeBuffer) Write(p []byte) (n int, err error) {
b.Mutex.Lock()
defer b.Mutex.Unlock()
return b.Buffer.Write(p)
}

View File

@ -81,35 +81,22 @@ type Context struct {
}
// BackgroundContext is a upgrade test execution context that will be passed
// down to each handler of BackgroundOperation. It contains a StopEvent channel
// which end user should use to obtain a testing.T for error reporting. Until
// StopEvent is sent user may use zap.SugaredLogger to log state of execution if
// necessary. The logs are stored in a threadSafeBuffer and flushed to the test
// output when the test fails.
// down to each handler of BackgroundOperation. It contains a stop event channel.
// Until stop event is sent user may use zap.SugaredLogger to log state of execution if
// necessary.
type BackgroundContext struct {
Log *zap.SugaredLogger
Stop <-chan StopEvent
logBuffer *threadSafeBuffer
}
// StopEvent represents an event that is to be received by background operation
// to indicate that is should stop it's operations and validate results using
// passed T. User should use Finished channel to signalize upgrade suite that
// all stop & verify operations are finished and it is safe to end tests.
type StopEvent struct {
T *testing.T
Finished chan<- struct{}
name string
logger *zap.SugaredLogger
T *testing.T
Log *zap.SugaredLogger
Stop <-chan struct{}
}
// WaitForStopEventConfiguration holds a values to be used be WaitForStopEvent
// function. OnStop will be called when StopEvent is sent. OnWait will be
// function. OnStop will be called when a stop event is sent. OnWait will be
// invoked in a loop while waiting, and each wait act is driven by WaitTime
// amount.
type WaitForStopEventConfiguration struct {
Name string
OnStop func(event StopEvent)
OnStop func()
OnWait func(bc BackgroundContext, self WaitForStopEventConfiguration)
WaitTime time.Duration
}
@ -117,24 +104,14 @@ type WaitForStopEventConfiguration struct {
// Configuration holds required and optional configuration to run upgrade tests.
type Configuration struct {
T *testing.T
// TODO(mgencur): Remove when dependent repositories migrate to LogConfig.
// Keep this for backwards compatibility.
Log *zap.Logger
LogConfig
}
func (c Configuration) logConfig() LogConfig {
if len(c.LogConfig.Config.OutputPaths) == 0 {
c.LogConfig.Config = zap.NewDevelopmentConfig()
}
return c.LogConfig
}
// LogConfig holds the logger configuration. It allows for passing just the
// logger configuration and also a custom function for building the resulting
// logger.
// logger configuration options.
type LogConfig struct {
// Config from which the zap.Logger be created.
// Deprecated: This config doesn't have effect. Use Options instead.
Config zap.Config
// Options holds options for the zap.Logger.
Options []zap.Option