Improve error handling

This commit is contained in:
Morten Torkildsen 2020-08-12 10:55:44 -07:00
parent 55b73eed69
commit 9135af4218
19 changed files with 392 additions and 132 deletions

View File

@ -34,7 +34,7 @@ func GetApplyRunner(f cmdutil.Factory, ioStreams genericclioptions.IOStreams) *A
RunE: r.RunE,
}
cmdutil.CheckErr(r.Applier.SetFlags(cmd))
r.Applier.SetFlags(cmd)
// The following flags are added, but hidden because other code
// depend on them when parsing flags. These flags are hidden and unused.
@ -86,13 +86,17 @@ type ApplyRunner struct {
}
func (r *ApplyRunner) RunE(cmd *cobra.Command, args []string) error {
cmdutil.CheckErr(setters2.CheckRequiredSettersSet())
if err := setters2.CheckRequiredSettersSet(); err != nil {
return err
}
prunePropPolicy, err := convertPropagationPolicy(r.prunePropagationPolicy)
if err != nil {
return err
}
cmdutil.CheckErr(r.Applier.Initialize(cmd))
if err := r.Applier.Initialize(cmd); err != nil {
return err
}
// Only emit status events if we are waiting for status.
//TODO: This is not the right way to do this. There are situations where
@ -149,8 +153,7 @@ func (r *ApplyRunner) RunE(cmd *cobra.Command, args []string) error {
// The printer will print updates from the channel. It will block
// until the channel is closed.
printer := printers.GetPrinter(r.output, r.ioStreams)
printer.Print(ch, common.DryRunNone)
return nil
return printer.Print(ch, common.DryRunNone)
}
// convertPropagationPolicy converts a propagationPolicy described as a

View File

@ -23,9 +23,12 @@ func NewCmdDestroy(f util.Factory, ioStreams genericclioptions.IOStreams) *cobra
Use: "destroy (DIRECTORY | STDIN)",
DisableFlagsInUseLine: true,
Short: i18n.T("Destroy all the resources related to configuration"),
Run: func(cmd *cobra.Command, args []string) {
RunE: func(cmd *cobra.Command, args []string) error {
paths := args
cmdutil.CheckErr(destroyer.Initialize(cmd, paths))
err := destroyer.Initialize(cmd, paths)
if err != nil {
return err
}
// Run the destroyer. It will return a channel where we can receive updates
// to keep track of progress and any issues.
@ -33,11 +36,12 @@ func NewCmdDestroy(f util.Factory, ioStreams genericclioptions.IOStreams) *cobra
// The printer will print updates from the channel. It will block
// until the channel is closed.
printer.Print(ch, destroyer.DryRunStrategy)
err = printer.Print(ch, destroyer.DryRunStrategy)
return err
},
}
cmdutil.CheckErr(destroyer.SetFlags(cmd))
destroyer.SetFlags(cmd)
// The following flags are added, but hidden because other code
// dependencies when parsing flags. These flags are hidden and unused.

View File

@ -6,7 +6,6 @@ package initcmd
import (
"github.com/spf13/cobra"
"k8s.io/cli-runtime/pkg/genericclioptions"
cmdutil "k8s.io/kubectl/pkg/cmd/util"
"k8s.io/kubectl/pkg/util/i18n"
"sigs.k8s.io/cli-utils/pkg/config"
)
@ -19,9 +18,12 @@ func NewCmdInit(ioStreams genericclioptions.IOStreams) *cobra.Command {
Use: "init DIRECTORY",
DisableFlagsInUseLine: true,
Short: i18n.T("Create a prune manifest ConfigMap as a inventory object"),
Run: func(cmd *cobra.Command, args []string) {
cmdutil.CheckErr(io.Complete(args))
cmdutil.CheckErr(io.Run())
RunE: func(cmd *cobra.Command, args []string) error {
err := io.Complete(args)
if err != nil {
return err
}
return io.Run()
},
}
cmd.Flags().StringVarP(&io.InventoryID, "inventory-id", "i", "", "Identifier for group of applied resources. Must be composed of valid label characters.")

View File

@ -18,6 +18,7 @@ import (
"sigs.k8s.io/cli-utils/cmd/initcmd"
"sigs.k8s.io/cli-utils/cmd/preview"
"sigs.k8s.io/cli-utils/cmd/status"
"sigs.k8s.io/cli-utils/pkg/errors"
"sigs.k8s.io/cli-utils/pkg/util/factory"
// This is here rather than in the libraries because of
@ -29,6 +30,10 @@ var cmd = &cobra.Command{
Use: "kapply",
Short: "Perform cluster operations using declarative configuration",
Long: "Perform cluster operations using declarative configuration",
// We silence error reporting from Cobra here since we want to improve
// the error messages coming from the commands.
SilenceErrors: true,
SilenceUsage: true,
}
func main() {
@ -69,7 +74,7 @@ func main() {
defer logs.FlushLogs()
if err := cmd.Execute(); err != nil {
os.Exit(1)
errors.CheckErr(cmd.ErrOrStderr(), err, "kapply")
}
}

View File

@ -38,10 +38,16 @@ func NewCmdPreview(f cmdutil.Factory, ioStreams genericclioptions.IOStreams) *co
DisableFlagsInUseLine: true,
Short: i18n.T("Preview the apply of a configuration"),
Args: cobra.MaximumNArgs(1),
Run: func(cmd *cobra.Command, args []string) {
cmdutil.CheckErr(setters2.CheckRequiredSettersSet())
RunE: func(cmd *cobra.Command, args []string) error {
err := setters2.CheckRequiredSettersSet()
if err != nil {
return err
}
var ch <-chan event.Event
cmdutil.CheckErr(destroyer.Initialize(cmd, args))
err = destroyer.Initialize(cmd, args)
if err != nil {
return err
}
drs := common.DryRunClient
if serverDryRun {
@ -55,13 +61,18 @@ func NewCmdPreview(f cmdutil.Factory, ioStreams genericclioptions.IOStreams) *co
// if destroy flag is set in preview, transmit it to destroyer DryRunStrategy flag
// and pivot execution to destroy with dry-run
if !destroyer.DryRunStrategy.ClientOrServerDryRun() {
cmdutil.CheckErr(applier.Initialize(cmd))
err = applier.Initialize(cmd)
if err != nil {
return err
}
// Create a context
ctx := context.Background()
_, err := common.DemandOneDirectory(args)
cmdutil.CheckErr(err)
if err != nil {
return err
}
var reader manifestreader.ManifestReader
readerOptions := manifestreader.ReaderOptions{
@ -81,7 +92,9 @@ func NewCmdPreview(f cmdutil.Factory, ioStreams genericclioptions.IOStreams) *co
}
}
infos, err := reader.Read()
cmdutil.CheckErr(err)
if err != nil {
return err
}
// Run the applier. It will return a channel where we can receive updates
// to keep track of progress and any issues.
@ -96,13 +109,13 @@ func NewCmdPreview(f cmdutil.Factory, ioStreams genericclioptions.IOStreams) *co
// The printer will print updates from the channel. It will block
// until the channel is closed.
printer.Print(ch, drs)
return printer.Print(ch, drs)
},
}
cmd.Flags().BoolVar(&noPrune, "no-prune", noPrune, "If true, do not prune previously applied objects.")
cmd.Flags().BoolVar(&serverDryRun, "server-side", serverDryRun, "If true, preview runs in the server instead of the client.")
cmdutil.CheckErr(applier.SetFlags(cmd))
applier.SetFlags(cmd)
// The following flags are added, but hidden because other code
// dependend on them when parsing flags. These flags are hidden and unused.

View File

@ -9,5 +9,5 @@ import (
)
type Printer interface {
Print(ch <-chan event.Event, previewStrategy common.DryRunStrategy)
Print(ch <-chan event.Event, previewStrategy common.DryRunStrategy) error
}

View File

@ -4,15 +4,12 @@
package table
import (
"bytes"
"fmt"
"sort"
"sync"
"k8s.io/apimachinery/pkg/api/meta"
"k8s.io/apimachinery/pkg/runtime"
"sigs.k8s.io/cli-utils/pkg/apply/event"
"sigs.k8s.io/cli-utils/pkg/apply/taskrunner"
pe "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event"
"sigs.k8s.io/cli-utils/pkg/kstatus/status"
"sigs.k8s.io/cli-utils/pkg/object"
@ -148,31 +145,39 @@ func (r *SubResourceInfo) SubResources() []table.Resource {
// The function returns a channel. When this channel is closed, the
// goroutine has processed all events in the eventChannel and
// exited.
func (r *ResourceStateCollector) Listen(eventChannel <-chan event.Event) <-chan struct{} {
completed := make(chan struct{})
func (r *ResourceStateCollector) Listen(eventChannel <-chan event.Event) <-chan listenerResult {
completed := make(chan listenerResult)
go func() {
defer close(completed)
for e := range eventChannel {
r.processEvent(e)
for ev := range eventChannel {
if err := r.processEvent(ev); err != nil {
completed <- listenerResult{err: err}
return
}
}
}()
return completed
}
type listenerResult struct {
err error
}
// processEvent processes an event and updates the state.
func (r *ResourceStateCollector) processEvent(e event.Event) {
func (r *ResourceStateCollector) processEvent(ev event.Event) error {
r.mux.Lock()
defer r.mux.Unlock()
switch e.Type {
switch ev.Type {
case event.StatusType:
r.processStatusEvent(e.StatusEvent)
r.processStatusEvent(ev.StatusEvent)
case event.ApplyType:
r.processApplyEvent(e.ApplyEvent)
r.processApplyEvent(ev.ApplyEvent)
case event.PruneType:
r.processPruneEvent(e.PruneEvent)
r.processPruneEvent(ev.PruneEvent)
case event.ErrorType:
r.processErrorEvent(e.ErrorEvent.Err)
return ev.ErrorEvent.Err
}
return nil
}
// processStatusEvent handles events pertaining to a status
@ -215,40 +220,9 @@ func (r *ResourceStateCollector) processPruneEvent(e event.PruneEvent) {
// processErrorEvent handles events for errors.
func (r *ResourceStateCollector) processErrorEvent(err error) {
if timeoutErr, ok := taskrunner.IsTimeoutError(err); ok {
r.err = r.handleTimeoutError(timeoutErr)
return
}
r.err = err
}
// handleTimeoutError transforms a TimeoutError into a new error which includes
// information about which resources failed to reach the desired status.
func (r *ResourceStateCollector) handleTimeoutError(timeoutErr taskrunner.TimeoutError) error {
var errInfo ResourceInfos
for _, id := range timeoutErr.Identifiers {
ri, ok := r.resourceInfos[id]
if !ok {
continue
}
if timeoutErr.Condition.Meets(ri.resourceStatus.Status) {
continue
}
errInfo = append(errInfo, ri)
}
sort.Sort(errInfo)
var b bytes.Buffer
_, _ = fmt.Fprint(&b, timeoutErr.Error()+"\n")
for i, ri := range errInfo {
_, _ = fmt.Fprintf(&b, "%s/%s %s %s", ri.identifier.GroupKind.Kind,
ri.identifier.Name, ri.resourceStatus.Status, ri.resourceStatus.Message)
if i != len(errInfo)-1 {
_, _ = fmt.Fprint(&b, "\n")
}
}
return fmt.Errorf(b.String())
}
// toIdentifier extracts the identifying information from an
// object.
func toIdentifier(o runtime.Object) object.ObjMetadata {

View File

@ -18,7 +18,7 @@ type Printer struct {
IOStreams genericclioptions.IOStreams
}
func (t *Printer) Print(ch <-chan event.Event, _ common.DryRunStrategy) {
func (t *Printer) Print(ch <-chan event.Event, _ common.DryRunStrategy) error {
// Wait for the init event that will give us the set of
// resources.
var initEvent event.InitEvent
@ -30,8 +30,7 @@ func (t *Printer) Print(ch <-chan event.Event, _ common.DryRunStrategy) {
// If we get an error event, we just print it and
// exit. The error event signals a fatal error.
if e.Type == event.ErrorType {
_, _ = fmt.Fprintf(t.IOStreams.Out, "Fatal error: %v\n", e.ErrorEvent.Err)
return
return e.ErrorEvent.Err
}
}
// Create a new collector and initialize it with the resources
@ -49,7 +48,10 @@ func (t *Printer) Print(ch <-chan event.Event, _ common.DryRunStrategy) {
// Block until all the collector has shut down. This means the
// eventChannel has been closed and all events have been processed.
<-done
var err error
for msg := range done {
err = msg.err
}
// Close the stop channel to notify the print goroutine that it should
// shut down.
@ -59,6 +61,7 @@ func (t *Printer) Print(ch <-chan event.Event, _ common.DryRunStrategy) {
// the printer has updated the UI with the latest state and
// exited from the goroutine.
<-printCompleted
return err
}
// columns defines the columns we want to print
@ -135,6 +138,7 @@ func (t *Printer) runPrintLoop(coll *ResourceStateCollector, stop chan struct{})
ticker.Stop()
latestState := coll.LatestState()
linesPrinted = baseTablePrinter.PrintTable(latestState, linesPrinted)
_, _ = fmt.Fprint(t.IOStreams.Out, "\n")
return
case <-ticker.C:
latestState := coll.LatestState()

View File

@ -113,12 +113,12 @@ func (a *Applier) Initialize(cmd *cobra.Command) error {
// SetFlags configures the command line flags needed for apply and
// status. This is a temporary solution as we should separate the configuration
// of cobra flags from the Applier.
func (a *Applier) SetFlags(cmd *cobra.Command) error {
func (a *Applier) SetFlags(cmd *cobra.Command) {
a.ApplyOptions.DeleteFlags.AddFlags(cmd)
for _, flag := range []string{"kustomize", "filename", "recursive"} {
err := cmd.Flags().MarkHidden(flag)
if err != nil {
return err
panic(err)
}
}
a.ApplyOptions.RecordFlags.AddFlags(cmd)
@ -129,7 +129,6 @@ func (a *Applier) SetFlags(cmd *cobra.Command) error {
_ = cmd.Flags().MarkHidden("timeout")
_ = cmd.Flags().MarkHidden("wait")
a.ApplyOptions.Overwrite = true
return nil
}
// infoHelperFactory returns a new instance of the InfoHelper.

View File

@ -219,7 +219,7 @@ func TestApplier(t *testing.T) {
applier := NewApplier(tf, ioStreams)
cmd := &cobra.Command{}
_ = applier.SetFlags(cmd)
applier.SetFlags(cmd)
var notUsedFlag bool
// This flag needs to be set as there is a dependency on it.
cmd.Flags().BoolVar(&notUsedFlag, "dry-run", notUsedFlag, "")

View File

@ -5,7 +5,6 @@ package apply
import (
"fmt"
"os"
"strings"
"k8s.io/apimachinery/pkg/api/meta"
@ -13,17 +12,11 @@ import (
"k8s.io/apimachinery/pkg/runtime/schema"
"k8s.io/cli-runtime/pkg/genericclioptions"
"sigs.k8s.io/cli-utils/pkg/apply/event"
"sigs.k8s.io/cli-utils/pkg/apply/taskrunner"
"sigs.k8s.io/cli-utils/pkg/common"
pollevent "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event"
"sigs.k8s.io/cli-utils/pkg/object"
)
const (
defaultExitErrorCode int = 1
timeoutExitErrorCode int = 3
)
// BasicPrinter is a simple implementation that just prints the events
// from the channel in the default format for kubectl.
// We need to support different printers for different output formats.
@ -96,7 +89,7 @@ func (sc *statusCollector) updateStatus(id object.ObjMetadata, se pollevent.Even
// format on StdOut. As we support other printer implementations
// this should probably be an interface.
// This function will block until the channel is closed.
func (b *BasicPrinter) Print(ch <-chan event.Event, previewStrategy common.DryRunStrategy) {
func (b *BasicPrinter) Print(ch <-chan event.Event, previewStrategy common.DryRunStrategy) error {
printFunc := b.getPrintFunc(previewStrategy)
applyStats := &applyStats{}
statusCollector := &statusCollector{
@ -108,7 +101,7 @@ func (b *BasicPrinter) Print(ch <-chan event.Event, previewStrategy common.DryRu
for e := range ch {
switch e.Type {
case event.ErrorType:
b.processErrorEvent(e.ErrorEvent, statusCollector, printFunc)
return e.ErrorEvent.Err
case event.ApplyType:
b.processApplyEvent(e.ApplyEvent, applyStats, statusCollector, printFunc)
case event.StatusType:
@ -119,27 +112,7 @@ func (b *BasicPrinter) Print(ch <-chan event.Event, previewStrategy common.DryRu
b.processDeleteEvent(e.DeleteEvent, deleteStats, printFunc)
}
}
}
func (b *BasicPrinter) processErrorEvent(ee event.ErrorEvent, c *statusCollector,
p printFunc) {
p("\nFatal error: %s", ee.Err.Error())
if timeoutErr, ok := taskrunner.IsTimeoutError(ee.Err); ok {
for _, id := range timeoutErr.Identifiers {
ls, found := c.latestStatus[id]
if !found {
continue
}
if timeoutErr.Condition.Meets(ls.Resource.Status) {
continue
}
p("%s/%s %s %s", id.GroupKind.Kind,
id.Name, ls.Resource.Status, ls.Resource.Message)
}
os.Exit(timeoutExitErrorCode)
}
os.Exit(defaultExitErrorCode)
return nil
}
func (b *BasicPrinter) processApplyEvent(ae event.ApplyEvent, as *applyStats,

View File

@ -155,12 +155,12 @@ func (d *Destroyer) Run() <-chan event.Event {
// SetFlags configures the command line flags needed for destroy
// This is a temporary solution as we should separate the configuration
// of cobra flags from the Destroyer.
func (d *Destroyer) SetFlags(cmd *cobra.Command) error {
func (d *Destroyer) SetFlags(cmd *cobra.Command) {
d.ApplyOptions.DeleteFlags.AddFlags(cmd)
for _, flag := range []string{"kustomize", "filename", "recursive"} {
err := cmd.Flags().MarkHidden(flag)
if err != nil {
return err
panic(err)
}
}
d.ApplyOptions.RecordFlags.AddFlags(cmd)
@ -171,7 +171,6 @@ func (d *Destroyer) SetFlags(cmd *cobra.Command) error {
_ = cmd.Flags().MarkHidden("timeout")
_ = cmd.Flags().MarkHidden("wait")
d.ApplyOptions.Overwrite = true
return nil
}
// runPruneEventTransformer creates a channel for events and

View File

@ -36,6 +36,7 @@ type resourceStatusCollector struct {
type resourceStatus struct {
Identifier object.ObjMetadata
CurrentStatus status.Status
Message string
Generation int64
}
@ -44,6 +45,7 @@ type resourceStatus struct {
func (a *resourceStatusCollector) resourceStatus(r *event.ResourceStatus) {
if ri, found := a.resourceMap[r.Identifier]; found {
ri.CurrentStatus = r.Status
ri.Message = r.Message
ri.Generation = getGeneration(r)
a.resourceMap[r.Identifier] = ri
}

View File

@ -12,6 +12,7 @@ import (
"sigs.k8s.io/cli-utils/pkg/apply/poller"
"sigs.k8s.io/cli-utils/pkg/kstatus/polling"
pollevent "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event"
"sigs.k8s.io/cli-utils/pkg/kstatus/status"
"sigs.k8s.io/cli-utils/pkg/object"
)
@ -205,6 +206,7 @@ func (b *baseRunner) run(ctx context.Context, taskQueue chan Task,
case msg := <-taskContext.TaskChannel():
currentTask.ClearTimeout()
if msg.Err != nil {
b.amendTimeoutError(msg.Err)
return msg.Err
}
if abort {
@ -227,6 +229,27 @@ func (b *baseRunner) run(ctx context.Context, taskQueue chan Task,
}
}
func (b *baseRunner) amendTimeoutError(err error) {
if timeoutErr, ok := err.(*TimeoutError); ok {
var timedOutResources []TimedOutResource
for _, id := range timeoutErr.Identifiers {
ls, found := b.collector.resourceMap[id]
if !found {
continue
}
if timeoutErr.Condition.Meets(ls.CurrentStatus) {
continue
}
timedOutResources = append(timedOutResources, TimedOutResource{
Identifier: id,
Status: ls.CurrentStatus,
Message: ls.Message,
})
}
timeoutErr.TimedOutResources = timedOutResources
}
}
// completeIfWaitTask checks if the current task is a wait task. If so,
// we invoke the complete function to complete it.
func completeIfWaitTask(currentTask Task, taskContext *TaskContext) {
@ -287,6 +310,16 @@ type TimeoutError struct {
// Condition defines the criteria for which the task was waiting.
Condition Condition
TimedOutResources []TimedOutResource
}
type TimedOutResource struct {
Identifier object.ObjMetadata
Status status.Status
Message string
}
func (te TimeoutError) Error() string {
@ -296,9 +329,9 @@ func (te TimeoutError) Error() string {
// IsTimeoutError checks whether a given error is
// a TimeoutError.
func IsTimeoutError(err error) (TimeoutError, bool) {
if e, ok := err.(TimeoutError); ok {
func IsTimeoutError(err error) (*TimeoutError, bool) {
if e, ok := err.(*TimeoutError); ok {
return e, true
}
return TimeoutError{}, false
return &TimeoutError{}, false
}

View File

@ -10,6 +10,7 @@ import (
"testing"
"time"
"github.com/stretchr/testify/assert"
"k8s.io/apimachinery/pkg/runtime/schema"
"sigs.k8s.io/cli-utils/pkg/apply/event"
pollevent "sigs.k8s.io/cli-utils/pkg/kstatus/polling/event"
@ -43,6 +44,8 @@ func TestBaseRunner(t *testing.T) {
statusEventsDelay time.Duration
statusEvents []pollevent.Event
expectedEventTypes []event.Type
expectedError error
expectedTimedOutResources []TimedOutResource
}{
"wait task runs until condition is met": {
identifiers: []object.ObjMetadata{depID, cmID},
@ -86,6 +89,33 @@ func TestBaseRunner(t *testing.T) {
event.PruneType,
},
},
"wait task times out eventually": {
identifiers: []object.ObjMetadata{depID, cmID},
tasks: []Task{
NewWaitTask([]object.ObjMetadata{depID, cmID}, AllCurrent,
2*time.Second),
},
statusEventsDelay: time.Second,
statusEvents: []pollevent.Event{
{
EventType: pollevent.ResourceUpdateEvent,
Resource: &pollevent.ResourceStatus{
Identifier: cmID,
Status: status.CurrentStatus,
},
},
},
expectedEventTypes: []event.Type{
event.StatusType,
},
expectedError: &TimeoutError{},
expectedTimedOutResources: []TimedOutResource{
{
Identifier: depID,
Status: status.UnknownStatus,
},
},
},
"tasks run in order": {
identifiers: []object.ObjMetadata{},
tasks: []Task{
@ -165,7 +195,14 @@ func TestBaseRunner(t *testing.T) {
close(eventChannel)
wg.Wait()
if err != nil {
if tc.expectedError != nil {
assert.IsType(t, tc.expectedError, err)
if timeoutError, ok := err.(*TimeoutError); ok {
assert.ElementsMatch(t, tc.expectedTimedOutResources,
timeoutError.TimedOutResources)
}
return
} else if err != nil {
t.Errorf("expected no error, but got %v", err)
}

View File

@ -83,7 +83,7 @@ func (w *WaitTask) setTimer(taskContext *TaskContext) {
// to the token first.
case <-w.token:
taskContext.TaskChannel() <- TaskResult{
Err: TimeoutError{
Err: &TimeoutError{
Identifiers: w.Identifiers,
Timeout: w.Timeout,
Condition: w.Condition,

126
pkg/errors/errors.go Normal file
View File

@ -0,0 +1,126 @@
// Copyright 2020 The Kubernetes Authors.
// SPDX-License-Identifier: Apache-2.0
package errors
import (
"bytes"
"fmt"
"io"
"os"
"reflect"
"strings"
"text/template"
cmdutil "k8s.io/kubectl/pkg/cmd/util"
"sigs.k8s.io/cli-utils/pkg/apply/taskrunner"
"sigs.k8s.io/cli-utils/pkg/inventory"
)
const (
DefaultErrorExitCode = 1
TimeoutErrorExitCode = 3
)
var errorMsgForType map[reflect.Type]string
var statusCodeForType map[reflect.Type]int
//nolint:gochecknoinits
func init() {
errorMsgForType = make(map[reflect.Type]string)
errorMsgForType[reflect.TypeOf(inventory.NoInventoryObjError{})] = `
Package uninitialized. Please run "{{.cmdNameBase}} init" command.
The package needs to be initialized to generate the template
which will store state for resource sets. This state is
necessary to perform functionality such as deleting an entire
package or automatically deleting omitted resources (pruning).
`
errorMsgForType[reflect.TypeOf(inventory.MultipleInventoryObjError{})] = `
Package has multiple inventory object templates.
The package should have one and only one inventory object template.
`
//nolint:lll
errorMsgForType[reflect.TypeOf(taskrunner.TimeoutError{})] = `
Timeout after {{printf "%.0f" .err.Timeout.Seconds}} seconds waiting for {{printf "%d" (len .err.TimedOutResources)}} out of {{printf "%d" (len .err.Identifiers)}} resources to reach condition {{ .err.Condition}}:
{{- range .err.TimedOutResources}}
{{printf "%s/%s %s %s" .Identifier.GroupKind.Kind .Identifier.Name .Status .Message }}
{{- end}}
`
statusCodeForType = make(map[reflect.Type]int)
statusCodeForType[reflect.TypeOf(taskrunner.TimeoutError{})] = TimeoutErrorExitCode
}
// CheckErr looks up the appropriate error message and exit status for known
// errors. It will print the information to the provided io.Writer. If we
// don't know the error, it delegates to the error handling in cmdutil.
func CheckErr(w io.Writer, err error, cmdNameBase string) {
errText, found := textForError(err, cmdNameBase)
if found {
exitStatus := findErrExitCode(err)
if len(errText) > 0 {
if !strings.HasSuffix(errText, "\n") {
errText += "\n"
}
fmt.Fprint(w, errText)
}
os.Exit(exitStatus)
}
cmdutil.CheckErr(err)
}
// textForError looks up the error message based on the type of the error.
func textForError(baseErr error, cmdNameBase string) (string, bool) {
errType := findErrType(baseErr)
tmplText, found := errorMsgForType[errType]
if !found {
return "", false
}
tmpl, err := template.New("errMsg").Parse(tmplText)
if err != nil {
// Just return false here instead of the error. It will just
// mean a less informative error message and we rather show the
// original error.
return "", false
}
var b bytes.Buffer
err = tmpl.Execute(&b, map[string]interface{}{
"cmdNameBase": cmdNameBase,
"err": baseErr,
})
if err != nil {
return "", false
}
return strings.TrimSpace(b.String()), true
}
// findErrType finds the type of the error. It returns the real type in the
// event the error is actually a pointer to a type.
func findErrType(err error) reflect.Type {
switch reflect.ValueOf(err).Kind() {
case reflect.Ptr:
// If the value of the interface is a pointer, we use the type
// of the real value.
return reflect.ValueOf(err).Elem().Type()
case reflect.Struct:
return reflect.TypeOf(err)
default:
panic("unexpected error type")
}
}
// findErrExitCode looks up if there is a defined error code for the provided
// error type.
func findErrExitCode(err error) int {
errType := findErrType(err)
if exitStatus, found := statusCodeForType[errType]; found {
return exitStatus
}
return DefaultErrorExitCode
}

100
pkg/errors/errors_test.go Normal file
View File

@ -0,0 +1,100 @@
// Copyright 2020 The Kubernetes Authors.
// SPDX-License-Identifier: Apache-2.0
package errors
import (
"fmt"
"strings"
"testing"
"time"
"github.com/stretchr/testify/assert"
"k8s.io/apimachinery/pkg/runtime/schema"
"sigs.k8s.io/cli-utils/pkg/apply/taskrunner"
"sigs.k8s.io/cli-utils/pkg/inventory"
"sigs.k8s.io/cli-utils/pkg/kstatus/status"
"sigs.k8s.io/cli-utils/pkg/object"
)
func TestTextForError(t *testing.T) {
testCases := map[string]struct {
err error
cmdNameBase string
expectFound bool
expectedErrText string
}{
"kapply command base name": {
err: inventory.NoInventoryObjError{},
cmdNameBase: "kapply",
expectFound: true,
expectedErrText: "Please run \"kapply init\" command.",
},
"different command base name": {
err: inventory.NoInventoryObjError{},
cmdNameBase: "mycommand",
expectFound: true,
expectedErrText: "Please run \"mycommand init\" command.",
},
"known error without directives in the template": {
err: inventory.MultipleInventoryObjError{},
cmdNameBase: "kapply",
expectFound: true,
expectedErrText: "Package has multiple inventory object templates.",
},
"unknown error": {
err: fmt.Errorf("this is a test"),
cmdNameBase: "kapply",
expectFound: false,
},
"timeout error": {
err: &taskrunner.TimeoutError{
Timeout: 2 * time.Second,
Identifiers: []object.ObjMetadata{
{
GroupKind: schema.GroupKind{
Kind: "Deployment",
Group: "apps",
},
Name: "foo",
},
},
Condition: taskrunner.AllCurrent,
TimedOutResources: []taskrunner.TimedOutResource{
{
Identifier: object.ObjMetadata{
GroupKind: schema.GroupKind{
Kind: "Deployment",
Group: "apps",
},
Name: "foo",
},
Status: status.InProgressStatus,
},
},
},
cmdNameBase: "kapply",
expectFound: true,
expectedErrText: `
Timeout after 2 seconds waiting for 1 out of 1 resources to reach condition AllCurrent:
Deployment/foo InProgress
`,
},
}
for tn, tc := range testCases {
t.Run(tn, func(t *testing.T) {
errText, found := textForError(tc.err, tc.cmdNameBase)
if !tc.expectFound {
assert.False(t, found)
return
}
assert.True(t, found)
fmt.Printf("%s\n", errText)
assert.Contains(t, errText, strings.TrimSpace(tc.expectedErrText))
})
}
}

View File

@ -91,12 +91,6 @@ func (t *BaseTablePrinter) PrintTable(rs ResourceStates,
linePrintCount += t.printSubTable(resource.SubResources(), "")
}
// If we have encountered an error, print that below the table.
if rs.Error() != nil {
lineCount := t.printError(rs.Error())
linePrintCount += lineCount
}
return linePrintCount
}
@ -143,14 +137,6 @@ func (t *BaseTablePrinter) printSubTable(resources []Resource,
return linePrintCount
}
//TODO: This should be able to return the correct number of printed lines,
// even if the error message has line breaks or is so long that it needs to
// be wrapped over multiple lines.
func (t *BaseTablePrinter) printError(err error) int {
t.printOrDie("\nFatal error: %v\n", err)
return 2 // This is the number of lines printed.
}
func (t *BaseTablePrinter) printOrDie(format string, a ...interface{}) {
_, err := fmt.Fprintf(t.IOStreams.Out, format, a...)
if err != nil {