mirror of
https://github.com/containers/podman.git
synced 2025-05-17 23:26:08 +08:00
rework event code to improve API errors
One of the problems with the Events() API was that you had to call it in a new goroutine. This meant the the error returned by it had to be read back via a second channel. This cuased other bugs in the past but here the biggest problem is that basic errors such as invalid since/until options were not directly returned to the caller. It meant in the API we were not able to write http code 200 quickly because we always waited for the first event or error from the channels. This in turn made some clients not happy as they assume the server hangs on time out if no such events are generated. To fix this we resturcture the entire event flow. First we spawn the goroutine inside the eventer Read() function so not all the callers have to. Then we can return the basic error quickly without the goroutine. The caller then checks the error like any normal function and the API can use this one to decide which status code to return. Second we now return errors/event in one channel then the callers can decide to ignore or log them which makes it a bit more clear. Fixes c46884aa93 ("podman events: check for an error after we finish reading events") Fixes #23712 Signed-off-by: Paul Holzinger <pholzing@redhat.com>
This commit is contained in:
@ -12,6 +12,7 @@ import (
|
||||
"github.com/containers/podman/v5/cmd/podman/validate"
|
||||
"github.com/containers/podman/v5/libpod/events"
|
||||
"github.com/containers/podman/v5/pkg/domain/entities"
|
||||
"github.com/sirupsen/logrus"
|
||||
"github.com/spf13/cobra"
|
||||
)
|
||||
|
||||
@ -139,9 +140,8 @@ func eventsCmd(cmd *cobra.Command, _ []string) error {
|
||||
if len(eventOptions.Since) > 0 || len(eventOptions.Until) > 0 {
|
||||
eventOptions.FromStart = true
|
||||
}
|
||||
eventChannel := make(chan *events.Event, 1)
|
||||
eventChannel := make(chan events.ReadResult, 1)
|
||||
eventOptions.EventChan = eventChannel
|
||||
errChannel := make(chan error)
|
||||
|
||||
var (
|
||||
rpt *report.Formatter
|
||||
@ -161,40 +161,31 @@ func eventsCmd(cmd *cobra.Command, _ []string) error {
|
||||
}
|
||||
}
|
||||
|
||||
go func() {
|
||||
errChannel <- registry.ContainerEngine().Events(context.Background(), eventOptions)
|
||||
close(errChannel)
|
||||
}()
|
||||
err := registry.ContainerEngine().Events(context.Background(), eventOptions)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
for {
|
||||
select {
|
||||
case event, ok := <-eventChannel:
|
||||
if !ok {
|
||||
// channel was closed we can exit
|
||||
// read the error channel blocking to make sure we are not missing any errors (#23165)
|
||||
return <-errChannel
|
||||
for evt := range eventChannel {
|
||||
if evt.Error != nil {
|
||||
logrus.Errorf("Failed to read event: %v", evt.Error)
|
||||
continue
|
||||
}
|
||||
switch {
|
||||
case doJSON:
|
||||
e := newEventFromLibpodEvent(event)
|
||||
e := newEventFromLibpodEvent(evt.Event)
|
||||
jsonStr, err := e.ToJSONString()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
fmt.Println(jsonStr)
|
||||
case cmd.Flags().Changed("format"):
|
||||
if err := rpt.Execute(newEventFromLibpodEvent(event)); err != nil {
|
||||
if err := rpt.Execute(newEventFromLibpodEvent(evt.Event)); err != nil {
|
||||
return err
|
||||
}
|
||||
default:
|
||||
fmt.Println(event.ToHumanReadable(!noTrunc))
|
||||
}
|
||||
case err := <-errChannel:
|
||||
// only exit in case of an error,
|
||||
// otherwise keep reading events until the event channel is closed
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
fmt.Println(evt.Event.ToHumanReadable(!noTrunc))
|
||||
}
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
@ -6,7 +6,6 @@ import (
|
||||
"context"
|
||||
"fmt"
|
||||
"path/filepath"
|
||||
"sync"
|
||||
|
||||
"github.com/containers/podman/v5/libpod/define"
|
||||
"github.com/containers/podman/v5/libpod/events"
|
||||
@ -187,7 +186,7 @@ func (r *Runtime) Events(ctx context.Context, options events.ReadOptions) error
|
||||
|
||||
// GetEvents reads the event log and returns events based on input filters
|
||||
func (r *Runtime) GetEvents(ctx context.Context, filters []string) ([]*events.Event, error) {
|
||||
eventChannel := make(chan *events.Event)
|
||||
eventChannel := make(chan events.ReadResult)
|
||||
options := events.ReadOptions{
|
||||
EventChannel: eventChannel,
|
||||
Filters: filters,
|
||||
@ -195,45 +194,21 @@ func (r *Runtime) GetEvents(ctx context.Context, filters []string) ([]*events.Ev
|
||||
Stream: false,
|
||||
}
|
||||
|
||||
logEvents := make([]*events.Event, 0, len(eventChannel))
|
||||
readLock := sync.Mutex{}
|
||||
readLock.Lock()
|
||||
go func() {
|
||||
for e := range eventChannel {
|
||||
logEvents = append(logEvents, e)
|
||||
}
|
||||
readLock.Unlock()
|
||||
}()
|
||||
|
||||
readErr := r.eventer.Read(ctx, options)
|
||||
readLock.Lock() // Wait for the events to be consumed.
|
||||
return logEvents, readErr
|
||||
}
|
||||
|
||||
// GetLastContainerEvent takes a container name or ID and an event status and returns
|
||||
// the last occurrence of the container event
|
||||
func (r *Runtime) GetLastContainerEvent(ctx context.Context, nameOrID string, containerEvent events.Status) (*events.Event, error) {
|
||||
// FIXME: events should be read in reverse order!
|
||||
// https://github.com/containers/podman/issues/14579
|
||||
|
||||
// check to make sure the event.Status is valid
|
||||
if _, err := events.StringToStatus(containerEvent.String()); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
filters := []string{
|
||||
fmt.Sprintf("container=%s", nameOrID),
|
||||
fmt.Sprintf("event=%s", containerEvent),
|
||||
"type=container",
|
||||
}
|
||||
containerEvents, err := r.GetEvents(ctx, filters)
|
||||
err := r.eventer.Read(ctx, options)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
if len(containerEvents) < 1 {
|
||||
return nil, fmt.Errorf("%s not found: %w", containerEvent.String(), events.ErrEventNotFound)
|
||||
|
||||
logEvents := make([]*events.Event, 0, len(eventChannel))
|
||||
for evt := range eventChannel {
|
||||
// we ignore any error here, this is only used on the backup
|
||||
// GetExecDiedEvent() died path as best effort anyway
|
||||
if evt.Error == nil {
|
||||
logEvents = append(logEvents, evt.Event)
|
||||
}
|
||||
// return the last element in the slice
|
||||
return containerEvents[len(containerEvents)-1], nil
|
||||
}
|
||||
|
||||
return logEvents, nil
|
||||
}
|
||||
|
||||
// GetExecDiedEvent takes a container name or ID, exec session ID, and returns
|
||||
|
@ -85,10 +85,15 @@ type Eventer interface {
|
||||
String() string
|
||||
}
|
||||
|
||||
type ReadResult struct {
|
||||
Event *Event
|
||||
Error error
|
||||
}
|
||||
|
||||
// ReadOptions describe the attributes needed to read event logs
|
||||
type ReadOptions struct {
|
||||
// EventChannel is the comm path back to user
|
||||
EventChannel chan *Event
|
||||
EventChannel chan ReadResult
|
||||
// Filters are key/value pairs that describe to limit output
|
||||
Filters []string
|
||||
// FromStart means you start reading from the start of the logs
|
||||
|
@ -7,6 +7,7 @@ import (
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"runtime"
|
||||
"strconv"
|
||||
"time"
|
||||
|
||||
@ -97,8 +98,9 @@ func (e EventJournalD) Write(ee Event) error {
|
||||
}
|
||||
|
||||
// Read reads events from the journal and sends qualified events to the event channel
|
||||
func (e EventJournalD) Read(ctx context.Context, options ReadOptions) error {
|
||||
defer close(options.EventChannel)
|
||||
func (e EventJournalD) Read(ctx context.Context, options ReadOptions) (retErr error) {
|
||||
runtime.LockOSThread()
|
||||
defer runtime.UnlockOSThread()
|
||||
filterMap, err := generateEventFilters(options.Filters, options.Since, options.Until)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to parse event filters: %w", err)
|
||||
@ -117,13 +119,15 @@ func (e EventJournalD) Read(ctx context.Context, options ReadOptions) error {
|
||||
return err
|
||||
}
|
||||
defer func() {
|
||||
if retErr != nil {
|
||||
if err := j.Close(); err != nil {
|
||||
logrus.Errorf("Unable to close journal :%v", err)
|
||||
}
|
||||
}
|
||||
}()
|
||||
err = j.SetDataThreshold(0)
|
||||
if err != nil {
|
||||
logrus.Warnf("cannot set data threshold: %v", err)
|
||||
return fmt.Errorf("cannot set data threshold for journal: %v", err)
|
||||
}
|
||||
// match only podman journal entries
|
||||
podmanJournal := sdjournal.Match{Field: "SYSLOG_IDENTIFIER", Value: "podman"}
|
||||
@ -158,14 +162,22 @@ func (e EventJournalD) Read(ctx context.Context, options ReadOptions) error {
|
||||
}
|
||||
}
|
||||
|
||||
go func() {
|
||||
defer close(options.EventChannel)
|
||||
defer func() {
|
||||
if err := j.Close(); err != nil {
|
||||
logrus.Errorf("Unable to close journal :%v", err)
|
||||
}
|
||||
}()
|
||||
for {
|
||||
entry, err := GetNextEntry(ctx, j, options.Stream, untilTime)
|
||||
if err != nil {
|
||||
return err
|
||||
options.EventChannel <- ReadResult{Error: err}
|
||||
break
|
||||
}
|
||||
// no entry == we hit the end
|
||||
if entry == nil {
|
||||
return nil
|
||||
break
|
||||
}
|
||||
|
||||
newEvent, err := newEventFromJournalEntry(entry)
|
||||
@ -174,14 +186,16 @@ func (e EventJournalD) Read(ctx context.Context, options ReadOptions) error {
|
||||
// Don't fail hard - that would make events unusable.
|
||||
// Instead, log and continue.
|
||||
if !errors.Is(err, ErrEventTypeBlank) {
|
||||
logrus.Errorf("Unable to decode event: %v", err)
|
||||
options.EventChannel <- ReadResult{Error: fmt.Errorf("unable to decode event: %v", err)}
|
||||
}
|
||||
continue
|
||||
}
|
||||
if applyFilters(newEvent, filterMap) {
|
||||
options.EventChannel <- newEvent
|
||||
options.EventChannel <- ReadResult{Event: newEvent}
|
||||
}
|
||||
}
|
||||
}()
|
||||
return nil
|
||||
}
|
||||
|
||||
func newEventFromJournalEntry(entry *sdjournal.JournalEntry) (*Event, error) {
|
||||
|
@ -108,7 +108,6 @@ func (e EventLogFile) readRotateEvent(event *Event) (begin bool, end bool, err e
|
||||
|
||||
// Reads from the log file
|
||||
func (e EventLogFile) Read(ctx context.Context, options ReadOptions) error {
|
||||
defer close(options.EventChannel)
|
||||
filterMap, err := generateEventFilters(options.Filters, options.Since, options.Until)
|
||||
if err != nil {
|
||||
return fmt.Errorf("failed to parse event filters: %w", err)
|
||||
@ -148,6 +147,8 @@ func (e EventLogFile) Read(ctx context.Context, options ReadOptions) error {
|
||||
return err
|
||||
}
|
||||
|
||||
go func() {
|
||||
defer close(options.EventChannel)
|
||||
var line *tail.Line
|
||||
var ok bool
|
||||
var skipRotate bool
|
||||
@ -156,18 +157,20 @@ func (e EventLogFile) Read(ctx context.Context, options ReadOptions) error {
|
||||
case <-ctx.Done():
|
||||
// the consumer has cancelled
|
||||
t.Kill(errors.New("hangup by client"))
|
||||
return nil
|
||||
return
|
||||
case line, ok = <-t.Lines:
|
||||
if !ok {
|
||||
// channel was closed
|
||||
return nil
|
||||
return
|
||||
}
|
||||
// fallthrough
|
||||
}
|
||||
|
||||
event, err := newEventFromJSONString(line.Text)
|
||||
if err != nil {
|
||||
return err
|
||||
err := fmt.Errorf("event type %s is not valid in %s", event.Type.String(), e.options.LogFilePath)
|
||||
options.EventChannel <- ReadResult{Error: err}
|
||||
continue
|
||||
}
|
||||
switch event.Type {
|
||||
case Image, Volume, Pod, Container, Network:
|
||||
@ -175,7 +178,8 @@ func (e EventLogFile) Read(ctx context.Context, options ReadOptions) error {
|
||||
case System:
|
||||
begin, end, err := e.readRotateEvent(event)
|
||||
if err != nil {
|
||||
return err
|
||||
options.EventChannel <- ReadResult{Error: err}
|
||||
continue
|
||||
}
|
||||
if begin && event.Time.After(readTime) {
|
||||
// If the rotation event happened _after_ we
|
||||
@ -189,15 +193,19 @@ func (e EventLogFile) Read(ctx context.Context, options ReadOptions) error {
|
||||
skipRotate = false
|
||||
}
|
||||
default:
|
||||
return fmt.Errorf("event type %s is not valid in %s", event.Type.String(), e.options.LogFilePath)
|
||||
err := fmt.Errorf("event type %s is not valid in %s", event.Type.String(), e.options.LogFilePath)
|
||||
options.EventChannel <- ReadResult{Error: err}
|
||||
continue
|
||||
}
|
||||
if skipRotate {
|
||||
continue
|
||||
}
|
||||
if applyFilters(event, filterMap) {
|
||||
options.EventChannel <- event
|
||||
options.EventChannel <- ReadResult{Event: event}
|
||||
}
|
||||
}
|
||||
}()
|
||||
return nil
|
||||
}
|
||||
|
||||
// String returns a string representation of the logger
|
||||
|
@ -48,11 +48,8 @@ func GetEvents(w http.ResponseWriter, r *http.Request) {
|
||||
utils.Error(w, http.StatusBadRequest, fmt.Errorf("failed to parse filters for %s: %w", r.URL.String(), err))
|
||||
return
|
||||
}
|
||||
eventChannel := make(chan *events.Event)
|
||||
errorChannel := make(chan error)
|
||||
eventChannel := make(chan events.ReadResult)
|
||||
|
||||
// Start reading events.
|
||||
go func() {
|
||||
readOpts := events.ReadOptions{
|
||||
FromStart: fromStart,
|
||||
Stream: query.Stream,
|
||||
@ -61,8 +58,11 @@ func GetEvents(w http.ResponseWriter, r *http.Request) {
|
||||
Since: query.Since,
|
||||
Until: query.Until,
|
||||
}
|
||||
errorChannel <- runtime.Events(r.Context(), readOpts)
|
||||
}()
|
||||
err = runtime.Events(r.Context(), readOpts)
|
||||
if err != nil {
|
||||
utils.InternalServerError(w, err)
|
||||
return
|
||||
}
|
||||
|
||||
flush := func() {}
|
||||
if flusher, ok := w.(http.Flusher); ok {
|
||||
@ -70,31 +70,29 @@ func GetEvents(w http.ResponseWriter, r *http.Request) {
|
||||
}
|
||||
|
||||
w.Header().Set("Content-Type", "application/json")
|
||||
wroteContent := false
|
||||
defer func() {
|
||||
if !wroteContent {
|
||||
w.WriteHeader(http.StatusOK)
|
||||
flush()
|
||||
}
|
||||
}()
|
||||
|
||||
coder := json.NewEncoder(w)
|
||||
coder.SetEscapeHTML(true)
|
||||
|
||||
for {
|
||||
select {
|
||||
case err := <-errorChannel:
|
||||
if err != nil {
|
||||
utils.InternalServerError(w, err)
|
||||
wroteContent = true
|
||||
}
|
||||
case <-r.Context().Done():
|
||||
return
|
||||
case evt := <-eventChannel:
|
||||
if evt == nil {
|
||||
case evt, ok := <-eventChannel:
|
||||
if !ok {
|
||||
return
|
||||
}
|
||||
if evt.Error != nil {
|
||||
logrus.Errorf("Unable to read event: %q", err)
|
||||
continue
|
||||
}
|
||||
if evt.Event == nil {
|
||||
continue
|
||||
}
|
||||
|
||||
e := entities.ConvertToEntitiesEvent(*evt)
|
||||
e := entities.ConvertToEntitiesEvent(*evt.Event)
|
||||
// Some events differ between Libpod and Docker endpoints.
|
||||
// Handle these differences for Docker-compat.
|
||||
if !utils.IsLibpodRequest(r) && e.Type == "image" && e.Status == "remove" {
|
||||
@ -110,10 +108,7 @@ func GetEvents(w http.ResponseWriter, r *http.Request) {
|
||||
if err := coder.Encode(e); err != nil {
|
||||
logrus.Errorf("Unable to write json: %q", err)
|
||||
}
|
||||
wroteContent = true
|
||||
flush()
|
||||
case <-r.Context().Done():
|
||||
return
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -236,8 +236,7 @@ func waitRemoved(ctrWait containerWaitFn) (int32, error) {
|
||||
func waitNextExit(ctx context.Context, containerName string) (int32, error) {
|
||||
runtime := ctx.Value(api.RuntimeKey).(*libpod.Runtime)
|
||||
containerEngine := &abi.ContainerEngine{Libpod: runtime}
|
||||
eventChannel := make(chan *events.Event)
|
||||
errChannel := make(chan error)
|
||||
eventChannel := make(chan events.ReadResult)
|
||||
opts := entities.EventsOptions{
|
||||
EventChan: eventChannel,
|
||||
Filter: []string{"event=died", fmt.Sprintf("container=%s", containerName)},
|
||||
@ -247,21 +246,22 @@ func waitNextExit(ctx context.Context, containerName string) (int32, error) {
|
||||
// ctx is used to cancel event watching goroutine
|
||||
ctx, cancel := context.WithCancel(ctx)
|
||||
defer cancel()
|
||||
go func() {
|
||||
errChannel <- containerEngine.Events(ctx, opts)
|
||||
}()
|
||||
err := containerEngine.Events(ctx, opts)
|
||||
if err != nil {
|
||||
return -1, err
|
||||
}
|
||||
|
||||
evt, ok := <-eventChannel
|
||||
if ok {
|
||||
if evt.ContainerExitCode != nil {
|
||||
return int32(*evt.ContainerExitCode), nil
|
||||
for evt := range eventChannel {
|
||||
if evt.Error == nil {
|
||||
if evt.Event.ContainerExitCode != nil {
|
||||
return int32(*evt.Event.ContainerExitCode), nil
|
||||
}
|
||||
return -1, nil
|
||||
}
|
||||
// if ok == false then containerEngine.Events() has exited
|
||||
}
|
||||
// if we are here then containerEngine.Events() has exited
|
||||
// it may happen if request was canceled (e.g. client closed connection prematurely) or
|
||||
// the server is in process of shutting down
|
||||
return -1, <-errChannel
|
||||
return -1, nil
|
||||
}
|
||||
|
||||
func waitNotRunning(ctrWait containerWaitFn) (int32, error) {
|
||||
|
@ -3,9 +3,7 @@ package system
|
||||
import (
|
||||
"context"
|
||||
"encoding/json"
|
||||
"errors"
|
||||
"fmt"
|
||||
"io"
|
||||
"net/http"
|
||||
"time"
|
||||
|
||||
@ -31,7 +29,6 @@ func Events(ctx context.Context, eventChan chan types.Event, cancelChan chan boo
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
defer response.Body.Close()
|
||||
|
||||
if cancelChan != nil {
|
||||
go func() {
|
||||
@ -43,9 +40,13 @@ func Events(ctx context.Context, eventChan chan types.Event, cancelChan chan boo
|
||||
}
|
||||
|
||||
if response.StatusCode != http.StatusOK {
|
||||
defer response.Body.Close()
|
||||
return response.Process(nil)
|
||||
}
|
||||
|
||||
go func() {
|
||||
defer response.Body.Close()
|
||||
defer close(eventChan)
|
||||
dec := json.NewDecoder(response.Body)
|
||||
for err = (error)(nil); err == nil; {
|
||||
var e = types.Event{}
|
||||
@ -54,15 +55,8 @@ func Events(ctx context.Context, eventChan chan types.Event, cancelChan chan boo
|
||||
eventChan <- e
|
||||
}
|
||||
}
|
||||
close(eventChan)
|
||||
switch {
|
||||
case err == nil:
|
||||
}()
|
||||
return nil
|
||||
case errors.Is(err, io.EOF):
|
||||
return nil
|
||||
default:
|
||||
return fmt.Errorf("unable to decode event response: %w", err)
|
||||
}
|
||||
}
|
||||
|
||||
// Prune removes all unused system data.
|
||||
|
@ -90,7 +90,7 @@ type DiffReport struct {
|
||||
|
||||
type EventsOptions struct {
|
||||
FromStart bool
|
||||
EventChan chan *events.Event
|
||||
EventChan chan events.ReadResult
|
||||
Filter []string
|
||||
Stream bool
|
||||
Since string
|
||||
|
@ -5,6 +5,7 @@ import (
|
||||
"fmt"
|
||||
"strings"
|
||||
|
||||
"github.com/containers/podman/v5/libpod/events"
|
||||
"github.com/containers/podman/v5/pkg/bindings/system"
|
||||
"github.com/containers/podman/v5/pkg/domain/entities"
|
||||
)
|
||||
@ -23,7 +24,7 @@ func (ic *ContainerEngine) Events(ctx context.Context, opts entities.EventsOptio
|
||||
binChan := make(chan entities.Event)
|
||||
go func() {
|
||||
for e := range binChan {
|
||||
opts.EventChan <- entities.ConvertToLibpodEvent(e)
|
||||
opts.EventChan <- events.ReadResult{Event: entities.ConvertToLibpodEvent(e)}
|
||||
}
|
||||
close(opts.EventChan)
|
||||
}()
|
||||
|
@ -32,4 +32,8 @@ t GET "events?stream=false&since=$START&type=remove" 200 \
|
||||
'select(.status | contains("remove")).Action=remove' \
|
||||
'select(.status | contains("remove")).Actor.Attributes.containerExitCode=1'
|
||||
|
||||
APIV2_TEST_EXPECT_TIMEOUT=1 t GET "events?stream=true" 999
|
||||
like "$(<$WORKDIR/curl.headers.out)" ".*HTTP.* 200 OK.*" \
|
||||
"Received headers from /events"
|
||||
|
||||
# vim: filetype=sh
|
||||
|
Reference in New Issue
Block a user