From e7e2c799c8cedca1b654e547c806ed664ad15949 Mon Sep 17 00:00:00 2001 From: Drew Bailey <2614075+drewbailey@users.noreply.github.com> Date: Mon, 31 Aug 2020 13:19:28 -0400 Subject: [PATCH 01/19] Event Buffer Implemenation adds an event buffer to hold events from raft changes. update events to use event buffer fix append call provide way to prune buffer items after TTL event publisher tests basic publish test wire up max item ttl rename package to stream, cleanup exploratory work subscription filtering subscription plumbing allow subscribers to consume events, handle closing subscriptions back out old exploratory ctx work fix lint remove unused ctx bits add a few comments fix test stop publisher on abandon --- nomad/event/event.go | 13 -- nomad/state/state_changes.go | 20 +- nomad/state/state_store.go | 23 +- nomad/stream/event.go | 14 ++ nomad/stream/event_buffer.go | 306 +++++++++++++++++++++++++++ nomad/stream/event_buffer_test.go | 201 ++++++++++++++++++ nomad/stream/event_publisher.go | 195 +++++++++++++++++ nomad/stream/event_publisher_test.go | 134 ++++++++++++ nomad/stream/subscription.go | 136 ++++++++++++ nomad/stream/subscription_test.go | 76 +++++++ 10 files changed, 1089 insertions(+), 29 deletions(-) delete mode 100644 nomad/event/event.go create mode 100644 nomad/stream/event.go create mode 100644 nomad/stream/event_buffer.go create mode 100644 nomad/stream/event_buffer_test.go create mode 100644 nomad/stream/event_publisher.go create mode 100644 nomad/stream/event_publisher_test.go create mode 100644 nomad/stream/subscription.go create mode 100644 nomad/stream/subscription_test.go diff --git a/nomad/event/event.go b/nomad/event/event.go deleted file mode 100644 index 49e809ac8031..000000000000 --- a/nomad/event/event.go +++ /dev/null @@ -1,13 +0,0 @@ -package event - -type Event struct { - Topic string - Key string - Index uint64 - Payload interface{} -} - -type EventPublisher struct{} - -func NewPublisher() *EventPublisher { return &EventPublisher{} } -func (e EventPublisher) Publish(events []Event) {} diff --git a/nomad/state/state_changes.go b/nomad/state/state_changes.go index 22e32f31b23f..a71056e0f9d0 100644 --- a/nomad/state/state_changes.go +++ b/nomad/state/state_changes.go @@ -4,7 +4,7 @@ import ( "fmt" "github.com/hashicorp/go-memdb" - "github.com/hashicorp/nomad/nomad/event" + "github.com/hashicorp/nomad/nomad/stream" ) // ReadTxn is implemented by memdb.Txn to perform read operations. @@ -29,28 +29,28 @@ type Changes struct { type changeTrackerDB struct { db *memdb.MemDB publisher eventPublisher - processChanges func(ReadTxn, Changes) ([]event.Event, error) + processChanges func(ReadTxn, Changes) ([]stream.Event, error) } func NewChangeTrackerDB(db *memdb.MemDB, publisher eventPublisher, changesFn changeProcessor) *changeTrackerDB { return &changeTrackerDB{ db: db, - publisher: event.NewPublisher(), + publisher: publisher, processChanges: changesFn, } } -type changeProcessor func(ReadTxn, Changes) ([]event.Event, error) +type changeProcessor func(ReadTxn, Changes) ([]stream.Event, error) type eventPublisher interface { - Publish(events []event.Event) + Publish(index uint64, events []stream.Event) } // noOpPublisher satisfies the eventPublisher interface and does nothing type noOpPublisher struct{} -func (n *noOpPublisher) Publish(events []event.Event) {} -func noOpProcessChanges(ReadTxn, Changes) ([]event.Event, error) { return []event.Event{}, nil } +func (n *noOpPublisher) Publish(index uint64, events []stream.Event) {} +func noOpProcessChanges(ReadTxn, Changes) ([]stream.Event, error) { return []stream.Event{}, nil } // ReadTxn returns a read-only transaction which behaves exactly the same as // memdb.Txn @@ -89,7 +89,7 @@ func (c *changeTrackerDB) publish(changes Changes) error { if err != nil { return fmt.Errorf("failed generating events from changes: %v", err) } - c.publisher.Publish(events) + c.publisher.Publish(changes.Index, events) return nil } @@ -146,7 +146,7 @@ func (tx *txn) Commit() error { return nil } -func processDBChanges(tx ReadTxn, changes Changes) ([]event.Event, error) { +func processDBChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { // TODO: add handlers here. - return []event.Event{}, nil + return []stream.Event{}, nil } diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 6bb71382d88e..d15913cd22d0 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -13,7 +13,7 @@ import ( "github.com/pkg/errors" "github.com/hashicorp/nomad/helper" - "github.com/hashicorp/nomad/nomad/event" + "github.com/hashicorp/nomad/nomad/stream" "github.com/hashicorp/nomad/nomad/structs" ) @@ -64,6 +64,10 @@ type StateStore struct { // abandonCh is used to signal watchers that this state store has been // abandoned (usually during a restore). This is only ever closed. abandonCh chan struct{} + + // TODO: refactor abondonCh to use a context so that both can use the same + // cancel mechanism. + stopEventPublisher func() } // NewStateStore is used to create a new state store @@ -75,12 +79,18 @@ func NewStateStore(config *StateStoreConfig) (*StateStore, error) { } // Create the state store + ctx, cancel := context.WithCancel(context.TODO()) s := &StateStore{ - logger: config.Logger.Named("state_store"), - config: config, - abandonCh: make(chan struct{}), - } - s.db = NewChangeTrackerDB(db, event.NewPublisher(), processDBChanges) + logger: config.Logger.Named("state_store"), + config: config, + abandonCh: make(chan struct{}), + stopEventPublisher: cancel, + } + publisher := stream.NewEventPublisher(ctx, stream.EventPublisherCfg{ + EventBufferTTL: 1 * time.Hour, + EventBufferSize: 250, + }) + s.db = NewChangeTrackerDB(db, publisher, processDBChanges) // Initialize the state store with required enterprise objects if err := s.enterpriseInit(); err != nil { @@ -189,6 +199,7 @@ func (s *StateStore) AbandonCh() <-chan struct{} { // Abandon is used to signal that the given state store has been abandoned. // Calling this more than one time will panic. func (s *StateStore) Abandon() { + s.stopEventPublisher() close(s.abandonCh) } diff --git a/nomad/stream/event.go b/nomad/stream/event.go new file mode 100644 index 000000000000..2625dede2049 --- /dev/null +++ b/nomad/stream/event.go @@ -0,0 +1,14 @@ +package stream + +const ( + AllKeys = "*" +) + +type Topic string + +type Event struct { + Topic Topic + Key string + Index uint64 + Payload interface{} +} diff --git a/nomad/stream/event_buffer.go b/nomad/stream/event_buffer.go new file mode 100644 index 000000000000..127ca5b9fdab --- /dev/null +++ b/nomad/stream/event_buffer.go @@ -0,0 +1,306 @@ +package stream + +import ( + "context" + "errors" + "fmt" + "sync/atomic" + "time" +) + +// eventBuffer is a single-writer, multiple-reader, fixed length concurrent +// buffer of events that have been published. The buffer is +// the head and tail of an atomically updated single-linked list. Atomic +// accesses are usually to be suspected as premature optimization but this +// specific design has several important features that significantly simplify a +// lot of our PubSub machinery. +// +// eventBuffer is an adaptation of conuls agent/stream/event eventBuffer but +// has been updated to be a max length buffer to work for Nomad's usecase. +// +// The eventBuffer only tracks the most recent set of published events, +// up to the max configured size, older events are dropped from the buffer +// but will only be garbage collected once the slowest reader drops the item. +// Consumers are notified of new events by closing a channel on the previous head +// allowing efficient broadcast to many watchers without having to run multiple +// goroutines or deliver to O(N) separate channels. +// +// Because eventBuffer is a linked list with atomically updated pointers, readers don't +// have to take a lock and can consume at their own pace. Slow readers can eventually +// append +// +// A new buffer is constructed with a sentinel "empty" bufferItem that has a nil +// Events array. This enables subscribers to start watching for the next update +// immediately. +// +// The zero value eventBuffer is _not_ usable, as it has not been +// initialized with an empty bufferItem so can not be used to wait for the first +// published event. Call newEventBuffer to construct a new buffer. +// +// Calls to Append or AppendBuffer that mutate the head must be externally +// synchronized. This allows systems that already serialize writes to append +// without lock overhead (e.g. a snapshot goroutine appending thousands of +// events). +type eventBuffer struct { + size *int64 + + head atomic.Value + tail atomic.Value + + maxSize int64 + maxItemTTL time.Duration +} + +// newEventBuffer creates an eventBuffer ready for use. +func newEventBuffer(size int64, maxItemTTL time.Duration) *eventBuffer { + zero := int64(0) + b := &eventBuffer{ + maxSize: size, + size: &zero, + maxItemTTL: maxItemTTL, + } + + item := newBufferItem(0, nil) + + b.head.Store(item) + b.tail.Store(item) + + return b +} + +// Append a set of events from one raft operation to the buffer and notify +// watchers. After calling append, the caller must not make any further +// mutations to the events as they may have been exposed to subscribers in other +// goroutines. Append only supports a single concurrent caller and must be +// externally synchronized with other Append, AppendBuffer or AppendErr calls. +func (b *eventBuffer) Append(index uint64, events []Event) { + b.appendItem(newBufferItem(index, events)) +} + +func (b *eventBuffer) appendItem(item *bufferItem) { + // Store the next item to the old tail + oldTail := b.Tail() + oldTail.link.next.Store(item) + + // Update the tail to the new item + b.tail.Store(item) + + // Increment the buffer size + size := atomic.AddInt64(b.size, 1) + + // Check if we need to advance the head to keep the list + // constrained to max size + if size > b.maxSize { + b.advanceHead() + } + + // notify waiters next event is available + close(oldTail.link.ch) + +} + +// advanceHead drops the current Head buffer item and notifies readers +// that the item should be discarded by closing droppedCh. +// Slow readers will prevent the old head from being GC'd until they +// discard it. +func (b *eventBuffer) advanceHead() { + old := b.Head() + next := old.link.next.Load() + + close(old.link.droppedCh) + b.head.Store(next) + atomic.AddInt64(b.size, -1) + +} + +// Head returns the current head of the buffer. It will always exist but it may +// be a "sentinel" empty item with a nil Events slice to allow consumers to +// watch for the next update. Consumers should always check for empty Events and +// treat them as no-ops. Will panic if eventBuffer was not initialized correctly +// with NewEventBuffer +func (b *eventBuffer) Head() *bufferItem { + return b.head.Load().(*bufferItem) +} + +// Tail returns the current tail of the buffer. It will always exist but it may +// be a "sentinel" empty item with a Nil Events slice to allow consumers to +// watch for the next update. Consumers should always check for empty Events and +// treat them as no-ops. Will panic if eventBuffer was not initialized correctly +// with NewEventBuffer +func (b *eventBuffer) Tail() *bufferItem { + return b.tail.Load().(*bufferItem) +} + +// StarStartAtClosest returns the closest bufferItem to a requested starting +// index as well as the offset between the requested index and returned one. +func (b *eventBuffer) StartAtClosest(index uint64) (*bufferItem, int) { + item := b.Head() + if index < item.Index { + return item, int(item.Index) - int(index) + } + if item.Index == index { + return item, 0 + } + + for { + prev := item + item = item.NextNoBlock() + if item == nil { + return prev, int(index) - int(prev.Index) + } + if index < item.Index { + return item, int(item.Index) - int(index) + } + if index == item.Index { + return item, 0 + } + } +} + +// Len returns the current length of the buffer +func (b *eventBuffer) Len() int { + return int(atomic.LoadInt64(b.size)) +} + +// prune advances the head of the buffer until the head buffer item TTL +// is no longer expired. It should be externally synchronized as it mutates +// the buffer of items. +func (b *eventBuffer) prune() { + for { + head := b.Head() + if b.Len() == 0 { + return + } + + if time.Since(head.createdAt) > b.maxItemTTL { + b.advanceHead() + } else { + return + } + } +} + +// bufferItem represents a set of events published by a single raft operation. +// The first item returned by a newly constructed buffer will have nil Events. +// It is a sentinel value which is used to wait on the next events via Next. +// +// To iterate to the next event, a Next method may be called which may block if +// there is no next element yet. +// +// Holding a pointer to the item keeps all the events published since in memory +// so it's important that subscribers don't hold pointers to buffer items after +// they have been delivered except where it's intentional to maintain a cache or +// trailing store of events for performance reasons. +// +// Subscribers must not mutate the bufferItem or the Events or Encoded payloads +// inside as these are shared between all readers. +type bufferItem struct { + // Events is the set of events published at one raft index. This may be nil as + // a sentinel value to allow watching for the first event in a buffer. Callers + // should check and skip nil Events at any point in the buffer. It will also + // be nil if the producer appends an Error event because they can't complete + // the request to populate the buffer. Err will be non-nil in this case. + Events []Event + + Index uint64 + + // Err is non-nil if the producer can't complete their task and terminates the + // buffer. Subscribers should return the error to clients and cease attempting + // to read from the buffer. + Err error + + // link holds the next pointer and channel. This extra bit of indirection + // allows us to splice buffers together at arbitrary points without including + // events in one buffer just for the side-effect of watching for the next set. + // The link may not be mutated once the event is appended to a buffer. + link *bufferLink + + createdAt time.Time +} + +type bufferLink struct { + // next is an atomically updated pointer to the next event in the buffer. It + // is written exactly once by the single published and will always be set if + // ch is closed. + next atomic.Value + + // ch is closed when the next event is published. It should never be mutated + // (e.g. set to nil) as that is racey, but is closed once when the next event + // is published. the next pointer will have been set by the time this is + // closed. + ch chan struct{} + + // droppedCh is closed when the event is dropped from the buffer due to + // sizing constraints. + droppedCh chan struct{} +} + +// newBufferItem returns a blank buffer item with a link and chan ready to have +// the fields set and be appended to a buffer. +func newBufferItem(index uint64, events []Event) *bufferItem { + return &bufferItem{ + link: &bufferLink{ + ch: make(chan struct{}), + droppedCh: make(chan struct{}), + }, + Events: events, + Index: index, + createdAt: time.Now(), + } +} + +// Next return the next buffer item in the buffer. It may block until ctx is +// cancelled or until the next item is published. +func (i *bufferItem) Next(ctx context.Context, forceClose <-chan struct{}) (*bufferItem, error) { + // See if there is already a next value, block if so. Note we don't rely on + // state change (chan nil) as that's not threadsafe but detecting close is. + select { + case <-ctx.Done(): + return nil, ctx.Err() + case <-forceClose: + return nil, fmt.Errorf("subscription closed") + case <-i.link.ch: + } + + // Check if the reader is too slow and the event buffer as discarded the event + select { + case <-i.link.droppedCh: + return nil, fmt.Errorf("event dropped from buffer") + default: + } + + // If channel closed, there must be a next item to read + nextRaw := i.link.next.Load() + if nextRaw == nil { + // shouldn't be possible + return nil, errors.New("invalid next item") + } + next := nextRaw.(*bufferItem) + if next.Err != nil { + return nil, next.Err + } + return next, nil +} + +// NextNoBlock returns the next item in the buffer without blocking. If it +// reaches the most recent item it will return nil. +func (i *bufferItem) NextNoBlock() *bufferItem { + nextRaw := i.link.next.Load() + if nextRaw == nil { + return nil + } + return nextRaw.(*bufferItem) +} + +// NextLink returns either the next item in the buffer if there is one, or +// an empty item (that will be ignored by subscribers) that has a pointer to +// the same link as this bufferItem (but none of the bufferItem content). +// When the link.ch is closed, subscriptions will be notified of the next item. +func (i *bufferItem) NextLink() *bufferItem { + next := i.NextNoBlock() + if next == nil { + // Return an empty item that can be followed to the next item published. + return &bufferItem{link: i.link} + } + return next +} diff --git a/nomad/stream/event_buffer_test.go b/nomad/stream/event_buffer_test.go new file mode 100644 index 000000000000..15a96a4e0d76 --- /dev/null +++ b/nomad/stream/event_buffer_test.go @@ -0,0 +1,201 @@ +package stream + +import ( + "context" + "fmt" + "math/rand" + "testing" + "time" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +func TestEventBufferFuzz(t *testing.T) { + nReaders := 1000 + nMessages := 1000 + + b := newEventBuffer(1000, DefaultTTL) + + // Start a write goroutine that will publish 10000 messages with sequential + // indexes and some jitter in timing (to allow clients to "catch up" and block + // waiting for updates). + go func() { + seed := time.Now().UnixNano() + t.Logf("Using seed %d", seed) + // z is a Zipfian distribution that gives us a number of milliseconds to + // sleep which are mostly low - near zero but occasionally spike up to near + // 100. + z := rand.NewZipf(rand.New(rand.NewSource(seed)), 1.5, 1.5, 50) + + for i := 0; i < nMessages; i++ { + // Event content is arbitrary and not valid for our use of buffers in + // streaming - here we only care about the semantics of the buffer. + e := Event{ + Index: uint64(i), // Indexes should be contiguous + } + b.Append(uint64(i), []Event{e}) + // Sleep sometimes for a while to let some subscribers catch up + wait := time.Duration(z.Uint64()) * time.Millisecond + time.Sleep(wait) + } + }() + + // Run n subscribers following and verifying + errCh := make(chan error, nReaders) + + // Load head here so all subscribers start from the same point or they might + // not run until several appends have already happened. + head := b.Head() + + for i := 0; i < nReaders; i++ { + go func(i int) { + expect := uint64(0) + item := head + var err error + for { + item, err = item.Next(context.Background(), nil) + if err != nil { + errCh <- fmt.Errorf("subscriber %05d failed getting next %d: %s", i, + expect, err) + return + } + if item.Events[0].Index != expect { + errCh <- fmt.Errorf("subscriber %05d got bad event want=%d, got=%d", i, + expect, item.Events[0].Index) + return + } + expect++ + if expect == uint64(nMessages) { + // Succeeded + errCh <- nil + return + } + } + }(i) + } + + // Wait for all readers to finish one way or other + for i := 0; i < nReaders; i++ { + err := <-errCh + assert.NoError(t, err) + } +} + +func TestEventBuffer_Slow_Reader(t *testing.T) { + + b := newEventBuffer(10, DefaultTTL) + + for i := 0; i < 10; i++ { + e := Event{ + Index: uint64(i), // Indexes should be contiguous + } + b.Append(uint64(i), []Event{e}) + } + + head := b.Head() + + for i := 10; i < 15; i++ { + e := Event{ + Index: uint64(i), // Indexes should be contiguous + } + b.Append(uint64(i), []Event{e}) + } + + // Ensure the slow reader errors to handle dropped events and + // fetch latest head + ev, err := head.Next(context.Background(), nil) + require.Error(t, err) + require.Nil(t, ev) + + newHead := b.Head() + require.Equal(t, 4, int(newHead.Index)) +} + +func TestEventBuffer_Size(t *testing.T) { + b := newEventBuffer(100, DefaultTTL) + + for i := 0; i < 10; i++ { + e := Event{ + Index: uint64(i), // Indexes should be contiguous + } + b.Append(uint64(i), []Event{e}) + } + + require.Equal(t, 10, b.Len()) +} + +// TestEventBuffer_Prune_AllOld tests the behavior when all items +// are past their TTL, the event buffer should prune down to the last message +// and hold onto the last item. +func TestEventBuffer_Prune_AllOld(t *testing.T) { + b := newEventBuffer(100, 1*time.Second) + + for i := 0; i < 10; i++ { + e := Event{ + Index: uint64(i), // Indexes should be contiguous + } + b.Append(uint64(i), []Event{e}) + } + + require.Equal(t, 10, int(b.Len())) + + time.Sleep(1 * time.Second) + + b.prune() + + require.Equal(t, 9, int(b.Head().Index)) + require.Equal(t, 0, b.Len()) +} + +func TestStartAt_CurrentIdx_Past_Start(t *testing.T) { + cases := []struct { + desc string + req uint64 + expected uint64 + offset int + }{ + { + desc: "requested index less than head receives head", + req: 10, + expected: 11, + offset: 1, + }, + { + desc: "requested exact match head", + req: 11, + expected: 11, + offset: 0, + }, + { + desc: "requested exact match", + req: 42, + expected: 42, + offset: 0, + }, + { + desc: "requested index greater than tail receives tail", + req: 500, + expected: 100, + offset: 400, + }, + } + + // buffer starts at index 11 goes to 100 + b := newEventBuffer(100, 1*time.Hour) + + for i := 11; i <= 100; i++ { + e := Event{ + Index: uint64(i), // Indexes should be contiguous + } + b.Append(uint64(i), []Event{e}) + } + + for _, tc := range cases { + t.Run(tc.desc, func(t *testing.T) { + got, offset := b.StartAtClosest(tc.req) + require.Equal(t, int(tc.expected), int(got.Index)) + require.Equal(t, tc.offset, offset) + }) + } +} diff --git a/nomad/stream/event_publisher.go b/nomad/stream/event_publisher.go new file mode 100644 index 000000000000..016119b6e0ca --- /dev/null +++ b/nomad/stream/event_publisher.go @@ -0,0 +1,195 @@ +package stream + +import ( + "context" + "sync" + "time" + + "github.com/hashicorp/go-hclog" +) + +const ( + DefaultTTL = 1 * time.Hour +) + +type EventPublisherCfg struct { + EventBufferSize int64 + EventBufferTTL time.Duration +} + +type EventPublisher struct { + // lock protects the eventbuffer + lock sync.Mutex + + // eventBuf stores a configurable amount of events in memory + eventBuf *eventBuffer + + // pruneTick is the duration to periodically prune events from the event + // buffer. Defaults to 5s + pruneTick time.Duration + + logger hclog.Logger + + subscriptions *subscriptions + + // publishCh is used to send messages from an active txn to a goroutine which + // publishes events, so that publishing can happen asynchronously from + // the Commit call in the FSM hot path. + publishCh chan changeEvents +} + +type subscriptions struct { + // lock for byToken. If both subscription.lock and EventPublisher.lock need + // to be held, EventPublisher.lock MUST always be acquired first. + lock sync.RWMutex + + // byToken is an mapping of active Subscriptions indexed by a token and + // a pointer to the request. + // When the token is modified all subscriptions under that token will be + // reloaded. + // A subscription may be unsubscribed by using the pointer to the request. + byToken map[string]map[*SubscribeRequest]*Subscription +} + +func NewEventPublisher(ctx context.Context, cfg EventPublisherCfg) *EventPublisher { + if cfg.EventBufferTTL == 0 { + cfg.EventBufferTTL = 1 * time.Hour + } + buffer := newEventBuffer(cfg.EventBufferSize, cfg.EventBufferTTL) + e := &EventPublisher{ + eventBuf: buffer, + publishCh: make(chan changeEvents), + subscriptions: &subscriptions{ + byToken: make(map[string]map[*SubscribeRequest]*Subscription), + }, + } + + go e.handleUpdates(ctx) + go e.periodicPrune(ctx) + + return e +} + +// Publish events to all subscribers of the event Topic. +func (e *EventPublisher) Publish(index uint64, events []Event) { + if len(events) > 0 { + e.publishCh <- changeEvents{index: index, events: events} + } +} + +// Subscribe returns a new Subscription for a given request. +func (e *EventPublisher) Subscribe(req *SubscribeRequest) (*Subscription, error) { + e.lock.Lock() + defer e.lock.Unlock() + + var head *bufferItem + var offset int + if req.Index != 0 { + head, offset = e.eventBuf.StartAtClosest(req.Index) + } else { + head = e.eventBuf.Head() + } + if offset > 0 { + e.logger.Warn("requested index no longer in buffer", "requsted", int(req.Index), "closest", int(head.Index)) + } + + sub := newSubscription(req, head, func() {}) + + e.subscriptions.add(req, sub) + return sub, nil +} + +func (e *EventPublisher) handleUpdates(ctx context.Context) { + for { + select { + case <-ctx.Done(): + e.subscriptions.closeAll() + return + case update := <-e.publishCh: + e.sendEvents(update) + } + } +} + +func (e *EventPublisher) periodicPrune(ctx context.Context) { + for { + select { + case <-ctx.Done(): + return + case <-time.After(e.pruneTick): + e.lock.Lock() + e.eventBuf.prune() + e.lock.Unlock() + } + } +} + +type changeEvents struct { + index uint64 + events []Event +} + +// sendEvents sends the given events to the publishers event buffer. +func (e *EventPublisher) sendEvents(update changeEvents) { + e.lock.Lock() + defer e.lock.Unlock() + + e.eventBuf.Append(update.index, update.events) +} + +func (s *subscriptions) add(req *SubscribeRequest, sub *Subscription) { + s.lock.Lock() + defer s.lock.Unlock() + + subsByToken, ok := s.byToken[req.Token] + if !ok { + subsByToken = make(map[*SubscribeRequest]*Subscription) + s.byToken[req.Token] = subsByToken + } + subsByToken[req] = sub +} + +func (s *subscriptions) closeSubscriptionsForTokens(tokenSecretIDs []string) { + s.lock.RLock() + defer s.lock.RUnlock() + + for _, secretID := range tokenSecretIDs { + if subs, ok := s.byToken[secretID]; ok { + for _, sub := range subs { + sub.forceClose() + } + } + } +} + +// unsubscribe returns a function that the subscription will call to remove +// itself from the subsByToken. +// This function is returned as a closure so that the caller doesn't need to keep +// track of the SubscriptionRequest, and can not accidentally call unsubscribe with the +// wrong pointer. +func (s *subscriptions) unsubscribe(req *SubscribeRequest) func() { + return func() { + s.lock.Lock() + defer s.lock.Unlock() + + subsByToken, ok := s.byToken[req.Token] + if !ok { + return + } + delete(subsByToken, req) + if len(subsByToken) == 0 { + delete(s.byToken, req.Token) + } + } +} + +func (s *subscriptions) closeAll() { + s.lock.Lock() + defer s.lock.Unlock() + + for _, byRequest := range s.byToken { + for _, sub := range byRequest { + sub.forceClose() + } + } +} diff --git a/nomad/stream/event_publisher_test.go b/nomad/stream/event_publisher_test.go new file mode 100644 index 000000000000..5085a6fd970b --- /dev/null +++ b/nomad/stream/event_publisher_test.go @@ -0,0 +1,134 @@ +package stream + +import ( + "context" + "testing" + "time" + + "github.com/stretchr/testify/require" +) + +func TestEventPublisher_PublishChangesAndSubscribe(t *testing.T) { + subscription := &SubscribeRequest{ + Topics: map[Topic][]string{ + "Test": []string{"sub-key"}, + }, + } + ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) + defer cancel() + + publisher := NewEventPublisher(ctx, EventPublisherCfg{EventBufferSize: 100, EventBufferTTL: DefaultTTL}) + sub, err := publisher.Subscribe(subscription) + require.NoError(t, err) + eventCh := consumeSubscription(ctx, sub) + + // Now subscriber should block waiting for updates + assertNoResult(t, eventCh) + + events := []Event{{ + Index: 1, + Topic: "Test", + Key: "sub-key", + Payload: "sample payload", + }} + publisher.Publish(1, events) + + // Subscriber should see the published event + result := nextResult(t, eventCh) + require.NoError(t, result.Err) + expected := []Event{{Payload: "sample payload", Key: "sub-key", Topic: "Test", Index: 1}} + require.Equal(t, expected, result.Events) + + // Now subscriber should block waiting for updates + assertNoResult(t, eventCh) + + // Publish a second event + events = []Event{{ + Index: 2, + Topic: "Test", + Key: "sub-key", + Payload: "sample payload 2", + }} + publisher.Publish(2, events) + + result = nextResult(t, eventCh) + require.NoError(t, result.Err) + expected = []Event{{Payload: "sample payload 2", Key: "sub-key", Topic: "Test", Index: 2}} + require.Equal(t, expected, result.Events) +} + +func TestEventPublisher_ShutdownClosesSubscriptions(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + t.Cleanup(cancel) + + publisher := NewEventPublisher(ctx, EventPublisherCfg{}) + + sub1, err := publisher.Subscribe(&SubscribeRequest{}) + require.NoError(t, err) + defer sub1.Unsubscribe() + + sub2, err := publisher.Subscribe(&SubscribeRequest{}) + require.NoError(t, err) + defer sub2.Unsubscribe() + + cancel() // Shutdown + + err = consumeSub(context.Background(), sub1) + require.Equal(t, err, ErrSubscriptionClosed) + + _, err = sub2.Next(context.Background()) + require.Equal(t, err, ErrSubscriptionClosed) +} + +func consumeSubscription(ctx context.Context, sub *Subscription) <-chan subNextResult { + eventCh := make(chan subNextResult, 1) + go func() { + for { + es, err := sub.Next(ctx) + eventCh <- subNextResult{ + Events: es, + Err: err, + } + if err != nil { + return + } + } + }() + return eventCh +} + +type subNextResult struct { + Events []Event + Err error +} + +func nextResult(t *testing.T, eventCh <-chan subNextResult) subNextResult { + t.Helper() + select { + case next := <-eventCh: + return next + case <-time.After(100 * time.Millisecond): + t.Fatalf("no event after 100ms") + } + return subNextResult{} +} + +func assertNoResult(t *testing.T, eventCh <-chan subNextResult) { + t.Helper() + select { + case next := <-eventCh: + require.NoError(t, next.Err) + require.Len(t, next.Events, 1) + t.Fatalf("received unexpected event: %#v", next.Events[0].Payload) + case <-time.After(100 * time.Millisecond): + } +} + +func consumeSub(ctx context.Context, sub *Subscription) error { + for { + _, err := sub.Next(ctx) + if err != nil { + return err + } + } +} diff --git a/nomad/stream/subscription.go b/nomad/stream/subscription.go new file mode 100644 index 000000000000..6846e005bd35 --- /dev/null +++ b/nomad/stream/subscription.go @@ -0,0 +1,136 @@ +package stream + +import ( + "context" + "errors" + "sync/atomic" +) + +const ( + // subscriptionStateOpen is the default state of a subscription. An open + // subscription may receive new events. + subscriptionStateOpen uint32 = 0 + + // subscriptionStateClosed indicates that the subscription was closed, possibly + // as a result of a change to an ACL token, and will not receive new events. + // The subscriber must issue a new Subscribe request. + subscriptionStateClosed uint32 = 1 +) + +// ErrSubscriptionClosed is a error signalling the subscription has been +// closed. The client should Unsubscribe, then re-Subscribe. +var ErrSubscriptionClosed = errors.New("subscription closed by server, client should resubscribe") + +// type Subscriber struct { +// logger hclog.Logger +// } + +type Subscription struct { + // state is accessed atomically 0 means open, 1 means closed with reload + state uint32 + + req *SubscribeRequest + + // currentItem stores the current buffer item we are on. It + // is mutated by calls to Next. + currentItem *bufferItem + + // forceClosed is closed when forceClose is called. It is used by + // EventPublisher to cancel Next(). + forceClosed chan struct{} + + // unsub is a function set by EventPublisher that is called to free resources + // when the subscription is no longer needed. + // It must be safe to call the function from multiple goroutines and the function + // must be idempotent. + unsub func() +} + +type SubscribeRequest struct { + Token string + Index uint64 + + Topics map[Topic][]string +} + +func newSubscription(req *SubscribeRequest, item *bufferItem, unsub func()) *Subscription { + return &Subscription{ + forceClosed: make(chan struct{}), + req: req, + currentItem: item, + unsub: unsub, + } +} + +func (s *Subscription) Next(ctx context.Context) ([]Event, error) { + if atomic.LoadUint32(&s.state) == subscriptionStateClosed { + return nil, ErrSubscriptionClosed + } + + for { + next, err := s.currentItem.Next(ctx, s.forceClosed) + switch { + case err != nil && atomic.LoadUint32(&s.state) == subscriptionStateClosed: + return nil, ErrSubscriptionClosed + case err != nil: + return nil, err + } + s.currentItem = next + + events := filter(s.req, next.Events) + if len(events) == 0 { + continue + } + return events, nil + } +} + +func (s *Subscription) forceClose() { + swapped := atomic.CompareAndSwapUint32(&s.state, subscriptionStateOpen, subscriptionStateClosed) + if swapped { + close(s.forceClosed) + } +} + +func (s *Subscription) Unsubscribe() { + s.unsub() +} + +// filter events to only those that match a subscriptions topic/keys +func filter(req *SubscribeRequest, events []Event) []Event { + if len(events) == 0 { + return events + } + + var count int + for _, e := range events { + if _, ok := req.Topics[e.Topic]; ok { + for _, k := range req.Topics[e.Topic] { + if e.Key == k || k == AllKeys { + count++ + } + } + } + } + + // Only allocate a new slice if some events need to be filtered out + switch count { + case 0: + return nil + case len(events): + return events + } + + // Return filtered events + result := make([]Event, 0, count) + for _, e := range events { + if _, ok := req.Topics[e.Topic]; ok { + for _, k := range req.Topics[e.Topic] { + if e.Key == k || k == AllKeys { + result = append(result, e) + } + } + } + } + return result +} diff --git a/nomad/stream/subscription_test.go b/nomad/stream/subscription_test.go new file mode 100644 index 000000000000..9d1f55720159 --- /dev/null +++ b/nomad/stream/subscription_test.go @@ -0,0 +1,76 @@ +package stream + +import ( + "testing" + + "github.com/stretchr/testify/require" +) + +func TestSubscription(t *testing.T) { + +} + +func TestFilter_AllKeys(t *testing.T) { + events := make([]Event, 0, 5) + events = append(events, Event{Topic: "Test", Key: "One"}, Event{Topic: "Test", Key: "Two"}) + + req := &SubscribeRequest{ + Topics: map[Topic][]string{ + "Test": []string{"*"}, + }, + } + actual := filter(req, events) + require.Equal(t, events, actual) + + // ensure new array was not allocated + require.Equal(t, cap(actual), 5) +} + +func TestFilter_PartialMatch_Topic(t *testing.T) { + events := make([]Event, 0, 5) + events = append(events, Event{Topic: "Test", Key: "One"}, Event{Topic: "Test", Key: "Two"}, Event{Topic: "Exclude", Key: "Two"}) + + req := &SubscribeRequest{ + Topics: map[Topic][]string{ + "Test": []string{"*"}, + }, + } + actual := filter(req, events) + expected := []Event{{Topic: "Test", Key: "One"}, {Topic: "Test", Key: "Two"}} + require.Equal(t, expected, actual) + + require.Equal(t, cap(actual), 2) +} + +func TestFilter_PartialMatch_Key(t *testing.T) { + events := make([]Event, 0, 5) + events = append(events, Event{Topic: "Test", Key: "One"}, Event{Topic: "Test", Key: "Two"}) + + req := &SubscribeRequest{ + Topics: map[Topic][]string{ + "Test": []string{"One"}, + }, + } + actual := filter(req, events) + expected := []Event{{Topic: "Test", Key: "One"}} + require.Equal(t, expected, actual) + + require.Equal(t, cap(actual), 1) +} + +func TestFilter_NoMatch(t *testing.T) { + events := make([]Event, 0, 5) + events = append(events, Event{Topic: "Test", Key: "One"}, Event{Topic: "Test", Key: "Two"}) + + req := &SubscribeRequest{ + Topics: map[Topic][]string{ + "NodeEvents": []string{"*"}, + "Test": []string{"Highly-Specific-Key"}, + }, + } + actual := filter(req, events) + var expected []Event + require.Equal(t, expected, actual) + + require.Equal(t, cap(actual), 0) +} From c9bf8ce5205cd3804e14ec2c106ff7c907df1175 Mon Sep 17 00:00:00 2001 From: Drew Bailey <2614075+drewbailey@users.noreply.github.com> Date: Wed, 16 Sep 2020 14:13:57 -0400 Subject: [PATCH 02/19] abandon current state on server shutdown --- nomad/fsm.go | 1 + 1 file changed, 1 insertion(+) diff --git a/nomad/fsm.go b/nomad/fsm.go index e4aeb33b916e..0fa7303412d6 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -162,6 +162,7 @@ func NewFSM(config *FSMConfig) (*nomadFSM, error) { // Close is used to cleanup resources associated with the FSM func (n *nomadFSM) Close() error { + n.state.Abandon() return nil } From 26263091fdfa25ac8079648878cccc2afd91f454 Mon Sep 17 00:00:00 2001 From: Drew Bailey <2614075+drewbailey@users.noreply.github.com> Date: Thu, 17 Sep 2020 11:22:16 -0400 Subject: [PATCH 03/19] Events/cfg enable publisher (#8916) * only enable publisher based on config * add default prune tick * back out state abandon changes on fsm close --- nomad/fsm.go | 1 - nomad/state/state_store.go | 17 ++++++++++++----- nomad/stream/event_publisher.go | 2 ++ 3 files changed, 14 insertions(+), 6 deletions(-) diff --git a/nomad/fsm.go b/nomad/fsm.go index 0fa7303412d6..e4aeb33b916e 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -162,7 +162,6 @@ func NewFSM(config *FSMConfig) (*nomadFSM, error) { // Close is used to cleanup resources associated with the FSM func (n *nomadFSM) Close() error { - n.state.Abandon() return nil } diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index d15913cd22d0..4440571dd9e6 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -45,6 +45,8 @@ type StateStoreConfig struct { // Region is the region of the server embedding the state store. Region string + + EnablePublisher bool } // The StateStore is responsible for maintaining all the Nomad @@ -86,11 +88,16 @@ func NewStateStore(config *StateStoreConfig) (*StateStore, error) { abandonCh: make(chan struct{}), stopEventPublisher: cancel, } - publisher := stream.NewEventPublisher(ctx, stream.EventPublisherCfg{ - EventBufferTTL: 1 * time.Hour, - EventBufferSize: 250, - }) - s.db = NewChangeTrackerDB(db, publisher, processDBChanges) + + if config.EnablePublisher { + publisher := stream.NewEventPublisher(ctx, stream.EventPublisherCfg{ + EventBufferTTL: 1 * time.Hour, + EventBufferSize: 250, + }) + s.db = NewChangeTrackerDB(db, publisher, processDBChanges) + } else { + s.db = NewChangeTrackerDB(db, &noOpPublisher{}, processDBChanges) + } // Initialize the state store with required enterprise objects if err := s.enterpriseInit(); err != nil { diff --git a/nomad/stream/event_publisher.go b/nomad/stream/event_publisher.go index 016119b6e0ca..7a0d78fe24cb 100644 --- a/nomad/stream/event_publisher.go +++ b/nomad/stream/event_publisher.go @@ -55,6 +55,7 @@ func NewEventPublisher(ctx context.Context, cfg EventPublisherCfg) *EventPublish if cfg.EventBufferTTL == 0 { cfg.EventBufferTTL = 1 * time.Hour } + buffer := newEventBuffer(cfg.EventBufferSize, cfg.EventBufferTTL) e := &EventPublisher{ eventBuf: buffer, @@ -62,6 +63,7 @@ func NewEventPublisher(ctx context.Context, cfg EventPublisherCfg) *EventPublish subscriptions: &subscriptions{ byToken: make(map[string]map[*SubscribeRequest]*Subscription), }, + pruneTick: 5 * time.Second, } go e.handleUpdates(ctx) From 8bd15b534fbd7b98fb12d47267aaa0f2a3d83561 Mon Sep 17 00:00:00 2001 From: Drew Bailey <2614075+drewbailey@users.noreply.github.com> Date: Wed, 23 Sep 2020 10:52:04 -0400 Subject: [PATCH 04/19] Events/event source node (#8918) * Node Register/Deregister event sourcing example upsert node with context fill in writetxnwithctx ctx passing to handle event type creation, wip test node deregistration event drop Node from registration event * node batch deregistration --- nomad/fsm.go | 17 ++- nomad/state/node_events.go | 75 ++++++++++++ nomad/state/node_events_test.go | 211 ++++++++++++++++++++++++++++++++ nomad/state/state_changes.go | 53 +++++++- nomad/state/state_store.go | 60 ++++++++- nomad/state/testing.go | 19 +++ 6 files changed, 423 insertions(+), 12 deletions(-) create mode 100644 nomad/state/node_events.go create mode 100644 nomad/state/node_events_test.go diff --git a/nomad/fsm.go b/nomad/fsm.go index e4aeb33b916e..aa525da0a176 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -1,6 +1,7 @@ package nomad import ( + "context" "fmt" "io" "reflect" @@ -195,9 +196,9 @@ func (n *nomadFSM) Apply(log *raft.Log) interface{} { switch msgType { case structs.NodeRegisterRequestType: - return n.applyUpsertNode(buf[1:], log.Index) + return n.applyUpsertNode(msgType, buf[1:], log.Index) case structs.NodeDeregisterRequestType: - return n.applyDeregisterNode(buf[1:], log.Index) + return n.applyDeregisterNode(msgType, buf[1:], log.Index) case structs.NodeUpdateStatusRequestType: return n.applyStatusUpdate(buf[1:], log.Index) case structs.NodeUpdateDrainRequestType: @@ -310,17 +311,19 @@ func (n *nomadFSM) applyClusterMetadata(buf []byte, index uint64) interface{} { return nil } -func (n *nomadFSM) applyUpsertNode(buf []byte, index uint64) interface{} { +func (n *nomadFSM) applyUpsertNode(reqType structs.MessageType, buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "register_node"}, time.Now()) var req structs.NodeRegisterRequest if err := structs.Decode(buf, &req); err != nil { panic(fmt.Errorf("failed to decode request: %v", err)) } + ctx := context.WithValue(context.Background(), state.CtxMsgType, reqType) + // Handle upgrade paths req.Node.Canonicalize() - if err := n.state.UpsertNode(index, req.Node); err != nil { + if err := n.state.UpsertNodeCtx(ctx, index, req.Node); err != nil { n.logger.Error("UpsertNode failed", "error", err) return err } @@ -334,14 +337,16 @@ func (n *nomadFSM) applyUpsertNode(buf []byte, index uint64) interface{} { return nil } -func (n *nomadFSM) applyDeregisterNode(buf []byte, index uint64) interface{} { +func (n *nomadFSM) applyDeregisterNode(reqType structs.MessageType, buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "deregister_node"}, time.Now()) var req structs.NodeDeregisterRequest if err := structs.Decode(buf, &req); err != nil { panic(fmt.Errorf("failed to decode request: %v", err)) } - if err := n.state.DeleteNode(index, []string{req.NodeID}); err != nil { + ctx := context.WithValue(context.Background(), state.CtxMsgType, reqType) + + if err := n.state.DeleteNodeCtx(ctx, index, []string{req.NodeID}); err != nil { n.logger.Error("DeleteNode failed", "error", err) return err } diff --git a/nomad/state/node_events.go b/nomad/state/node_events.go new file mode 100644 index 000000000000..b2889e3a3076 --- /dev/null +++ b/nomad/state/node_events.go @@ -0,0 +1,75 @@ +package state + +import ( + "fmt" + + "github.com/hashicorp/nomad/nomad/stream" + "github.com/hashicorp/nomad/nomad/structs" +) + +const ( + TopicNodeRegistration = "NodeRegistration" + TopicNodeDeregistration = "NodeDeregistration" +) + +type NodeRegistrationEvent struct { + Event *structs.NodeEvent + NodeStatus string +} + +type NodeDeregistrationEvent struct { + NodeID string +} + +// NodeRegisterEventFromChanges generates a NodeRegistrationEvent from a set +// of transaction changes. +func NodeRegisterEventFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { + var events []stream.Event + for _, change := range changes.Changes { + switch change.Table { + case "nodes": + after, ok := change.After.(*structs.Node) + if !ok { + return nil, fmt.Errorf("transaction change was not a Node") + } + + event := stream.Event{ + Topic: TopicNodeRegistration, + Index: changes.Index, + Key: after.ID, + Payload: &NodeRegistrationEvent{ + Event: after.Events[len(after.Events)-1], + NodeStatus: after.Status, + }, + } + events = append(events, event) + } + } + return events, nil +} + +// NodeDeregisterEventFromChanges generates a NodeDeregistrationEvent from a set +// of transaction changes. +func NodeDeregisterEventFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { + var events []stream.Event + for _, change := range changes.Changes { + switch change.Table { + case "nodes": + before, ok := change.Before.(*structs.Node) + if !ok { + return nil, fmt.Errorf("transaction change was not a Node") + } + + event := stream.Event{ + Topic: TopicNodeDeregistration, + Index: changes.Index, + Key: before.ID, + Payload: &NodeDeregistrationEvent{ + NodeID: before.ID, + }, + } + events = append(events, event) + } + } + return events, nil +} diff --git a/nomad/state/node_events_test.go b/nomad/state/node_events_test.go new file mode 100644 index 000000000000..10b9458a635a --- /dev/null +++ b/nomad/state/node_events_test.go @@ -0,0 +1,211 @@ +package state + +import ( + "testing" + + "github.com/hashicorp/nomad/nomad/mock" + "github.com/hashicorp/nomad/nomad/stream" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/stretchr/testify/require" +) + +func TestNodeRegisterEventFromChanges(t *testing.T) { + cases := []struct { + Name string + MsgType structs.MessageType + Setup func(s *StateStore, tx *txn) error + Mutate func(s *StateStore, tx *txn) error + WantEvents []stream.Event + WantErr bool + WantTopic string + }{ + { + MsgType: structs.NodeRegisterRequestType, + WantTopic: TopicNodeRegistration, + Name: "node registered", + Mutate: func(s *StateStore, tx *txn) error { + return upsertNodeTxn(tx, tx.Index, testNode()) + }, + WantEvents: []stream.Event{{ + Topic: TopicNodeRegistration, + Key: testNodeID(), + Index: 100, + Payload: &NodeRegistrationEvent{ + Event: &structs.NodeEvent{ + Message: "Node registered", + Subsystem: "Cluster", + }, + NodeStatus: structs.NodeStatusReady, + }, + }}, + WantErr: false, + }, + { + MsgType: structs.NodeRegisterRequestType, + WantTopic: TopicNodeRegistration, + Name: "node registered initializing", + Mutate: func(s *StateStore, tx *txn) error { + return upsertNodeTxn(tx, tx.Index, testNode(nodeNotReady)) + }, + WantEvents: []stream.Event{{ + Topic: TopicNodeRegistration, + Key: testNodeID(), + Index: 100, + Payload: &NodeRegistrationEvent{ + Event: &structs.NodeEvent{ + Message: "Node registered", + Subsystem: "Cluster", + }, + NodeStatus: structs.NodeStatusInit, + }, + }}, + WantErr: false, + }, + { + MsgType: structs.NodeDeregisterRequestType, + WantTopic: TopicNodeDeregistration, + Name: "node deregistered", + Setup: func(s *StateStore, tx *txn) error { + return upsertNodeTxn(tx, tx.Index, testNode()) + }, + Mutate: func(s *StateStore, tx *txn) error { + return deleteNodeTxn(tx, tx.Index, []string{testNodeID()}) + }, + WantEvents: []stream.Event{{ + Topic: TopicNodeDeregistration, + Key: testNodeID(), + Index: 100, + Payload: &NodeDeregistrationEvent{ + NodeID: testNodeID(), + }, + }}, + WantErr: false, + }, + { + MsgType: structs.NodeDeregisterRequestType, + WantTopic: TopicNodeDeregistration, + Name: "batch node deregistered", + Setup: func(s *StateStore, tx *txn) error { + require.NoError(t, upsertNodeTxn(tx, tx.Index, testNode())) + return upsertNodeTxn(tx, tx.Index, testNode(nodeIDTwo)) + }, + Mutate: func(s *StateStore, tx *txn) error { + return deleteNodeTxn(tx, tx.Index, []string{testNodeID(), testNodeIDTwo()}) + }, + WantEvents: []stream.Event{ + { + Topic: TopicNodeDeregistration, + Key: testNodeID(), + Index: 100, + Payload: &NodeDeregistrationEvent{ + NodeID: testNodeID(), + }, + }, + { + Topic: TopicNodeDeregistration, + Key: testNodeIDTwo(), + Index: 100, + Payload: &NodeDeregistrationEvent{ + NodeID: testNodeIDTwo(), + }, + }, + }, + WantErr: false, + }, + } + + for _, tc := range cases { + t.Run(tc.Name, func(t *testing.T) { + s := TestStateStoreCfg(t, TestStateStorePublisher(t)) + defer s.StopEventPublisher() + + if tc.Setup != nil { + // Bypass publish mechanism for setup + setupTx := s.db.WriteTxn(10) + require.NoError(t, tc.Setup(s, setupTx)) + setupTx.Txn.Commit() + } + + tx := s.db.WriteTxn(100) + require.NoError(t, tc.Mutate(s, tx)) + + changes := Changes{Changes: tx.Changes(), Index: 100, MsgType: tc.MsgType} + got, err := processDBChanges(tx, changes) + + if tc.WantErr { + require.Error(t, err) + return + } + require.NoError(t, err) + + require.Equal(t, len(tc.WantEvents), len(got)) + for idx, g := range got { + switch tc.MsgType { + case structs.NodeRegisterRequestType: + requireNodeRegistrationEventEqual(t, tc.WantEvents[idx], g) + case structs.NodeDeregisterRequestType: + requireNodeDeregistrationEventEqual(t, tc.WantEvents[idx], g) + } + } + }) + } +} + +func requireNodeRegistrationEventEqual(t *testing.T, want, got stream.Event) { + t.Helper() + + require.Equal(t, want.Index, got.Index) + require.Equal(t, want.Key, got.Key) + require.Equal(t, want.Topic, got.Topic) + + wantPayload := want.Payload.(*NodeRegistrationEvent) + gotPayload := got.Payload.(*NodeRegistrationEvent) + + // Check payload equality for the fields that we can easily control + require.Equal(t, wantPayload.NodeStatus, gotPayload.NodeStatus) + require.Equal(t, wantPayload.Event.Message, gotPayload.Event.Message) + require.Equal(t, wantPayload.Event.Subsystem, gotPayload.Event.Subsystem) +} + +func requireNodeDeregistrationEventEqual(t *testing.T, want, got stream.Event) { + t.Helper() + + require.Equal(t, want.Index, got.Index) + require.Equal(t, want.Key, got.Key) + require.Equal(t, want.Topic, got.Topic) + + wantPayload := want.Payload.(*NodeDeregistrationEvent) + gotPayload := got.Payload.(*NodeDeregistrationEvent) + + require.Equal(t, wantPayload, gotPayload) +} + +type nodeOpts func(n *structs.Node) + +func nodeNotReady(n *structs.Node) { + n.Status = structs.NodeStatusInit +} + +func nodeIDTwo(n *structs.Node) { + n.ID = testNodeIDTwo() +} + +func testNode(opts ...nodeOpts) *structs.Node { + n := mock.Node() + n.ID = testNodeID() + + n.SecretID = "ab9812d3-6a21-40d3-973d-d9d2174a23ee" + + for _, opt := range opts { + opt(n) + } + return n +} + +func testNodeID() string { + return "9d5741c1-3899-498a-98dd-eb3c05665863" +} + +func testNodeIDTwo() string { + return "694ff31d-8c59-4030-ac83-e15692560c8d" +} diff --git a/nomad/state/state_changes.go b/nomad/state/state_changes.go index a71056e0f9d0..f105f49b5f35 100644 --- a/nomad/state/state_changes.go +++ b/nomad/state/state_changes.go @@ -1,10 +1,16 @@ package state import ( + "context" "fmt" "github.com/hashicorp/go-memdb" "github.com/hashicorp/nomad/nomad/stream" + "github.com/hashicorp/nomad/nomad/structs" +) + +const ( + CtxMsgType = "type" ) // ReadTxn is implemented by memdb.Txn to perform read operations. @@ -21,6 +27,7 @@ type Changes struct { // Index is the latest index at the time these changes were committed. Index uint64 Changes memdb.Changes + MsgType structs.MessageType } // changeTrackerDB is a thin wrapper around memdb.DB which enables TrackChanges on @@ -81,6 +88,18 @@ func (c *changeTrackerDB) WriteTxn(idx uint64) *txn { return t } +// WriteTxnCtx is identical to WriteTxn but takes a ctx used for event sourcing +func (c *changeTrackerDB) WriteTxnCtx(ctx context.Context, idx uint64) *txn { + t := &txn{ + ctx: ctx, + Txn: c.db.Txn(true), + Index: idx, + publish: c.publish, + } + t.Txn.TrackChanges() + return t +} + func (c *changeTrackerDB) publish(changes Changes) error { readOnlyTx := c.db.Txn(false) defer readOnlyTx.Abort() @@ -113,6 +132,9 @@ func (c *changeTrackerDB) WriteTxnRestore() *txn { // error. Any errors from the callback would be lost, which would result in a // missing change event, even though the state store had changed. type txn struct { + // ctx is used to hold message type information from an FSM request + ctx context.Context + *memdb.Txn // Index in raft where the write is occurring. The value is zero for a // read-only, or WriteTxnRestore transaction. @@ -136,6 +158,7 @@ func (tx *txn) Commit() error { changes := Changes{ Index: tx.Index, Changes: tx.Txn.Changes(), + MsgType: tx.MsgType(), } if err := tx.publish(changes); err != nil { return err @@ -146,7 +169,35 @@ func (tx *txn) Commit() error { return nil } +// MsgType returns a MessageType from the txn's context. +// If the context is empty or the value isn't set IgnoreUnknownTypeFlag will +// be returned to signal that the MsgType is unknown. +func (tx *txn) MsgType() structs.MessageType { + if tx.ctx == nil { + return structs.IgnoreUnknownTypeFlag + } + + raw := tx.ctx.Value(CtxMsgType) + if raw == nil { + return structs.IgnoreUnknownTypeFlag + } + + msgType, ok := raw.(structs.MessageType) + if !ok { + return structs.IgnoreUnknownTypeFlag + } + return msgType +} + func processDBChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { - // TODO: add handlers here. + switch changes.MsgType { + case structs.IgnoreUnknownTypeFlag: + // unknown event type + return []stream.Event{}, nil + case structs.NodeRegisterRequestType: + return NodeRegisterEventFromChanges(tx, changes) + case structs.NodeDeregisterRequestType: + return NodeDeregisterEventFromChanges(tx, changes) + } return []stream.Event{}, nil } diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 4440571dd9e6..dca07ebf86d7 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -206,10 +206,16 @@ func (s *StateStore) AbandonCh() <-chan struct{} { // Abandon is used to signal that the given state store has been abandoned. // Calling this more than one time will panic. func (s *StateStore) Abandon() { - s.stopEventPublisher() + s.StopEventPublisher() close(s.abandonCh) } +// StopStopEventPublisher calls the cancel func for the state stores event +// publisher. It should be called during server shutdown. +func (s *StateStore) StopEventPublisher() { + s.stopEventPublisher() +} + // QueryFn is the definition of a function that can be used to implement a basic // blocking query against the state store. type QueryFn func(memdb.WatchSet, *StateStore) (resp interface{}, index uint64, err error) @@ -740,6 +746,21 @@ func (s *StateStore) ScalingEventsByJob(ws memdb.WatchSet, namespace, jobID stri return nil, 0, nil } +// UpsertNodeCtx is used to register a node or update a node definition +// This is assumed to be triggered by the client, so we retain the value +// of drain/eligibility which is set by the scheduler. +func (s *StateStore) UpsertNodeCtx(ctx context.Context, index uint64, node *structs.Node) error { + txn := s.db.WriteTxnCtx(ctx, index) + defer txn.Abort() + + err := upsertNodeTxn(txn, index, node) + if err != nil { + return nil + } + txn.Commit() + return nil +} + // UpsertNode is used to register a node or update a node definition // This is assumed to be triggered by the client, so we retain the value // of drain/eligibility which is set by the scheduler. @@ -747,6 +768,15 @@ func (s *StateStore) UpsertNode(index uint64, node *structs.Node) error { txn := s.db.WriteTxn(index) defer txn.Abort() + err := upsertNodeTxn(txn, index, node) + if err != nil { + return nil + } + txn.Commit() + return nil +} + +func upsertNodeTxn(txn *txn, index uint64, node *structs.Node) error { // Check if the node already exists existing, err := txn.First("nodes", "id", node.ID) if err != nil { @@ -795,19 +825,40 @@ func (s *StateStore) UpsertNode(index uint64, node *structs.Node) error { return fmt.Errorf("csi plugin update failed: %v", err) } + return nil +} + +// DeleteNode deregisters a batch of nodes +func (s *StateStore) DeleteNodeCtx(ctx context.Context, index uint64, nodes []string) error { + txn := s.db.WriteTxnCtx(ctx, index) + defer txn.Abort() + + err := deleteNodeTxn(txn, index, nodes) + if err != nil { + return nil + } txn.Commit() return nil } // DeleteNode deregisters a batch of nodes func (s *StateStore) DeleteNode(index uint64, nodes []string) error { + txn := s.db.WriteTxn(index) + defer txn.Abort() + + err := deleteNodeTxn(txn, index, nodes) + if err != nil { + return nil + } + txn.Commit() + return nil +} + +func deleteNodeTxn(txn *txn, index uint64, nodes []string) error { if len(nodes) == 0 { return fmt.Errorf("node ids missing") } - txn := s.db.WriteTxn(index) - defer txn.Abort() - for _, nodeID := range nodes { existing, err := txn.First("nodes", "id", nodeID) if err != nil { @@ -832,7 +883,6 @@ func (s *StateStore) DeleteNode(index uint64, nodes []string) error { return fmt.Errorf("index update failed: %v", err) } - txn.Commit() return nil } diff --git a/nomad/state/testing.go b/nomad/state/testing.go index de069e4098b4..6aa9039f4d79 100644 --- a/nomad/state/testing.go +++ b/nomad/state/testing.go @@ -24,6 +24,25 @@ func TestStateStore(t testing.T) *StateStore { return state } +func TestStateStorePublisher(t testing.T) *StateStoreConfig { + return &StateStoreConfig{ + Logger: testlog.HCLogger(t), + Region: "global", + EnablePublisher: true, + } +} +func TestStateStoreCfg(t testing.T, cfg *StateStoreConfig) *StateStore { + state, err := NewStateStore(cfg) + if err != nil { + t.Fatalf("err: %v", err) + } + + if state == nil { + t.Fatalf("missing state") + } + return state +} + // CreateTestCSIPlugin is a helper that generates the node + fingerprint results necessary // to create a CSIPlugin by directly inserting into the state store. The plugin requires a // controller. From b825ba3bf11e7dcba7f6326d2863ec8390b074dc Mon Sep 17 00:00:00 2001 From: Drew Bailey <2614075+drewbailey@users.noreply.github.com> Date: Mon, 28 Sep 2020 10:13:10 -0400 Subject: [PATCH 05/19] Event Stream API/RPC (#8947) This Commit adds an /v1/events/stream endpoint to stream events from. The stream framer has been updated to include a SendFull method which does not fragment the data between multiple frames. This essentially treats the stream framer as a envelope to adhere to the stream framer interface in the UI. If the `encode` query parameter is omitted events will be streamed as newline delimted JSON. --- command/agent/event_endpoint.go | 169 +++++++++++++++ command/agent/event_endpoint_test.go | 148 ++++++++++++++ command/agent/http.go | 2 + nomad/config.go | 5 + nomad/event_endpoint.go | 213 +++++++++++++++++++ nomad/event_endpoint_test.go | 295 +++++++++++++++++++++++++++ nomad/fsm.go | 8 +- nomad/server.go | 15 +- nomad/state/state_changes.go | 16 +- nomad/state/state_store.go | 13 +- nomad/stream/event_buffer.go | 2 +- nomad/stream/event_publisher.go | 13 +- nomad/stream/ndjson.go | 114 +++++++++++ nomad/stream/ndjson_test.go | 72 +++++++ nomad/stream/subscription.go | 26 ++- nomad/stream/subscription_test.go | 16 ++ nomad/structs/structs.go | 32 +++ nomad/testing.go | 1 + 18 files changed, 1129 insertions(+), 31 deletions(-) create mode 100644 command/agent/event_endpoint.go create mode 100644 command/agent/event_endpoint_test.go create mode 100644 nomad/event_endpoint.go create mode 100644 nomad/event_endpoint_test.go create mode 100644 nomad/stream/ndjson.go create mode 100644 nomad/stream/ndjson_test.go diff --git a/command/agent/event_endpoint.go b/command/agent/event_endpoint.go new file mode 100644 index 000000000000..5684d9c14e86 --- /dev/null +++ b/command/agent/event_endpoint.go @@ -0,0 +1,169 @@ +package agent + +import ( + "bytes" + "context" + "fmt" + "io" + "net" + "net/http" + "net/url" + "strconv" + "strings" + + "github.com/docker/docker/pkg/ioutils" + "github.com/hashicorp/go-msgpack/codec" + "github.com/hashicorp/nomad/nomad/stream" + "github.com/hashicorp/nomad/nomad/structs" +) + +func (s *HTTPServer) EventStream(resp http.ResponseWriter, req *http.Request) (interface{}, error) { + query := req.URL.Query() + + indexStr := query.Get("index") + if indexStr == "" { + indexStr = "0" + } + index, err := strconv.Atoi(indexStr) + if err != nil { + return nil, CodedError(400, fmt.Sprintf("Unable to parse index: %v", err)) + } + + topics, err := parseEventTopics(query) + if err != nil { + return nil, CodedError(400, fmt.Sprintf("Invalid topic query: %v", err)) + } + + args := &structs.EventStreamRequest{ + Topics: topics, + Index: index, + } + resp.Header().Set("Content-Type", "application/json") + resp.Header().Set("Cache-Control", "no-cache") + + s.parse(resp, req, &args.QueryOptions.Region, &args.QueryOptions) + + // Make the RPC + var handler structs.StreamingRpcHandler + var handlerErr error + + if server := s.agent.Server(); server != nil { + handler, handlerErr = server.StreamingRpcHandler("Event.Stream") + } else if client := s.agent.Client(); client != nil { + handler, handlerErr = client.RemoteStreamingRpcHandler("Event.Stream") + } else { + handlerErr = fmt.Errorf("misconfigured connection") + } + + if handlerErr != nil { + return nil, CodedError(500, handlerErr.Error()) + } + + httpPipe, handlerPipe := net.Pipe() + decoder := codec.NewDecoder(httpPipe, structs.MsgpackHandle) + encoder := codec.NewEncoder(httpPipe, structs.MsgpackHandle) + + // Create a goroutine that closes the pipe if the connection closes + ctx, cancel := context.WithCancel(req.Context()) + defer cancel() + go func() { + <-ctx.Done() + httpPipe.Close() + }() + + // Create an output that gets flushed on every write + output := ioutils.NewWriteFlusher(resp) + + // create an error channel to handle errors + errCh := make(chan HTTPCodedError, 2) + + go func() { + defer cancel() + + // Send the request + if err := encoder.Encode(args); err != nil { + errCh <- CodedError(500, err.Error()) + return + } + + for { + select { + case <-ctx.Done(): + errCh <- nil + return + default: + } + + // Decode the response + var res structs.EventStreamWrapper + if err := decoder.Decode(&res); err != nil { + if err == io.EOF || err == io.ErrClosedPipe { + return + } + errCh <- CodedError(500, err.Error()) + return + } + decoder.Reset(httpPipe) + + if err := res.Error; err != nil { + if err.Code != nil { + errCh <- CodedError(int(*err.Code), err.Error()) + return + } + } + + // Flush json entry to response + if _, err := io.Copy(output, bytes.NewReader(res.Event.Data)); err != nil { + errCh <- CodedError(500, err.Error()) + return + } + } + }() + + // invoke handler + handler(handlerPipe) + cancel() + codedErr := <-errCh + + if codedErr != nil && + (codedErr == io.EOF || + strings.Contains(codedErr.Error(), io.ErrClosedPipe.Error())) { + codedErr = nil + } + + return nil, codedErr +} + +func parseEventTopics(query url.Values) (map[stream.Topic][]string, error) { + raw, ok := query["topic"] + if !ok { + return allTopics(), nil + } + topics := make(map[stream.Topic][]string) + + for _, topic := range raw { + k, v, err := parseTopic(topic) + if err != nil { + return nil, fmt.Errorf("error parsing topics: %w", err) + } + + if topics[stream.Topic(k)] == nil { + topics[stream.Topic(k)] = []string{v} + } else { + topics[stream.Topic(k)] = append(topics[stream.Topic(k)], v) + } + } + return topics, nil +} + +func parseTopic(topic string) (string, string, error) { + parts := strings.Split(topic, ":") + if len(parts) != 2 { + return "", "", fmt.Errorf("Invalid key value pair for topic, topic: %s", topic) + } + return parts[0], parts[1], nil +} + +func allTopics() map[stream.Topic][]string { + return map[stream.Topic][]string{"*": {"*"}} +} diff --git a/command/agent/event_endpoint_test.go b/command/agent/event_endpoint_test.go new file mode 100644 index 000000000000..88987e3d9de5 --- /dev/null +++ b/command/agent/event_endpoint_test.go @@ -0,0 +1,148 @@ +package agent + +import ( + "context" + "fmt" + "net/http" + "net/http/httptest" + "net/url" + "strings" + "testing" + "time" + + "github.com/hashicorp/nomad/nomad/stream" + "github.com/hashicorp/nomad/testutil" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" +) + +type testEvent struct { + ID string +} + +func TestEventStream(t *testing.T) { + t.Parallel() + + httpTest(t, nil, func(s *TestAgent) { + ctx, cancel := context.WithCancel(context.Background()) + req, err := http.NewRequestWithContext(ctx, "GET", "/v1/event/stream", nil) + require.Nil(t, err) + resp := httptest.NewRecorder() + + respErrCh := make(chan error) + go func() { + _, err = s.Server.EventStream(resp, req) + respErrCh <- err + assert.NoError(t, err) + }() + + pub, err := s.Agent.server.State().EventPublisher() + require.NoError(t, err) + pub.Publish(100, []stream.Event{{Payload: testEvent{ID: "123"}}}) + + testutil.WaitForResult(func() (bool, error) { + got := resp.Body.String() + want := `{"ID":"123"}` + if strings.Contains(got, want) { + return true, nil + } + + return false, fmt.Errorf("missing expected json, got: %v, want: %v", got, want) + }, func(err error) { + cancel() + require.Fail(t, err.Error()) + }) + + // wait for response to close to prevent race between subscription + // shutdown and server shutdown returning subscription closed by server err + // resp.Close() + cancel() + select { + case err := <-respErrCh: + require.Nil(t, err) + case <-time.After(1 * time.Second): + require.Fail(t, "waiting for request cancellation") + } + }) +} + +func TestEventStream_QueryParse(t *testing.T) { + t.Parallel() + + cases := []struct { + desc string + query string + want map[stream.Topic][]string + wantErr bool + }{ + { + desc: "all topics and keys specified", + query: "?topic=*:*", + want: map[stream.Topic][]string{ + "*": []string{"*"}, + }, + }, + { + desc: "all topics and keys inferred", + query: "", + want: map[stream.Topic][]string{ + "*": []string{"*"}, + }, + }, + { + desc: "invalid key value formatting", + query: "?topic=NodeDrain:*:*", + wantErr: true, + }, + { + desc: "invalid key value formatting no value", + query: "?topic=NodeDrain", + wantErr: true, + }, + { + desc: "single topic and key", + query: "?topic=NodeDrain:*", + want: map[stream.Topic][]string{ + "NodeDrain": []string{"*"}, + }, + }, + { + desc: "single topic multiple keys", + query: "?topic=NodeDrain:*&topic=NodeDrain:3caace09-f1f4-4d23-b37a-9ab5eb75069d", + want: map[stream.Topic][]string{ + "NodeDrain": []string{ + "*", + "3caace09-f1f4-4d23-b37a-9ab5eb75069d", + }, + }, + }, + { + desc: "multiple topics", + query: "?topic=NodeRegister:*&topic=NodeDrain:3caace09-f1f4-4d23-b37a-9ab5eb75069d", + want: map[stream.Topic][]string{ + "NodeDrain": []string{ + "3caace09-f1f4-4d23-b37a-9ab5eb75069d", + }, + "NodeRegister": []string{ + "*", + }, + }, + }, + } + + for _, tc := range cases { + t.Run(tc.desc, func(t *testing.T) { + raw := fmt.Sprintf("http://localhost:80/v1/events%s", tc.query) + req, err := url.Parse(raw) + require.NoError(t, err) + + got, err := parseEventTopics(req.Query()) + if tc.wantErr { + require.Error(t, err) + return + } + require.NoError(t, err) + require.Equal(t, tc.want, got) + }) + } +} diff --git a/command/agent/http.go b/command/agent/http.go index 87b51c0aca89..777ff5974e9b 100644 --- a/command/agent/http.go +++ b/command/agent/http.go @@ -326,6 +326,8 @@ func (s *HTTPServer) registerHandlers(enableDebug bool) { s.mux.HandleFunc("/v1/operator/scheduler/configuration", s.wrap(s.OperatorSchedulerConfiguration)) + s.mux.HandleFunc("/v1/event/stream", s.wrap(s.EventStream)) + if uiEnabled { s.mux.Handle("/ui/", http.StripPrefix("/ui/", s.handleUI(http.FileServer(&UIAssetWrapper{FileSystem: assetFS()})))) } else { diff --git a/nomad/config.go b/nomad/config.go index 764ebb1ed894..081d83e02601 100644 --- a/nomad/config.go +++ b/nomad/config.go @@ -78,6 +78,10 @@ type Config struct { // in the absence of ACLs EnableDebug bool + // EnableEventPublisher is used to enable or disable the state stores + // event publishing + EnableEventPublisher bool + // LogOutput is the location to write logs to. If this is not set, // logs will go to stderr. LogOutput io.Writer @@ -413,6 +417,7 @@ func DefaultConfig() *Config { ReplicationBackoff: 30 * time.Second, SentinelGCInterval: 30 * time.Second, LicenseConfig: &LicenseConfig{}, + EnableEventPublisher: true, AutopilotConfig: &structs.AutopilotConfig{ CleanupDeadServers: true, LastContactThreshold: 200 * time.Millisecond, diff --git a/nomad/event_endpoint.go b/nomad/event_endpoint.go new file mode 100644 index 000000000000..5661200d43f7 --- /dev/null +++ b/nomad/event_endpoint.go @@ -0,0 +1,213 @@ +package nomad + +import ( + "context" + "io" + "time" + + "github.com/hashicorp/go-msgpack/codec" + "github.com/hashicorp/nomad/helper" + "github.com/hashicorp/nomad/helper/uuid" + "github.com/hashicorp/nomad/nomad/stream" + "github.com/hashicorp/nomad/nomad/structs" +) + +type Event struct { + srv *Server +} + +func (e *Event) register() { + e.srv.streamingRpcs.Register("Event.Stream", e.stream) +} + +func (e *Event) stream(conn io.ReadWriteCloser) { + defer conn.Close() + + var args structs.EventStreamRequest + decoder := codec.NewDecoder(conn, structs.MsgpackHandle) + encoder := codec.NewEncoder(conn, structs.MsgpackHandle) + + if err := decoder.Decode(&args); err != nil { + handleJsonResultError(err, helper.Int64ToPtr(500), encoder) + return + } + + // forward to appropriate region + if args.Region != e.srv.config.Region { + err := e.forwardStreamingRPC(args.Region, "Event.Stream", args, conn) + if err != nil { + handleJsonResultError(err, helper.Int64ToPtr(500), encoder) + } + return + } + + // ACL check + // TODO(drew) ACL checks need to be per topic + // All Events Management + // System Events Management + // Node Events NamespaceCapabilityReadEvents + // Job/Alloc Events NamespaceCapabilityReadEvents + if aclObj, err := e.srv.ResolveToken(args.AuthToken); err != nil { + handleJsonResultError(err, nil, encoder) + return + } else if aclObj != nil && !aclObj.IsManagement() { + handleJsonResultError(structs.ErrPermissionDenied, helper.Int64ToPtr(403), encoder) + return + } + + // TODO(drew) handle streams without ACLS + reqToken := args.AuthToken + if reqToken == "" { + // generate a random request token + reqToken = uuid.Generate() + } + subReq := &stream.SubscribeRequest{ + Token: reqToken, + Topics: args.Topics, + Index: uint64(args.Index), + } + publisher, err := e.srv.State().EventPublisher() + if err != nil { + handleJsonResultError(err, helper.Int64ToPtr(500), encoder) + return + } + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + // start subscription to publisher + subscription, err := publisher.Subscribe(subReq) + if err != nil { + handleJsonResultError(err, helper.Int64ToPtr(500), encoder) + return + } + defer subscription.Unsubscribe() + + ndJsonCh := make(chan *stream.NDJson) + errCh := make(chan error) + + jsonStream := stream.NewNDJsonStream(ndJsonCh, 30*time.Second) + jsonStream.Run(ctx) + + // goroutine to detect remote side closing + go func() { + if _, err := conn.Read(nil); err != nil { + // One end of the pipe explicitly closed, exit + cancel() + return + } + select { + case <-errCh: + case <-ctx.Done(): + return + } + }() + + go func() { + defer cancel() + LOOP: + for { + events, err := subscription.Next(ctx) + if err != nil { + select { + case errCh <- err: + case <-ctx.Done(): + } + break LOOP + } + + // Continue if there are no events + if events == nil { + continue + } + + // Send each event as its own frame + for _, e := range events { + if err := jsonStream.Send(e); err != nil { + select { + case errCh <- err: + case <-ctx.Done(): + } + break LOOP + } + } + } + }() + + var streamErr error +OUTER: + for { + select { + case streamErr = <-errCh: + break OUTER + case <-ctx.Done(): + break OUTER + case eventJSON, ok := <-ndJsonCh: + // check if ndjson may have been closed when an error occurred, + // check once more for an error. + if !ok { + select { + case streamErr = <-errCh: + // There was a pending error + default: + } + break OUTER + } + + var resp structs.EventStreamWrapper + resp.Event = eventJSON + + if err := encoder.Encode(resp); err != nil { + streamErr = err + break OUTER + } + encoder.Reset(conn) + } + + } + + if streamErr != nil { + handleJsonResultError(streamErr, helper.Int64ToPtr(500), encoder) + return + } + +} + +func (e *Event) forwardStreamingRPC(region string, method string, args interface{}, in io.ReadWriteCloser) error { + server, err := e.srv.findRegionServer(region) + if err != nil { + return err + } + + return e.forwardStreamingRPCToServer(server, method, args, in) +} + +func (e *Event) forwardStreamingRPCToServer(server *serverParts, method string, args interface{}, in io.ReadWriteCloser) error { + srvConn, err := e.srv.streamingRpc(server, method) + if err != nil { + return err + } + defer srvConn.Close() + + outEncoder := codec.NewEncoder(srvConn, structs.MsgpackHandle) + if err := outEncoder.Encode(args); err != nil { + return err + } + + structs.Bridge(in, srvConn) + return nil +} + +// handleJsonResultError is a helper for sending an error with a potential +// error code. The transmission of the error is ignored if the error has been +// generated by the closing of the underlying transport. +func handleJsonResultError(err error, code *int64, encoder *codec.Encoder) { + // Nothing to do as the conn is closed + if err == io.EOF { + return + } + + encoder.Encode(&structs.EventStreamWrapper{ + Error: structs.NewRpcError(err, code), + }) +} diff --git a/nomad/event_endpoint_test.go b/nomad/event_endpoint_test.go new file mode 100644 index 000000000000..9e4256192614 --- /dev/null +++ b/nomad/event_endpoint_test.go @@ -0,0 +1,295 @@ +package nomad + +import ( + "encoding/json" + "fmt" + "io" + "net" + "strings" + "testing" + "time" + + "github.com/hashicorp/go-msgpack/codec" + "github.com/hashicorp/nomad/nomad/mock" + "github.com/hashicorp/nomad/nomad/stream" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/testutil" + "github.com/mitchellh/mapstructure" + "github.com/stretchr/testify/require" +) + +func TestEventStream(t *testing.T) { + t.Parallel() + + s1, cleanupS1 := TestServer(t, func(c *Config) { + c.EnableEventPublisher = true + }) + defer cleanupS1() + + // Create request for all topics and keys + req := structs.EventStreamRequest{ + Topics: map[stream.Topic][]string{"*": []string{"*"}}, + QueryOptions: structs.QueryOptions{ + Region: s1.Region(), + }, + } + + handler, err := s1.StreamingRpcHandler("Event.Stream") + require.Nil(t, err) + + p1, p2 := net.Pipe() + defer p1.Close() + defer p2.Close() + + errCh := make(chan error) + streamMsg := make(chan *structs.EventStreamWrapper) + + // invoke handler + go handler(p2) + + // send request + go func() { + decoder := codec.NewDecoder(p1, structs.MsgpackHandle) + for { + var msg structs.EventStreamWrapper + if err := decoder.Decode(&msg); err != nil { + if err == io.EOF || strings.Contains(err.Error(), "closed") { + return + } + errCh <- fmt.Errorf("error decoding: %w", err) + } + + streamMsg <- &msg + } + }() + + // retrieve publisher for server, send event + publisher, err := s1.State().EventPublisher() + require.NoError(t, err) + + node := mock.Node() + publisher.Publish(uint64(1), []stream.Event{{Topic: "test", Payload: node}}) + + encoder := codec.NewEncoder(p1, structs.MsgpackHandle) + require.Nil(t, encoder.Encode(req)) + + timeout := time.After(3 * time.Second) +OUTER: + for { + select { + case <-timeout: + t.Fatal("timeout waiting for event stream") + case err := <-errCh: + t.Fatal(err) + case msg := <-streamMsg: + if msg.Error != nil { + t.Fatalf("Got error: %v", msg.Error.Error()) + } + + // ignore heartbeat + if msg.Event == stream.NDJsonHeartbeat { + continue + } + + var event stream.Event + err = json.Unmarshal(msg.Event.Data, &event) + require.NoError(t, err) + + // decode fully to ensure we received expected out + var out structs.Node + cfg := &mapstructure.DecoderConfig{ + Metadata: nil, + Result: &out, + } + dec, err := mapstructure.NewDecoder(cfg) + dec.Decode(event.Payload) + require.NoError(t, err) + require.Equal(t, node.ID, out.ID) + break OUTER + } + } +} + +// TestEventStream_StreamErr asserts an error is returned when an event publisher +// closes its subscriptions +func TestEventStream_StreamErr(t *testing.T) { + t.Parallel() + + s1, cleanupS1 := TestServer(t, func(c *Config) { + c.EnableEventPublisher = true + }) + defer cleanupS1() + + testutil.WaitForLeader(t, s1.RPC) + + req := structs.EventStreamRequest{ + Topics: map[stream.Topic][]string{"*": []string{"*"}}, + QueryOptions: structs.QueryOptions{ + Region: s1.Region(), + }, + } + + handler, err := s1.StreamingRpcHandler("Event.Stream") + require.Nil(t, err) + + p1, p2 := net.Pipe() + defer p1.Close() + defer p2.Close() + + errCh := make(chan error) + streamMsg := make(chan *structs.EventStreamWrapper) + + go handler(p2) + + go func() { + decoder := codec.NewDecoder(p1, structs.MsgpackHandle) + for { + var msg structs.EventStreamWrapper + if err := decoder.Decode(&msg); err != nil { + if err == io.EOF || strings.Contains(err.Error(), "closed") { + return + } + errCh <- fmt.Errorf("error decoding: %w", err) + } + + streamMsg <- &msg + } + }() + + publisher, err := s1.State().EventPublisher() + require.NoError(t, err) + + node := mock.Node() + publisher.Publish(uint64(1), []stream.Event{{Topic: "test", Payload: node}}) + + // send req + encoder := codec.NewEncoder(p1, structs.MsgpackHandle) + require.Nil(t, encoder.Encode(req)) + + // stop the publisher to force an error on subscription side + s1.State().StopEventPublisher() + + timeout := time.After(5 * time.Second) +OUTER: + for { + select { + case <-timeout: + t.Fatal("timeout waiting for event stream") + case err := <-errCh: + t.Fatal(err) + case msg := <-streamMsg: + if msg.Error == nil { + // race between error and receiving an event + // continue trying for error + continue + } + require.NotNil(t, msg.Error) + require.Contains(t, msg.Error.Error(), "subscription closed by server") + break OUTER + } + } +} + +// TestEventStream_RegionForward tests event streaming from one server +// to another in a different region +func TestEventStream_RegionForward(t *testing.T) { + t.Parallel() + + s1, cleanupS1 := TestServer(t, func(c *Config) { + c.EnableEventPublisher = true + }) + defer cleanupS1() + + s2, cleanupS2 := TestServer(t, func(c *Config) { + c.EnableEventPublisher = true + c.Region = "foo" + }) + defer cleanupS2() + + TestJoin(t, s1, s2) + + // Create request targed for region foo + req := structs.EventStreamRequest{ + Topics: map[stream.Topic][]string{"*": []string{"*"}}, + QueryOptions: structs.QueryOptions{ + Region: "foo", + }, + } + + // Query s1 handler + handler, err := s1.StreamingRpcHandler("Event.Stream") + require.Nil(t, err) + + p1, p2 := net.Pipe() + defer p1.Close() + defer p2.Close() + + errCh := make(chan error) + streamMsg := make(chan *structs.EventStreamWrapper) + + go handler(p2) + + go func() { + decoder := codec.NewDecoder(p1, structs.MsgpackHandle) + for { + var msg structs.EventStreamWrapper + if err := decoder.Decode(&msg); err != nil { + if err == io.EOF || strings.Contains(err.Error(), "closed") { + return + } + errCh <- fmt.Errorf("error decoding: %w", err) + } + + streamMsg <- &msg + } + }() + + // publish with server 2 + publisher, err := s2.State().EventPublisher() + require.NoError(t, err) + + node := mock.Node() + publisher.Publish(uint64(1), []stream.Event{{Topic: "test", Payload: node}}) + + // send req + encoder := codec.NewEncoder(p1, structs.MsgpackHandle) + require.Nil(t, encoder.Encode(req)) + + timeout := time.After(3 * time.Second) +OUTER: + for { + select { + case <-timeout: + t.Fatal("timeout waiting for event stream") + case err := <-errCh: + t.Fatal(err) + case msg := <-streamMsg: + if msg.Error != nil { + t.Fatalf("Got error: %v", msg.Error.Error()) + } + + if msg.Event == stream.NDJsonHeartbeat { + continue + } + + var event stream.Event + err = json.Unmarshal(msg.Event.Data, &event) + require.NoError(t, err) + + var out structs.Node + cfg := &mapstructure.DecoderConfig{ + Metadata: nil, + Result: &out, + } + dec, err := mapstructure.NewDecoder(cfg) + dec.Decode(event.Payload) + require.NoError(t, err) + require.Equal(t, node.ID, out.ID) + break OUTER + } + } +} + +// TODO(drew) acl test +func TestEventStream_ACL(t *testing.T) { +} diff --git a/nomad/fsm.go b/nomad/fsm.go index aa525da0a176..d18a1d43b734 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -126,14 +126,17 @@ type FSMConfig struct { // Region is the region of the server embedding the FSM Region string + + EnableEventPublisher bool } // NewFSMPath is used to construct a new FSM with a blank state func NewFSM(config *FSMConfig) (*nomadFSM, error) { // Create a state store sconfig := &state.StateStoreConfig{ - Logger: config.Logger, - Region: config.Region, + Logger: config.Logger, + Region: config.Region, + EnablePublisher: config.EnableEventPublisher, } state, err := state.NewStateStore(sconfig) if err != nil { @@ -163,6 +166,7 @@ func NewFSM(config *FSMConfig) (*nomadFSM, error) { // Close is used to cleanup resources associated with the FSM func (n *nomadFSM) Close() error { + n.state.StopEventPublisher() return nil } diff --git a/nomad/server.go b/nomad/server.go index d0977c1e7b4a..dea6e29c1e6c 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -275,6 +275,7 @@ type endpoints struct { ACL *ACL Scaling *Scaling Enterprise *EnterpriseEndpoints + Event *Event // Client endpoints ClientStats *ClientStats @@ -1162,6 +1163,9 @@ func (s *Server) setupRpcServer(server *rpc.Server, ctx *RPCContext) { s.staticEndpoints.Agent = &Agent{srv: s} s.staticEndpoints.Agent.register() + + s.staticEndpoints.Event = &Event{srv: s} + s.staticEndpoints.Event.register() } // Register the static handlers @@ -1207,11 +1211,12 @@ func (s *Server) setupRaft() error { // Create the FSM fsmConfig := &FSMConfig{ - EvalBroker: s.evalBroker, - Periodic: s.periodicDispatcher, - Blocked: s.blockedEvals, - Logger: s.logger, - Region: s.Region(), + EvalBroker: s.evalBroker, + Periodic: s.periodicDispatcher, + Blocked: s.blockedEvals, + Logger: s.logger, + Region: s.Region(), + EnableEventPublisher: s.config.EnableEventPublisher, } var err error s.fsm, err = NewFSM(fsmConfig) diff --git a/nomad/state/state_changes.go b/nomad/state/state_changes.go index f105f49b5f35..be80c53c6bb9 100644 --- a/nomad/state/state_changes.go +++ b/nomad/state/state_changes.go @@ -32,14 +32,14 @@ type Changes struct { // changeTrackerDB is a thin wrapper around memdb.DB which enables TrackChanges on // all write transactions. When the transaction is committed the changes are -// sent to the eventPublisher which will create and emit change events. +// sent to the EventPublisher which will create and emit change events. type changeTrackerDB struct { db *memdb.MemDB - publisher eventPublisher + publisher *stream.EventPublisher processChanges func(ReadTxn, Changes) ([]stream.Event, error) } -func NewChangeTrackerDB(db *memdb.MemDB, publisher eventPublisher, changesFn changeProcessor) *changeTrackerDB { +func NewChangeTrackerDB(db *memdb.MemDB, publisher *stream.EventPublisher, changesFn changeProcessor) *changeTrackerDB { return &changeTrackerDB{ db: db, publisher: publisher, @@ -49,15 +49,7 @@ func NewChangeTrackerDB(db *memdb.MemDB, publisher eventPublisher, changesFn cha type changeProcessor func(ReadTxn, Changes) ([]stream.Event, error) -type eventPublisher interface { - Publish(index uint64, events []stream.Event) -} - -// noOpPublisher satisfies the eventPublisher interface and does nothing -type noOpPublisher struct{} - -func (n *noOpPublisher) Publish(index uint64, events []stream.Event) {} -func noOpProcessChanges(ReadTxn, Changes) ([]stream.Event, error) { return []stream.Event{}, nil } +func noOpProcessChanges(ReadTxn, Changes) ([]stream.Event, error) { return []stream.Event{}, nil } // ReadTxn returns a read-only transaction which behaves exactly the same as // memdb.Txn diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index dca07ebf86d7..b1902eb55f07 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -93,10 +93,11 @@ func NewStateStore(config *StateStoreConfig) (*StateStore, error) { publisher := stream.NewEventPublisher(ctx, stream.EventPublisherCfg{ EventBufferTTL: 1 * time.Hour, EventBufferSize: 250, + Logger: config.Logger, }) s.db = NewChangeTrackerDB(db, publisher, processDBChanges) } else { - s.db = NewChangeTrackerDB(db, &noOpPublisher{}, processDBChanges) + s.db = NewChangeTrackerDB(db, nil, noOpProcessChanges) } // Initialize the state store with required enterprise objects @@ -107,6 +108,13 @@ func NewStateStore(config *StateStoreConfig) (*StateStore, error) { return s, nil } +func (s *StateStore) EventPublisher() (*stream.EventPublisher, error) { + if s.db.publisher == nil { + return nil, fmt.Errorf("EventPublisher not configured") + } + return s.db.publisher, nil +} + // Config returns the state store configuration. func (s *StateStore) Config() *StateStoreConfig { return s.config @@ -123,7 +131,8 @@ func (s *StateStore) Snapshot() (*StateSnapshot, error) { config: s.config, } - store.db = NewChangeTrackerDB(memDBSnap, &noOpPublisher{}, noOpProcessChanges) + // Create a new change tracker DB that does not publish or track changes + store.db = NewChangeTrackerDB(memDBSnap, nil, noOpProcessChanges) snap := &StateSnapshot{ StateStore: store, diff --git a/nomad/stream/event_buffer.go b/nomad/stream/event_buffer.go index 127ca5b9fdab..145776225a2c 100644 --- a/nomad/stream/event_buffer.go +++ b/nomad/stream/event_buffer.go @@ -256,7 +256,7 @@ func (i *bufferItem) Next(ctx context.Context, forceClose <-chan struct{}) (*buf // state change (chan nil) as that's not threadsafe but detecting close is. select { case <-ctx.Done(): - return nil, ctx.Err() + return nil, fmt.Errorf("waiting for next event: %w", ctx.Err()) case <-forceClose: return nil, fmt.Errorf("subscription closed") case <-i.link.ch: diff --git a/nomad/stream/event_publisher.go b/nomad/stream/event_publisher.go index 7a0d78fe24cb..aedc4bc101b5 100644 --- a/nomad/stream/event_publisher.go +++ b/nomad/stream/event_publisher.go @@ -15,6 +15,7 @@ const ( type EventPublisherCfg struct { EventBufferSize int64 EventBufferTTL time.Duration + Logger hclog.Logger } type EventPublisher struct { @@ -56,8 +57,13 @@ func NewEventPublisher(ctx context.Context, cfg EventPublisherCfg) *EventPublish cfg.EventBufferTTL = 1 * time.Hour } + if cfg.Logger == nil { + cfg.Logger = hclog.NewNullLogger() + } + buffer := newEventBuffer(cfg.EventBufferSize, cfg.EventBufferTTL) e := &EventPublisher{ + logger: cfg.Logger.Named("event_publisher"), eventBuf: buffer, publishCh: make(chan changeEvents), subscriptions: &subscriptions{ @@ -95,7 +101,12 @@ func (e *EventPublisher) Subscribe(req *SubscribeRequest) (*Subscription, error) e.logger.Warn("requested index no longer in buffer", "requsted", int(req.Index), "closest", int(head.Index)) } - sub := newSubscription(req, head, func() {}) + // Empty head so that calling Next on sub + start := newBufferItem(req.Index, []Event{}) + start.link.next.Store(head) + close(start.link.ch) + + sub := newSubscription(req, start, e.subscriptions.unsubscribe(req)) e.subscriptions.add(req, sub) return sub, nil diff --git a/nomad/stream/ndjson.go b/nomad/stream/ndjson.go new file mode 100644 index 000000000000..cb5195e54ef0 --- /dev/null +++ b/nomad/stream/ndjson.go @@ -0,0 +1,114 @@ +package stream + +import ( + "bytes" + "context" + "encoding/json" + "fmt" + "sync" + "time" +) + +var ( + // NDJsonHeartbeat is the NDJson to send as a heartbeat + // Avoids creating many heartbeat instances + NDJsonHeartbeat = &NDJson{Data: []byte("{}\n")} +) + +// NDJsonStream is used to send new line delimited JSON and heartbeats +// to a destination (out channel) +type NDJsonStream struct { + out chan<- *NDJson + + // heartbeat is the interval to send heartbeat messages to keep a connection + // open. + heartbeat *time.Ticker + + publishCh chan NDJson + exitCh chan struct{} + + l sync.Mutex + running bool +} + +// NNDJson is a wrapper for a Newline Delimited JSON object +type NDJson struct { + Data []byte +} + +// NewNNewNDJsonStream creates a new NDJson stream that will output NDJson structs +// to the passed output channel +func NewNDJsonStream(out chan<- *NDJson, heartbeat time.Duration) *NDJsonStream { + return &NDJsonStream{ + out: out, + heartbeat: time.NewTicker(heartbeat), + exitCh: make(chan struct{}), + publishCh: make(chan NDJson), + } +} + +// Run starts a long lived goroutine that handles sending +// heartbeats and processed json objects to the streams out channel as well +func (n *NDJsonStream) Run(ctx context.Context) { + n.l.Lock() + if n.running { + return + } + n.running = true + n.l.Unlock() + + go n.run(ctx) +} + +func (n *NDJsonStream) run(ctx context.Context) { + defer func() { + n.l.Lock() + n.running = false + n.l.Unlock() + close(n.exitCh) + }() + + for { + select { + case <-ctx.Done(): + return + case msg := <-n.publishCh: + n.out <- msg.Copy() + case <-n.heartbeat.C: + // Send a heartbeat frame + select { + case n.out <- NDJsonHeartbeat: + case <-ctx.Done(): + return + } + } + } +} + +// Send encodes an object into Newline delimited json. An error is returned +// if json encoding fails or if the stream is no longer running. +func (n *NDJsonStream) Send(obj interface{}) error { + n.l.Lock() + defer n.l.Unlock() + + buf := bytes.NewBuffer(nil) + if err := json.NewEncoder(buf).Encode(obj); err != nil { + return fmt.Errorf("marshaling json for stream: %w", err) + } + + select { + case n.publishCh <- NDJson{Data: buf.Bytes()}: + case <-n.exitCh: + return fmt.Errorf("stream is no longer running") + } + + return nil +} + +func (j *NDJson) Copy() *NDJson { + n := new(NDJson) + *n = *j + n.Data = make([]byte, len(j.Data)) + copy(n.Data, j.Data) + return n +} diff --git a/nomad/stream/ndjson_test.go b/nomad/stream/ndjson_test.go new file mode 100644 index 000000000000..8e807938fb7a --- /dev/null +++ b/nomad/stream/ndjson_test.go @@ -0,0 +1,72 @@ +package stream + +import ( + "bytes" + "context" + "testing" + "time" + + "github.com/stretchr/testify/require" +) + +type testObj struct { + Name string `json:"name"` +} + +func TestNDJson(t *testing.T) { + t.Parallel() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + out := make(chan *NDJson) + s := NewNDJsonStream(out, 1*time.Second) + s.Run(ctx) + + require.NoError(t, s.Send(testObj{Name: "test"})) + + out1 := <-out + + var expected bytes.Buffer + expected.Write([]byte(`{"name":"test"}`)) + expected.Write([]byte("\n")) + + require.Equal(t, expected.Bytes(), out1.Data) + select { + case _ = <-out: + t.Fatalf("Did not expect another message") + case <-time.After(100 * time.Millisecond): + } +} + +func TestNDJson_Send_After_Stop(t *testing.T) { + t.Parallel() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + out := make(chan *NDJson) + s := NewNDJsonStream(out, 1*time.Second) + s.Run(ctx) + + // stop the stream + cancel() + + time.Sleep(10 * time.Millisecond) + require.Error(t, s.Send(testObj{})) +} + +func TestNDJson_HeartBeat(t *testing.T) { + t.Parallel() + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + out := make(chan *NDJson) + s := NewNDJsonStream(out, 10*time.Millisecond) + s.Run(ctx) + + heartbeat := <-out + + require.Equal(t, NDJsonHeartbeat, heartbeat) +} diff --git a/nomad/stream/subscription.go b/nomad/stream/subscription.go index 6846e005bd35..0fc64512c19a 100644 --- a/nomad/stream/subscription.go +++ b/nomad/stream/subscription.go @@ -21,10 +21,6 @@ const ( // closed. The client should Unsubscribe, then re-Subscribe. var ErrSubscriptionClosed = errors.New("subscription closed by server, client should resubscribe") -// type Subscriber struct { -// logger hclog.Logger -// } - type Subscription struct { // state is accessed atomically 0 means open, 1 means closed with reload state uint32 @@ -104,8 +100,15 @@ func filter(req *SubscribeRequest, events []Event) []Event { var count int for _, e := range events { - if _, ok := req.Topics[e.Topic]; ok { - for _, k := range req.Topics[e.Topic] { + _, allTopics := req.Topics[AllKeys] + if _, ok := req.Topics[e.Topic]; ok || allTopics { + var keys []string + if allTopics { + keys = req.Topics[AllKeys] + } else { + keys = req.Topics[e.Topic] + } + for _, k := range keys { if e.Key == k || k == AllKeys { count++ } @@ -124,8 +127,15 @@ func filter(req *SubscribeRequest, events []Event) []Event { // Return filtered events result := make([]Event, 0, count) for _, e := range events { - if _, ok := req.Topics[e.Topic]; ok { - for _, k := range req.Topics[e.Topic] { + _, allTopics := req.Topics[AllKeys] + if _, ok := req.Topics[e.Topic]; ok || allTopics { + var keys []string + if allTopics { + keys = req.Topics[AllKeys] + } else { + keys = req.Topics[e.Topic] + } + for _, k := range keys { if e.Key == k || k == AllKeys { result = append(result, e) } diff --git a/nomad/stream/subscription_test.go b/nomad/stream/subscription_test.go index 9d1f55720159..8dd841bd2be2 100644 --- a/nomad/stream/subscription_test.go +++ b/nomad/stream/subscription_test.go @@ -10,6 +10,22 @@ func TestSubscription(t *testing.T) { } +func TestFilter_AllTopics(t *testing.T) { + events := make([]Event, 0, 5) + events = append(events, Event{Topic: "Test", Key: "One"}, Event{Topic: "Test", Key: "Two"}) + + req := &SubscribeRequest{ + Topics: map[Topic][]string{ + "*": []string{"*"}, + }, + } + actual := filter(req, events) + require.Equal(t, events, actual) + + // ensure new array was not allocated + require.Equal(t, cap(actual), 5) +} + func TestFilter_AllKeys(t *testing.T) { events := make([]Event, 0, 5) events = append(events, Event{Topic: "Test", Key: "One"}, Event{Topic: "Test", Key: "Two"}) diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 59d5ae9b4978..653aadc0c30c 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -39,6 +39,7 @@ import ( "github.com/hashicorp/nomad/helper/constraints/semver" "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/lib/kheap" + "github.com/hashicorp/nomad/nomad/stream" psstructs "github.com/hashicorp/nomad/plugins/shared/structs" ) @@ -10694,3 +10695,34 @@ type ACLTokenUpsertResponse struct { Tokens []*ACLToken WriteMeta } + +// EEventStreamRequest is used to stream events from a servers +// EventPublisher +type EventStreamRequest struct { + Topics map[stream.Topic][]string + Index int + + QueryOptions +} + +type EventStreamWrapper struct { + Error *RpcError + Event *stream.NDJson +} + +// RpcError is used for serializing errors with a potential error code +type RpcError struct { + Message string + Code *int64 +} + +func NewRpcError(err error, code *int64) *RpcError { + return &RpcError{ + Message: err.Error(), + Code: code, + } +} + +func (r *RpcError) Error() string { + return r.Message +} diff --git a/nomad/testing.go b/nomad/testing.go index f575139ccc2a..edb2d1c8e49a 100644 --- a/nomad/testing.go +++ b/nomad/testing.go @@ -47,6 +47,7 @@ func TestServer(t testing.T, cb func(*Config)) (*Server, func()) { config.Logger = testlog.HCLogger(t) config.Build = version.Version + "+unittest" config.DevMode = true + config.EnableEventPublisher = true config.BootstrapExpect = 1 nodeNum := atomic.AddUint32(&nodeNumber, 1) config.NodeName = fmt.Sprintf("nomad-%03d", nodeNum) From 8d16846695a57a55167ec7639c9a495c69a194ec Mon Sep 17 00:00:00 2001 From: Drew Bailey <2614075+drewbailey@users.noreply.github.com> Date: Thu, 1 Oct 2020 14:43:28 -0400 Subject: [PATCH 06/19] Events/deployment events (#9004) * Node Drain events and Node Events (#8980) Deployment status updates handle deployment status updates (paused, failed, resume) deployment alloc health generate events from apply plan result txn err check, slim down deployment event one ndjson line per index * consolidate down to node event + type * fix UpdateDeploymentAllocHealth test invocations * fix test --- command/agent/event_endpoint_test.go | 12 +- .../deployments_watcher_test.go | 13 +- nomad/deploymentwatcher/testutil_test.go | 7 +- nomad/eval_endpoint_test.go | 3 +- nomad/event_endpoint.go | 15 +- nomad/event_endpoint_test.go | 12 +- nomad/fsm.go | 48 +++-- nomad/plan_apply.go | 2 +- nomad/state/apply_plan_events.go | 74 +++++++ nomad/state/deployment_event_test.go | 189 ++++++++++++++++ nomad/state/deployment_events.go | 84 ++++++++ nomad/state/events.go | 68 ++++++ nomad/state/node_events.go | 104 +++++++-- nomad/state/node_events_test.go | 202 ++++++++++++++---- nomad/state/state_changes.go | 12 ++ nomad/state/state_store.go | 52 ++++- nomad/state/state_store_test.go | 42 ++-- nomad/stream/event.go | 15 +- nomad/stream/event_publisher.go | 2 +- nomad/stream/event_publisher_test.go | 2 +- nomad/stream/subscription.go | 28 ++- nomad/structs/structs.go | 9 + scheduler/testing.go | 3 +- 23 files changed, 842 insertions(+), 156 deletions(-) create mode 100644 nomad/state/apply_plan_events.go create mode 100644 nomad/state/deployment_event_test.go create mode 100644 nomad/state/deployment_events.go create mode 100644 nomad/state/events.go diff --git a/command/agent/event_endpoint_test.go b/command/agent/event_endpoint_test.go index 88987e3d9de5..e97b4a30d363 100644 --- a/command/agent/event_endpoint_test.go +++ b/command/agent/event_endpoint_test.go @@ -79,14 +79,14 @@ func TestEventStream_QueryParse(t *testing.T) { desc: "all topics and keys specified", query: "?topic=*:*", want: map[stream.Topic][]string{ - "*": []string{"*"}, + "*": {"*"}, }, }, { desc: "all topics and keys inferred", query: "", want: map[stream.Topic][]string{ - "*": []string{"*"}, + "*": {"*"}, }, }, { @@ -103,14 +103,14 @@ func TestEventStream_QueryParse(t *testing.T) { desc: "single topic and key", query: "?topic=NodeDrain:*", want: map[stream.Topic][]string{ - "NodeDrain": []string{"*"}, + "NodeDrain": {"*"}, }, }, { desc: "single topic multiple keys", query: "?topic=NodeDrain:*&topic=NodeDrain:3caace09-f1f4-4d23-b37a-9ab5eb75069d", want: map[stream.Topic][]string{ - "NodeDrain": []string{ + "NodeDrain": { "*", "3caace09-f1f4-4d23-b37a-9ab5eb75069d", }, @@ -120,10 +120,10 @@ func TestEventStream_QueryParse(t *testing.T) { desc: "multiple topics", query: "?topic=NodeRegister:*&topic=NodeDrain:3caace09-f1f4-4d23-b37a-9ab5eb75069d", want: map[stream.Topic][]string{ - "NodeDrain": []string{ + "NodeDrain": { "3caace09-f1f4-4d23-b37a-9ab5eb75069d", }, - "NodeRegister": []string{ + "NodeRegister": { "*", }, }, diff --git a/nomad/deploymentwatcher/deployments_watcher_test.go b/nomad/deploymentwatcher/deployments_watcher_test.go index 8373a900697b..c4c503f35c21 100644 --- a/nomad/deploymentwatcher/deployments_watcher_test.go +++ b/nomad/deploymentwatcher/deployments_watcher_test.go @@ -1,6 +1,7 @@ package deploymentwatcher import ( + "context" "fmt" "testing" "time" @@ -917,7 +918,7 @@ func TestDeploymentWatcher_Watch_NoProgressDeadline(t *testing.T) { HealthyAllocationIDs: []string{a.ID}, }, } - require.Nil(m.state.UpdateDeploymentAllocHealth(m.nextIndex(), req), "UpsertDeploymentAllocHealth") + require.Nil(m.state.UpdateDeploymentAllocHealth(context.Background(), m.nextIndex(), req), "UpsertDeploymentAllocHealth") } // Wait for there to be one eval @@ -945,7 +946,7 @@ func TestDeploymentWatcher_Watch_NoProgressDeadline(t *testing.T) { UnhealthyAllocationIDs: []string{a.ID}, }, } - require.Nil(m.state.UpdateDeploymentAllocHealth(m.nextIndex(), req2), "UpsertDeploymentAllocHealth") + require.Nil(m.state.UpdateDeploymentAllocHealth(context.Background(), m.nextIndex(), req2), "UpsertDeploymentAllocHealth") // Wait for there to be one eval testutil.WaitForResult(func() (bool, error) { @@ -1453,7 +1454,7 @@ func TestDeploymentWatcher_RollbackFailed(t *testing.T) { HealthyAllocationIDs: []string{a.ID}, }, } - require.Nil(m.state.UpdateDeploymentAllocHealth(m.nextIndex(), req), "UpsertDeploymentAllocHealth") + require.Nil(m.state.UpdateDeploymentAllocHealth(context.Background(), m.nextIndex(), req), "UpsertDeploymentAllocHealth") } // Wait for there to be one eval @@ -1481,7 +1482,7 @@ func TestDeploymentWatcher_RollbackFailed(t *testing.T) { UnhealthyAllocationIDs: []string{a.ID}, }, } - require.Nil(m.state.UpdateDeploymentAllocHealth(m.nextIndex(), req2), "UpsertDeploymentAllocHealth") + require.Nil(m.state.UpdateDeploymentAllocHealth(context.Background(), m.nextIndex(), req2), "UpsertDeploymentAllocHealth") // Wait for there to be one eval testutil.WaitForResult(func() (bool, error) { @@ -1562,7 +1563,7 @@ func TestWatcher_BatchAllocUpdates(t *testing.T) { HealthyAllocationIDs: []string{a1.ID}, }, } - require.Nil(m.state.UpdateDeploymentAllocHealth(m.nextIndex(), req), "UpsertDeploymentAllocHealth") + require.Nil(m.state.UpdateDeploymentAllocHealth(context.Background(), m.nextIndex(), req), "UpsertDeploymentAllocHealth") req2 := &structs.ApplyDeploymentAllocHealthRequest{ DeploymentAllocHealthRequest: structs.DeploymentAllocHealthRequest{ @@ -1570,7 +1571,7 @@ func TestWatcher_BatchAllocUpdates(t *testing.T) { HealthyAllocationIDs: []string{a2.ID}, }, } - require.Nil(m.state.UpdateDeploymentAllocHealth(m.nextIndex(), req2), "UpsertDeploymentAllocHealth") + require.Nil(m.state.UpdateDeploymentAllocHealth(context.Background(), m.nextIndex(), req2), "UpsertDeploymentAllocHealth") // Wait for there to be one eval for each job testutil.WaitForResult(func() (bool, error) { diff --git a/nomad/deploymentwatcher/testutil_test.go b/nomad/deploymentwatcher/testutil_test.go index 0542c4469556..cc2e9d844fa9 100644 --- a/nomad/deploymentwatcher/testutil_test.go +++ b/nomad/deploymentwatcher/testutil_test.go @@ -1,6 +1,7 @@ package deploymentwatcher import ( + "context" "reflect" "strings" "sync" @@ -95,7 +96,7 @@ func (m *mockBackend) UpsertJob(job *structs.Job) (uint64, error) { func (m *mockBackend) UpdateDeploymentStatus(u *structs.DeploymentStatusUpdateRequest) (uint64, error) { m.Called(u) i := m.nextIndex() - return i, m.state.UpdateDeploymentStatus(i, u) + return i, m.state.UpdateDeploymentStatus(context.Background(), i, u) } // matchDeploymentStatusUpdateConfig is used to configure the matching @@ -149,7 +150,7 @@ func matchDeploymentStatusUpdateRequest(c *matchDeploymentStatusUpdateConfig) fu func (m *mockBackend) UpdateDeploymentPromotion(req *structs.ApplyDeploymentPromoteRequest) (uint64, error) { m.Called(req) i := m.nextIndex() - return i, m.state.UpdateDeploymentPromotion(i, req) + return i, m.state.UpdateDeploymentPromotion(context.Background(), i, req) } // matchDeploymentPromoteRequestConfig is used to configure the matching @@ -179,7 +180,7 @@ func matchDeploymentPromoteRequest(c *matchDeploymentPromoteRequestConfig) func( func (m *mockBackend) UpdateDeploymentAllocHealth(req *structs.ApplyDeploymentAllocHealthRequest) (uint64, error) { m.Called(req) i := m.nextIndex() - return i, m.state.UpdateDeploymentAllocHealth(i, req) + return i, m.state.UpdateDeploymentAllocHealth(context.Background(), i, req) } // matchDeploymentAllocHealthRequestConfig is used to configure the matching diff --git a/nomad/eval_endpoint_test.go b/nomad/eval_endpoint_test.go index 1e77faa7354e..e494466559e2 100644 --- a/nomad/eval_endpoint_test.go +++ b/nomad/eval_endpoint_test.go @@ -1,6 +1,7 @@ package nomad import ( + "context" "fmt" "reflect" "strings" @@ -374,7 +375,7 @@ func TestEvalEndpoint_Dequeue_UpdateWaitIndex(t *testing.T) { EvalID: eval.ID, } assert := assert.New(t) - err := state.UpsertPlanResults(1000, &res) + err := state.UpsertPlanResults(context.Background(), 1000, &res) assert.Nil(err) // Dequeue the eval diff --git a/nomad/event_endpoint.go b/nomad/event_endpoint.go index 5661200d43f7..781adb5ad3d5 100644 --- a/nomad/event_endpoint.go +++ b/nomad/event_endpoint.go @@ -117,19 +117,16 @@ func (e *Event) stream(conn io.ReadWriteCloser) { } // Continue if there are no events - if events == nil { + if len(events.Events) == 0 { continue } - // Send each event as its own frame - for _, e := range events { - if err := jsonStream.Send(e); err != nil { - select { - case errCh <- err: - case <-ctx.Done(): - } - break LOOP + if err := jsonStream.Send(events); err != nil { + select { + case errCh <- err: + case <-ctx.Done(): } + break LOOP } } }() diff --git a/nomad/event_endpoint_test.go b/nomad/event_endpoint_test.go index 9e4256192614..793c198a40a7 100644 --- a/nomad/event_endpoint_test.go +++ b/nomad/event_endpoint_test.go @@ -91,7 +91,7 @@ OUTER: continue } - var event stream.Event + var event stream.Events err = json.Unmarshal(msg.Event.Data, &event) require.NoError(t, err) @@ -102,7 +102,7 @@ OUTER: Result: &out, } dec, err := mapstructure.NewDecoder(cfg) - dec.Decode(event.Payload) + dec.Decode(event.Events[0].Payload) require.NoError(t, err) require.Equal(t, node.ID, out.ID) break OUTER @@ -123,7 +123,7 @@ func TestEventStream_StreamErr(t *testing.T) { testutil.WaitForLeader(t, s1.RPC) req := structs.EventStreamRequest{ - Topics: map[stream.Topic][]string{"*": []string{"*"}}, + Topics: map[stream.Topic][]string{"*": {"*"}}, QueryOptions: structs.QueryOptions{ Region: s1.Region(), }, @@ -210,7 +210,7 @@ func TestEventStream_RegionForward(t *testing.T) { // Create request targed for region foo req := structs.EventStreamRequest{ - Topics: map[stream.Topic][]string{"*": []string{"*"}}, + Topics: map[stream.Topic][]string{"*": {"*"}}, QueryOptions: structs.QueryOptions{ Region: "foo", }, @@ -272,7 +272,7 @@ OUTER: continue } - var event stream.Event + var event stream.Events err = json.Unmarshal(msg.Event.Data, &event) require.NoError(t, err) @@ -282,7 +282,7 @@ OUTER: Result: &out, } dec, err := mapstructure.NewDecoder(cfg) - dec.Decode(event.Payload) + dec.Decode(event.Events[0].Payload) require.NoError(t, err) require.Equal(t, node.ID, out.ID) break OUTER diff --git a/nomad/fsm.go b/nomad/fsm.go index d18a1d43b734..c4e4aa5bf858 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -206,7 +206,7 @@ func (n *nomadFSM) Apply(log *raft.Log) interface{} { case structs.NodeUpdateStatusRequestType: return n.applyStatusUpdate(buf[1:], log.Index) case structs.NodeUpdateDrainRequestType: - return n.applyDrainUpdate(buf[1:], log.Index) + return n.applyDrainUpdate(msgType, buf[1:], log.Index) case structs.JobRegisterRequestType: return n.applyUpsertJob(buf[1:], log.Index) case structs.JobDeregisterRequestType: @@ -226,13 +226,13 @@ func (n *nomadFSM) Apply(log *raft.Log) interface{} { case structs.VaultAccessorDeregisterRequestType: return n.applyDeregisterVaultAccessor(buf[1:], log.Index) case structs.ApplyPlanResultsRequestType: - return n.applyPlanResults(buf[1:], log.Index) + return n.applyPlanResults(msgType, buf[1:], log.Index) case structs.DeploymentStatusUpdateRequestType: - return n.applyDeploymentStatusUpdate(buf[1:], log.Index) + return n.applyDeploymentStatusUpdate(msgType, buf[1:], log.Index) case structs.DeploymentPromoteRequestType: - return n.applyDeploymentPromotion(buf[1:], log.Index) + return n.applyDeploymentPromotion(msgType, buf[1:], log.Index) case structs.DeploymentAllocHealthRequestType: - return n.applyDeploymentAllocHealth(buf[1:], log.Index) + return n.applyDeploymentAllocHealth(msgType, buf[1:], log.Index) case structs.DeploymentDeleteRequestType: return n.applyDeploymentDelete(buf[1:], log.Index) case structs.JobStabilityRequestType: @@ -250,7 +250,7 @@ func (n *nomadFSM) Apply(log *raft.Log) interface{} { case structs.AutopilotRequestType: return n.applyAutopilotUpdate(buf[1:], log.Index) case structs.UpsertNodeEventsType: - return n.applyUpsertNodeEvent(buf[1:], log.Index) + return n.applyUpsertNodeEvent(msgType, buf[1:], log.Index) case structs.JobBatchDeregisterRequestType: return n.applyBatchDeregisterJob(buf[1:], log.Index) case structs.AllocUpdateDesiredTransitionRequestType: @@ -402,13 +402,15 @@ func (n *nomadFSM) applyStatusUpdate(buf []byte, index uint64) interface{} { return nil } -func (n *nomadFSM) applyDrainUpdate(buf []byte, index uint64) interface{} { +func (n *nomadFSM) applyDrainUpdate(reqType structs.MessageType, buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "node_drain_update"}, time.Now()) var req structs.NodeUpdateDrainRequest if err := structs.Decode(buf, &req); err != nil { panic(fmt.Errorf("failed to decode request: %v", err)) } + ctx := context.WithValue(context.Background(), state.CtxMsgType, reqType) + // COMPAT Remove in version 0.10 // As part of Nomad 0.8 we have deprecated the drain boolean in favor of a // drain strategy but we need to handle the upgrade path where the Raft log @@ -423,7 +425,7 @@ func (n *nomadFSM) applyDrainUpdate(buf []byte, index uint64) interface{} { } } - if err := n.state.UpdateNodeDrain(index, req.NodeID, req.DrainStrategy, req.MarkEligible, req.UpdatedAt, req.NodeEvent); err != nil { + if err := n.state.UpdateNodeDrainCtx(ctx, index, req.NodeID, req.DrainStrategy, req.MarkEligible, req.UpdatedAt, req.NodeEvent); err != nil { n.logger.Error("UpdateNodeDrain failed", "error", err) return err } @@ -874,14 +876,16 @@ func (n *nomadFSM) applyReconcileSummaries(buf []byte, index uint64) interface{} } // applyUpsertNodeEvent tracks the given node events. -func (n *nomadFSM) applyUpsertNodeEvent(buf []byte, index uint64) interface{} { +func (n *nomadFSM) applyUpsertNodeEvent(msgType structs.MessageType, buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "upsert_node_events"}, time.Now()) var req structs.EmitNodeEventsRequest if err := structs.Decode(buf, &req); err != nil { panic(fmt.Errorf("failed to decode EmitNodeEventsRequest: %v", err)) } - if err := n.state.UpsertNodeEvents(index, req.NodeEvents); err != nil { + ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType) + + if err := n.state.UpsertNodeEventsCtx(ctx, index, req.NodeEvents); err != nil { n.logger.Error("failed to add node events", "error", err) return err } @@ -953,14 +957,16 @@ func (n *nomadFSM) applyDeregisterSIAccessor(buf []byte, index uint64) interface } // applyPlanApply applies the results of a plan application -func (n *nomadFSM) applyPlanResults(buf []byte, index uint64) interface{} { +func (n *nomadFSM) applyPlanResults(msgType structs.MessageType, buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "apply_plan_results"}, time.Now()) var req structs.ApplyPlanResultsRequest if err := structs.Decode(buf, &req); err != nil { panic(fmt.Errorf("failed to decode request: %v", err)) } - if err := n.state.UpsertPlanResults(index, &req); err != nil { + ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType) + + if err := n.state.UpsertPlanResults(ctx, index, &req); err != nil { n.logger.Error("ApplyPlan failed", "error", err) return err } @@ -972,14 +978,16 @@ func (n *nomadFSM) applyPlanResults(buf []byte, index uint64) interface{} { // applyDeploymentStatusUpdate is used to update the status of an existing // deployment -func (n *nomadFSM) applyDeploymentStatusUpdate(buf []byte, index uint64) interface{} { +func (n *nomadFSM) applyDeploymentStatusUpdate(msgType structs.MessageType, buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "apply_deployment_status_update"}, time.Now()) var req structs.DeploymentStatusUpdateRequest if err := structs.Decode(buf, &req); err != nil { panic(fmt.Errorf("failed to decode request: %v", err)) } - if err := n.state.UpdateDeploymentStatus(index, &req); err != nil { + ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType) + + if err := n.state.UpdateDeploymentStatus(ctx, index, &req); err != nil { n.logger.Error("UpsertDeploymentStatusUpdate failed", "error", err) return err } @@ -989,14 +997,16 @@ func (n *nomadFSM) applyDeploymentStatusUpdate(buf []byte, index uint64) interfa } // applyDeploymentPromotion is used to promote canaries in a deployment -func (n *nomadFSM) applyDeploymentPromotion(buf []byte, index uint64) interface{} { +func (n *nomadFSM) applyDeploymentPromotion(msgType structs.MessageType, buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "apply_deployment_promotion"}, time.Now()) var req structs.ApplyDeploymentPromoteRequest if err := structs.Decode(buf, &req); err != nil { panic(fmt.Errorf("failed to decode request: %v", err)) } - if err := n.state.UpdateDeploymentPromotion(index, &req); err != nil { + ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType) + + if err := n.state.UpdateDeploymentPromotion(ctx, index, &req); err != nil { n.logger.Error("UpsertDeploymentPromotion failed", "error", err) return err } @@ -1007,14 +1017,16 @@ func (n *nomadFSM) applyDeploymentPromotion(buf []byte, index uint64) interface{ // applyDeploymentAllocHealth is used to set the health of allocations as part // of a deployment -func (n *nomadFSM) applyDeploymentAllocHealth(buf []byte, index uint64) interface{} { +func (n *nomadFSM) applyDeploymentAllocHealth(msgType structs.MessageType, buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "apply_deployment_alloc_health"}, time.Now()) var req structs.ApplyDeploymentAllocHealthRequest if err := structs.Decode(buf, &req); err != nil { panic(fmt.Errorf("failed to decode request: %v", err)) } - if err := n.state.UpdateDeploymentAllocHealth(index, &req); err != nil { + ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType) + + if err := n.state.UpdateDeploymentAllocHealth(ctx, index, &req); err != nil { n.logger.Error("UpsertDeploymentAllocHealth failed", "error", err) return err } diff --git a/nomad/plan_apply.go b/nomad/plan_apply.go index 4f527018b283..08d050026019 100644 --- a/nomad/plan_apply.go +++ b/nomad/plan_apply.go @@ -310,7 +310,7 @@ func (p *planner) applyPlan(plan *structs.Plan, result *structs.PlanResult, snap // Optimistically apply to our state view if snap != nil { nextIdx := p.raft.AppliedIndex() + 1 - if err := snap.UpsertPlanResults(nextIdx, &req); err != nil { + if err := snap.UpsertPlanResults(context.Background(), nextIdx, &req); err != nil { return future, err } } diff --git a/nomad/state/apply_plan_events.go b/nomad/state/apply_plan_events.go new file mode 100644 index 000000000000..470c48790b9e --- /dev/null +++ b/nomad/state/apply_plan_events.go @@ -0,0 +1,74 @@ +package state + +import ( + "fmt" + + "github.com/hashicorp/nomad/nomad/stream" + "github.com/hashicorp/nomad/nomad/structs" +) + +func ApplyPlanResultEventsFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { + var events []stream.Event + for _, change := range changes.Changes { + switch change.Table { + case "deployment": + after, ok := change.After.(*structs.Deployment) + if !ok { + return nil, fmt.Errorf("transaction change was not a Deployment") + } + + event := stream.Event{ + Topic: TopicDeployment, + Type: TypeDeploymentUpdate, + Index: changes.Index, + Key: after.ID, + Payload: &DeploymentEvent{ + Deployment: after, + }, + } + events = append(events, event) + case "evals": + after, ok := change.After.(*structs.Evaluation) + if !ok { + return nil, fmt.Errorf("transaction change was not an Evaluation") + } + + event := stream.Event{ + Topic: TopicEval, + Index: changes.Index, + Key: after.ID, + Payload: &EvalEvent{ + Eval: after, + }, + } + + events = append(events, event) + case "allocs": + after, ok := change.After.(*structs.Allocation) + if !ok { + return nil, fmt.Errorf("transaction change was not an Allocation") + } + before := change.Before + var msg string + if before == nil { + msg = TypeAllocCreated + } else { + msg = TypeAllocUpdated + } + + event := stream.Event{ + Topic: TopicAlloc, + Type: msg, + Index: changes.Index, + Key: after.ID, + Payload: &AllocEvent{ + Alloc: after, + }, + } + + events = append(events, event) + } + } + + return events, nil +} diff --git a/nomad/state/deployment_event_test.go b/nomad/state/deployment_event_test.go new file mode 100644 index 000000000000..d71ed8d2f335 --- /dev/null +++ b/nomad/state/deployment_event_test.go @@ -0,0 +1,189 @@ +package state + +import ( + "context" + "testing" + "time" + + "github.com/hashicorp/nomad/helper" + "github.com/hashicorp/nomad/nomad/mock" + "github.com/hashicorp/nomad/nomad/stream" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/stretchr/testify/require" +) + +func TestDeploymentEventFromChanges(t *testing.T) { + t.Parallel() + s := TestStateStoreCfg(t, TestStateStorePublisher(t)) + defer s.StopEventPublisher() + + // setup + setupTx := s.db.WriteTxn(10) + + j := mock.Job() + e := mock.Eval() + e.JobID = j.ID + + d := mock.Deployment() + d.JobID = j.ID + + require.NoError(t, s.upsertJobImpl(10, j, false, setupTx)) + require.NoError(t, s.upsertDeploymentImpl(10, d, setupTx)) + + setupTx.Txn.Commit() + + ctx := context.WithValue(context.Background(), CtxMsgType, structs.DeploymentStatusUpdateRequestType) + + req := &structs.DeploymentStatusUpdateRequest{ + DeploymentUpdate: &structs.DeploymentStatusUpdate{ + DeploymentID: d.ID, + Status: structs.DeploymentStatusPaused, + StatusDescription: structs.DeploymentStatusDescriptionPaused, + }, + Eval: e, + // Exlude Job and assert its added + } + + require.NoError(t, s.UpdateDeploymentStatus(ctx, 100, req)) + + events := WaitForEvents(t, s, 100, 1, 1*time.Second) + require.Len(t, events, 2) + + got := events[0] + require.Equal(t, uint64(100), got.Index) + require.Equal(t, d.ID, got.Key) + + de := got.Payload.(*DeploymentEvent) + require.Equal(t, structs.DeploymentStatusPaused, de.Deployment.Status) + require.Contains(t, got.FilterKeys, j.ID) + +} + +func TestDeploymentEventFromChanges_Promotion(t *testing.T) { + t.Parallel() + s := TestStateStoreCfg(t, TestStateStorePublisher(t)) + defer s.StopEventPublisher() + + // setup + setupTx := s.db.WriteTxn(10) + + j := mock.Job() + tg1 := j.TaskGroups[0] + tg2 := tg1.Copy() + tg2.Name = "foo" + j.TaskGroups = append(j.TaskGroups, tg2) + require.NoError(t, s.upsertJobImpl(10, j, false, setupTx)) + + d := mock.Deployment() + d.StatusDescription = structs.DeploymentStatusDescriptionRunningNeedsPromotion + d.JobID = j.ID + d.TaskGroups = map[string]*structs.DeploymentState{ + "web": { + DesiredTotal: 10, + DesiredCanaries: 1, + }, + "foo": { + DesiredTotal: 10, + DesiredCanaries: 1, + }, + } + require.NoError(t, s.upsertDeploymentImpl(10, d, setupTx)) + + // create set of allocs + c1 := mock.Alloc() + c1.JobID = j.ID + c1.DeploymentID = d.ID + d.TaskGroups[c1.TaskGroup].PlacedCanaries = append(d.TaskGroups[c1.TaskGroup].PlacedCanaries, c1.ID) + c1.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + c2 := mock.Alloc() + c2.JobID = j.ID + c2.DeploymentID = d.ID + d.TaskGroups[c2.TaskGroup].PlacedCanaries = append(d.TaskGroups[c2.TaskGroup].PlacedCanaries, c2.ID) + c2.TaskGroup = tg2.Name + c2.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + + require.NoError(t, s.upsertAllocsImpl(10, []*structs.Allocation{c1, c2}, setupTx)) + + // commit setup transaction + setupTx.Txn.Commit() + + e := mock.Eval() + // Request to promote canaries + ctx := context.WithValue(context.Background(), CtxMsgType, structs.DeploymentPromoteRequestType) + req := &structs.ApplyDeploymentPromoteRequest{ + DeploymentPromoteRequest: structs.DeploymentPromoteRequest{ + DeploymentID: d.ID, + All: true, + }, + Eval: e, + } + + require.NoError(t, s.UpdateDeploymentPromotion(ctx, 100, req)) + + events := WaitForEvents(t, s, 100, 1, 1*time.Second) + require.Len(t, events, 2) + + got := events[0] + require.Equal(t, uint64(100), got.Index) + require.Equal(t, d.ID, got.Key) + + de := got.Payload.(*DeploymentEvent) + require.Equal(t, structs.DeploymentStatusRunning, de.Deployment.Status) +} + +func WaitForEvents(t *testing.T, s *StateStore, index uint64, minEvents int, timeout time.Duration) []stream.Event { + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + go func() { + select { + case <-ctx.Done(): + return + case <-time.After(timeout): + require.Fail(t, "timeout waiting for events") + } + }() + + maxAttempts := 10 + for { + got := EventsForIndex(t, s, index) + if len(got) >= minEvents { + return got + } + maxAttempts-- + if maxAttempts == 0 { + require.Fail(t, "reached max attempts waiting for desired event count") + } + time.Sleep(10 * time.Millisecond) + } +} + +func EventsForIndex(t *testing.T, s *StateStore, index uint64) []stream.Event { + pub, err := s.EventPublisher() + require.NoError(t, err) + + sub, err := pub.Subscribe(&stream.SubscribeRequest{ + Topics: map[stream.Topic][]string{ + "*": []string{"*"}, + }, + Index: index, + }) + defer sub.Unsubscribe() + + require.NoError(t, err) + + var events []stream.Event + for { + e, err := sub.NextNoBlock() + require.NoError(t, err) + if e == nil { + break + } + events = append(events, e...) + } + return events +} diff --git a/nomad/state/deployment_events.go b/nomad/state/deployment_events.go new file mode 100644 index 000000000000..5f9838f28e25 --- /dev/null +++ b/nomad/state/deployment_events.go @@ -0,0 +1,84 @@ +package state + +import ( + "fmt" + + "github.com/hashicorp/nomad/nomad/stream" + "github.com/hashicorp/nomad/nomad/structs" +) + +func DeploymentEventFromChanges(msgType structs.MessageType, tx ReadTxn, changes Changes) ([]stream.Event, error) { + var events []stream.Event + + var eventType string + switch msgType { + case structs.DeploymentStatusUpdateRequestType: + eventType = TypeDeploymentUpdate + case structs.DeploymentPromoteRequestType: + eventType = TypeDeploymentPromotion + case structs.DeploymentAllocHealthRequestType: + eventType = TypeDeploymentAllocHealth + } + + for _, change := range changes.Changes { + switch change.Table { + case "deployment": + after, ok := change.After.(*structs.Deployment) + if !ok { + return nil, fmt.Errorf("transaction change was not a Deployment") + } + + event := stream.Event{ + Topic: TopicDeployment, + Type: eventType, + Index: changes.Index, + Key: after.ID, + FilterKeys: []string{after.JobID}, + Payload: &DeploymentEvent{ + Deployment: after, + }, + } + + events = append(events, event) + case "jobs": + after, ok := change.After.(*structs.Job) + if !ok { + return nil, fmt.Errorf("transaction change was not a Job") + } + + event := stream.Event{ + Topic: TopicJob, + Type: eventType, + Index: changes.Index, + Key: after.ID, + Payload: &JobEvent{ + Job: after, + }, + } + + events = append(events, event) + case "allocs": + // TODO(drew) determine how to handle alloc updates during deployment + case "evals": + after, ok := change.After.(*structs.Evaluation) + if !ok { + return nil, fmt.Errorf("transaction change was not an Evaluation") + } + + event := stream.Event{ + Topic: TopicEval, + Type: eventType, + Index: changes.Index, + Key: after.ID, + FilterKeys: []string{after.DeploymentID, after.JobID}, + Payload: &EvalEvent{ + Eval: after, + }, + } + + events = append(events, event) + } + } + + return events, nil +} diff --git a/nomad/state/events.go b/nomad/state/events.go new file mode 100644 index 000000000000..824623409f62 --- /dev/null +++ b/nomad/state/events.go @@ -0,0 +1,68 @@ +package state + +import ( + "github.com/hashicorp/nomad/nomad/stream" + "github.com/hashicorp/nomad/nomad/structs" +) + +const ( + TopicDeployment stream.Topic = "Deployment" + TopicEval stream.Topic = "Eval" + TopicAlloc stream.Topic = "Alloc" + TopicJob stream.Topic = "Job" + // TopicNodeRegistration stream.Topic = "NodeRegistration" + // TopicNodeDeregistration stream.Topic = "NodeDeregistration" + // TopicNodeDrain stream.Topic = "NodeDrain" + TopicNode stream.Topic = "Node" + + // TODO(drew) Node Events use TopicNode + Type + TypeNodeRegistration = "NodeRegistration" + TypeNodeDeregistration = "NodeDeregistration" + TypeNodeDrain = "NodeDrain" + TypeNodeEvent = "NodeEvent" + + TypeDeploymentUpdate = "DeploymentStatusUpdate" + TypeDeploymentPromotion = "DeploymentPromotion" + TypeDeploymentAllocHealth = "DeploymentAllocHealth" + + TypeAllocCreated = "AllocCreated" + TypeAllocUpdated = "AllocUpdated" +) + +type JobEvent struct { + Job *structs.Job +} + +type EvalEvent struct { + Eval *structs.Evaluation +} + +type AllocEvent struct { + Alloc *structs.Allocation +} + +type DeploymentEvent struct { + Deployment *structs.Deployment +} + +type NodeEvent struct { + Node *structs.Node +} + +// NNodeDrainEvent is the Payload for a NodeDrain event. It contains +// information related to the Node being drained as well as high level +// information about the current allocations on the Node +type NodeDrainEvent struct { + Node *structs.Node + JobAllocs map[string]*JobDrainDetails +} + +type NodeDrainAllocDetails struct { + ID string + Migrate *structs.MigrateStrategy +} + +type JobDrainDetails struct { + Type string + AllocDetails map[string]NodeDrainAllocDetails +} diff --git a/nomad/state/node_events.go b/nomad/state/node_events.go index b2889e3a3076..a9e55aeb10b9 100644 --- a/nomad/state/node_events.go +++ b/nomad/state/node_events.go @@ -7,20 +7,6 @@ import ( "github.com/hashicorp/nomad/nomad/structs" ) -const ( - TopicNodeRegistration = "NodeRegistration" - TopicNodeDeregistration = "NodeDeregistration" -) - -type NodeRegistrationEvent struct { - Event *structs.NodeEvent - NodeStatus string -} - -type NodeDeregistrationEvent struct { - NodeID string -} - // NodeRegisterEventFromChanges generates a NodeRegistrationEvent from a set // of transaction changes. func NodeRegisterEventFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { @@ -34,12 +20,12 @@ func NodeRegisterEventFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, } event := stream.Event{ - Topic: TopicNodeRegistration, + Topic: TopicNode, + Type: TypeNodeRegistration, Index: changes.Index, Key: after.ID, - Payload: &NodeRegistrationEvent{ - Event: after.Events[len(after.Events)-1], - NodeStatus: after.Status, + Payload: &NodeEvent{ + Node: after, }, } events = append(events, event) @@ -61,11 +47,87 @@ func NodeDeregisterEventFromChanges(tx ReadTxn, changes Changes) ([]stream.Event } event := stream.Event{ - Topic: TopicNodeDeregistration, + Topic: TopicNode, + Type: TypeNodeDeregistration, Index: changes.Index, Key: before.ID, - Payload: &NodeDeregistrationEvent{ - NodeID: before.ID, + Payload: &NodeEvent{ + Node: before, + }, + } + events = append(events, event) + } + } + return events, nil +} + +// NodeEventFromChanges generates a NodeDeregistrationEvent from a set +// of transaction changes. +func NodeEventFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { + var events []stream.Event + for _, change := range changes.Changes { + switch change.Table { + case "nodes": + after, ok := change.After.(*structs.Node) + if !ok { + return nil, fmt.Errorf("transaction change was not a Node") + } + + event := stream.Event{ + Topic: TopicNode, + Type: TypeNodeEvent, + Index: changes.Index, + Key: after.ID, + Payload: &NodeEvent{ + Node: after, + }, + } + events = append(events, event) + } + } + return events, nil +} + +func NodeDrainEventFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { + var events []stream.Event + for _, change := range changes.Changes { + switch change.Table { + case "nodes": + after, ok := change.After.(*structs.Node) + if !ok { + return nil, fmt.Errorf("transaction change was not a Node") + } + + // retrieve allocations currently on node + allocs, err := allocsByNodeTxn(tx, nil, after.ID) + if err != nil { + return nil, fmt.Errorf("retrieving allocations for node drain event: %w", err) + } + + // build job/alloc details for node drain + jobAllocs := make(map[string]*JobDrainDetails) + for _, a := range allocs { + if _, ok := jobAllocs[a.Job.Name]; !ok { + jobAllocs[a.Job.Name] = &JobDrainDetails{ + AllocDetails: make(map[string]NodeDrainAllocDetails), + Type: a.Job.Type, + } + } + + jobAllocs[a.Job.Name].AllocDetails[a.ID] = NodeDrainAllocDetails{ + Migrate: a.MigrateStrategy(), + ID: a.ID, + } + } + + event := stream.Event{ + Topic: TopicNode, + Type: TypeNodeDrain, + Index: changes.Index, + Key: after.ID, + Payload: &NodeDrainEvent{ + Node: after, + JobAllocs: jobAllocs, }, } events = append(events, event) diff --git a/nomad/state/node_events_test.go b/nomad/state/node_events_test.go index 10b9458a635a..78918db23578 100644 --- a/nomad/state/node_events_test.go +++ b/nomad/state/node_events_test.go @@ -2,6 +2,7 @@ package state import ( "testing" + "time" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/stream" @@ -9,7 +10,7 @@ import ( "github.com/stretchr/testify/require" ) -func TestNodeRegisterEventFromChanges(t *testing.T) { +func TestNodeEventsFromChanges(t *testing.T) { cases := []struct { Name string MsgType structs.MessageType @@ -17,53 +18,47 @@ func TestNodeRegisterEventFromChanges(t *testing.T) { Mutate func(s *StateStore, tx *txn) error WantEvents []stream.Event WantErr bool - WantTopic string + WantTopic stream.Topic }{ { MsgType: structs.NodeRegisterRequestType, - WantTopic: TopicNodeRegistration, + WantTopic: TopicNode, Name: "node registered", Mutate: func(s *StateStore, tx *txn) error { return upsertNodeTxn(tx, tx.Index, testNode()) }, WantEvents: []stream.Event{{ - Topic: TopicNodeRegistration, + Topic: TopicNode, + Type: TypeNodeRegistration, Key: testNodeID(), Index: 100, - Payload: &NodeRegistrationEvent{ - Event: &structs.NodeEvent{ - Message: "Node registered", - Subsystem: "Cluster", - }, - NodeStatus: structs.NodeStatusReady, + Payload: &NodeEvent{ + Node: testNode(), }, }}, WantErr: false, }, { MsgType: structs.NodeRegisterRequestType, - WantTopic: TopicNodeRegistration, + WantTopic: TopicNode, Name: "node registered initializing", Mutate: func(s *StateStore, tx *txn) error { return upsertNodeTxn(tx, tx.Index, testNode(nodeNotReady)) }, WantEvents: []stream.Event{{ - Topic: TopicNodeRegistration, + Topic: TopicNode, + Type: TypeNodeRegistration, Key: testNodeID(), Index: 100, - Payload: &NodeRegistrationEvent{ - Event: &structs.NodeEvent{ - Message: "Node registered", - Subsystem: "Cluster", - }, - NodeStatus: structs.NodeStatusInit, + Payload: &NodeEvent{ + Node: testNode(nodeNotReady), }, }}, WantErr: false, }, { MsgType: structs.NodeDeregisterRequestType, - WantTopic: TopicNodeDeregistration, + WantTopic: TopicNode, Name: "node deregistered", Setup: func(s *StateStore, tx *txn) error { return upsertNodeTxn(tx, tx.Index, testNode()) @@ -72,18 +67,19 @@ func TestNodeRegisterEventFromChanges(t *testing.T) { return deleteNodeTxn(tx, tx.Index, []string{testNodeID()}) }, WantEvents: []stream.Event{{ - Topic: TopicNodeDeregistration, + Topic: TopicNode, + Type: TypeNodeDeregistration, Key: testNodeID(), Index: 100, - Payload: &NodeDeregistrationEvent{ - NodeID: testNodeID(), + Payload: &NodeEvent{ + Node: testNode(), }, }}, WantErr: false, }, { MsgType: structs.NodeDeregisterRequestType, - WantTopic: TopicNodeDeregistration, + WantTopic: TopicNode, Name: "batch node deregistered", Setup: func(s *StateStore, tx *txn) error { require.NoError(t, upsertNodeTxn(tx, tx.Index, testNode())) @@ -94,19 +90,73 @@ func TestNodeRegisterEventFromChanges(t *testing.T) { }, WantEvents: []stream.Event{ { - Topic: TopicNodeDeregistration, + Topic: TopicNode, + Type: TypeNodeDeregistration, Key: testNodeID(), Index: 100, - Payload: &NodeDeregistrationEvent{ - NodeID: testNodeID(), + Payload: &NodeEvent{ + Node: testNode(), }, }, { - Topic: TopicNodeDeregistration, + Topic: TopicNode, + Type: TypeNodeDeregistration, Key: testNodeIDTwo(), Index: 100, - Payload: &NodeDeregistrationEvent{ - NodeID: testNodeIDTwo(), + Payload: &NodeEvent{ + Node: testNode(nodeIDTwo), + }, + }, + }, + WantErr: false, + }, + { + MsgType: structs.UpsertNodeEventsType, + WantTopic: TopicNode, + Name: "batch node events upserted", + Setup: func(s *StateStore, tx *txn) error { + require.NoError(t, upsertNodeTxn(tx, tx.Index, testNode())) + return upsertNodeTxn(tx, tx.Index, testNode(nodeIDTwo)) + }, + Mutate: func(s *StateStore, tx *txn) error { + eventFn := func(id string) []*structs.NodeEvent { + return []*structs.NodeEvent{ + { + Message: "test event one", + Subsystem: "Cluster", + Details: map[string]string{ + "NodeID": id, + }, + }, + { + Message: "test event two", + Subsystem: "Cluster", + Details: map[string]string{ + "NodeID": id, + }, + }, + } + } + require.NoError(t, s.upsertNodeEvents(tx.Index, testNodeID(), eventFn(testNodeID()), tx)) + return s.upsertNodeEvents(tx.Index, testNodeIDTwo(), eventFn(testNodeIDTwo()), tx) + }, + WantEvents: []stream.Event{ + { + Topic: TopicNode, + Type: TypeNodeEvent, + Key: testNodeID(), + Index: 100, + Payload: &NodeEvent{ + Node: testNode(), + }, + }, + { + Topic: TopicNode, + Type: TypeNodeEvent, + Key: testNodeIDTwo(), + Index: 100, + Payload: &NodeEvent{ + Node: testNode(nodeIDTwo), }, }, }, @@ -140,44 +190,104 @@ func TestNodeRegisterEventFromChanges(t *testing.T) { require.Equal(t, len(tc.WantEvents), len(got)) for idx, g := range got { + // assert equality of shared fields + + want := tc.WantEvents[idx] + require.Equal(t, want.Index, g.Index) + require.Equal(t, want.Key, g.Key) + require.Equal(t, want.Topic, g.Topic) + switch tc.MsgType { case structs.NodeRegisterRequestType: requireNodeRegistrationEventEqual(t, tc.WantEvents[idx], g) case structs.NodeDeregisterRequestType: requireNodeDeregistrationEventEqual(t, tc.WantEvents[idx], g) + case structs.UpsertNodeEventsType: + requireNodeEventEqual(t, tc.WantEvents[idx], g) + default: + require.Fail(t, "unhandled message type") } } }) } } +func TestNodeDrainEventFromChanges(t *testing.T) { + t.Parallel() + s := TestStateStoreCfg(t, TestStateStorePublisher(t)) + defer s.StopEventPublisher() + + // setup + setupTx := s.db.WriteTxn(10) + + node := mock.Node() + alloc1 := mock.Alloc() + alloc2 := mock.Alloc() + alloc1.NodeID = node.ID + alloc2.NodeID = node.ID + + require.NoError(t, upsertNodeTxn(setupTx, 10, node)) + require.NoError(t, s.upsertAllocsImpl(100, []*structs.Allocation{alloc1, alloc2}, setupTx)) + setupTx.Txn.Commit() + + // changes + tx := s.db.WriteTxn(100) + + strat := &structs.DrainStrategy{ + DrainSpec: structs.DrainSpec{ + Deadline: 10 * time.Minute, + IgnoreSystemJobs: false, + }, + StartedAt: time.Now(), + } + markEligible := false + updatedAt := time.Now() + event := &structs.NodeEvent{} + + require.NoError(t, s.updateNodeDrainImpl(tx, 100, node.ID, strat, markEligible, updatedAt.UnixNano(), event)) + changes := Changes{Changes: tx.Changes(), Index: 100, MsgType: structs.NodeUpdateDrainRequestType} + got, err := processDBChanges(tx, changes) + require.NoError(t, err) + + require.Len(t, got, 1) + + require.Equal(t, TopicNode, got[0].Topic) + require.Equal(t, TypeNodeDrain, got[0].Type) + require.Equal(t, uint64(100), got[0].Index) + + nodeEvent, ok := got[0].Payload.(*NodeDrainEvent) + require.True(t, ok) + + require.Equal(t, structs.NodeSchedulingIneligible, nodeEvent.Node.SchedulingEligibility) + require.Equal(t, strat, nodeEvent.Node.DrainStrategy) +} + func requireNodeRegistrationEventEqual(t *testing.T, want, got stream.Event) { t.Helper() - require.Equal(t, want.Index, got.Index) - require.Equal(t, want.Key, got.Key) - require.Equal(t, want.Topic, got.Topic) - - wantPayload := want.Payload.(*NodeRegistrationEvent) - gotPayload := got.Payload.(*NodeRegistrationEvent) + wantPayload := want.Payload.(*NodeEvent) + gotPayload := got.Payload.(*NodeEvent) // Check payload equality for the fields that we can easily control - require.Equal(t, wantPayload.NodeStatus, gotPayload.NodeStatus) - require.Equal(t, wantPayload.Event.Message, gotPayload.Event.Message) - require.Equal(t, wantPayload.Event.Subsystem, gotPayload.Event.Subsystem) + require.Equal(t, wantPayload.Node.Status, gotPayload.Node.Status) + require.Equal(t, wantPayload.Node.ID, gotPayload.Node.ID) + require.NotEqual(t, wantPayload.Node.Events, gotPayload.Node.Events) } func requireNodeDeregistrationEventEqual(t *testing.T, want, got stream.Event) { t.Helper() - require.Equal(t, want.Index, got.Index) - require.Equal(t, want.Key, got.Key) - require.Equal(t, want.Topic, got.Topic) + wantPayload := want.Payload.(*NodeEvent) + gotPayload := got.Payload.(*NodeEvent) + + require.Equal(t, wantPayload.Node.ID, gotPayload.Node.ID) + require.NotEqual(t, wantPayload.Node.Events, gotPayload.Node.Events) +} - wantPayload := want.Payload.(*NodeDeregistrationEvent) - gotPayload := got.Payload.(*NodeDeregistrationEvent) +func requireNodeEventEqual(t *testing.T, want, got stream.Event) { + gotPayload := got.Payload.(*NodeEvent) - require.Equal(t, wantPayload, gotPayload) + require.Len(t, gotPayload.Node.Events, 3) } type nodeOpts func(n *structs.Node) @@ -186,6 +296,10 @@ func nodeNotReady(n *structs.Node) { n.Status = structs.NodeStatusInit } +func nodeReady(n *structs.Node) { + n.Status = structs.NodeStatusReady +} + func nodeIDTwo(n *structs.Node) { n.ID = testNodeIDTwo() } diff --git a/nomad/state/state_changes.go b/nomad/state/state_changes.go index be80c53c6bb9..ad5e77e0459a 100644 --- a/nomad/state/state_changes.go +++ b/nomad/state/state_changes.go @@ -190,6 +190,18 @@ func processDBChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { return NodeRegisterEventFromChanges(tx, changes) case structs.NodeDeregisterRequestType: return NodeDeregisterEventFromChanges(tx, changes) + case structs.NodeUpdateDrainRequestType: + return NodeDrainEventFromChanges(tx, changes) + case structs.UpsertNodeEventsType: + return NodeEventFromChanges(tx, changes) + case structs.DeploymentStatusUpdateRequestType: + return DeploymentEventFromChanges(changes.MsgType, tx, changes) + case structs.DeploymentPromoteRequestType: + return DeploymentEventFromChanges(changes.MsgType, tx, changes) + case structs.DeploymentAllocHealthRequestType: + return DeploymentEventFromChanges(changes.MsgType, tx, changes) + case structs.ApplyPlanResultsRequestType: + return ApplyPlanResultEventsFromChanges(tx, changes) } return []stream.Event{}, nil } diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index b1902eb55f07..68f85ca80fdd 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -270,7 +270,7 @@ RUN_QUERY: } // UpsertPlanResults is used to upsert the results of a plan. -func (s *StateStore) UpsertPlanResults(index uint64, results *structs.ApplyPlanResultsRequest) error { +func (s *StateStore) UpsertPlanResults(ctx context.Context, index uint64, results *structs.ApplyPlanResultsRequest) error { snapshot, err := s.Snapshot() if err != nil { return err @@ -292,7 +292,7 @@ func (s *StateStore) UpsertPlanResults(index uint64, results *structs.ApplyPlanR return err } - txn := s.db.WriteTxn(index) + txn := s.db.WriteTxnCtx(ctx, index) defer txn.Abort() // Upsert the newly created or updated deployment @@ -356,7 +356,10 @@ func (s *StateStore) UpsertPlanResults(index uint64, results *structs.ApplyPlanR } } - txn.Commit() + if err := txn.Commit(); err != nil { + return err + } + return nil } @@ -956,6 +959,19 @@ func (s *StateStore) BatchUpdateNodeDrain(index uint64, updatedAt int64, updates return nil } +// UpdateNodeDrain is used to update the drain of a node +func (s *StateStore) UpdateNodeDrainCtx(ctx context.Context, index uint64, nodeID string, + drain *structs.DrainStrategy, markEligible bool, updatedAt int64, event *structs.NodeEvent) error { + + txn := s.db.WriteTxnCtx(ctx, index) + defer txn.Abort() + if err := s.updateNodeDrainImpl(txn, index, nodeID, drain, markEligible, updatedAt, event); err != nil { + return err + } + txn.Commit() + return nil +} + // UpdateNodeDrain is used to update the drain of a node func (s *StateStore) UpdateNodeDrain(index uint64, nodeID string, drain *structs.DrainStrategy, markEligible bool, updatedAt int64, event *structs.NodeEvent) error { @@ -1059,6 +1075,20 @@ func (s *StateStore) UpdateNodeEligibility(index uint64, nodeID string, eligibil return nil } +func (s *StateStore) UpsertNodeEventsCtx(ctx context.Context, index uint64, nodeEvents map[string][]*structs.NodeEvent) error { + txn := s.db.WriteTxnCtx(ctx, index) + defer txn.Abort() + + for nodeID, events := range nodeEvents { + if err := s.upsertNodeEvents(index, nodeID, events, txn); err != nil { + return err + } + } + + txn.Commit() + return nil +} + // UpsertNodeEvents adds the node events to the nodes, rotating events as // necessary. func (s *StateStore) UpsertNodeEvents(index uint64, nodeEvents map[string][]*structs.NodeEvent) error { @@ -3375,6 +3405,10 @@ func (s *StateStore) AllocsByIDPrefixInNSes(ws memdb.WatchSet, namespaces map[st func (s *StateStore) AllocsByNode(ws memdb.WatchSet, node string) ([]*structs.Allocation, error) { txn := s.db.ReadTxn() + return allocsByNodeTxn(txn, ws, node) +} + +func allocsByNodeTxn(txn ReadTxn, ws memdb.WatchSet, node string) ([]*structs.Allocation, error) { // Get an iterator over the node allocations, using only the // node prefix which ignores the terminal status iter, err := txn.Get("allocs", "node_prefix", node) @@ -3796,8 +3830,8 @@ func (s *StateStore) SITokenAccessorsByNode(ws memdb.WatchSet, nodeID string) ([ // UpdateDeploymentStatus is used to make deployment status updates and // potentially make a evaluation -func (s *StateStore) UpdateDeploymentStatus(index uint64, req *structs.DeploymentStatusUpdateRequest) error { - txn := s.db.WriteTxn(index) +func (s *StateStore) UpdateDeploymentStatus(ctx context.Context, index uint64, req *structs.DeploymentStatusUpdateRequest) error { + txn := s.db.WriteTxnCtx(ctx, index) defer txn.Abort() if err := s.updateDeploymentStatusImpl(index, req.DeploymentUpdate, txn); err != nil { @@ -3900,8 +3934,8 @@ func (s *StateStore) updateJobStabilityImpl(index uint64, namespace, jobID strin // UpdateDeploymentPromotion is used to promote canaries in a deployment and // potentially make a evaluation -func (s *StateStore) UpdateDeploymentPromotion(index uint64, req *structs.ApplyDeploymentPromoteRequest) error { - txn := s.db.WriteTxn(index) +func (s *StateStore) UpdateDeploymentPromotion(ctx context.Context, index uint64, req *structs.ApplyDeploymentPromoteRequest) error { + txn := s.db.WriteTxnCtx(ctx, index) defer txn.Abort() // Retrieve deployment and ensure it is not terminal and is active @@ -4043,8 +4077,8 @@ func (s *StateStore) UpdateDeploymentPromotion(index uint64, req *structs.ApplyD // UpdateDeploymentAllocHealth is used to update the health of allocations as // part of the deployment and potentially make a evaluation -func (s *StateStore) UpdateDeploymentAllocHealth(index uint64, req *structs.ApplyDeploymentAllocHealthRequest) error { - txn := s.db.WriteTxn(index) +func (s *StateStore) UpdateDeploymentAllocHealth(ctx context.Context, index uint64, req *structs.ApplyDeploymentAllocHealthRequest) error { + txn := s.db.WriteTxnCtx(ctx, index) defer txn.Abort() // Retrieve deployment and ensure it is not terminal and is active diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index 6c3a16aff495..ba0e94b53df6 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -127,7 +127,7 @@ func TestStateStore_UpsertPlanResults_AllocationsCreated_Denormalized(t *testing EvalID: eval.ID, } assert := assert.New(t) - err := state.UpsertPlanResults(1000, &res) + err := state.UpsertPlanResults(context.Background(), 1000, &res) assert.Nil(err) ws := memdb.NewWatchSet() @@ -203,7 +203,7 @@ func TestStateStore_UpsertPlanResults_AllocationsDenormalized(t *testing.T) { } assert := assert.New(t) planModifyIndex := uint64(1000) - err := state.UpsertPlanResults(planModifyIndex, &res) + err := state.UpsertPlanResults(context.Background(), planModifyIndex, &res) require.NoError(err) ws := memdb.NewWatchSet() @@ -284,7 +284,7 @@ func TestStateStore_UpsertPlanResults_Deployment(t *testing.T) { EvalID: eval.ID, } - err := state.UpsertPlanResults(1000, &res) + err := state.UpsertPlanResults(context.Background(), 1000, &res) if err != nil { t.Fatalf("err: %v", err) } @@ -332,7 +332,7 @@ func TestStateStore_UpsertPlanResults_Deployment(t *testing.T) { EvalID: eval.ID, } - err = state.UpsertPlanResults(1001, &res) + err = state.UpsertPlanResults(context.Background(), 1001, &res) if err != nil { t.Fatalf("err: %v", err) } @@ -400,7 +400,7 @@ func TestStateStore_UpsertPlanResults_PreemptedAllocs(t *testing.T) { PreemptionEvals: []*structs.Evaluation{eval2}, } - err = state.UpsertPlanResults(1000, &res) + err = state.UpsertPlanResults(context.Background(), 1000, &res) require.NoError(err) ws := memdb.NewWatchSet() @@ -486,7 +486,7 @@ func TestStateStore_UpsertPlanResults_DeploymentUpdates(t *testing.T) { EvalID: eval.ID, } - err := state.UpsertPlanResults(1000, &res) + err := state.UpsertPlanResults(context.Background(), 1000, &res) if err != nil { t.Fatalf("err: %v", err) } @@ -6600,7 +6600,7 @@ func TestStateStore_UpsertDeploymentStatusUpdate_Nonexistent(t *testing.T) { Status: structs.DeploymentStatusRunning, }, } - err := state.UpdateDeploymentStatus(2, req) + err := state.UpdateDeploymentStatus(context.Background(), 2, req) if err == nil || !strings.Contains(err.Error(), "does not exist") { t.Fatalf("expected error updating the status because the deployment doesn't exist") } @@ -6627,7 +6627,7 @@ func TestStateStore_UpsertDeploymentStatusUpdate_Terminal(t *testing.T) { Status: structs.DeploymentStatusRunning, }, } - err := state.UpdateDeploymentStatus(2, req) + err := state.UpdateDeploymentStatus(context.Background(), 2, req) if err == nil || !strings.Contains(err.Error(), "has terminal status") { t.Fatalf("expected error updating the status because the deployment is terminal") } @@ -6661,7 +6661,7 @@ func TestStateStore_UpsertDeploymentStatusUpdate_NonTerminal(t *testing.T) { Job: j, Eval: e, } - err := state.UpdateDeploymentStatus(2, req) + err := state.UpdateDeploymentStatus(context.Background(), 2, req) if err != nil { t.Fatalf("bad: %v", err) } @@ -6722,7 +6722,7 @@ func TestStateStore_UpsertDeploymentStatusUpdate_Successful(t *testing.T) { StatusDescription: structs.DeploymentStatusDescriptionSuccessful, }, } - err := state.UpdateDeploymentStatus(3, req) + err := state.UpdateDeploymentStatus(context.Background(), 3, req) if err != nil { t.Fatalf("bad: %v", err) } @@ -6820,7 +6820,7 @@ func TestStateStore_UpsertDeploymentPromotion_Nonexistent(t *testing.T) { All: true, }, } - err := state.UpdateDeploymentPromotion(2, req) + err := state.UpdateDeploymentPromotion(context.Background(), 2, req) if err == nil || !strings.Contains(err.Error(), "does not exist") { t.Fatalf("expected error promoting because the deployment doesn't exist") } @@ -6847,7 +6847,7 @@ func TestStateStore_UpsertDeploymentPromotion_Terminal(t *testing.T) { All: true, }, } - err := state.UpdateDeploymentPromotion(2, req) + err := state.UpdateDeploymentPromotion(context.Background(), 2, req) if err == nil || !strings.Contains(err.Error(), "has terminal status") { t.Fatalf("expected error updating the status because the deployment is terminal: %v", err) } @@ -6897,7 +6897,7 @@ func TestStateStore_UpsertDeploymentPromotion_Unhealthy(t *testing.T) { All: true, }, } - err := state.UpdateDeploymentPromotion(4, req) + err := state.UpdateDeploymentPromotion(context.Background(), 4, req) require.NotNil(err) require.Contains(err.Error(), `Task group "web" has 0/2 healthy allocations`) } @@ -6926,7 +6926,7 @@ func TestStateStore_UpsertDeploymentPromotion_NoCanaries(t *testing.T) { All: true, }, } - err := state.UpdateDeploymentPromotion(4, req) + err := state.UpdateDeploymentPromotion(context.Background(), 4, req) require.NotNil(err) require.Contains(err.Error(), `Task group "web" has 0/2 healthy allocations`) } @@ -6997,7 +6997,7 @@ func TestStateStore_UpsertDeploymentPromotion_All(t *testing.T) { }, Eval: e, } - err := state.UpdateDeploymentPromotion(4, req) + err := state.UpdateDeploymentPromotion(context.Background(), 4, req) if err != nil { t.Fatalf("bad: %v", err) } @@ -7103,7 +7103,7 @@ func TestStateStore_UpsertDeploymentPromotion_Subset(t *testing.T) { }, Eval: e, } - require.Nil(state.UpdateDeploymentPromotion(4, req)) + require.Nil(state.UpdateDeploymentPromotion(context.Background(), 4, req)) // Check that the status per task group was updated properly ws := memdb.NewWatchSet() @@ -7146,7 +7146,7 @@ func TestStateStore_UpsertDeploymentAllocHealth_Nonexistent(t *testing.T) { HealthyAllocationIDs: []string{uuid.Generate()}, }, } - err := state.UpdateDeploymentAllocHealth(2, req) + err := state.UpdateDeploymentAllocHealth(context.Background(), 2, req) if err == nil || !strings.Contains(err.Error(), "does not exist") { t.Fatalf("expected error because the deployment doesn't exist: %v", err) } @@ -7173,7 +7173,7 @@ func TestStateStore_UpsertDeploymentAllocHealth_Terminal(t *testing.T) { HealthyAllocationIDs: []string{uuid.Generate()}, }, } - err := state.UpdateDeploymentAllocHealth(2, req) + err := state.UpdateDeploymentAllocHealth(context.Background(), 2, req) if err == nil || !strings.Contains(err.Error(), "has terminal status") { t.Fatalf("expected error because the deployment is terminal: %v", err) } @@ -7198,7 +7198,7 @@ func TestStateStore_UpsertDeploymentAllocHealth_BadAlloc_Nonexistent(t *testing. HealthyAllocationIDs: []string{uuid.Generate()}, }, } - err := state.UpdateDeploymentAllocHealth(2, req) + err := state.UpdateDeploymentAllocHealth(context.Background(), 2, req) if err == nil || !strings.Contains(err.Error(), "unknown alloc") { t.Fatalf("expected error because the alloc doesn't exist: %v", err) } @@ -7338,7 +7338,7 @@ func TestStateStore_UpsertDeploymentAllocHealth_BadAlloc_MismatchDeployment(t *t HealthyAllocationIDs: []string{a.ID}, }, } - err := state.UpdateDeploymentAllocHealth(4, req) + err := state.UpdateDeploymentAllocHealth(context.Background(), 4, req) if err == nil || !strings.Contains(err.Error(), "not part of deployment") { t.Fatalf("expected error because the alloc isn't part of the deployment: %v", err) } @@ -7395,7 +7395,7 @@ func TestStateStore_UpsertDeploymentAllocHealth(t *testing.T) { DeploymentUpdate: u, Timestamp: ts, } - err := state.UpdateDeploymentAllocHealth(3, req) + err := state.UpdateDeploymentAllocHealth(context.Background(), 3, req) if err != nil { t.Fatalf("bad: %v", err) } diff --git a/nomad/stream/event.go b/nomad/stream/event.go index 2625dede2049..33679723e177 100644 --- a/nomad/stream/event.go +++ b/nomad/stream/event.go @@ -7,8 +7,15 @@ const ( type Topic string type Event struct { - Topic Topic - Key string - Index uint64 - Payload interface{} + Topic Topic + Type string + Key string + FilterKeys []string + Index uint64 + Payload interface{} +} + +type Events struct { + Index uint64 + Events []Event } diff --git a/nomad/stream/event_publisher.go b/nomad/stream/event_publisher.go index aedc4bc101b5..0a6bb79cdfc4 100644 --- a/nomad/stream/event_publisher.go +++ b/nomad/stream/event_publisher.go @@ -65,7 +65,7 @@ func NewEventPublisher(ctx context.Context, cfg EventPublisherCfg) *EventPublish e := &EventPublisher{ logger: cfg.Logger.Named("event_publisher"), eventBuf: buffer, - publishCh: make(chan changeEvents), + publishCh: make(chan changeEvents, 64), subscriptions: &subscriptions{ byToken: make(map[string]map[*SubscribeRequest]*Subscription), }, diff --git a/nomad/stream/event_publisher_test.go b/nomad/stream/event_publisher_test.go index 5085a6fd970b..691574a65a9a 100644 --- a/nomad/stream/event_publisher_test.go +++ b/nomad/stream/event_publisher_test.go @@ -86,7 +86,7 @@ func consumeSubscription(ctx context.Context, sub *Subscription) <-chan subNextR for { es, err := sub.Next(ctx) eventCh <- subNextResult{ - Events: es, + Events: es.Events, Err: err, } if err != nil { diff --git a/nomad/stream/subscription.go b/nomad/stream/subscription.go index 0fc64512c19a..bae3d091ac2e 100644 --- a/nomad/stream/subscription.go +++ b/nomad/stream/subscription.go @@ -58,18 +58,38 @@ func newSubscription(req *SubscribeRequest, item *bufferItem, unsub func()) *Sub } } -func (s *Subscription) Next(ctx context.Context) ([]Event, error) { +func (s *Subscription) Next(ctx context.Context) (Events, error) { if atomic.LoadUint32(&s.state) == subscriptionStateClosed { - return nil, ErrSubscriptionClosed + return Events{}, ErrSubscriptionClosed } for { next, err := s.currentItem.Next(ctx, s.forceClosed) switch { case err != nil && atomic.LoadUint32(&s.state) == subscriptionStateClosed: - return nil, ErrSubscriptionClosed + return Events{}, ErrSubscriptionClosed case err != nil: - return nil, err + return Events{}, err + } + s.currentItem = next + + events := filter(s.req, next.Events) + if len(events) == 0 { + continue + } + return Events{Index: next.Index, Events: events}, nil + } +} + +func (s *Subscription) NextNoBlock() ([]Event, error) { + if atomic.LoadUint32(&s.state) == subscriptionStateClosed { + return nil, ErrSubscriptionClosed + } + + for { + next := s.currentItem.NextNoBlock() + if next == nil { + return nil, nil } s.currentItem = next diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 653aadc0c30c..6638dc3b358b 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -8989,6 +8989,15 @@ func (a *Allocation) ReschedulePolicy() *ReschedulePolicy { return tg.ReschedulePolicy } +// MigrateStrategy returns the migrate strategy based on the task group +func (a *Allocation) MigrateStrategy() *MigrateStrategy { + tg := a.Job.LookupTaskGroup(a.TaskGroup) + if tg == nil { + return nil + } + return tg.Migrate +} + // NextRescheduleTime returns a time on or after which the allocation is eligible to be rescheduled, // and whether the next reschedule time is within policy's interval if the policy doesn't allow unlimited reschedules func (a *Allocation) NextRescheduleTime() (time.Time, bool) { diff --git a/scheduler/testing.go b/scheduler/testing.go index c1d8776b4ad3..9b43be42f0db 100644 --- a/scheduler/testing.go +++ b/scheduler/testing.go @@ -1,6 +1,7 @@ package scheduler import ( + "context" "fmt" "sync" "time" @@ -170,7 +171,7 @@ func (h *Harness) SubmitPlan(plan *structs.Plan) (*structs.PlanResult, State, er } // Apply the full plan - err := h.State.UpsertPlanResults(index, &req) + err := h.State.UpsertPlanResults(context.Background(), index, &req) return result, nil, err } From 4f97bf8ef7a9649225d747f526137129ba67040a Mon Sep 17 00:00:00 2001 From: Drew Bailey <2614075+drewbailey@users.noreply.github.com> Date: Fri, 2 Oct 2020 14:23:30 -0400 Subject: [PATCH 07/19] Events/eval alloc events (#9012) * generic eval update event first pass at alloc client update events * api/event client --- api/event.go | 93 +++++++++++++++++++ api/event_test.go | 50 ++++++++++ nomad/alloc_endpoint_test.go | 3 +- nomad/deployment_endpoint_test.go | 3 +- .../deployments_watcher_test.go | 6 +- nomad/drainer_int_test.go | 2 +- nomad/fsm.go | 26 +++--- nomad/node_endpoint_test.go | 3 +- nomad/state/events.go | 57 ++++++++++++ nomad/state/state_changes.go | 4 + nomad/state/state_store.go | 16 +++- nomad/state/state_store_test.go | 32 +++---- scheduler/generic_sched_test.go | 7 +- scheduler/system_sched_test.go | 3 +- 14 files changed, 265 insertions(+), 40 deletions(-) create mode 100644 api/event.go create mode 100644 api/event_test.go diff --git a/api/event.go b/api/event.go new file mode 100644 index 000000000000..e9009419e83f --- /dev/null +++ b/api/event.go @@ -0,0 +1,93 @@ +package api + +import ( + "context" + "encoding/json" + "fmt" +) + +type Events struct { + Index uint64 + Events []Event +} + +type Topic string + +type Event struct { + Topic Topic + Type string + Key string + FilterKeys []string + Index uint64 + Payload interface{} +} + +func (e *Events) IsHeartBeat() bool { + return e.Index == 0 && len(e.Events) == 0 +} + +type EventStream struct { + client *Client +} + +func (c *Client) EventStream() *EventStream { + return &EventStream{client: c} +} + +func (e *EventStream) Stream(ctx context.Context, topics map[Topic][]string, index uint64, q *QueryOptions) (<-chan *Events, <-chan error) { + + errCh := make(chan error, 1) + + r, err := e.client.newRequest("GET", "/v1/event/stream") + if err != nil { + errCh <- err + return nil, errCh + } + r.setQueryOptions(q) + + // Build topic query params + for topic, keys := range topics { + for _, k := range keys { + r.params.Add("topic", fmt.Sprintf("%s:%s", topic, k)) + } + } + + _, resp, err := requireOK(e.client.doRequest(r)) + + if err != nil { + errCh <- err + return nil, errCh + } + + eventsCh := make(chan *Events, 10) + go func() { + defer resp.Body.Close() + + dec := json.NewDecoder(resp.Body) + + for { + select { + case <-ctx.Done(): + close(eventsCh) + return + default: + } + + // Decode next newline delimited json of events + var events Events + if err := dec.Decode(&events); err != nil { + close(eventsCh) + errCh <- err + return + } + if events.IsHeartBeat() { + continue + } + + eventsCh <- &events + + } + }() + + return eventsCh, errCh +} diff --git a/api/event_test.go b/api/event_test.go new file mode 100644 index 000000000000..1ea1c6e4274b --- /dev/null +++ b/api/event_test.go @@ -0,0 +1,50 @@ +package api + +import ( + "context" + "testing" + "time" + + "github.com/stretchr/testify/require" +) + +func TestEvent_Stream(t *testing.T) { + t.Parallel() + + c, s := makeClient(t, nil, nil) + defer s.Stop() + + // register job to generate events + jobs := c.Jobs() + job := testJob() + resp2, _, err := jobs.Register(job, nil) + require.Nil(t, err) + require.NotNil(t, resp2) + + // build event stream request + events := c.EventStream() + q := &QueryOptions{} + topics := map[Topic][]string{ + "Eval": {"*"}, + } + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + streamCh, errCh := events.Stream(ctx, topics, 0, q) + +OUTER: + for { + select { + case event := <-streamCh: + require.Equal(t, len(event.Events), 1) + require.Equal(t, "Eval", string(event.Events[0].Topic)) + + break OUTER + case err := <-errCh: + require.Fail(t, err.Error()) + case <-time.After(5 * time.Second): + require.Fail(t, "failed waiting for event stream event") + } + } +} diff --git a/nomad/alloc_endpoint_test.go b/nomad/alloc_endpoint_test.go index d9123b03f56c..012e95dfaf4c 100644 --- a/nomad/alloc_endpoint_test.go +++ b/nomad/alloc_endpoint_test.go @@ -1,6 +1,7 @@ package nomad import ( + "context" "reflect" "testing" "time" @@ -193,7 +194,7 @@ func TestAllocEndpoint_List_Blocking(t *testing.T) { alloc2.ClientStatus = structs.AllocClientStatusRunning time.AfterFunc(100*time.Millisecond, func() { state.UpsertJobSummary(3, mock.JobSummary(alloc2.JobID)) - if err := state.UpdateAllocsFromClient(4, []*structs.Allocation{alloc2}); err != nil { + if err := state.UpdateAllocsFromClient(context.Background(), 4, []*structs.Allocation{alloc2}); err != nil { t.Fatalf("err: %v", err) } }) diff --git a/nomad/deployment_endpoint_test.go b/nomad/deployment_endpoint_test.go index 88b85620d447..ad7d222dc56d 100644 --- a/nomad/deployment_endpoint_test.go +++ b/nomad/deployment_endpoint_test.go @@ -1,6 +1,7 @@ package nomad import ( + "context" "testing" "time" @@ -1300,7 +1301,7 @@ func TestDeploymentEndpoint_Allocations_Blocking(t *testing.T) { a2.ClientStatus = structs.AllocClientStatusRunning time.AfterFunc(100*time.Millisecond, func() { assert.Nil(state.UpsertJobSummary(5, mock.JobSummary(a2.JobID)), "UpsertJobSummary") - assert.Nil(state.UpdateAllocsFromClient(6, []*structs.Allocation{a2}), "updateAllocsFromClient") + assert.Nil(state.UpdateAllocsFromClient(context.Background(), 6, []*structs.Allocation{a2}), "updateAllocsFromClient") }) req.MinQueryIndex = 4 diff --git a/nomad/deploymentwatcher/deployments_watcher_test.go b/nomad/deploymentwatcher/deployments_watcher_test.go index c4c503f35c21..d1285d76986e 100644 --- a/nomad/deploymentwatcher/deployments_watcher_test.go +++ b/nomad/deploymentwatcher/deployments_watcher_test.go @@ -1025,7 +1025,7 @@ func TestDeploymentWatcher_Watch_ProgressDeadline(t *testing.T) { Healthy: helper.BoolToPtr(false), Timestamp: now, } - require.Nil(m.state.UpdateAllocsFromClient(100, []*structs.Allocation{a2})) + require.Nil(m.state.UpdateAllocsFromClient(context.Background(), 100, []*structs.Allocation{a2})) // Wait for the deployment to be failed testutil.WaitForResult(func() (bool, error) { @@ -1209,7 +1209,7 @@ func TestDeploymentWatcher_Watch_ProgressDeadline_Canaries(t *testing.T) { Healthy: helper.BoolToPtr(true), Timestamp: now, } - require.Nil(m.state.UpdateAllocsFromClient(m.nextIndex(), []*structs.Allocation{a2})) + require.Nil(m.state.UpdateAllocsFromClient(context.Background(), m.nextIndex(), []*structs.Allocation{a2})) // Wait for the deployment to cross the deadline dout, err := m.state.DeploymentByID(nil, d.ID) @@ -1382,7 +1382,7 @@ func TestDeploymentWatcher_Watch_StartWithoutProgressDeadline(t *testing.T) { Healthy: helper.BoolToPtr(false), Timestamp: time.Now(), } - require.Nil(m.state.UpdateAllocsFromClient(m.nextIndex(), []*structs.Allocation{a2})) + require.Nil(m.state.UpdateAllocsFromClient(context.Background(), m.nextIndex(), []*structs.Allocation{a2})) // Wait for the alloc's DesiredState to set reschedule testutil.WaitForResult(func() (bool, error) { diff --git a/nomad/drainer_int_test.go b/nomad/drainer_int_test.go index ce480bd33ed9..49896c7ce96d 100644 --- a/nomad/drainer_int_test.go +++ b/nomad/drainer_int_test.go @@ -673,7 +673,7 @@ func TestDrainer_AllTypes_NoDeadline(t *testing.T) { new.ClientStatus = structs.AllocClientStatusComplete updates = append(updates, new) } - require.Nil(state.UpdateAllocsFromClient(1000, updates)) + require.Nil(state.UpdateAllocsFromClient(context.Background(), 1000, updates)) // Check that the node drain is removed testutil.WaitForResult(func() (bool, error) { diff --git a/nomad/fsm.go b/nomad/fsm.go index c4e4aa5bf858..fd483434ac2a 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -212,13 +212,13 @@ func (n *nomadFSM) Apply(log *raft.Log) interface{} { case structs.JobDeregisterRequestType: return n.applyDeregisterJob(buf[1:], log.Index) case structs.EvalUpdateRequestType: - return n.applyUpdateEval(buf[1:], log.Index) + return n.applyUpdateEval(msgType, buf[1:], log.Index) case structs.EvalDeleteRequestType: return n.applyDeleteEval(buf[1:], log.Index) case structs.AllocUpdateRequestType: return n.applyAllocUpdate(buf[1:], log.Index) case structs.AllocClientUpdateRequestType: - return n.applyAllocClientUpdate(buf[1:], log.Index) + return n.applyAllocClientUpdate(msgType, buf[1:], log.Index) case structs.ReconcileJobSummariesRequestType: return n.applyReconcileSummaries(buf[1:], log.Index) case structs.VaultAccessorRegisterRequestType: @@ -570,7 +570,7 @@ func (n *nomadFSM) applyUpsertJob(buf []byte, index uint64) interface{} { // so this may be nil during server upgrades. if req.Eval != nil { req.Eval.JobModifyIndex = index - if err := n.upsertEvals(index, []*structs.Evaluation{req.Eval}); err != nil { + if err := n.upsertEvals(context.Background(), index, []*structs.Evaluation{req.Eval}); err != nil { return err } } @@ -602,7 +602,7 @@ func (n *nomadFSM) applyDeregisterJob(buf []byte, index uint64) interface{} { // always attempt upsert eval even if job deregister fail if req.Eval != nil { req.Eval.JobModifyIndex = index - if err := n.upsertEvals(index, []*structs.Evaluation{req.Eval}); err != nil { + if err := n.upsertEvals(context.Background(), index, []*structs.Evaluation{req.Eval}); err != nil { return err } } @@ -689,17 +689,20 @@ func (n *nomadFSM) handleJobDeregister(index uint64, jobID, namespace string, pu return nil } -func (n *nomadFSM) applyUpdateEval(buf []byte, index uint64) interface{} { +func (n *nomadFSM) applyUpdateEval(msgType structs.MessageType, buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "update_eval"}, time.Now()) var req structs.EvalUpdateRequest if err := structs.Decode(buf, &req); err != nil { panic(fmt.Errorf("failed to decode request: %v", err)) } - return n.upsertEvals(index, req.Evals) + + ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType) + + return n.upsertEvals(ctx, index, req.Evals) } -func (n *nomadFSM) upsertEvals(index uint64, evals []*structs.Evaluation) error { - if err := n.state.UpsertEvals(index, evals); err != nil { +func (n *nomadFSM) upsertEvals(ctx context.Context, index uint64, evals []*structs.Evaluation) error { + if err := n.state.UpsertEvalsCtx(ctx, index, evals); err != nil { n.logger.Error("UpsertEvals failed", "error", err) return err } @@ -786,7 +789,7 @@ func (n *nomadFSM) applyAllocUpdate(buf []byte, index uint64) interface{} { return nil } -func (n *nomadFSM) applyAllocClientUpdate(buf []byte, index uint64) interface{} { +func (n *nomadFSM) applyAllocClientUpdate(msgType structs.MessageType, buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "alloc_client_update"}, time.Now()) var req structs.AllocUpdateRequest if err := structs.Decode(buf, &req); err != nil { @@ -807,15 +810,16 @@ func (n *nomadFSM) applyAllocClientUpdate(buf []byte, index uint64) interface{} } } + ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType) // Update all the client allocations - if err := n.state.UpdateAllocsFromClient(index, req.Alloc); err != nil { + if err := n.state.UpdateAllocsFromClient(ctx, index, req.Alloc); err != nil { n.logger.Error("UpdateAllocFromClient failed", "error", err) return err } // Update any evals if len(req.Evals) > 0 { - if err := n.upsertEvals(index, req.Evals); err != nil { + if err := n.upsertEvals(ctx, index, req.Evals); err != nil { n.logger.Error("applyAllocClientUpdate failed to update evaluations", "error", err) return err } diff --git a/nomad/node_endpoint_test.go b/nomad/node_endpoint_test.go index 1be5728c598b..db1b46a5c940 100644 --- a/nomad/node_endpoint_test.go +++ b/nomad/node_endpoint_test.go @@ -1,6 +1,7 @@ package nomad import ( + "context" "errors" "fmt" "net" @@ -2048,7 +2049,7 @@ func TestClientEndpoint_GetAllocs_Blocking(t *testing.T) { allocUpdate.ID = alloc.ID allocUpdate.ClientStatus = structs.AllocClientStatusRunning state.UpsertJobSummary(199, mock.JobSummary(allocUpdate.JobID)) - err := state.UpdateAllocsFromClient(200, []*structs.Allocation{allocUpdate}) + err := state.UpdateAllocsFromClient(context.Background(), 200, []*structs.Allocation{allocUpdate}) if err != nil { t.Fatalf("err: %v", err) } diff --git a/nomad/state/events.go b/nomad/state/events.go index 824623409f62..921d7a8947ea 100644 --- a/nomad/state/events.go +++ b/nomad/state/events.go @@ -1,6 +1,8 @@ package state import ( + "fmt" + "github.com/hashicorp/nomad/nomad/stream" "github.com/hashicorp/nomad/nomad/structs" ) @@ -27,6 +29,8 @@ const ( TypeAllocCreated = "AllocCreated" TypeAllocUpdated = "AllocUpdated" + + TypeEvalUpdated = "EvalUpdated" ) type JobEvent struct { @@ -66,3 +70,56 @@ type JobDrainDetails struct { Type string AllocDetails map[string]NodeDrainAllocDetails } + +func GenericEventsFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { + var eventType string + switch changes.MsgType { + case structs.EvalUpdateRequestType: + eventType = TypeEvalUpdated + case structs.AllocClientUpdateRequestType: + eventType = TypeAllocUpdated + } + + var events []stream.Event + for _, change := range changes.Changes { + switch change.Table { + case "evals": + after, ok := change.After.(*structs.Evaluation) + if !ok { + return nil, fmt.Errorf("transaction change was not an Evaluation") + } + + event := stream.Event{ + Topic: TopicEval, + Type: eventType, + Index: changes.Index, + Key: after.ID, + Payload: &EvalEvent{ + Eval: after, + }, + } + + events = append(events, event) + + case "allocs": + after, ok := change.After.(*structs.Allocation) + if !ok { + return nil, fmt.Errorf("transaction change was not an Allocation") + } + + event := stream.Event{ + Topic: TopicAlloc, + Type: eventType, + Index: changes.Index, + Key: after.ID, + Payload: &AllocEvent{ + Alloc: after, + }, + } + + events = append(events, event) + } + } + + return events, nil +} diff --git a/nomad/state/state_changes.go b/nomad/state/state_changes.go index ad5e77e0459a..14b7b7cd1307 100644 --- a/nomad/state/state_changes.go +++ b/nomad/state/state_changes.go @@ -202,6 +202,10 @@ func processDBChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { return DeploymentEventFromChanges(changes.MsgType, tx, changes) case structs.ApplyPlanResultsRequestType: return ApplyPlanResultEventsFromChanges(tx, changes) + case structs.EvalUpdateRequestType: + return GenericEventsFromChanges(tx, changes) + case structs.AllocClientUpdateRequestType: + return GenericEventsFromChanges(tx, changes) } return []stream.Event{}, nil } diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 68f85ca80fdd..295b414f63c8 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -2705,6 +2705,18 @@ func (s *StateStore) UpsertEvals(index uint64, evals []*structs.Evaluation) erro return err } +// UpsertEvals is used to upsert a set of evaluations +func (s *StateStore) UpsertEvalsCtx(ctx context.Context, index uint64, evals []*structs.Evaluation) error { + txn := s.db.WriteTxnCtx(ctx, index) + defer txn.Abort() + + err := s.UpsertEvalsTxn(index, evals, txn) + if err == nil { + txn.Commit() + } + return err +} + // UpsertEvals is used to upsert a set of evaluations, like UpsertEvals // but in a transaction. Useful for when making multiple modifications atomically func (s *StateStore) UpsertEvalsTxn(index uint64, evals []*structs.Evaluation, txn Txn) error { @@ -3017,8 +3029,8 @@ func (s *StateStore) EvalsByNamespace(ws memdb.WatchSet, namespace string) (memd // most things, some updates are authoritative from the client. Specifically, // the desired state comes from the schedulers, while the actual state comes // from clients. -func (s *StateStore) UpdateAllocsFromClient(index uint64, allocs []*structs.Allocation) error { - txn := s.db.WriteTxn(index) +func (s *StateStore) UpdateAllocsFromClient(ctx context.Context, index uint64, allocs []*structs.Allocation) error { + txn := s.db.WriteTxnCtx(ctx, index) defer txn.Abort() // Handle each of the updated allocations diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index ba0e94b53df6..c683606015c3 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -4467,7 +4467,7 @@ func TestStateStore_UpdateAllocsFromClient(t *testing.T) { JobID: alloc.JobID, TaskGroup: alloc.TaskGroup, } - err = state.UpdateAllocsFromClient(1001, []*structs.Allocation{update}) + err = state.UpdateAllocsFromClient(context.Background(), 1001, []*structs.Allocation{update}) if err != nil { t.Fatalf("err: %v", err) } @@ -4565,7 +4565,7 @@ func TestStateStore_UpdateAllocsFromClient_ChildJob(t *testing.T) { TaskGroup: alloc2.TaskGroup, } - err = state.UpdateAllocsFromClient(1001, []*structs.Allocation{update, update2}) + err = state.UpdateAllocsFromClient(context.Background(), 1001, []*structs.Allocation{update, update2}) if err != nil { t.Fatalf("err: %v", err) } @@ -4666,7 +4666,7 @@ func TestStateStore_UpdateMultipleAllocsFromClient(t *testing.T) { TaskGroup: alloc.TaskGroup, } - err = state.UpdateAllocsFromClient(1001, []*structs.Allocation{update, update2}) + err = state.UpdateAllocsFromClient(context.Background(), 1001, []*structs.Allocation{update, update2}) if err != nil { t.Fatalf("err: %v", err) } @@ -4735,7 +4735,7 @@ func TestStateStore_UpdateAllocsFromClient_Deployment(t *testing.T) { Timestamp: healthy, }, } - require.Nil(state.UpdateAllocsFromClient(1001, []*structs.Allocation{update})) + require.Nil(state.UpdateAllocsFromClient(context.Background(), 1001, []*structs.Allocation{update})) // Check that the deployment state was updated because the healthy // deployment @@ -4780,7 +4780,7 @@ func TestStateStore_UpdateAllocsFromClient_DeploymentStateMerges(t *testing.T) { Canary: false, }, } - require.Nil(state.UpdateAllocsFromClient(1001, []*structs.Allocation{update})) + require.Nil(state.UpdateAllocsFromClient(context.Background(), 1001, []*structs.Allocation{update})) // Check that the merging of the deployment status was correct out, err := state.AllocByID(nil, alloc.ID) @@ -5161,7 +5161,7 @@ func TestStateStore_UpdateAlloc_NoJob(t *testing.T) { // Update the client state of the allocation to complete allocCopy1 := allocCopy.Copy() allocCopy1.ClientStatus = structs.AllocClientStatusComplete - if err := state.UpdateAllocsFromClient(1003, []*structs.Allocation{allocCopy1}); err != nil { + if err := state.UpdateAllocsFromClient(context.Background(), 1003, []*structs.Allocation{allocCopy1}); err != nil { t.Fatalf("err: %v", err) } @@ -5272,12 +5272,12 @@ func TestStateStore_JobSummary(t *testing.T) { alloc1 := alloc.Copy() alloc1.ClientStatus = structs.AllocClientStatusPending alloc1.DesiredStatus = "" - state.UpdateAllocsFromClient(920, []*structs.Allocation{alloc}) + state.UpdateAllocsFromClient(context.Background(), 920, []*structs.Allocation{alloc}) alloc3 := alloc.Copy() alloc3.ClientStatus = structs.AllocClientStatusRunning alloc3.DesiredStatus = "" - state.UpdateAllocsFromClient(930, []*structs.Allocation{alloc3}) + state.UpdateAllocsFromClient(context.Background(), 930, []*structs.Allocation{alloc3}) // Upsert the alloc alloc4 := alloc.Copy() @@ -5320,7 +5320,7 @@ func TestStateStore_JobSummary(t *testing.T) { alloc6 := alloc.Copy() alloc6.ClientStatus = structs.AllocClientStatusRunning alloc6.DesiredStatus = "" - state.UpdateAllocsFromClient(990, []*structs.Allocation{alloc6}) + state.UpdateAllocsFromClient(context.Background(), 990, []*structs.Allocation{alloc6}) // We shouldn't have any summary at this point summary, _ = state.JobSummaryByID(ws, job.Namespace, job.ID) @@ -5347,7 +5347,7 @@ func TestStateStore_JobSummary(t *testing.T) { alloc7.Job = outJob alloc7.ClientStatus = structs.AllocClientStatusComplete alloc7.DesiredStatus = structs.AllocDesiredStatusRun - state.UpdateAllocsFromClient(1020, []*structs.Allocation{alloc7}) + state.UpdateAllocsFromClient(context.Background(), 1020, []*structs.Allocation{alloc7}) expectedSummary = structs.JobSummary{ JobID: job.ID, @@ -5392,7 +5392,7 @@ func TestStateStore_ReconcileJobSummary(t *testing.T) { // Change the state of the first alloc to running alloc3 := alloc.Copy() alloc3.ClientStatus = structs.AllocClientStatusRunning - state.UpdateAllocsFromClient(120, []*structs.Allocation{alloc3}) + state.UpdateAllocsFromClient(context.Background(), 120, []*structs.Allocation{alloc3}) //Add some more allocs to the second tg alloc4 := mock.Alloc() @@ -5425,7 +5425,7 @@ func TestStateStore_ReconcileJobSummary(t *testing.T) { state.UpsertAllocs(130, []*structs.Allocation{alloc4, alloc6, alloc8, alloc10}) - state.UpdateAllocsFromClient(150, []*structs.Allocation{alloc5, alloc7, alloc9, alloc11}) + state.UpdateAllocsFromClient(context.Background(), 150, []*structs.Allocation{alloc5, alloc7, alloc9, alloc11}) // DeleteJobSummary is a helper method and doesn't modify the indexes table state.DeleteJobSummary(130, alloc.Namespace, alloc.Job.ID) @@ -5564,7 +5564,7 @@ func TestStateStore_UpdateAlloc_JobNotPresent(t *testing.T) { alloc1.ClientStatus = structs.AllocClientStatusRunning // Updating allocation should not throw any error - if err := state.UpdateAllocsFromClient(400, []*structs.Allocation{alloc1}); err != nil { + if err := state.UpdateAllocsFromClient(context.Background(), 400, []*structs.Allocation{alloc1}); err != nil { t.Fatalf("expect err: %v", err) } @@ -5574,7 +5574,7 @@ func TestStateStore_UpdateAlloc_JobNotPresent(t *testing.T) { // Update the alloc again alloc2 := alloc.Copy() alloc2.ClientStatus = structs.AllocClientStatusComplete - if err := state.UpdateAllocsFromClient(400, []*structs.Allocation{alloc1}); err != nil { + if err := state.UpdateAllocsFromClient(context.Background(), 400, []*structs.Allocation{alloc1}); err != nil { t.Fatalf("expect err: %v", err) } @@ -6484,7 +6484,7 @@ func TestStateJobSummary_UpdateJobCount(t *testing.T) { alloc5.JobID = alloc3.JobID alloc5.ClientStatus = structs.AllocClientStatusComplete - if err := state.UpdateAllocsFromClient(1004, []*structs.Allocation{alloc4, alloc5}); err != nil { + if err := state.UpdateAllocsFromClient(context.Background(), 1004, []*structs.Allocation{alloc4, alloc5}); err != nil { t.Fatalf("err: %v", err) } @@ -6561,7 +6561,7 @@ func TestJobSummary_UpdateClientStatus(t *testing.T) { alloc6.JobID = alloc.JobID alloc6.ClientStatus = structs.AllocClientStatusRunning - if err := state.UpdateAllocsFromClient(1002, []*structs.Allocation{alloc4, alloc5, alloc6}); err != nil { + if err := state.UpdateAllocsFromClient(context.Background(), 1002, []*structs.Allocation{alloc4, alloc5, alloc6}); err != nil { t.Fatalf("err: %v", err) } diff --git a/scheduler/generic_sched_test.go b/scheduler/generic_sched_test.go index 7861cc82b303..61a16326c687 100644 --- a/scheduler/generic_sched_test.go +++ b/scheduler/generic_sched_test.go @@ -1,6 +1,7 @@ package scheduler import ( + "context" "fmt" "reflect" "sort" @@ -2957,7 +2958,7 @@ func TestServiceSched_NodeUpdate(t *testing.T) { for i := 0; i < 4; i++ { out, _ := h.State.AllocByID(ws, allocs[i].ID) out.ClientStatus = structs.AllocClientStatusRunning - require.NoError(t, h.State.UpdateAllocsFromClient(h.NextIndex(), []*structs.Allocation{out})) + require.NoError(t, h.State.UpdateAllocsFromClient(context.Background(), h.NextIndex(), []*structs.Allocation{out})) } // Create a mock evaluation which won't trigger any new placements @@ -3107,7 +3108,7 @@ func TestServiceSched_NodeDrain_Down(t *testing.T) { newAlloc.ClientStatus = structs.AllocClientStatusRunning running = append(running, newAlloc) } - require.NoError(t, h.State.UpdateAllocsFromClient(h.NextIndex(), running)) + require.NoError(t, h.State.UpdateAllocsFromClient(context.Background(), h.NextIndex(), running)) // Mark some of the allocations as complete var complete []*structs.Allocation @@ -3126,7 +3127,7 @@ func TestServiceSched_NodeDrain_Down(t *testing.T) { newAlloc.ClientStatus = structs.AllocClientStatusComplete complete = append(complete, newAlloc) } - require.NoError(t, h.State.UpdateAllocsFromClient(h.NextIndex(), complete)) + require.NoError(t, h.State.UpdateAllocsFromClient(context.Background(), h.NextIndex(), complete)) // Create a mock evaluation to deal with the node update eval := &structs.Evaluation{ diff --git a/scheduler/system_sched_test.go b/scheduler/system_sched_test.go index cb964989f153..3ba722e1385d 100644 --- a/scheduler/system_sched_test.go +++ b/scheduler/system_sched_test.go @@ -1,6 +1,7 @@ package scheduler import ( + "context" "fmt" "reflect" "sort" @@ -132,7 +133,7 @@ func TestSystemSched_JobRegister_StickyAllocs(t *testing.T) { // Get an allocation and mark it as failed alloc := planned[4].Copy() alloc.ClientStatus = structs.AllocClientStatusFailed - require.NoError(t, h.State.UpdateAllocsFromClient(h.NextIndex(), []*structs.Allocation{alloc})) + require.NoError(t, h.State.UpdateAllocsFromClient(context.Background(), h.NextIndex(), []*structs.Allocation{alloc})) // Create a mock evaluation to handle the update eval = &structs.Evaluation{ From 8a57ee85f029a33dace266b85e6cbfc7d16b0b79 Mon Sep 17 00:00:00 2001 From: Drew Bailey <2614075+drewbailey@users.noreply.github.com> Date: Fri, 2 Oct 2020 14:37:37 -0400 Subject: [PATCH 08/19] api comments --- api/event.go | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/api/event.go b/api/event.go index e9009419e83f..b52f4eb8e8b9 100644 --- a/api/event.go +++ b/api/event.go @@ -6,13 +6,18 @@ import ( "fmt" ) +// Events is a set of events for a corresponding index. Events returned for the +// index depend on which topics are subscribed to when a request is made. type Events struct { Index uint64 Events []Event } +// Topic is an event Topic type Topic string +// Event holds information related to an event that occurred in Nomad. +// The Payload is a hydrated object related to the Topic type Event struct { Topic Topic Type string @@ -22,18 +27,24 @@ type Event struct { Payload interface{} } +// IsHeartBeat specifies if the event is an empty heartbeat used to +// keep a connection alive. func (e *Events) IsHeartBeat() bool { return e.Index == 0 && len(e.Events) == 0 } +// EventStream is used to stream events from Nomad type EventStream struct { client *Client } +// EventStream returns a handle to the Events endpoint func (c *Client) EventStream() *EventStream { return &EventStream{client: c} } +// Stream establishes a new subscription to Nomad's event stream and streams +// results back to the returned channel. func (e *EventStream) Stream(ctx context.Context, topics map[Topic][]string, index uint64, q *QueryOptions) (<-chan *Events, <-chan error) { errCh := make(chan error, 1) From e7e6df394f5d417f01611cb21c7b123d94ea67c5 Mon Sep 17 00:00:00 2001 From: Drew Bailey <2614075+drewbailey@users.noreply.github.com> Date: Fri, 2 Oct 2020 15:04:05 -0400 Subject: [PATCH 09/19] wire up enable_event_publisher --- command/agent/agent.go | 3 +++ command/agent/agent_test.go | 2 ++ command/agent/command.go | 1 + command/agent/config.go | 13 +++++++++++-- command/agent/config_parse_test.go | 1 + command/agent/testdata/basic.hcl | 1 + command/agent/testdata/basic.json | 1 + 7 files changed, 20 insertions(+), 2 deletions(-) diff --git a/command/agent/agent.go b/command/agent/agent.go index 016b77d16ebf..16f3c20d55dc 100644 --- a/command/agent/agent.go +++ b/command/agent/agent.go @@ -243,6 +243,9 @@ func convertServerConfig(agentConfig *Config) (*nomad.Config, error) { if agentConfig.Server.UpgradeVersion != "" { conf.UpgradeVersion = agentConfig.Server.UpgradeVersion } + if agentConfig.Server.EnableEventPublisher { + conf.EnableEventPublisher = agentConfig.Server.EnableEventPublisher + } if agentConfig.Autopilot != nil { if agentConfig.Autopilot.CleanupDeadServers != nil { conf.AutopilotConfig.CleanupDeadServers = *agentConfig.Autopilot.CleanupDeadServers diff --git a/command/agent/agent_test.go b/command/agent/agent_test.go index d9290bc59304..1ac900444656 100644 --- a/command/agent/agent_test.go +++ b/command/agent/agent_test.go @@ -57,6 +57,8 @@ func TestAgent_ServerConfig(t *testing.T) { out, err := a.serverConfig() require.NoError(t, err) + require.True(t, out.EnableEventPublisher) + serfAddr := out.SerfConfig.MemberlistConfig.AdvertiseAddr require.Equal(t, "127.0.0.1", serfAddr) diff --git a/command/agent/command.go b/command/agent/command.go index 0ed3ff33814e..199f47133541 100644 --- a/command/agent/command.go +++ b/command/agent/command.go @@ -94,6 +94,7 @@ func (c *Command) readConfig() *Config { flags.Var((*flaghelper.StringFlag)(&cmdConfig.Server.ServerJoin.StartJoin), "join", "") flags.Var((*flaghelper.StringFlag)(&cmdConfig.Server.ServerJoin.RetryJoin), "retry-join", "") flags.IntVar(&cmdConfig.Server.ServerJoin.RetryMaxAttempts, "retry-max", 0, "") + flags.BoolVar(&cmdConfig.Server.EnableEventPublisher, "event-publisher", false, "") flags.Var((flaghelper.FuncDurationVar)(func(d time.Duration) error { cmdConfig.Server.ServerJoin.RetryInterval = d return nil diff --git a/command/agent/config.go b/command/agent/config.go index 5fd3d9487d7a..392eaf7b1dc5 100644 --- a/command/agent/config.go +++ b/command/agent/config.go @@ -484,6 +484,10 @@ type ServerConfig struct { // This value is ignored. DefaultSchedulerConfig *structs.SchedulerConfiguration `hcl:"default_scheduler_config"` + // EnableEventPublisher configures whether this server's state store + // will generate events for its event stream. + EnableEventPublisher bool `hcl:"enable_event_publisher"` + // ExtraKeysHCL is used by hcl to surface unexpected keys ExtraKeysHCL []string `hcl:",unusedKeys" json:"-"` } @@ -874,8 +878,9 @@ func DefaultConfig() *Config { BindWildcardDefaultHostNetwork: true, }, Server: &ServerConfig{ - Enabled: false, - StartJoin: []string{}, + Enabled: false, + EnableEventPublisher: true, + StartJoin: []string{}, ServerJoin: &ServerJoin{ RetryJoin: []string{}, RetryInterval: 30 * time.Second, @@ -1399,6 +1404,10 @@ func (a *ServerConfig) Merge(b *ServerConfig) *ServerConfig { result.ServerJoin = result.ServerJoin.Merge(b.ServerJoin) } + if b.EnableEventPublisher { + result.EnableEventPublisher = true + } + if b.DefaultSchedulerConfig != nil { c := *b.DefaultSchedulerConfig result.DefaultSchedulerConfig = &c diff --git a/command/agent/config_parse_test.go b/command/agent/config_parse_test.go index 91108710c24e..a288b5f80c4b 100644 --- a/command/agent/config_parse_test.go +++ b/command/agent/config_parse_test.go @@ -122,6 +122,7 @@ var basicConfig = &Config{ RedundancyZone: "foo", UpgradeVersion: "0.8.0", EncryptKey: "abc", + EnableEventPublisher: true, ServerJoin: &ServerJoin{ RetryJoin: []string{"1.1.1.1", "2.2.2.2"}, RetryInterval: time.Duration(15) * time.Second, diff --git a/command/agent/testdata/basic.hcl b/command/agent/testdata/basic.hcl index 43499861f305..36a420551710 100644 --- a/command/agent/testdata/basic.hcl +++ b/command/agent/testdata/basic.hcl @@ -130,6 +130,7 @@ server { upgrade_version = "0.8.0" encrypt = "abc" raft_multiplier = 4 + enable_event_publisher = true server_join { retry_join = ["1.1.1.1", "2.2.2.2"] diff --git a/command/agent/testdata/basic.json b/command/agent/testdata/basic.json index e97f4d3f01ab..5aa897d73411 100644 --- a/command/agent/testdata/basic.json +++ b/command/agent/testdata/basic.json @@ -261,6 +261,7 @@ "data_dir": "/tmp/data", "deployment_gc_threshold": "12h", "enabled": true, + "enable_event_publisher": true, "enabled_schedulers": [ "test" ], From 47d1a33eb5b9849e8076493ab946b39d9a62967a Mon Sep 17 00:00:00 2001 From: Drew Bailey <2614075+drewbailey@users.noreply.github.com> Date: Fri, 2 Oct 2020 16:13:49 -0400 Subject: [PATCH 10/19] writetxn can return error, add alloc and job generic events. Add events table for durability --- nomad/alloc_endpoint_test.go | 3 +- nomad/deployment_endpoint_test.go | 3 +- .../deployments_watcher_test.go | 19 ++- nomad/deploymentwatcher/testutil_test.go | 7 +- nomad/drainer_int_test.go | 2 +- nomad/eval_endpoint_test.go | 3 +- nomad/fsm.go | 85 ++++++-------- nomad/node_endpoint_test.go | 5 +- nomad/plan_apply.go | 2 +- nomad/state/deployment_event_test.go | 8 +- nomad/state/events.go | 41 +++++++ nomad/state/schema.go | 17 +++ nomad/state/schema_test.go | 68 +++++++++++ nomad/state/state_changes.go | 34 +++--- nomad/state/state_store.go | 110 ++++++++++-------- nomad/state/state_store_test.go | 78 ++++++------- nomad/structs/structs.go | 4 + scheduler/generic_sched_test.go | 2 +- scheduler/system_sched_test.go | 3 +- scheduler/testing.go | 3 +- 20 files changed, 303 insertions(+), 194 deletions(-) diff --git a/nomad/alloc_endpoint_test.go b/nomad/alloc_endpoint_test.go index 012e95dfaf4c..09da45aee6e2 100644 --- a/nomad/alloc_endpoint_test.go +++ b/nomad/alloc_endpoint_test.go @@ -1,7 +1,6 @@ package nomad import ( - "context" "reflect" "testing" "time" @@ -194,7 +193,7 @@ func TestAllocEndpoint_List_Blocking(t *testing.T) { alloc2.ClientStatus = structs.AllocClientStatusRunning time.AfterFunc(100*time.Millisecond, func() { state.UpsertJobSummary(3, mock.JobSummary(alloc2.JobID)) - if err := state.UpdateAllocsFromClient(context.Background(), 4, []*structs.Allocation{alloc2}); err != nil { + if err := state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 4, []*structs.Allocation{alloc2}); err != nil { t.Fatalf("err: %v", err) } }) diff --git a/nomad/deployment_endpoint_test.go b/nomad/deployment_endpoint_test.go index ad7d222dc56d..80f638032bc1 100644 --- a/nomad/deployment_endpoint_test.go +++ b/nomad/deployment_endpoint_test.go @@ -1,7 +1,6 @@ package nomad import ( - "context" "testing" "time" @@ -1301,7 +1300,7 @@ func TestDeploymentEndpoint_Allocations_Blocking(t *testing.T) { a2.ClientStatus = structs.AllocClientStatusRunning time.AfterFunc(100*time.Millisecond, func() { assert.Nil(state.UpsertJobSummary(5, mock.JobSummary(a2.JobID)), "UpsertJobSummary") - assert.Nil(state.UpdateAllocsFromClient(context.Background(), 6, []*structs.Allocation{a2}), "updateAllocsFromClient") + assert.Nil(state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 6, []*structs.Allocation{a2}), "updateAllocsFromClient") }) req.MinQueryIndex = 4 diff --git a/nomad/deploymentwatcher/deployments_watcher_test.go b/nomad/deploymentwatcher/deployments_watcher_test.go index d1285d76986e..f4805cd83c0a 100644 --- a/nomad/deploymentwatcher/deployments_watcher_test.go +++ b/nomad/deploymentwatcher/deployments_watcher_test.go @@ -1,7 +1,6 @@ package deploymentwatcher import ( - "context" "fmt" "testing" "time" @@ -918,7 +917,7 @@ func TestDeploymentWatcher_Watch_NoProgressDeadline(t *testing.T) { HealthyAllocationIDs: []string{a.ID}, }, } - require.Nil(m.state.UpdateDeploymentAllocHealth(context.Background(), m.nextIndex(), req), "UpsertDeploymentAllocHealth") + require.Nil(m.state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, m.nextIndex(), req), "UpsertDeploymentAllocHealth") } // Wait for there to be one eval @@ -946,7 +945,7 @@ func TestDeploymentWatcher_Watch_NoProgressDeadline(t *testing.T) { UnhealthyAllocationIDs: []string{a.ID}, }, } - require.Nil(m.state.UpdateDeploymentAllocHealth(context.Background(), m.nextIndex(), req2), "UpsertDeploymentAllocHealth") + require.Nil(m.state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, m.nextIndex(), req2), "UpsertDeploymentAllocHealth") // Wait for there to be one eval testutil.WaitForResult(func() (bool, error) { @@ -1025,7 +1024,7 @@ func TestDeploymentWatcher_Watch_ProgressDeadline(t *testing.T) { Healthy: helper.BoolToPtr(false), Timestamp: now, } - require.Nil(m.state.UpdateAllocsFromClient(context.Background(), 100, []*structs.Allocation{a2})) + require.Nil(m.state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 100, []*structs.Allocation{a2})) // Wait for the deployment to be failed testutil.WaitForResult(func() (bool, error) { @@ -1209,7 +1208,7 @@ func TestDeploymentWatcher_Watch_ProgressDeadline_Canaries(t *testing.T) { Healthy: helper.BoolToPtr(true), Timestamp: now, } - require.Nil(m.state.UpdateAllocsFromClient(context.Background(), m.nextIndex(), []*structs.Allocation{a2})) + require.Nil(m.state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, m.nextIndex(), []*structs.Allocation{a2})) // Wait for the deployment to cross the deadline dout, err := m.state.DeploymentByID(nil, d.ID) @@ -1382,7 +1381,7 @@ func TestDeploymentWatcher_Watch_StartWithoutProgressDeadline(t *testing.T) { Healthy: helper.BoolToPtr(false), Timestamp: time.Now(), } - require.Nil(m.state.UpdateAllocsFromClient(context.Background(), m.nextIndex(), []*structs.Allocation{a2})) + require.Nil(m.state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, m.nextIndex(), []*structs.Allocation{a2})) // Wait for the alloc's DesiredState to set reschedule testutil.WaitForResult(func() (bool, error) { @@ -1454,7 +1453,7 @@ func TestDeploymentWatcher_RollbackFailed(t *testing.T) { HealthyAllocationIDs: []string{a.ID}, }, } - require.Nil(m.state.UpdateDeploymentAllocHealth(context.Background(), m.nextIndex(), req), "UpsertDeploymentAllocHealth") + require.Nil(m.state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, m.nextIndex(), req), "UpsertDeploymentAllocHealth") } // Wait for there to be one eval @@ -1482,7 +1481,7 @@ func TestDeploymentWatcher_RollbackFailed(t *testing.T) { UnhealthyAllocationIDs: []string{a.ID}, }, } - require.Nil(m.state.UpdateDeploymentAllocHealth(context.Background(), m.nextIndex(), req2), "UpsertDeploymentAllocHealth") + require.Nil(m.state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, m.nextIndex(), req2), "UpsertDeploymentAllocHealth") // Wait for there to be one eval testutil.WaitForResult(func() (bool, error) { @@ -1563,7 +1562,7 @@ func TestWatcher_BatchAllocUpdates(t *testing.T) { HealthyAllocationIDs: []string{a1.ID}, }, } - require.Nil(m.state.UpdateDeploymentAllocHealth(context.Background(), m.nextIndex(), req), "UpsertDeploymentAllocHealth") + require.Nil(m.state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, m.nextIndex(), req), "UpsertDeploymentAllocHealth") req2 := &structs.ApplyDeploymentAllocHealthRequest{ DeploymentAllocHealthRequest: structs.DeploymentAllocHealthRequest{ @@ -1571,7 +1570,7 @@ func TestWatcher_BatchAllocUpdates(t *testing.T) { HealthyAllocationIDs: []string{a2.ID}, }, } - require.Nil(m.state.UpdateDeploymentAllocHealth(context.Background(), m.nextIndex(), req2), "UpsertDeploymentAllocHealth") + require.Nil(m.state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, m.nextIndex(), req2), "UpsertDeploymentAllocHealth") // Wait for there to be one eval for each job testutil.WaitForResult(func() (bool, error) { diff --git a/nomad/deploymentwatcher/testutil_test.go b/nomad/deploymentwatcher/testutil_test.go index cc2e9d844fa9..a3e21d7df26a 100644 --- a/nomad/deploymentwatcher/testutil_test.go +++ b/nomad/deploymentwatcher/testutil_test.go @@ -1,7 +1,6 @@ package deploymentwatcher import ( - "context" "reflect" "strings" "sync" @@ -96,7 +95,7 @@ func (m *mockBackend) UpsertJob(job *structs.Job) (uint64, error) { func (m *mockBackend) UpdateDeploymentStatus(u *structs.DeploymentStatusUpdateRequest) (uint64, error) { m.Called(u) i := m.nextIndex() - return i, m.state.UpdateDeploymentStatus(context.Background(), i, u) + return i, m.state.UpdateDeploymentStatus(structs.MsgTypeTestSetup, i, u) } // matchDeploymentStatusUpdateConfig is used to configure the matching @@ -150,7 +149,7 @@ func matchDeploymentStatusUpdateRequest(c *matchDeploymentStatusUpdateConfig) fu func (m *mockBackend) UpdateDeploymentPromotion(req *structs.ApplyDeploymentPromoteRequest) (uint64, error) { m.Called(req) i := m.nextIndex() - return i, m.state.UpdateDeploymentPromotion(context.Background(), i, req) + return i, m.state.UpdateDeploymentPromotion(structs.MsgTypeTestSetup, i, req) } // matchDeploymentPromoteRequestConfig is used to configure the matching @@ -180,7 +179,7 @@ func matchDeploymentPromoteRequest(c *matchDeploymentPromoteRequestConfig) func( func (m *mockBackend) UpdateDeploymentAllocHealth(req *structs.ApplyDeploymentAllocHealthRequest) (uint64, error) { m.Called(req) i := m.nextIndex() - return i, m.state.UpdateDeploymentAllocHealth(context.Background(), i, req) + return i, m.state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, i, req) } // matchDeploymentAllocHealthRequestConfig is used to configure the matching diff --git a/nomad/drainer_int_test.go b/nomad/drainer_int_test.go index 49896c7ce96d..eac10b525876 100644 --- a/nomad/drainer_int_test.go +++ b/nomad/drainer_int_test.go @@ -673,7 +673,7 @@ func TestDrainer_AllTypes_NoDeadline(t *testing.T) { new.ClientStatus = structs.AllocClientStatusComplete updates = append(updates, new) } - require.Nil(state.UpdateAllocsFromClient(context.Background(), 1000, updates)) + require.Nil(state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1000, updates)) // Check that the node drain is removed testutil.WaitForResult(func() (bool, error) { diff --git a/nomad/eval_endpoint_test.go b/nomad/eval_endpoint_test.go index e494466559e2..7dacc726302d 100644 --- a/nomad/eval_endpoint_test.go +++ b/nomad/eval_endpoint_test.go @@ -1,7 +1,6 @@ package nomad import ( - "context" "fmt" "reflect" "strings" @@ -375,7 +374,7 @@ func TestEvalEndpoint_Dequeue_UpdateWaitIndex(t *testing.T) { EvalID: eval.ID, } assert := assert.New(t) - err := state.UpsertPlanResults(context.Background(), 1000, &res) + err := state.UpsertPlanResults(structs.MsgTypeTestSetup, 1000, &res) assert.Nil(err) // Dequeue the eval diff --git a/nomad/fsm.go b/nomad/fsm.go index fd483434ac2a..cf6d06a369f2 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -1,7 +1,6 @@ package nomad import ( - "context" "fmt" "io" "reflect" @@ -204,19 +203,19 @@ func (n *nomadFSM) Apply(log *raft.Log) interface{} { case structs.NodeDeregisterRequestType: return n.applyDeregisterNode(msgType, buf[1:], log.Index) case structs.NodeUpdateStatusRequestType: - return n.applyStatusUpdate(buf[1:], log.Index) + return n.applyStatusUpdate(msgType, buf[1:], log.Index) case structs.NodeUpdateDrainRequestType: return n.applyDrainUpdate(msgType, buf[1:], log.Index) case structs.JobRegisterRequestType: - return n.applyUpsertJob(buf[1:], log.Index) + return n.applyUpsertJob(msgType, buf[1:], log.Index) case structs.JobDeregisterRequestType: - return n.applyDeregisterJob(buf[1:], log.Index) + return n.applyDeregisterJob(msgType, buf[1:], log.Index) case structs.EvalUpdateRequestType: return n.applyUpdateEval(msgType, buf[1:], log.Index) case structs.EvalDeleteRequestType: return n.applyDeleteEval(buf[1:], log.Index) case structs.AllocUpdateRequestType: - return n.applyAllocUpdate(buf[1:], log.Index) + return n.applyAllocUpdate(msgType, buf[1:], log.Index) case structs.AllocClientUpdateRequestType: return n.applyAllocClientUpdate(msgType, buf[1:], log.Index) case structs.ReconcileJobSummariesRequestType: @@ -252,16 +251,21 @@ func (n *nomadFSM) Apply(log *raft.Log) interface{} { case structs.UpsertNodeEventsType: return n.applyUpsertNodeEvent(msgType, buf[1:], log.Index) case structs.JobBatchDeregisterRequestType: - return n.applyBatchDeregisterJob(buf[1:], log.Index) + // TODO(drew) higher priority event + return n.applyBatchDeregisterJob(msgType, buf[1:], log.Index) case structs.AllocUpdateDesiredTransitionRequestType: + // TODO(drew) higher priority event return n.applyAllocUpdateDesiredTransition(buf[1:], log.Index) case structs.NodeUpdateEligibilityRequestType: + // TODO(drew) higher priority event return n.applyNodeEligibilityUpdate(buf[1:], log.Index) case structs.BatchNodeUpdateDrainRequestType: + // TODO(drew) higher priority event return n.applyBatchDrainUpdate(buf[1:], log.Index) case structs.SchedulerConfigRequestType: return n.applySchedulerConfigUpdate(buf[1:], log.Index) case structs.NodeBatchDeregisterRequestType: + // TODO(drew) higher priority event return n.applyDeregisterNodeBatch(buf[1:], log.Index) case structs.ClusterMetadataRequestType: return n.applyClusterMetadata(buf[1:], log.Index) @@ -322,12 +326,10 @@ func (n *nomadFSM) applyUpsertNode(reqType structs.MessageType, buf []byte, inde panic(fmt.Errorf("failed to decode request: %v", err)) } - ctx := context.WithValue(context.Background(), state.CtxMsgType, reqType) - // Handle upgrade paths req.Node.Canonicalize() - if err := n.state.UpsertNodeCtx(ctx, index, req.Node); err != nil { + if err := n.state.UpsertNodeMsgType(reqType, index, req.Node); err != nil { n.logger.Error("UpsertNode failed", "error", err) return err } @@ -348,9 +350,7 @@ func (n *nomadFSM) applyDeregisterNode(reqType structs.MessageType, buf []byte, panic(fmt.Errorf("failed to decode request: %v", err)) } - ctx := context.WithValue(context.Background(), state.CtxMsgType, reqType) - - if err := n.state.DeleteNodeCtx(ctx, index, []string{req.NodeID}); err != nil { + if err := n.state.DeleteNodeMsgType(reqType, index, []string{req.NodeID}); err != nil { n.logger.Error("DeleteNode failed", "error", err) return err } @@ -373,14 +373,14 @@ func (n *nomadFSM) applyDeregisterNodeBatch(buf []byte, index uint64) interface{ return nil } -func (n *nomadFSM) applyStatusUpdate(buf []byte, index uint64) interface{} { +func (n *nomadFSM) applyStatusUpdate(msgType structs.MessageType, buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "node_status_update"}, time.Now()) var req structs.NodeUpdateStatusRequest if err := structs.Decode(buf, &req); err != nil { panic(fmt.Errorf("failed to decode request: %v", err)) } - if err := n.state.UpdateNodeStatus(index, req.NodeID, req.Status, req.UpdatedAt, req.NodeEvent); err != nil { + if err := n.state.UpdateNodeStatus(msgType, index, req.NodeID, req.Status, req.UpdatedAt, req.NodeEvent); err != nil { n.logger.Error("UpdateNodeStatus failed", "error", err) return err } @@ -409,8 +409,6 @@ func (n *nomadFSM) applyDrainUpdate(reqType structs.MessageType, buf []byte, ind panic(fmt.Errorf("failed to decode request: %v", err)) } - ctx := context.WithValue(context.Background(), state.CtxMsgType, reqType) - // COMPAT Remove in version 0.10 // As part of Nomad 0.8 we have deprecated the drain boolean in favor of a // drain strategy but we need to handle the upgrade path where the Raft log @@ -425,7 +423,7 @@ func (n *nomadFSM) applyDrainUpdate(reqType structs.MessageType, buf []byte, ind } } - if err := n.state.UpdateNodeDrainCtx(ctx, index, req.NodeID, req.DrainStrategy, req.MarkEligible, req.UpdatedAt, req.NodeEvent); err != nil { + if err := n.state.UpdateNodeDrainMsgType(reqType, index, req.NodeID, req.DrainStrategy, req.MarkEligible, req.UpdatedAt, req.NodeEvent); err != nil { n.logger.Error("UpdateNodeDrain failed", "error", err) return err } @@ -476,7 +474,7 @@ func (n *nomadFSM) applyNodeEligibilityUpdate(buf []byte, index uint64) interfac return nil } -func (n *nomadFSM) applyUpsertJob(buf []byte, index uint64) interface{} { +func (n *nomadFSM) applyUpsertJob(msgType structs.MessageType, buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "register_job"}, time.Now()) var req structs.JobRegisterRequest if err := structs.Decode(buf, &req); err != nil { @@ -492,7 +490,7 @@ func (n *nomadFSM) applyUpsertJob(buf []byte, index uint64) interface{} { */ req.Job.Canonicalize() - if err := n.state.UpsertJob(index, req.Job); err != nil { + if err := n.state.UpsertJobMsgType(msgType, index, req.Job); err != nil { n.logger.Error("UpsertJob failed", "error", err) return err } @@ -570,7 +568,7 @@ func (n *nomadFSM) applyUpsertJob(buf []byte, index uint64) interface{} { // so this may be nil during server upgrades. if req.Eval != nil { req.Eval.JobModifyIndex = index - if err := n.upsertEvals(context.Background(), index, []*structs.Evaluation{req.Eval}); err != nil { + if err := n.upsertEvals(msgType, index, []*structs.Evaluation{req.Eval}); err != nil { return err } } @@ -578,14 +576,14 @@ func (n *nomadFSM) applyUpsertJob(buf []byte, index uint64) interface{} { return nil } -func (n *nomadFSM) applyDeregisterJob(buf []byte, index uint64) interface{} { +func (n *nomadFSM) applyDeregisterJob(msgType structs.MessageType, buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "deregister_job"}, time.Now()) var req structs.JobDeregisterRequest if err := structs.Decode(buf, &req); err != nil { panic(fmt.Errorf("failed to decode request: %v", err)) } - err := n.state.WithWriteTransaction(index, func(tx state.Txn) error { + err := n.state.WithWriteTransaction(msgType, index, func(tx state.Txn) error { err := n.handleJobDeregister(index, req.JobID, req.Namespace, req.Purge, tx) if err != nil { @@ -602,7 +600,7 @@ func (n *nomadFSM) applyDeregisterJob(buf []byte, index uint64) interface{} { // always attempt upsert eval even if job deregister fail if req.Eval != nil { req.Eval.JobModifyIndex = index - if err := n.upsertEvals(context.Background(), index, []*structs.Evaluation{req.Eval}); err != nil { + if err := n.upsertEvals(msgType, index, []*structs.Evaluation{req.Eval}); err != nil { return err } } @@ -614,7 +612,7 @@ func (n *nomadFSM) applyDeregisterJob(buf []byte, index uint64) interface{} { return nil } -func (n *nomadFSM) applyBatchDeregisterJob(buf []byte, index uint64) interface{} { +func (n *nomadFSM) applyBatchDeregisterJob(msgType structs.MessageType, buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "batch_deregister_job"}, time.Now()) var req structs.JobBatchDeregisterRequest if err := structs.Decode(buf, &req); err != nil { @@ -624,7 +622,7 @@ func (n *nomadFSM) applyBatchDeregisterJob(buf []byte, index uint64) interface{} // Perform all store updates atomically to ensure a consistent view for store readers. // A partial update may increment the snapshot index, allowing eval brokers to process // evals for jobs whose deregistering didn't get committed yet. - err := n.state.WithWriteTransaction(index, func(tx state.Txn) error { + err := n.state.WithWriteTransaction(msgType, index, func(tx state.Txn) error { for jobNS, options := range req.Jobs { if err := n.handleJobDeregister(index, jobNS.ID, jobNS.Namespace, options.Purge, tx); err != nil { n.logger.Error("deregistering job failed", "job", jobNS.ID, "error", err) @@ -696,13 +694,11 @@ func (n *nomadFSM) applyUpdateEval(msgType structs.MessageType, buf []byte, inde panic(fmt.Errorf("failed to decode request: %v", err)) } - ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType) - - return n.upsertEvals(ctx, index, req.Evals) + return n.upsertEvals(msgType, index, req.Evals) } -func (n *nomadFSM) upsertEvals(ctx context.Context, index uint64, evals []*structs.Evaluation) error { - if err := n.state.UpsertEvalsCtx(ctx, index, evals); err != nil { +func (n *nomadFSM) upsertEvals(msgType structs.MessageType, index uint64, evals []*structs.Evaluation) error { + if err := n.state.UpsertEvalsMsgType(msgType, index, evals); err != nil { n.logger.Error("UpsertEvals failed", "error", err) return err } @@ -751,7 +747,7 @@ func (n *nomadFSM) applyDeleteEval(buf []byte, index uint64) interface{} { return nil } -func (n *nomadFSM) applyAllocUpdate(buf []byte, index uint64) interface{} { +func (n *nomadFSM) applyAllocUpdate(msgType structs.MessageType, buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "alloc_update"}, time.Now()) var req structs.AllocUpdateRequest if err := structs.Decode(buf, &req); err != nil { @@ -782,7 +778,7 @@ func (n *nomadFSM) applyAllocUpdate(buf []byte, index uint64) interface{} { alloc.Canonicalize() } - if err := n.state.UpsertAllocs(index, req.Alloc); err != nil { + if err := n.state.UpsertAllocsMsgType(msgType, index, req.Alloc); err != nil { n.logger.Error("UpsertAllocs failed", "error", err) return err } @@ -810,16 +806,15 @@ func (n *nomadFSM) applyAllocClientUpdate(msgType structs.MessageType, buf []byt } } - ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType) // Update all the client allocations - if err := n.state.UpdateAllocsFromClient(ctx, index, req.Alloc); err != nil { + if err := n.state.UpdateAllocsFromClient(msgType, index, req.Alloc); err != nil { n.logger.Error("UpdateAllocFromClient failed", "error", err) return err } // Update any evals if len(req.Evals) > 0 { - if err := n.upsertEvals(ctx, index, req.Evals); err != nil { + if err := n.upsertEvals(msgType, index, req.Evals); err != nil { n.logger.Error("applyAllocClientUpdate failed to update evaluations", "error", err) return err } @@ -887,9 +882,7 @@ func (n *nomadFSM) applyUpsertNodeEvent(msgType structs.MessageType, buf []byte, panic(fmt.Errorf("failed to decode EmitNodeEventsRequest: %v", err)) } - ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType) - - if err := n.state.UpsertNodeEventsCtx(ctx, index, req.NodeEvents); err != nil { + if err := n.state.UpsertNodeEventsMsgType(msgType, index, req.NodeEvents); err != nil { n.logger.Error("failed to add node events", "error", err) return err } @@ -968,9 +961,7 @@ func (n *nomadFSM) applyPlanResults(msgType structs.MessageType, buf []byte, ind panic(fmt.Errorf("failed to decode request: %v", err)) } - ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType) - - if err := n.state.UpsertPlanResults(ctx, index, &req); err != nil { + if err := n.state.UpsertPlanResults(msgType, index, &req); err != nil { n.logger.Error("ApplyPlan failed", "error", err) return err } @@ -989,9 +980,7 @@ func (n *nomadFSM) applyDeploymentStatusUpdate(msgType structs.MessageType, buf panic(fmt.Errorf("failed to decode request: %v", err)) } - ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType) - - if err := n.state.UpdateDeploymentStatus(ctx, index, &req); err != nil { + if err := n.state.UpdateDeploymentStatus(msgType, index, &req); err != nil { n.logger.Error("UpsertDeploymentStatusUpdate failed", "error", err) return err } @@ -1008,9 +997,7 @@ func (n *nomadFSM) applyDeploymentPromotion(msgType structs.MessageType, buf []b panic(fmt.Errorf("failed to decode request: %v", err)) } - ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType) - - if err := n.state.UpdateDeploymentPromotion(ctx, index, &req); err != nil { + if err := n.state.UpdateDeploymentPromotion(msgType, index, &req); err != nil { n.logger.Error("UpsertDeploymentPromotion failed", "error", err) return err } @@ -1028,9 +1015,7 @@ func (n *nomadFSM) applyDeploymentAllocHealth(msgType structs.MessageType, buf [ panic(fmt.Errorf("failed to decode request: %v", err)) } - ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType) - - if err := n.state.UpdateDeploymentAllocHealth(ctx, index, &req); err != nil { + if err := n.state.UpdateDeploymentAllocHealth(msgType, index, &req); err != nil { n.logger.Error("UpsertDeploymentAllocHealth failed", "error", err) return err } diff --git a/nomad/node_endpoint_test.go b/nomad/node_endpoint_test.go index db1b46a5c940..23baba2e41c4 100644 --- a/nomad/node_endpoint_test.go +++ b/nomad/node_endpoint_test.go @@ -1,7 +1,6 @@ package nomad import ( - "context" "errors" "fmt" "net" @@ -2049,7 +2048,7 @@ func TestClientEndpoint_GetAllocs_Blocking(t *testing.T) { allocUpdate.ID = alloc.ID allocUpdate.ClientStatus = structs.AllocClientStatusRunning state.UpsertJobSummary(199, mock.JobSummary(allocUpdate.JobID)) - err := state.UpdateAllocsFromClient(context.Background(), 200, []*structs.Allocation{allocUpdate}) + err := state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 200, []*structs.Allocation{allocUpdate}) if err != nil { t.Fatalf("err: %v", err) } @@ -2748,7 +2747,7 @@ func TestClientEndpoint_ListNodes_Blocking(t *testing.T) { // Node status update triggers watches time.AfterFunc(100*time.Millisecond, func() { - errCh <- state.UpdateNodeStatus(40, node.ID, structs.NodeStatusDown, 0, nil) + errCh <- state.UpdateNodeStatus(structs.MsgTypeTestSetup, 40, node.ID, structs.NodeStatusDown, 0, nil) }) req.MinQueryIndex = 38 diff --git a/nomad/plan_apply.go b/nomad/plan_apply.go index 08d050026019..f7a3f19fe741 100644 --- a/nomad/plan_apply.go +++ b/nomad/plan_apply.go @@ -310,7 +310,7 @@ func (p *planner) applyPlan(plan *structs.Plan, result *structs.PlanResult, snap // Optimistically apply to our state view if snap != nil { nextIdx := p.raft.AppliedIndex() + 1 - if err := snap.UpsertPlanResults(context.Background(), nextIdx, &req); err != nil { + if err := snap.UpsertPlanResults(structs.ApplyPlanResultsRequestType, nextIdx, &req); err != nil { return future, err } } diff --git a/nomad/state/deployment_event_test.go b/nomad/state/deployment_event_test.go index d71ed8d2f335..14fb3482edbb 100644 --- a/nomad/state/deployment_event_test.go +++ b/nomad/state/deployment_event_test.go @@ -32,7 +32,7 @@ func TestDeploymentEventFromChanges(t *testing.T) { setupTx.Txn.Commit() - ctx := context.WithValue(context.Background(), CtxMsgType, structs.DeploymentStatusUpdateRequestType) + msgType := structs.DeploymentStatusUpdateRequestType req := &structs.DeploymentStatusUpdateRequest{ DeploymentUpdate: &structs.DeploymentStatusUpdate{ @@ -44,7 +44,7 @@ func TestDeploymentEventFromChanges(t *testing.T) { // Exlude Job and assert its added } - require.NoError(t, s.UpdateDeploymentStatus(ctx, 100, req)) + require.NoError(t, s.UpdateDeploymentStatus(msgType, 100, req)) events := WaitForEvents(t, s, 100, 1, 1*time.Second) require.Len(t, events, 2) @@ -113,7 +113,7 @@ func TestDeploymentEventFromChanges_Promotion(t *testing.T) { e := mock.Eval() // Request to promote canaries - ctx := context.WithValue(context.Background(), CtxMsgType, structs.DeploymentPromoteRequestType) + msgType := structs.DeploymentPromoteRequestType req := &structs.ApplyDeploymentPromoteRequest{ DeploymentPromoteRequest: structs.DeploymentPromoteRequest{ DeploymentID: d.ID, @@ -122,7 +122,7 @@ func TestDeploymentEventFromChanges_Promotion(t *testing.T) { Eval: e, } - require.NoError(t, s.UpdateDeploymentPromotion(ctx, 100, req)) + require.NoError(t, s.UpdateDeploymentPromotion(msgType, 100, req)) events := WaitForEvents(t, s, 100, 1, 1*time.Second) require.Len(t, events, 2) diff --git a/nomad/state/events.go b/nomad/state/events.go index 921d7a8947ea..7e0d95daab9f 100644 --- a/nomad/state/events.go +++ b/nomad/state/events.go @@ -31,6 +31,8 @@ const ( TypeAllocUpdated = "AllocUpdated" TypeEvalUpdated = "EvalUpdated" + + TypeJobRegistered = "JobRegistered" ) type JobEvent struct { @@ -78,6 +80,12 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, erro eventType = TypeEvalUpdated case structs.AllocClientUpdateRequestType: eventType = TypeAllocUpdated + case structs.JobRegisterRequestType: + eventType = TypeJobRegistered + case structs.AllocUpdateRequestType: + eventType = TypeAllocUpdated + case structs.NodeUpdateStatusRequestType: + eventType = TypeNodeEvent } var events []stream.Event @@ -117,6 +125,39 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, erro }, } + events = append(events, event) + case "jobs": + after, ok := change.After.(*structs.Job) + if !ok { + return nil, fmt.Errorf("transaction change was not an Allocation") + } + + event := stream.Event{ + Topic: TopicAlloc, + Type: eventType, + Index: changes.Index, + Key: after.ID, + Payload: &JobEvent{ + Job: after, + }, + } + + events = append(events, event) + case "nodes": + after, ok := change.After.(*structs.Node) + if !ok { + return nil, fmt.Errorf("transaction change was not a Node") + } + + event := stream.Event{ + Topic: TopicNode, + Type: eventType, + Index: changes.Index, + Key: after.ID, + Payload: &NodeEvent{ + Node: after, + }, + } events = append(events, event) } } diff --git a/nomad/state/schema.go b/nomad/state/schema.go index 3d1308859ae9..7f73bb4fcd6b 100644 --- a/nomad/state/schema.go +++ b/nomad/state/schema.go @@ -52,6 +52,7 @@ func init() { csiPluginTableSchema, scalingPolicyTableSchema, scalingEventTableSchema, + eventTableSchema, }...) } @@ -900,3 +901,19 @@ func scalingEventTableSchema() *memdb.TableSchema { }, } } + +func eventTableSchema() *memdb.TableSchema { + return &memdb.TableSchema{ + Name: "events", + Indexes: map[string]*memdb.IndexSchema{ + "id": { + Name: "id", + AllowMissing: true, + Unique: true, + Indexer: &memdb.UintFieldIndex{ + Field: "Index", + }, + }, + }, + } +} diff --git a/nomad/state/schema_test.go b/nomad/state/schema_test.go index f5b1b620fc45..267c6f8742b0 100644 --- a/nomad/state/schema_test.go +++ b/nomad/state/schema_test.go @@ -5,6 +5,7 @@ import ( memdb "github.com/hashicorp/go-memdb" "github.com/hashicorp/nomad/nomad/mock" + "github.com/hashicorp/nomad/nomad/stream" "github.com/stretchr/testify/require" ) @@ -144,3 +145,70 @@ func TestState_ScalingPolicyTargetFieldIndex_FromObject(t *testing.T) { require.Error(err) require.Equal("", string(val)) } + +func TestEventTableUintIndex(t *testing.T) { + + require := require.New(t) + + const ( + eventsTable = "events" + uintIDIdx = "id" + ) + + db, err := memdb.NewMemDB(&memdb.DBSchema{ + Tables: map[string]*memdb.TableSchema{ + eventsTable: eventTableSchema(), + }, + }) + require.NoError(err) + + // numRecords in table counts all the items in the table, which is expected + // to always be 1 since that's the point of the singletonRecord Indexer. + numRecordsInTable := func() int { + txn := db.Txn(false) + defer txn.Abort() + + iter, err := txn.Get(eventsTable, uintIDIdx) + require.NoError(err) + + num := 0 + for item := iter.Next(); item != nil; item = iter.Next() { + num++ + } + return num + } + + insertEvents := func(e *stream.Events) { + txn := db.Txn(true) + err := txn.Insert(eventsTable, e) + require.NoError(err) + txn.Commit() + } + + get := func(idx uint64) *stream.Events { + txn := db.Txn(false) + defer txn.Abort() + record, err := txn.First("events", "id", idx) + require.NoError(err) + s, ok := record.(*stream.Events) + require.True(ok) + return s + } + + firstEvent := &stream.Events{Index: 10, Events: []stream.Event{{Index: 10}, {Index: 10}}} + secondEvent := &stream.Events{Index: 11, Events: []stream.Event{{Index: 11}, {Index: 11}}} + thirdEvent := &stream.Events{Index: 202, Events: []stream.Event{{Index: 202}, {Index: 202}}} + insertEvents(firstEvent) + insertEvents(secondEvent) + insertEvents(thirdEvent) + require.Equal(3, numRecordsInTable()) + + gotFirst := get(10) + require.Equal(firstEvent, gotFirst) + + gotSecond := get(11) + require.Equal(secondEvent, gotSecond) + + gotThird := get(202) + require.Equal(thirdEvent, gotThird) +} diff --git a/nomad/state/state_changes.go b/nomad/state/state_changes.go index 14b7b7cd1307..117eebc29ea5 100644 --- a/nomad/state/state_changes.go +++ b/nomad/state/state_changes.go @@ -1,7 +1,6 @@ package state import ( - "context" "fmt" "github.com/hashicorp/go-memdb" @@ -80,10 +79,9 @@ func (c *changeTrackerDB) WriteTxn(idx uint64) *txn { return t } -// WriteTxnCtx is identical to WriteTxn but takes a ctx used for event sourcing -func (c *changeTrackerDB) WriteTxnCtx(ctx context.Context, idx uint64) *txn { +func (c *changeTrackerDB) WriteTxnMsgT(msgType structs.MessageType, idx uint64) *txn { t := &txn{ - ctx: ctx, + msgType: msgType, Txn: c.db.Txn(true), Index: idx, publish: c.publish, @@ -124,8 +122,8 @@ func (c *changeTrackerDB) WriteTxnRestore() *txn { // error. Any errors from the callback would be lost, which would result in a // missing change event, even though the state store had changed. type txn struct { - // ctx is used to hold message type information from an FSM request - ctx context.Context + // msgType is used to inform event sourcing which type of event to create + msgType structs.MessageType *memdb.Txn // Index in raft where the write is occurring. The value is zero for a @@ -165,20 +163,7 @@ func (tx *txn) Commit() error { // If the context is empty or the value isn't set IgnoreUnknownTypeFlag will // be returned to signal that the MsgType is unknown. func (tx *txn) MsgType() structs.MessageType { - if tx.ctx == nil { - return structs.IgnoreUnknownTypeFlag - } - - raw := tx.ctx.Value(CtxMsgType) - if raw == nil { - return structs.IgnoreUnknownTypeFlag - } - - msgType, ok := raw.(structs.MessageType) - if !ok { - return structs.IgnoreUnknownTypeFlag - } - return msgType + return tx.msgType } func processDBChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { @@ -188,6 +173,9 @@ func processDBChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { return []stream.Event{}, nil case structs.NodeRegisterRequestType: return NodeRegisterEventFromChanges(tx, changes) + case structs.NodeUpdateStatusRequestType: + // TODO(drew) test + return GenericEventsFromChanges(tx, changes) case structs.NodeDeregisterRequestType: return NodeDeregisterEventFromChanges(tx, changes) case structs.NodeUpdateDrainRequestType: @@ -206,6 +194,12 @@ func processDBChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { return GenericEventsFromChanges(tx, changes) case structs.AllocClientUpdateRequestType: return GenericEventsFromChanges(tx, changes) + case structs.JobRegisterRequestType: + // TODO(drew) test + return GenericEventsFromChanges(tx, changes) + case structs.AllocUpdateRequestType: + // TODO(drew) test + return GenericEventsFromChanges(tx, changes) } return []stream.Event{}, nil } diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 295b414f63c8..a2271676fb42 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -270,7 +270,7 @@ RUN_QUERY: } // UpsertPlanResults is used to upsert the results of a plan. -func (s *StateStore) UpsertPlanResults(ctx context.Context, index uint64, results *structs.ApplyPlanResultsRequest) error { +func (s *StateStore) UpsertPlanResults(msgType structs.MessageType, index uint64, results *structs.ApplyPlanResultsRequest) error { snapshot, err := s.Snapshot() if err != nil { return err @@ -292,7 +292,7 @@ func (s *StateStore) UpsertPlanResults(ctx context.Context, index uint64, result return err } - txn := s.db.WriteTxnCtx(ctx, index) + txn := s.db.WriteTxnMsgT(msgType, index) defer txn.Abort() // Upsert the newly created or updated deployment @@ -302,7 +302,7 @@ func (s *StateStore) UpsertPlanResults(ctx context.Context, index uint64, result } } - // Update the status of deployments effected by the plan. + // Update the status of dmsgType structs.MessageType by the plan. if len(results.DeploymentUpdates) != 0 { s.upsertDeploymentUpdates(index, results.DeploymentUpdates, txn) } @@ -356,11 +356,7 @@ func (s *StateStore) UpsertPlanResults(ctx context.Context, index uint64, result } } - if err := txn.Commit(); err != nil { - return err - } - - return nil + return txn.Commit() } // addComputedAllocAttrs adds the computed/derived attributes to the allocation. @@ -758,19 +754,18 @@ func (s *StateStore) ScalingEventsByJob(ws memdb.WatchSet, namespace, jobID stri return nil, 0, nil } -// UpsertNodeCtx is used to register a node or update a node definition +// UpsertNodeMsgType is used to register a node or update a node definition // This is assumed to be triggered by the client, so we retain the value // of drain/eligibility which is set by the scheduler. -func (s *StateStore) UpsertNodeCtx(ctx context.Context, index uint64, node *structs.Node) error { - txn := s.db.WriteTxnCtx(ctx, index) +func (s *StateStore) UpsertNodeMsgType(msgType structs.MessageType, index uint64, node *structs.Node) error { + txn := s.db.WriteTxnMsgT(msgType, index) defer txn.Abort() err := upsertNodeTxn(txn, index, node) if err != nil { return nil } - txn.Commit() - return nil + return txn.Commit() } // UpsertNode is used to register a node or update a node definition @@ -841,16 +836,15 @@ func upsertNodeTxn(txn *txn, index uint64, node *structs.Node) error { } // DeleteNode deregisters a batch of nodes -func (s *StateStore) DeleteNodeCtx(ctx context.Context, index uint64, nodes []string) error { - txn := s.db.WriteTxnCtx(ctx, index) +func (s *StateStore) DeleteNodeMsgType(msgType structs.MessageType, index uint64, nodes []string) error { + txn := s.db.WriteTxnMsgT(msgType, index) defer txn.Abort() err := deleteNodeTxn(txn, index, nodes) if err != nil { return nil } - txn.Commit() - return nil + return txn.Commit() } // DeleteNode deregisters a batch of nodes @@ -899,16 +893,15 @@ func deleteNodeTxn(txn *txn, index uint64, nodes []string) error { } // UpdateNodeStatus is used to update the status of a node -func (s *StateStore) UpdateNodeStatus(index uint64, nodeID, status string, updatedAt int64, event *structs.NodeEvent) error { - txn := s.db.WriteTxn(index) +func (s *StateStore) UpdateNodeStatus(msgType structs.MessageType, index uint64, nodeID, status string, updatedAt int64, event *structs.NodeEvent) error { + txn := s.db.WriteTxnMsgT(msgType, index) defer txn.Abort() if err := s.updateNodeStatusTxn(txn, nodeID, status, updatedAt, event); err != nil { return err } - txn.Commit() - return nil + return txn.Commit() } func (s *StateStore) updateNodeStatusTxn(txn *txn, nodeID, status string, updatedAt int64, event *structs.NodeEvent) error { @@ -960,16 +953,15 @@ func (s *StateStore) BatchUpdateNodeDrain(index uint64, updatedAt int64, updates } // UpdateNodeDrain is used to update the drain of a node -func (s *StateStore) UpdateNodeDrainCtx(ctx context.Context, index uint64, nodeID string, +func (s *StateStore) UpdateNodeDrainMsgType(msgType structs.MessageType, index uint64, nodeID string, drain *structs.DrainStrategy, markEligible bool, updatedAt int64, event *structs.NodeEvent) error { - txn := s.db.WriteTxnCtx(ctx, index) + txn := s.db.WriteTxnMsgT(msgType, index) defer txn.Abort() if err := s.updateNodeDrainImpl(txn, index, nodeID, drain, markEligible, updatedAt, event); err != nil { return err } - txn.Commit() - return nil + return txn.Commit() } // UpdateNodeDrain is used to update the drain of a node @@ -1075,8 +1067,8 @@ func (s *StateStore) UpdateNodeEligibility(index uint64, nodeID string, eligibil return nil } -func (s *StateStore) UpsertNodeEventsCtx(ctx context.Context, index uint64, nodeEvents map[string][]*structs.NodeEvent) error { - txn := s.db.WriteTxnCtx(ctx, index) +func (s *StateStore) UpsertNodeEventsMsgType(msgType structs.MessageType, index uint64, nodeEvents map[string][]*structs.NodeEvent) error { + txn := s.db.WriteTxnMsgT(msgType, index) defer txn.Abort() for nodeID, events := range nodeEvents { @@ -1085,8 +1077,7 @@ func (s *StateStore) UpsertNodeEventsCtx(ctx context.Context, index uint64, node } } - txn.Commit() - return nil + return txn.Commit() } // UpsertNodeEvents adds the node events to the nodes, rotating events as @@ -1492,6 +1483,16 @@ func (s *StateStore) UpsertJob(index uint64, job *structs.Job) error { return nil } +// UpsertJob is used to register a job or update a job definition +func (s *StateStore) UpsertJobMsgType(msgType structs.MessageType, index uint64, job *structs.Job) error { + txn := s.db.WriteTxnMsgT(msgType, index) + defer txn.Abort() + if err := s.upsertJobImpl(index, job, false, txn); err != nil { + return err + } + return txn.Commit() +} + // UpsertJobTxn is used to register a job or update a job definition, like UpsertJob, // but in a transaction. Useful for when making multiple modifications atomically func (s *StateStore) UpsertJobTxn(index uint64, job *structs.Job, txn Txn) error { @@ -2706,13 +2707,13 @@ func (s *StateStore) UpsertEvals(index uint64, evals []*structs.Evaluation) erro } // UpsertEvals is used to upsert a set of evaluations -func (s *StateStore) UpsertEvalsCtx(ctx context.Context, index uint64, evals []*structs.Evaluation) error { - txn := s.db.WriteTxnCtx(ctx, index) +func (s *StateStore) UpsertEvalsMsgType(msgType structs.MessageType, index uint64, evals []*structs.Evaluation) error { + txn := s.db.WriteTxnMsgT(msgType, index) defer txn.Abort() err := s.UpsertEvalsTxn(index, evals, txn) if err == nil { - txn.Commit() + return txn.Commit() } return err } @@ -3029,8 +3030,8 @@ func (s *StateStore) EvalsByNamespace(ws memdb.WatchSet, namespace string) (memd // most things, some updates are authoritative from the client. Specifically, // the desired state comes from the schedulers, while the actual state comes // from clients. -func (s *StateStore) UpdateAllocsFromClient(ctx context.Context, index uint64, allocs []*structs.Allocation) error { - txn := s.db.WriteTxnCtx(ctx, index) +func (s *StateStore) UpdateAllocsFromClient(msgType structs.MessageType, index uint64, allocs []*structs.Allocation) error { + txn := s.db.WriteTxnMsgT(msgType, index) defer txn.Abort() // Handle each of the updated allocations @@ -3045,8 +3046,7 @@ func (s *StateStore) UpdateAllocsFromClient(ctx context.Context, index uint64, a return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // nestedUpdateAllocFromClient is used to nest an update of an allocation with client status @@ -3149,6 +3149,17 @@ func (s *StateStore) UpsertAllocs(index uint64, allocs []*structs.Allocation) er return nil } +// UpsertAllocsMsgType is used to evict a set of allocations and allocate new ones at +// the same time. +func (s *StateStore) UpsertAllocsMsgType(msgType structs.MessageType, index uint64, allocs []*structs.Allocation) error { + txn := s.db.WriteTxnMsgT(msgType, index) + defer txn.Abort() + if err := s.upsertAllocsImpl(index, allocs, txn); err != nil { + return err + } + return txn.Commit() +} + // upsertAllocs is the actual implementation of UpsertAllocs so that it may be // used with an existing transaction. func (s *StateStore) upsertAllocsImpl(index uint64, allocs []*structs.Allocation, txn *txn) error { @@ -3842,8 +3853,8 @@ func (s *StateStore) SITokenAccessorsByNode(ws memdb.WatchSet, nodeID string) ([ // UpdateDeploymentStatus is used to make deployment status updates and // potentially make a evaluation -func (s *StateStore) UpdateDeploymentStatus(ctx context.Context, index uint64, req *structs.DeploymentStatusUpdateRequest) error { - txn := s.db.WriteTxnCtx(ctx, index) +func (s *StateStore) UpdateDeploymentStatus(msgType structs.MessageType, index uint64, req *structs.DeploymentStatusUpdateRequest) error { + txn := s.db.WriteTxnMsgT(msgType, index) defer txn.Abort() if err := s.updateDeploymentStatusImpl(index, req.DeploymentUpdate, txn); err != nil { @@ -3864,8 +3875,7 @@ func (s *StateStore) UpdateDeploymentStatus(ctx context.Context, index uint64, r } } - txn.Commit() - return nil + return txn.Commit() } // updateDeploymentStatusImpl is used to make deployment status updates @@ -3946,8 +3956,8 @@ func (s *StateStore) updateJobStabilityImpl(index uint64, namespace, jobID strin // UpdateDeploymentPromotion is used to promote canaries in a deployment and // potentially make a evaluation -func (s *StateStore) UpdateDeploymentPromotion(ctx context.Context, index uint64, req *structs.ApplyDeploymentPromoteRequest) error { - txn := s.db.WriteTxnCtx(ctx, index) +func (s *StateStore) UpdateDeploymentPromotion(msgType structs.MessageType, index uint64, req *structs.ApplyDeploymentPromoteRequest) error { + txn := s.db.WriteTxnMsgT(msgType, index) defer txn.Abort() // Retrieve deployment and ensure it is not terminal and is active @@ -4083,14 +4093,13 @@ func (s *StateStore) UpdateDeploymentPromotion(ctx context.Context, index uint64 return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // UpdateDeploymentAllocHealth is used to update the health of allocations as // part of the deployment and potentially make a evaluation -func (s *StateStore) UpdateDeploymentAllocHealth(ctx context.Context, index uint64, req *structs.ApplyDeploymentAllocHealthRequest) error { - txn := s.db.WriteTxnCtx(ctx, index) +func (s *StateStore) UpdateDeploymentAllocHealth(msgType structs.MessageType, index uint64, req *structs.ApplyDeploymentAllocHealthRequest) error { + txn := s.db.WriteTxnMsgT(msgType, index) defer txn.Abort() // Retrieve deployment and ensure it is not terminal and is active @@ -4179,8 +4188,7 @@ func (s *StateStore) UpdateDeploymentAllocHealth(ctx context.Context, index uint } } - txn.Commit() - return nil + return txn.Commit() } // LastIndex returns the greatest index value for all indexes @@ -5407,13 +5415,13 @@ func (s *StateStore) ClusterSetMetadata(index uint64, meta *structs.ClusterMetad // WithWriteTransaction executes the passed function within a write transaction, // and returns its result. If the invocation returns no error, the transaction // is committed; otherwise, it's aborted. -func (s *StateStore) WithWriteTransaction(index uint64, fn func(Txn) error) error { - tx := s.db.WriteTxn(index) +func (s *StateStore) WithWriteTransaction(msgType structs.MessageType, index uint64, fn func(Txn) error) error { + tx := s.db.WriteTxnMsgT(msgType, index) defer tx.Abort() err := fn(tx) if err == nil { - tx.Commit() + return tx.Commit() } return err } diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index c683606015c3..a6ed60a44e74 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -127,7 +127,7 @@ func TestStateStore_UpsertPlanResults_AllocationsCreated_Denormalized(t *testing EvalID: eval.ID, } assert := assert.New(t) - err := state.UpsertPlanResults(context.Background(), 1000, &res) + err := state.UpsertPlanResults(structs.MsgTypeTestSetup, 1000, &res) assert.Nil(err) ws := memdb.NewWatchSet() @@ -203,7 +203,7 @@ func TestStateStore_UpsertPlanResults_AllocationsDenormalized(t *testing.T) { } assert := assert.New(t) planModifyIndex := uint64(1000) - err := state.UpsertPlanResults(context.Background(), planModifyIndex, &res) + err := state.UpsertPlanResults(structs.MsgTypeTestSetup, planModifyIndex, &res) require.NoError(err) ws := memdb.NewWatchSet() @@ -284,7 +284,7 @@ func TestStateStore_UpsertPlanResults_Deployment(t *testing.T) { EvalID: eval.ID, } - err := state.UpsertPlanResults(context.Background(), 1000, &res) + err := state.UpsertPlanResults(structs.MsgTypeTestSetup, 1000, &res) if err != nil { t.Fatalf("err: %v", err) } @@ -332,7 +332,7 @@ func TestStateStore_UpsertPlanResults_Deployment(t *testing.T) { EvalID: eval.ID, } - err = state.UpsertPlanResults(context.Background(), 1001, &res) + err = state.UpsertPlanResults(structs.MsgTypeTestSetup, 1001, &res) if err != nil { t.Fatalf("err: %v", err) } @@ -400,7 +400,7 @@ func TestStateStore_UpsertPlanResults_PreemptedAllocs(t *testing.T) { PreemptionEvals: []*structs.Evaluation{eval2}, } - err = state.UpsertPlanResults(context.Background(), 1000, &res) + err = state.UpsertPlanResults(structs.MsgTypeTestSetup, 1000, &res) require.NoError(err) ws := memdb.NewWatchSet() @@ -486,7 +486,7 @@ func TestStateStore_UpsertPlanResults_DeploymentUpdates(t *testing.T) { EvalID: eval.ID, } - err := state.UpsertPlanResults(context.Background(), 1000, &res) + err := state.UpsertPlanResults(structs.MsgTypeTestSetup, 1000, &res) if err != nil { t.Fatalf("err: %v", err) } @@ -898,7 +898,7 @@ func TestStateStore_UpdateNodeStatus_Node(t *testing.T) { Timestamp: time.Now(), } - require.NoError(state.UpdateNodeStatus(801, node.ID, structs.NodeStatusReady, 70, event)) + require.NoError(state.UpdateNodeStatus(structs.MsgTypeTestSetup, 801, node.ID, structs.NodeStatusReady, 70, event)) require.True(watchFired(ws)) ws = memdb.NewWatchSet() @@ -1903,7 +1903,7 @@ func TestStateStore_DeleteJobTxn_BatchDeletes(t *testing.T) { // Actually delete const deletionIndex = uint64(10001) - err = state.WithWriteTransaction(deletionIndex, func(txn Txn) error { + err = state.WithWriteTransaction(structs.MsgTypeTestSetup, deletionIndex, func(txn Txn) error { for i, job := range jobs { err := state.DeleteJobTxn(deletionIndex, job.Namespace, job.ID, txn) require.NoError(t, err, "failed at %d %e", i, err) @@ -4467,7 +4467,7 @@ func TestStateStore_UpdateAllocsFromClient(t *testing.T) { JobID: alloc.JobID, TaskGroup: alloc.TaskGroup, } - err = state.UpdateAllocsFromClient(context.Background(), 1001, []*structs.Allocation{update}) + err = state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1001, []*structs.Allocation{update}) if err != nil { t.Fatalf("err: %v", err) } @@ -4565,7 +4565,7 @@ func TestStateStore_UpdateAllocsFromClient_ChildJob(t *testing.T) { TaskGroup: alloc2.TaskGroup, } - err = state.UpdateAllocsFromClient(context.Background(), 1001, []*structs.Allocation{update, update2}) + err = state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1001, []*structs.Allocation{update, update2}) if err != nil { t.Fatalf("err: %v", err) } @@ -4666,7 +4666,7 @@ func TestStateStore_UpdateMultipleAllocsFromClient(t *testing.T) { TaskGroup: alloc.TaskGroup, } - err = state.UpdateAllocsFromClient(context.Background(), 1001, []*structs.Allocation{update, update2}) + err = state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1001, []*structs.Allocation{update, update2}) if err != nil { t.Fatalf("err: %v", err) } @@ -4735,7 +4735,7 @@ func TestStateStore_UpdateAllocsFromClient_Deployment(t *testing.T) { Timestamp: healthy, }, } - require.Nil(state.UpdateAllocsFromClient(context.Background(), 1001, []*structs.Allocation{update})) + require.Nil(state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1001, []*structs.Allocation{update})) // Check that the deployment state was updated because the healthy // deployment @@ -4780,7 +4780,7 @@ func TestStateStore_UpdateAllocsFromClient_DeploymentStateMerges(t *testing.T) { Canary: false, }, } - require.Nil(state.UpdateAllocsFromClient(context.Background(), 1001, []*structs.Allocation{update})) + require.Nil(state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1001, []*structs.Allocation{update})) // Check that the merging of the deployment status was correct out, err := state.AllocByID(nil, alloc.ID) @@ -5161,7 +5161,7 @@ func TestStateStore_UpdateAlloc_NoJob(t *testing.T) { // Update the client state of the allocation to complete allocCopy1 := allocCopy.Copy() allocCopy1.ClientStatus = structs.AllocClientStatusComplete - if err := state.UpdateAllocsFromClient(context.Background(), 1003, []*structs.Allocation{allocCopy1}); err != nil { + if err := state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1003, []*structs.Allocation{allocCopy1}); err != nil { t.Fatalf("err: %v", err) } @@ -5272,12 +5272,12 @@ func TestStateStore_JobSummary(t *testing.T) { alloc1 := alloc.Copy() alloc1.ClientStatus = structs.AllocClientStatusPending alloc1.DesiredStatus = "" - state.UpdateAllocsFromClient(context.Background(), 920, []*structs.Allocation{alloc}) + state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 920, []*structs.Allocation{alloc}) alloc3 := alloc.Copy() alloc3.ClientStatus = structs.AllocClientStatusRunning alloc3.DesiredStatus = "" - state.UpdateAllocsFromClient(context.Background(), 930, []*structs.Allocation{alloc3}) + state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 930, []*structs.Allocation{alloc3}) // Upsert the alloc alloc4 := alloc.Copy() @@ -5320,7 +5320,7 @@ func TestStateStore_JobSummary(t *testing.T) { alloc6 := alloc.Copy() alloc6.ClientStatus = structs.AllocClientStatusRunning alloc6.DesiredStatus = "" - state.UpdateAllocsFromClient(context.Background(), 990, []*structs.Allocation{alloc6}) + state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 990, []*structs.Allocation{alloc6}) // We shouldn't have any summary at this point summary, _ = state.JobSummaryByID(ws, job.Namespace, job.ID) @@ -5347,7 +5347,7 @@ func TestStateStore_JobSummary(t *testing.T) { alloc7.Job = outJob alloc7.ClientStatus = structs.AllocClientStatusComplete alloc7.DesiredStatus = structs.AllocDesiredStatusRun - state.UpdateAllocsFromClient(context.Background(), 1020, []*structs.Allocation{alloc7}) + state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1020, []*structs.Allocation{alloc7}) expectedSummary = structs.JobSummary{ JobID: job.ID, @@ -5392,7 +5392,7 @@ func TestStateStore_ReconcileJobSummary(t *testing.T) { // Change the state of the first alloc to running alloc3 := alloc.Copy() alloc3.ClientStatus = structs.AllocClientStatusRunning - state.UpdateAllocsFromClient(context.Background(), 120, []*structs.Allocation{alloc3}) + state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 120, []*structs.Allocation{alloc3}) //Add some more allocs to the second tg alloc4 := mock.Alloc() @@ -5425,7 +5425,7 @@ func TestStateStore_ReconcileJobSummary(t *testing.T) { state.UpsertAllocs(130, []*structs.Allocation{alloc4, alloc6, alloc8, alloc10}) - state.UpdateAllocsFromClient(context.Background(), 150, []*structs.Allocation{alloc5, alloc7, alloc9, alloc11}) + state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 150, []*structs.Allocation{alloc5, alloc7, alloc9, alloc11}) // DeleteJobSummary is a helper method and doesn't modify the indexes table state.DeleteJobSummary(130, alloc.Namespace, alloc.Job.ID) @@ -5564,7 +5564,7 @@ func TestStateStore_UpdateAlloc_JobNotPresent(t *testing.T) { alloc1.ClientStatus = structs.AllocClientStatusRunning // Updating allocation should not throw any error - if err := state.UpdateAllocsFromClient(context.Background(), 400, []*structs.Allocation{alloc1}); err != nil { + if err := state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 400, []*structs.Allocation{alloc1}); err != nil { t.Fatalf("expect err: %v", err) } @@ -5574,7 +5574,7 @@ func TestStateStore_UpdateAlloc_JobNotPresent(t *testing.T) { // Update the alloc again alloc2 := alloc.Copy() alloc2.ClientStatus = structs.AllocClientStatusComplete - if err := state.UpdateAllocsFromClient(context.Background(), 400, []*structs.Allocation{alloc1}); err != nil { + if err := state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 400, []*structs.Allocation{alloc1}); err != nil { t.Fatalf("expect err: %v", err) } @@ -6484,7 +6484,7 @@ func TestStateJobSummary_UpdateJobCount(t *testing.T) { alloc5.JobID = alloc3.JobID alloc5.ClientStatus = structs.AllocClientStatusComplete - if err := state.UpdateAllocsFromClient(context.Background(), 1004, []*structs.Allocation{alloc4, alloc5}); err != nil { + if err := state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1004, []*structs.Allocation{alloc4, alloc5}); err != nil { t.Fatalf("err: %v", err) } @@ -6561,7 +6561,7 @@ func TestJobSummary_UpdateClientStatus(t *testing.T) { alloc6.JobID = alloc.JobID alloc6.ClientStatus = structs.AllocClientStatusRunning - if err := state.UpdateAllocsFromClient(context.Background(), 1002, []*structs.Allocation{alloc4, alloc5, alloc6}); err != nil { + if err := state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1002, []*structs.Allocation{alloc4, alloc5, alloc6}); err != nil { t.Fatalf("err: %v", err) } @@ -6600,7 +6600,7 @@ func TestStateStore_UpsertDeploymentStatusUpdate_Nonexistent(t *testing.T) { Status: structs.DeploymentStatusRunning, }, } - err := state.UpdateDeploymentStatus(context.Background(), 2, req) + err := state.UpdateDeploymentStatus(structs.MsgTypeTestSetup, 2, req) if err == nil || !strings.Contains(err.Error(), "does not exist") { t.Fatalf("expected error updating the status because the deployment doesn't exist") } @@ -6627,7 +6627,7 @@ func TestStateStore_UpsertDeploymentStatusUpdate_Terminal(t *testing.T) { Status: structs.DeploymentStatusRunning, }, } - err := state.UpdateDeploymentStatus(context.Background(), 2, req) + err := state.UpdateDeploymentStatus(structs.MsgTypeTestSetup, 2, req) if err == nil || !strings.Contains(err.Error(), "has terminal status") { t.Fatalf("expected error updating the status because the deployment is terminal") } @@ -6661,7 +6661,7 @@ func TestStateStore_UpsertDeploymentStatusUpdate_NonTerminal(t *testing.T) { Job: j, Eval: e, } - err := state.UpdateDeploymentStatus(context.Background(), 2, req) + err := state.UpdateDeploymentStatus(structs.MsgTypeTestSetup, 2, req) if err != nil { t.Fatalf("bad: %v", err) } @@ -6722,7 +6722,7 @@ func TestStateStore_UpsertDeploymentStatusUpdate_Successful(t *testing.T) { StatusDescription: structs.DeploymentStatusDescriptionSuccessful, }, } - err := state.UpdateDeploymentStatus(context.Background(), 3, req) + err := state.UpdateDeploymentStatus(structs.MsgTypeTestSetup, 3, req) if err != nil { t.Fatalf("bad: %v", err) } @@ -6820,7 +6820,7 @@ func TestStateStore_UpsertDeploymentPromotion_Nonexistent(t *testing.T) { All: true, }, } - err := state.UpdateDeploymentPromotion(context.Background(), 2, req) + err := state.UpdateDeploymentPromotion(structs.MsgTypeTestSetup, 2, req) if err == nil || !strings.Contains(err.Error(), "does not exist") { t.Fatalf("expected error promoting because the deployment doesn't exist") } @@ -6847,7 +6847,7 @@ func TestStateStore_UpsertDeploymentPromotion_Terminal(t *testing.T) { All: true, }, } - err := state.UpdateDeploymentPromotion(context.Background(), 2, req) + err := state.UpdateDeploymentPromotion(structs.MsgTypeTestSetup, 2, req) if err == nil || !strings.Contains(err.Error(), "has terminal status") { t.Fatalf("expected error updating the status because the deployment is terminal: %v", err) } @@ -6897,7 +6897,7 @@ func TestStateStore_UpsertDeploymentPromotion_Unhealthy(t *testing.T) { All: true, }, } - err := state.UpdateDeploymentPromotion(context.Background(), 4, req) + err := state.UpdateDeploymentPromotion(structs.MsgTypeTestSetup, 4, req) require.NotNil(err) require.Contains(err.Error(), `Task group "web" has 0/2 healthy allocations`) } @@ -6926,7 +6926,7 @@ func TestStateStore_UpsertDeploymentPromotion_NoCanaries(t *testing.T) { All: true, }, } - err := state.UpdateDeploymentPromotion(context.Background(), 4, req) + err := state.UpdateDeploymentPromotion(structs.MsgTypeTestSetup, 4, req) require.NotNil(err) require.Contains(err.Error(), `Task group "web" has 0/2 healthy allocations`) } @@ -6997,7 +6997,7 @@ func TestStateStore_UpsertDeploymentPromotion_All(t *testing.T) { }, Eval: e, } - err := state.UpdateDeploymentPromotion(context.Background(), 4, req) + err := state.UpdateDeploymentPromotion(structs.MsgTypeTestSetup, 4, req) if err != nil { t.Fatalf("bad: %v", err) } @@ -7103,7 +7103,7 @@ func TestStateStore_UpsertDeploymentPromotion_Subset(t *testing.T) { }, Eval: e, } - require.Nil(state.UpdateDeploymentPromotion(context.Background(), 4, req)) + require.Nil(state.UpdateDeploymentPromotion(structs.MsgTypeTestSetup, 4, req)) // Check that the status per task group was updated properly ws := memdb.NewWatchSet() @@ -7146,7 +7146,7 @@ func TestStateStore_UpsertDeploymentAllocHealth_Nonexistent(t *testing.T) { HealthyAllocationIDs: []string{uuid.Generate()}, }, } - err := state.UpdateDeploymentAllocHealth(context.Background(), 2, req) + err := state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, 2, req) if err == nil || !strings.Contains(err.Error(), "does not exist") { t.Fatalf("expected error because the deployment doesn't exist: %v", err) } @@ -7173,7 +7173,7 @@ func TestStateStore_UpsertDeploymentAllocHealth_Terminal(t *testing.T) { HealthyAllocationIDs: []string{uuid.Generate()}, }, } - err := state.UpdateDeploymentAllocHealth(context.Background(), 2, req) + err := state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, 2, req) if err == nil || !strings.Contains(err.Error(), "has terminal status") { t.Fatalf("expected error because the deployment is terminal: %v", err) } @@ -7198,7 +7198,7 @@ func TestStateStore_UpsertDeploymentAllocHealth_BadAlloc_Nonexistent(t *testing. HealthyAllocationIDs: []string{uuid.Generate()}, }, } - err := state.UpdateDeploymentAllocHealth(context.Background(), 2, req) + err := state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, 2, req) if err == nil || !strings.Contains(err.Error(), "unknown alloc") { t.Fatalf("expected error because the alloc doesn't exist: %v", err) } @@ -7338,7 +7338,7 @@ func TestStateStore_UpsertDeploymentAllocHealth_BadAlloc_MismatchDeployment(t *t HealthyAllocationIDs: []string{a.ID}, }, } - err := state.UpdateDeploymentAllocHealth(context.Background(), 4, req) + err := state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, 4, req) if err == nil || !strings.Contains(err.Error(), "not part of deployment") { t.Fatalf("expected error because the alloc isn't part of the deployment: %v", err) } @@ -7395,7 +7395,7 @@ func TestStateStore_UpsertDeploymentAllocHealth(t *testing.T) { DeploymentUpdate: u, Timestamp: ts, } - err := state.UpdateDeploymentAllocHealth(context.Background(), 3, req) + err := state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, 3, req) if err != nil { t.Fatalf("bad: %v", err) } diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 6638dc3b358b..c00aa1d490c6 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -107,6 +107,10 @@ const ( // old servers to crash when the FSM attempts to process them. IgnoreUnknownTypeFlag MessageType = 128 + // MsgTypeTestSetup is used during testing when calling state store + // methods directly that require an FSM MessageType + MsgTypeTestSetup MessageType = IgnoreUnknownTypeFlag + // ApiMajorVersion is returned as part of the Status.Version request. // It should be incremented anytime the APIs are changed in a way // that would break clients for sane client versioning. diff --git a/scheduler/generic_sched_test.go b/scheduler/generic_sched_test.go index 61a16326c687..1294604d277f 100644 --- a/scheduler/generic_sched_test.go +++ b/scheduler/generic_sched_test.go @@ -2958,7 +2958,7 @@ func TestServiceSched_NodeUpdate(t *testing.T) { for i := 0; i < 4; i++ { out, _ := h.State.AllocByID(ws, allocs[i].ID) out.ClientStatus = structs.AllocClientStatusRunning - require.NoError(t, h.State.UpdateAllocsFromClient(context.Background(), h.NextIndex(), []*structs.Allocation{out})) + require.NoError(t, h.State.UpdateAllocsFromClient(structs.MsgTypeTestSetup, h.NextIndex(), []*structs.Allocation{out})) } // Create a mock evaluation which won't trigger any new placements diff --git a/scheduler/system_sched_test.go b/scheduler/system_sched_test.go index 3ba722e1385d..937cc14d8dc9 100644 --- a/scheduler/system_sched_test.go +++ b/scheduler/system_sched_test.go @@ -1,7 +1,6 @@ package scheduler import ( - "context" "fmt" "reflect" "sort" @@ -133,7 +132,7 @@ func TestSystemSched_JobRegister_StickyAllocs(t *testing.T) { // Get an allocation and mark it as failed alloc := planned[4].Copy() alloc.ClientStatus = structs.AllocClientStatusFailed - require.NoError(t, h.State.UpdateAllocsFromClient(context.Background(), h.NextIndex(), []*structs.Allocation{alloc})) + require.NoError(t, h.State.UpdateAllocsFromClient(structs.MsgTypeTestSetup, h.NextIndex(), []*structs.Allocation{alloc})) // Create a mock evaluation to handle the update eval = &structs.Evaluation{ diff --git a/scheduler/testing.go b/scheduler/testing.go index 9b43be42f0db..dada4d105a93 100644 --- a/scheduler/testing.go +++ b/scheduler/testing.go @@ -1,7 +1,6 @@ package scheduler import ( - "context" "fmt" "sync" "time" @@ -171,7 +170,7 @@ func (h *Harness) SubmitPlan(plan *structs.Plan) (*structs.PlanResult, State, er } // Apply the full plan - err := h.State.UpsertPlanResults(context.Background(), index, &req) + err := h.State.UpsertPlanResults(structs.MsgTypeTestSetup, index, &req) return result, nil, err } From 004f634868a7e462a47a2fba85c0fc562db511cc Mon Sep 17 00:00:00 2001 From: Drew Bailey <2614075+drewbailey@users.noreply.github.com> Date: Sun, 4 Oct 2020 15:12:35 -0400 Subject: [PATCH 11/19] use Events to wrap index and events, store in events table --- command/agent/event_endpoint.go | 15 +- command/agent/event_endpoint_test.go | 16 +- go.mod | 3 +- go.sum | 5 +- nomad/event_endpoint.go | 2 +- nomad/event_endpoint_test.go | 16 +- nomad/fsm.go | 82 +++++++- nomad/fsm_test.go | 70 +++++++ nomad/mock/mock.go | 22 +++ nomad/state/apply_plan_events.go | 19 +- nomad/state/deployment_events.go | 19 +- ...vent_test.go => deployment_events_test.go} | 8 +- nomad/state/events.go | 33 ++-- nomad/state/node_events.go | 43 +++-- nomad/state/node_events_test.go | 33 ++-- nomad/state/schema_test.go | 14 +- nomad/state/state_changes.go | 61 ++++-- nomad/state/state_store.go | 63 ++++++- nomad/state/testing.go | 7 +- nomad/stream/event.go | 21 --- nomad/stream/event_buffer.go | 16 +- nomad/stream/event_buffer_test.go | 25 +-- nomad/stream/event_publisher.go | 23 +-- nomad/stream/event_publisher_test.go | 17 +- nomad/stream/ndjson.go | 27 +-- nomad/stream/ndjson_test.go | 7 +- nomad/stream/subscription.go | 21 ++- nomad/stream/subscription_test.go | 37 ++-- nomad/structs/structs.go | 34 +++- scheduler/generic_sched_test.go | 5 +- .../hashicorp/go-immutable-radix/CHANGELOG.md | 14 +- .../hashicorp/go-immutable-radix/iter.go | 2 +- .../hashicorp/go-immutable-radix/node.go | 30 +++ .../hashicorp/go-immutable-radix/raw_iter.go | 2 +- .../go-immutable-radix/reverse_iter.go | 177 ++++++++++++++++++ vendor/github.com/hashicorp/go-memdb/go.mod | 2 +- vendor/github.com/hashicorp/go-memdb/go.sum | 4 +- vendor/github.com/hashicorp/go-memdb/txn.go | 111 +++++++++++ .../github.com/hashicorp/nomad/api/event.go | 104 ++++++++++ vendor/modules.txt | 5 +- 40 files changed, 949 insertions(+), 266 deletions(-) rename nomad/state/{deployment_event_test.go => deployment_events_test.go} (97%) delete mode 100644 nomad/stream/event.go create mode 100644 vendor/github.com/hashicorp/go-immutable-radix/reverse_iter.go create mode 100644 vendor/github.com/hashicorp/nomad/api/event.go diff --git a/command/agent/event_endpoint.go b/command/agent/event_endpoint.go index 5684d9c14e86..d68563969130 100644 --- a/command/agent/event_endpoint.go +++ b/command/agent/event_endpoint.go @@ -13,7 +13,6 @@ import ( "github.com/docker/docker/pkg/ioutils" "github.com/hashicorp/go-msgpack/codec" - "github.com/hashicorp/nomad/nomad/stream" "github.com/hashicorp/nomad/nomad/structs" ) @@ -134,12 +133,12 @@ func (s *HTTPServer) EventStream(resp http.ResponseWriter, req *http.Request) (i return nil, codedErr } -func parseEventTopics(query url.Values) (map[stream.Topic][]string, error) { +func parseEventTopics(query url.Values) (map[structs.Topic][]string, error) { raw, ok := query["topic"] if !ok { return allTopics(), nil } - topics := make(map[stream.Topic][]string) + topics := make(map[structs.Topic][]string) for _, topic := range raw { k, v, err := parseTopic(topic) @@ -147,10 +146,10 @@ func parseEventTopics(query url.Values) (map[stream.Topic][]string, error) { return nil, fmt.Errorf("error parsing topics: %w", err) } - if topics[stream.Topic(k)] == nil { - topics[stream.Topic(k)] = []string{v} + if topics[structs.Topic(k)] == nil { + topics[structs.Topic(k)] = []string{v} } else { - topics[stream.Topic(k)] = append(topics[stream.Topic(k)], v) + topics[structs.Topic(k)] = append(topics[structs.Topic(k)], v) } } return topics, nil @@ -164,6 +163,6 @@ func parseTopic(topic string) (string, string, error) { return parts[0], parts[1], nil } -func allTopics() map[stream.Topic][]string { - return map[stream.Topic][]string{"*": {"*"}} +func allTopics() map[structs.Topic][]string { + return map[structs.Topic][]string{"*": {"*"}} } diff --git a/command/agent/event_endpoint_test.go b/command/agent/event_endpoint_test.go index e97b4a30d363..af86cce7cadd 100644 --- a/command/agent/event_endpoint_test.go +++ b/command/agent/event_endpoint_test.go @@ -3,6 +3,7 @@ package agent import ( "context" "fmt" + "github.com/hashicorp/nomad/nomad/structs" "net/http" "net/http/httptest" "net/url" @@ -10,7 +11,6 @@ import ( "testing" "time" - "github.com/hashicorp/nomad/nomad/stream" "github.com/hashicorp/nomad/testutil" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -38,7 +38,7 @@ func TestEventStream(t *testing.T) { pub, err := s.Agent.server.State().EventPublisher() require.NoError(t, err) - pub.Publish(100, []stream.Event{{Payload: testEvent{ID: "123"}}}) + pub.Publish(structs.Events{Index: 100, Events: []structs.Event{{Payload: testEvent{ID: "123"}}}}) testutil.WaitForResult(func() (bool, error) { got := resp.Body.String() @@ -72,20 +72,20 @@ func TestEventStream_QueryParse(t *testing.T) { cases := []struct { desc string query string - want map[stream.Topic][]string + want map[structs.Topic][]string wantErr bool }{ { desc: "all topics and keys specified", query: "?topic=*:*", - want: map[stream.Topic][]string{ + want: map[structs.Topic][]string{ "*": {"*"}, }, }, { desc: "all topics and keys inferred", query: "", - want: map[stream.Topic][]string{ + want: map[structs.Topic][]string{ "*": {"*"}, }, }, @@ -102,14 +102,14 @@ func TestEventStream_QueryParse(t *testing.T) { { desc: "single topic and key", query: "?topic=NodeDrain:*", - want: map[stream.Topic][]string{ + want: map[structs.Topic][]string{ "NodeDrain": {"*"}, }, }, { desc: "single topic multiple keys", query: "?topic=NodeDrain:*&topic=NodeDrain:3caace09-f1f4-4d23-b37a-9ab5eb75069d", - want: map[stream.Topic][]string{ + want: map[structs.Topic][]string{ "NodeDrain": { "*", "3caace09-f1f4-4d23-b37a-9ab5eb75069d", @@ -119,7 +119,7 @@ func TestEventStream_QueryParse(t *testing.T) { { desc: "multiple topics", query: "?topic=NodeRegister:*&topic=NodeDrain:3caace09-f1f4-4d23-b37a-9ab5eb75069d", - want: map[stream.Topic][]string{ + want: map[structs.Topic][]string{ "NodeDrain": { "3caace09-f1f4-4d23-b37a-9ab5eb75069d", }, diff --git a/go.mod b/go.mod index 0995d5df889b..d399bf5f9c79 100644 --- a/go.mod +++ b/go.mod @@ -9,6 +9,7 @@ replace ( github.com/godbus/dbus => github.com/godbus/dbus v5.0.1+incompatible github.com/golang/protobuf => github.com/golang/protobuf v1.3.4 github.com/hashicorp/go-discover => github.com/hashicorp/go-discover v0.0.0-20200812215701-c4b85f6ed31f + github.com/hashicorp/go-memdb => /home/drew/work/go/go-memdb github.com/hashicorp/nomad/api => ./api github.com/kr/pty => github.com/kr/pty v1.1.5 github.com/shirou/gopsutil => github.com/hashicorp/gopsutil v2.18.13-0.20200531184148-5aca383d4f9d+incompatible @@ -62,7 +63,7 @@ require ( github.com/hashicorp/go-envparse v0.0.0-20180119215841-310ca1881b22 github.com/hashicorp/go-getter v1.3.1-0.20190822194507-f5101da01173 github.com/hashicorp/go-hclog v0.12.0 - github.com/hashicorp/go-immutable-radix v1.2.0 + github.com/hashicorp/go-immutable-radix v1.3.0 github.com/hashicorp/go-memdb v1.2.1 github.com/hashicorp/go-msgpack v1.1.5 github.com/hashicorp/go-multierror v1.1.0 diff --git a/go.sum b/go.sum index 795748067c42..ed55baff18f1 100644 --- a/go.sum +++ b/go.sum @@ -392,9 +392,8 @@ github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjh github.com/hashicorp/go-immutable-radix v1.1.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= github.com/hashicorp/go-immutable-radix v1.2.0 h1:l6UW37iCXwZkZoAbEYnptSHVE/cQ5bOTPYG5W3vf9+8= github.com/hashicorp/go-immutable-radix v1.2.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= -github.com/hashicorp/go-memdb v1.0.3/go.mod h1:LWQ8R70vPrS4OEY9k28D2z8/Zzyu34NVzeRibGAzHO0= -github.com/hashicorp/go-memdb v1.2.1 h1:wI9btDjYUOJJHTCnRlAG/TkRyD/ij7meJMrLK9X31Cc= -github.com/hashicorp/go-memdb v1.2.1/go.mod h1:OSvLJ662Jim8hMM+gWGyhktyWk2xPCnWMc7DWIqtkGA= +github.com/hashicorp/go-immutable-radix v1.3.0 h1:8exGP7ego3OmkfksihtSouGMZ+hQrhxx+FVELeXpVPE= +github.com/hashicorp/go-immutable-radix v1.3.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= github.com/hashicorp/go-msgpack v0.5.5/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= github.com/hashicorp/go-msgpack v1.1.5 h1:9byZdVjKTe5mce63pRVNP1L7UAmdHOTEMGehn6KvJWs= diff --git a/nomad/event_endpoint.go b/nomad/event_endpoint.go index 781adb5ad3d5..1e1c94081f6e 100644 --- a/nomad/event_endpoint.go +++ b/nomad/event_endpoint.go @@ -83,7 +83,7 @@ func (e *Event) stream(conn io.ReadWriteCloser) { } defer subscription.Unsubscribe() - ndJsonCh := make(chan *stream.NDJson) + ndJsonCh := make(chan *structs.NDJson) errCh := make(chan error) jsonStream := stream.NewNDJsonStream(ndJsonCh, 30*time.Second) diff --git a/nomad/event_endpoint_test.go b/nomad/event_endpoint_test.go index 793c198a40a7..b05aef2b8906 100644 --- a/nomad/event_endpoint_test.go +++ b/nomad/event_endpoint_test.go @@ -28,7 +28,7 @@ func TestEventStream(t *testing.T) { // Create request for all topics and keys req := structs.EventStreamRequest{ - Topics: map[stream.Topic][]string{"*": []string{"*"}}, + Topics: map[structs.Topic][]string{"*": []string{"*"}}, QueryOptions: structs.QueryOptions{ Region: s1.Region(), }, @@ -68,7 +68,7 @@ func TestEventStream(t *testing.T) { require.NoError(t, err) node := mock.Node() - publisher.Publish(uint64(1), []stream.Event{{Topic: "test", Payload: node}}) + publisher.Publish(structs.Events{Index: uint64(1), Events: []structs.Event{{Topic: "test", Payload: node}}}) encoder := codec.NewEncoder(p1, structs.MsgpackHandle) require.Nil(t, encoder.Encode(req)) @@ -91,7 +91,7 @@ OUTER: continue } - var event stream.Events + var event structs.Events err = json.Unmarshal(msg.Event.Data, &event) require.NoError(t, err) @@ -123,7 +123,7 @@ func TestEventStream_StreamErr(t *testing.T) { testutil.WaitForLeader(t, s1.RPC) req := structs.EventStreamRequest{ - Topics: map[stream.Topic][]string{"*": {"*"}}, + Topics: map[structs.Topic][]string{"*": {"*"}}, QueryOptions: structs.QueryOptions{ Region: s1.Region(), }, @@ -160,7 +160,7 @@ func TestEventStream_StreamErr(t *testing.T) { require.NoError(t, err) node := mock.Node() - publisher.Publish(uint64(1), []stream.Event{{Topic: "test", Payload: node}}) + publisher.Publish(structs.Events{uint64(1), []structs.Event{{Topic: "test", Payload: node}}}) // send req encoder := codec.NewEncoder(p1, structs.MsgpackHandle) @@ -210,7 +210,7 @@ func TestEventStream_RegionForward(t *testing.T) { // Create request targed for region foo req := structs.EventStreamRequest{ - Topics: map[stream.Topic][]string{"*": {"*"}}, + Topics: map[structs.Topic][]string{"*": {"*"}}, QueryOptions: structs.QueryOptions{ Region: "foo", }, @@ -249,7 +249,7 @@ func TestEventStream_RegionForward(t *testing.T) { require.NoError(t, err) node := mock.Node() - publisher.Publish(uint64(1), []stream.Event{{Topic: "test", Payload: node}}) + publisher.Publish(structs.Events{uint64(1), []structs.Event{{Topic: "test", Payload: node}}}) // send req encoder := codec.NewEncoder(p1, structs.MsgpackHandle) @@ -272,7 +272,7 @@ OUTER: continue } - var event stream.Events + var event structs.Events err = json.Unmarshal(msg.Event.Data, &event) require.NoError(t, err) diff --git a/nomad/fsm.go b/nomad/fsm.go index cf6d06a369f2..ce5caaafd5af 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -52,6 +52,7 @@ const ( CSIPluginSnapshot CSIVolumeSnapshot ScalingEventsSnapshot + EventSnapshot ) // LogApplier is the definition of a function that can apply a Raft log @@ -136,6 +137,8 @@ func NewFSM(config *FSMConfig) (*nomadFSM, error) { Logger: config.Logger, Region: config.Region, EnablePublisher: config.EnableEventPublisher, + // TODO(drew) plumb cfg + EnableDurability: true, } state, err := state.NewStateStore(sconfig) if err != nil { @@ -1266,8 +1269,11 @@ func (n *nomadFSM) Restore(old io.ReadCloser) error { // Create a new state store config := &state.StateStoreConfig{ - Logger: n.config.Logger, - Region: n.config.Region, + Logger: n.config.Logger, + Region: n.config.Region, + EnablePublisher: n.config.EnableEventPublisher, + // TODO(drew) plumb cfg + EnableDurability: true, } newState, err := state.NewStateStore(config) if err != nil { @@ -1511,7 +1517,14 @@ func (n *nomadFSM) Restore(old io.ReadCloser) error { if err := restore.CSIVolumeRestore(plugin); err != nil { return err } - + case EventSnapshot: + event := new(structs.Events) + if err := dec.Decode(event); err != nil { + return err + } + if err := restore.EventRestore(event); err != nil { + return err + } default: // Check if this is an enterprise only object being restored restorer, ok := n.enterpriseRestorers[snapType] @@ -1546,6 +1559,24 @@ func (n *nomadFSM) Restore(old io.ReadCloser) error { // blocking queries won't see any changes and need to be woken up. stateOld.Abandon() + // Rehydrate the new state store's event publisher with the events + // persisted in the snapshot + // pub, err := n.state.EventPublisher() + // if err != nil { + // n.logger.Warn("Snapshot Restore: new state event publisher not configured") + // } + // events, err := n.state.Events(nil) + // if err != nil { + // n.logger.Warn("Snapshot Restore: unable to retrieve current events") + // } + // for { + // raw := events.Next() + // if raw == nil { + // break + // } + // e := raw.(*structs.Events) + // pub.Publish(e) + // } return nil } @@ -1823,6 +1854,10 @@ func (s *nomadSnapshot) Persist(sink raft.SnapshotSink) error { sink.Cancel() return err } + if err := s.persistEvents(sink, encoder); err != nil { + sink.Cancel() + return err + } return nil } @@ -2326,6 +2361,47 @@ func (s *nomadSnapshot) persistCSIVolumes(sink raft.SnapshotSink, return nil } +func (s *nomadSnapshot) persistEvents(sink raft.SnapshotSink, encoder *codec.Encoder) error { + var durableCount int + if s.snap.Config() != nil && !s.snap.Config().EnableDurability { + return nil + } else { + durableCount = s.snap.Config().DurableCount + } + + events, err := s.snap.LatestEventsReverse(nil) + if err != nil { + return err + } + + count := 0 + for { + // Get the next item + raw := events.Next() + if raw == nil { + break + } + + // Prepare the request struct + event := raw.(*structs.Events) + + eventCount := len(event.Events) + + // Write out a volume snapshot + sink.Write([]byte{byte(EventSnapshot)}) + if err := encoder.Encode(event); err != nil { + return err + } + count += eventCount + + // Only write to sink until durableCount has been reached + if count >= durableCount { + return nil + } + } + return nil +} + // Release is a no-op, as we just need to GC the pointer // to the state store snapshot. There is nothing to explicitly // cleanup. diff --git a/nomad/fsm_test.go b/nomad/fsm_test.go index 1a0857178e88..81e583568aa2 100644 --- a/nomad/fsm_test.go +++ b/nomad/fsm_test.go @@ -3199,3 +3199,73 @@ func TestFSM_ClusterMetadata(t *testing.T) { r.Equal(clusterID, storedMetadata.ClusterID) r.Equal(now, storedMetadata.CreateTime) } + +func TestFSM_SnapshotRestore_Events_WithDurability(t *testing.T) { + t.Parallel() + // Add some state + fsm := testFSM(t) + state := fsm.State() + cfg := state.Config() + cfg.EnableDurability = true + // DurableCount = 4 each mock events wrapper contains 2 events + cfg.DurableCount = 4 + + e1 := mock.Events(1000) + e2 := mock.Events(1001) + e3 := mock.Events(1002) + + require.NoError(t, state.UpsertEvents(1000, e1)) + require.NoError(t, state.UpsertEvents(1001, e2)) + require.NoError(t, state.UpsertEvents(1002, e3)) + + // Verify the contents + fsm2 := testSnapshotRestore(t, fsm) + state2 := fsm2.State() + + // latest events iterator is newest to oldest + iter, err := state2.LatestEventsReverse(nil) + require.NoError(t, err) + + raw3 := iter.Next() + require.NotNil(t, raw3) + + out3, ok := raw3.(*structs.Events) + require.True(t, ok) + require.Equal(t, e3.Index, out3.Index) + + raw2 := iter.Next() + require.NotNil(t, raw2) + + out2, ok := raw2.(*structs.Events) + require.True(t, ok) + require.Equal(t, e2.Index, out2.Index) + + // Durable count was 4 so e1 events should be excluded + raw1 := iter.Next() + require.Nil(t, raw1) +} + +func TestFSM_SnapshotRestore_Events_NoDurability(t *testing.T) { + t.Parallel() + // Add some state + fsm := testFSM(t) + state := fsm.State() + cfg := state.Config() + cfg.EnableDurability = false + + e1 := mock.Events(1000) + e2 := mock.Events(1001) + + require.NoError(t, state.UpsertEvents(1000, e1)) + require.NoError(t, state.UpsertEvents(1001, e2)) + + // Verify the contents + fsm2 := testSnapshotRestore(t, fsm) + state2 := fsm2.State() + // ws := memdb.NewWatchSet() + out, err := state2.LatestEventsReverse(nil) + require.NoError(t, err) + + raw := out.Next() + require.Nil(t, raw) +} diff --git a/nomad/mock/mock.go b/nomad/mock/mock.go index 338b0f1b5c49..765ca0730fae 100644 --- a/nomad/mock/mock.go +++ b/nomad/mock/mock.go @@ -1471,3 +1471,25 @@ func CSIVolume(plugin *structs.CSIPlugin) *structs.CSIVolume { NodesExpected: len(plugin.Nodes), } } + +func Events(index uint64) *structs.Events { + return &structs.Events{ + Index: index, + Events: []structs.Event{ + { + Index: index, + Topic: "Node", + Type: "update", + Key: uuid.Generate(), + Payload: Node(), + }, + { + Index: index, + Topic: "Eval", + Type: "update", + Key: uuid.Generate(), + Payload: Eval(), + }, + }, + } +} diff --git a/nomad/state/apply_plan_events.go b/nomad/state/apply_plan_events.go index 470c48790b9e..1132dafd7726 100644 --- a/nomad/state/apply_plan_events.go +++ b/nomad/state/apply_plan_events.go @@ -3,21 +3,20 @@ package state import ( "fmt" - "github.com/hashicorp/nomad/nomad/stream" "github.com/hashicorp/nomad/nomad/structs" ) -func ApplyPlanResultEventsFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { - var events []stream.Event +func ApplyPlanResultEventsFromChanges(tx ReadTxn, changes Changes) (structs.Events, error) { + var events []structs.Event for _, change := range changes.Changes { switch change.Table { case "deployment": after, ok := change.After.(*structs.Deployment) if !ok { - return nil, fmt.Errorf("transaction change was not a Deployment") + return structs.Events{}, fmt.Errorf("transaction change was not a Deployment") } - event := stream.Event{ + event := structs.Event{ Topic: TopicDeployment, Type: TypeDeploymentUpdate, Index: changes.Index, @@ -30,10 +29,10 @@ func ApplyPlanResultEventsFromChanges(tx ReadTxn, changes Changes) ([]stream.Eve case "evals": after, ok := change.After.(*structs.Evaluation) if !ok { - return nil, fmt.Errorf("transaction change was not an Evaluation") + return structs.Events{}, fmt.Errorf("transaction change was not an Evaluation") } - event := stream.Event{ + event := structs.Event{ Topic: TopicEval, Index: changes.Index, Key: after.ID, @@ -46,7 +45,7 @@ func ApplyPlanResultEventsFromChanges(tx ReadTxn, changes Changes) ([]stream.Eve case "allocs": after, ok := change.After.(*structs.Allocation) if !ok { - return nil, fmt.Errorf("transaction change was not an Allocation") + return structs.Events{}, fmt.Errorf("transaction change was not an Allocation") } before := change.Before var msg string @@ -56,7 +55,7 @@ func ApplyPlanResultEventsFromChanges(tx ReadTxn, changes Changes) ([]stream.Eve msg = TypeAllocUpdated } - event := stream.Event{ + event := structs.Event{ Topic: TopicAlloc, Type: msg, Index: changes.Index, @@ -70,5 +69,5 @@ func ApplyPlanResultEventsFromChanges(tx ReadTxn, changes Changes) ([]stream.Eve } } - return events, nil + return structs.Events{Index: changes.Index, Events: events}, nil } diff --git a/nomad/state/deployment_events.go b/nomad/state/deployment_events.go index 5f9838f28e25..3f11d351d3ac 100644 --- a/nomad/state/deployment_events.go +++ b/nomad/state/deployment_events.go @@ -3,12 +3,11 @@ package state import ( "fmt" - "github.com/hashicorp/nomad/nomad/stream" "github.com/hashicorp/nomad/nomad/structs" ) -func DeploymentEventFromChanges(msgType structs.MessageType, tx ReadTxn, changes Changes) ([]stream.Event, error) { - var events []stream.Event +func DeploymentEventFromChanges(msgType structs.MessageType, tx ReadTxn, changes Changes) (structs.Events, error) { + var events []structs.Event var eventType string switch msgType { @@ -25,10 +24,10 @@ func DeploymentEventFromChanges(msgType structs.MessageType, tx ReadTxn, changes case "deployment": after, ok := change.After.(*structs.Deployment) if !ok { - return nil, fmt.Errorf("transaction change was not a Deployment") + return structs.Events{}, fmt.Errorf("transaction change was not a Deployment") } - event := stream.Event{ + event := structs.Event{ Topic: TopicDeployment, Type: eventType, Index: changes.Index, @@ -43,10 +42,10 @@ func DeploymentEventFromChanges(msgType structs.MessageType, tx ReadTxn, changes case "jobs": after, ok := change.After.(*structs.Job) if !ok { - return nil, fmt.Errorf("transaction change was not a Job") + return structs.Events{}, fmt.Errorf("transaction change was not a Job") } - event := stream.Event{ + event := structs.Event{ Topic: TopicJob, Type: eventType, Index: changes.Index, @@ -62,10 +61,10 @@ func DeploymentEventFromChanges(msgType structs.MessageType, tx ReadTxn, changes case "evals": after, ok := change.After.(*structs.Evaluation) if !ok { - return nil, fmt.Errorf("transaction change was not an Evaluation") + return structs.Events{}, fmt.Errorf("transaction change was not an Evaluation") } - event := stream.Event{ + event := structs.Event{ Topic: TopicEval, Type: eventType, Index: changes.Index, @@ -80,5 +79,5 @@ func DeploymentEventFromChanges(msgType structs.MessageType, tx ReadTxn, changes } } - return events, nil + return structs.Events{Index: changes.Index, Events: events}, nil } diff --git a/nomad/state/deployment_event_test.go b/nomad/state/deployment_events_test.go similarity index 97% rename from nomad/state/deployment_event_test.go rename to nomad/state/deployment_events_test.go index 14fb3482edbb..d85c9d395bd8 100644 --- a/nomad/state/deployment_event_test.go +++ b/nomad/state/deployment_events_test.go @@ -135,7 +135,7 @@ func TestDeploymentEventFromChanges_Promotion(t *testing.T) { require.Equal(t, structs.DeploymentStatusRunning, de.Deployment.Status) } -func WaitForEvents(t *testing.T, s *StateStore, index uint64, minEvents int, timeout time.Duration) []stream.Event { +func WaitForEvents(t *testing.T, s *StateStore, index uint64, minEvents int, timeout time.Duration) []structs.Event { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -162,12 +162,12 @@ func WaitForEvents(t *testing.T, s *StateStore, index uint64, minEvents int, tim } } -func EventsForIndex(t *testing.T, s *StateStore, index uint64) []stream.Event { +func EventsForIndex(t *testing.T, s *StateStore, index uint64) []structs.Event { pub, err := s.EventPublisher() require.NoError(t, err) sub, err := pub.Subscribe(&stream.SubscribeRequest{ - Topics: map[stream.Topic][]string{ + Topics: map[structs.Topic][]string{ "*": []string{"*"}, }, Index: index, @@ -176,7 +176,7 @@ func EventsForIndex(t *testing.T, s *StateStore, index uint64) []stream.Event { require.NoError(t, err) - var events []stream.Event + var events []structs.Event for { e, err := sub.NextNoBlock() require.NoError(t, err) diff --git a/nomad/state/events.go b/nomad/state/events.go index 7e0d95daab9f..52f9450ccca8 100644 --- a/nomad/state/events.go +++ b/nomad/state/events.go @@ -3,19 +3,18 @@ package state import ( "fmt" - "github.com/hashicorp/nomad/nomad/stream" "github.com/hashicorp/nomad/nomad/structs" ) const ( - TopicDeployment stream.Topic = "Deployment" - TopicEval stream.Topic = "Eval" - TopicAlloc stream.Topic = "Alloc" - TopicJob stream.Topic = "Job" + TopicDeployment structs.Topic = "Deployment" + TopicEval structs.Topic = "Eval" + TopicAlloc structs.Topic = "Alloc" + TopicJob structs.Topic = "Job" // TopicNodeRegistration stream.Topic = "NodeRegistration" // TopicNodeDeregistration stream.Topic = "NodeDeregistration" // TopicNodeDrain stream.Topic = "NodeDrain" - TopicNode stream.Topic = "Node" + TopicNode structs.Topic = "Node" // TODO(drew) Node Events use TopicNode + Type TypeNodeRegistration = "NodeRegistration" @@ -73,7 +72,7 @@ type JobDrainDetails struct { AllocDetails map[string]NodeDrainAllocDetails } -func GenericEventsFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { +func GenericEventsFromChanges(tx ReadTxn, changes Changes) (structs.Events, error) { var eventType string switch changes.MsgType { case structs.EvalUpdateRequestType: @@ -88,16 +87,16 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, erro eventType = TypeNodeEvent } - var events []stream.Event + var events []structs.Event for _, change := range changes.Changes { switch change.Table { case "evals": after, ok := change.After.(*structs.Evaluation) if !ok { - return nil, fmt.Errorf("transaction change was not an Evaluation") + return structs.Events{}, fmt.Errorf("transaction change was not an Evaluation") } - event := stream.Event{ + event := structs.Event{ Topic: TopicEval, Type: eventType, Index: changes.Index, @@ -112,10 +111,10 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, erro case "allocs": after, ok := change.After.(*structs.Allocation) if !ok { - return nil, fmt.Errorf("transaction change was not an Allocation") + return structs.Events{}, fmt.Errorf("transaction change was not an Allocation") } - event := stream.Event{ + event := structs.Event{ Topic: TopicAlloc, Type: eventType, Index: changes.Index, @@ -129,10 +128,10 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, erro case "jobs": after, ok := change.After.(*structs.Job) if !ok { - return nil, fmt.Errorf("transaction change was not an Allocation") + return structs.Events{}, fmt.Errorf("transaction change was not an Allocation") } - event := stream.Event{ + event := structs.Event{ Topic: TopicAlloc, Type: eventType, Index: changes.Index, @@ -146,10 +145,10 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, erro case "nodes": after, ok := change.After.(*structs.Node) if !ok { - return nil, fmt.Errorf("transaction change was not a Node") + return structs.Events{}, fmt.Errorf("transaction change was not a Node") } - event := stream.Event{ + event := structs.Event{ Topic: TopicNode, Type: eventType, Index: changes.Index, @@ -162,5 +161,5 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, erro } } - return events, nil + return structs.Events{Index: changes.Index, Events: events}, nil } diff --git a/nomad/state/node_events.go b/nomad/state/node_events.go index a9e55aeb10b9..e709f3cf32a0 100644 --- a/nomad/state/node_events.go +++ b/nomad/state/node_events.go @@ -3,23 +3,22 @@ package state import ( "fmt" - "github.com/hashicorp/nomad/nomad/stream" "github.com/hashicorp/nomad/nomad/structs" ) // NodeRegisterEventFromChanges generates a NodeRegistrationEvent from a set // of transaction changes. -func NodeRegisterEventFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { - var events []stream.Event +func NodeRegisterEventFromChanges(tx ReadTxn, changes Changes) (structs.Events, error) { + var events []structs.Event for _, change := range changes.Changes { switch change.Table { case "nodes": after, ok := change.After.(*structs.Node) if !ok { - return nil, fmt.Errorf("transaction change was not a Node") + return structs.Events{}, fmt.Errorf("transaction change was not a Node") } - event := stream.Event{ + event := structs.Event{ Topic: TopicNode, Type: TypeNodeRegistration, Index: changes.Index, @@ -31,22 +30,22 @@ func NodeRegisterEventFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, events = append(events, event) } } - return events, nil + return structs.Events{Index: changes.Index, Events: events}, nil } // NodeDeregisterEventFromChanges generates a NodeDeregistrationEvent from a set // of transaction changes. -func NodeDeregisterEventFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { - var events []stream.Event +func NodeDeregisterEventFromChanges(tx ReadTxn, changes Changes) (structs.Events, error) { + var events []structs.Event for _, change := range changes.Changes { switch change.Table { case "nodes": before, ok := change.Before.(*structs.Node) if !ok { - return nil, fmt.Errorf("transaction change was not a Node") + return structs.Events{}, fmt.Errorf("transaction change was not a Node") } - event := stream.Event{ + event := structs.Event{ Topic: TopicNode, Type: TypeNodeDeregistration, Index: changes.Index, @@ -58,22 +57,22 @@ func NodeDeregisterEventFromChanges(tx ReadTxn, changes Changes) ([]stream.Event events = append(events, event) } } - return events, nil + return structs.Events{Index: changes.Index, Events: events}, nil } // NodeEventFromChanges generates a NodeDeregistrationEvent from a set // of transaction changes. -func NodeEventFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { - var events []stream.Event +func NodeEventFromChanges(tx ReadTxn, changes Changes) (structs.Events, error) { + var events []structs.Event for _, change := range changes.Changes { switch change.Table { case "nodes": after, ok := change.After.(*structs.Node) if !ok { - return nil, fmt.Errorf("transaction change was not a Node") + return structs.Events{}, fmt.Errorf("transaction change was not a Node") } - event := stream.Event{ + event := structs.Event{ Topic: TopicNode, Type: TypeNodeEvent, Index: changes.Index, @@ -85,23 +84,23 @@ func NodeEventFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { events = append(events, event) } } - return events, nil + return structs.Events{Index: changes.Index, Events: events}, nil } -func NodeDrainEventFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { - var events []stream.Event +func NodeDrainEventFromChanges(tx ReadTxn, changes Changes) (structs.Events, error) { + var events []structs.Event for _, change := range changes.Changes { switch change.Table { case "nodes": after, ok := change.After.(*structs.Node) if !ok { - return nil, fmt.Errorf("transaction change was not a Node") + return structs.Events{}, fmt.Errorf("transaction change was not a Node") } // retrieve allocations currently on node allocs, err := allocsByNodeTxn(tx, nil, after.ID) if err != nil { - return nil, fmt.Errorf("retrieving allocations for node drain event: %w", err) + return structs.Events{}, fmt.Errorf("retrieving allocations for node drain event: %w", err) } // build job/alloc details for node drain @@ -120,7 +119,7 @@ func NodeDrainEventFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, err } } - event := stream.Event{ + event := structs.Event{ Topic: TopicNode, Type: TypeNodeDrain, Index: changes.Index, @@ -133,5 +132,5 @@ func NodeDrainEventFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, err events = append(events, event) } } - return events, nil + return structs.Events{Index: changes.Index, Events: events}, nil } diff --git a/nomad/state/node_events_test.go b/nomad/state/node_events_test.go index 78918db23578..e32e338ec85a 100644 --- a/nomad/state/node_events_test.go +++ b/nomad/state/node_events_test.go @@ -5,7 +5,6 @@ import ( "time" "github.com/hashicorp/nomad/nomad/mock" - "github.com/hashicorp/nomad/nomad/stream" "github.com/hashicorp/nomad/nomad/structs" "github.com/stretchr/testify/require" ) @@ -16,9 +15,9 @@ func TestNodeEventsFromChanges(t *testing.T) { MsgType structs.MessageType Setup func(s *StateStore, tx *txn) error Mutate func(s *StateStore, tx *txn) error - WantEvents []stream.Event + WantEvents []structs.Event WantErr bool - WantTopic stream.Topic + WantTopic structs.Topic }{ { MsgType: structs.NodeRegisterRequestType, @@ -27,7 +26,7 @@ func TestNodeEventsFromChanges(t *testing.T) { Mutate: func(s *StateStore, tx *txn) error { return upsertNodeTxn(tx, tx.Index, testNode()) }, - WantEvents: []stream.Event{{ + WantEvents: []structs.Event{{ Topic: TopicNode, Type: TypeNodeRegistration, Key: testNodeID(), @@ -45,7 +44,7 @@ func TestNodeEventsFromChanges(t *testing.T) { Mutate: func(s *StateStore, tx *txn) error { return upsertNodeTxn(tx, tx.Index, testNode(nodeNotReady)) }, - WantEvents: []stream.Event{{ + WantEvents: []structs.Event{{ Topic: TopicNode, Type: TypeNodeRegistration, Key: testNodeID(), @@ -66,7 +65,7 @@ func TestNodeEventsFromChanges(t *testing.T) { Mutate: func(s *StateStore, tx *txn) error { return deleteNodeTxn(tx, tx.Index, []string{testNodeID()}) }, - WantEvents: []stream.Event{{ + WantEvents: []structs.Event{{ Topic: TopicNode, Type: TypeNodeDeregistration, Key: testNodeID(), @@ -88,7 +87,7 @@ func TestNodeEventsFromChanges(t *testing.T) { Mutate: func(s *StateStore, tx *txn) error { return deleteNodeTxn(tx, tx.Index, []string{testNodeID(), testNodeIDTwo()}) }, - WantEvents: []stream.Event{ + WantEvents: []structs.Event{ { Topic: TopicNode, Type: TypeNodeDeregistration, @@ -140,7 +139,7 @@ func TestNodeEventsFromChanges(t *testing.T) { require.NoError(t, s.upsertNodeEvents(tx.Index, testNodeID(), eventFn(testNodeID()), tx)) return s.upsertNodeEvents(tx.Index, testNodeIDTwo(), eventFn(testNodeIDTwo()), tx) }, - WantEvents: []stream.Event{ + WantEvents: []structs.Event{ { Topic: TopicNode, Type: TypeNodeEvent, @@ -188,8 +187,8 @@ func TestNodeEventsFromChanges(t *testing.T) { } require.NoError(t, err) - require.Equal(t, len(tc.WantEvents), len(got)) - for idx, g := range got { + require.Equal(t, len(tc.WantEvents), len(got.Events)) + for idx, g := range got.Events { // assert equality of shared fields want := tc.WantEvents[idx] @@ -251,18 +250,18 @@ func TestNodeDrainEventFromChanges(t *testing.T) { require.Len(t, got, 1) - require.Equal(t, TopicNode, got[0].Topic) - require.Equal(t, TypeNodeDrain, got[0].Type) - require.Equal(t, uint64(100), got[0].Index) + require.Equal(t, TopicNode, got.Events[0].Topic) + require.Equal(t, TypeNodeDrain, got.Events[0].Type) + require.Equal(t, uint64(100), got.Events[0].Index) - nodeEvent, ok := got[0].Payload.(*NodeDrainEvent) + nodeEvent, ok := got.Events[0].Payload.(*NodeDrainEvent) require.True(t, ok) require.Equal(t, structs.NodeSchedulingIneligible, nodeEvent.Node.SchedulingEligibility) require.Equal(t, strat, nodeEvent.Node.DrainStrategy) } -func requireNodeRegistrationEventEqual(t *testing.T, want, got stream.Event) { +func requireNodeRegistrationEventEqual(t *testing.T, want, got structs.Event) { t.Helper() wantPayload := want.Payload.(*NodeEvent) @@ -274,7 +273,7 @@ func requireNodeRegistrationEventEqual(t *testing.T, want, got stream.Event) { require.NotEqual(t, wantPayload.Node.Events, gotPayload.Node.Events) } -func requireNodeDeregistrationEventEqual(t *testing.T, want, got stream.Event) { +func requireNodeDeregistrationEventEqual(t *testing.T, want, got structs.Event) { t.Helper() wantPayload := want.Payload.(*NodeEvent) @@ -284,7 +283,7 @@ func requireNodeDeregistrationEventEqual(t *testing.T, want, got stream.Event) { require.NotEqual(t, wantPayload.Node.Events, gotPayload.Node.Events) } -func requireNodeEventEqual(t *testing.T, want, got stream.Event) { +func requireNodeEventEqual(t *testing.T, want, got structs.Event) { gotPayload := got.Payload.(*NodeEvent) require.Len(t, gotPayload.Node.Events, 3) diff --git a/nomad/state/schema_test.go b/nomad/state/schema_test.go index 267c6f8742b0..fd2f04b4c3f9 100644 --- a/nomad/state/schema_test.go +++ b/nomad/state/schema_test.go @@ -1,11 +1,11 @@ package state import ( + "github.com/hashicorp/nomad/nomad/structs" "testing" memdb "github.com/hashicorp/go-memdb" "github.com/hashicorp/nomad/nomad/mock" - "github.com/hashicorp/nomad/nomad/stream" "github.com/stretchr/testify/require" ) @@ -178,26 +178,26 @@ func TestEventTableUintIndex(t *testing.T) { return num } - insertEvents := func(e *stream.Events) { + insertEvents := func(e *structs.Events) { txn := db.Txn(true) err := txn.Insert(eventsTable, e) require.NoError(err) txn.Commit() } - get := func(idx uint64) *stream.Events { + get := func(idx uint64) *structs.Events { txn := db.Txn(false) defer txn.Abort() record, err := txn.First("events", "id", idx) require.NoError(err) - s, ok := record.(*stream.Events) + s, ok := record.(*structs.Events) require.True(ok) return s } - firstEvent := &stream.Events{Index: 10, Events: []stream.Event{{Index: 10}, {Index: 10}}} - secondEvent := &stream.Events{Index: 11, Events: []stream.Event{{Index: 11}, {Index: 11}}} - thirdEvent := &stream.Events{Index: 202, Events: []stream.Event{{Index: 202}, {Index: 202}}} + firstEvent := &structs.Events{Index: 10, Events: []structs.Event{{Index: 10}, {Index: 10}}} + secondEvent := &structs.Events{Index: 11, Events: []structs.Event{{Index: 11}, {Index: 11}}} + thirdEvent := &structs.Events{Index: 202, Events: []structs.Event{{Index: 202}, {Index: 202}}} insertEvents(firstEvent) insertEvents(secondEvent) insertEvents(thirdEvent) diff --git a/nomad/state/state_changes.go b/nomad/state/state_changes.go index 117eebc29ea5..a8c48b1ff1fb 100644 --- a/nomad/state/state_changes.go +++ b/nomad/state/state_changes.go @@ -34,21 +34,35 @@ type Changes struct { // sent to the EventPublisher which will create and emit change events. type changeTrackerDB struct { db *memdb.MemDB + durableEvents bool + durableCount int publisher *stream.EventPublisher - processChanges func(ReadTxn, Changes) ([]stream.Event, error) + processChanges func(ReadTxn, Changes) (structs.Events, error) } -func NewChangeTrackerDB(db *memdb.MemDB, publisher *stream.EventPublisher, changesFn changeProcessor) *changeTrackerDB { +// ChangeConfig +type ChangeConfig struct { + DurableEvents bool + DurableCount int +} + +func NewChangeTrackerDB(db *memdb.MemDB, publisher *stream.EventPublisher, changesFn changeProcessor, cfg *ChangeConfig) *changeTrackerDB { + if cfg == nil { + cfg = &ChangeConfig{} + } + return &changeTrackerDB{ db: db, publisher: publisher, processChanges: changesFn, + durableEvents: cfg.DurableEvents, + durableCount: cfg.DurableCount, } } -type changeProcessor func(ReadTxn, Changes) ([]stream.Event, error) +type changeProcessor func(ReadTxn, Changes) (structs.Events, error) -func noOpProcessChanges(ReadTxn, Changes) ([]stream.Event, error) { return []stream.Event{}, nil } +func noOpProcessChanges(ReadTxn, Changes) (structs.Events, error) { return structs.Events{}, nil } // ReadTxn returns a read-only transaction which behaves exactly the same as // memdb.Txn @@ -81,25 +95,27 @@ func (c *changeTrackerDB) WriteTxn(idx uint64) *txn { func (c *changeTrackerDB) WriteTxnMsgT(msgType structs.MessageType, idx uint64) *txn { t := &txn{ - msgType: msgType, - Txn: c.db.Txn(true), - Index: idx, - publish: c.publish, + msgType: msgType, + Txn: c.db.Txn(true), + Index: idx, + publish: c.publish, + persistChanges: c.durableEvents, } t.Txn.TrackChanges() return t } -func (c *changeTrackerDB) publish(changes Changes) error { +func (c *changeTrackerDB) publish(changes Changes) (structs.Events, error) { readOnlyTx := c.db.Txn(false) defer readOnlyTx.Abort() events, err := c.processChanges(readOnlyTx, changes) if err != nil { - return fmt.Errorf("failed generating events from changes: %v", err) + return structs.Events{}, fmt.Errorf("failed generating events from changes: %v", err) } - c.publisher.Publish(changes.Index, events) - return nil + + c.publisher.Publish(events) + return events, nil } // WriteTxnRestore returns a wrapped RW transaction that does NOT have change @@ -125,13 +141,15 @@ type txn struct { // msgType is used to inform event sourcing which type of event to create msgType structs.MessageType + persistChanges bool + *memdb.Txn // Index in raft where the write is occurring. The value is zero for a // read-only, or WriteTxnRestore transaction. // Index is stored so that it may be passed along to any subscribers as part // of a change event. Index uint64 - publish func(changes Changes) error + publish func(changes Changes) (structs.Events, error) } // Commit first pushes changes to EventPublisher, then calls Commit on the @@ -150,9 +168,18 @@ func (tx *txn) Commit() error { Changes: tx.Txn.Changes(), MsgType: tx.MsgType(), } - if err := tx.publish(changes); err != nil { + events, err := tx.publish(changes) + if err != nil { return err } + + if tx.persistChanges { + // persist events after processing changes + err := tx.Txn.Insert("events", events) + if err != nil { + return err + } + } } tx.Txn.Commit() @@ -166,11 +193,11 @@ func (tx *txn) MsgType() structs.MessageType { return tx.msgType } -func processDBChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { +func processDBChanges(tx ReadTxn, changes Changes) (structs.Events, error) { switch changes.MsgType { case structs.IgnoreUnknownTypeFlag: // unknown event type - return []stream.Event{}, nil + return structs.Events{}, nil case structs.NodeRegisterRequestType: return NodeRegisterEventFromChanges(tx, changes) case structs.NodeUpdateStatusRequestType: @@ -201,5 +228,5 @@ func processDBChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) { // TODO(drew) test return GenericEventsFromChanges(tx, changes) } - return []stream.Event{}, nil + return structs.Events{}, nil } diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index a2271676fb42..18efdfd36ba9 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -47,6 +47,9 @@ type StateStoreConfig struct { Region string EnablePublisher bool + + EnableDurability bool + DurableCount int } // The StateStore is responsible for maintaining all the Nomad @@ -90,14 +93,18 @@ func NewStateStore(config *StateStoreConfig) (*StateStore, error) { } if config.EnablePublisher { + cfg := &ChangeConfig{ + DurableEvents: config.EnableDurability, + DurableCount: 1000, + } publisher := stream.NewEventPublisher(ctx, stream.EventPublisherCfg{ EventBufferTTL: 1 * time.Hour, EventBufferSize: 250, Logger: config.Logger, }) - s.db = NewChangeTrackerDB(db, publisher, processDBChanges) + s.db = NewChangeTrackerDB(db, publisher, processDBChanges, cfg) } else { - s.db = NewChangeTrackerDB(db, nil, noOpProcessChanges) + s.db = NewChangeTrackerDB(db, nil, noOpProcessChanges, nil) } // Initialize the state store with required enterprise objects @@ -132,7 +139,7 @@ func (s *StateStore) Snapshot() (*StateSnapshot, error) { } // Create a new change tracker DB that does not publish or track changes - store.db = NewChangeTrackerDB(memDBSnap, nil, noOpProcessChanges) + store.db = NewChangeTrackerDB(memDBSnap, nil, noOpProcessChanges, nil) snap := &StateSnapshot{ StateStore: store, @@ -5752,6 +5759,49 @@ func (s *StateStore) ScalingPolicyByTargetAndType(ws memdb.WatchSet, target map[ return nil, nil } +// LatestEventsReverse returns the unfiltered list of all volumes +func (s *StateStore) LatestEventsReverse(ws memdb.WatchSet) (memdb.ResultIterator, error) { + txn := s.db.ReadTxn() + defer txn.Abort() + + iter, err := txn.GetReverse("events", "id") + if err != nil { + return nil, fmt.Errorf("events lookup failed: %v", err) + } + + ws.Add(iter.WatchCh()) + + return iter, nil +} + +// Events returns the unfiltered list of all volumes +func (s *StateStore) Events(ws memdb.WatchSet) (memdb.ResultIterator, error) { + txn := s.db.ReadTxn() + defer txn.Abort() + + iter, err := txn.Get("events", "id") + if err != nil { + return nil, fmt.Errorf("events lookup failed: %v", err) + } + + ws.Add(iter.WatchCh()) + + return iter, nil +} + +// UpsertEvents is used to insert events. It should only be used for testing. +// Normal use events are inserted to go-memdb during transaction commit +func (s *StateStore) UpsertEvents(index uint64, events *structs.Events) error { + txn := s.db.WriteTxn(index) + defer txn.Abort() + + if err := txn.Insert("events", events); err != nil { + return err + } + txn.Commit() + return nil +} + // StateSnapshot is used to provide a point-in-time snapshot type StateSnapshot struct { StateStore @@ -5999,6 +6049,13 @@ func (r *StateRestore) CSIVolumeRestore(volume *structs.CSIVolume) error { return nil } +func (r *StateRestore) EventRestore(events *structs.Events) error { + if err := r.txn.Insert("events", events); err != nil { + return fmt.Errorf("events insert failed: %v", err) + } + return nil +} + func (r *StateRestore) ScalingEventsRestore(jobEvents *structs.JobScalingEvents) error { if err := r.txn.Insert("scaling_event", jobEvents); err != nil { return fmt.Errorf("scaling event insert failed: %v", err) diff --git a/nomad/state/testing.go b/nomad/state/testing.go index 6aa9039f4d79..86acaf9e7820 100644 --- a/nomad/state/testing.go +++ b/nomad/state/testing.go @@ -26,9 +26,10 @@ func TestStateStore(t testing.T) *StateStore { func TestStateStorePublisher(t testing.T) *StateStoreConfig { return &StateStoreConfig{ - Logger: testlog.HCLogger(t), - Region: "global", - EnablePublisher: true, + Logger: testlog.HCLogger(t), + Region: "global", + EnablePublisher: true, + EnableDurability: true, } } func TestStateStoreCfg(t testing.T, cfg *StateStoreConfig) *StateStore { diff --git a/nomad/stream/event.go b/nomad/stream/event.go deleted file mode 100644 index 33679723e177..000000000000 --- a/nomad/stream/event.go +++ /dev/null @@ -1,21 +0,0 @@ -package stream - -const ( - AllKeys = "*" -) - -type Topic string - -type Event struct { - Topic Topic - Type string - Key string - FilterKeys []string - Index uint64 - Payload interface{} -} - -type Events struct { - Index uint64 - Events []Event -} diff --git a/nomad/stream/event_buffer.go b/nomad/stream/event_buffer.go index 145776225a2c..fda04fa4f355 100644 --- a/nomad/stream/event_buffer.go +++ b/nomad/stream/event_buffer.go @@ -6,6 +6,8 @@ import ( "fmt" "sync/atomic" "time" + + "github.com/hashicorp/nomad/nomad/structs" ) // eventBuffer is a single-writer, multiple-reader, fixed length concurrent @@ -60,7 +62,7 @@ func newEventBuffer(size int64, maxItemTTL time.Duration) *eventBuffer { maxItemTTL: maxItemTTL, } - item := newBufferItem(0, nil) + item := newBufferItem(structs.Events{Index: 0, Events: nil}) b.head.Store(item) b.tail.Store(item) @@ -73,8 +75,8 @@ func newEventBuffer(size int64, maxItemTTL time.Duration) *eventBuffer { // mutations to the events as they may have been exposed to subscribers in other // goroutines. Append only supports a single concurrent caller and must be // externally synchronized with other Append, AppendBuffer or AppendErr calls. -func (b *eventBuffer) Append(index uint64, events []Event) { - b.appendItem(newBufferItem(index, events)) +func (b *eventBuffer) Append(events structs.Events) { + b.appendItem(newBufferItem(events)) } func (b *eventBuffer) appendItem(item *bufferItem) { @@ -200,7 +202,7 @@ type bufferItem struct { // should check and skip nil Events at any point in the buffer. It will also // be nil if the producer appends an Error event because they can't complete // the request to populate the buffer. Err will be non-nil in this case. - Events []Event + Events []structs.Event Index uint64 @@ -237,14 +239,14 @@ type bufferLink struct { // newBufferItem returns a blank buffer item with a link and chan ready to have // the fields set and be appended to a buffer. -func newBufferItem(index uint64, events []Event) *bufferItem { +func newBufferItem(events structs.Events) *bufferItem { return &bufferItem{ link: &bufferLink{ ch: make(chan struct{}), droppedCh: make(chan struct{}), }, - Events: events, - Index: index, + Events: events.Events, + Index: events.Index, createdAt: time.Now(), } } diff --git a/nomad/stream/event_buffer_test.go b/nomad/stream/event_buffer_test.go index 15a96a4e0d76..5b1b6e62333e 100644 --- a/nomad/stream/event_buffer_test.go +++ b/nomad/stream/event_buffer_test.go @@ -3,6 +3,7 @@ package stream import ( "context" "fmt" + "github.com/hashicorp/nomad/nomad/structs" "math/rand" "testing" "time" @@ -31,10 +32,10 @@ func TestEventBufferFuzz(t *testing.T) { for i := 0; i < nMessages; i++ { // Event content is arbitrary and not valid for our use of buffers in // streaming - here we only care about the semantics of the buffer. - e := Event{ + e := structs.Event{ Index: uint64(i), // Indexes should be contiguous } - b.Append(uint64(i), []Event{e}) + b.Append(structs.Events{Index: uint64(i), Events: []structs.Event{e}}) // Sleep sometimes for a while to let some subscribers catch up wait := time.Duration(z.Uint64()) * time.Millisecond time.Sleep(wait) @@ -87,19 +88,19 @@ func TestEventBuffer_Slow_Reader(t *testing.T) { b := newEventBuffer(10, DefaultTTL) for i := 0; i < 10; i++ { - e := Event{ + e := structs.Event{ Index: uint64(i), // Indexes should be contiguous } - b.Append(uint64(i), []Event{e}) + b.Append(structs.Events{uint64(i), []structs.Event{e}}) } head := b.Head() for i := 10; i < 15; i++ { - e := Event{ + e := structs.Event{ Index: uint64(i), // Indexes should be contiguous } - b.Append(uint64(i), []Event{e}) + b.Append(structs.Events{uint64(i), []structs.Event{e}}) } // Ensure the slow reader errors to handle dropped events and @@ -116,10 +117,10 @@ func TestEventBuffer_Size(t *testing.T) { b := newEventBuffer(100, DefaultTTL) for i := 0; i < 10; i++ { - e := Event{ + e := structs.Event{ Index: uint64(i), // Indexes should be contiguous } - b.Append(uint64(i), []Event{e}) + b.Append(structs.Events{uint64(i), []structs.Event{e}}) } require.Equal(t, 10, b.Len()) @@ -132,10 +133,10 @@ func TestEventBuffer_Prune_AllOld(t *testing.T) { b := newEventBuffer(100, 1*time.Second) for i := 0; i < 10; i++ { - e := Event{ + e := structs.Event{ Index: uint64(i), // Indexes should be contiguous } - b.Append(uint64(i), []Event{e}) + b.Append(structs.Events{uint64(i), []structs.Event{e}}) } require.Equal(t, 10, int(b.Len())) @@ -185,10 +186,10 @@ func TestStartAt_CurrentIdx_Past_Start(t *testing.T) { b := newEventBuffer(100, 1*time.Hour) for i := 11; i <= 100; i++ { - e := Event{ + e := structs.Event{ Index: uint64(i), // Indexes should be contiguous } - b.Append(uint64(i), []Event{e}) + b.Append(structs.Events{uint64(i), []structs.Event{e}}) } for _, tc := range cases { diff --git a/nomad/stream/event_publisher.go b/nomad/stream/event_publisher.go index 0a6bb79cdfc4..16e7f0a73fc8 100644 --- a/nomad/stream/event_publisher.go +++ b/nomad/stream/event_publisher.go @@ -5,6 +5,8 @@ import ( "sync" "time" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/go-hclog" ) @@ -36,7 +38,7 @@ type EventPublisher struct { // publishCh is used to send messages from an active txn to a goroutine which // publishes events, so that publishing can happen asynchronously from // the Commit call in the FSM hot path. - publishCh chan changeEvents + publishCh chan structs.Events } type subscriptions struct { @@ -65,7 +67,7 @@ func NewEventPublisher(ctx context.Context, cfg EventPublisherCfg) *EventPublish e := &EventPublisher{ logger: cfg.Logger.Named("event_publisher"), eventBuf: buffer, - publishCh: make(chan changeEvents, 64), + publishCh: make(chan structs.Events, 64), subscriptions: &subscriptions{ byToken: make(map[string]map[*SubscribeRequest]*Subscription), }, @@ -79,9 +81,9 @@ func NewEventPublisher(ctx context.Context, cfg EventPublisherCfg) *EventPublish } // Publish events to all subscribers of the event Topic. -func (e *EventPublisher) Publish(index uint64, events []Event) { - if len(events) > 0 { - e.publishCh <- changeEvents{index: index, events: events} +func (e *EventPublisher) Publish(events structs.Events) { + if len(events.Events) > 0 { + e.publishCh <- events } } @@ -102,7 +104,7 @@ func (e *EventPublisher) Subscribe(req *SubscribeRequest) (*Subscription, error) } // Empty head so that calling Next on sub - start := newBufferItem(req.Index, []Event{}) + start := newBufferItem(structs.Events{Index: req.Index}) start.link.next.Store(head) close(start.link.ch) @@ -137,17 +139,12 @@ func (e *EventPublisher) periodicPrune(ctx context.Context) { } } -type changeEvents struct { - index uint64 - events []Event -} - // sendEvents sends the given events to the publishers event buffer. -func (e *EventPublisher) sendEvents(update changeEvents) { +func (e *EventPublisher) sendEvents(update structs.Events) { e.lock.Lock() defer e.lock.Unlock() - e.eventBuf.Append(update.index, update.events) + e.eventBuf.Append(update) } func (s *subscriptions) add(req *SubscribeRequest, sub *Subscription) { diff --git a/nomad/stream/event_publisher_test.go b/nomad/stream/event_publisher_test.go index 691574a65a9a..1cdd0f3472b0 100644 --- a/nomad/stream/event_publisher_test.go +++ b/nomad/stream/event_publisher_test.go @@ -2,6 +2,7 @@ package stream import ( "context" + "github.com/hashicorp/nomad/nomad/structs" "testing" "time" @@ -10,7 +11,7 @@ import ( func TestEventPublisher_PublishChangesAndSubscribe(t *testing.T) { subscription := &SubscribeRequest{ - Topics: map[Topic][]string{ + Topics: map[structs.Topic][]string{ "Test": []string{"sub-key"}, }, } @@ -25,35 +26,35 @@ func TestEventPublisher_PublishChangesAndSubscribe(t *testing.T) { // Now subscriber should block waiting for updates assertNoResult(t, eventCh) - events := []Event{{ + events := []structs.Event{{ Index: 1, Topic: "Test", Key: "sub-key", Payload: "sample payload", }} - publisher.Publish(1, events) + publisher.Publish(structs.Events{Index: 1, Events: events}) // Subscriber should see the published event result := nextResult(t, eventCh) require.NoError(t, result.Err) - expected := []Event{{Payload: "sample payload", Key: "sub-key", Topic: "Test", Index: 1}} + expected := []structs.Event{{Payload: "sample payload", Key: "sub-key", Topic: "Test", Index: 1}} require.Equal(t, expected, result.Events) // Now subscriber should block waiting for updates assertNoResult(t, eventCh) // Publish a second event - events = []Event{{ + events = []structs.Event{{ Index: 2, Topic: "Test", Key: "sub-key", Payload: "sample payload 2", }} - publisher.Publish(2, events) + publisher.Publish(structs.Events{Index: 2, Events: events}) result = nextResult(t, eventCh) require.NoError(t, result.Err) - expected = []Event{{Payload: "sample payload 2", Key: "sub-key", Topic: "Test", Index: 2}} + expected = []structs.Event{{Payload: "sample payload 2", Key: "sub-key", Topic: "Test", Index: 2}} require.Equal(t, expected, result.Events) } @@ -98,7 +99,7 @@ func consumeSubscription(ctx context.Context, sub *Subscription) <-chan subNextR } type subNextResult struct { - Events []Event + Events []structs.Event Err error } diff --git a/nomad/stream/ndjson.go b/nomad/stream/ndjson.go index cb5195e54ef0..cd3befc7ea19 100644 --- a/nomad/stream/ndjson.go +++ b/nomad/stream/ndjson.go @@ -7,43 +7,40 @@ import ( "fmt" "sync" "time" + + "github.com/hashicorp/nomad/nomad/structs" ) var ( // NDJsonHeartbeat is the NDJson to send as a heartbeat // Avoids creating many heartbeat instances - NDJsonHeartbeat = &NDJson{Data: []byte("{}\n")} + NDJsonHeartbeat = &structs.NDJson{Data: []byte("{}\n")} ) // NDJsonStream is used to send new line delimited JSON and heartbeats // to a destination (out channel) type NDJsonStream struct { - out chan<- *NDJson + out chan<- *structs.NDJson // heartbeat is the interval to send heartbeat messages to keep a connection // open. heartbeat *time.Ticker - publishCh chan NDJson + publishCh chan structs.NDJson exitCh chan struct{} l sync.Mutex running bool } -// NNDJson is a wrapper for a Newline Delimited JSON object -type NDJson struct { - Data []byte -} - // NewNNewNDJsonStream creates a new NDJson stream that will output NDJson structs // to the passed output channel -func NewNDJsonStream(out chan<- *NDJson, heartbeat time.Duration) *NDJsonStream { +func NewNDJsonStream(out chan<- *structs.NDJson, heartbeat time.Duration) *NDJsonStream { return &NDJsonStream{ out: out, heartbeat: time.NewTicker(heartbeat), exitCh: make(chan struct{}), - publishCh: make(chan NDJson), + publishCh: make(chan structs.NDJson), } } @@ -97,18 +94,10 @@ func (n *NDJsonStream) Send(obj interface{}) error { } select { - case n.publishCh <- NDJson{Data: buf.Bytes()}: + case n.publishCh <- structs.NDJson{Data: buf.Bytes()}: case <-n.exitCh: return fmt.Errorf("stream is no longer running") } return nil } - -func (j *NDJson) Copy() *NDJson { - n := new(NDJson) - *n = *j - n.Data = make([]byte, len(j.Data)) - copy(n.Data, j.Data) - return n -} diff --git a/nomad/stream/ndjson_test.go b/nomad/stream/ndjson_test.go index 8e807938fb7a..589cde1a7190 100644 --- a/nomad/stream/ndjson_test.go +++ b/nomad/stream/ndjson_test.go @@ -6,6 +6,7 @@ import ( "testing" "time" + "github.com/hashicorp/nomad/nomad/structs" "github.com/stretchr/testify/require" ) @@ -19,7 +20,7 @@ func TestNDJson(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - out := make(chan *NDJson) + out := make(chan *structs.NDJson) s := NewNDJsonStream(out, 1*time.Second) s.Run(ctx) @@ -45,7 +46,7 @@ func TestNDJson_Send_After_Stop(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - out := make(chan *NDJson) + out := make(chan *structs.NDJson) s := NewNDJsonStream(out, 1*time.Second) s.Run(ctx) @@ -62,7 +63,7 @@ func TestNDJson_HeartBeat(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - out := make(chan *NDJson) + out := make(chan *structs.NDJson) s := NewNDJsonStream(out, 10*time.Millisecond) s.Run(ctx) diff --git a/nomad/stream/subscription.go b/nomad/stream/subscription.go index bae3d091ac2e..555be583477e 100644 --- a/nomad/stream/subscription.go +++ b/nomad/stream/subscription.go @@ -4,9 +4,12 @@ import ( "context" "errors" "sync/atomic" + + "github.com/hashicorp/nomad/nomad/structs" ) const ( + AllKeys = "*" // subscriptionStateOpen is the default state of a subscription. An open // subscription may receive new events. subscriptionStateOpen uint32 = 0 @@ -46,7 +49,7 @@ type SubscribeRequest struct { Token string Index uint64 - Topics map[Topic][]string + Topics map[structs.Topic][]string } func newSubscription(req *SubscribeRequest, item *bufferItem, unsub func()) *Subscription { @@ -58,18 +61,18 @@ func newSubscription(req *SubscribeRequest, item *bufferItem, unsub func()) *Sub } } -func (s *Subscription) Next(ctx context.Context) (Events, error) { +func (s *Subscription) Next(ctx context.Context) (structs.Events, error) { if atomic.LoadUint32(&s.state) == subscriptionStateClosed { - return Events{}, ErrSubscriptionClosed + return structs.Events{}, ErrSubscriptionClosed } for { next, err := s.currentItem.Next(ctx, s.forceClosed) switch { case err != nil && atomic.LoadUint32(&s.state) == subscriptionStateClosed: - return Events{}, ErrSubscriptionClosed + return structs.Events{}, ErrSubscriptionClosed case err != nil: - return Events{}, err + return structs.Events{}, err } s.currentItem = next @@ -77,11 +80,11 @@ func (s *Subscription) Next(ctx context.Context) (Events, error) { if len(events) == 0 { continue } - return Events{Index: next.Index, Events: events}, nil + return structs.Events{Index: next.Index, Events: events}, nil } } -func (s *Subscription) NextNoBlock() ([]Event, error) { +func (s *Subscription) NextNoBlock() ([]structs.Event, error) { if atomic.LoadUint32(&s.state) == subscriptionStateClosed { return nil, ErrSubscriptionClosed } @@ -113,7 +116,7 @@ func (s *Subscription) Unsubscribe() { } // filter events to only those that match a subscriptions topic/keys -func filter(req *SubscribeRequest, events []Event) []Event { +func filter(req *SubscribeRequest, events []structs.Event) []structs.Event { if len(events) == 0 { return events } @@ -145,7 +148,7 @@ func filter(req *SubscribeRequest, events []Event) []Event { } // Return filtered events - result := make([]Event, 0, count) + result := make([]structs.Event, 0, count) for _, e := range events { _, allTopics := req.Topics[AllKeys] if _, ok := req.Topics[e.Topic]; ok || allTopics { diff --git a/nomad/stream/subscription_test.go b/nomad/stream/subscription_test.go index 8dd841bd2be2..659a7f1a2538 100644 --- a/nomad/stream/subscription_test.go +++ b/nomad/stream/subscription_test.go @@ -1,6 +1,7 @@ package stream import ( + "github.com/hashicorp/nomad/nomad/structs" "testing" "github.com/stretchr/testify/require" @@ -11,11 +12,11 @@ func TestSubscription(t *testing.T) { } func TestFilter_AllTopics(t *testing.T) { - events := make([]Event, 0, 5) - events = append(events, Event{Topic: "Test", Key: "One"}, Event{Topic: "Test", Key: "Two"}) + events := make([]structs.Event, 0, 5) + events = append(events, structs.Event{Topic: "Test", Key: "One"}, structs.Event{Topic: "Test", Key: "Two"}) req := &SubscribeRequest{ - Topics: map[Topic][]string{ + Topics: map[structs.Topic][]string{ "*": []string{"*"}, }, } @@ -27,11 +28,11 @@ func TestFilter_AllTopics(t *testing.T) { } func TestFilter_AllKeys(t *testing.T) { - events := make([]Event, 0, 5) - events = append(events, Event{Topic: "Test", Key: "One"}, Event{Topic: "Test", Key: "Two"}) + events := make([]structs.Event, 0, 5) + events = append(events, structs.Event{Topic: "Test", Key: "One"}, structs.Event{Topic: "Test", Key: "Two"}) req := &SubscribeRequest{ - Topics: map[Topic][]string{ + Topics: map[structs.Topic][]string{ "Test": []string{"*"}, }, } @@ -43,49 +44,49 @@ func TestFilter_AllKeys(t *testing.T) { } func TestFilter_PartialMatch_Topic(t *testing.T) { - events := make([]Event, 0, 5) - events = append(events, Event{Topic: "Test", Key: "One"}, Event{Topic: "Test", Key: "Two"}, Event{Topic: "Exclude", Key: "Two"}) + events := make([]structs.Event, 0, 5) + events = append(events, structs.Event{Topic: "Test", Key: "One"}, structs.Event{Topic: "Test", Key: "Two"}, structs.Event{Topic: "Exclude", Key: "Two"}) req := &SubscribeRequest{ - Topics: map[Topic][]string{ + Topics: map[structs.Topic][]string{ "Test": []string{"*"}, }, } actual := filter(req, events) - expected := []Event{{Topic: "Test", Key: "One"}, {Topic: "Test", Key: "Two"}} + expected := []structs.Event{{Topic: "Test", Key: "One"}, {Topic: "Test", Key: "Two"}} require.Equal(t, expected, actual) require.Equal(t, cap(actual), 2) } func TestFilter_PartialMatch_Key(t *testing.T) { - events := make([]Event, 0, 5) - events = append(events, Event{Topic: "Test", Key: "One"}, Event{Topic: "Test", Key: "Two"}) + events := make([]structs.Event, 0, 5) + events = append(events, structs.Event{Topic: "Test", Key: "One"}, structs.Event{Topic: "Test", Key: "Two"}) req := &SubscribeRequest{ - Topics: map[Topic][]string{ + Topics: map[structs.Topic][]string{ "Test": []string{"One"}, }, } actual := filter(req, events) - expected := []Event{{Topic: "Test", Key: "One"}} + expected := []structs.Event{{Topic: "Test", Key: "One"}} require.Equal(t, expected, actual) require.Equal(t, cap(actual), 1) } func TestFilter_NoMatch(t *testing.T) { - events := make([]Event, 0, 5) - events = append(events, Event{Topic: "Test", Key: "One"}, Event{Topic: "Test", Key: "Two"}) + events := make([]structs.Event, 0, 5) + events = append(events, structs.Event{Topic: "Test", Key: "One"}, structs.Event{Topic: "Test", Key: "Two"}) req := &SubscribeRequest{ - Topics: map[Topic][]string{ + Topics: map[structs.Topic][]string{ "NodeEvents": []string{"*"}, "Test": []string{"Highly-Specific-Key"}, }, } actual := filter(req, events) - var expected []Event + var expected []structs.Event require.Equal(t, expected, actual) require.Equal(t, cap(actual), 0) diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index c00aa1d490c6..4192a27b0c04 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -39,7 +39,6 @@ import ( "github.com/hashicorp/nomad/helper/constraints/semver" "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/lib/kheap" - "github.com/hashicorp/nomad/nomad/stream" psstructs "github.com/hashicorp/nomad/plugins/shared/structs" ) @@ -10712,7 +10711,7 @@ type ACLTokenUpsertResponse struct { // EEventStreamRequest is used to stream events from a servers // EventPublisher type EventStreamRequest struct { - Topics map[stream.Topic][]string + Topics map[Topic][]string Index int QueryOptions @@ -10720,7 +10719,7 @@ type EventStreamRequest struct { type EventStreamWrapper struct { Error *RpcError - Event *stream.NDJson + Event *NDJson } // RpcError is used for serializing errors with a potential error code @@ -10739,3 +10738,32 @@ func NewRpcError(err error, code *int64) *RpcError { func (r *RpcError) Error() string { return r.Message } + +type Topic string + +type Event struct { + Topic Topic + Type string + Key string + FilterKeys []string + Index uint64 + Payload interface{} +} + +type Events struct { + Index uint64 + Events []Event +} + +// NNDJson is a wrapper for a Newline Delimited JSON object +type NDJson struct { + Data []byte +} + +func (j *NDJson) Copy() *NDJson { + n := new(NDJson) + *n = *j + n.Data = make([]byte, len(j.Data)) + copy(n.Data, j.Data) + return n +} diff --git a/scheduler/generic_sched_test.go b/scheduler/generic_sched_test.go index 1294604d277f..b0aa7be3bab3 100644 --- a/scheduler/generic_sched_test.go +++ b/scheduler/generic_sched_test.go @@ -1,7 +1,6 @@ package scheduler import ( - "context" "fmt" "reflect" "sort" @@ -3108,7 +3107,7 @@ func TestServiceSched_NodeDrain_Down(t *testing.T) { newAlloc.ClientStatus = structs.AllocClientStatusRunning running = append(running, newAlloc) } - require.NoError(t, h.State.UpdateAllocsFromClient(context.Background(), h.NextIndex(), running)) + require.NoError(t, h.State.UpdateAllocsFromClient(structs.MsgTypeTestSetup, h.NextIndex(), running)) // Mark some of the allocations as complete var complete []*structs.Allocation @@ -3127,7 +3126,7 @@ func TestServiceSched_NodeDrain_Down(t *testing.T) { newAlloc.ClientStatus = structs.AllocClientStatusComplete complete = append(complete, newAlloc) } - require.NoError(t, h.State.UpdateAllocsFromClient(context.Background(), h.NextIndex(), complete)) + require.NoError(t, h.State.UpdateAllocsFromClient(structs.MsgTypeTestSetup, h.NextIndex(), complete)) // Create a mock evaluation to deal with the node update eval := &structs.Evaluation{ diff --git a/vendor/github.com/hashicorp/go-immutable-radix/CHANGELOG.md b/vendor/github.com/hashicorp/go-immutable-radix/CHANGELOG.md index a967ae456da8..6331af921bab 100644 --- a/vendor/github.com/hashicorp/go-immutable-radix/CHANGELOG.md +++ b/vendor/github.com/hashicorp/go-immutable-radix/CHANGELOG.md @@ -1,4 +1,16 @@ -# UNRELEASED +# 1.3.0 (September 17th, 2020) + +FEATURES + +* Add reverse tree traversal [[GH-30](https://github.com/hashicorp/go-immutable-radix/pull/30)] + +# 1.2.0 (March 18th, 2020) + +FEATURES + +* Adds a `Clone` method to `Txn` allowing transactions to be split either into two independently mutable trees. [[GH-26](https://github.com/hashicorp/go-immutable-radix/pull/26)] + +# 1.1.0 (May 22nd, 2019) FEATURES diff --git a/vendor/github.com/hashicorp/go-immutable-radix/iter.go b/vendor/github.com/hashicorp/go-immutable-radix/iter.go index 1ecaf831c785..cd16d3beae42 100644 --- a/vendor/github.com/hashicorp/go-immutable-radix/iter.go +++ b/vendor/github.com/hashicorp/go-immutable-radix/iter.go @@ -155,7 +155,7 @@ func (i *Iterator) Next() ([]byte, interface{}, bool) { // Initialize our stack if needed if i.stack == nil && i.node != nil { i.stack = []edges{ - edges{ + { edge{node: i.node}, }, } diff --git a/vendor/github.com/hashicorp/go-immutable-radix/node.go b/vendor/github.com/hashicorp/go-immutable-radix/node.go index 3ab904edcebd..35985480872c 100644 --- a/vendor/github.com/hashicorp/go-immutable-radix/node.go +++ b/vendor/github.com/hashicorp/go-immutable-radix/node.go @@ -211,6 +211,12 @@ func (n *Node) Iterator() *Iterator { return &Iterator{node: n} } +// ReverseIterator is used to return an iterator at +// the given node to walk the tree backwards +func (n *Node) ReverseIterator() *ReverseIterator { + return NewReverseIterator(n) +} + // rawIterator is used to return a raw iterator at the given node to walk the // tree. func (n *Node) rawIterator() *rawIterator { @@ -224,6 +230,11 @@ func (n *Node) Walk(fn WalkFn) { recursiveWalk(n, fn) } +// WalkBackwards is used to walk the tree in reverse order +func (n *Node) WalkBackwards(fn WalkFn) { + reverseRecursiveWalk(n, fn) +} + // WalkPrefix is used to walk the tree under a prefix func (n *Node) WalkPrefix(prefix []byte, fn WalkFn) { search := prefix @@ -302,3 +313,22 @@ func recursiveWalk(n *Node, fn WalkFn) bool { } return false } + +// reverseRecursiveWalk is used to do a reverse pre-order +// walk of a node recursively. Returns true if the walk +// should be aborted +func reverseRecursiveWalk(n *Node, fn WalkFn) bool { + // Visit the leaf values if any + if n.leaf != nil && fn(n.leaf.key, n.leaf.val) { + return true + } + + // Recurse on the children in reverse order + for i := len(n.edges) - 1; i >= 0; i-- { + e := n.edges[i] + if reverseRecursiveWalk(e.node, fn) { + return true + } + } + return false +} diff --git a/vendor/github.com/hashicorp/go-immutable-radix/raw_iter.go b/vendor/github.com/hashicorp/go-immutable-radix/raw_iter.go index 04814c1323fb..3c6a22525c8e 100644 --- a/vendor/github.com/hashicorp/go-immutable-radix/raw_iter.go +++ b/vendor/github.com/hashicorp/go-immutable-radix/raw_iter.go @@ -41,7 +41,7 @@ func (i *rawIterator) Next() { // Initialize our stack if needed. if i.stack == nil && i.node != nil { i.stack = []rawStackEntry{ - rawStackEntry{ + { edges: edges{ edge{node: i.node}, }, diff --git a/vendor/github.com/hashicorp/go-immutable-radix/reverse_iter.go b/vendor/github.com/hashicorp/go-immutable-radix/reverse_iter.go new file mode 100644 index 000000000000..762471bc36fb --- /dev/null +++ b/vendor/github.com/hashicorp/go-immutable-radix/reverse_iter.go @@ -0,0 +1,177 @@ +package iradix + +import ( + "bytes" +) + +// ReverseIterator is used to iterate over a set of nodes +// in reverse in-order +type ReverseIterator struct { + i *Iterator +} + +// NewReverseIterator returns a new ReverseIterator at a node +func NewReverseIterator(n *Node) *ReverseIterator { + return &ReverseIterator{ + i: &Iterator{node: n}, + } +} + +// SeekPrefixWatch is used to seek the iterator to a given prefix +// and returns the watch channel of the finest granularity +func (ri *ReverseIterator) SeekPrefixWatch(prefix []byte) (watch <-chan struct{}) { + return ri.i.SeekPrefixWatch(prefix) +} + +// SeekPrefix is used to seek the iterator to a given prefix +func (ri *ReverseIterator) SeekPrefix(prefix []byte) { + ri.i.SeekPrefixWatch(prefix) +} + +func (ri *ReverseIterator) recurseMax(n *Node) *Node { + // Traverse to the maximum child + if n.leaf != nil { + return n + } + if len(n.edges) > 0 { + // Add all the other edges to the stack (the max node will be added as + // we recurse) + m := len(n.edges) + ri.i.stack = append(ri.i.stack, n.edges[:m-1]) + return ri.recurseMax(n.edges[m-1].node) + } + // Shouldn't be possible + return nil +} + +// SeekReverseLowerBound is used to seek the iterator to the largest key that is +// lower or equal to the given key. There is no watch variant as it's hard to +// predict based on the radix structure which node(s) changes might affect the +// result. +func (ri *ReverseIterator) SeekReverseLowerBound(key []byte) { + // Wipe the stack. Unlike Prefix iteration, we need to build the stack as we + // go because we need only a subset of edges of many nodes in the path to the + // leaf with the lower bound. + ri.i.stack = []edges{} + n := ri.i.node + search := key + + found := func(n *Node) { + ri.i.node = n + ri.i.stack = append(ri.i.stack, edges{edge{node: n}}) + } + + for { + // Compare current prefix with the search key's same-length prefix. + var prefixCmp int + if len(n.prefix) < len(search) { + prefixCmp = bytes.Compare(n.prefix, search[0:len(n.prefix)]) + } else { + prefixCmp = bytes.Compare(n.prefix, search) + } + + if prefixCmp < 0 { + // Prefix is smaller than search prefix, that means there is no lower bound. + // But we are looking in reverse, so the reverse lower bound will be the + // largest leaf under this subtree, since it is the value that would come + // right before the current search prefix if it were in the tree. So we need + // to follow the maximum path in this subtree to find it. + n = ri.recurseMax(n) + if n != nil { + found(n) + } + return + } + + if prefixCmp > 0 { + // Prefix is larger than search prefix, that means there is no reverse lower + // bound since nothing comes before our current search prefix. + ri.i.node = nil + return + } + + // Prefix is equal, we are still heading for an exact match. If this is a + // leaf we're done. + if n.leaf != nil { + if bytes.Compare(n.leaf.key, key) < 0 { + ri.i.node = nil + return + } + found(n) + return + } + + // Consume the search prefix + if len(n.prefix) > len(search) { + search = []byte{} + } else { + search = search[len(n.prefix):] + } + + // Otherwise, take the lower bound next edge. + idx, lbNode := n.getLowerBoundEdge(search[0]) + + // From here, we need to update the stack with all values lower than + // the lower bound edge. Since getLowerBoundEdge() returns -1 when the + // search prefix is larger than all edges, we need to place idx at the + // last edge index so they can all be place in the stack, since they + // come before our search prefix. + if idx == -1 { + idx = len(n.edges) + } + + // Create stack edges for the all strictly lower edges in this node. + if len(n.edges[:idx]) > 0 { + ri.i.stack = append(ri.i.stack, n.edges[:idx]) + } + + // Exit if there's not lower bound edge. The stack will have the + // previous nodes already. + if lbNode == nil { + ri.i.node = nil + return + } + + ri.i.node = lbNode + // Recurse + n = lbNode + } +} + +// Previous returns the previous node in reverse order +func (ri *ReverseIterator) Previous() ([]byte, interface{}, bool) { + // Initialize our stack if needed + if ri.i.stack == nil && ri.i.node != nil { + ri.i.stack = []edges{ + { + edge{node: ri.i.node}, + }, + } + } + + for len(ri.i.stack) > 0 { + // Inspect the last element of the stack + n := len(ri.i.stack) + last := ri.i.stack[n-1] + m := len(last) + elem := last[m-1].node + + // Update the stack + if m > 1 { + ri.i.stack[n-1] = last[:m-1] + } else { + ri.i.stack = ri.i.stack[:n-1] + } + + // Push the edges onto the frontier + if len(elem.edges) > 0 { + ri.i.stack = append(ri.i.stack, elem.edges) + } + + // Return the leaf values if any + if elem.leaf != nil { + return elem.leaf.key, elem.leaf.val, true + } + } + return nil, nil, false +} diff --git a/vendor/github.com/hashicorp/go-memdb/go.mod b/vendor/github.com/hashicorp/go-memdb/go.mod index 4b37934cb2b2..242f5fac2dda 100644 --- a/vendor/github.com/hashicorp/go-memdb/go.mod +++ b/vendor/github.com/hashicorp/go-memdb/go.mod @@ -3,6 +3,6 @@ module github.com/hashicorp/go-memdb go 1.12 require ( - github.com/hashicorp/go-immutable-radix v1.2.0 + github.com/hashicorp/go-immutable-radix v1.3.0 github.com/hashicorp/golang-lru v0.5.4 // indirect ) diff --git a/vendor/github.com/hashicorp/go-memdb/go.sum b/vendor/github.com/hashicorp/go-memdb/go.sum index 8d330e75ab5b..eaff521cec66 100644 --- a/vendor/github.com/hashicorp/go-memdb/go.sum +++ b/vendor/github.com/hashicorp/go-memdb/go.sum @@ -1,5 +1,5 @@ -github.com/hashicorp/go-immutable-radix v1.2.0 h1:l6UW37iCXwZkZoAbEYnptSHVE/cQ5bOTPYG5W3vf9+8= -github.com/hashicorp/go-immutable-radix v1.2.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= +github.com/hashicorp/go-immutable-radix v1.3.0 h1:8exGP7ego3OmkfksihtSouGMZ+hQrhxx+FVELeXpVPE= +github.com/hashicorp/go-immutable-radix v1.3.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= github.com/hashicorp/go-uuid v1.0.0 h1:RS8zrF7PhGwyNPOtxSClXXj9HA8feRnJzgnI1RJCSnM= github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/golang-lru v0.5.0 h1:CL2msUPvZTLb5O648aiLNJw3hnBxN2+1Jq8rCOH9wdo= diff --git a/vendor/github.com/hashicorp/go-memdb/txn.go b/vendor/github.com/hashicorp/go-memdb/txn.go index 6466845933ad..68734e37c86a 100644 --- a/vendor/github.com/hashicorp/go-memdb/txn.go +++ b/vendor/github.com/hashicorp/go-memdb/txn.go @@ -536,6 +536,34 @@ func (txn *Txn) FirstWatch(table, index string, args ...interface{}) (<-chan str return watch, value, nil } +// LastWatch is used to return the last matching object for +// the given constraints on the index along with the watch channel +func (txn *Txn) LastWatch(table, index string, args ...interface{}) (<-chan struct{}, interface{}, error) { + // Get the index value + indexSchema, val, err := txn.getIndexValue(table, index, args...) + if err != nil { + return nil, nil, err + } + + // Get the index itself + indexTxn := txn.readableIndex(table, indexSchema.Name) + + // Do an exact lookup + if indexSchema.Unique && val != nil && indexSchema.Name == index { + watch, obj, ok := indexTxn.GetWatch(val) + if !ok { + return watch, nil, nil + } + return watch, obj, nil + } + + // Handle non-unique index by using an iterator and getting the last value + iter := indexTxn.Root().ReverseIterator() + watch := iter.SeekPrefixWatch(val) + _, value, _ := iter.Previous() + return watch, value, nil +} + // First is used to return the first matching object for // the given constraints on the index func (txn *Txn) First(table, index string, args ...interface{}) (interface{}, error) { @@ -543,6 +571,13 @@ func (txn *Txn) First(table, index string, args ...interface{}) (interface{}, er return val, err } +// Last is used to return the last matching object for +// the given constraints on the index +func (txn *Txn) Last(table, index string, args ...interface{}) (interface{}, error) { + _, val, err := txn.LastWatch(table, index, args...) + return val, err +} + // LongestPrefix is used to fetch the longest prefix match for the given // constraints on the index. Note that this will not work with the memdb // StringFieldIndex because it adds null terminators which prevent the @@ -654,6 +689,26 @@ func (txn *Txn) Get(table, index string, args ...interface{}) (ResultIterator, e return iter, nil } +// GetReverse is used to construct a Reverse ResultIterator over all the +// rows that match the given constraints of an index. +// The returned ResultIterator's Next() will return the next Previous value +func (txn *Txn) GetReverse(table, index string, args ...interface{}) (ResultIterator, error) { + indexIter, val, err := txn.getIndexIteratorReverse(table, index, args...) + if err != nil { + return nil, err + } + + // Seek the iterator to the appropriate sub-set + watchCh := indexIter.SeekPrefixWatch(val) + + // Create an iterator + iter := &radixReverseIterator{ + iter: indexIter, + watchCh: watchCh, + } + return iter, nil +} + // LowerBound is used to construct a ResultIterator over all the the range of // rows that have an index value greater than or equal to the provide args. // Calling this then iterating until the rows are larger than required allows @@ -676,6 +731,29 @@ func (txn *Txn) LowerBound(table, index string, args ...interface{}) (ResultIter return iter, nil } +// ReverseLowerBound is used to construct a Reverse ResultIterator over all the +// the range of rows that have an index value less than or equal to the +// provide args. Calling this then iterating until the rows are lower than +// required allows range scans within an index. It is not possible to watch the +// resulting iterator since the radix tree doesn't efficiently allow watching +// on lower bound changes. The WatchCh returned will be nill and so will block +// forever. +func (txn *Txn) ReverseLowerBound(table, index string, args ...interface{}) (ResultIterator, error) { + indexIter, val, err := txn.getIndexIteratorReverse(table, index, args...) + if err != nil { + return nil, err + } + + // Seek the iterator to the appropriate sub-set + indexIter.SeekReverseLowerBound(val) + + // Create an iterator + iter := &radixReverseIterator{ + iter: indexIter, + } + return iter, nil +} + // objectID is a tuple of table name and the raw internal id byte slice // converted to a string. It's only converted to a string to make it comparable // so this struct can be used as a map index. @@ -777,6 +855,22 @@ func (txn *Txn) getIndexIterator(table, index string, args ...interface{}) (*ira return indexIter, val, nil } +func (txn *Txn) getIndexIteratorReverse(table, index string, args ...interface{}) (*iradix.ReverseIterator, []byte, error) { + // Get the index value to scan + indexSchema, val, err := txn.getIndexValue(table, index, args...) + if err != nil { + return nil, nil, err + } + + // Get the index itself + indexTxn := txn.readableIndex(table, indexSchema.Name) + indexRoot := indexTxn.Root() + + // Get an interator over the index + indexIter := indexRoot.ReverseIterator() + return indexIter, val, nil +} + // Defer is used to push a new arbitrary function onto a stack which // gets called when a transaction is committed and finished. Deferred // functions are called in LIFO order, and only invoked at the end of @@ -805,6 +899,23 @@ func (r *radixIterator) Next() interface{} { return value } +type radixReverseIterator struct { + iter *iradix.ReverseIterator + watchCh <-chan struct{} +} + +func (r *radixReverseIterator) Next() interface{} { + _, value, ok := r.iter.Previous() + if !ok { + return nil + } + return value +} + +func (r *radixReverseIterator) WatchCh() <-chan struct{} { + return r.watchCh +} + // Snapshot creates a snapshot of the current state of the transaction. // Returns a new read-only transaction or nil if the transaction is already // aborted or committed. diff --git a/vendor/github.com/hashicorp/nomad/api/event.go b/vendor/github.com/hashicorp/nomad/api/event.go new file mode 100644 index 000000000000..b52f4eb8e8b9 --- /dev/null +++ b/vendor/github.com/hashicorp/nomad/api/event.go @@ -0,0 +1,104 @@ +package api + +import ( + "context" + "encoding/json" + "fmt" +) + +// Events is a set of events for a corresponding index. Events returned for the +// index depend on which topics are subscribed to when a request is made. +type Events struct { + Index uint64 + Events []Event +} + +// Topic is an event Topic +type Topic string + +// Event holds information related to an event that occurred in Nomad. +// The Payload is a hydrated object related to the Topic +type Event struct { + Topic Topic + Type string + Key string + FilterKeys []string + Index uint64 + Payload interface{} +} + +// IsHeartBeat specifies if the event is an empty heartbeat used to +// keep a connection alive. +func (e *Events) IsHeartBeat() bool { + return e.Index == 0 && len(e.Events) == 0 +} + +// EventStream is used to stream events from Nomad +type EventStream struct { + client *Client +} + +// EventStream returns a handle to the Events endpoint +func (c *Client) EventStream() *EventStream { + return &EventStream{client: c} +} + +// Stream establishes a new subscription to Nomad's event stream and streams +// results back to the returned channel. +func (e *EventStream) Stream(ctx context.Context, topics map[Topic][]string, index uint64, q *QueryOptions) (<-chan *Events, <-chan error) { + + errCh := make(chan error, 1) + + r, err := e.client.newRequest("GET", "/v1/event/stream") + if err != nil { + errCh <- err + return nil, errCh + } + r.setQueryOptions(q) + + // Build topic query params + for topic, keys := range topics { + for _, k := range keys { + r.params.Add("topic", fmt.Sprintf("%s:%s", topic, k)) + } + } + + _, resp, err := requireOK(e.client.doRequest(r)) + + if err != nil { + errCh <- err + return nil, errCh + } + + eventsCh := make(chan *Events, 10) + go func() { + defer resp.Body.Close() + + dec := json.NewDecoder(resp.Body) + + for { + select { + case <-ctx.Done(): + close(eventsCh) + return + default: + } + + // Decode next newline delimited json of events + var events Events + if err := dec.Decode(&events); err != nil { + close(eventsCh) + errCh <- err + return + } + if events.IsHeartBeat() { + continue + } + + eventsCh <- &events + + } + }() + + return eventsCh, errCh +} diff --git a/vendor/modules.txt b/vendor/modules.txt index 218ee4eb754a..5af8e500481e 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -408,10 +408,10 @@ github.com/hashicorp/go-getter/helper/url # github.com/hashicorp/go-hclog v0.12.0 ## explicit github.com/hashicorp/go-hclog -# github.com/hashicorp/go-immutable-radix v1.2.0 +# github.com/hashicorp/go-immutable-radix v1.3.0 ## explicit github.com/hashicorp/go-immutable-radix -# github.com/hashicorp/go-memdb v1.2.1 +# github.com/hashicorp/go-memdb v1.2.1 => /home/drew/work/go/go-memdb ## explicit github.com/hashicorp/go-memdb # github.com/hashicorp/go-msgpack v1.1.5 @@ -1019,6 +1019,7 @@ honnef.co/go/tools/version # github.com/godbus/dbus => github.com/godbus/dbus v5.0.1+incompatible # github.com/golang/protobuf => github.com/golang/protobuf v1.3.4 # github.com/hashicorp/go-discover => github.com/hashicorp/go-discover v0.0.0-20200812215701-c4b85f6ed31f +# github.com/hashicorp/go-memdb => /home/drew/work/go/go-memdb # github.com/hashicorp/nomad/api => ./api # github.com/kr/pty => github.com/kr/pty v1.1.5 # github.com/shirou/gopsutil => github.com/hashicorp/gopsutil v2.18.13-0.20200531184148-5aca383d4f9d+incompatible From 8a3130a356958a58eb5ef4b6da5ec0ad0fb1171e Mon Sep 17 00:00:00 2001 From: Drew Bailey <2614075+drewbailey@users.noreply.github.com> Date: Mon, 5 Oct 2020 19:40:06 -0400 Subject: [PATCH 12/19] event durability count and cfg --- command/agent/config.go | 9 +++++++++ command/agent/config_parse_test.go | 1 + command/agent/testdata/basic.hcl | 1 + command/agent/testdata/basic.json | 1 + nomad/fsm.go | 27 +++++++++++++++------------ nomad/fsm_test.go | 7 +++---- nomad/state/state_changes.go | 6 ++---- nomad/state/state_store.go | 9 +++++---- nomad/state/testing.go | 8 ++++---- 9 files changed, 41 insertions(+), 28 deletions(-) diff --git a/command/agent/config.go b/command/agent/config.go index 392eaf7b1dc5..3ba23eceeae9 100644 --- a/command/agent/config.go +++ b/command/agent/config.go @@ -488,6 +488,10 @@ type ServerConfig struct { // will generate events for its event stream. EnableEventPublisher bool `hcl:"enable_event_publisher"` + // DurableEventCount specifies the amount of events to persist during snapshot generation. + // A count of 0 signals that no events should be persisted. + DurableEventCount int `hcl:"durable_event_count"` + // ExtraKeysHCL is used by hcl to surface unexpected keys ExtraKeysHCL []string `hcl:",unusedKeys" json:"-"` } @@ -880,6 +884,7 @@ func DefaultConfig() *Config { Server: &ServerConfig{ Enabled: false, EnableEventPublisher: true, + DurableEventCount: 100, StartJoin: []string{}, ServerJoin: &ServerJoin{ RetryJoin: []string{}, @@ -1408,6 +1413,10 @@ func (a *ServerConfig) Merge(b *ServerConfig) *ServerConfig { result.EnableEventPublisher = true } + if b.DurableEventCount != 0 { + result.DurableEventCount = b.DurableEventCount + } + if b.DefaultSchedulerConfig != nil { c := *b.DefaultSchedulerConfig result.DefaultSchedulerConfig = &c diff --git a/command/agent/config_parse_test.go b/command/agent/config_parse_test.go index a288b5f80c4b..9cce0f52eb46 100644 --- a/command/agent/config_parse_test.go +++ b/command/agent/config_parse_test.go @@ -123,6 +123,7 @@ var basicConfig = &Config{ UpgradeVersion: "0.8.0", EncryptKey: "abc", EnableEventPublisher: true, + DurableEventCount: 100, ServerJoin: &ServerJoin{ RetryJoin: []string{"1.1.1.1", "2.2.2.2"}, RetryInterval: time.Duration(15) * time.Second, diff --git a/command/agent/testdata/basic.hcl b/command/agent/testdata/basic.hcl index 36a420551710..da08118dbe18 100644 --- a/command/agent/testdata/basic.hcl +++ b/command/agent/testdata/basic.hcl @@ -131,6 +131,7 @@ server { encrypt = "abc" raft_multiplier = 4 enable_event_publisher = true + durable_event_count = 100 server_join { retry_join = ["1.1.1.1", "2.2.2.2"] diff --git a/command/agent/testdata/basic.json b/command/agent/testdata/basic.json index 5aa897d73411..2d671dd1a99e 100644 --- a/command/agent/testdata/basic.json +++ b/command/agent/testdata/basic.json @@ -262,6 +262,7 @@ "deployment_gc_threshold": "12h", "enabled": true, "enable_event_publisher": true, + "durable_event_count": 100, "enabled_schedulers": [ "test" ], diff --git a/nomad/fsm.go b/nomad/fsm.go index ce5caaafd5af..db1aee98cf7e 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -128,17 +128,21 @@ type FSMConfig struct { Region string EnableEventPublisher bool + + // Durable count specifies the amount of events generated by the state store + // to save to disk during snapshot generation. The most recent events + // limited to count will be saved. + DurableEventCount int } // NewFSMPath is used to construct a new FSM with a blank state func NewFSM(config *FSMConfig) (*nomadFSM, error) { // Create a state store sconfig := &state.StateStoreConfig{ - Logger: config.Logger, - Region: config.Region, - EnablePublisher: config.EnableEventPublisher, - // TODO(drew) plumb cfg - EnableDurability: true, + Logger: config.Logger, + Region: config.Region, + EnablePublisher: config.EnableEventPublisher, + DurableEventCount: config.DurableEventCount, } state, err := state.NewStateStore(sconfig) if err != nil { @@ -1269,11 +1273,10 @@ func (n *nomadFSM) Restore(old io.ReadCloser) error { // Create a new state store config := &state.StateStoreConfig{ - Logger: n.config.Logger, - Region: n.config.Region, - EnablePublisher: n.config.EnableEventPublisher, - // TODO(drew) plumb cfg - EnableDurability: true, + Logger: n.config.Logger, + Region: n.config.Region, + EnablePublisher: n.config.EnableEventPublisher, + DurableEventCount: n.config.DurableEventCount, } newState, err := state.NewStateStore(config) if err != nil { @@ -2363,10 +2366,10 @@ func (s *nomadSnapshot) persistCSIVolumes(sink raft.SnapshotSink, func (s *nomadSnapshot) persistEvents(sink raft.SnapshotSink, encoder *codec.Encoder) error { var durableCount int - if s.snap.Config() != nil && !s.snap.Config().EnableDurability { + if s.snap.Config() != nil && s.snap.Config().DurableEventCount == 0 { return nil } else { - durableCount = s.snap.Config().DurableCount + durableCount = s.snap.Config().DurableEventCount } events, err := s.snap.LatestEventsReverse(nil) diff --git a/nomad/fsm_test.go b/nomad/fsm_test.go index 81e583568aa2..98b82feac5b3 100644 --- a/nomad/fsm_test.go +++ b/nomad/fsm_test.go @@ -3206,9 +3206,8 @@ func TestFSM_SnapshotRestore_Events_WithDurability(t *testing.T) { fsm := testFSM(t) state := fsm.State() cfg := state.Config() - cfg.EnableDurability = true - // DurableCount = 4 each mock events wrapper contains 2 events - cfg.DurableCount = 4 + // DurableEventCount = 4 each mock events wrapper contains 2 events + cfg.DurableEventCount = 4 e1 := mock.Events(1000) e2 := mock.Events(1001) @@ -3251,7 +3250,7 @@ func TestFSM_SnapshotRestore_Events_NoDurability(t *testing.T) { fsm := testFSM(t) state := fsm.State() cfg := state.Config() - cfg.EnableDurability = false + cfg.DurableEventCount = 0 e1 := mock.Events(1000) e2 := mock.Events(1001) diff --git a/nomad/state/state_changes.go b/nomad/state/state_changes.go index a8c48b1ff1fb..6e4903f024db 100644 --- a/nomad/state/state_changes.go +++ b/nomad/state/state_changes.go @@ -42,8 +42,7 @@ type changeTrackerDB struct { // ChangeConfig type ChangeConfig struct { - DurableEvents bool - DurableCount int + DurableEventCount int } func NewChangeTrackerDB(db *memdb.MemDB, publisher *stream.EventPublisher, changesFn changeProcessor, cfg *ChangeConfig) *changeTrackerDB { @@ -55,8 +54,7 @@ func NewChangeTrackerDB(db *memdb.MemDB, publisher *stream.EventPublisher, chang db: db, publisher: publisher, processChanges: changesFn, - durableEvents: cfg.DurableEvents, - durableCount: cfg.DurableCount, + durableCount: cfg.DurableEventCount, } } diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 18efdfd36ba9..4484c0f1bbe3 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -46,10 +46,12 @@ type StateStoreConfig struct { // Region is the region of the server embedding the state store. Region string + // EnablePublisher is used to enable or disable the event publisher EnablePublisher bool - EnableDurability bool - DurableCount int + // DurableEventCount is the amount of events to persist during the snapshot + // process. + DurableEventCount int } // The StateStore is responsible for maintaining all the Nomad @@ -94,8 +96,7 @@ func NewStateStore(config *StateStoreConfig) (*StateStore, error) { if config.EnablePublisher { cfg := &ChangeConfig{ - DurableEvents: config.EnableDurability, - DurableCount: 1000, + DurableEventCount: 1000, } publisher := stream.NewEventPublisher(ctx, stream.EventPublisherCfg{ EventBufferTTL: 1 * time.Hour, diff --git a/nomad/state/testing.go b/nomad/state/testing.go index 86acaf9e7820..19421cf42312 100644 --- a/nomad/state/testing.go +++ b/nomad/state/testing.go @@ -26,10 +26,10 @@ func TestStateStore(t testing.T) *StateStore { func TestStateStorePublisher(t testing.T) *StateStoreConfig { return &StateStoreConfig{ - Logger: testlog.HCLogger(t), - Region: "global", - EnablePublisher: true, - EnableDurability: true, + Logger: testlog.HCLogger(t), + Region: "global", + EnablePublisher: true, + DurableEventCount: 100, } } func TestStateStoreCfg(t testing.T, cfg *StateStoreConfig) *StateStore { From 66bda4455aa83579cd2b66c44763ec170df44d2e Mon Sep 17 00:00:00 2001 From: Drew Bailey <2614075+drewbailey@users.noreply.github.com> Date: Tue, 6 Oct 2020 09:47:16 -0400 Subject: [PATCH 13/19] go-memdb v1.3.0 --- go.mod | 3 +-- go.sum | 3 +++ vendor/modules.txt | 3 +-- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/go.mod b/go.mod index d399bf5f9c79..63877eb50450 100644 --- a/go.mod +++ b/go.mod @@ -9,7 +9,6 @@ replace ( github.com/godbus/dbus => github.com/godbus/dbus v5.0.1+incompatible github.com/golang/protobuf => github.com/golang/protobuf v1.3.4 github.com/hashicorp/go-discover => github.com/hashicorp/go-discover v0.0.0-20200812215701-c4b85f6ed31f - github.com/hashicorp/go-memdb => /home/drew/work/go/go-memdb github.com/hashicorp/nomad/api => ./api github.com/kr/pty => github.com/kr/pty v1.1.5 github.com/shirou/gopsutil => github.com/hashicorp/gopsutil v2.18.13-0.20200531184148-5aca383d4f9d+incompatible @@ -64,7 +63,7 @@ require ( github.com/hashicorp/go-getter v1.3.1-0.20190822194507-f5101da01173 github.com/hashicorp/go-hclog v0.12.0 github.com/hashicorp/go-immutable-radix v1.3.0 - github.com/hashicorp/go-memdb v1.2.1 + github.com/hashicorp/go-memdb v1.3.0 github.com/hashicorp/go-msgpack v1.1.5 github.com/hashicorp/go-multierror v1.1.0 github.com/hashicorp/go-plugin v1.0.2-0.20191004171845-809113480b55 diff --git a/go.sum b/go.sum index ed55baff18f1..c034bd861295 100644 --- a/go.sum +++ b/go.sum @@ -394,6 +394,9 @@ github.com/hashicorp/go-immutable-radix v1.2.0 h1:l6UW37iCXwZkZoAbEYnptSHVE/cQ5b github.com/hashicorp/go-immutable-radix v1.2.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= github.com/hashicorp/go-immutable-radix v1.3.0 h1:8exGP7ego3OmkfksihtSouGMZ+hQrhxx+FVELeXpVPE= github.com/hashicorp/go-immutable-radix v1.3.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= +github.com/hashicorp/go-memdb v1.0.3/go.mod h1:LWQ8R70vPrS4OEY9k28D2z8/Zzyu34NVzeRibGAzHO0= +github.com/hashicorp/go-memdb v1.3.0 h1:xdXq34gBOMEloa9rlGStLxmfX/dyIK8htOv36dQUwHU= +github.com/hashicorp/go-memdb v1.3.0/go.mod h1:Mluclgwib3R93Hk5fxEfiRhB+6Dar64wWh71LpNSe3g= github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= github.com/hashicorp/go-msgpack v0.5.5/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= github.com/hashicorp/go-msgpack v1.1.5 h1:9byZdVjKTe5mce63pRVNP1L7UAmdHOTEMGehn6KvJWs= diff --git a/vendor/modules.txt b/vendor/modules.txt index 5af8e500481e..e43b07720b45 100644 --- a/vendor/modules.txt +++ b/vendor/modules.txt @@ -411,7 +411,7 @@ github.com/hashicorp/go-hclog # github.com/hashicorp/go-immutable-radix v1.3.0 ## explicit github.com/hashicorp/go-immutable-radix -# github.com/hashicorp/go-memdb v1.2.1 => /home/drew/work/go/go-memdb +# github.com/hashicorp/go-memdb v1.3.0 ## explicit github.com/hashicorp/go-memdb # github.com/hashicorp/go-msgpack v1.1.5 @@ -1019,7 +1019,6 @@ honnef.co/go/tools/version # github.com/godbus/dbus => github.com/godbus/dbus v5.0.1+incompatible # github.com/golang/protobuf => github.com/golang/protobuf v1.3.4 # github.com/hashicorp/go-discover => github.com/hashicorp/go-discover v0.0.0-20200812215701-c4b85f6ed31f -# github.com/hashicorp/go-memdb => /home/drew/work/go/go-memdb # github.com/hashicorp/nomad/api => ./api # github.com/kr/pty => github.com/kr/pty v1.1.5 # github.com/shirou/gopsutil => github.com/hashicorp/gopsutil v2.18.13-0.20200531184148-5aca383d4f9d+incompatible From 1288b18b27519a99f3f24e80b76216846f711682 Mon Sep 17 00:00:00 2001 From: Drew Bailey <2614075+drewbailey@users.noreply.github.com> Date: Tue, 6 Oct 2020 11:08:12 -0400 Subject: [PATCH 14/19] rehydrate event publisher on snapshot restore address pr feedback --- api/event.go | 44 ++++++++----------- api/event_test.go | 35 +++++++++++++-- command/agent/command.go | 4 ++ command/agent/event_endpoint.go | 5 ++- command/agent/event_endpoint_test.go | 10 +++-- nomad/event_endpoint.go | 6 +-- nomad/fsm.go | 43 +++++++++++------- nomad/fsm_test.go | 28 +++++++++--- nomad/state/node_events_test.go | 2 +- nomad/stream/event_publisher.go | 4 ++ .../github.com/hashicorp/nomad/api/event.go | 44 ++++++++----------- 11 files changed, 140 insertions(+), 85 deletions(-) diff --git a/api/event.go b/api/event.go index b52f4eb8e8b9..dce3c265fe8c 100644 --- a/api/event.go +++ b/api/event.go @@ -6,11 +6,12 @@ import ( "fmt" ) -// Events is a set of events for a corresponding index. Events returned for the +// Ebvents is a set of events for a corresponding index. Events returned for the // index depend on which topics are subscribed to when a request is made. type Events struct { Index uint64 Events []Event + Err error } // Topic is an event Topic @@ -24,12 +25,12 @@ type Event struct { Key string FilterKeys []string Index uint64 - Payload interface{} + Payload map[string]interface{} } -// IsHeartBeat specifies if the event is an empty heartbeat used to +// IsHeartbeat specifies if the event is an empty heartbeat used to // keep a connection alive. -func (e *Events) IsHeartBeat() bool { +func (e *Events) IsHeartbeat() bool { return e.Index == 0 && len(e.Events) == 0 } @@ -45,14 +46,11 @@ func (c *Client) EventStream() *EventStream { // Stream establishes a new subscription to Nomad's event stream and streams // results back to the returned channel. -func (e *EventStream) Stream(ctx context.Context, topics map[Topic][]string, index uint64, q *QueryOptions) (<-chan *Events, <-chan error) { - - errCh := make(chan error, 1) +func (e *EventStream) Stream(ctx context.Context, topics map[Topic][]string, index uint64, q *QueryOptions) (<-chan *Events, error) { r, err := e.client.newRequest("GET", "/v1/event/stream") if err != nil { - errCh <- err - return nil, errCh + return nil, err } r.setQueryOptions(q) @@ -66,39 +64,35 @@ func (e *EventStream) Stream(ctx context.Context, topics map[Topic][]string, ind _, resp, err := requireOK(e.client.doRequest(r)) if err != nil { - errCh <- err - return nil, errCh + return nil, err } eventsCh := make(chan *Events, 10) go func() { defer resp.Body.Close() + defer close(eventsCh) dec := json.NewDecoder(resp.Body) - for { - select { - case <-ctx.Done(): - close(eventsCh) - return - default: - } - + for ctx.Err() == nil { // Decode next newline delimited json of events var events Events if err := dec.Decode(&events); err != nil { - close(eventsCh) - errCh <- err + events = Events{Err: err} + eventsCh <- &events return } - if events.IsHeartBeat() { + if events.IsHeartbeat() { continue } - eventsCh <- &events - + select { + case <-ctx.Done(): + return + case eventsCh <- &events: + } } }() - return eventsCh, errCh + return eventsCh, nil } diff --git a/api/event_test.go b/api/event_test.go index 1ea1c6e4274b..9bd4ae3a4dc1 100644 --- a/api/event_test.go +++ b/api/event_test.go @@ -31,20 +31,49 @@ func TestEvent_Stream(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - streamCh, errCh := events.Stream(ctx, topics, 0, q) + streamCh, err := events.Stream(ctx, topics, 0, q) + require.NoError(t, err) OUTER: for { select { case event := <-streamCh: + if event.Err != nil { + require.Fail(t, err.Error()) + } require.Equal(t, len(event.Events), 1) require.Equal(t, "Eval", string(event.Events[0].Topic)) break OUTER - case err := <-errCh: - require.Fail(t, err.Error()) case <-time.After(5 * time.Second): require.Fail(t, "failed waiting for event stream event") } } } + +func TestEvent_Stream_Err_InvalidQueryParam(t *testing.T) { + t.Parallel() + + c, s := makeClient(t, nil, nil) + defer s.Stop() + + // register job to generate events + jobs := c.Jobs() + job := testJob() + resp2, _, err := jobs.Register(job, nil) + require.Nil(t, err) + require.NotNil(t, resp2) + + // build event stream request + events := c.EventStream() + q := &QueryOptions{} + topics := map[Topic][]string{ + "Eval": {"::*"}, + } + + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + + _, err = events.Stream(ctx, topics, 0, q) + require.Error(t, err) +} diff --git a/command/agent/command.go b/command/agent/command.go index 199f47133541..71beabbeea4f 100644 --- a/command/agent/command.go +++ b/command/agent/command.go @@ -597,6 +597,7 @@ func (c *Command) AutocompleteFlags() complete.Flags { "-vault-tls-server-name": complete.PredictAnything, "-acl-enabled": complete.PredictNothing, "-acl-replication-token": complete.PredictAnything, + "-event-publisher": complete.PredictNothing, } } @@ -1279,6 +1280,9 @@ Server Options: -rejoin Ignore a previous leave and attempts to rejoin the cluster. + -event-publisher + Whether to enable or disable the servers event publisher. + Client Options: -client diff --git a/command/agent/event_endpoint.go b/command/agent/event_endpoint.go index d68563969130..828fbb01ba12 100644 --- a/command/agent/event_endpoint.go +++ b/command/agent/event_endpoint.go @@ -157,7 +157,10 @@ func parseEventTopics(query url.Values) (map[structs.Topic][]string, error) { func parseTopic(topic string) (string, string, error) { parts := strings.Split(topic, ":") - if len(parts) != 2 { + // infer wildcard if only given a topic + if len(parts) == 1 { + return topic, "*", nil + } else if len(parts) != 2 { return "", "", fmt.Errorf("Invalid key value pair for topic, topic: %s", topic) } return parts[0], parts[1], nil diff --git a/command/agent/event_endpoint_test.go b/command/agent/event_endpoint_test.go index af86cce7cadd..915029efc509 100644 --- a/command/agent/event_endpoint_test.go +++ b/command/agent/event_endpoint_test.go @@ -3,7 +3,6 @@ package agent import ( "context" "fmt" - "github.com/hashicorp/nomad/nomad/structs" "net/http" "net/http/httptest" "net/url" @@ -11,6 +10,8 @@ import ( "testing" "time" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/testutil" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -95,9 +96,12 @@ func TestEventStream_QueryParse(t *testing.T) { wantErr: true, }, { - desc: "invalid key value formatting no value", + desc: "Infer wildcard if absent", query: "?topic=NodeDrain", - wantErr: true, + wantErr: false, + want: map[structs.Topic][]string{ + "NodeDrain": {"*"}, + }, }, { desc: "single topic and key", diff --git a/nomad/event_endpoint.go b/nomad/event_endpoint.go index 1e1c94081f6e..6c84d3783b65 100644 --- a/nomad/event_endpoint.go +++ b/nomad/event_endpoint.go @@ -97,7 +97,6 @@ func (e *Event) stream(conn io.ReadWriteCloser) { return } select { - case <-errCh: case <-ctx.Done(): return } @@ -105,7 +104,6 @@ func (e *Event) stream(conn io.ReadWriteCloser) { go func() { defer cancel() - LOOP: for { events, err := subscription.Next(ctx) if err != nil { @@ -113,7 +111,7 @@ func (e *Event) stream(conn io.ReadWriteCloser) { case errCh <- err: case <-ctx.Done(): } - break LOOP + return } // Continue if there are no events @@ -126,7 +124,7 @@ func (e *Event) stream(conn io.ReadWriteCloser) { case errCh <- err: case <-ctx.Done(): } - break LOOP + return } } }() diff --git a/nomad/fsm.go b/nomad/fsm.go index db1aee98cf7e..5bab15b5ee5f 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -1564,22 +1564,33 @@ func (n *nomadFSM) Restore(old io.ReadCloser) error { // Rehydrate the new state store's event publisher with the events // persisted in the snapshot - // pub, err := n.state.EventPublisher() - // if err != nil { - // n.logger.Warn("Snapshot Restore: new state event publisher not configured") - // } - // events, err := n.state.Events(nil) - // if err != nil { - // n.logger.Warn("Snapshot Restore: unable to retrieve current events") - // } - // for { - // raw := events.Next() - // if raw == nil { - // break - // } - // e := raw.(*structs.Events) - // pub.Publish(e) - // } + if n.config.EnableEventPublisher { + if err := rehydratePublisherFromState(n.state); err != nil { + n.logger.Error("Error re-hydrating event publisher during restore", "error", err) + } + } + + return nil +} + +func rehydratePublisherFromState(s *state.StateStore) error { + pub, err := s.EventPublisher() + if err != nil { + return err + } + + events, err := s.Events(nil) + if err != nil { + return err + } + for { + raw := events.Next() + if raw == nil { + break + } + e := raw.(*structs.Events) + pub.Publish(*e) + } return nil } diff --git a/nomad/fsm_test.go b/nomad/fsm_test.go index 98b82feac5b3..d832b797b615 100644 --- a/nomad/fsm_test.go +++ b/nomad/fsm_test.go @@ -2,6 +2,7 @@ package nomad import ( "bytes" + "errors" "fmt" "reflect" "strings" @@ -50,11 +51,12 @@ func testFSM(t *testing.T) *nomadFSM { dispatcher, _ := testPeriodicDispatcher(t) logger := testlog.HCLogger(t) fsmConfig := &FSMConfig{ - EvalBroker: broker, - Periodic: dispatcher, - Blocked: NewBlockedEvals(broker, logger), - Logger: logger, - Region: "global", + EvalBroker: broker, + Periodic: dispatcher, + Blocked: NewBlockedEvals(broker, logger), + Logger: logger, + Region: "global", + EnableEventPublisher: true, } fsm, err := NewFSM(fsmConfig) if err != nil { @@ -3204,6 +3206,8 @@ func TestFSM_SnapshotRestore_Events_WithDurability(t *testing.T) { t.Parallel() // Add some state fsm := testFSM(t) + fsm.config.EnableEventPublisher = true + state := fsm.State() cfg := state.Config() // DurableEventCount = 4 each mock events wrapper contains 2 events @@ -3242,15 +3246,25 @@ func TestFSM_SnapshotRestore_Events_WithDurability(t *testing.T) { // Durable count was 4 so e1 events should be excluded raw1 := iter.Next() require.Nil(t, raw1) + + pub, err := state2.EventPublisher() + require.NoError(t, err) + testutil.WaitForResult(func() (bool, error) { + if pub.Len() == 2 { + return true, nil + } + return false, errors.New("expected publisher to be populated") + }, func(err error) { + require.Fail(t, err.Error()) + }) } func TestFSM_SnapshotRestore_Events_NoDurability(t *testing.T) { t.Parallel() // Add some state fsm := testFSM(t) + fsm.config.EnableEventPublisher = false state := fsm.State() - cfg := state.Config() - cfg.DurableEventCount = 0 e1 := mock.Events(1000) e2 := mock.Events(1001) diff --git a/nomad/state/node_events_test.go b/nomad/state/node_events_test.go index e32e338ec85a..bb24657b16ce 100644 --- a/nomad/state/node_events_test.go +++ b/nomad/state/node_events_test.go @@ -248,7 +248,7 @@ func TestNodeDrainEventFromChanges(t *testing.T) { got, err := processDBChanges(tx, changes) require.NoError(t, err) - require.Len(t, got, 1) + require.Len(t, got.Events, 1) require.Equal(t, TopicNode, got.Events[0].Topic) require.Equal(t, TypeNodeDrain, got.Events[0].Type) diff --git a/nomad/stream/event_publisher.go b/nomad/stream/event_publisher.go index 16e7f0a73fc8..45b8ef8566ad 100644 --- a/nomad/stream/event_publisher.go +++ b/nomad/stream/event_publisher.go @@ -80,6 +80,10 @@ func NewEventPublisher(ctx context.Context, cfg EventPublisherCfg) *EventPublish return e } +func (e *EventPublisher) Len() int { + return e.eventBuf.Len() +} + // Publish events to all subscribers of the event Topic. func (e *EventPublisher) Publish(events structs.Events) { if len(events.Events) > 0 { diff --git a/vendor/github.com/hashicorp/nomad/api/event.go b/vendor/github.com/hashicorp/nomad/api/event.go index b52f4eb8e8b9..dce3c265fe8c 100644 --- a/vendor/github.com/hashicorp/nomad/api/event.go +++ b/vendor/github.com/hashicorp/nomad/api/event.go @@ -6,11 +6,12 @@ import ( "fmt" ) -// Events is a set of events for a corresponding index. Events returned for the +// Ebvents is a set of events for a corresponding index. Events returned for the // index depend on which topics are subscribed to when a request is made. type Events struct { Index uint64 Events []Event + Err error } // Topic is an event Topic @@ -24,12 +25,12 @@ type Event struct { Key string FilterKeys []string Index uint64 - Payload interface{} + Payload map[string]interface{} } -// IsHeartBeat specifies if the event is an empty heartbeat used to +// IsHeartbeat specifies if the event is an empty heartbeat used to // keep a connection alive. -func (e *Events) IsHeartBeat() bool { +func (e *Events) IsHeartbeat() bool { return e.Index == 0 && len(e.Events) == 0 } @@ -45,14 +46,11 @@ func (c *Client) EventStream() *EventStream { // Stream establishes a new subscription to Nomad's event stream and streams // results back to the returned channel. -func (e *EventStream) Stream(ctx context.Context, topics map[Topic][]string, index uint64, q *QueryOptions) (<-chan *Events, <-chan error) { - - errCh := make(chan error, 1) +func (e *EventStream) Stream(ctx context.Context, topics map[Topic][]string, index uint64, q *QueryOptions) (<-chan *Events, error) { r, err := e.client.newRequest("GET", "/v1/event/stream") if err != nil { - errCh <- err - return nil, errCh + return nil, err } r.setQueryOptions(q) @@ -66,39 +64,35 @@ func (e *EventStream) Stream(ctx context.Context, topics map[Topic][]string, ind _, resp, err := requireOK(e.client.doRequest(r)) if err != nil { - errCh <- err - return nil, errCh + return nil, err } eventsCh := make(chan *Events, 10) go func() { defer resp.Body.Close() + defer close(eventsCh) dec := json.NewDecoder(resp.Body) - for { - select { - case <-ctx.Done(): - close(eventsCh) - return - default: - } - + for ctx.Err() == nil { // Decode next newline delimited json of events var events Events if err := dec.Decode(&events); err != nil { - close(eventsCh) - errCh <- err + events = Events{Err: err} + eventsCh <- &events return } - if events.IsHeartBeat() { + if events.IsHeartbeat() { continue } - eventsCh <- &events - + select { + case <-ctx.Done(): + return + case eventsCh <- &events: + } } }() - return eventsCh, errCh + return eventsCh, nil } From 39ef3263cab436531fa1c586b98f4246fa83294c Mon Sep 17 00:00:00 2001 From: Drew Bailey <2614075+drewbailey@users.noreply.github.com> Date: Tue, 6 Oct 2020 16:21:58 -0400 Subject: [PATCH 15/19] Add EvictCallbackFn to handle removing entries from go-memdb when they are removed from the event buffer. Wire up event buffer size config, use pointers for structs.Events instead of copying. --- api/event.go | 6 +- command/agent/agent.go | 7 +- command/agent/command.go | 5 - command/agent/config.go | 18 +++- command/agent/config_parse_test.go | 3 +- command/agent/config_test.go | 40 +++++++ command/agent/event_endpoint.go | 6 +- command/agent/event_endpoint_test.go | 3 +- command/agent/testdata/basic.hcl | 3 +- command/agent/testdata/basic.json | 3 +- nomad/config.go | 5 +- nomad/deploymentwatcher/testutil_test.go | 2 +- nomad/event_endpoint.go | 14 +-- nomad/event_endpoint_test.go | 130 +++++++++++++++++++++-- nomad/fsm.go | 46 ++++---- nomad/fsm_test.go | 19 ++-- nomad/node_endpoint_test.go | 2 +- nomad/server.go | 1 + nomad/state/apply_plan_events.go | 10 +- nomad/state/deployment_events.go | 10 +- nomad/state/events.go | 78 ++++++++++---- nomad/state/node_events.go | 47 ++------ nomad/state/node_events_test.go | 3 +- nomad/state/state_changes.go | 56 ++++++---- nomad/state/state_store.go | 51 +++++++-- nomad/state/state_store_events_test.go | 125 ++++++++++++++++++++++ nomad/state/state_store_test.go | 12 +-- nomad/state/testing.go | 7 +- nomad/stream/event_buffer.go | 98 +++++++++-------- nomad/stream/event_buffer_test.go | 81 ++++++++++---- nomad/stream/event_publisher.go | 43 +++----- nomad/stream/event_publisher_test.go | 7 +- nomad/stream/subscription.go | 6 +- 33 files changed, 669 insertions(+), 278 deletions(-) create mode 100644 nomad/state/state_store_events_test.go diff --git a/api/event.go b/api/event.go index dce3c265fe8c..2bf08fba5793 100644 --- a/api/event.go +++ b/api/event.go @@ -6,7 +6,7 @@ import ( "fmt" ) -// Ebvents is a set of events for a corresponding index. Events returned for the +// Events is a set of events for a corresponding index. Events returned for the // index depend on which topics are subscribed to when a request is made. type Events struct { Index uint64 @@ -78,9 +78,9 @@ func (e *EventStream) Stream(ctx context.Context, topics map[Topic][]string, ind // Decode next newline delimited json of events var events Events if err := dec.Decode(&events); err != nil { + // set error and fallthrough to + // select eventsCh events = Events{Err: err} - eventsCh <- &events - return } if events.IsHeartbeat() { continue diff --git a/command/agent/agent.go b/command/agent/agent.go index 16f3c20d55dc..54de24470f65 100644 --- a/command/agent/agent.go +++ b/command/agent/agent.go @@ -243,8 +243,11 @@ func convertServerConfig(agentConfig *Config) (*nomad.Config, error) { if agentConfig.Server.UpgradeVersion != "" { conf.UpgradeVersion = agentConfig.Server.UpgradeVersion } - if agentConfig.Server.EnableEventPublisher { - conf.EnableEventPublisher = agentConfig.Server.EnableEventPublisher + if agentConfig.Server.EnableEventPublisher != nil { + conf.EnableEventPublisher = *agentConfig.Server.EnableEventPublisher + } + if agentConfig.Server.EventBufferSize > 0 { + conf.EventBufferSize = int64(agentConfig.Server.EventBufferSize) } if agentConfig.Autopilot != nil { if agentConfig.Autopilot.CleanupDeadServers != nil { diff --git a/command/agent/command.go b/command/agent/command.go index 71beabbeea4f..0ed3ff33814e 100644 --- a/command/agent/command.go +++ b/command/agent/command.go @@ -94,7 +94,6 @@ func (c *Command) readConfig() *Config { flags.Var((*flaghelper.StringFlag)(&cmdConfig.Server.ServerJoin.StartJoin), "join", "") flags.Var((*flaghelper.StringFlag)(&cmdConfig.Server.ServerJoin.RetryJoin), "retry-join", "") flags.IntVar(&cmdConfig.Server.ServerJoin.RetryMaxAttempts, "retry-max", 0, "") - flags.BoolVar(&cmdConfig.Server.EnableEventPublisher, "event-publisher", false, "") flags.Var((flaghelper.FuncDurationVar)(func(d time.Duration) error { cmdConfig.Server.ServerJoin.RetryInterval = d return nil @@ -597,7 +596,6 @@ func (c *Command) AutocompleteFlags() complete.Flags { "-vault-tls-server-name": complete.PredictAnything, "-acl-enabled": complete.PredictNothing, "-acl-replication-token": complete.PredictAnything, - "-event-publisher": complete.PredictNothing, } } @@ -1280,9 +1278,6 @@ Server Options: -rejoin Ignore a previous leave and attempts to rejoin the cluster. - -event-publisher - Whether to enable or disable the servers event publisher. - Client Options: -client diff --git a/command/agent/config.go b/command/agent/config.go index 3ba23eceeae9..9665fcaa9ade 100644 --- a/command/agent/config.go +++ b/command/agent/config.go @@ -486,7 +486,12 @@ type ServerConfig struct { // EnableEventPublisher configures whether this server's state store // will generate events for its event stream. - EnableEventPublisher bool `hcl:"enable_event_publisher"` + EnableEventPublisher *bool `hcl:"enable_event_publisher"` + + // EventBufferSize configure the amount of events to be held in memory. + // If EnableEventPublisher is set to true, the minimum allowable value + // for the EventBufferSize is 1. + EventBufferSize int `hcl:"event_buffer_size"` // DurableEventCount specifies the amount of events to persist during snapshot generation. // A count of 0 signals that no events should be persisted. @@ -883,7 +888,8 @@ func DefaultConfig() *Config { }, Server: &ServerConfig{ Enabled: false, - EnableEventPublisher: true, + EnableEventPublisher: helper.BoolToPtr(true), + EventBufferSize: 100, DurableEventCount: 100, StartJoin: []string{}, ServerJoin: &ServerJoin{ @@ -1409,8 +1415,12 @@ func (a *ServerConfig) Merge(b *ServerConfig) *ServerConfig { result.ServerJoin = result.ServerJoin.Merge(b.ServerJoin) } - if b.EnableEventPublisher { - result.EnableEventPublisher = true + if b.EnableEventPublisher != nil { + result.EnableEventPublisher = b.EnableEventPublisher + } + + if b.EventBufferSize != 0 { + result.EventBufferSize = b.EventBufferSize } if b.DurableEventCount != 0 { diff --git a/command/agent/config_parse_test.go b/command/agent/config_parse_test.go index 9cce0f52eb46..8999fa281208 100644 --- a/command/agent/config_parse_test.go +++ b/command/agent/config_parse_test.go @@ -122,7 +122,8 @@ var basicConfig = &Config{ RedundancyZone: "foo", UpgradeVersion: "0.8.0", EncryptKey: "abc", - EnableEventPublisher: true, + EnableEventPublisher: helper.BoolToPtr(false), + EventBufferSize: 200, DurableEventCount: 100, ServerJoin: &ServerJoin{ RetryJoin: []string{"1.1.1.1", "2.2.2.2"}, diff --git a/command/agent/config_test.go b/command/agent/config_test.go index 32115d32dbc2..59aa42e0572a 100644 --- a/command/agent/config_test.go +++ b/command/agent/config_test.go @@ -138,6 +138,7 @@ func TestConfig_Merge(t *testing.T) { MaxHeartbeatsPerSecond: 30.0, RedundancyZone: "foo", UpgradeVersion: "foo", + EnableEventPublisher: helper.BoolToPtr(false), }, ACL: &ACLConfig{ Enabled: true, @@ -328,6 +329,7 @@ func TestConfig_Merge(t *testing.T) { NonVotingServer: true, RedundancyZone: "bar", UpgradeVersion: "bar", + EnableEventPublisher: helper.BoolToPtr(true), }, ACL: &ACLConfig{ Enabled: true, @@ -1163,3 +1165,41 @@ func TestTelemetry_Parse(t *testing.T) { require.Exactly([]string{"+nomad.raft"}, config.Telemetry.PrefixFilter) require.True(config.Telemetry.DisableDispatchedJobSummaryMetrics) } + +func TestEventPublisher_Parse(t *testing.T) { + + require := require.New(t) + + { + a := &ServerConfig{ + EnableEventPublisher: helper.BoolToPtr(false), + } + b := DefaultConfig().Server + b.EnableEventPublisher = nil + + result := a.Merge(b) + require.Equal(false, *result.EnableEventPublisher) + } + + { + a := &ServerConfig{ + EnableEventPublisher: helper.BoolToPtr(true), + } + b := DefaultConfig().Server + b.EnableEventPublisher = nil + + result := a.Merge(b) + require.Equal(true, *result.EnableEventPublisher) + } + + { + a := &ServerConfig{ + EnableEventPublisher: helper.BoolToPtr(false), + } + b := DefaultConfig().Server + b.EnableEventPublisher = helper.BoolToPtr(true) + + result := a.Merge(b) + require.Equal(true, *result.EnableEventPublisher) + } +} diff --git a/command/agent/event_endpoint.go b/command/agent/event_endpoint.go index 828fbb01ba12..af46620acf5c 100644 --- a/command/agent/event_endpoint.go +++ b/command/agent/event_endpoint.go @@ -74,7 +74,7 @@ func (s *HTTPServer) EventStream(resp http.ResponseWriter, req *http.Request) (i output := ioutils.NewWriteFlusher(resp) // create an error channel to handle errors - errCh := make(chan HTTPCodedError, 2) + errCh := make(chan HTTPCodedError, 1) go func() { defer cancel() @@ -124,9 +124,7 @@ func (s *HTTPServer) EventStream(resp http.ResponseWriter, req *http.Request) (i cancel() codedErr := <-errCh - if codedErr != nil && - (codedErr == io.EOF || - strings.Contains(codedErr.Error(), io.ErrClosedPipe.Error())) { + if codedErr != nil && strings.Contains(codedErr.Error(), io.ErrClosedPipe.Error()) { codedErr = nil } diff --git a/command/agent/event_endpoint_test.go b/command/agent/event_endpoint_test.go index 915029efc509..546dd6516dd0 100644 --- a/command/agent/event_endpoint_test.go +++ b/command/agent/event_endpoint_test.go @@ -39,7 +39,7 @@ func TestEventStream(t *testing.T) { pub, err := s.Agent.server.State().EventPublisher() require.NoError(t, err) - pub.Publish(structs.Events{Index: 100, Events: []structs.Event{{Payload: testEvent{ID: "123"}}}}) + pub.Publish(&structs.Events{Index: 100, Events: []structs.Event{{Payload: testEvent{ID: "123"}}}}) testutil.WaitForResult(func() (bool, error) { got := resp.Body.String() @@ -56,7 +56,6 @@ func TestEventStream(t *testing.T) { // wait for response to close to prevent race between subscription // shutdown and server shutdown returning subscription closed by server err - // resp.Close() cancel() select { case err := <-respErrCh: diff --git a/command/agent/testdata/basic.hcl b/command/agent/testdata/basic.hcl index da08118dbe18..be35093e9802 100644 --- a/command/agent/testdata/basic.hcl +++ b/command/agent/testdata/basic.hcl @@ -130,7 +130,8 @@ server { upgrade_version = "0.8.0" encrypt = "abc" raft_multiplier = 4 - enable_event_publisher = true + enable_event_publisher = false + event_buffer_size = 200 durable_event_count = 100 server_join { diff --git a/command/agent/testdata/basic.json b/command/agent/testdata/basic.json index 2d671dd1a99e..d9db2d5d0f19 100644 --- a/command/agent/testdata/basic.json +++ b/command/agent/testdata/basic.json @@ -261,7 +261,8 @@ "data_dir": "/tmp/data", "deployment_gc_threshold": "12h", "enabled": true, - "enable_event_publisher": true, + "enable_event_publisher": false, + "event_buffer_size": 200, "durable_event_count": 100, "enabled_schedulers": [ "test" diff --git a/nomad/config.go b/nomad/config.go index 081d83e02601..0a2272be8848 100644 --- a/nomad/config.go +++ b/nomad/config.go @@ -78,10 +78,13 @@ type Config struct { // in the absence of ACLs EnableDebug bool - // EnableEventPublisher is used to enable or disable the state stores + // EnableEventPublisher is used to enable or disable state store // event publishing EnableEventPublisher bool + // EventBufferSize is the amount of events to hold in memory. + EventBufferSize int64 + // LogOutput is the location to write logs to. If this is not set, // logs will go to stderr. LogOutput io.Writer diff --git a/nomad/deploymentwatcher/testutil_test.go b/nomad/deploymentwatcher/testutil_test.go index a3e21d7df26a..87cd492fe61b 100644 --- a/nomad/deploymentwatcher/testutil_test.go +++ b/nomad/deploymentwatcher/testutil_test.go @@ -38,7 +38,7 @@ func (m *mockBackend) nextIndex() uint64 { func (m *mockBackend) UpdateAllocDesiredTransition(u *structs.AllocUpdateDesiredTransitionRequest) (uint64, error) { m.Called(u) i := m.nextIndex() - return i, m.state.UpdateAllocsDesiredTransitions(i, u.Allocs, u.Evals) + return i, m.state.UpdateAllocsDesiredTransitions(structs.MsgTypeTestSetup, i, u.Allocs, u.Evals) } // matchUpdateAllocDesiredTransitions is used to match an upsert request diff --git a/nomad/event_endpoint.go b/nomad/event_endpoint.go index 6c84d3783b65..202fcf99b8b8 100644 --- a/nomad/event_endpoint.go +++ b/nomad/event_endpoint.go @@ -55,14 +55,16 @@ func (e *Event) stream(conn io.ReadWriteCloser) { return } - // TODO(drew) handle streams without ACLS - reqToken := args.AuthToken - if reqToken == "" { - // generate a random request token - reqToken = uuid.Generate() + // authToken is passed to the subscribe request so the event stream + // can handle closing a subscription if the authToken expires. + // If ACLs are disabled, a random token is generated and it will + // never be closed due to expiry. + authToken := args.AuthToken + if authToken == "" { + authToken = uuid.Generate() } subReq := &stream.SubscribeRequest{ - Token: reqToken, + Token: authToken, Topics: args.Topics, Index: uint64(args.Index), } diff --git a/nomad/event_endpoint_test.go b/nomad/event_endpoint_test.go index b05aef2b8906..573084ff3139 100644 --- a/nomad/event_endpoint_test.go +++ b/nomad/event_endpoint_test.go @@ -10,6 +10,7 @@ import ( "time" "github.com/hashicorp/go-msgpack/codec" + "github.com/hashicorp/nomad/acl" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/stream" "github.com/hashicorp/nomad/nomad/structs" @@ -28,7 +29,7 @@ func TestEventStream(t *testing.T) { // Create request for all topics and keys req := structs.EventStreamRequest{ - Topics: map[structs.Topic][]string{"*": []string{"*"}}, + Topics: map[structs.Topic][]string{"*": {"*"}}, QueryOptions: structs.QueryOptions{ Region: s1.Region(), }, @@ -47,7 +48,7 @@ func TestEventStream(t *testing.T) { // invoke handler go handler(p2) - // send request + // decode request responses go func() { decoder := codec.NewDecoder(p1, structs.MsgpackHandle) for { @@ -68,8 +69,9 @@ func TestEventStream(t *testing.T) { require.NoError(t, err) node := mock.Node() - publisher.Publish(structs.Events{Index: uint64(1), Events: []structs.Event{{Topic: "test", Payload: node}}}) + publisher.Publish(&structs.Events{Index: uint64(1), Events: []structs.Event{{Topic: "test", Payload: node}}}) + // Send request encoder := codec.NewEncoder(p1, structs.MsgpackHandle) require.Nil(t, encoder.Encode(req)) @@ -160,14 +162,14 @@ func TestEventStream_StreamErr(t *testing.T) { require.NoError(t, err) node := mock.Node() - publisher.Publish(structs.Events{uint64(1), []structs.Event{{Topic: "test", Payload: node}}}) // send req encoder := codec.NewEncoder(p1, structs.MsgpackHandle) require.Nil(t, encoder.Encode(req)) - // stop the publisher to force an error on subscription side - s1.State().StopEventPublisher() + // publish some events + publisher.Publish(&structs.Events{Index: uint64(1), Events: []structs.Event{{Topic: "test", Payload: node}}}) + publisher.Publish(&structs.Events{Index: uint64(2), Events: []structs.Event{{Topic: "test", Payload: node}}}) timeout := time.After(5 * time.Second) OUTER: @@ -178,8 +180,10 @@ OUTER: case err := <-errCh: t.Fatal(err) case msg := <-streamMsg: + // close the publishers subscriptions forcing an error + // after an initial event is received + publisher.CloseAll() if msg.Error == nil { - // race between error and receiving an event // continue trying for error continue } @@ -249,7 +253,7 @@ func TestEventStream_RegionForward(t *testing.T) { require.NoError(t, err) node := mock.Node() - publisher.Publish(structs.Events{uint64(1), []structs.Event{{Topic: "test", Payload: node}}}) + publisher.Publish(&structs.Events{Index: uint64(1), Events: []structs.Event{{Topic: "test", Payload: node}}}) // send req encoder := codec.NewEncoder(p1, structs.MsgpackHandle) @@ -290,6 +294,114 @@ OUTER: } } -// TODO(drew) acl test func TestEventStream_ACL(t *testing.T) { + t.Parallel() + require := require.New(t) + + // start server + s, root, cleanupS := TestACLServer(t, nil) + defer cleanupS() + testutil.WaitForLeader(t, s.RPC) + + policyBad := mock.NamespacePolicy("other", "", []string{acl.NamespaceCapabilityReadFS}) + tokenBad := mock.CreatePolicyAndToken(t, s.State(), 1005, "invalid", policyBad) + + cases := []struct { + Name string + Token string + ExpectedErr string + }{ + { + Name: "no token", + Token: "", + ExpectedErr: structs.ErrPermissionDenied.Error(), + }, + { + Name: "bad token", + Token: tokenBad.SecretID, + ExpectedErr: structs.ErrPermissionDenied.Error(), + }, + { + Name: "root token", + Token: root.SecretID, + ExpectedErr: "subscription closed by server", + }, + } + + for _, tc := range cases { + t.Run(tc.Name, func(t *testing.T) { + // Create request for all topics and keys + req := structs.EventStreamRequest{ + Topics: map[structs.Topic][]string{"*": {"*"}}, + QueryOptions: structs.QueryOptions{ + Region: s.Region(), + AuthToken: tc.Token, + }, + } + + handler, err := s.StreamingRpcHandler("Event.Stream") + require.Nil(err) + + // create pipe + p1, p2 := net.Pipe() + defer p1.Close() + defer p2.Close() + + errCh := make(chan error) + streamMsg := make(chan *structs.EventStreamWrapper) + + go handler(p2) + + // Start decoder + go func() { + decoder := codec.NewDecoder(p1, structs.MsgpackHandle) + for { + var msg structs.EventStreamWrapper + if err := decoder.Decode(&msg); err != nil { + if err == io.EOF || strings.Contains(err.Error(), "closed") { + return + } + errCh <- fmt.Errorf("error decoding: %w", err) + } + + streamMsg <- &msg + } + }() + + // send request + encoder := codec.NewEncoder(p1, structs.MsgpackHandle) + require.Nil(encoder.Encode(req)) + + publisher, err := s.State().EventPublisher() + require.NoError(err) + + // publish some events + node := mock.Node() + publisher.Publish(&structs.Events{Index: uint64(1), Events: []structs.Event{{Topic: "test", Payload: node}}}) + publisher.Publish(&structs.Events{Index: uint64(2), Events: []structs.Event{{Topic: "test", Payload: node}}}) + + timeout := time.After(5 * time.Second) + OUTER: + for { + select { + case <-timeout: + require.Fail("timeout waiting for response") + case err := <-errCh: + t.Fatal(err) + case msg := <-streamMsg: + // force error by closing all subscriptions + publisher.CloseAll() + if msg.Error == nil { + continue + } + + if strings.Contains(msg.Error.Error(), tc.ExpectedErr) { + break OUTER + } else { + require.Fail("Unexpected error", msg.Error) + } + } + } + }) + } } diff --git a/nomad/fsm.go b/nomad/fsm.go index 5bab15b5ee5f..ead6d43e03f2 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -100,8 +100,9 @@ type nomadFSM struct { // state in a way that can be accessed concurrently with operations // that may modify the live state. type nomadSnapshot struct { - snap *state.StateSnapshot - timetable *TimeTable + snap *state.StateSnapshot + timetable *TimeTable + durableEventCount int } // snapshotHeader is the first entry in our snapshot @@ -129,6 +130,8 @@ type FSMConfig struct { EnableEventPublisher bool + EventBufferSize int64 + // Durable count specifies the amount of events generated by the state store // to save to disk during snapshot generation. The most recent events // limited to count will be saved. @@ -142,6 +145,7 @@ func NewFSM(config *FSMConfig) (*nomadFSM, error) { Logger: config.Logger, Region: config.Region, EnablePublisher: config.EnableEventPublisher, + EventBufferSize: config.EventBufferSize, DurableEventCount: config.DurableEventCount, } state, err := state.NewStateStore(sconfig) @@ -258,21 +262,16 @@ func (n *nomadFSM) Apply(log *raft.Log) interface{} { case structs.UpsertNodeEventsType: return n.applyUpsertNodeEvent(msgType, buf[1:], log.Index) case structs.JobBatchDeregisterRequestType: - // TODO(drew) higher priority event return n.applyBatchDeregisterJob(msgType, buf[1:], log.Index) case structs.AllocUpdateDesiredTransitionRequestType: - // TODO(drew) higher priority event - return n.applyAllocUpdateDesiredTransition(buf[1:], log.Index) + return n.applyAllocUpdateDesiredTransition(msgType, buf[1:], log.Index) case structs.NodeUpdateEligibilityRequestType: - // TODO(drew) higher priority event - return n.applyNodeEligibilityUpdate(buf[1:], log.Index) + return n.applyNodeEligibilityUpdate(msgType, buf[1:], log.Index) case structs.BatchNodeUpdateDrainRequestType: - // TODO(drew) higher priority event - return n.applyBatchDrainUpdate(buf[1:], log.Index) + return n.applyBatchDrainUpdate(msgType, buf[1:], log.Index) case structs.SchedulerConfigRequestType: return n.applySchedulerConfigUpdate(buf[1:], log.Index) case structs.NodeBatchDeregisterRequestType: - // TODO(drew) higher priority event return n.applyDeregisterNodeBatch(buf[1:], log.Index) case structs.ClusterMetadataRequestType: return n.applyClusterMetadata(buf[1:], log.Index) @@ -437,21 +436,21 @@ func (n *nomadFSM) applyDrainUpdate(reqType structs.MessageType, buf []byte, ind return nil } -func (n *nomadFSM) applyBatchDrainUpdate(buf []byte, index uint64) interface{} { +func (n *nomadFSM) applyBatchDrainUpdate(msgType structs.MessageType, buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "batch_node_drain_update"}, time.Now()) var req structs.BatchNodeUpdateDrainRequest if err := structs.Decode(buf, &req); err != nil { panic(fmt.Errorf("failed to decode request: %v", err)) } - if err := n.state.BatchUpdateNodeDrain(index, req.UpdatedAt, req.Updates, req.NodeEvents); err != nil { + if err := n.state.BatchUpdateNodeDrain(msgType, index, req.UpdatedAt, req.Updates, req.NodeEvents); err != nil { n.logger.Error("BatchUpdateNodeDrain failed", "error", err) return err } return nil } -func (n *nomadFSM) applyNodeEligibilityUpdate(buf []byte, index uint64) interface{} { +func (n *nomadFSM) applyNodeEligibilityUpdate(msgType structs.MessageType, buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "node_eligibility_update"}, time.Now()) var req structs.NodeUpdateEligibilityRequest if err := structs.Decode(buf, &req); err != nil { @@ -465,7 +464,7 @@ func (n *nomadFSM) applyNodeEligibilityUpdate(buf []byte, index uint64) interfac return err } - if err := n.state.UpdateNodeEligibility(index, req.NodeID, req.Eligibility, req.UpdatedAt, req.NodeEvent); err != nil { + if err := n.state.UpdateNodeEligibility(msgType, index, req.NodeID, req.Eligibility, req.UpdatedAt, req.NodeEvent); err != nil { n.logger.Error("UpdateNodeEligibility failed", "error", err) return err } @@ -857,14 +856,14 @@ func (n *nomadFSM) applyAllocClientUpdate(msgType structs.MessageType, buf []byt // applyAllocUpdateDesiredTransition is used to update the desired transitions // of a set of allocations. -func (n *nomadFSM) applyAllocUpdateDesiredTransition(buf []byte, index uint64) interface{} { +func (n *nomadFSM) applyAllocUpdateDesiredTransition(msgType structs.MessageType, buf []byte, index uint64) interface{} { defer metrics.MeasureSince([]string{"nomad", "fsm", "alloc_update_desired_transition"}, time.Now()) var req structs.AllocUpdateDesiredTransitionRequest if err := structs.Decode(buf, &req); err != nil { panic(fmt.Errorf("failed to decode request: %v", err)) } - if err := n.state.UpdateAllocsDesiredTransitions(index, req.Allocs, req.Evals); err != nil { + if err := n.state.UpdateAllocsDesiredTransitions(msgType, index, req.Allocs, req.Evals); err != nil { n.logger.Error("UpdateAllocsDesiredTransitions failed", "error", err) return err } @@ -1262,8 +1261,9 @@ func (n *nomadFSM) Snapshot() (raft.FSMSnapshot, error) { } ns := &nomadSnapshot{ - snap: snap, - timetable: n.timetable, + snap: snap, + timetable: n.timetable, + durableEventCount: n.config.DurableEventCount, } return ns, nil } @@ -1276,6 +1276,7 @@ func (n *nomadFSM) Restore(old io.ReadCloser) error { Logger: n.config.Logger, Region: n.config.Region, EnablePublisher: n.config.EnableEventPublisher, + EventBufferSize: n.config.EventBufferSize, DurableEventCount: n.config.DurableEventCount, } newState, err := state.NewStateStore(config) @@ -1589,7 +1590,7 @@ func rehydratePublisherFromState(s *state.StateStore) error { break } e := raw.(*structs.Events) - pub.Publish(*e) + pub.Publish(e) } return nil } @@ -2376,11 +2377,8 @@ func (s *nomadSnapshot) persistCSIVolumes(sink raft.SnapshotSink, } func (s *nomadSnapshot) persistEvents(sink raft.SnapshotSink, encoder *codec.Encoder) error { - var durableCount int - if s.snap.Config() != nil && s.snap.Config().DurableEventCount == 0 { + if s.durableEventCount == 0 { return nil - } else { - durableCount = s.snap.Config().DurableEventCount } events, err := s.snap.LatestEventsReverse(nil) @@ -2409,7 +2407,7 @@ func (s *nomadSnapshot) persistEvents(sink raft.SnapshotSink, encoder *codec.Enc count += eventCount // Only write to sink until durableCount has been reached - if count >= durableCount { + if count >= s.durableEventCount { return nil } } diff --git a/nomad/fsm_test.go b/nomad/fsm_test.go index d832b797b615..b1f8d83dd2eb 100644 --- a/nomad/fsm_test.go +++ b/nomad/fsm_test.go @@ -2,7 +2,6 @@ package nomad import ( "bytes" - "errors" "fmt" "reflect" "strings" @@ -57,6 +56,7 @@ func testFSM(t *testing.T) *nomadFSM { Logger: logger, Region: "global", EnableEventPublisher: true, + EventBufferSize: 100, } fsm, err := NewFSM(fsmConfig) if err != nil { @@ -3207,11 +3207,10 @@ func TestFSM_SnapshotRestore_Events_WithDurability(t *testing.T) { // Add some state fsm := testFSM(t) fsm.config.EnableEventPublisher = true + // DurableEventCount = 4 each mock events wrapper contains 2 events + fsm.config.DurableEventCount = 4 state := fsm.State() - cfg := state.Config() - // DurableEventCount = 4 each mock events wrapper contains 2 events - cfg.DurableEventCount = 4 e1 := mock.Events(1000) e2 := mock.Events(1001) @@ -3249,11 +3248,13 @@ func TestFSM_SnapshotRestore_Events_WithDurability(t *testing.T) { pub, err := state2.EventPublisher() require.NoError(t, err) + testutil.WaitForResult(func() (bool, error) { - if pub.Len() == 2 { + plen := pub.Len() + if plen == 4 { return true, nil } - return false, errors.New("expected publisher to be populated") + return false, fmt.Errorf("expected publisher to have len 2 got: %d", plen) }, func(err error) { require.Fail(t, err.Error()) }) @@ -3261,9 +3262,11 @@ func TestFSM_SnapshotRestore_Events_WithDurability(t *testing.T) { func TestFSM_SnapshotRestore_Events_NoDurability(t *testing.T) { t.Parallel() - // Add some state fsm := testFSM(t) - fsm.config.EnableEventPublisher = false + // Enable event publisher with durable event count of zero + fsm.config.EnableEventPublisher = true + fsm.config.DurableEventCount = 0 + state := fsm.State() e1 := mock.Events(1000) diff --git a/nomad/node_endpoint_test.go b/nomad/node_endpoint_test.go index 23baba2e41c4..0f685dd67c2b 100644 --- a/nomad/node_endpoint_test.go +++ b/nomad/node_endpoint_test.go @@ -1917,7 +1917,7 @@ func TestClientEndpoint_GetClientAllocs_Blocking_GC(t *testing.T) { if time.Since(start) < 100*time.Millisecond { t.Fatalf("too fast") } - assert.EqualValues(200, resp3.Index) + assert.EqualValues(200, int(resp3.Index)) if assert.Len(resp3.Allocs, 1) { assert.EqualValues(100, resp3.Allocs[alloc1.ID]) } diff --git a/nomad/server.go b/nomad/server.go index dea6e29c1e6c..29757dd3c080 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -1217,6 +1217,7 @@ func (s *Server) setupRaft() error { Logger: s.logger, Region: s.Region(), EnableEventPublisher: s.config.EnableEventPublisher, + EventBufferSize: s.config.EventBufferSize, } var err error s.fsm, err = NewFSM(fsmConfig) diff --git a/nomad/state/apply_plan_events.go b/nomad/state/apply_plan_events.go index 1132dafd7726..9bf4b813dc9a 100644 --- a/nomad/state/apply_plan_events.go +++ b/nomad/state/apply_plan_events.go @@ -6,14 +6,14 @@ import ( "github.com/hashicorp/nomad/nomad/structs" ) -func ApplyPlanResultEventsFromChanges(tx ReadTxn, changes Changes) (structs.Events, error) { +func ApplyPlanResultEventsFromChanges(tx ReadTxn, changes Changes) (*structs.Events, error) { var events []structs.Event for _, change := range changes.Changes { switch change.Table { case "deployment": after, ok := change.After.(*structs.Deployment) if !ok { - return structs.Events{}, fmt.Errorf("transaction change was not a Deployment") + return nil, fmt.Errorf("transaction change was not a Deployment") } event := structs.Event{ @@ -29,7 +29,7 @@ func ApplyPlanResultEventsFromChanges(tx ReadTxn, changes Changes) (structs.Even case "evals": after, ok := change.After.(*structs.Evaluation) if !ok { - return structs.Events{}, fmt.Errorf("transaction change was not an Evaluation") + return nil, fmt.Errorf("transaction change was not an Evaluation") } event := structs.Event{ @@ -45,7 +45,7 @@ func ApplyPlanResultEventsFromChanges(tx ReadTxn, changes Changes) (structs.Even case "allocs": after, ok := change.After.(*structs.Allocation) if !ok { - return structs.Events{}, fmt.Errorf("transaction change was not an Allocation") + return nil, fmt.Errorf("transaction change was not an Allocation") } before := change.Before var msg string @@ -69,5 +69,5 @@ func ApplyPlanResultEventsFromChanges(tx ReadTxn, changes Changes) (structs.Even } } - return structs.Events{Index: changes.Index, Events: events}, nil + return &structs.Events{Index: changes.Index, Events: events}, nil } diff --git a/nomad/state/deployment_events.go b/nomad/state/deployment_events.go index 3f11d351d3ac..01c51992c23e 100644 --- a/nomad/state/deployment_events.go +++ b/nomad/state/deployment_events.go @@ -6,7 +6,7 @@ import ( "github.com/hashicorp/nomad/nomad/structs" ) -func DeploymentEventFromChanges(msgType structs.MessageType, tx ReadTxn, changes Changes) (structs.Events, error) { +func DeploymentEventFromChanges(msgType structs.MessageType, tx ReadTxn, changes Changes) (*structs.Events, error) { var events []structs.Event var eventType string @@ -24,7 +24,7 @@ func DeploymentEventFromChanges(msgType structs.MessageType, tx ReadTxn, changes case "deployment": after, ok := change.After.(*structs.Deployment) if !ok { - return structs.Events{}, fmt.Errorf("transaction change was not a Deployment") + return nil, fmt.Errorf("transaction change was not a Deployment") } event := structs.Event{ @@ -42,7 +42,7 @@ func DeploymentEventFromChanges(msgType structs.MessageType, tx ReadTxn, changes case "jobs": after, ok := change.After.(*structs.Job) if !ok { - return structs.Events{}, fmt.Errorf("transaction change was not a Job") + return nil, fmt.Errorf("transaction change was not a Job") } event := structs.Event{ @@ -61,7 +61,7 @@ func DeploymentEventFromChanges(msgType structs.MessageType, tx ReadTxn, changes case "evals": after, ok := change.After.(*structs.Evaluation) if !ok { - return structs.Events{}, fmt.Errorf("transaction change was not an Evaluation") + return nil, fmt.Errorf("transaction change was not an Evaluation") } event := structs.Event{ @@ -79,5 +79,5 @@ func DeploymentEventFromChanges(msgType structs.MessageType, tx ReadTxn, changes } } - return structs.Events{Index: changes.Index, Events: events}, nil + return &structs.Events{Index: changes.Index, Events: events}, nil } diff --git a/nomad/state/events.go b/nomad/state/events.go index 52f9450ccca8..62060137d538 100644 --- a/nomad/state/events.go +++ b/nomad/state/events.go @@ -17,21 +17,22 @@ const ( TopicNode structs.Topic = "Node" // TODO(drew) Node Events use TopicNode + Type - TypeNodeRegistration = "NodeRegistration" - TypeNodeDeregistration = "NodeDeregistration" - TypeNodeDrain = "NodeDrain" - TypeNodeEvent = "NodeEvent" - - TypeDeploymentUpdate = "DeploymentStatusUpdate" - TypeDeploymentPromotion = "DeploymentPromotion" - TypeDeploymentAllocHealth = "DeploymentAllocHealth" - - TypeAllocCreated = "AllocCreated" - TypeAllocUpdated = "AllocUpdated" - - TypeEvalUpdated = "EvalUpdated" - - TypeJobRegistered = "JobRegistered" + TypeNodeRegistration = "NodeRegistration" + TypeNodeDeregistration = "NodeDeregistration" + TypeNodeEligibilityUpdate = "NodeEligibility" + TypeNodeDrain = "NodeDrain" + TypeNodeEvent = "NodeEvent" + + TypeDeploymentUpdate = "DeploymentStatusUpdate" + TypeDeploymentPromotion = "DeploymentPromotion" + TypeDeploymentAllocHealth = "DeploymentAllocHealth" + TypeAllocCreated = "AllocCreated" + TypeAllocUpdated = "AllocUpdated" + TypeAllocUpdateDesiredStatus = "AllocUpdateDesiredStatus" + TypeEvalUpdated = "EvalUpdated" + TypeJobRegistered = "JobRegistered" + TypeJobDeregistered = "JobDeregistered" + TypeJobBatchDeregistered = "JobBatchDeregistered" ) type JobEvent struct { @@ -72,9 +73,13 @@ type JobDrainDetails struct { AllocDetails map[string]NodeDrainAllocDetails } -func GenericEventsFromChanges(tx ReadTxn, changes Changes) (structs.Events, error) { +func GenericEventsFromChanges(tx ReadTxn, changes Changes) (*structs.Events, error) { var eventType string switch changes.MsgType { + case structs.NodeRegisterRequestType: + eventType = TypeNodeRegistration + case structs.UpsertNodeEventsType: + eventType = TypeNodeEvent case structs.EvalUpdateRequestType: eventType = TypeEvalUpdated case structs.AllocClientUpdateRequestType: @@ -85,15 +90,31 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) (structs.Events, erro eventType = TypeAllocUpdated case structs.NodeUpdateStatusRequestType: eventType = TypeNodeEvent + case structs.JobDeregisterRequestType: + eventType = TypeJobDeregistered + case structs.JobBatchDeregisterRequestType: + eventType = TypeJobBatchDeregistered + case structs.AllocUpdateDesiredTransitionRequestType: + eventType = TypeAllocUpdateDesiredStatus + case structs.NodeUpdateEligibilityRequestType: + eventType = TypeNodeDrain + case structs.BatchNodeUpdateDrainRequestType: + eventType = TypeNodeDrain + default: + // unknown request type + return nil, nil } var events []structs.Event for _, change := range changes.Changes { switch change.Table { case "evals": + if change.Deleted() { + return nil, nil + } after, ok := change.After.(*structs.Evaluation) if !ok { - return structs.Events{}, fmt.Errorf("transaction change was not an Evaluation") + return nil, fmt.Errorf("transaction change was not an Evaluation") } event := structs.Event{ @@ -109,26 +130,36 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) (structs.Events, erro events = append(events, event) case "allocs": + if change.Deleted() { + return nil, nil + } after, ok := change.After.(*structs.Allocation) if !ok { - return structs.Events{}, fmt.Errorf("transaction change was not an Allocation") + return nil, fmt.Errorf("transaction change was not an Allocation") } + alloc := after.Copy() + // remove job info to help keep size of alloc event down + alloc.Job = nil + event := structs.Event{ Topic: TopicAlloc, Type: eventType, Index: changes.Index, Key: after.ID, Payload: &AllocEvent{ - Alloc: after, + Alloc: alloc, }, } events = append(events, event) case "jobs": + if change.Deleted() { + return nil, nil + } after, ok := change.After.(*structs.Job) if !ok { - return structs.Events{}, fmt.Errorf("transaction change was not an Allocation") + return nil, fmt.Errorf("transaction change was not an Allocation") } event := structs.Event{ @@ -143,9 +174,12 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) (structs.Events, erro events = append(events, event) case "nodes": + if change.Deleted() { + return nil, nil + } after, ok := change.After.(*structs.Node) if !ok { - return structs.Events{}, fmt.Errorf("transaction change was not a Node") + return nil, fmt.Errorf("transaction change was not a Node") } event := structs.Event{ @@ -161,5 +195,5 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) (structs.Events, erro } } - return structs.Events{Index: changes.Index, Events: events}, nil + return &structs.Events{Index: changes.Index, Events: events}, nil } diff --git a/nomad/state/node_events.go b/nomad/state/node_events.go index e709f3cf32a0..1c85ec5723e8 100644 --- a/nomad/state/node_events.go +++ b/nomad/state/node_events.go @@ -8,14 +8,14 @@ import ( // NodeRegisterEventFromChanges generates a NodeRegistrationEvent from a set // of transaction changes. -func NodeRegisterEventFromChanges(tx ReadTxn, changes Changes) (structs.Events, error) { +func NodeRegisterEventFromChanges(tx ReadTxn, changes Changes) (*structs.Events, error) { var events []structs.Event for _, change := range changes.Changes { switch change.Table { case "nodes": after, ok := change.After.(*structs.Node) if !ok { - return structs.Events{}, fmt.Errorf("transaction change was not a Node") + return nil, fmt.Errorf("transaction change was not a Node") } event := structs.Event{ @@ -30,19 +30,19 @@ func NodeRegisterEventFromChanges(tx ReadTxn, changes Changes) (structs.Events, events = append(events, event) } } - return structs.Events{Index: changes.Index, Events: events}, nil + return &structs.Events{Index: changes.Index, Events: events}, nil } // NodeDeregisterEventFromChanges generates a NodeDeregistrationEvent from a set // of transaction changes. -func NodeDeregisterEventFromChanges(tx ReadTxn, changes Changes) (structs.Events, error) { +func NodeDeregisterEventFromChanges(tx ReadTxn, changes Changes) (*structs.Events, error) { var events []structs.Event for _, change := range changes.Changes { switch change.Table { case "nodes": before, ok := change.Before.(*structs.Node) if !ok { - return structs.Events{}, fmt.Errorf("transaction change was not a Node") + return nil, fmt.Errorf("transaction change was not a Node") } event := structs.Event{ @@ -57,50 +57,23 @@ func NodeDeregisterEventFromChanges(tx ReadTxn, changes Changes) (structs.Events events = append(events, event) } } - return structs.Events{Index: changes.Index, Events: events}, nil + return &structs.Events{Index: changes.Index, Events: events}, nil } -// NodeEventFromChanges generates a NodeDeregistrationEvent from a set -// of transaction changes. -func NodeEventFromChanges(tx ReadTxn, changes Changes) (structs.Events, error) { - var events []structs.Event - for _, change := range changes.Changes { - switch change.Table { - case "nodes": - after, ok := change.After.(*structs.Node) - if !ok { - return structs.Events{}, fmt.Errorf("transaction change was not a Node") - } - - event := structs.Event{ - Topic: TopicNode, - Type: TypeNodeEvent, - Index: changes.Index, - Key: after.ID, - Payload: &NodeEvent{ - Node: after, - }, - } - events = append(events, event) - } - } - return structs.Events{Index: changes.Index, Events: events}, nil -} - -func NodeDrainEventFromChanges(tx ReadTxn, changes Changes) (structs.Events, error) { +func NodeDrainEventFromChanges(tx ReadTxn, changes Changes) (*structs.Events, error) { var events []structs.Event for _, change := range changes.Changes { switch change.Table { case "nodes": after, ok := change.After.(*structs.Node) if !ok { - return structs.Events{}, fmt.Errorf("transaction change was not a Node") + return nil, fmt.Errorf("transaction change was not a Node") } // retrieve allocations currently on node allocs, err := allocsByNodeTxn(tx, nil, after.ID) if err != nil { - return structs.Events{}, fmt.Errorf("retrieving allocations for node drain event: %w", err) + return nil, fmt.Errorf("retrieving allocations for node drain event: %w", err) } // build job/alloc details for node drain @@ -132,5 +105,5 @@ func NodeDrainEventFromChanges(tx ReadTxn, changes Changes) (structs.Events, err events = append(events, event) } } - return structs.Events{Index: changes.Index, Events: events}, nil + return &structs.Events{Index: changes.Index, Events: events}, nil } diff --git a/nomad/state/node_events_test.go b/nomad/state/node_events_test.go index bb24657b16ce..86874814c364 100644 --- a/nomad/state/node_events_test.go +++ b/nomad/state/node_events_test.go @@ -175,7 +175,7 @@ func TestNodeEventsFromChanges(t *testing.T) { setupTx.Txn.Commit() } - tx := s.db.WriteTxn(100) + tx := s.db.WriteTxnMsgT(tc.MsgType, 100) require.NoError(t, tc.Mutate(s, tx)) changes := Changes{Changes: tx.Changes(), Index: 100, MsgType: tc.MsgType} @@ -186,6 +186,7 @@ func TestNodeEventsFromChanges(t *testing.T) { return } require.NoError(t, err) + require.NotNil(t, got) require.Equal(t, len(tc.WantEvents), len(got.Events)) for idx, g := range got.Events { diff --git a/nomad/state/state_changes.go b/nomad/state/state_changes.go index 6e4903f024db..4e72485157b4 100644 --- a/nomad/state/state_changes.go +++ b/nomad/state/state_changes.go @@ -8,10 +8,6 @@ import ( "github.com/hashicorp/nomad/nomad/structs" ) -const ( - CtxMsgType = "type" -) - // ReadTxn is implemented by memdb.Txn to perform read operations. type ReadTxn interface { Get(table, index string, args ...interface{}) (memdb.ResultIterator, error) @@ -34,10 +30,9 @@ type Changes struct { // sent to the EventPublisher which will create and emit change events. type changeTrackerDB struct { db *memdb.MemDB - durableEvents bool - durableCount int + durableCount int64 publisher *stream.EventPublisher - processChanges func(ReadTxn, Changes) (structs.Events, error) + processChanges func(ReadTxn, Changes) (*structs.Events, error) } // ChangeConfig @@ -54,13 +49,13 @@ func NewChangeTrackerDB(db *memdb.MemDB, publisher *stream.EventPublisher, chang db: db, publisher: publisher, processChanges: changesFn, - durableCount: cfg.DurableEventCount, + durableCount: int64(cfg.DurableEventCount), } } -type changeProcessor func(ReadTxn, Changes) (structs.Events, error) +type changeProcessor func(ReadTxn, Changes) (*structs.Events, error) -func noOpProcessChanges(ReadTxn, Changes) (structs.Events, error) { return structs.Events{}, nil } +func noOpProcessChanges(ReadTxn, Changes) (*structs.Events, error) { return nil, nil } // ReadTxn returns a read-only transaction which behaves exactly the same as // memdb.Txn @@ -92,27 +87,31 @@ func (c *changeTrackerDB) WriteTxn(idx uint64) *txn { } func (c *changeTrackerDB) WriteTxnMsgT(msgType structs.MessageType, idx uint64) *txn { + persistChanges := c.durableCount > 0 + t := &txn{ msgType: msgType, Txn: c.db.Txn(true), Index: idx, publish: c.publish, - persistChanges: c.durableEvents, + persistChanges: persistChanges, } t.Txn.TrackChanges() return t } -func (c *changeTrackerDB) publish(changes Changes) (structs.Events, error) { +func (c *changeTrackerDB) publish(changes Changes) (*structs.Events, error) { readOnlyTx := c.db.Txn(false) defer readOnlyTx.Abort() events, err := c.processChanges(readOnlyTx, changes) if err != nil { - return structs.Events{}, fmt.Errorf("failed generating events from changes: %v", err) + return nil, fmt.Errorf("failed generating events from changes: %v", err) } - c.publisher.Publish(events) + if events != nil { + c.publisher.Publish(events) + } return events, nil } @@ -147,7 +146,7 @@ type txn struct { // Index is stored so that it may be passed along to any subscribers as part // of a change event. Index uint64 - publish func(changes Changes) (structs.Events, error) + publish func(changes Changes) (*structs.Events, error) } // Commit first pushes changes to EventPublisher, then calls Commit on the @@ -171,7 +170,7 @@ func (tx *txn) Commit() error { return err } - if tx.persistChanges { + if tx.persistChanges && events != nil { // persist events after processing changes err := tx.Txn.Insert("events", events) if err != nil { @@ -191,13 +190,13 @@ func (tx *txn) MsgType() structs.MessageType { return tx.msgType } -func processDBChanges(tx ReadTxn, changes Changes) (structs.Events, error) { +func processDBChanges(tx ReadTxn, changes Changes) (*structs.Events, error) { switch changes.MsgType { case structs.IgnoreUnknownTypeFlag: // unknown event type - return structs.Events{}, nil + return nil, nil case structs.NodeRegisterRequestType: - return NodeRegisterEventFromChanges(tx, changes) + return GenericEventsFromChanges(tx, changes) case structs.NodeUpdateStatusRequestType: // TODO(drew) test return GenericEventsFromChanges(tx, changes) @@ -206,7 +205,7 @@ func processDBChanges(tx ReadTxn, changes Changes) (structs.Events, error) { case structs.NodeUpdateDrainRequestType: return NodeDrainEventFromChanges(tx, changes) case structs.UpsertNodeEventsType: - return NodeEventFromChanges(tx, changes) + return GenericEventsFromChanges(tx, changes) case structs.DeploymentStatusUpdateRequestType: return DeploymentEventFromChanges(changes.MsgType, tx, changes) case structs.DeploymentPromoteRequestType: @@ -225,6 +224,21 @@ func processDBChanges(tx ReadTxn, changes Changes) (structs.Events, error) { case structs.AllocUpdateRequestType: // TODO(drew) test return GenericEventsFromChanges(tx, changes) + // case structs.JobDeregisterRequestType: + // TODO(drew) test / handle delete + // return GenericEventsFromChanges(tx, changes) + // case structs.JobBatchDeregisterRequestType: + // TODO(drew) test & handle delete + // return GenericEventsFromChanges(tx, changes) + case structs.AllocUpdateDesiredTransitionRequestType: + // TODO(drew) drain + return GenericEventsFromChanges(tx, changes) + case structs.NodeUpdateEligibilityRequestType: + // TODO(drew) test, drain + return GenericEventsFromChanges(tx, changes) + case structs.BatchNodeUpdateDrainRequestType: + // TODO(drew) test, drain + return GenericEventsFromChanges(tx, changes) } - return structs.Events{}, nil + return nil, nil } diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 4484c0f1bbe3..0a2bf582f91a 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -49,8 +49,11 @@ type StateStoreConfig struct { // EnablePublisher is used to enable or disable the event publisher EnablePublisher bool - // DurableEventCount is the amount of events to persist during the snapshot - // process. + // EventBufferSize configures the amount of events to hold in memory + EventBufferSize int64 + + // DurableEventCount is used to determine if events from transaction changes + // should be saved in go-memdb DurableEventCount int } @@ -96,12 +99,15 @@ func NewStateStore(config *StateStoreConfig) (*StateStore, error) { if config.EnablePublisher { cfg := &ChangeConfig{ - DurableEventCount: 1000, + DurableEventCount: config.DurableEventCount, } + + // Create new event publisher using provided config publisher := stream.NewEventPublisher(ctx, stream.EventPublisherCfg{ EventBufferTTL: 1 * time.Hour, - EventBufferSize: 250, + EventBufferSize: config.EventBufferSize, Logger: config.Logger, + OnEvict: s.eventPublisherEvict, }) s.db = NewChangeTrackerDB(db, publisher, processDBChanges, cfg) } else { @@ -123,6 +129,30 @@ func (s *StateStore) EventPublisher() (*stream.EventPublisher, error) { return s.db.publisher, nil } +// eventPublisherEvict is used as a callback to delete an evicted events +// entry from go-memdb. +func (s *StateStore) eventPublisherEvict(events *structs.Events) { + if err := s.deleteEvent(events); err != nil { + if err == memdb.ErrNotFound { + s.logger.Info("Evicted event was not found in go-memdb table", "event index", events.Index) + } else { + s.logger.Error("Error deleting event from events table", "error", err) + } + } +} + +func (s *StateStore) deleteEvent(events *structs.Events) error { + txn := s.db.db.Txn(true) + defer txn.Abort() + + if err := txn.Delete("events", events); err != nil { + return err + } + + txn.Commit() + return nil +} + // Config returns the state store configuration. func (s *StateStore) Config() *StateStoreConfig { return s.config @@ -948,7 +978,7 @@ func (s *StateStore) updateNodeStatusTxn(txn *txn, nodeID, status string, update } // BatchUpdateNodeDrain is used to update the drain of a node set of nodes -func (s *StateStore) BatchUpdateNodeDrain(index uint64, updatedAt int64, updates map[string]*structs.DrainUpdate, events map[string]*structs.NodeEvent) error { +func (s *StateStore) BatchUpdateNodeDrain(msgType structs.MessageType, index uint64, updatedAt int64, updates map[string]*structs.DrainUpdate, events map[string]*structs.NodeEvent) error { txn := s.db.WriteTxn(index) defer txn.Abort() for node, update := range updates { @@ -1030,9 +1060,9 @@ func (s *StateStore) updateNodeDrainImpl(txn *txn, index uint64, nodeID string, } // UpdateNodeEligibility is used to update the scheduling eligibility of a node -func (s *StateStore) UpdateNodeEligibility(index uint64, nodeID string, eligibility string, updatedAt int64, event *structs.NodeEvent) error { +func (s *StateStore) UpdateNodeEligibility(msgType structs.MessageType, index uint64, nodeID string, eligibility string, updatedAt int64, event *structs.NodeEvent) error { - txn := s.db.WriteTxn(index) + txn := s.db.WriteTxnMsgT(msgType, index) defer txn.Abort() // Lookup the node @@ -2919,8 +2949,7 @@ func (s *StateStore) DeleteEval(index uint64, evals []string, allocs []string) e return fmt.Errorf("setting job status failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // EvalByID is used to lookup an eval by its ID @@ -3291,10 +3320,10 @@ func (s *StateStore) upsertAllocsImpl(index uint64, allocs []*structs.Allocation // UpdateAllocsDesiredTransitions is used to update a set of allocations // desired transitions. -func (s *StateStore) UpdateAllocsDesiredTransitions(index uint64, allocs map[string]*structs.DesiredTransition, +func (s *StateStore) UpdateAllocsDesiredTransitions(msgType structs.MessageType, index uint64, allocs map[string]*structs.DesiredTransition, evals []*structs.Evaluation) error { - txn := s.db.WriteTxn(index) + txn := s.db.WriteTxnMsgT(msgType, index) defer txn.Abort() // Handle each of the updated allocations diff --git a/nomad/state/state_store_events_test.go b/nomad/state/state_store_events_test.go new file mode 100644 index 000000000000..08a5b88b770a --- /dev/null +++ b/nomad/state/state_store_events_test.go @@ -0,0 +1,125 @@ +package state + +import ( + "errors" + "fmt" + "testing" + + "github.com/hashicorp/nomad/helper/testlog" + "github.com/hashicorp/nomad/nomad/mock" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/testutil" + "github.com/stretchr/testify/require" +) + +// TestStateStore_Events_OnEvict tests that events in the state stores +// event publisher and go-memdb are evicted together when the event buffer +// size reaches its max. +func TestStateStore_Events_OnEvict(t *testing.T) { + t.Parallel() + + cfg := &StateStoreConfig{ + Logger: testlog.HCLogger(t), + Region: "global", + EnablePublisher: true, + EventBufferSize: 10, + DurableEventCount: 10, + } + s := TestStateStoreCfg(t, cfg) + + _, err := s.EventPublisher() + require.NoError(t, err) + + // force 3 evictions + for i := 1; i < 13; i++ { + require.NoError(t, + s.UpsertNodeMsgType(structs.NodeRegisterRequestType, uint64(i), mock.Node()), + ) + } + + get := func() []*structs.Events { + var out []*structs.Events + iter, err := s.Events(nil) + require.NoError(t, err) + for { + raw := iter.Next() + if raw == nil { + break + } + e := raw.(*structs.Events) + + out = append(out, e) + } + return out + } + + // event publisher is async so wait for it to prune + testutil.WaitForResult(func() (bool, error) { + out := get() + if len(out) != 10 { + return false, errors.New("Expected event count to be pruned to 10") + } + return true, nil + }, func(err error) { + require.Fail(t, err.Error()) + t.Fatalf("err: %s", err) + }) + + out := get() + require.Equal(t, 3, int(out[0].Index)) + +} + +// TestStateStore_Events_OnEvict_Missing tests behavior when the event publisher +// evicts an event and there is no corresponding go-memdb entry due to durability +// settings +func TestStateStore_Events_OnEvict_Missing(t *testing.T) { + t.Parallel() + + cfg := &StateStoreConfig{ + Logger: testlog.HCLogger(t), + Region: "global", + EnablePublisher: true, + EventBufferSize: 10, + DurableEventCount: 0, + } + s := TestStateStoreCfg(t, cfg) + + _, err := s.EventPublisher() + require.NoError(t, err) + + getEvents := func() []*structs.Events { + var out []*structs.Events + iter, err := s.Events(nil) + require.NoError(t, err) + for { + raw := iter.Next() + if raw == nil { + break + } + e := raw.(*structs.Events) + + out = append(out, e) + } + return out + } + + // Publish 13 events to fill buffer and force 3 evictions + for i := 1; i < 13; i++ { + require.NoError(t, + s.UpsertNodeMsgType(structs.NodeRegisterRequestType, uint64(i), mock.Node()), + ) + } + + // event publisher is async so wait for it to prune + testutil.WaitForResult(func() (bool, error) { + out := getEvents() + if len(out) != 0 { + return false, fmt.Errorf("Expected event count to be %d, got: %d", 0, len(out)) + } + return true, nil + }, func(err error) { + require.Fail(t, err.Error()) + t.Fatalf("err: %s", err) + }) +} diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index a6ed60a44e74..1c0f345f57eb 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -956,7 +956,7 @@ func TestStateStore_BatchUpdateNodeDrain(t *testing.T) { n2.ID: event, } - require.Nil(state.BatchUpdateNodeDrain(1002, 7, update, events)) + require.Nil(state.BatchUpdateNodeDrain(structs.MsgTypeTestSetup, 1002, 7, update, events)) require.True(watchFired(ws)) ws = memdb.NewWatchSet() @@ -1190,7 +1190,7 @@ func TestStateStore_UpdateNodeEligibility(t *testing.T) { Subsystem: structs.NodeEventSubsystemCluster, Timestamp: time.Now(), } - require.Nil(state.UpdateNodeEligibility(1001, node.ID, expectedEligibility, 7, event)) + require.Nil(state.UpdateNodeEligibility(structs.MsgTypeTestSetup, 1001, node.ID, expectedEligibility, 7, event)) require.True(watchFired(ws)) ws = memdb.NewWatchSet() @@ -1216,7 +1216,7 @@ func TestStateStore_UpdateNodeEligibility(t *testing.T) { require.Nil(state.UpdateNodeDrain(1002, node.ID, expectedDrain, false, 7, nil)) // Try to set the node to eligible - err = state.UpdateNodeEligibility(1003, node.ID, structs.NodeSchedulingEligible, 9, nil) + err = state.UpdateNodeEligibility(structs.MsgTypeTestSetup, 1003, node.ID, structs.NodeSchedulingEligible, 9, nil) require.NotNil(err) require.Contains(err.Error(), "while it is draining") } @@ -5203,7 +5203,7 @@ func TestStateStore_UpdateAllocDesiredTransition(t *testing.T) { evals := []*structs.Evaluation{eval} m := map[string]*structs.DesiredTransition{alloc.ID: t1} - require.Nil(state.UpdateAllocsDesiredTransitions(1001, m, evals)) + require.Nil(state.UpdateAllocsDesiredTransitions(structs.MsgTypeTestSetup, 1001, m, evals)) ws := memdb.NewWatchSet() out, err := state.AllocByID(ws, alloc.ID) @@ -5223,7 +5223,7 @@ func TestStateStore_UpdateAllocDesiredTransition(t *testing.T) { require.NotNil(eout) m = map[string]*structs.DesiredTransition{alloc.ID: t2} - require.Nil(state.UpdateAllocsDesiredTransitions(1002, m, evals)) + require.Nil(state.UpdateAllocsDesiredTransitions(structs.MsgTypeTestSetup, 1002, m, evals)) ws = memdb.NewWatchSet() out, err = state.AllocByID(ws, alloc.ID) @@ -5239,7 +5239,7 @@ func TestStateStore_UpdateAllocDesiredTransition(t *testing.T) { // Try with a bogus alloc id m = map[string]*structs.DesiredTransition{uuid.Generate(): t2} - require.Nil(state.UpdateAllocsDesiredTransitions(1003, m, evals)) + require.Nil(state.UpdateAllocsDesiredTransitions(structs.MsgTypeTestSetup, 1003, m, evals)) } func TestStateStore_JobSummary(t *testing.T) { diff --git a/nomad/state/testing.go b/nomad/state/testing.go index 19421cf42312..6aa9039f4d79 100644 --- a/nomad/state/testing.go +++ b/nomad/state/testing.go @@ -26,10 +26,9 @@ func TestStateStore(t testing.T) *StateStore { func TestStateStorePublisher(t testing.T) *StateStoreConfig { return &StateStoreConfig{ - Logger: testlog.HCLogger(t), - Region: "global", - EnablePublisher: true, - DurableEventCount: 100, + Logger: testlog.HCLogger(t), + Region: "global", + EnablePublisher: true, } } func TestStateStoreCfg(t testing.T, cfg *StateStoreConfig) *StateStore { diff --git a/nomad/stream/event_buffer.go b/nomad/stream/event_buffer.go index fda04fa4f355..42c9dc3a4c25 100644 --- a/nomad/stream/event_buffer.go +++ b/nomad/stream/event_buffer.go @@ -10,6 +10,8 @@ import ( "github.com/hashicorp/nomad/nomad/structs" ) +type EvictCallbackFn func(events *structs.Events) + // eventBuffer is a single-writer, multiple-reader, fixed length concurrent // buffer of events that have been published. The buffer is // the head and tail of an atomically updated single-linked list. Atomic @@ -28,8 +30,8 @@ import ( // goroutines or deliver to O(N) separate channels. // // Because eventBuffer is a linked list with atomically updated pointers, readers don't -// have to take a lock and can consume at their own pace. Slow readers can eventually -// append +// have to take a lock and can consume at their own pace. Slow readers will eventually +// be forced to reconnect to the lastest head by being notified via a bufferItem's droppedCh. // // A new buffer is constructed with a sentinel "empty" bufferItem that has a nil // Events array. This enables subscribers to start watching for the next update @@ -39,10 +41,9 @@ import ( // initialized with an empty bufferItem so can not be used to wait for the first // published event. Call newEventBuffer to construct a new buffer. // -// Calls to Append or AppendBuffer that mutate the head must be externally +// Calls to Append or purne that mutate the head must be externally // synchronized. This allows systems that already serialize writes to append -// without lock overhead (e.g. a snapshot goroutine appending thousands of -// events). +// without lock overhead. type eventBuffer struct { size *int64 @@ -51,18 +52,20 @@ type eventBuffer struct { maxSize int64 maxItemTTL time.Duration + onEvict EvictCallbackFn } // newEventBuffer creates an eventBuffer ready for use. -func newEventBuffer(size int64, maxItemTTL time.Duration) *eventBuffer { +func newEventBuffer(size int64, maxItemTTL time.Duration, onEvict EvictCallbackFn) *eventBuffer { zero := int64(0) b := &eventBuffer{ maxSize: size, size: &zero, maxItemTTL: maxItemTTL, + onEvict: onEvict, } - item := newBufferItem(structs.Events{Index: 0, Events: nil}) + item := newBufferItem(&structs.Events{Index: 0, Events: nil}) b.head.Store(item) b.tail.Store(item) @@ -74,8 +77,8 @@ func newEventBuffer(size int64, maxItemTTL time.Duration) *eventBuffer { // watchers. After calling append, the caller must not make any further // mutations to the events as they may have been exposed to subscribers in other // goroutines. Append only supports a single concurrent caller and must be -// externally synchronized with other Append, AppendBuffer or AppendErr calls. -func (b *eventBuffer) Append(events structs.Events) { +// externally synchronized with other Append, or prune calls. +func (b *eventBuffer) Append(events *structs.Events) { b.appendItem(newBufferItem(events)) } @@ -88,11 +91,10 @@ func (b *eventBuffer) appendItem(item *bufferItem) { b.tail.Store(item) // Increment the buffer size - size := atomic.AddInt64(b.size, 1) + atomic.AddInt64(b.size, int64(len(item.Events.Events))) - // Check if we need to advance the head to keep the list - // constrained to max size - if size > b.maxSize { + // Advance Head until we are under allowable size + for atomic.LoadInt64(b.size) > b.maxSize { b.advanceHead() } @@ -101,18 +103,43 @@ func (b *eventBuffer) appendItem(item *bufferItem) { } +func newSentinelItem() *bufferItem { + return newBufferItem(&structs.Events{Index: 0, Events: nil}) +} + // advanceHead drops the current Head buffer item and notifies readers // that the item should be discarded by closing droppedCh. // Slow readers will prevent the old head from being GC'd until they // discard it. func (b *eventBuffer) advanceHead() { old := b.Head() + next := old.link.next.Load() + // if the next item is nil replace it with a sentinel value + if next == nil { + next = newSentinelItem() + } + // notify readers that old is being dropped close(old.link.droppedCh) + + // store the next value to head b.head.Store(next) - atomic.AddInt64(b.size, -1) + // If the old head is equal to the tail + // update the tail value as well + if old == b.Tail() { + b.tail.Store(next) + } + + // update the amount of events we have in the buffer + rmCount := len(old.Events.Events) + atomic.AddInt64(b.size, -int64(rmCount)) + + // Call evict callback if the item isn't a sentinel value + if b.onEvict != nil && old.Events.Index != 0 { + b.onEvict(old.Events) + } } // Head returns the current head of the buffer. It will always exist but it may @@ -137,10 +164,10 @@ func (b *eventBuffer) Tail() *bufferItem { // index as well as the offset between the requested index and returned one. func (b *eventBuffer) StartAtClosest(index uint64) (*bufferItem, int) { item := b.Head() - if index < item.Index { - return item, int(item.Index) - int(index) + if index < item.Events.Index { + return item, int(item.Events.Index) - int(index) } - if item.Index == index { + if item.Events.Index == index { return item, 0 } @@ -148,12 +175,12 @@ func (b *eventBuffer) StartAtClosest(index uint64) (*bufferItem, int) { prev := item item = item.NextNoBlock() if item == nil { - return prev, int(index) - int(prev.Index) + return prev, int(index) - int(prev.Events.Index) } - if index < item.Index { - return item, int(item.Index) - int(index) + if index < item.Events.Index { + return item, int(item.Events.Index) - int(index) } - if index == item.Index { + if index == item.Events.Index { return item, 0 } } @@ -168,13 +195,14 @@ func (b *eventBuffer) Len() int { // is no longer expired. It should be externally synchronized as it mutates // the buffer of items. func (b *eventBuffer) prune() { + now := time.Now() for { head := b.Head() if b.Len() == 0 { return } - if time.Since(head.createdAt) > b.maxItemTTL { + if now.Sub(head.createdAt) > b.maxItemTTL { b.advanceHead() } else { return @@ -202,9 +230,7 @@ type bufferItem struct { // should check and skip nil Events at any point in the buffer. It will also // be nil if the producer appends an Error event because they can't complete // the request to populate the buffer. Err will be non-nil in this case. - Events []structs.Event - - Index uint64 + Events *structs.Events // Err is non-nil if the producer can't complete their task and terminates the // buffer. Subscribers should return the error to clients and cease attempting @@ -239,14 +265,13 @@ type bufferLink struct { // newBufferItem returns a blank buffer item with a link and chan ready to have // the fields set and be appended to a buffer. -func newBufferItem(events structs.Events) *bufferItem { +func newBufferItem(events *structs.Events) *bufferItem { return &bufferItem{ link: &bufferLink{ ch: make(chan struct{}), droppedCh: make(chan struct{}), }, - Events: events.Events, - Index: events.Index, + Events: events, createdAt: time.Now(), } } @@ -258,13 +283,15 @@ func (i *bufferItem) Next(ctx context.Context, forceClose <-chan struct{}) (*buf // state change (chan nil) as that's not threadsafe but detecting close is. select { case <-ctx.Done(): - return nil, fmt.Errorf("waiting for next event: %w", ctx.Err()) + return nil, ctx.Err() case <-forceClose: return nil, fmt.Errorf("subscription closed") case <-i.link.ch: } // Check if the reader is too slow and the event buffer as discarded the event + // This must happen after the above select to prevent a random selection + // between linkCh and droppedCh select { case <-i.link.droppedCh: return nil, fmt.Errorf("event dropped from buffer") @@ -293,16 +320,3 @@ func (i *bufferItem) NextNoBlock() *bufferItem { } return nextRaw.(*bufferItem) } - -// NextLink returns either the next item in the buffer if there is one, or -// an empty item (that will be ignored by subscribers) that has a pointer to -// the same link as this bufferItem (but none of the bufferItem content). -// When the link.ch is closed, subscriptions will be notified of the next item. -func (i *bufferItem) NextLink() *bufferItem { - next := i.NextNoBlock() - if next == nil { - // Return an empty item that can be followed to the next item published. - return &bufferItem{link: i.link} - } - return next -} diff --git a/nomad/stream/event_buffer_test.go b/nomad/stream/event_buffer_test.go index 5b1b6e62333e..84f0b8524733 100644 --- a/nomad/stream/event_buffer_test.go +++ b/nomad/stream/event_buffer_test.go @@ -3,11 +3,12 @@ package stream import ( "context" "fmt" - "github.com/hashicorp/nomad/nomad/structs" "math/rand" "testing" "time" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" ) @@ -16,7 +17,7 @@ func TestEventBufferFuzz(t *testing.T) { nReaders := 1000 nMessages := 1000 - b := newEventBuffer(1000, DefaultTTL) + b := newEventBuffer(1000, DefaultTTL, nil) // Start a write goroutine that will publish 10000 messages with sequential // indexes and some jitter in timing (to allow clients to "catch up" and block @@ -35,7 +36,7 @@ func TestEventBufferFuzz(t *testing.T) { e := structs.Event{ Index: uint64(i), // Indexes should be contiguous } - b.Append(structs.Events{Index: uint64(i), Events: []structs.Event{e}}) + b.Append(&structs.Events{Index: uint64(i), Events: []structs.Event{e}}) // Sleep sometimes for a while to let some subscribers catch up wait := time.Duration(z.Uint64()) * time.Millisecond time.Sleep(wait) @@ -61,9 +62,9 @@ func TestEventBufferFuzz(t *testing.T) { expect, err) return } - if item.Events[0].Index != expect { + if item.Events.Events[0].Index != expect { errCh <- fmt.Errorf("subscriber %05d got bad event want=%d, got=%d", i, - expect, item.Events[0].Index) + expect, item.Events.Events[0].Index) return } expect++ @@ -84,14 +85,13 @@ func TestEventBufferFuzz(t *testing.T) { } func TestEventBuffer_Slow_Reader(t *testing.T) { - - b := newEventBuffer(10, DefaultTTL) + b := newEventBuffer(10, DefaultTTL, nil) for i := 0; i < 10; i++ { e := structs.Event{ Index: uint64(i), // Indexes should be contiguous } - b.Append(structs.Events{uint64(i), []structs.Event{e}}) + b.Append(&structs.Events{Index: uint64(i), Events: []structs.Event{e}}) } head := b.Head() @@ -100,7 +100,7 @@ func TestEventBuffer_Slow_Reader(t *testing.T) { e := structs.Event{ Index: uint64(i), // Indexes should be contiguous } - b.Append(structs.Events{uint64(i), []structs.Event{e}}) + b.Append(&structs.Events{Index: uint64(i), Events: []structs.Event{e}}) } // Ensure the slow reader errors to handle dropped events and @@ -110,17 +110,17 @@ func TestEventBuffer_Slow_Reader(t *testing.T) { require.Nil(t, ev) newHead := b.Head() - require.Equal(t, 4, int(newHead.Index)) + require.Equal(t, 5, int(newHead.Events.Index)) } func TestEventBuffer_Size(t *testing.T) { - b := newEventBuffer(100, DefaultTTL) + b := newEventBuffer(100, DefaultTTL, nil) for i := 0; i < 10; i++ { e := structs.Event{ Index: uint64(i), // Indexes should be contiguous } - b.Append(structs.Events{uint64(i), []structs.Event{e}}) + b.Append(&structs.Events{Index: uint64(i), Events: []structs.Event{e}}) } require.Equal(t, 10, b.Len()) @@ -130,26 +130,46 @@ func TestEventBuffer_Size(t *testing.T) { // are past their TTL, the event buffer should prune down to the last message // and hold onto the last item. func TestEventBuffer_Prune_AllOld(t *testing.T) { - b := newEventBuffer(100, 1*time.Second) + b := newEventBuffer(100, 1*time.Second, nil) for i := 0; i < 10; i++ { e := structs.Event{ Index: uint64(i), // Indexes should be contiguous } - b.Append(structs.Events{uint64(i), []structs.Event{e}}) + b.Append(&structs.Events{Index: uint64(i), Events: []structs.Event{e}}) } require.Equal(t, 10, int(b.Len())) time.Sleep(1 * time.Second) + // prune old messages, which will bring the event buffer down + // to a single sentinel value b.prune() - require.Equal(t, 9, int(b.Head().Index)) + // head and tail are now a sentinel value + head := b.Head() + tail := b.Tail() + require.Equal(t, 0, int(head.Events.Index)) require.Equal(t, 0, b.Len()) + require.Equal(t, head, tail) + + e := structs.Event{ + Index: uint64(100), + } + b.Append(&structs.Events{Index: uint64(100), Events: []structs.Event{e}}) + + ctx, cancel := context.WithDeadline(context.Background(), time.Now().Add(1*time.Second)) + defer cancel() + + next, err := head.Next(ctx, make(chan struct{})) + require.NoError(t, err) + require.NotNil(t, next) + require.Equal(t, uint64(100), next.Events.Index) + } -func TestStartAt_CurrentIdx_Past_Start(t *testing.T) { +func TestEventBuffer_StartAt_CurrentIdx_Past_Start(t *testing.T) { cases := []struct { desc string req uint64 @@ -183,20 +203,43 @@ func TestStartAt_CurrentIdx_Past_Start(t *testing.T) { } // buffer starts at index 11 goes to 100 - b := newEventBuffer(100, 1*time.Hour) + b := newEventBuffer(100, 1*time.Hour, nil) for i := 11; i <= 100; i++ { e := structs.Event{ Index: uint64(i), // Indexes should be contiguous } - b.Append(structs.Events{uint64(i), []structs.Event{e}}) + b.Append(&structs.Events{Index: uint64(i), Events: []structs.Event{e}}) } for _, tc := range cases { t.Run(tc.desc, func(t *testing.T) { got, offset := b.StartAtClosest(tc.req) - require.Equal(t, int(tc.expected), int(got.Index)) + require.Equal(t, int(tc.expected), int(got.Events.Index)) require.Equal(t, tc.offset, offset) }) } } + +func TestEventBuffer_OnEvict(t *testing.T) { + called := make(chan struct{}) + testOnEvict := func(events *structs.Events) { + close(called) + } + b := newEventBuffer(2, DefaultTTL, testOnEvict) + + // start at 1 since new event buffer is built with a starting sentinel value + for i := 1; i < 4; i++ { + e := structs.Event{ + Index: uint64(i), // Indexes should be contiguous + } + b.Append(&structs.Events{Index: uint64(i), Events: []structs.Event{e}}) + } + + select { + case <-called: + // testOnEvict called + case <-time.After(100 * time.Millisecond): + require.Fail(t, "expected testOnEvict to be called") + } +} diff --git a/nomad/stream/event_publisher.go b/nomad/stream/event_publisher.go index 45b8ef8566ad..f28131dddec6 100644 --- a/nomad/stream/event_publisher.go +++ b/nomad/stream/event_publisher.go @@ -18,6 +18,7 @@ type EventPublisherCfg struct { EventBufferSize int64 EventBufferTTL time.Duration Logger hclog.Logger + OnEvict EvictCallbackFn } type EventPublisher struct { @@ -27,10 +28,6 @@ type EventPublisher struct { // eventBuf stores a configurable amount of events in memory eventBuf *eventBuffer - // pruneTick is the duration to periodically prune events from the event - // buffer. Defaults to 5s - pruneTick time.Duration - logger hclog.Logger subscriptions *subscriptions @@ -38,7 +35,7 @@ type EventPublisher struct { // publishCh is used to send messages from an active txn to a goroutine which // publishes events, so that publishing can happen asynchronously from // the Commit call in the FSM hot path. - publishCh chan structs.Events + publishCh chan *structs.Events } type subscriptions struct { @@ -63,19 +60,22 @@ func NewEventPublisher(ctx context.Context, cfg EventPublisherCfg) *EventPublish cfg.Logger = hclog.NewNullLogger() } - buffer := newEventBuffer(cfg.EventBufferSize, cfg.EventBufferTTL) + // Set the event buffer size to a minimum + if cfg.EventBufferSize == 0 { + cfg.EventBufferSize = 100 + } + + buffer := newEventBuffer(cfg.EventBufferSize, cfg.EventBufferTTL, cfg.OnEvict) e := &EventPublisher{ logger: cfg.Logger.Named("event_publisher"), eventBuf: buffer, - publishCh: make(chan structs.Events, 64), + publishCh: make(chan *structs.Events, 64), subscriptions: &subscriptions{ byToken: make(map[string]map[*SubscribeRequest]*Subscription), }, - pruneTick: 5 * time.Second, } go e.handleUpdates(ctx) - go e.periodicPrune(ctx) return e } @@ -85,7 +85,7 @@ func (e *EventPublisher) Len() int { } // Publish events to all subscribers of the event Topic. -func (e *EventPublisher) Publish(events structs.Events) { +func (e *EventPublisher) Publish(events *structs.Events) { if len(events.Events) > 0 { e.publishCh <- events } @@ -104,11 +104,11 @@ func (e *EventPublisher) Subscribe(req *SubscribeRequest) (*Subscription, error) head = e.eventBuf.Head() } if offset > 0 { - e.logger.Warn("requested index no longer in buffer", "requsted", int(req.Index), "closest", int(head.Index)) + e.logger.Warn("requested index no longer in buffer", "requsted", int(req.Index), "closest", int(head.Events.Index)) } // Empty head so that calling Next on sub - start := newBufferItem(structs.Events{Index: req.Index}) + start := newBufferItem(&structs.Events{Index: req.Index}) start.link.next.Store(head) close(start.link.ch) @@ -118,6 +118,10 @@ func (e *EventPublisher) Subscribe(req *SubscribeRequest) (*Subscription, error) return sub, nil } +func (e *EventPublisher) CloseAll() { + e.subscriptions.closeAll() +} + func (e *EventPublisher) handleUpdates(ctx context.Context) { for { select { @@ -130,21 +134,8 @@ func (e *EventPublisher) handleUpdates(ctx context.Context) { } } -func (e *EventPublisher) periodicPrune(ctx context.Context) { - for { - select { - case <-ctx.Done(): - return - case <-time.After(e.pruneTick): - e.lock.Lock() - e.eventBuf.prune() - e.lock.Unlock() - } - } -} - // sendEvents sends the given events to the publishers event buffer. -func (e *EventPublisher) sendEvents(update structs.Events) { +func (e *EventPublisher) sendEvents(update *structs.Events) { e.lock.Lock() defer e.lock.Unlock() diff --git a/nomad/stream/event_publisher_test.go b/nomad/stream/event_publisher_test.go index 1cdd0f3472b0..af92d0595667 100644 --- a/nomad/stream/event_publisher_test.go +++ b/nomad/stream/event_publisher_test.go @@ -2,10 +2,11 @@ package stream import ( "context" - "github.com/hashicorp/nomad/nomad/structs" "testing" "time" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/stretchr/testify/require" ) @@ -32,7 +33,7 @@ func TestEventPublisher_PublishChangesAndSubscribe(t *testing.T) { Key: "sub-key", Payload: "sample payload", }} - publisher.Publish(structs.Events{Index: 1, Events: events}) + publisher.Publish(&structs.Events{Index: 1, Events: events}) // Subscriber should see the published event result := nextResult(t, eventCh) @@ -50,7 +51,7 @@ func TestEventPublisher_PublishChangesAndSubscribe(t *testing.T) { Key: "sub-key", Payload: "sample payload 2", }} - publisher.Publish(structs.Events{Index: 2, Events: events}) + publisher.Publish(&structs.Events{Index: 2, Events: events}) result = nextResult(t, eventCh) require.NoError(t, result.Err) diff --git a/nomad/stream/subscription.go b/nomad/stream/subscription.go index 555be583477e..0d2b8fe39f55 100644 --- a/nomad/stream/subscription.go +++ b/nomad/stream/subscription.go @@ -76,11 +76,11 @@ func (s *Subscription) Next(ctx context.Context) (structs.Events, error) { } s.currentItem = next - events := filter(s.req, next.Events) + events := filter(s.req, next.Events.Events) if len(events) == 0 { continue } - return structs.Events{Index: next.Index, Events: events}, nil + return structs.Events{Index: next.Events.Index, Events: events}, nil } } @@ -96,7 +96,7 @@ func (s *Subscription) NextNoBlock() ([]structs.Event, error) { } s.currentItem = next - events := filter(s.req, next.Events) + events := filter(s.req, next.Events.Events) if len(events) == 0 { continue } From 8711376e6a91f5f6a814face084fafa8addc2eb0 Mon Sep 17 00:00:00 2001 From: Drew Bailey <2614075+drewbailey@users.noreply.github.com> Date: Thu, 8 Oct 2020 10:25:24 -0400 Subject: [PATCH 16/19] handle txn returning error --- nomad/state/state_store.go | 117 +++++++++++++------------------------ 1 file changed, 42 insertions(+), 75 deletions(-) diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 0a2bf582f91a..b50c809308d8 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -463,8 +463,7 @@ func (s *StateStore) UpsertJobSummary(index uint64, jobSummary *structs.JobSumma return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // DeleteJobSummary deletes the job summary with the given ID. This is for @@ -480,8 +479,7 @@ func (s *StateStore) DeleteJobSummary(index uint64, namespace, id string) error if err := txn.Insert("index", &IndexEntry{"job_summary", index}); err != nil { return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // UpsertDeployment is used to insert a new deployment. If cancelPrior is set to @@ -492,8 +490,7 @@ func (s *StateStore) UpsertDeployment(index uint64, deployment *structs.Deployme if err := s.upsertDeploymentImpl(index, deployment, txn); err != nil { return err } - txn.Commit() - return nil + return txn.Commit() } func (s *StateStore) upsertDeploymentImpl(index uint64, deployment *structs.Deployment, txn *txn) error { @@ -705,8 +702,7 @@ func (s *StateStore) DeleteDeployment(index uint64, deploymentIDs []string) erro return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // UpsertScalingEvent is used to insert a new scaling event. @@ -757,8 +753,7 @@ func (s *StateStore) UpsertScalingEvent(index uint64, req *structs.ScalingEventR return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // ScalingEvents returns an iterator over all the job scaling events @@ -817,8 +812,7 @@ func (s *StateStore) UpsertNode(index uint64, node *structs.Node) error { if err != nil { return nil } - txn.Commit() - return nil + return txn.Commit() } func upsertNodeTxn(txn *txn, index uint64, node *structs.Node) error { @@ -894,8 +888,7 @@ func (s *StateStore) DeleteNode(index uint64, nodes []string) error { if err != nil { return nil } - txn.Commit() - return nil + return txn.Commit() } func deleteNodeTxn(txn *txn, index uint64, nodes []string) error { @@ -986,8 +979,7 @@ func (s *StateStore) BatchUpdateNodeDrain(msgType structs.MessageType, index uin return err } } - txn.Commit() - return nil + return txn.Commit() } // UpdateNodeDrain is used to update the drain of a node @@ -1011,8 +1003,7 @@ func (s *StateStore) UpdateNodeDrain(index uint64, nodeID string, if err := s.updateNodeDrainImpl(txn, index, nodeID, drain, markEligible, updatedAt, event); err != nil { return err } - txn.Commit() - return nil + return txn.Commit() } func (s *StateStore) updateNodeDrainImpl(txn *txn, index uint64, nodeID string, @@ -1101,8 +1092,7 @@ func (s *StateStore) UpdateNodeEligibility(msgType structs.MessageType, index ui return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } func (s *StateStore) UpsertNodeEventsMsgType(msgType structs.MessageType, index uint64, nodeEvents map[string][]*structs.NodeEvent) error { @@ -1130,8 +1120,7 @@ func (s *StateStore) UpsertNodeEvents(index uint64, nodeEvents map[string][]*str } } - txn.Commit() - return nil + return txn.Commit() } // upsertNodeEvent upserts a node event for a respective node. It also maintains @@ -1517,8 +1506,7 @@ func (s *StateStore) UpsertJob(index uint64, job *structs.Job) error { if err := s.upsertJobImpl(index, job, false, txn); err != nil { return err } - txn.Commit() - return nil + return txn.Commit() } // UpsertJob is used to register a job or update a job definition @@ -1629,7 +1617,7 @@ func (s *StateStore) DeleteJob(index uint64, namespace, jobID string) error { err := s.DeleteJobTxn(index, namespace, jobID, txn) if err == nil { - txn.Commit() + return txn.Commit() } return err } @@ -2152,8 +2140,7 @@ func (s *StateStore) CSIVolumeRegister(index uint64, volumes []*structs.CSIVolum return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // CSIVolumes returns the unfiltered list of all volumes @@ -2342,8 +2329,7 @@ func (s *StateStore) CSIVolumeClaim(index uint64, namespace, id string, claim *s return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // CSIVolumeDeregister removes the volume from the server @@ -2385,8 +2371,7 @@ func (s *StateStore) CSIVolumeDeregister(index uint64, namespace string, ids []s return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // volSafeToForce checks if the any of the remaining allocations @@ -2597,8 +2582,7 @@ func (s *StateStore) UpsertCSIPlugin(index uint64, plug *structs.CSIPlugin) erro if err := txn.Insert("index", &IndexEntry{"csi_plugins", index}); err != nil { return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // DeleteCSIPlugin deletes the plugin if it's not in use. @@ -2628,8 +2612,7 @@ func (s *StateStore) DeleteCSIPlugin(index uint64, id string) error { if err != nil { return fmt.Errorf("csi_plugins delete error: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // UpsertPeriodicLaunch is used to register a launch or update it. @@ -2660,8 +2643,7 @@ func (s *StateStore) UpsertPeriodicLaunch(index uint64, launch *structs.Periodic return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // DeletePeriodicLaunch is used to delete the periodic launch @@ -2671,7 +2653,7 @@ func (s *StateStore) DeletePeriodicLaunch(index uint64, namespace, jobID string) err := s.DeletePeriodicLaunchTxn(index, namespace, jobID, txn) if err == nil { - txn.Commit() + return txn.Commit() } return err } @@ -2739,7 +2721,7 @@ func (s *StateStore) UpsertEvals(index uint64, evals []*structs.Evaluation) erro err := s.UpsertEvalsTxn(index, evals, txn) if err == nil { - txn.Commit() + return txn.Commit() } return err } @@ -3182,8 +3164,7 @@ func (s *StateStore) UpsertAllocs(index uint64, allocs []*structs.Allocation) er if err := s.upsertAllocsImpl(index, allocs, txn); err != nil { return err } - txn.Commit() - return nil + return txn.Commit() } // UpsertAllocsMsgType is used to evict a set of allocations and allocate new ones at @@ -3344,8 +3325,7 @@ func (s *StateStore) UpdateAllocsDesiredTransitions(msgType structs.MessageType, return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // nestedUpdateAllocDesiredTransition is used to nest an update of an @@ -3654,8 +3634,7 @@ func (s *StateStore) UpsertVaultAccessor(index uint64, accessors []*structs.Vaul return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // DeleteVaultAccessors is used to delete a set of Vault Accessors @@ -3675,8 +3654,7 @@ func (s *StateStore) DeleteVaultAccessors(index uint64, accessors []*structs.Vau return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // VaultAccessor returns the given Vault accessor @@ -3786,8 +3764,7 @@ func (s *StateStore) UpsertSITokenAccessors(index uint64, accessors []*structs.S return errors.Wrap(err, "index update failed") } - txn.Commit() - return nil + return txn.Commit() } // DeleteSITokenAccessors is used to delete a set of Service Identity token accessors. @@ -3808,8 +3785,7 @@ func (s *StateStore) DeleteSITokenAccessors(index uint64, accessors []*structs.S return errors.Wrap(err, "index update failed") } - txn.Commit() - return nil + return txn.Commit() } // SITokenAccessor returns the given Service Identity token accessor. @@ -3964,8 +3940,7 @@ func (s *StateStore) UpdateJobStability(index uint64, namespace, jobID string, j return err } - txn.Commit() - return nil + return txn.Commit() } // updateJobStabilityImpl updates the stability of the given job and version @@ -4436,8 +4411,7 @@ func (s *StateStore) ReconcileJobSummaries(index uint64) error { if err := txn.Insert("index", &IndexEntry{"job_summary", index}); err != nil { return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // setJobStatuses is a helper for calling setJobStatus on multiple jobs by ID. @@ -5125,8 +5099,7 @@ func (s *StateStore) UpsertACLPolicies(index uint64, policies []*structs.ACLPoli return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // DeleteACLPolicies deletes the policies with the given names @@ -5143,8 +5116,7 @@ func (s *StateStore) DeleteACLPolicies(index uint64, names []string) error { if err := txn.Insert("index", &IndexEntry{"acl_policy", index}); err != nil { return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // ACLPolicyByName is used to lookup a policy by name @@ -5232,8 +5204,7 @@ func (s *StateStore) UpsertACLTokens(index uint64, tokens []*structs.ACLToken) e if err := txn.Insert("index", &IndexEntry{"acl_token", index}); err != nil { return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // DeleteACLTokens deletes the tokens with the given accessor ids @@ -5250,8 +5221,7 @@ func (s *StateStore) DeleteACLTokens(index uint64, ids []string) error { if err := txn.Insert("index", &IndexEntry{"acl_token", index}); err != nil { return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // ACLTokenByAccessorID is used to lookup a token by accessor ID @@ -5385,8 +5355,7 @@ func (s *StateStore) BootstrapACLTokens(index, resetIndex uint64, token *structs if err := txn.Insert("index", &IndexEntry{"acl_token_bootstrap", index}); err != nil { return fmt.Errorf("index update failed: %v", err) } - txn.Commit() - return nil + return txn.Commit() } // SchedulerConfig is used to get the current Scheduler configuration. @@ -5415,8 +5384,7 @@ func (s *StateStore) SchedulerSetConfig(index uint64, config *structs.SchedulerC s.schedulerSetConfigTxn(index, tx, config) - tx.Commit() - return nil + return tx.Commit() } func (s *StateStore) ClusterMetadata(ws memdb.WatchSet) (*structs.ClusterMetadata, error) { @@ -5445,8 +5413,7 @@ func (s *StateStore) ClusterSetMetadata(index uint64, meta *structs.ClusterMetad return errors.Wrap(err, "set cluster metadata failed") } - txn.Commit() - return nil + return txn.Commit() } // WithWriteTransaction executes the passed function within a write transaction, @@ -5486,7 +5453,9 @@ func (s *StateStore) SchedulerCASConfig(index, cidx uint64, config *structs.Sche s.schedulerSetConfigTxn(index, tx, config) - tx.Commit() + if err := tx.Commit(); err != nil { + return false, err + } return true, nil } @@ -5543,8 +5512,7 @@ func (s *StateStore) UpsertScalingPolicies(index uint64, scalingPolicies []*stru return err } - txn.Commit() - return nil + return txn.Commit() } // upsertScalingPolicy is used to insert a new scaling policy. @@ -5613,7 +5581,7 @@ func (s *StateStore) DeleteScalingPolicies(index uint64, ids []string) error { err := s.DeleteScalingPoliciesTxn(index, ids, txn) if err == nil { - txn.Commit() + return txn.Commit() } return err @@ -5828,8 +5796,7 @@ func (s *StateStore) UpsertEvents(index uint64, events *structs.Events) error { if err := txn.Insert("events", events); err != nil { return err } - txn.Commit() - return nil + return txn.Commit() } // StateSnapshot is used to provide a point-in-time snapshot From 71028207dbbea50523677431e3bc7261990b3bea Mon Sep 17 00:00:00 2001 From: Drew Bailey <2614075+drewbailey@users.noreply.github.com> Date: Thu, 8 Oct 2020 11:57:21 -0400 Subject: [PATCH 17/19] namespace filtering --- command/agent/event_endpoint_test.go | 51 ++++++++++++++++ nomad/event_endpoint.go | 7 ++- nomad/state/apply_plan_events.go | 73 ----------------------- nomad/state/deployment_events.go | 83 --------------------------- nomad/state/deployment_events_test.go | 2 +- nomad/state/events.go | 57 ++++++++++++++---- nomad/state/node_events.go | 27 --------- nomad/state/state_changes.go | 21 +++---- nomad/stream/subscription.go | 16 +++++- nomad/stream/subscription_test.go | 35 ++++++++--- nomad/structs/structs.go | 1 + 11 files changed, 154 insertions(+), 219 deletions(-) delete mode 100644 nomad/state/apply_plan_events.go delete mode 100644 nomad/state/deployment_events.go diff --git a/command/agent/event_endpoint_test.go b/command/agent/event_endpoint_test.go index 546dd6516dd0..3b95488d193f 100644 --- a/command/agent/event_endpoint_test.go +++ b/command/agent/event_endpoint_test.go @@ -66,6 +66,57 @@ func TestEventStream(t *testing.T) { }) } +func TestEventStream_NamespaceQuery(t *testing.T) { + t.Parallel() + + httpTest(t, nil, func(s *TestAgent) { + ctx, cancel := context.WithCancel(context.Background()) + req, err := http.NewRequestWithContext(ctx, "GET", "/v1/event/stream?namespace=foo", nil) + require.Nil(t, err) + resp := httptest.NewRecorder() + + respErrCh := make(chan error) + go func() { + _, err = s.Server.EventStream(resp, req) + respErrCh <- err + assert.NoError(t, err) + }() + + pub, err := s.Agent.server.State().EventPublisher() + require.NoError(t, err) + + pub.Publish(&structs.Events{Index: 100, Events: []structs.Event{{Namespace: "bar", Payload: testEvent{ID: "123"}}}}) + pub.Publish(&structs.Events{Index: 101, Events: []structs.Event{{Namespace: "foo", Payload: testEvent{ID: "456"}}}}) + + testutil.WaitForResult(func() (bool, error) { + got := resp.Body.String() + want := `"Namespace":"foo"` + bad := `123` + if strings.Contains(got, bad) { + return false, fmt.Errorf("expected non matching namespace to be filtered, got:%v", got) + } + if strings.Contains(got, want) { + return true, nil + } + + return false, fmt.Errorf("missing expected json, got: %v, want: %v", got, want) + }, func(err error) { + cancel() + require.Fail(t, err.Error()) + }) + + // wait for response to close to prevent race between subscription + // shutdown and server shutdown returning subscription closed by server err + cancel() + select { + case err := <-respErrCh: + require.Nil(t, err) + case <-time.After(1 * time.Second): + require.Fail(t, "waiting for request cancellation") + } + }) +} + func TestEventStream_QueryParse(t *testing.T) { t.Parallel() diff --git a/nomad/event_endpoint.go b/nomad/event_endpoint.go index 202fcf99b8b8..3ee249cab2d1 100644 --- a/nomad/event_endpoint.go +++ b/nomad/event_endpoint.go @@ -64,9 +64,10 @@ func (e *Event) stream(conn io.ReadWriteCloser) { authToken = uuid.Generate() } subReq := &stream.SubscribeRequest{ - Token: authToken, - Topics: args.Topics, - Index: uint64(args.Index), + Token: authToken, + Topics: args.Topics, + Index: uint64(args.Index), + Namespace: args.Namespace, } publisher, err := e.srv.State().EventPublisher() if err != nil { diff --git a/nomad/state/apply_plan_events.go b/nomad/state/apply_plan_events.go deleted file mode 100644 index 9bf4b813dc9a..000000000000 --- a/nomad/state/apply_plan_events.go +++ /dev/null @@ -1,73 +0,0 @@ -package state - -import ( - "fmt" - - "github.com/hashicorp/nomad/nomad/structs" -) - -func ApplyPlanResultEventsFromChanges(tx ReadTxn, changes Changes) (*structs.Events, error) { - var events []structs.Event - for _, change := range changes.Changes { - switch change.Table { - case "deployment": - after, ok := change.After.(*structs.Deployment) - if !ok { - return nil, fmt.Errorf("transaction change was not a Deployment") - } - - event := structs.Event{ - Topic: TopicDeployment, - Type: TypeDeploymentUpdate, - Index: changes.Index, - Key: after.ID, - Payload: &DeploymentEvent{ - Deployment: after, - }, - } - events = append(events, event) - case "evals": - after, ok := change.After.(*structs.Evaluation) - if !ok { - return nil, fmt.Errorf("transaction change was not an Evaluation") - } - - event := structs.Event{ - Topic: TopicEval, - Index: changes.Index, - Key: after.ID, - Payload: &EvalEvent{ - Eval: after, - }, - } - - events = append(events, event) - case "allocs": - after, ok := change.After.(*structs.Allocation) - if !ok { - return nil, fmt.Errorf("transaction change was not an Allocation") - } - before := change.Before - var msg string - if before == nil { - msg = TypeAllocCreated - } else { - msg = TypeAllocUpdated - } - - event := structs.Event{ - Topic: TopicAlloc, - Type: msg, - Index: changes.Index, - Key: after.ID, - Payload: &AllocEvent{ - Alloc: after, - }, - } - - events = append(events, event) - } - } - - return &structs.Events{Index: changes.Index, Events: events}, nil -} diff --git a/nomad/state/deployment_events.go b/nomad/state/deployment_events.go deleted file mode 100644 index 01c51992c23e..000000000000 --- a/nomad/state/deployment_events.go +++ /dev/null @@ -1,83 +0,0 @@ -package state - -import ( - "fmt" - - "github.com/hashicorp/nomad/nomad/structs" -) - -func DeploymentEventFromChanges(msgType structs.MessageType, tx ReadTxn, changes Changes) (*structs.Events, error) { - var events []structs.Event - - var eventType string - switch msgType { - case structs.DeploymentStatusUpdateRequestType: - eventType = TypeDeploymentUpdate - case structs.DeploymentPromoteRequestType: - eventType = TypeDeploymentPromotion - case structs.DeploymentAllocHealthRequestType: - eventType = TypeDeploymentAllocHealth - } - - for _, change := range changes.Changes { - switch change.Table { - case "deployment": - after, ok := change.After.(*structs.Deployment) - if !ok { - return nil, fmt.Errorf("transaction change was not a Deployment") - } - - event := structs.Event{ - Topic: TopicDeployment, - Type: eventType, - Index: changes.Index, - Key: after.ID, - FilterKeys: []string{after.JobID}, - Payload: &DeploymentEvent{ - Deployment: after, - }, - } - - events = append(events, event) - case "jobs": - after, ok := change.After.(*structs.Job) - if !ok { - return nil, fmt.Errorf("transaction change was not a Job") - } - - event := structs.Event{ - Topic: TopicJob, - Type: eventType, - Index: changes.Index, - Key: after.ID, - Payload: &JobEvent{ - Job: after, - }, - } - - events = append(events, event) - case "allocs": - // TODO(drew) determine how to handle alloc updates during deployment - case "evals": - after, ok := change.After.(*structs.Evaluation) - if !ok { - return nil, fmt.Errorf("transaction change was not an Evaluation") - } - - event := structs.Event{ - Topic: TopicEval, - Type: eventType, - Index: changes.Index, - Key: after.ID, - FilterKeys: []string{after.DeploymentID, after.JobID}, - Payload: &EvalEvent{ - Eval: after, - }, - } - - events = append(events, event) - } - } - - return &structs.Events{Index: changes.Index, Events: events}, nil -} diff --git a/nomad/state/deployment_events_test.go b/nomad/state/deployment_events_test.go index d85c9d395bd8..bbaeb7996eaa 100644 --- a/nomad/state/deployment_events_test.go +++ b/nomad/state/deployment_events_test.go @@ -125,7 +125,7 @@ func TestDeploymentEventFromChanges_Promotion(t *testing.T) { require.NoError(t, s.UpdateDeploymentPromotion(msgType, 100, req)) events := WaitForEvents(t, s, 100, 1, 1*time.Second) - require.Len(t, events, 2) + require.Len(t, events, 4) got := events[0] require.Equal(t, uint64(100), got.Index) diff --git a/nomad/state/events.go b/nomad/state/events.go index 62060137d538..f396975f67a7 100644 --- a/nomad/state/events.go +++ b/nomad/state/events.go @@ -33,6 +33,7 @@ const ( TypeJobRegistered = "JobRegistered" TypeJobDeregistered = "JobDeregistered" TypeJobBatchDeregistered = "JobBatchDeregistered" + TypePlanResult = "PlanResult" ) type JobEvent struct { @@ -100,6 +101,14 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) (*structs.Events, err eventType = TypeNodeDrain case structs.BatchNodeUpdateDrainRequestType: eventType = TypeNodeDrain + case structs.DeploymentStatusUpdateRequestType: + eventType = TypeDeploymentUpdate + case structs.DeploymentPromoteRequestType: + eventType = TypeDeploymentPromotion + case structs.DeploymentAllocHealthRequestType: + eventType = TypeDeploymentAllocHealth + case structs.ApplyPlanResultsRequestType: + eventType = TypePlanResult default: // unknown request type return nil, nil @@ -118,10 +127,11 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) (*structs.Events, err } event := structs.Event{ - Topic: TopicEval, - Type: eventType, - Index: changes.Index, - Key: after.ID, + Topic: TopicEval, + Type: eventType, + Index: changes.Index, + Key: after.ID, + Namespace: after.Namespace, Payload: &EvalEvent{ Eval: after, }, @@ -143,10 +153,11 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) (*structs.Events, err alloc.Job = nil event := structs.Event{ - Topic: TopicAlloc, - Type: eventType, - Index: changes.Index, - Key: after.ID, + Topic: TopicAlloc, + Type: eventType, + Index: changes.Index, + Key: after.ID, + Namespace: after.Namespace, Payload: &AllocEvent{ Alloc: alloc, }, @@ -163,10 +174,11 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) (*structs.Events, err } event := structs.Event{ - Topic: TopicAlloc, - Type: eventType, - Index: changes.Index, - Key: after.ID, + Topic: TopicAlloc, + Type: eventType, + Index: changes.Index, + Key: after.ID, + Namespace: after.Namespace, Payload: &JobEvent{ Job: after, }, @@ -192,6 +204,27 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) (*structs.Events, err }, } events = append(events, event) + case "deployment": + if change.Deleted() { + return nil, nil + } + after, ok := change.After.(*structs.Deployment) + if !ok { + return nil, fmt.Errorf("transaction change was not a Node") + } + + event := structs.Event{ + Topic: TopicNode, + Type: eventType, + Index: changes.Index, + Key: after.ID, + Namespace: after.Namespace, + FilterKeys: []string{after.JobID}, + Payload: &DeploymentEvent{ + Deployment: after, + }, + } + events = append(events, event) } } diff --git a/nomad/state/node_events.go b/nomad/state/node_events.go index 1c85ec5723e8..4742b1aa5878 100644 --- a/nomad/state/node_events.go +++ b/nomad/state/node_events.go @@ -6,33 +6,6 @@ import ( "github.com/hashicorp/nomad/nomad/structs" ) -// NodeRegisterEventFromChanges generates a NodeRegistrationEvent from a set -// of transaction changes. -func NodeRegisterEventFromChanges(tx ReadTxn, changes Changes) (*structs.Events, error) { - var events []structs.Event - for _, change := range changes.Changes { - switch change.Table { - case "nodes": - after, ok := change.After.(*structs.Node) - if !ok { - return nil, fmt.Errorf("transaction change was not a Node") - } - - event := structs.Event{ - Topic: TopicNode, - Type: TypeNodeRegistration, - Index: changes.Index, - Key: after.ID, - Payload: &NodeEvent{ - Node: after, - }, - } - events = append(events, event) - } - } - return &structs.Events{Index: changes.Index, Events: events}, nil -} - // NodeDeregisterEventFromChanges generates a NodeDeregistrationEvent from a set // of transaction changes. func NodeDeregisterEventFromChanges(tx ReadTxn, changes Changes) (*structs.Events, error) { diff --git a/nomad/state/state_changes.go b/nomad/state/state_changes.go index 4e72485157b4..c94545d30d86 100644 --- a/nomad/state/state_changes.go +++ b/nomad/state/state_changes.go @@ -207,13 +207,14 @@ func processDBChanges(tx ReadTxn, changes Changes) (*structs.Events, error) { case structs.UpsertNodeEventsType: return GenericEventsFromChanges(tx, changes) case structs.DeploymentStatusUpdateRequestType: - return DeploymentEventFromChanges(changes.MsgType, tx, changes) + return GenericEventsFromChanges(tx, changes) case structs.DeploymentPromoteRequestType: - return DeploymentEventFromChanges(changes.MsgType, tx, changes) + return GenericEventsFromChanges(tx, changes) case structs.DeploymentAllocHealthRequestType: - return DeploymentEventFromChanges(changes.MsgType, tx, changes) + return GenericEventsFromChanges(tx, changes) case structs.ApplyPlanResultsRequestType: - return ApplyPlanResultEventsFromChanges(tx, changes) + // TODO test + return GenericEventsFromChanges(tx, changes) case structs.EvalUpdateRequestType: return GenericEventsFromChanges(tx, changes) case structs.AllocClientUpdateRequestType: @@ -224,12 +225,12 @@ func processDBChanges(tx ReadTxn, changes Changes) (*structs.Events, error) { case structs.AllocUpdateRequestType: // TODO(drew) test return GenericEventsFromChanges(tx, changes) - // case structs.JobDeregisterRequestType: - // TODO(drew) test / handle delete - // return GenericEventsFromChanges(tx, changes) - // case structs.JobBatchDeregisterRequestType: - // TODO(drew) test & handle delete - // return GenericEventsFromChanges(tx, changes) + case structs.JobDeregisterRequestType: + // TODO(drew) test / handle delete + return GenericEventsFromChanges(tx, changes) + case structs.JobBatchDeregisterRequestType: + // TODO(drew) test & handle delete + return GenericEventsFromChanges(tx, changes) case structs.AllocUpdateDesiredTransitionRequestType: // TODO(drew) drain return GenericEventsFromChanges(tx, changes) diff --git a/nomad/stream/subscription.go b/nomad/stream/subscription.go index 0d2b8fe39f55..b85838f3d9c3 100644 --- a/nomad/stream/subscription.go +++ b/nomad/stream/subscription.go @@ -46,8 +46,9 @@ type Subscription struct { } type SubscribeRequest struct { - Token string - Index uint64 + Token string + Index uint64 + Namespace string Topics map[structs.Topic][]string } @@ -115,7 +116,7 @@ func (s *Subscription) Unsubscribe() { s.unsub() } -// filter events to only those that match a subscriptions topic/keys +// filter events to only those that match a subscriptions topic/keys/namespace func filter(req *SubscribeRequest, events []structs.Event) []structs.Event { if len(events) == 0 { return events @@ -131,7 +132,12 @@ func filter(req *SubscribeRequest, events []structs.Event) []structs.Event { } else { keys = req.Topics[e.Topic] } + if req.Namespace != "" && e.Namespace != "" && e.Namespace != req.Namespace { + continue + } for _, k := range keys { + // if req.Namespace != "" && e.Namespace != "" && e.Namespace == + // if e.Namespace != "" && e.Namespace if e.Key == k || k == AllKeys { count++ } @@ -158,6 +164,10 @@ func filter(req *SubscribeRequest, events []structs.Event) []structs.Event { } else { keys = req.Topics[e.Topic] } + // filter out non matching namespaces + if req.Namespace != "" && e.Namespace != "" && e.Namespace != req.Namespace { + continue + } for _, k := range keys { if e.Key == k || k == AllKeys { result = append(result, e) diff --git a/nomad/stream/subscription_test.go b/nomad/stream/subscription_test.go index 659a7f1a2538..a83be8809869 100644 --- a/nomad/stream/subscription_test.go +++ b/nomad/stream/subscription_test.go @@ -1,9 +1,10 @@ package stream import ( - "github.com/hashicorp/nomad/nomad/structs" "testing" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/stretchr/testify/require" ) @@ -17,7 +18,7 @@ func TestFilter_AllTopics(t *testing.T) { req := &SubscribeRequest{ Topics: map[structs.Topic][]string{ - "*": []string{"*"}, + "*": {"*"}, }, } actual := filter(req, events) @@ -33,7 +34,7 @@ func TestFilter_AllKeys(t *testing.T) { req := &SubscribeRequest{ Topics: map[structs.Topic][]string{ - "Test": []string{"*"}, + "Test": {"*"}, }, } actual := filter(req, events) @@ -49,7 +50,7 @@ func TestFilter_PartialMatch_Topic(t *testing.T) { req := &SubscribeRequest{ Topics: map[structs.Topic][]string{ - "Test": []string{"*"}, + "Test": {"*"}, }, } actual := filter(req, events) @@ -65,7 +66,7 @@ func TestFilter_PartialMatch_Key(t *testing.T) { req := &SubscribeRequest{ Topics: map[structs.Topic][]string{ - "Test": []string{"One"}, + "Test": {"One"}, }, } actual := filter(req, events) @@ -81,8 +82,8 @@ func TestFilter_NoMatch(t *testing.T) { req := &SubscribeRequest{ Topics: map[structs.Topic][]string{ - "NodeEvents": []string{"*"}, - "Test": []string{"Highly-Specific-Key"}, + "NodeEvents": {"*"}, + "Test": {"Highly-Specific-Key"}, }, } actual := filter(req, events) @@ -91,3 +92,23 @@ func TestFilter_NoMatch(t *testing.T) { require.Equal(t, cap(actual), 0) } + +func TestFilter_Namespace(t *testing.T) { + events := make([]structs.Event, 0, 5) + events = append(events, structs.Event{Topic: "Test", Key: "One", Namespace: "foo"}, structs.Event{Topic: "Test", Key: "Two"}, structs.Event{Topic: "Test", Key: "Two", Namespace: "bar"}) + + req := &SubscribeRequest{ + Topics: map[structs.Topic][]string{ + "*": {"*"}, + }, + Namespace: "foo", + } + actual := filter(req, events) + expected := []structs.Event{ + {Topic: "Test", Key: "One", Namespace: "foo"}, + {Topic: "Test", Key: "Two"}, + } + require.Equal(t, expected, actual) + + require.Equal(t, cap(actual), 2) +} diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 4192a27b0c04..75db4c35cab1 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -10745,6 +10745,7 @@ type Event struct { Topic Topic Type string Key string + Namespace string FilterKeys []string Index uint64 Payload interface{} From 3c15f41411593bddc3d233a98f2393923d9e3ab9 Mon Sep 17 00:00:00 2001 From: Drew Bailey <2614075+drewbailey@users.noreply.github.com> Date: Thu, 8 Oct 2020 14:27:52 -0400 Subject: [PATCH 18/19] filter on additional filter keys, remove switch statement duplication properly wire up durable event count move newline responsibility moves newline creation from NDJson to the http handler, json stream only encodes and sends now ignore snapshot restore if broker is disabled enable dev mode to access event steam without acl use mapping instead of switch use pointers for config sizes, remove unused ttl, simplify closed conn logic --- api/event.go | 4 +- api/event_test.go | 60 +- command/agent/agent.go | 11 +- command/agent/agent_test.go | 3 +- command/agent/config.go | 28 +- command/agent/config_parse_test.go | 6 +- command/agent/config_test.go | 47 +- command/agent/event_endpoint.go | 45 +- command/agent/event_endpoint_test.go | 14 +- command/agent/testdata/basic.hcl | 4 +- command/agent/testdata/basic.json | 4 +- nomad/config.go | 12 +- nomad/event_endpoint.go | 99 +-- nomad/event_endpoint_test.go | 143 ++++- nomad/fsm.go | 48 +- nomad/fsm_test.go | 20 +- nomad/node_endpoint_test.go | 2 +- nomad/server.go | 15 +- nomad/state/deployment_events_test.go | 89 +-- nomad/state/events.go | 111 ++-- nomad/state/events_test.go | 567 ++++++++++++++++++ nomad/state/node_events.go | 4 +- nomad/state/node_events_test.go | 34 +- nomad/state/state_changes.go | 77 +-- nomad/state/state_store.go | 56 +- nomad/state/state_store_events_test.go | 4 +- nomad/state/state_store_test.go | 36 +- nomad/stream/event_buffer.go | 37 +- nomad/stream/event_buffer_test.go | 28 +- nomad/stream/event_publisher.go | 132 ++-- nomad/stream/event_publisher_test.go | 37 +- nomad/stream/ndjson.go | 93 ++- nomad/stream/ndjson_test.go | 37 +- nomad/stream/subscription.go | 36 +- nomad/stream/subscription_test.go | 23 +- nomad/structs/structs.go | 51 +- nomad/testing.go | 2 +- .../github.com/hashicorp/nomad/api/event.go | 10 +- 38 files changed, 1369 insertions(+), 660 deletions(-) create mode 100644 nomad/state/events_test.go diff --git a/api/event.go b/api/event.go index 2bf08fba5793..f89e222848f7 100644 --- a/api/event.go +++ b/api/event.go @@ -47,11 +47,11 @@ func (c *Client) EventStream() *EventStream { // Stream establishes a new subscription to Nomad's event stream and streams // results back to the returned channel. func (e *EventStream) Stream(ctx context.Context, topics map[Topic][]string, index uint64, q *QueryOptions) (<-chan *Events, error) { - r, err := e.client.newRequest("GET", "/v1/event/stream") if err != nil { return nil, err } + q = q.WithContext(ctx) r.setQueryOptions(q) // Build topic query params @@ -82,7 +82,7 @@ func (e *EventStream) Stream(ctx context.Context, topics map[Topic][]string, ind // select eventsCh events = Events{Err: err} } - if events.IsHeartbeat() { + if events.Err == nil && events.IsHeartbeat() { continue } diff --git a/api/event_test.go b/api/event_test.go index 9bd4ae3a4dc1..80820e97a803 100644 --- a/api/event_test.go +++ b/api/event_test.go @@ -34,20 +34,15 @@ func TestEvent_Stream(t *testing.T) { streamCh, err := events.Stream(ctx, topics, 0, q) require.NoError(t, err) -OUTER: - for { - select { - case event := <-streamCh: - if event.Err != nil { - require.Fail(t, err.Error()) - } - require.Equal(t, len(event.Events), 1) - require.Equal(t, "Eval", string(event.Events[0].Topic)) - - break OUTER - case <-time.After(5 * time.Second): - require.Fail(t, "failed waiting for event stream event") + select { + case event := <-streamCh: + if event.Err != nil { + require.Fail(t, err.Error()) } + require.Equal(t, len(event.Events), 1) + require.Equal(t, "Eval", string(event.Events[0].Topic)) + case <-time.After(5 * time.Second): + require.Fail(t, "failed waiting for event stream event") } } @@ -76,4 +71,43 @@ func TestEvent_Stream_Err_InvalidQueryParam(t *testing.T) { _, err = events.Stream(ctx, topics, 0, q) require.Error(t, err) + require.Contains(t, err.Error(), "400") + require.Contains(t, err.Error(), "Invalid key value pair") +} + +func TestEvent_Stream_CloseCtx(t *testing.T) { + t.Parallel() + + c, s := makeClient(t, nil, nil) + defer s.Stop() + + // register job to generate events + jobs := c.Jobs() + job := testJob() + resp2, _, err := jobs.Register(job, nil) + require.Nil(t, err) + require.NotNil(t, resp2) + + // build event stream request + events := c.EventStream() + q := &QueryOptions{} + topics := map[Topic][]string{ + "Eval": {"*"}, + } + + ctx, cancel := context.WithCancel(context.Background()) + + streamCh, err := events.Stream(ctx, topics, 0, q) + require.NoError(t, err) + + // cancel the request + cancel() + + select { + case event, ok := <-streamCh: + require.False(t, ok) + require.Nil(t, event) + case <-time.After(5 * time.Second): + require.Fail(t, "failed waiting for event stream event") + } } diff --git a/command/agent/agent.go b/command/agent/agent.go index 54de24470f65..e9d9f882bd57 100644 --- a/command/agent/agent.go +++ b/command/agent/agent.go @@ -243,11 +243,14 @@ func convertServerConfig(agentConfig *Config) (*nomad.Config, error) { if agentConfig.Server.UpgradeVersion != "" { conf.UpgradeVersion = agentConfig.Server.UpgradeVersion } - if agentConfig.Server.EnableEventPublisher != nil { - conf.EnableEventPublisher = *agentConfig.Server.EnableEventPublisher + if agentConfig.Server.EnableEventBroker != nil { + conf.EnableEventBroker = *agentConfig.Server.EnableEventBroker } - if agentConfig.Server.EventBufferSize > 0 { - conf.EventBufferSize = int64(agentConfig.Server.EventBufferSize) + if agentConfig.Server.EventBufferSize != nil { + conf.EventBufferSize = int64(*agentConfig.Server.EventBufferSize) + } + if agentConfig.Server.DurableEventCount != nil { + conf.DurableEventCount = int64(*agentConfig.Server.DurableEventCount) } if agentConfig.Autopilot != nil { if agentConfig.Autopilot.CleanupDeadServers != nil { diff --git a/command/agent/agent_test.go b/command/agent/agent_test.go index 1ac900444656..5c71a70b3c61 100644 --- a/command/agent/agent_test.go +++ b/command/agent/agent_test.go @@ -57,7 +57,8 @@ func TestAgent_ServerConfig(t *testing.T) { out, err := a.serverConfig() require.NoError(t, err) - require.True(t, out.EnableEventPublisher) + require.True(t, out.EnableEventBroker) + require.Equal(t, int64(100), out.DurableEventCount) serfAddr := out.SerfConfig.MemberlistConfig.AdvertiseAddr require.Equal(t, "127.0.0.1", serfAddr) diff --git a/command/agent/config.go b/command/agent/config.go index 9665fcaa9ade..f9690efffab2 100644 --- a/command/agent/config.go +++ b/command/agent/config.go @@ -484,18 +484,18 @@ type ServerConfig struct { // This value is ignored. DefaultSchedulerConfig *structs.SchedulerConfiguration `hcl:"default_scheduler_config"` - // EnableEventPublisher configures whether this server's state store + // EnableEventBroker configures whether this server's state store // will generate events for its event stream. - EnableEventPublisher *bool `hcl:"enable_event_publisher"` + EnableEventBroker *bool `hcl:"enable_event_broker"` // EventBufferSize configure the amount of events to be held in memory. - // If EnableEventPublisher is set to true, the minimum allowable value + // If EnableEventBroker is set to true, the minimum allowable value // for the EventBufferSize is 1. - EventBufferSize int `hcl:"event_buffer_size"` + EventBufferSize *int `hcl:"event_buffer_size"` // DurableEventCount specifies the amount of events to persist during snapshot generation. // A count of 0 signals that no events should be persisted. - DurableEventCount int `hcl:"durable_event_count"` + DurableEventCount *int `hcl:"durable_event_count"` // ExtraKeysHCL is used by hcl to surface unexpected keys ExtraKeysHCL []string `hcl:",unusedKeys" json:"-"` @@ -887,11 +887,11 @@ func DefaultConfig() *Config { BindWildcardDefaultHostNetwork: true, }, Server: &ServerConfig{ - Enabled: false, - EnableEventPublisher: helper.BoolToPtr(true), - EventBufferSize: 100, - DurableEventCount: 100, - StartJoin: []string{}, + Enabled: false, + EnableEventBroker: helper.BoolToPtr(true), + EventBufferSize: helper.IntToPtr(100), + DurableEventCount: helper.IntToPtr(100), + StartJoin: []string{}, ServerJoin: &ServerJoin{ RetryJoin: []string{}, RetryInterval: 30 * time.Second, @@ -1415,15 +1415,15 @@ func (a *ServerConfig) Merge(b *ServerConfig) *ServerConfig { result.ServerJoin = result.ServerJoin.Merge(b.ServerJoin) } - if b.EnableEventPublisher != nil { - result.EnableEventPublisher = b.EnableEventPublisher + if b.EnableEventBroker != nil { + result.EnableEventBroker = b.EnableEventBroker } - if b.EventBufferSize != 0 { + if b.EventBufferSize != nil { result.EventBufferSize = b.EventBufferSize } - if b.DurableEventCount != 0 { + if b.DurableEventCount != nil { result.DurableEventCount = b.DurableEventCount } diff --git a/command/agent/config_parse_test.go b/command/agent/config_parse_test.go index 8999fa281208..771ff19731fd 100644 --- a/command/agent/config_parse_test.go +++ b/command/agent/config_parse_test.go @@ -122,9 +122,9 @@ var basicConfig = &Config{ RedundancyZone: "foo", UpgradeVersion: "0.8.0", EncryptKey: "abc", - EnableEventPublisher: helper.BoolToPtr(false), - EventBufferSize: 200, - DurableEventCount: 100, + EnableEventBroker: helper.BoolToPtr(false), + EventBufferSize: helper.IntToPtr(200), + DurableEventCount: helper.IntToPtr(0), ServerJoin: &ServerJoin{ RetryJoin: []string{"1.1.1.1", "2.2.2.2"}, RetryInterval: time.Duration(15) * time.Second, diff --git a/command/agent/config_test.go b/command/agent/config_test.go index 59aa42e0572a..4e0aa934609e 100644 --- a/command/agent/config_test.go +++ b/command/agent/config_test.go @@ -138,7 +138,9 @@ func TestConfig_Merge(t *testing.T) { MaxHeartbeatsPerSecond: 30.0, RedundancyZone: "foo", UpgradeVersion: "foo", - EnableEventPublisher: helper.BoolToPtr(false), + EnableEventBroker: helper.BoolToPtr(false), + EventBufferSize: helper.IntToPtr(0), + DurableEventCount: helper.IntToPtr(0), }, ACL: &ACLConfig{ Enabled: true, @@ -329,7 +331,9 @@ func TestConfig_Merge(t *testing.T) { NonVotingServer: true, RedundancyZone: "bar", UpgradeVersion: "bar", - EnableEventPublisher: helper.BoolToPtr(true), + EnableEventBroker: helper.BoolToPtr(true), + DurableEventCount: helper.IntToPtr(100), + EventBufferSize: helper.IntToPtr(100), }, ACL: &ACLConfig{ Enabled: true, @@ -1166,40 +1170,57 @@ func TestTelemetry_Parse(t *testing.T) { require.True(config.Telemetry.DisableDispatchedJobSummaryMetrics) } -func TestEventPublisher_Parse(t *testing.T) { +func TestEventBroker_Parse(t *testing.T) { require := require.New(t) - { a := &ServerConfig{ - EnableEventPublisher: helper.BoolToPtr(false), + EnableEventBroker: helper.BoolToPtr(false), + EventBufferSize: helper.IntToPtr(0), + DurableEventCount: helper.IntToPtr(0), } b := DefaultConfig().Server - b.EnableEventPublisher = nil + b.EnableEventBroker = nil + b.EventBufferSize = nil + b.DurableEventCount = nil result := a.Merge(b) - require.Equal(false, *result.EnableEventPublisher) + require.Equal(false, *result.EnableEventBroker) + require.Equal(0, *result.EventBufferSize) + require.Equal(0, *result.DurableEventCount) } { a := &ServerConfig{ - EnableEventPublisher: helper.BoolToPtr(true), + EnableEventBroker: helper.BoolToPtr(true), + EventBufferSize: helper.IntToPtr(5000), + DurableEventCount: helper.IntToPtr(200), } b := DefaultConfig().Server - b.EnableEventPublisher = nil + b.EnableEventBroker = nil + b.EventBufferSize = nil + b.DurableEventCount = nil result := a.Merge(b) - require.Equal(true, *result.EnableEventPublisher) + require.Equal(true, *result.EnableEventBroker) + require.Equal(5000, *result.EventBufferSize) + require.Equal(200, *result.DurableEventCount) } { a := &ServerConfig{ - EnableEventPublisher: helper.BoolToPtr(false), + EnableEventBroker: helper.BoolToPtr(false), + EventBufferSize: helper.IntToPtr(0), + DurableEventCount: helper.IntToPtr(0), } b := DefaultConfig().Server - b.EnableEventPublisher = helper.BoolToPtr(true) + b.EnableEventBroker = helper.BoolToPtr(true) + b.EventBufferSize = helper.IntToPtr(20000) + b.DurableEventCount = helper.IntToPtr(1000) result := a.Merge(b) - require.Equal(true, *result.EnableEventPublisher) + require.Equal(true, *result.EnableEventBroker) + require.Equal(20000, *result.EventBufferSize) + require.Equal(1000, *result.DurableEventCount) } } diff --git a/command/agent/event_endpoint.go b/command/agent/event_endpoint.go index af46620acf5c..d28ca84e772d 100644 --- a/command/agent/event_endpoint.go +++ b/command/agent/event_endpoint.go @@ -14,6 +14,7 @@ import ( "github.com/docker/docker/pkg/ioutils" "github.com/hashicorp/go-msgpack/codec" "github.com/hashicorp/nomad/nomad/structs" + "golang.org/x/sync/errgroup" ) func (s *HTTPServer) EventStream(resp http.ResponseWriter, req *http.Request) (interface{}, error) { @@ -40,12 +41,12 @@ func (s *HTTPServer) EventStream(resp http.ResponseWriter, req *http.Request) (i resp.Header().Set("Content-Type", "application/json") resp.Header().Set("Cache-Control", "no-cache") + // Set region, namespace and authtoken to args s.parse(resp, req, &args.QueryOptions.Region, &args.QueryOptions) - // Make the RPC + // Determine the RPC handler to use to find a server var handler structs.StreamingRpcHandler var handlerErr error - if server := s.agent.Server(); server != nil { handler, handlerErr = server.StreamingRpcHandler("Event.Stream") } else if client := s.agent.Client(); client != nil { @@ -73,57 +74,51 @@ func (s *HTTPServer) EventStream(resp http.ResponseWriter, req *http.Request) (i // Create an output that gets flushed on every write output := ioutils.NewWriteFlusher(resp) - // create an error channel to handle errors - errCh := make(chan HTTPCodedError, 1) - - go func() { + // send request and decode events + errs, errCtx := errgroup.WithContext(ctx) + errs.Go(func() error { defer cancel() // Send the request if err := encoder.Encode(args); err != nil { - errCh <- CodedError(500, err.Error()) - return + return CodedError(500, err.Error()) } for { select { - case <-ctx.Done(): - errCh <- nil - return + case <-errCtx.Done(): + return nil default: } // Decode the response var res structs.EventStreamWrapper if err := decoder.Decode(&res); err != nil { - if err == io.EOF || err == io.ErrClosedPipe { - return - } - errCh <- CodedError(500, err.Error()) - return + return CodedError(500, err.Error()) } decoder.Reset(httpPipe) if err := res.Error; err != nil { if err.Code != nil { - errCh <- CodedError(int(*err.Code), err.Error()) - return + return CodedError(int(*err.Code), err.Error()) } } // Flush json entry to response if _, err := io.Copy(output, bytes.NewReader(res.Event.Data)); err != nil { - errCh <- CodedError(500, err.Error()) - return + return CodedError(500, err.Error()) } + // Each entry is its own new line according to ndjson.org + // append new line to each entry + fmt.Fprint(output, "\n") } - }() + }) // invoke handler handler(handlerPipe) cancel() - codedErr := <-errCh + codedErr := errs.Wait() if codedErr != nil && strings.Contains(codedErr.Error(), io.ErrClosedPipe.Error()) { codedErr = nil } @@ -144,11 +139,7 @@ func parseEventTopics(query url.Values) (map[structs.Topic][]string, error) { return nil, fmt.Errorf("error parsing topics: %w", err) } - if topics[structs.Topic(k)] == nil { - topics[structs.Topic(k)] = []string{v} - } else { - topics[structs.Topic(k)] = append(topics[structs.Topic(k)], v) - } + topics[structs.Topic(k)] = append(topics[structs.Topic(k)], v) } return topics, nil } diff --git a/command/agent/event_endpoint_test.go b/command/agent/event_endpoint_test.go index 3b95488d193f..c450a9179880 100644 --- a/command/agent/event_endpoint_test.go +++ b/command/agent/event_endpoint_test.go @@ -10,6 +10,7 @@ import ( "testing" "time" + "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/nomad/testutil" @@ -37,7 +38,7 @@ func TestEventStream(t *testing.T) { assert.NoError(t, err) }() - pub, err := s.Agent.server.State().EventPublisher() + pub, err := s.Agent.server.State().EventBroker() require.NoError(t, err) pub.Publish(&structs.Events{Index: 100, Events: []structs.Event{{Payload: testEvent{ID: "123"}}}}) @@ -71,6 +72,8 @@ func TestEventStream_NamespaceQuery(t *testing.T) { httpTest(t, nil, func(s *TestAgent) { ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + req, err := http.NewRequestWithContext(ctx, "GET", "/v1/event/stream?namespace=foo", nil) require.Nil(t, err) resp := httptest.NewRecorder() @@ -82,17 +85,17 @@ func TestEventStream_NamespaceQuery(t *testing.T) { assert.NoError(t, err) }() - pub, err := s.Agent.server.State().EventPublisher() + pub, err := s.Agent.server.State().EventBroker() require.NoError(t, err) - pub.Publish(&structs.Events{Index: 100, Events: []structs.Event{{Namespace: "bar", Payload: testEvent{ID: "123"}}}}) + badID := uuid.Generate() + pub.Publish(&structs.Events{Index: 100, Events: []structs.Event{{Namespace: "bar", Payload: testEvent{ID: badID}}}}) pub.Publish(&structs.Events{Index: 101, Events: []structs.Event{{Namespace: "foo", Payload: testEvent{ID: "456"}}}}) testutil.WaitForResult(func() (bool, error) { got := resp.Body.String() want := `"Namespace":"foo"` - bad := `123` - if strings.Contains(got, bad) { + if strings.Contains(got, badID) { return false, fmt.Errorf("expected non matching namespace to be filtered, got:%v", got) } if strings.Contains(got, want) { @@ -101,7 +104,6 @@ func TestEventStream_NamespaceQuery(t *testing.T) { return false, fmt.Errorf("missing expected json, got: %v, want: %v", got, want) }, func(err error) { - cancel() require.Fail(t, err.Error()) }) diff --git a/command/agent/testdata/basic.hcl b/command/agent/testdata/basic.hcl index be35093e9802..66d9f9d0f103 100644 --- a/command/agent/testdata/basic.hcl +++ b/command/agent/testdata/basic.hcl @@ -130,9 +130,9 @@ server { upgrade_version = "0.8.0" encrypt = "abc" raft_multiplier = 4 - enable_event_publisher = false + enable_event_broker = false event_buffer_size = 200 - durable_event_count = 100 + durable_event_count = 0 server_join { retry_join = ["1.1.1.1", "2.2.2.2"] diff --git a/command/agent/testdata/basic.json b/command/agent/testdata/basic.json index d9db2d5d0f19..c20889fc6359 100644 --- a/command/agent/testdata/basic.json +++ b/command/agent/testdata/basic.json @@ -261,9 +261,9 @@ "data_dir": "/tmp/data", "deployment_gc_threshold": "12h", "enabled": true, - "enable_event_publisher": false, + "enable_event_broker": false, "event_buffer_size": 200, - "durable_event_count": 100, + "durable_event_count": 0, "enabled_schedulers": [ "test" ], diff --git a/nomad/config.go b/nomad/config.go index 0a2272be8848..93abdc6f1329 100644 --- a/nomad/config.go +++ b/nomad/config.go @@ -78,13 +78,17 @@ type Config struct { // in the absence of ACLs EnableDebug bool - // EnableEventPublisher is used to enable or disable state store + // EnableEventBroker is used to enable or disable state store // event publishing - EnableEventPublisher bool + EnableEventBroker bool // EventBufferSize is the amount of events to hold in memory. EventBufferSize int64 + // DurableEventCount is the amount of events to save to disk when + // snapshotting + DurableEventCount int64 + // LogOutput is the location to write logs to. If this is not set, // logs will go to stderr. LogOutput io.Writer @@ -420,7 +424,9 @@ func DefaultConfig() *Config { ReplicationBackoff: 30 * time.Second, SentinelGCInterval: 30 * time.Second, LicenseConfig: &LicenseConfig{}, - EnableEventPublisher: true, + EnableEventBroker: true, + EventBufferSize: 100, + DurableEventCount: 100, AutopilotConfig: &structs.AutopilotConfig{ CleanupDeadServers: true, LastContactThreshold: 200 * time.Millisecond, diff --git a/nomad/event_endpoint.go b/nomad/event_endpoint.go index 3ee249cab2d1..3b980981496a 100644 --- a/nomad/event_endpoint.go +++ b/nomad/event_endpoint.go @@ -2,12 +2,14 @@ package nomad import ( "context" + "fmt" "io" + "io/ioutil" "time" "github.com/hashicorp/go-msgpack/codec" + "github.com/hashicorp/nomad/acl" "github.com/hashicorp/nomad/helper" - "github.com/hashicorp/nomad/helper/uuid" "github.com/hashicorp/nomad/nomad/stream" "github.com/hashicorp/nomad/nomad/structs" ) @@ -41,35 +43,29 @@ func (e *Event) stream(conn io.ReadWriteCloser) { return } - // ACL check - // TODO(drew) ACL checks need to be per topic - // All Events Management - // System Events Management - // Node Events NamespaceCapabilityReadEvents - // Job/Alloc Events NamespaceCapabilityReadEvents - if aclObj, err := e.srv.ResolveToken(args.AuthToken); err != nil { + aclObj, err := e.srv.ResolveToken(args.AuthToken) + if err != nil { handleJsonResultError(err, nil, encoder) return - } else if aclObj != nil && !aclObj.IsManagement() { - handleJsonResultError(structs.ErrPermissionDenied, helper.Int64ToPtr(403), encoder) - return } - // authToken is passed to the subscribe request so the event stream - // can handle closing a subscription if the authToken expires. - // If ACLs are disabled, a random token is generated and it will - // never be closed due to expiry. - authToken := args.AuthToken - if authToken == "" { - authToken = uuid.Generate() - } subReq := &stream.SubscribeRequest{ - Token: authToken, + Token: args.AuthToken, Topics: args.Topics, Index: uint64(args.Index), Namespace: args.Namespace, } - publisher, err := e.srv.State().EventPublisher() + + // Check required ACL permissions for requested Topics + if aclObj != nil { + if err := aclCheckForEvents(subReq, aclObj); err != nil { + handleJsonResultError(structs.ErrPermissionDenied, helper.Int64ToPtr(403), encoder) + return + } + } + + // Get the servers broker and subscribe + publisher, err := e.srv.State().EventBroker() if err != nil { handleJsonResultError(err, helper.Int64ToPtr(500), encoder) return @@ -86,23 +82,14 @@ func (e *Event) stream(conn io.ReadWriteCloser) { } defer subscription.Unsubscribe() - ndJsonCh := make(chan *structs.NDJson) errCh := make(chan error) - jsonStream := stream.NewNDJsonStream(ndJsonCh, 30*time.Second) - jsonStream.Run(ctx) + jsonStream := stream.NewJsonStream(ctx, 30*time.Second) // goroutine to detect remote side closing go func() { - if _, err := conn.Read(nil); err != nil { - // One end of the pipe explicitly closed, exit - cancel() - return - } - select { - case <-ctx.Done(): - return - } + io.Copy(ioutil.Discard, conn) + cancel() }() go func() { @@ -140,7 +127,7 @@ OUTER: break OUTER case <-ctx.Done(): break OUTER - case eventJSON, ok := <-ndJsonCh: + case eventJSON, ok := <-jsonStream.OutCh(): // check if ndjson may have been closed when an error occurred, // check once more for an error. if !ok { @@ -209,3 +196,47 @@ func handleJsonResultError(err error, code *int64, encoder *codec.Encoder) { Error: structs.NewRpcError(err, code), }) } + +func aclCheckForEvents(subReq *stream.SubscribeRequest, aclObj *acl.ACL) error { + if len(subReq.Topics) == 0 { + return fmt.Errorf("invalid topic request") + } + + reqPolicies := make(map[string]struct{}) + var required = struct{}{} + + for topic := range subReq.Topics { + switch topic { + case structs.TopicDeployment, structs.TopicEval, + structs.TopicAlloc, structs.TopicJob: + if _, ok := reqPolicies[acl.NamespaceCapabilityReadJob]; !ok { + reqPolicies[acl.NamespaceCapabilityReadJob] = required + } + case structs.TopicNode: + reqPolicies["node-read"] = required + case structs.TopicAll: + reqPolicies["management"] = required + default: + return fmt.Errorf("unknown topic %s", topic) + } + } + + for checks := range reqPolicies { + switch checks { + case acl.NamespaceCapabilityReadJob: + if ok := aclObj.AllowNsOp(subReq.Namespace, acl.NamespaceCapabilityReadJob); !ok { + return structs.ErrPermissionDenied + } + case "node-read": + if ok := aclObj.AllowNodeRead(); !ok { + return structs.ErrPermissionDenied + } + case "management": + if ok := aclObj.IsManagement(); !ok { + return structs.ErrPermissionDenied + } + } + } + + return nil +} diff --git a/nomad/event_endpoint_test.go b/nomad/event_endpoint_test.go index 573084ff3139..26757d37ca3e 100644 --- a/nomad/event_endpoint_test.go +++ b/nomad/event_endpoint_test.go @@ -23,7 +23,7 @@ func TestEventStream(t *testing.T) { t.Parallel() s1, cleanupS1 := TestServer(t, func(c *Config) { - c.EnableEventPublisher = true + c.EnableEventBroker = true }) defer cleanupS1() @@ -65,7 +65,7 @@ func TestEventStream(t *testing.T) { }() // retrieve publisher for server, send event - publisher, err := s1.State().EventPublisher() + publisher, err := s1.State().EventBroker() require.NoError(t, err) node := mock.Node() @@ -75,7 +75,12 @@ func TestEventStream(t *testing.T) { encoder := codec.NewEncoder(p1, structs.MsgpackHandle) require.Nil(t, encoder.Encode(req)) + publisher.Publish(&structs.Events{Index: uint64(2), Events: []structs.Event{{Topic: "test", Payload: node}}}) + publisher.Publish(&structs.Events{Index: uint64(3), Events: []structs.Event{{Topic: "test", Payload: node}}}) + timeout := time.After(3 * time.Second) + got := 0 + want := 3 OUTER: for { select { @@ -89,7 +94,7 @@ OUTER: } // ignore heartbeat - if msg.Event == stream.NDJsonHeartbeat { + if msg.Event == stream.JsonHeartbeat { continue } @@ -107,7 +112,11 @@ OUTER: dec.Decode(event.Events[0].Payload) require.NoError(t, err) require.Equal(t, node.ID, out.ID) - break OUTER + + got++ + if got == want { + break OUTER + } } } } @@ -118,7 +127,7 @@ func TestEventStream_StreamErr(t *testing.T) { t.Parallel() s1, cleanupS1 := TestServer(t, func(c *Config) { - c.EnableEventPublisher = true + c.EnableEventBroker = true }) defer cleanupS1() @@ -158,7 +167,7 @@ func TestEventStream_StreamErr(t *testing.T) { } }() - publisher, err := s1.State().EventPublisher() + publisher, err := s1.State().EventBroker() require.NoError(t, err) node := mock.Node() @@ -200,12 +209,12 @@ func TestEventStream_RegionForward(t *testing.T) { t.Parallel() s1, cleanupS1 := TestServer(t, func(c *Config) { - c.EnableEventPublisher = true + c.EnableEventBroker = true }) defer cleanupS1() s2, cleanupS2 := TestServer(t, func(c *Config) { - c.EnableEventPublisher = true + c.EnableEventBroker = true c.Region = "foo" }) defer cleanupS2() @@ -249,7 +258,7 @@ func TestEventStream_RegionForward(t *testing.T) { }() // publish with server 2 - publisher, err := s2.State().EventPublisher() + publisher, err := s2.State().EventBroker() require.NoError(t, err) node := mock.Node() @@ -272,7 +281,7 @@ OUTER: t.Fatalf("Got error: %v", msg.Error.Error()) } - if msg.Event == stream.NDJsonHeartbeat { + if msg.Event == stream.JsonHeartbeat { continue } @@ -306,35 +315,124 @@ func TestEventStream_ACL(t *testing.T) { policyBad := mock.NamespacePolicy("other", "", []string{acl.NamespaceCapabilityReadFS}) tokenBad := mock.CreatePolicyAndToken(t, s.State(), 1005, "invalid", policyBad) + policyNsGood := mock.NamespacePolicy("foo", "", []string{acl.NamespaceCapabilityReadJob}) + tokenNsFoo := mock.CreatePolicyAndToken(t, s.State(), 1006, "valid", policyNsGood) + + policyNsNode := mock.NamespacePolicy("foo", "", []string{acl.NamespaceCapabilityReadJob}) + policyNsNode += "\n" + mock.NodePolicy("read") + tokenNsNode := mock.CreatePolicyAndToken(t, s.State(), 1007, "validnNsNode", policyNsNode) + cases := []struct { Name string Token string + Topics map[structs.Topic][]string + Namespace string ExpectedErr string + PublishFn func(p *stream.EventBroker) }{ { - Name: "no token", - Token: "", + Name: "no token", + Token: "", + Topics: map[structs.Topic][]string{ + "*": {"*"}, + }, + ExpectedErr: structs.ErrPermissionDenied.Error(), + }, + { + Name: "bad token", + Token: tokenBad.SecretID, + Topics: map[structs.Topic][]string{ + "*": {"*"}, + }, + ExpectedErr: structs.ErrPermissionDenied.Error(), + }, + { + Name: "root token", + Token: root.SecretID, + Topics: map[structs.Topic][]string{ + "*": {"*"}, + }, + ExpectedErr: "subscription closed by server", + }, + { + Name: "job namespace token - correct ns", + Token: tokenNsFoo.SecretID, + Topics: map[structs.Topic][]string{ + "Job": {"*"}, + "Eval": {"*"}, + "Alloc": {"*"}, + "Deployment": {"*"}, + }, + Namespace: "foo", + ExpectedErr: "subscription closed by server", + PublishFn: func(p *stream.EventBroker) { + p.Publish(&structs.Events{Index: uint64(1000), Events: []structs.Event{{Topic: "Job", Namespace: "foo", Payload: mock.Job()}}}) + }, + }, + { + Name: "job namespace token - incorrect ns", + Token: tokenNsFoo.SecretID, + Topics: map[structs.Topic][]string{ + "Job": {"*"}, // good + }, + Namespace: "bar", // bad ExpectedErr: structs.ErrPermissionDenied.Error(), + PublishFn: func(p *stream.EventBroker) { + p.Publish(&structs.Events{Index: uint64(1000), Events: []structs.Event{{Topic: "Job", Namespace: "foo", Payload: mock.Job()}}}) + }, }, { - Name: "bad token", - Token: tokenBad.SecretID, + Name: "job namespace token - request management topic", + Token: tokenNsFoo.SecretID, + Topics: map[structs.Topic][]string{ + "*": {"*"}, // bad + }, + Namespace: "foo", ExpectedErr: structs.ErrPermissionDenied.Error(), + PublishFn: func(p *stream.EventBroker) { + p.Publish(&structs.Events{Index: uint64(1000), Events: []structs.Event{{Topic: "Job", Namespace: "foo", Payload: mock.Job()}}}) + }, }, { - Name: "root token", - Token: root.SecretID, + Name: "job namespace token - request invalid node topic", + Token: tokenNsFoo.SecretID, + Topics: map[structs.Topic][]string{ + "Eval": {"*"}, // good + "Node": {"*"}, // bad + }, + Namespace: "foo", + ExpectedErr: structs.ErrPermissionDenied.Error(), + PublishFn: func(p *stream.EventBroker) { + p.Publish(&structs.Events{Index: uint64(1000), Events: []structs.Event{{Topic: "Job", Namespace: "foo", Payload: mock.Job()}}}) + }, + }, + { + Name: "job+node namespace token, valid", + Token: tokenNsNode.SecretID, + Topics: map[structs.Topic][]string{ + "Eval": {"*"}, // good + "Node": {"*"}, // good + }, + Namespace: "foo", ExpectedErr: "subscription closed by server", + PublishFn: func(p *stream.EventBroker) { + p.Publish(&structs.Events{Index: uint64(1000), Events: []structs.Event{{Topic: "Node", Payload: mock.Node()}}}) + }, }, } for _, tc := range cases { t.Run(tc.Name, func(t *testing.T) { + var ns string + if tc.Namespace != "" { + ns = tc.Namespace + } // Create request for all topics and keys req := structs.EventStreamRequest{ - Topics: map[structs.Topic][]string{"*": {"*"}}, + Topics: tc.Topics, QueryOptions: structs.QueryOptions{ Region: s.Region(), + Namespace: ns, AuthToken: tc.Token, }, } @@ -372,20 +470,25 @@ func TestEventStream_ACL(t *testing.T) { encoder := codec.NewEncoder(p1, structs.MsgpackHandle) require.Nil(encoder.Encode(req)) - publisher, err := s.State().EventPublisher() + publisher, err := s.State().EventBroker() require.NoError(err) // publish some events node := mock.Node() + publisher.Publish(&structs.Events{Index: uint64(1), Events: []structs.Event{{Topic: "test", Payload: node}}}) publisher.Publish(&structs.Events{Index: uint64(2), Events: []structs.Event{{Topic: "test", Payload: node}}}) + if tc.PublishFn != nil { + tc.PublishFn(publisher) + } + timeout := time.After(5 * time.Second) OUTER: for { select { case <-timeout: - require.Fail("timeout waiting for response") + t.Fatal("timeout waiting for events") case err := <-errCh: t.Fatal(err) case msg := <-streamMsg: @@ -398,7 +501,7 @@ func TestEventStream_ACL(t *testing.T) { if strings.Contains(msg.Error.Error(), tc.ExpectedErr) { break OUTER } else { - require.Fail("Unexpected error", msg.Error) + t.Fatalf("unexpected error %v", msg.Error) } } } diff --git a/nomad/fsm.go b/nomad/fsm.go index ead6d43e03f2..d30c15d4528d 100644 --- a/nomad/fsm.go +++ b/nomad/fsm.go @@ -8,6 +8,7 @@ import ( "time" metrics "github.com/armon/go-metrics" + "github.com/hashicorp/go-hclog" log "github.com/hashicorp/go-hclog" memdb "github.com/hashicorp/go-memdb" "github.com/hashicorp/go-msgpack/codec" @@ -102,7 +103,7 @@ type nomadFSM struct { type nomadSnapshot struct { snap *state.StateSnapshot timetable *TimeTable - durableEventCount int + durableEventCount int64 } // snapshotHeader is the first entry in our snapshot @@ -128,14 +129,17 @@ type FSMConfig struct { // Region is the region of the server embedding the FSM Region string - EnableEventPublisher bool + // EnableEventBroker specifies if the FSMs state store should enable + // it's event publisher. + EnableEventBroker bool + // EventBufferSize is the amount of messages to hold in memory EventBufferSize int64 // Durable count specifies the amount of events generated by the state store // to save to disk during snapshot generation. The most recent events // limited to count will be saved. - DurableEventCount int + DurableEventCount int64 } // NewFSMPath is used to construct a new FSM with a blank state @@ -144,7 +148,7 @@ func NewFSM(config *FSMConfig) (*nomadFSM, error) { sconfig := &state.StateStoreConfig{ Logger: config.Logger, Region: config.Region, - EnablePublisher: config.EnableEventPublisher, + EnablePublisher: config.EnableEventBroker, EventBufferSize: config.EventBufferSize, DurableEventCount: config.DurableEventCount, } @@ -176,7 +180,7 @@ func NewFSM(config *FSMConfig) (*nomadFSM, error) { // Close is used to cleanup resources associated with the FSM func (n *nomadFSM) Close() error { - n.state.StopEventPublisher() + n.state.StopEventBroker() return nil } @@ -1275,7 +1279,7 @@ func (n *nomadFSM) Restore(old io.ReadCloser) error { config := &state.StateStoreConfig{ Logger: n.config.Logger, Region: n.config.Region, - EnablePublisher: n.config.EnableEventPublisher, + EnablePublisher: n.config.EnableEventBroker, EventBufferSize: n.config.EventBufferSize, DurableEventCount: n.config.DurableEventCount, } @@ -1522,6 +1526,12 @@ func (n *nomadFSM) Restore(old io.ReadCloser) error { return err } case EventSnapshot: + // If the event broker is disabled but the snapshot from potentially + // a remote server has events, ignore them + if !n.config.EnableEventBroker { + return nil + } + event := new(structs.Events) if err := dec.Decode(event); err != nil { return err @@ -1543,7 +1553,9 @@ func (n *nomadFSM) Restore(old io.ReadCloser) error { } } - restore.Commit() + if err := restore.Commit(); err != nil { + return err + } // COMPAT Remove in 0.10 // Clean up active deployments that do not have a job @@ -1565,8 +1577,9 @@ func (n *nomadFSM) Restore(old io.ReadCloser) error { // Rehydrate the new state store's event publisher with the events // persisted in the snapshot - if n.config.EnableEventPublisher { - if err := rehydratePublisherFromState(n.state); err != nil { + if n.config.EnableEventBroker { + n.logger.Debug("Rehydrating event broker events from snapshot") + if err := rehydratePublisherFromState(n.state, n.logger); err != nil { n.logger.Error("Error re-hydrating event publisher during restore", "error", err) } } @@ -1574,8 +1587,11 @@ func (n *nomadFSM) Restore(old io.ReadCloser) error { return nil } -func rehydratePublisherFromState(s *state.StateStore) error { - pub, err := s.EventPublisher() +// rehydratePublisherFromState is used during a snapshot restore to +// add the persisted events from that snapshot that were just added to memdb +// back into the event publisher +func rehydratePublisherFromState(s *state.StateStore, l hclog.Logger) error { + pub, err := s.EventBroker() if err != nil { return err } @@ -1584,6 +1600,7 @@ func rehydratePublisherFromState(s *state.StateStore) error { if err != nil { return err } + count := 0 for { raw := events.Next() if raw == nil { @@ -1591,7 +1608,10 @@ func rehydratePublisherFromState(s *state.StateStore) error { } e := raw.(*structs.Events) pub.Publish(e) + count++ } + + l.Debug("finished hydrating event broker from snapshot", "events", count) return nil } @@ -2386,7 +2406,7 @@ func (s *nomadSnapshot) persistEvents(sink raft.SnapshotSink, encoder *codec.Enc return err } - count := 0 + var count int64 for { // Get the next item raw := events.Next() @@ -2397,14 +2417,12 @@ func (s *nomadSnapshot) persistEvents(sink raft.SnapshotSink, encoder *codec.Enc // Prepare the request struct event := raw.(*structs.Events) - eventCount := len(event.Events) - // Write out a volume snapshot sink.Write([]byte{byte(EventSnapshot)}) if err := encoder.Encode(event); err != nil { return err } - count += eventCount + count += int64(len(event.Events)) // Only write to sink until durableCount has been reached if count >= s.durableEventCount { diff --git a/nomad/fsm_test.go b/nomad/fsm_test.go index b1f8d83dd2eb..2557e59e86a2 100644 --- a/nomad/fsm_test.go +++ b/nomad/fsm_test.go @@ -50,13 +50,13 @@ func testFSM(t *testing.T) *nomadFSM { dispatcher, _ := testPeriodicDispatcher(t) logger := testlog.HCLogger(t) fsmConfig := &FSMConfig{ - EvalBroker: broker, - Periodic: dispatcher, - Blocked: NewBlockedEvals(broker, logger), - Logger: logger, - Region: "global", - EnableEventPublisher: true, - EventBufferSize: 100, + EvalBroker: broker, + Periodic: dispatcher, + Blocked: NewBlockedEvals(broker, logger), + Logger: logger, + Region: "global", + EnableEventBroker: true, + EventBufferSize: 100, } fsm, err := NewFSM(fsmConfig) if err != nil { @@ -3206,7 +3206,7 @@ func TestFSM_SnapshotRestore_Events_WithDurability(t *testing.T) { t.Parallel() // Add some state fsm := testFSM(t) - fsm.config.EnableEventPublisher = true + fsm.config.EnableEventBroker = true // DurableEventCount = 4 each mock events wrapper contains 2 events fsm.config.DurableEventCount = 4 @@ -3246,7 +3246,7 @@ func TestFSM_SnapshotRestore_Events_WithDurability(t *testing.T) { raw1 := iter.Next() require.Nil(t, raw1) - pub, err := state2.EventPublisher() + pub, err := state2.EventBroker() require.NoError(t, err) testutil.WaitForResult(func() (bool, error) { @@ -3264,7 +3264,7 @@ func TestFSM_SnapshotRestore_Events_NoDurability(t *testing.T) { t.Parallel() fsm := testFSM(t) // Enable event publisher with durable event count of zero - fsm.config.EnableEventPublisher = true + fsm.config.EnableEventBroker = true fsm.config.DurableEventCount = 0 state := fsm.State() diff --git a/nomad/node_endpoint_test.go b/nomad/node_endpoint_test.go index 0f685dd67c2b..23baba2e41c4 100644 --- a/nomad/node_endpoint_test.go +++ b/nomad/node_endpoint_test.go @@ -1917,7 +1917,7 @@ func TestClientEndpoint_GetClientAllocs_Blocking_GC(t *testing.T) { if time.Since(start) < 100*time.Millisecond { t.Fatalf("too fast") } - assert.EqualValues(200, int(resp3.Index)) + assert.EqualValues(200, resp3.Index) if assert.Len(resp3.Allocs, 1) { assert.EqualValues(100, resp3.Allocs[alloc1.ID]) } diff --git a/nomad/server.go b/nomad/server.go index 29757dd3c080..44de55471b9b 100644 --- a/nomad/server.go +++ b/nomad/server.go @@ -1211,13 +1211,14 @@ func (s *Server) setupRaft() error { // Create the FSM fsmConfig := &FSMConfig{ - EvalBroker: s.evalBroker, - Periodic: s.periodicDispatcher, - Blocked: s.blockedEvals, - Logger: s.logger, - Region: s.Region(), - EnableEventPublisher: s.config.EnableEventPublisher, - EventBufferSize: s.config.EventBufferSize, + EvalBroker: s.evalBroker, + Periodic: s.periodicDispatcher, + Blocked: s.blockedEvals, + Logger: s.logger, + Region: s.Region(), + EnableEventBroker: s.config.EnableEventBroker, + DurableEventCount: s.config.DurableEventCount, + EventBufferSize: s.config.EventBufferSize, } var err error s.fsm, err = NewFSM(fsmConfig) diff --git a/nomad/state/deployment_events_test.go b/nomad/state/deployment_events_test.go index bbaeb7996eaa..08e06e45a8d3 100644 --- a/nomad/state/deployment_events_test.go +++ b/nomad/state/deployment_events_test.go @@ -5,7 +5,6 @@ import ( "testing" "time" - "github.com/hashicorp/nomad/helper" "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/stream" "github.com/hashicorp/nomad/nomad/structs" @@ -15,7 +14,7 @@ import ( func TestDeploymentEventFromChanges(t *testing.T) { t.Parallel() s := TestStateStoreCfg(t, TestStateStorePublisher(t)) - defer s.StopEventPublisher() + defer s.StopEventBroker() // setup setupTx := s.db.WriteTxn(10) @@ -59,82 +58,6 @@ func TestDeploymentEventFromChanges(t *testing.T) { } -func TestDeploymentEventFromChanges_Promotion(t *testing.T) { - t.Parallel() - s := TestStateStoreCfg(t, TestStateStorePublisher(t)) - defer s.StopEventPublisher() - - // setup - setupTx := s.db.WriteTxn(10) - - j := mock.Job() - tg1 := j.TaskGroups[0] - tg2 := tg1.Copy() - tg2.Name = "foo" - j.TaskGroups = append(j.TaskGroups, tg2) - require.NoError(t, s.upsertJobImpl(10, j, false, setupTx)) - - d := mock.Deployment() - d.StatusDescription = structs.DeploymentStatusDescriptionRunningNeedsPromotion - d.JobID = j.ID - d.TaskGroups = map[string]*structs.DeploymentState{ - "web": { - DesiredTotal: 10, - DesiredCanaries: 1, - }, - "foo": { - DesiredTotal: 10, - DesiredCanaries: 1, - }, - } - require.NoError(t, s.upsertDeploymentImpl(10, d, setupTx)) - - // create set of allocs - c1 := mock.Alloc() - c1.JobID = j.ID - c1.DeploymentID = d.ID - d.TaskGroups[c1.TaskGroup].PlacedCanaries = append(d.TaskGroups[c1.TaskGroup].PlacedCanaries, c1.ID) - c1.DeploymentStatus = &structs.AllocDeploymentStatus{ - Healthy: helper.BoolToPtr(true), - } - c2 := mock.Alloc() - c2.JobID = j.ID - c2.DeploymentID = d.ID - d.TaskGroups[c2.TaskGroup].PlacedCanaries = append(d.TaskGroups[c2.TaskGroup].PlacedCanaries, c2.ID) - c2.TaskGroup = tg2.Name - c2.DeploymentStatus = &structs.AllocDeploymentStatus{ - Healthy: helper.BoolToPtr(true), - } - - require.NoError(t, s.upsertAllocsImpl(10, []*structs.Allocation{c1, c2}, setupTx)) - - // commit setup transaction - setupTx.Txn.Commit() - - e := mock.Eval() - // Request to promote canaries - msgType := structs.DeploymentPromoteRequestType - req := &structs.ApplyDeploymentPromoteRequest{ - DeploymentPromoteRequest: structs.DeploymentPromoteRequest{ - DeploymentID: d.ID, - All: true, - }, - Eval: e, - } - - require.NoError(t, s.UpdateDeploymentPromotion(msgType, 100, req)) - - events := WaitForEvents(t, s, 100, 1, 1*time.Second) - require.Len(t, events, 4) - - got := events[0] - require.Equal(t, uint64(100), got.Index) - require.Equal(t, d.ID, got.Key) - - de := got.Payload.(*DeploymentEvent) - require.Equal(t, structs.DeploymentStatusRunning, de.Deployment.Status) -} - func WaitForEvents(t *testing.T, s *StateStore, index uint64, minEvents int, timeout time.Duration) []structs.Event { ctx, cancel := context.WithCancel(context.Background()) defer cancel() @@ -163,15 +86,19 @@ func WaitForEvents(t *testing.T, s *StateStore, index uint64, minEvents int, tim } func EventsForIndex(t *testing.T, s *StateStore, index uint64) []structs.Event { - pub, err := s.EventPublisher() + pub, err := s.EventBroker() require.NoError(t, err) sub, err := pub.Subscribe(&stream.SubscribeRequest{ Topics: map[structs.Topic][]string{ - "*": []string{"*"}, + "*": {"*"}, }, - Index: index, + Index: index, + StartExactlyAtIndex: true, }) + if err != nil { + return []structs.Event{} + } defer sub.Unsubscribe() require.NoError(t, err) diff --git a/nomad/state/events.go b/nomad/state/events.go index f396975f67a7..a5a65a02b8c7 100644 --- a/nomad/state/events.go +++ b/nomad/state/events.go @@ -7,22 +7,11 @@ import ( ) const ( - TopicDeployment structs.Topic = "Deployment" - TopicEval structs.Topic = "Eval" - TopicAlloc structs.Topic = "Alloc" - TopicJob structs.Topic = "Job" - // TopicNodeRegistration stream.Topic = "NodeRegistration" - // TopicNodeDeregistration stream.Topic = "NodeDeregistration" - // TopicNodeDrain stream.Topic = "NodeDrain" - TopicNode structs.Topic = "Node" - - // TODO(drew) Node Events use TopicNode + Type - TypeNodeRegistration = "NodeRegistration" - TypeNodeDeregistration = "NodeDeregistration" - TypeNodeEligibilityUpdate = "NodeEligibility" - TypeNodeDrain = "NodeDrain" - TypeNodeEvent = "NodeEvent" - + TypeNodeRegistration = "NodeRegistration" + TypeNodeDeregistration = "NodeDeregistration" + TypeNodeEligibilityUpdate = "NodeEligibility" + TypeNodeDrain = "NodeDrain" + TypeNodeEvent = "NodeEvent" TypeDeploymentUpdate = "DeploymentStatusUpdate" TypeDeploymentPromotion = "DeploymentPromotion" TypeDeploymentAllocHealth = "DeploymentAllocHealth" @@ -36,22 +25,28 @@ const ( TypePlanResult = "PlanResult" ) +// JobEvent holds a newly updated Job. type JobEvent struct { Job *structs.Job } +// EvalEvent holds a newly updated Eval. type EvalEvent struct { Eval *structs.Evaluation } +// AllocEvent holds a newly updated Allocation. The +// Allocs embedded Job has been removed to reduce size. type AllocEvent struct { Alloc *structs.Allocation } +// DeploymentEvent holds a newly updated Deployment. type DeploymentEvent struct { Deployment *structs.Deployment } +// NodeEvent holds a newly updated Node type NodeEvent struct { Node *structs.Node } @@ -74,43 +69,30 @@ type JobDrainDetails struct { AllocDetails map[string]NodeDrainAllocDetails } +var MsgTypeEvents = map[structs.MessageType]string{ + structs.NodeRegisterRequestType: TypeNodeRegistration, + structs.UpsertNodeEventsType: TypeNodeEvent, + structs.EvalUpdateRequestType: TypeEvalUpdated, + structs.AllocClientUpdateRequestType: TypeAllocUpdated, + structs.JobRegisterRequestType: TypeJobRegistered, + structs.AllocUpdateRequestType: TypeAllocUpdated, + structs.NodeUpdateStatusRequestType: TypeNodeEvent, + structs.JobDeregisterRequestType: TypeJobDeregistered, + structs.JobBatchDeregisterRequestType: TypeJobBatchDeregistered, + structs.AllocUpdateDesiredTransitionRequestType: TypeAllocUpdateDesiredStatus, + structs.NodeUpdateEligibilityRequestType: TypeNodeDrain, + structs.BatchNodeUpdateDrainRequestType: TypeNodeDrain, + structs.DeploymentStatusUpdateRequestType: TypeDeploymentUpdate, + structs.DeploymentPromoteRequestType: TypeDeploymentPromotion, + structs.DeploymentAllocHealthRequestType: TypeDeploymentAllocHealth, + structs.ApplyPlanResultsRequestType: TypePlanResult, +} + +// GenericEventsFromChanges returns a set of events for a given set of +// transaction changes. It currently ignores Delete operations. func GenericEventsFromChanges(tx ReadTxn, changes Changes) (*structs.Events, error) { - var eventType string - switch changes.MsgType { - case structs.NodeRegisterRequestType: - eventType = TypeNodeRegistration - case structs.UpsertNodeEventsType: - eventType = TypeNodeEvent - case structs.EvalUpdateRequestType: - eventType = TypeEvalUpdated - case structs.AllocClientUpdateRequestType: - eventType = TypeAllocUpdated - case structs.JobRegisterRequestType: - eventType = TypeJobRegistered - case structs.AllocUpdateRequestType: - eventType = TypeAllocUpdated - case structs.NodeUpdateStatusRequestType: - eventType = TypeNodeEvent - case structs.JobDeregisterRequestType: - eventType = TypeJobDeregistered - case structs.JobBatchDeregisterRequestType: - eventType = TypeJobBatchDeregistered - case structs.AllocUpdateDesiredTransitionRequestType: - eventType = TypeAllocUpdateDesiredStatus - case structs.NodeUpdateEligibilityRequestType: - eventType = TypeNodeDrain - case structs.BatchNodeUpdateDrainRequestType: - eventType = TypeNodeDrain - case structs.DeploymentStatusUpdateRequestType: - eventType = TypeDeploymentUpdate - case structs.DeploymentPromoteRequestType: - eventType = TypeDeploymentPromotion - case structs.DeploymentAllocHealthRequestType: - eventType = TypeDeploymentAllocHealth - case structs.ApplyPlanResultsRequestType: - eventType = TypePlanResult - default: - // unknown request type + eventType, ok := MsgTypeEvents[changes.MsgType] + if !ok { return nil, nil } @@ -127,7 +109,7 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) (*structs.Events, err } event := structs.Event{ - Topic: TopicEval, + Topic: structs.TopicEval, Type: eventType, Index: changes.Index, Key: after.ID, @@ -149,15 +131,22 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) (*structs.Events, err } alloc := after.Copy() + + filterKeys := []string{ + alloc.JobID, + alloc.DeploymentID, + } + // remove job info to help keep size of alloc event down alloc.Job = nil event := structs.Event{ - Topic: TopicAlloc, - Type: eventType, - Index: changes.Index, - Key: after.ID, - Namespace: after.Namespace, + Topic: structs.TopicAlloc, + Type: eventType, + Index: changes.Index, + Key: after.ID, + FilterKeys: filterKeys, + Namespace: after.Namespace, Payload: &AllocEvent{ Alloc: alloc, }, @@ -174,7 +163,7 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) (*structs.Events, err } event := structs.Event{ - Topic: TopicAlloc, + Topic: structs.TopicJob, Type: eventType, Index: changes.Index, Key: after.ID, @@ -195,7 +184,7 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) (*structs.Events, err } event := structs.Event{ - Topic: TopicNode, + Topic: structs.TopicNode, Type: eventType, Index: changes.Index, Key: after.ID, @@ -214,7 +203,7 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) (*structs.Events, err } event := structs.Event{ - Topic: TopicNode, + Topic: structs.TopicDeployment, Type: eventType, Index: changes.Index, Key: after.ID, diff --git a/nomad/state/events_test.go b/nomad/state/events_test.go new file mode 100644 index 000000000000..ba93adedb955 --- /dev/null +++ b/nomad/state/events_test.go @@ -0,0 +1,567 @@ +package state + +import ( + "testing" + "time" + + "github.com/hashicorp/nomad/helper" + "github.com/hashicorp/nomad/helper/uuid" + "github.com/hashicorp/nomad/nomad/mock" + "github.com/hashicorp/nomad/nomad/structs" + "github.com/stretchr/testify/require" +) + +// structs.AllocClientUpdateRequestType: +// structs.AllocUpdateRequestType +// JobDeregisterRequestType +// jobregisterrequesttype + +func TestGenericEventsFromChanges_DeploymentUpdate(t *testing.T) { + t.Parallel() + s := TestStateStoreCfg(t, TestStateStorePublisher(t)) + defer s.StopEventBroker() + + // setup + setupTx := s.db.WriteTxn(10) + + j := mock.Job() + e := mock.Eval() + e.JobID = j.ID + + d := mock.Deployment() + d.JobID = j.ID + + require.NoError(t, s.upsertJobImpl(10, j, false, setupTx)) + require.NoError(t, s.upsertDeploymentImpl(10, d, setupTx)) + + setupTx.Txn.Commit() + + msgType := structs.DeploymentStatusUpdateRequestType + + req := &structs.DeploymentStatusUpdateRequest{ + DeploymentUpdate: &structs.DeploymentStatusUpdate{ + DeploymentID: d.ID, + Status: structs.DeploymentStatusPaused, + StatusDescription: structs.DeploymentStatusDescriptionPaused, + }, + Eval: e, + // Exlude Job and assert its added + } + + require.NoError(t, s.UpdateDeploymentStatus(msgType, 100, req)) + + events := WaitForEvents(t, s, 100, 1, 1*time.Second) + require.Len(t, events, 2) + + got := events[0] + require.Equal(t, uint64(100), got.Index) + require.Equal(t, d.ID, got.Key) + + de := got.Payload.(*DeploymentEvent) + require.Equal(t, structs.DeploymentStatusPaused, de.Deployment.Status) + require.Contains(t, got.FilterKeys, j.ID) +} + +func TestGenericEventsFromChanges_DeploymentPromotion(t *testing.T) { + t.Parallel() + s := TestStateStoreCfg(t, TestStateStorePublisher(t)) + defer s.StopEventBroker() + + // setup + setupTx := s.db.WriteTxn(10) + + j := mock.Job() + tg1 := j.TaskGroups[0] + tg2 := tg1.Copy() + tg2.Name = "foo" + j.TaskGroups = append(j.TaskGroups, tg2) + require.NoError(t, s.upsertJobImpl(10, j, false, setupTx)) + + d := mock.Deployment() + d.StatusDescription = structs.DeploymentStatusDescriptionRunningNeedsPromotion + d.JobID = j.ID + d.TaskGroups = map[string]*structs.DeploymentState{ + "web": { + DesiredTotal: 10, + DesiredCanaries: 1, + }, + "foo": { + DesiredTotal: 10, + DesiredCanaries: 1, + }, + } + require.NoError(t, s.upsertDeploymentImpl(10, d, setupTx)) + + // create set of allocs + c1 := mock.Alloc() + c1.JobID = j.ID + c1.DeploymentID = d.ID + d.TaskGroups[c1.TaskGroup].PlacedCanaries = append(d.TaskGroups[c1.TaskGroup].PlacedCanaries, c1.ID) + c1.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + c2 := mock.Alloc() + c2.JobID = j.ID + c2.DeploymentID = d.ID + d.TaskGroups[c2.TaskGroup].PlacedCanaries = append(d.TaskGroups[c2.TaskGroup].PlacedCanaries, c2.ID) + c2.TaskGroup = tg2.Name + c2.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + + require.NoError(t, s.upsertAllocsImpl(10, []*structs.Allocation{c1, c2}, setupTx)) + + // commit setup transaction + setupTx.Txn.Commit() + + e := mock.Eval() + // Request to promote canaries + msgType := structs.DeploymentPromoteRequestType + req := &structs.ApplyDeploymentPromoteRequest{ + DeploymentPromoteRequest: structs.DeploymentPromoteRequest{ + DeploymentID: d.ID, + All: true, + }, + Eval: e, + } + + require.NoError(t, s.UpdateDeploymentPromotion(msgType, 100, req)) + + events := WaitForEvents(t, s, 100, 1, 1*time.Second) + require.Len(t, events, 4) + + got := events[0] + require.Equal(t, uint64(100), got.Index) + require.Equal(t, d.ID, got.Key) + + de := got.Payload.(*DeploymentEvent) + require.Equal(t, structs.DeploymentStatusRunning, de.Deployment.Status) + require.Equal(t, TypeDeploymentPromotion, got.Type) +} + +func TestGenericEventsFromChanges_DeploymentAllocHealthRequestType(t *testing.T) { + t.Parallel() + s := TestStateStoreCfg(t, TestStateStorePublisher(t)) + defer s.StopEventBroker() + + // setup + setupTx := s.db.WriteTxn(10) + + j := mock.Job() + tg1 := j.TaskGroups[0] + tg2 := tg1.Copy() + tg2.Name = "foo" + j.TaskGroups = append(j.TaskGroups, tg2) + require.NoError(t, s.upsertJobImpl(10, j, false, setupTx)) + + d := mock.Deployment() + d.StatusDescription = structs.DeploymentStatusDescriptionRunningNeedsPromotion + d.JobID = j.ID + d.TaskGroups = map[string]*structs.DeploymentState{ + "web": { + DesiredTotal: 10, + DesiredCanaries: 1, + }, + "foo": { + DesiredTotal: 10, + DesiredCanaries: 1, + }, + } + require.NoError(t, s.upsertDeploymentImpl(10, d, setupTx)) + + // create set of allocs + c1 := mock.Alloc() + c1.JobID = j.ID + c1.DeploymentID = d.ID + d.TaskGroups[c1.TaskGroup].PlacedCanaries = append(d.TaskGroups[c1.TaskGroup].PlacedCanaries, c1.ID) + c1.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + c2 := mock.Alloc() + c2.JobID = j.ID + c2.DeploymentID = d.ID + d.TaskGroups[c2.TaskGroup].PlacedCanaries = append(d.TaskGroups[c2.TaskGroup].PlacedCanaries, c2.ID) + c2.TaskGroup = tg2.Name + c2.DeploymentStatus = &structs.AllocDeploymentStatus{ + Healthy: helper.BoolToPtr(true), + } + + require.NoError(t, s.upsertAllocsImpl(10, []*structs.Allocation{c1, c2}, setupTx)) + + // Commit setup + setupTx.Commit() + + msgType := structs.DeploymentAllocHealthRequestType + + req := &structs.ApplyDeploymentAllocHealthRequest{ + DeploymentAllocHealthRequest: structs.DeploymentAllocHealthRequest{ + DeploymentID: d.ID, + HealthyAllocationIDs: []string{c1.ID}, + UnhealthyAllocationIDs: []string{c2.ID}, + }, + DeploymentUpdate: &structs.DeploymentStatusUpdate{ + DeploymentID: d.ID, + }, + } + + require.NoError(t, s.UpdateDeploymentAllocHealth(msgType, 100, req)) + + events := WaitForEvents(t, s, 100, 1, 1*time.Second) + require.Len(t, events, 3) + + var allocEvents []structs.Event + var deploymentEvent []structs.Event + for _, e := range events { + if e.Topic == structs.TopicAlloc { + allocEvents = append(allocEvents, e) + } else if e.Topic == structs.TopicDeployment { + deploymentEvent = append(deploymentEvent, e) + } + } + + require.Len(t, allocEvents, 2) + for _, e := range allocEvents { + require.Equal(t, 100, int(e.Index)) + require.Equal(t, TypeDeploymentAllocHealth, e.Type) + require.Equal(t, structs.TopicAlloc, e.Topic) + } + + require.Len(t, deploymentEvent, 1) + for _, e := range deploymentEvent { + require.Equal(t, 100, int(e.Index)) + require.Equal(t, TypeDeploymentAllocHealth, e.Type) + require.Equal(t, structs.TopicDeployment, e.Topic) + require.Equal(t, d.ID, e.Key) + } +} + +func TestGenericEventsFromChanges_UpsertNodeEventsType(t *testing.T) { + t.Parallel() + s := TestStateStoreCfg(t, TestStateStorePublisher(t)) + defer s.StopEventBroker() + + // setup + n1 := mock.Node() + n2 := mock.Node() + + require.NoError(t, s.UpsertNode(10, n1)) + require.NoError(t, s.UpsertNode(12, n2)) + + msgType := structs.UpsertNodeEventsType + req := &structs.EmitNodeEventsRequest{ + NodeEvents: map[string][]*structs.NodeEvent{ + n1.ID: { + { + Message: "update", + }, + }, + n2.ID: { + { + Message: "update", + }, + }, + }, + } + + require.NoError(t, s.UpsertNodeEventsMsgType(msgType, 100, req.NodeEvents)) + events := WaitForEvents(t, s, 100, 1, 1*time.Second) + require.Len(t, events, 2) + + for _, e := range events { + require.Equal(t, structs.TopicNode, e.Topic) + require.Equal(t, TypeNodeEvent, e.Type) + event := e.Payload.(*NodeEvent) + require.Equal(t, "update", event.Node.Events[len(event.Node.Events)-1].Message) + } + +} + +func TestGenericEventsFromChanges_NodeUpdateStatusRequest(t *testing.T) { + t.Parallel() + s := TestStateStoreCfg(t, TestStateStorePublisher(t)) + defer s.StopEventBroker() + + // setup + n1 := mock.Node() + + require.NoError(t, s.UpsertNode(10, n1)) + + updated := time.Now() + msgType := structs.NodeUpdateStatusRequestType + req := &structs.NodeUpdateStatusRequest{ + NodeID: n1.ID, + Status: structs.NodeStatusDown, + UpdatedAt: updated.UnixNano(), + NodeEvent: &structs.NodeEvent{Message: "down"}, + } + + require.NoError(t, s.UpdateNodeStatus(msgType, 100, req.NodeID, req.Status, req.UpdatedAt, req.NodeEvent)) + events := WaitForEvents(t, s, 100, 1, 1*time.Second) + require.Len(t, events, 1) + + e := events[0] + require.Equal(t, structs.TopicNode, e.Topic) + require.Equal(t, TypeNodeEvent, e.Type) + event := e.Payload.(*NodeEvent) + require.Equal(t, "down", event.Node.Events[len(event.Node.Events)-1].Message) + require.Equal(t, structs.NodeStatusDown, event.Node.Status) +} + +func TestGenericEventsFromChanges_EvalUpdateRequestType(t *testing.T) { + t.Parallel() + s := TestStateStoreCfg(t, TestStateStorePublisher(t)) + defer s.StopEventBroker() + + // setup + e1 := mock.Eval() + + require.NoError(t, s.UpsertEvals(10, []*structs.Evaluation{e1})) + + e2 := mock.Eval() + e2.ID = e1.ID + e2.JobID = e1.JobID + e2.Status = structs.EvalStatusBlocked + + msgType := structs.EvalUpdateRequestType + req := &structs.EvalUpdateRequest{ + Evals: []*structs.Evaluation{e2}, + } + + require.NoError(t, s.UpsertEvalsMsgType(msgType, 100, req.Evals)) + + events := WaitForEvents(t, s, 100, 1, 1*time.Second) + require.Len(t, events, 1) + + e := events[0] + require.Equal(t, structs.TopicEval, e.Topic) + require.Equal(t, TypeEvalUpdated, e.Type) + event := e.Payload.(*EvalEvent) + require.Equal(t, "blocked", event.Eval.Status) +} + +func TestGenericEventsFromChanges_ApplyPlanResultsRequestType(t *testing.T) { + t.Parallel() + s := TestStateStoreCfg(t, TestStateStorePublisher(t)) + defer s.StopEventBroker() + + // setup + alloc := mock.Alloc() + alloc2 := mock.Alloc() + job := alloc.Job + alloc.Job = nil + alloc2.Job = nil + + d := mock.Deployment() + alloc.DeploymentID = d.ID + alloc2.DeploymentID = d.ID + + require.NoError(t, s.UpsertJob(9, job)) + + eval := mock.Eval() + eval.JobID = job.ID + + // Create an eval + require.NoError(t, s.UpsertEvals(10, []*structs.Evaluation{eval})) + + msgType := structs.ApplyPlanResultsRequestType + req := &structs.ApplyPlanResultsRequest{ + AllocUpdateRequest: structs.AllocUpdateRequest{ + Alloc: []*structs.Allocation{alloc, alloc2}, + Job: job, + }, + Deployment: d, + EvalID: eval.ID, + } + + require.NoError(t, s.UpsertPlanResults(msgType, 100, req)) + + events := WaitForEvents(t, s, 100, 1, 1*time.Second) + require.Len(t, events, 5) + + var allocs []structs.Event + var evals []structs.Event + var jobs []structs.Event + var deploys []structs.Event + for _, e := range events { + if e.Topic == structs.TopicAlloc { + allocs = append(allocs, e) + } else if e.Topic == structs.TopicEval { + evals = append(evals, e) + } else if e.Topic == structs.TopicJob { + jobs = append(jobs, e) + } else if e.Topic == structs.TopicDeployment { + deploys = append(deploys, e) + } + require.Equal(t, TypePlanResult, e.Type) + } + require.Len(t, allocs, 2) + require.Len(t, evals, 1) + require.Len(t, jobs, 1) + require.Len(t, deploys, 1) +} + +func TestGenericEventsFromChanges_BatchNodeUpdateDrainRequestType(t *testing.T) { + t.Parallel() + s := TestStateStoreCfg(t, TestStateStorePublisher(t)) + defer s.StopEventBroker() + + // setup + n1 := mock.Node() + n2 := mock.Node() + + require.NoError(t, s.UpsertNode(10, n1)) + require.NoError(t, s.UpsertNode(11, n2)) + + updated := time.Now() + msgType := structs.BatchNodeUpdateDrainRequestType + + expectedDrain := &structs.DrainStrategy{ + DrainSpec: structs.DrainSpec{ + Deadline: -1 * time.Second, + }, + } + event := &structs.NodeEvent{ + Message: "Drain strategy enabled", + Subsystem: structs.NodeEventSubsystemDrain, + Timestamp: time.Now(), + } + req := structs.BatchNodeUpdateDrainRequest{ + Updates: map[string]*structs.DrainUpdate{ + n1.ID: { + DrainStrategy: expectedDrain, + }, + n2.ID: { + DrainStrategy: expectedDrain, + }, + }, + NodeEvents: map[string]*structs.NodeEvent{ + n1.ID: event, + n2.ID: event, + }, + UpdatedAt: updated.UnixNano(), + } + + require.NoError(t, s.BatchUpdateNodeDrain(msgType, 100, req.UpdatedAt, req.Updates, req.NodeEvents)) + + events := WaitForEvents(t, s, 100, 1, 1*time.Second) + require.Len(t, events, 2) + + for _, e := range events { + require.Equal(t, 100, int(e.Index)) + require.Equal(t, TypeNodeDrain, e.Type) + require.Equal(t, structs.TopicNode, e.Topic) + ne := e.Payload.(*NodeEvent) + require.Equal(t, event.Message, ne.Node.Events[len(ne.Node.Events)-1].Message) + } +} + +func TestGenericEventsFromChanges_NodeUpdateEligibilityRequestType(t *testing.T) { + t.Parallel() + s := TestStateStoreCfg(t, TestStateStorePublisher(t)) + defer s.StopEventBroker() + + // setup + n1 := mock.Node() + + require.NoError(t, s.UpsertNode(10, n1)) + + msgType := structs.NodeUpdateEligibilityRequestType + + event := &structs.NodeEvent{ + Message: "Node marked as ineligible", + Subsystem: structs.NodeEventSubsystemCluster, + Timestamp: time.Now(), + } + + req := structs.NodeUpdateEligibilityRequest{ + NodeID: n1.ID, + NodeEvent: event, + Eligibility: structs.NodeSchedulingIneligible, + UpdatedAt: time.Now().UnixNano(), + } + + require.NoError(t, s.UpdateNodeEligibility(msgType, 100, req.NodeID, req.Eligibility, req.UpdatedAt, req.NodeEvent)) + + events := WaitForEvents(t, s, 100, 1, 1*time.Second) + require.Len(t, events, 1) + + for _, e := range events { + require.Equal(t, 100, int(e.Index)) + require.Equal(t, TypeNodeDrain, e.Type) + require.Equal(t, structs.TopicNode, e.Topic) + ne := e.Payload.(*NodeEvent) + require.Equal(t, event.Message, ne.Node.Events[len(ne.Node.Events)-1].Message) + require.Equal(t, structs.NodeSchedulingIneligible, ne.Node.SchedulingEligibility) + } +} + +func TestGenericEventsFromChanges_AllocUpdateDesiredTransitionRequestType(t *testing.T) { + t.Parallel() + s := TestStateStoreCfg(t, TestStateStorePublisher(t)) + defer s.StopEventBroker() + + alloc := mock.Alloc() + + require.Nil(t, s.UpsertJob(10, alloc.Job)) + require.Nil(t, s.UpsertAllocs(11, []*structs.Allocation{alloc})) + + msgType := structs.AllocUpdateDesiredTransitionRequestType + + eval := &structs.Evaluation{ + ID: uuid.Generate(), + Namespace: alloc.Namespace, + Priority: alloc.Job.Priority, + Type: alloc.Job.Type, + TriggeredBy: structs.EvalTriggerNodeDrain, + JobID: alloc.Job.ID, + JobModifyIndex: alloc.Job.ModifyIndex, + Status: structs.EvalStatusPending, + } + evals := []*structs.Evaluation{eval} + + req := &structs.AllocUpdateDesiredTransitionRequest{ + Allocs: map[string]*structs.DesiredTransition{ + alloc.ID: {Migrate: helper.BoolToPtr(true)}, + }, + Evals: evals, + } + + require.NoError(t, s.UpdateAllocsDesiredTransitions(msgType, 100, req.Allocs, req.Evals)) + + events := WaitForEvents(t, s, 100, 1, 1*time.Second) + require.Len(t, events, 2) + + var allocs []structs.Event + var evalEvents []structs.Event + for _, e := range events { + if e.Topic == structs.TopicEval { + evalEvents = append(evalEvents, e) + } else if e.Topic == structs.TopicAlloc { + allocs = append(allocs, e) + } else { + require.Fail(t, "unexpected event type") + } + + require.Equal(t, TypeAllocUpdateDesiredStatus, e.Type) + } + + require.Len(t, allocs, 1) + require.Len(t, evalEvents, 1) +} + +func TestGenericEventsFromChanges_JobBatchDeregisterRequestType(t *testing.T) { + // TODO Job batch deregister logic mostly occurs in the FSM + t.SkipNow() + +} +func TestGenericEventsFromChanges_AllocClientUpdateRequestType(t *testing.T) { + t.SkipNow() +} + +func TestGenericEventsFromChanges_AllocUpdateRequestType(t *testing.T) { + t.SkipNow() +} + +func TestGenericEventsFromChanges_JobDeregisterRequestType(t *testing.T) { + t.SkipNow() +} diff --git a/nomad/state/node_events.go b/nomad/state/node_events.go index 4742b1aa5878..178262fb99d4 100644 --- a/nomad/state/node_events.go +++ b/nomad/state/node_events.go @@ -19,7 +19,7 @@ func NodeDeregisterEventFromChanges(tx ReadTxn, changes Changes) (*structs.Event } event := structs.Event{ - Topic: TopicNode, + Topic: structs.TopicNode, Type: TypeNodeDeregistration, Index: changes.Index, Key: before.ID, @@ -66,7 +66,7 @@ func NodeDrainEventFromChanges(tx ReadTxn, changes Changes) (*structs.Events, er } event := structs.Event{ - Topic: TopicNode, + Topic: structs.TopicNode, Type: TypeNodeDrain, Index: changes.Index, Key: after.ID, diff --git a/nomad/state/node_events_test.go b/nomad/state/node_events_test.go index 86874814c364..2789ae80f338 100644 --- a/nomad/state/node_events_test.go +++ b/nomad/state/node_events_test.go @@ -21,13 +21,13 @@ func TestNodeEventsFromChanges(t *testing.T) { }{ { MsgType: structs.NodeRegisterRequestType, - WantTopic: TopicNode, + WantTopic: structs.TopicNode, Name: "node registered", Mutate: func(s *StateStore, tx *txn) error { return upsertNodeTxn(tx, tx.Index, testNode()) }, WantEvents: []structs.Event{{ - Topic: TopicNode, + Topic: structs.TopicNode, Type: TypeNodeRegistration, Key: testNodeID(), Index: 100, @@ -39,13 +39,13 @@ func TestNodeEventsFromChanges(t *testing.T) { }, { MsgType: structs.NodeRegisterRequestType, - WantTopic: TopicNode, + WantTopic: structs.TopicNode, Name: "node registered initializing", Mutate: func(s *StateStore, tx *txn) error { return upsertNodeTxn(tx, tx.Index, testNode(nodeNotReady)) }, WantEvents: []structs.Event{{ - Topic: TopicNode, + Topic: structs.TopicNode, Type: TypeNodeRegistration, Key: testNodeID(), Index: 100, @@ -57,7 +57,7 @@ func TestNodeEventsFromChanges(t *testing.T) { }, { MsgType: structs.NodeDeregisterRequestType, - WantTopic: TopicNode, + WantTopic: structs.TopicNode, Name: "node deregistered", Setup: func(s *StateStore, tx *txn) error { return upsertNodeTxn(tx, tx.Index, testNode()) @@ -66,7 +66,7 @@ func TestNodeEventsFromChanges(t *testing.T) { return deleteNodeTxn(tx, tx.Index, []string{testNodeID()}) }, WantEvents: []structs.Event{{ - Topic: TopicNode, + Topic: structs.TopicNode, Type: TypeNodeDeregistration, Key: testNodeID(), Index: 100, @@ -78,7 +78,7 @@ func TestNodeEventsFromChanges(t *testing.T) { }, { MsgType: structs.NodeDeregisterRequestType, - WantTopic: TopicNode, + WantTopic: structs.TopicNode, Name: "batch node deregistered", Setup: func(s *StateStore, tx *txn) error { require.NoError(t, upsertNodeTxn(tx, tx.Index, testNode())) @@ -89,7 +89,7 @@ func TestNodeEventsFromChanges(t *testing.T) { }, WantEvents: []structs.Event{ { - Topic: TopicNode, + Topic: structs.TopicNode, Type: TypeNodeDeregistration, Key: testNodeID(), Index: 100, @@ -98,7 +98,7 @@ func TestNodeEventsFromChanges(t *testing.T) { }, }, { - Topic: TopicNode, + Topic: structs.TopicNode, Type: TypeNodeDeregistration, Key: testNodeIDTwo(), Index: 100, @@ -111,7 +111,7 @@ func TestNodeEventsFromChanges(t *testing.T) { }, { MsgType: structs.UpsertNodeEventsType, - WantTopic: TopicNode, + WantTopic: structs.TopicNode, Name: "batch node events upserted", Setup: func(s *StateStore, tx *txn) error { require.NoError(t, upsertNodeTxn(tx, tx.Index, testNode())) @@ -141,7 +141,7 @@ func TestNodeEventsFromChanges(t *testing.T) { }, WantEvents: []structs.Event{ { - Topic: TopicNode, + Topic: structs.TopicNode, Type: TypeNodeEvent, Key: testNodeID(), Index: 100, @@ -150,7 +150,7 @@ func TestNodeEventsFromChanges(t *testing.T) { }, }, { - Topic: TopicNode, + Topic: structs.TopicNode, Type: TypeNodeEvent, Key: testNodeIDTwo(), Index: 100, @@ -166,7 +166,7 @@ func TestNodeEventsFromChanges(t *testing.T) { for _, tc := range cases { t.Run(tc.Name, func(t *testing.T) { s := TestStateStoreCfg(t, TestStateStorePublisher(t)) - defer s.StopEventPublisher() + defer s.StopEventBroker() if tc.Setup != nil { // Bypass publish mechanism for setup @@ -215,7 +215,7 @@ func TestNodeEventsFromChanges(t *testing.T) { func TestNodeDrainEventFromChanges(t *testing.T) { t.Parallel() s := TestStateStoreCfg(t, TestStateStorePublisher(t)) - defer s.StopEventPublisher() + defer s.StopEventBroker() // setup setupTx := s.db.WriteTxn(10) @@ -251,7 +251,7 @@ func TestNodeDrainEventFromChanges(t *testing.T) { require.Len(t, got.Events, 1) - require.Equal(t, TopicNode, got.Events[0].Topic) + require.Equal(t, structs.TopicNode, got.Events[0].Topic) require.Equal(t, TypeNodeDrain, got.Events[0].Type) require.Equal(t, uint64(100), got.Events[0].Index) @@ -296,10 +296,6 @@ func nodeNotReady(n *structs.Node) { n.Status = structs.NodeStatusInit } -func nodeReady(n *structs.Node) { - n.Status = structs.NodeStatusReady -} - func nodeIDTwo(n *structs.Node) { n.ID = testNodeIDTwo() } diff --git a/nomad/state/state_changes.go b/nomad/state/state_changes.go index c94545d30d86..c7a2d9634e34 100644 --- a/nomad/state/state_changes.go +++ b/nomad/state/state_changes.go @@ -27,29 +27,20 @@ type Changes struct { // changeTrackerDB is a thin wrapper around memdb.DB which enables TrackChanges on // all write transactions. When the transaction is committed the changes are -// sent to the EventPublisher which will create and emit change events. +// sent to the EventBroker which will create and emit change events. type changeTrackerDB struct { - db *memdb.MemDB + memdb *memdb.MemDB durableCount int64 - publisher *stream.EventPublisher + publisher *stream.EventBroker processChanges func(ReadTxn, Changes) (*structs.Events, error) } -// ChangeConfig -type ChangeConfig struct { - DurableEventCount int -} - -func NewChangeTrackerDB(db *memdb.MemDB, publisher *stream.EventPublisher, changesFn changeProcessor, cfg *ChangeConfig) *changeTrackerDB { - if cfg == nil { - cfg = &ChangeConfig{} - } - +func NewChangeTrackerDB(db *memdb.MemDB, publisher *stream.EventBroker, changesFn changeProcessor, durableCount int64) *changeTrackerDB { return &changeTrackerDB{ - db: db, + memdb: db, publisher: publisher, processChanges: changesFn, - durableCount: int64(cfg.DurableEventCount), + durableCount: durableCount, } } @@ -63,7 +54,7 @@ func noOpProcessChanges(ReadTxn, Changes) (*structs.Events, error) { return nil, // TODO: this could return a regular memdb.Txn if all the state functions accepted // the ReadTxn interface func (c *changeTrackerDB) ReadTxn() *txn { - return &txn{Txn: c.db.Txn(false)} + return &txn{Txn: c.memdb.Txn(false)} } // WriteTxn returns a wrapped memdb.Txn suitable for writes to the state store. @@ -78,7 +69,7 @@ func (c *changeTrackerDB) ReadTxn() *txn { // data directly into the DB. These cases may use WriteTxnRestore. func (c *changeTrackerDB) WriteTxn(idx uint64) *txn { t := &txn{ - Txn: c.db.Txn(true), + Txn: c.memdb.Txn(true), Index: idx, publish: c.publish, } @@ -91,7 +82,7 @@ func (c *changeTrackerDB) WriteTxnMsgT(msgType structs.MessageType, idx uint64) t := &txn{ msgType: msgType, - Txn: c.db.Txn(true), + Txn: c.memdb.Txn(true), Index: idx, publish: c.publish, persistChanges: persistChanges, @@ -101,7 +92,7 @@ func (c *changeTrackerDB) WriteTxnMsgT(msgType structs.MessageType, idx uint64) } func (c *changeTrackerDB) publish(changes Changes) (*structs.Events, error) { - readOnlyTx := c.db.Txn(false) + readOnlyTx := c.memdb.Txn(false) defer readOnlyTx.Abort() events, err := c.processChanges(readOnlyTx, changes) @@ -123,12 +114,12 @@ func (c *changeTrackerDB) publish(changes Changes) (*structs.Events, error) { // written across many indexes. func (c *changeTrackerDB) WriteTxnRestore() *txn { return &txn{ - Txn: c.db.Txn(true), + Txn: c.memdb.Txn(true), Index: 0, } } -// txn wraps a memdb.Txn to capture changes and send them to the EventPublisher. +// txn wraps a memdb.Txn to capture changes and send them to the EventBroker. // // This can not be done with txn.Defer because the callback passed to Defer is // invoked after commit completes, and because the callback can not return an @@ -149,7 +140,7 @@ type txn struct { publish func(changes Changes) (*structs.Events, error) } -// Commit first pushes changes to EventPublisher, then calls Commit on the +// Commit first pushes changes to EventBroker, then calls Commit on the // underlying transaction. // // Note that this function, unlike memdb.Txn, returns an error which must be checked @@ -195,51 +186,11 @@ func processDBChanges(tx ReadTxn, changes Changes) (*structs.Events, error) { case structs.IgnoreUnknownTypeFlag: // unknown event type return nil, nil - case structs.NodeRegisterRequestType: - return GenericEventsFromChanges(tx, changes) - case structs.NodeUpdateStatusRequestType: - // TODO(drew) test - return GenericEventsFromChanges(tx, changes) case structs.NodeDeregisterRequestType: return NodeDeregisterEventFromChanges(tx, changes) case structs.NodeUpdateDrainRequestType: return NodeDrainEventFromChanges(tx, changes) - case structs.UpsertNodeEventsType: - return GenericEventsFromChanges(tx, changes) - case structs.DeploymentStatusUpdateRequestType: - return GenericEventsFromChanges(tx, changes) - case structs.DeploymentPromoteRequestType: - return GenericEventsFromChanges(tx, changes) - case structs.DeploymentAllocHealthRequestType: - return GenericEventsFromChanges(tx, changes) - case structs.ApplyPlanResultsRequestType: - // TODO test - return GenericEventsFromChanges(tx, changes) - case structs.EvalUpdateRequestType: - return GenericEventsFromChanges(tx, changes) - case structs.AllocClientUpdateRequestType: - return GenericEventsFromChanges(tx, changes) - case structs.JobRegisterRequestType: - // TODO(drew) test - return GenericEventsFromChanges(tx, changes) - case structs.AllocUpdateRequestType: - // TODO(drew) test - return GenericEventsFromChanges(tx, changes) - case structs.JobDeregisterRequestType: - // TODO(drew) test / handle delete - return GenericEventsFromChanges(tx, changes) - case structs.JobBatchDeregisterRequestType: - // TODO(drew) test & handle delete - return GenericEventsFromChanges(tx, changes) - case structs.AllocUpdateDesiredTransitionRequestType: - // TODO(drew) drain - return GenericEventsFromChanges(tx, changes) - case structs.NodeUpdateEligibilityRequestType: - // TODO(drew) test, drain - return GenericEventsFromChanges(tx, changes) - case structs.BatchNodeUpdateDrainRequestType: - // TODO(drew) test, drain + default: return GenericEventsFromChanges(tx, changes) } - return nil, nil } diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index b50c809308d8..ebb3f0a661d8 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -54,7 +54,7 @@ type StateStoreConfig struct { // DurableEventCount is used to determine if events from transaction changes // should be saved in go-memdb - DurableEventCount int + DurableEventCount int64 } // The StateStore is responsible for maintaining all the Nomad @@ -77,7 +77,7 @@ type StateStore struct { // TODO: refactor abondonCh to use a context so that both can use the same // cancel mechanism. - stopEventPublisher func() + stopEventBroker func() } // NewStateStore is used to create a new state store @@ -91,27 +91,22 @@ func NewStateStore(config *StateStoreConfig) (*StateStore, error) { // Create the state store ctx, cancel := context.WithCancel(context.TODO()) s := &StateStore{ - logger: config.Logger.Named("state_store"), - config: config, - abandonCh: make(chan struct{}), - stopEventPublisher: cancel, + logger: config.Logger.Named("state_store"), + config: config, + abandonCh: make(chan struct{}), + stopEventBroker: cancel, } if config.EnablePublisher { - cfg := &ChangeConfig{ - DurableEventCount: config.DurableEventCount, - } - // Create new event publisher using provided config - publisher := stream.NewEventPublisher(ctx, stream.EventPublisherCfg{ - EventBufferTTL: 1 * time.Hour, + broker := stream.NewEventBroker(ctx, stream.EventBrokerCfg{ EventBufferSize: config.EventBufferSize, Logger: config.Logger, - OnEvict: s.eventPublisherEvict, + OnEvict: s.eventBrokerEvict, }) - s.db = NewChangeTrackerDB(db, publisher, processDBChanges, cfg) + s.db = NewChangeTrackerDB(db, broker, processDBChanges, config.DurableEventCount) } else { - s.db = NewChangeTrackerDB(db, nil, noOpProcessChanges, nil) + s.db = NewChangeTrackerDB(db, nil, noOpProcessChanges, 0) } // Initialize the state store with required enterprise objects @@ -122,16 +117,16 @@ func NewStateStore(config *StateStoreConfig) (*StateStore, error) { return s, nil } -func (s *StateStore) EventPublisher() (*stream.EventPublisher, error) { +func (s *StateStore) EventBroker() (*stream.EventBroker, error) { if s.db.publisher == nil { - return nil, fmt.Errorf("EventPublisher not configured") + return nil, fmt.Errorf("EventBroker not configured") } return s.db.publisher, nil } -// eventPublisherEvict is used as a callback to delete an evicted events +// eventBrokerEvict is used as a callback to delete an evicted events // entry from go-memdb. -func (s *StateStore) eventPublisherEvict(events *structs.Events) { +func (s *StateStore) eventBrokerEvict(events *structs.Events) { if err := s.deleteEvent(events); err != nil { if err == memdb.ErrNotFound { s.logger.Info("Evicted event was not found in go-memdb table", "event index", events.Index) @@ -142,7 +137,7 @@ func (s *StateStore) eventPublisherEvict(events *structs.Events) { } func (s *StateStore) deleteEvent(events *structs.Events) error { - txn := s.db.db.Txn(true) + txn := s.db.memdb.Txn(true) defer txn.Abort() if err := txn.Delete("events", events); err != nil { @@ -162,7 +157,7 @@ func (s *StateStore) Config() *StateStoreConfig { // we use MemDB, we just need to snapshot the state of the underlying // database. func (s *StateStore) Snapshot() (*StateSnapshot, error) { - memDBSnap := s.db.db.Snapshot() + memDBSnap := s.db.memdb.Snapshot() store := StateStore{ logger: s.logger, @@ -170,7 +165,7 @@ func (s *StateStore) Snapshot() (*StateSnapshot, error) { } // Create a new change tracker DB that does not publish or track changes - store.db = NewChangeTrackerDB(memDBSnap, nil, noOpProcessChanges, nil) + store.db = NewChangeTrackerDB(memDBSnap, nil, noOpProcessChanges, 0) snap := &StateSnapshot{ StateStore: store, @@ -253,14 +248,14 @@ func (s *StateStore) AbandonCh() <-chan struct{} { // Abandon is used to signal that the given state store has been abandoned. // Calling this more than one time will panic. func (s *StateStore) Abandon() { - s.StopEventPublisher() + s.StopEventBroker() close(s.abandonCh) } -// StopStopEventPublisher calls the cancel func for the state stores event +// StopStopEventBroker calls the cancel func for the state stores event // publisher. It should be called during server shutdown. -func (s *StateStore) StopEventPublisher() { - s.stopEventPublisher() +func (s *StateStore) StopEventBroker() { + s.stopEventBroker() } // QueryFn is the definition of a function that can be used to implement a basic @@ -340,7 +335,7 @@ func (s *StateStore) UpsertPlanResults(msgType structs.MessageType, index uint64 } } - // Update the status of dmsgType structs.MessageType by the plan. + // Update the status of deployments effected by the plan. if len(results.DeploymentUpdates) != 0 { s.upsertDeploymentUpdates(index, results.DeploymentUpdates, txn) } @@ -790,6 +785,7 @@ func (s *StateStore) ScalingEventsByJob(ws memdb.WatchSet, namespace, jobID stri // UpsertNodeMsgType is used to register a node or update a node definition // This is assumed to be triggered by the client, so we retain the value // of drain/eligibility which is set by the scheduler. +// TODO(drew) remove this and update all test callers of UpsertNode to use msgType func (s *StateStore) UpsertNodeMsgType(msgType structs.MessageType, index uint64, node *structs.Node) error { txn := s.db.WriteTxnMsgT(msgType, index) defer txn.Abort() @@ -972,7 +968,7 @@ func (s *StateStore) updateNodeStatusTxn(txn *txn, nodeID, status string, update // BatchUpdateNodeDrain is used to update the drain of a node set of nodes func (s *StateStore) BatchUpdateNodeDrain(msgType structs.MessageType, index uint64, updatedAt int64, updates map[string]*structs.DrainUpdate, events map[string]*structs.NodeEvent) error { - txn := s.db.WriteTxn(index) + txn := s.db.WriteTxnMsgT(msgType, index) defer txn.Abort() for node, update := range updates { if err := s.updateNodeDrainImpl(txn, index, node, update.DrainStrategy, update.MarkEligible, updatedAt, events[node]); err != nil { @@ -5900,8 +5896,8 @@ func (s *StateRestore) Abort() { } // Commit is used to commit the restore operation -func (s *StateRestore) Commit() { - s.txn.Commit() +func (s *StateRestore) Commit() error { + return s.txn.Commit() } // NodeRestore is used to restore a node diff --git a/nomad/state/state_store_events_test.go b/nomad/state/state_store_events_test.go index 08a5b88b770a..146b792e9724 100644 --- a/nomad/state/state_store_events_test.go +++ b/nomad/state/state_store_events_test.go @@ -27,7 +27,7 @@ func TestStateStore_Events_OnEvict(t *testing.T) { } s := TestStateStoreCfg(t, cfg) - _, err := s.EventPublisher() + _, err := s.EventBroker() require.NoError(t, err) // force 3 evictions @@ -85,7 +85,7 @@ func TestStateStore_Events_OnEvict_Missing(t *testing.T) { } s := TestStateStoreCfg(t, cfg) - _, err := s.EventPublisher() + _, err := s.EventBroker() require.NoError(t, err) getEvents := func() []*structs.Events { diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index 1c0f345f57eb..7fed591a4a4b 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -1368,7 +1368,7 @@ func TestStateStore_RestoreNode(t *testing.T) { if err != nil { t.Fatalf("err: %v", err) } - restore.Commit() + require.NoError(t, restore.Commit()) ws := memdb.NewWatchSet() out, err := state.NodeByID(ws, node.ID) @@ -2441,7 +2441,7 @@ func TestStateStore_RestoreJob(t *testing.T) { if err != nil { t.Fatalf("err: %v", err) } - restore.Commit() + require.NoError(t, restore.Commit()) ws := memdb.NewWatchSet() out, err := state.JobByID(ws, job.Namespace, job.ID) @@ -2711,7 +2711,7 @@ func TestStateStore_RestorePeriodicLaunch(t *testing.T) { if err != nil { t.Fatalf("err: %v", err) } - restore.Commit() + require.NoError(t, restore.Commit()) ws := memdb.NewWatchSet() out, err := state.PeriodicLaunchByID(ws, job.Namespace, job.ID) @@ -2743,7 +2743,7 @@ func TestStateStore_RestoreJobVersion(t *testing.T) { if err != nil { t.Fatalf("err: %v", err) } - restore.Commit() + require.NoError(t, restore.Commit()) ws := memdb.NewWatchSet() out, err := state.JobByIDAndVersion(ws, job.Namespace, job.ID, job.Version) @@ -2775,7 +2775,7 @@ func TestStateStore_RestoreDeployment(t *testing.T) { if err != nil { t.Fatalf("err: %v", err) } - restore.Commit() + require.NoError(t, restore.Commit()) ws := memdb.NewWatchSet() out, err := state.DeploymentByID(ws, d.ID) @@ -2815,7 +2815,7 @@ func TestStateStore_RestoreJobSummary(t *testing.T) { if err != nil { t.Fatalf("err: %v", err) } - restore.Commit() + require.NoError(t, restore.Commit()) ws := memdb.NewWatchSet() out, err := state.JobSummaryByID(ws, job.Namespace, job.ID) @@ -3617,7 +3617,7 @@ func TestStateStore_RestoreCSIPlugin(t *testing.T) { err = restore.CSIPluginRestore(plugin) require.NoError(err) - restore.Commit() + require.NoError(restore.Commit()) ws := memdb.NewWatchSet() out, err := state.CSIPluginByID(ws, plugin.ID) @@ -3731,7 +3731,7 @@ func TestStateStore_RestoreIndex(t *testing.T) { t.Fatalf("err: %v", err) } - restore.Commit() + require.NoError(t, restore.Commit()) out, err := state.Index("jobs") if err != nil { @@ -4397,7 +4397,7 @@ func TestStateStore_RestoreEval(t *testing.T) { if err != nil { t.Fatalf("err: %v", err) } - restore.Commit() + require.NoError(t, restore.Commit()) ws := memdb.NewWatchSet() out, err := state.EvalByID(ws, eval.ID) @@ -6033,7 +6033,7 @@ func TestStateStore_RestoreAlloc(t *testing.T) { t.Fatalf("err: %v", err) } - restore.Commit() + require.NoError(t, restore.Commit()) ws := memdb.NewWatchSet() out, err := state.AllocByID(ws, alloc.ID) @@ -7701,7 +7701,7 @@ func TestStateStore_RestoreVaultAccessor(t *testing.T) { if err != nil { t.Fatalf("err: %v", err) } - restore.Commit() + require.NoError(t, restore.Commit()) ws := memdb.NewWatchSet() out, err := state.VaultAccessor(ws, a.Accessor) @@ -7902,7 +7902,7 @@ func TestStateStore_RestoreSITokenAccessor(t *testing.T) { err = restore.SITokenAccessorRestore(a1) r.NoError(err) - restore.Commit() + require.NoError(t, restore.Commit()) ws := memdb.NewWatchSet() result, err := state.SITokenAccessor(ws, a1.AccessorID) @@ -8382,7 +8382,7 @@ func TestStateStore_RestoreACLPolicy(t *testing.T) { if err != nil { t.Fatalf("err: %v", err) } - restore.Commit() + require.NoError(t, restore.Commit()) ws := memdb.NewWatchSet() out, err := state.ACLPolicyByName(ws, policy.Name) @@ -8441,7 +8441,7 @@ func TestStateStore_RestoreACLToken(t *testing.T) { if err != nil { t.Fatalf("err: %v", err) } - restore.Commit() + require.NoError(t, restore.Commit()) ws := memdb.NewWatchSet() out, err := state.ACLTokenByAccessorID(ws, token.AccessorID) @@ -8470,7 +8470,7 @@ func TestStateStore_SchedulerConfig(t *testing.T) { err = restore.SchedulerConfigRestore(schedConfig) require.Nil(err) - restore.Commit() + require.NoError(restore.Commit()) modIndex, out, err := state.SchedulerConfig() require.Nil(err) @@ -8510,7 +8510,7 @@ func TestStateStore_ClusterMetadataRestore(t *testing.T) { err = restore.ClusterMetadataRestore(meta) require.NoError(err) - restore.Commit() + require.NoError(restore.Commit()) out, err := state.ClusterMetadata(nil) require.NoError(err) @@ -8530,7 +8530,7 @@ func TestStateStore_RestoreScalingPolicy(t *testing.T) { err = restore.ScalingPolicyRestore(scalingPolicy) require.NoError(err) - restore.Commit() + require.NoError(restore.Commit()) ws := memdb.NewWatchSet() out, err := state.ScalingPolicyByID(ws, scalingPolicy.ID) @@ -9543,7 +9543,7 @@ func TestStateStore_RestoreScalingEvents(t *testing.T) { err = restore.ScalingEventsRestore(jobScalingEvents) require.NoError(err) - restore.Commit() + require.NoError(restore.Commit()) ws := memdb.NewWatchSet() out, _, err := state.ScalingEventsByJob(ws, jobScalingEvents.Namespace, diff --git a/nomad/stream/event_buffer.go b/nomad/stream/event_buffer.go index 42c9dc3a4c25..419b27f98371 100644 --- a/nomad/stream/event_buffer.go +++ b/nomad/stream/event_buffer.go @@ -50,19 +50,17 @@ type eventBuffer struct { head atomic.Value tail atomic.Value - maxSize int64 - maxItemTTL time.Duration - onEvict EvictCallbackFn + maxSize int64 + onEvict EvictCallbackFn } // newEventBuffer creates an eventBuffer ready for use. -func newEventBuffer(size int64, maxItemTTL time.Duration, onEvict EvictCallbackFn) *eventBuffer { +func newEventBuffer(size int64, onEvict EvictCallbackFn) *eventBuffer { zero := int64(0) b := &eventBuffer{ - maxSize: size, - size: &zero, - maxItemTTL: maxItemTTL, - onEvict: onEvict, + maxSize: size, + size: &zero, + onEvict: onEvict, } item := newBufferItem(&structs.Events{Index: 0, Events: nil}) @@ -77,7 +75,7 @@ func newEventBuffer(size int64, maxItemTTL time.Duration, onEvict EvictCallbackF // watchers. After calling append, the caller must not make any further // mutations to the events as they may have been exposed to subscribers in other // goroutines. Append only supports a single concurrent caller and must be -// externally synchronized with other Append, or prune calls. +// externally synchronized with other Append calls. func (b *eventBuffer) Append(events *structs.Events) { b.appendItem(newBufferItem(events)) } @@ -104,7 +102,7 @@ func (b *eventBuffer) appendItem(item *bufferItem) { } func newSentinelItem() *bufferItem { - return newBufferItem(&structs.Events{Index: 0, Events: nil}) + return newBufferItem(&structs.Events{}) } // advanceHead drops the current Head buffer item and notifies readers @@ -191,25 +189,6 @@ func (b *eventBuffer) Len() int { return int(atomic.LoadInt64(b.size)) } -// prune advances the head of the buffer until the head buffer item TTL -// is no longer expired. It should be externally synchronized as it mutates -// the buffer of items. -func (b *eventBuffer) prune() { - now := time.Now() - for { - head := b.Head() - if b.Len() == 0 { - return - } - - if now.Sub(head.createdAt) > b.maxItemTTL { - b.advanceHead() - } else { - return - } - } -} - // bufferItem represents a set of events published by a single raft operation. // The first item returned by a newly constructed buffer will have nil Events. // It is a sentinel value which is used to wait on the next events via Next. diff --git a/nomad/stream/event_buffer_test.go b/nomad/stream/event_buffer_test.go index 84f0b8524733..45501d12372c 100644 --- a/nomad/stream/event_buffer_test.go +++ b/nomad/stream/event_buffer_test.go @@ -17,7 +17,7 @@ func TestEventBufferFuzz(t *testing.T) { nReaders := 1000 nMessages := 1000 - b := newEventBuffer(1000, DefaultTTL, nil) + b := newEventBuffer(1000, nil) // Start a write goroutine that will publish 10000 messages with sequential // indexes and some jitter in timing (to allow clients to "catch up" and block @@ -85,7 +85,7 @@ func TestEventBufferFuzz(t *testing.T) { } func TestEventBuffer_Slow_Reader(t *testing.T) { - b := newEventBuffer(10, DefaultTTL, nil) + b := newEventBuffer(10, nil) for i := 0; i < 10; i++ { e := structs.Event{ @@ -114,7 +114,7 @@ func TestEventBuffer_Slow_Reader(t *testing.T) { } func TestEventBuffer_Size(t *testing.T) { - b := newEventBuffer(100, DefaultTTL, nil) + b := newEventBuffer(100, nil) for i := 0; i < 10; i++ { e := structs.Event{ @@ -126,11 +126,11 @@ func TestEventBuffer_Size(t *testing.T) { require.Equal(t, 10, b.Len()) } -// TestEventBuffer_Prune_AllOld tests the behavior when all items -// are past their TTL, the event buffer should prune down to the last message -// and hold onto the last item. -func TestEventBuffer_Prune_AllOld(t *testing.T) { - b := newEventBuffer(100, 1*time.Second, nil) +// TestEventBuffer_Emptying_Buffer tests the behavior when all items +// are removed, the event buffer should advance its head down to the last message +// and insert a placeholder sentinel value. +func TestEventBuffer_Emptying_Buffer(t *testing.T) { + b := newEventBuffer(10, nil) for i := 0; i < 10; i++ { e := structs.Event{ @@ -141,11 +141,11 @@ func TestEventBuffer_Prune_AllOld(t *testing.T) { require.Equal(t, 10, int(b.Len())) - time.Sleep(1 * time.Second) - - // prune old messages, which will bring the event buffer down + // empty the buffer, which will bring the event buffer down // to a single sentinel value - b.prune() + for i := 0; i < 11; i++ { + b.advanceHead() + } // head and tail are now a sentinel value head := b.Head() @@ -203,7 +203,7 @@ func TestEventBuffer_StartAt_CurrentIdx_Past_Start(t *testing.T) { } // buffer starts at index 11 goes to 100 - b := newEventBuffer(100, 1*time.Hour, nil) + b := newEventBuffer(100, nil) for i := 11; i <= 100; i++ { e := structs.Event{ @@ -226,7 +226,7 @@ func TestEventBuffer_OnEvict(t *testing.T) { testOnEvict := func(events *structs.Events) { close(called) } - b := newEventBuffer(2, DefaultTTL, testOnEvict) + b := newEventBuffer(2, testOnEvict) // start at 1 since new event buffer is built with a starting sentinel value for i := 1; i < 4; i++ { diff --git a/nomad/stream/event_publisher.go b/nomad/stream/event_publisher.go index f28131dddec6..bcf8a8fc52b0 100644 --- a/nomad/stream/event_publisher.go +++ b/nomad/stream/event_publisher.go @@ -2,9 +2,11 @@ package stream import ( "context" + "fmt" "sync" "time" + "github.com/armon/go-metrics" "github.com/hashicorp/nomad/nomad/structs" "github.com/hashicorp/go-hclog" @@ -14,48 +16,34 @@ const ( DefaultTTL = 1 * time.Hour ) -type EventPublisherCfg struct { +type EventBrokerCfg struct { EventBufferSize int64 - EventBufferTTL time.Duration Logger hclog.Logger OnEvict EvictCallbackFn } -type EventPublisher struct { - // lock protects the eventbuffer - lock sync.Mutex +type EventBroker struct { + // mu protects the eventbuffer and subscriptions + mu sync.Mutex // eventBuf stores a configurable amount of events in memory eventBuf *eventBuffer - logger hclog.Logger - subscriptions *subscriptions // publishCh is used to send messages from an active txn to a goroutine which // publishes events, so that publishing can happen asynchronously from // the Commit call in the FSM hot path. publishCh chan *structs.Events -} - -type subscriptions struct { - // lock for byToken. If both subscription.lock and EventPublisher.lock need - // to be held, EventPublisher.lock MUST always be acquired first. - lock sync.RWMutex - // byToken is an mapping of active Subscriptions indexed by a token and - // a pointer to the request. - // When the token is modified all subscriptions under that token will be - // reloaded. - // A subscription may be unsubscribed by using the pointer to the request. - byToken map[string]map[*SubscribeRequest]*Subscription + logger hclog.Logger } -func NewEventPublisher(ctx context.Context, cfg EventPublisherCfg) *EventPublisher { - if cfg.EventBufferTTL == 0 { - cfg.EventBufferTTL = 1 * time.Hour - } - +// NewEventBroker returns an EventBroker for publishing change events. +// A goroutine is run in the background to publish events to an event buffer. +// Cancelling the context will shutdown the goroutine to free resources, and stop +// all publishing. +func NewEventBroker(ctx context.Context, cfg EventBrokerCfg) *EventBroker { if cfg.Logger == nil { cfg.Logger = hclog.NewNullLogger() } @@ -65,9 +53,9 @@ func NewEventPublisher(ctx context.Context, cfg EventPublisherCfg) *EventPublish cfg.EventBufferSize = 100 } - buffer := newEventBuffer(cfg.EventBufferSize, cfg.EventBufferTTL, cfg.OnEvict) - e := &EventPublisher{ - logger: cfg.Logger.Named("event_publisher"), + buffer := newEventBuffer(cfg.EventBufferSize, cfg.OnEvict) + e := &EventBroker{ + logger: cfg.Logger.Named("event_broker"), eventBuf: buffer, publishCh: make(chan *structs.Events, 64), subscriptions: &subscriptions{ @@ -80,21 +68,35 @@ func NewEventPublisher(ctx context.Context, cfg EventPublisherCfg) *EventPublish return e } -func (e *EventPublisher) Len() int { +// Returns the current length of the event buffer +func (e *EventBroker) Len() int { + e.mu.Lock() + defer e.mu.Unlock() return e.eventBuf.Len() } // Publish events to all subscribers of the event Topic. -func (e *EventPublisher) Publish(events *structs.Events) { +func (e *EventBroker) Publish(events *structs.Events) { if len(events.Events) > 0 { e.publishCh <- events } } -// Subscribe returns a new Subscription for a given request. -func (e *EventPublisher) Subscribe(req *SubscribeRequest) (*Subscription, error) { - e.lock.Lock() - defer e.lock.Unlock() +// Subscribe returns a new Subscription for a given request. A Subscription +// will receive an initial empty currentItem value which points to the first item +// in the buffer. This allows the new subscription to call Next() without first checking +// for the current Item. +// +// A Subscription will start at the requested index, or as close as possible to +// the requested index if it is no longer in the buffer. If StartExactlyAtIndex is +// set and the index is no longer in the buffer or not yet in the buffer an error +// will be returned. +// +// When a caller is finished with the subscription it must call Subscription.Unsubscribe +// to free ACL tracking resources. TODO(drew) ACL tracking +func (e *EventBroker) Subscribe(req *SubscribeRequest) (*Subscription, error) { + e.mu.Lock() + defer e.mu.Unlock() var head *bufferItem var offset int @@ -103,8 +105,11 @@ func (e *EventPublisher) Subscribe(req *SubscribeRequest) (*Subscription, error) } else { head = e.eventBuf.Head() } - if offset > 0 { - e.logger.Warn("requested index no longer in buffer", "requsted", int(req.Index), "closest", int(head.Events.Index)) + if offset > 0 && req.StartExactlyAtIndex { + return nil, fmt.Errorf("requested index not in buffer") + } else if offset > 0 { + metrics.SetGauge([]string{"nomad", "event_broker", "subscription", "request_offset"}, float32(offset)) + e.logger.Debug("requested index no longer in buffer", "requsted", int(req.Index), "closest", int(head.Events.Index)) } // Empty head so that calling Next on sub @@ -112,17 +117,18 @@ func (e *EventPublisher) Subscribe(req *SubscribeRequest) (*Subscription, error) start.link.next.Store(head) close(start.link.ch) - sub := newSubscription(req, start, e.subscriptions.unsubscribe(req)) + sub := newSubscription(req, start, e.subscriptions.unsubscribeFn(req)) e.subscriptions.add(req, sub) return sub, nil } -func (e *EventPublisher) CloseAll() { +// CloseAll closes all subscriptions +func (e *EventBroker) CloseAll() { e.subscriptions.closeAll() } -func (e *EventPublisher) handleUpdates(ctx context.Context) { +func (e *EventBroker) handleUpdates(ctx context.Context) { for { select { case <-ctx.Done(): @@ -135,16 +141,29 @@ func (e *EventPublisher) handleUpdates(ctx context.Context) { } // sendEvents sends the given events to the publishers event buffer. -func (e *EventPublisher) sendEvents(update *structs.Events) { - e.lock.Lock() - defer e.lock.Unlock() +func (e *EventBroker) sendEvents(update *structs.Events) { + e.mu.Lock() + defer e.mu.Unlock() e.eventBuf.Append(update) } +type subscriptions struct { + // mu for byToken. If both subscription.mu and EventBroker.mu need + // to be held, EventBroker mutex MUST always be acquired first. + mu sync.RWMutex + + // byToken is an mapping of active Subscriptions indexed by a token and + // a pointer to the request. + // When the token is modified all subscriptions under that token will be + // reloaded. + // A subscription may be unsubscribed by using the pointer to the request. + byToken map[string]map[*SubscribeRequest]*Subscription +} + func (s *subscriptions) add(req *SubscribeRequest, sub *Subscription) { - s.lock.Lock() - defer s.lock.Unlock() + s.mu.Lock() + defer s.mu.Unlock() subsByToken, ok := s.byToken[req.Token] if !ok { @@ -155,8 +174,8 @@ func (s *subscriptions) add(req *SubscribeRequest, sub *Subscription) { } func (s *subscriptions) closeSubscriptionsForTokens(tokenSecretIDs []string) { - s.lock.RLock() - defer s.lock.RUnlock() + s.mu.RLock() + defer s.mu.RUnlock() for _, secretID := range tokenSecretIDs { if subs, ok := s.byToken[secretID]; ok { @@ -167,20 +186,29 @@ func (s *subscriptions) closeSubscriptionsForTokens(tokenSecretIDs []string) { } } -// unsubscribe returns a function that the subscription will call to remove +// unsubscribeFn returns a function that the subscription will call to remove // itself from the subsByToken. // This function is returned as a closure so that the caller doesn't need to keep -// track of the SubscriptionRequest, and can not accidentally call unsubscribe with the +// track of the SubscriptionRequest, and can not accidentally call unsubscribeFn with the // wrong pointer. -func (s *subscriptions) unsubscribe(req *SubscribeRequest) func() { +func (s *subscriptions) unsubscribeFn(req *SubscribeRequest) func() { return func() { - s.lock.Lock() - defer s.lock.Unlock() + s.mu.Lock() + defer s.mu.Unlock() subsByToken, ok := s.byToken[req.Token] if !ok { return } + + sub := subsByToken[req] + if sub == nil { + return + } + + // close the subscription + sub.forceClose() + delete(subsByToken, req) if len(subsByToken) == 0 { delete(s.byToken, req.Token) @@ -189,8 +217,8 @@ func (s *subscriptions) unsubscribe(req *SubscribeRequest) func() { } func (s *subscriptions) closeAll() { - s.lock.Lock() - defer s.lock.Unlock() + s.mu.Lock() + defer s.mu.Unlock() for _, byRequest := range s.byToken { for _, sub := range byRequest { diff --git a/nomad/stream/event_publisher_test.go b/nomad/stream/event_publisher_test.go index af92d0595667..0587a81f520c 100644 --- a/nomad/stream/event_publisher_test.go +++ b/nomad/stream/event_publisher_test.go @@ -2,6 +2,7 @@ package stream import ( "context" + "sync/atomic" "testing" "time" @@ -10,16 +11,16 @@ import ( "github.com/stretchr/testify/require" ) -func TestEventPublisher_PublishChangesAndSubscribe(t *testing.T) { +func TestEventBroker_PublishChangesAndSubscribe(t *testing.T) { subscription := &SubscribeRequest{ Topics: map[structs.Topic][]string{ - "Test": []string{"sub-key"}, + "Test": {"sub-key"}, }, } ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second) defer cancel() - publisher := NewEventPublisher(ctx, EventPublisherCfg{EventBufferSize: 100, EventBufferTTL: DefaultTTL}) + publisher := NewEventBroker(ctx, EventBrokerCfg{EventBufferSize: 100}) sub, err := publisher.Subscribe(subscription) require.NoError(t, err) eventCh := consumeSubscription(ctx, sub) @@ -59,11 +60,11 @@ func TestEventPublisher_PublishChangesAndSubscribe(t *testing.T) { require.Equal(t, expected, result.Events) } -func TestEventPublisher_ShutdownClosesSubscriptions(t *testing.T) { +func TestEventBroker_ShutdownClosesSubscriptions(t *testing.T) { ctx, cancel := context.WithCancel(context.Background()) t.Cleanup(cancel) - publisher := NewEventPublisher(ctx, EventPublisherCfg{}) + publisher := NewEventBroker(ctx, EventBrokerCfg{}) sub1, err := publisher.Subscribe(&SubscribeRequest{}) require.NoError(t, err) @@ -82,6 +83,32 @@ func TestEventPublisher_ShutdownClosesSubscriptions(t *testing.T) { require.Equal(t, err, ErrSubscriptionClosed) } +// TestEventBroker_EmptyReqToken_DistinctSubscriptions tests subscription +// hanlding behavior when ACLs are disabled (request Token is empty). +// Subscriptions are mapped by their request token. when that token is empty, +// the subscriptions should still be handled indeppendtly of each other when +// unssubscribing. +func TestEventBroker_EmptyReqToken_DistinctSubscriptions(t *testing.T) { + ctx, cancel := context.WithCancel(context.Background()) + t.Cleanup(cancel) + + publisher := NewEventBroker(ctx, EventBrokerCfg{}) + + // first subscription, empty token + sub1, err := publisher.Subscribe(&SubscribeRequest{}) + require.NoError(t, err) + defer sub1.Unsubscribe() + + // second subscription, empty token + sub2, err := publisher.Subscribe(&SubscribeRequest{}) + require.NoError(t, err) + require.NotNil(t, sub2) + + sub1.Unsubscribe() + + require.Equal(t, subscriptionStateOpen, atomic.LoadUint32(&sub2.state)) +} + func consumeSubscription(ctx context.Context, sub *Subscription) <-chan subNextResult { eventCh := make(chan subNextResult, 1) go func() { diff --git a/nomad/stream/ndjson.go b/nomad/stream/ndjson.go index cd3befc7ea19..7e7ad0928104 100644 --- a/nomad/stream/ndjson.go +++ b/nomad/stream/ndjson.go @@ -1,81 +1,63 @@ package stream import ( - "bytes" "context" "encoding/json" "fmt" - "sync" "time" "github.com/hashicorp/nomad/nomad/structs" ) var ( - // NDJsonHeartbeat is the NDJson to send as a heartbeat + // JsonHeartbeat is an empty JSON object to send as a heartbeat // Avoids creating many heartbeat instances - NDJsonHeartbeat = &structs.NDJson{Data: []byte("{}\n")} + JsonHeartbeat = &structs.EventJson{Data: []byte("{}")} ) -// NDJsonStream is used to send new line delimited JSON and heartbeats +// JsonStream is used to send new line delimited JSON and heartbeats // to a destination (out channel) -type NDJsonStream struct { - out chan<- *structs.NDJson +type JsonStream struct { + // ctx is a passed in context used to notify the json stream + // when it should terminate + ctx context.Context + + outCh chan *structs.EventJson // heartbeat is the interval to send heartbeat messages to keep a connection // open. - heartbeat *time.Ticker - - publishCh chan structs.NDJson - exitCh chan struct{} - - l sync.Mutex - running bool + heartbeatTick *time.Ticker } -// NewNNewNDJsonStream creates a new NDJson stream that will output NDJson structs -// to the passed output channel -func NewNDJsonStream(out chan<- *structs.NDJson, heartbeat time.Duration) *NDJsonStream { - return &NDJsonStream{ - out: out, - heartbeat: time.NewTicker(heartbeat), - exitCh: make(chan struct{}), - publishCh: make(chan structs.NDJson), +// NewJsonStream creates a new json stream that will output Json structs +// to the passed output channel. The constructor starts a goroutine +// to begin heartbeating on its set interval. +func NewJsonStream(ctx context.Context, heartbeat time.Duration) *JsonStream { + s := &JsonStream{ + ctx: ctx, + outCh: make(chan *structs.EventJson, 10), + heartbeatTick: time.NewTicker(heartbeat), } -} -// Run starts a long lived goroutine that handles sending -// heartbeats and processed json objects to the streams out channel as well -func (n *NDJsonStream) Run(ctx context.Context) { - n.l.Lock() - if n.running { - return - } - n.running = true - n.l.Unlock() + go s.heartbeat() - go n.run(ctx) + return s } -func (n *NDJsonStream) run(ctx context.Context) { - defer func() { - n.l.Lock() - n.running = false - n.l.Unlock() - close(n.exitCh) - }() +func (n *JsonStream) OutCh() chan *structs.EventJson { + return n.outCh +} +func (n *JsonStream) heartbeat() { for { select { - case <-ctx.Done(): + case <-n.ctx.Done(): return - case msg := <-n.publishCh: - n.out <- msg.Copy() - case <-n.heartbeat.C: + case <-n.heartbeatTick.C: // Send a heartbeat frame select { - case n.out <- NDJsonHeartbeat: - case <-ctx.Done(): + case n.outCh <- JsonHeartbeat: + case <-n.ctx.Done(): return } } @@ -84,19 +66,20 @@ func (n *NDJsonStream) run(ctx context.Context) { // Send encodes an object into Newline delimited json. An error is returned // if json encoding fails or if the stream is no longer running. -func (n *NDJsonStream) Send(obj interface{}) error { - n.l.Lock() - defer n.l.Unlock() +func (n *JsonStream) Send(v interface{}) error { + if n.ctx.Err() != nil { + return n.ctx.Err() + } - buf := bytes.NewBuffer(nil) - if err := json.NewEncoder(buf).Encode(obj); err != nil { - return fmt.Errorf("marshaling json for stream: %w", err) + buf, err := json.Marshal(v) + if err != nil { + return fmt.Errorf("error marshaling json for stream: %w", err) } select { - case n.publishCh <- structs.NDJson{Data: buf.Bytes()}: - case <-n.exitCh: - return fmt.Errorf("stream is no longer running") + case <-n.ctx.Done(): + return fmt.Errorf("error stream is no longer running: %w", err) + case n.outCh <- &structs.EventJson{Data: buf}: } return nil diff --git a/nomad/stream/ndjson_test.go b/nomad/stream/ndjson_test.go index 589cde1a7190..0c7c4de787dd 100644 --- a/nomad/stream/ndjson_test.go +++ b/nomad/stream/ndjson_test.go @@ -6,7 +6,6 @@ import ( "testing" "time" - "github.com/hashicorp/nomad/nomad/structs" "github.com/stretchr/testify/require" ) @@ -14,15 +13,14 @@ type testObj struct { Name string `json:"name"` } -func TestNDJson(t *testing.T) { +func TestJsonStream(t *testing.T) { t.Parallel() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - out := make(chan *structs.NDJson) - s := NewNDJsonStream(out, 1*time.Second) - s.Run(ctx) + s := NewJsonStream(ctx, 1*time.Second) + out := s.OutCh() require.NoError(t, s.Send(testObj{Name: "test"})) @@ -30,25 +28,31 @@ func TestNDJson(t *testing.T) { var expected bytes.Buffer expected.Write([]byte(`{"name":"test"}`)) - expected.Write([]byte("\n")) require.Equal(t, expected.Bytes(), out1.Data) select { - case _ = <-out: - t.Fatalf("Did not expect another message") + case msg := <-out: + require.Failf(t, "Did not expect another message", "%#v", msg) case <-time.After(100 * time.Millisecond): } + + require.NoError(t, s.Send(testObj{Name: "test2"})) + + out2 := <-out + expected.Reset() + + expected.Write([]byte(`{"name":"test2"}`)) + require.Equal(t, expected.Bytes(), out2.Data) + } -func TestNDJson_Send_After_Stop(t *testing.T) { +func TestJson_Send_After_Stop(t *testing.T) { t.Parallel() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - out := make(chan *structs.NDJson) - s := NewNDJsonStream(out, 1*time.Second) - s.Run(ctx) + s := NewJsonStream(ctx, 1*time.Second) // stop the stream cancel() @@ -57,17 +61,16 @@ func TestNDJson_Send_After_Stop(t *testing.T) { require.Error(t, s.Send(testObj{})) } -func TestNDJson_HeartBeat(t *testing.T) { +func TestJson_HeartBeat(t *testing.T) { t.Parallel() ctx, cancel := context.WithCancel(context.Background()) defer cancel() - out := make(chan *structs.NDJson) - s := NewNDJsonStream(out, 10*time.Millisecond) - s.Run(ctx) + s := NewJsonStream(ctx, 10*time.Millisecond) + out := s.OutCh() heartbeat := <-out - require.Equal(t, NDJsonHeartbeat, heartbeat) + require.Equal(t, JsonHeartbeat, heartbeat) } diff --git a/nomad/stream/subscription.go b/nomad/stream/subscription.go index b85838f3d9c3..70b133a63829 100644 --- a/nomad/stream/subscription.go +++ b/nomad/stream/subscription.go @@ -9,7 +9,6 @@ import ( ) const ( - AllKeys = "*" // subscriptionStateOpen is the default state of a subscription. An open // subscription may receive new events. subscriptionStateOpen uint32 = 0 @@ -25,7 +24,7 @@ const ( var ErrSubscriptionClosed = errors.New("subscription closed by server, client should resubscribe") type Subscription struct { - // state is accessed atomically 0 means open, 1 means closed with reload + // state must be accessed atomically 0 means open, 1 means closed with reload state uint32 req *SubscribeRequest @@ -35,10 +34,10 @@ type Subscription struct { currentItem *bufferItem // forceClosed is closed when forceClose is called. It is used by - // EventPublisher to cancel Next(). + // EventBroker to cancel Next(). forceClosed chan struct{} - // unsub is a function set by EventPublisher that is called to free resources + // unsub is a function set by EventBroker that is called to free resources // when the subscription is no longer needed. // It must be safe to call the function from multiple goroutines and the function // must be idempotent. @@ -51,6 +50,12 @@ type SubscribeRequest struct { Namespace string Topics map[structs.Topic][]string + + // StartExactlyAtIndex specifies if a subscription needs to + // start exactly at the requested Index. If set to false, + // the closest index in the buffer will be returned if there is not + // an exact match + StartExactlyAtIndex bool } func newSubscription(req *SubscribeRequest, item *bufferItem, unsub func()) *Subscription { @@ -124,11 +129,11 @@ func filter(req *SubscribeRequest, events []structs.Event) []structs.Event { var count int for _, e := range events { - _, allTopics := req.Topics[AllKeys] + _, allTopics := req.Topics[structs.TopicAll] if _, ok := req.Topics[e.Topic]; ok || allTopics { var keys []string if allTopics { - keys = req.Topics[AllKeys] + keys = req.Topics[structs.TopicAll] } else { keys = req.Topics[e.Topic] } @@ -136,9 +141,7 @@ func filter(req *SubscribeRequest, events []structs.Event) []structs.Event { continue } for _, k := range keys { - // if req.Namespace != "" && e.Namespace != "" && e.Namespace == - // if e.Namespace != "" && e.Namespace - if e.Key == k || k == AllKeys { + if e.Key == k || k == string(structs.TopicAll) || filterKeyContains(e.FilterKeys, k) { count++ } } @@ -156,11 +159,11 @@ func filter(req *SubscribeRequest, events []structs.Event) []structs.Event { // Return filtered events result := make([]structs.Event, 0, count) for _, e := range events { - _, allTopics := req.Topics[AllKeys] + _, allTopics := req.Topics[structs.TopicAll] if _, ok := req.Topics[e.Topic]; ok || allTopics { var keys []string if allTopics { - keys = req.Topics[AllKeys] + keys = req.Topics[structs.TopicAll] } else { keys = req.Topics[e.Topic] } @@ -169,7 +172,7 @@ func filter(req *SubscribeRequest, events []structs.Event) []structs.Event { continue } for _, k := range keys { - if e.Key == k || k == AllKeys { + if e.Key == k || k == string(structs.TopicAll) || filterKeyContains(e.FilterKeys, k) { result = append(result, e) } } @@ -177,3 +180,12 @@ func filter(req *SubscribeRequest, events []structs.Event) []structs.Event { } return result } + +func filterKeyContains(filterKeys []string, key string) bool { + for _, fk := range filterKeys { + if fk == key { + return true + } + } + return false +} diff --git a/nomad/stream/subscription_test.go b/nomad/stream/subscription_test.go index a83be8809869..c54bc4fa557c 100644 --- a/nomad/stream/subscription_test.go +++ b/nomad/stream/subscription_test.go @@ -8,10 +8,6 @@ import ( "github.com/stretchr/testify/require" ) -func TestSubscription(t *testing.T) { - -} - func TestFilter_AllTopics(t *testing.T) { events := make([]structs.Event, 0, 5) events = append(events, structs.Event{Topic: "Test", Key: "One"}, structs.Event{Topic: "Test", Key: "Two"}) @@ -112,3 +108,22 @@ func TestFilter_Namespace(t *testing.T) { require.Equal(t, cap(actual), 2) } + +func TestFilter_FilterKeys(t *testing.T) { + events := make([]structs.Event, 0, 5) + events = append(events, structs.Event{Topic: "Test", Key: "One", FilterKeys: []string{"extra-key"}}, structs.Event{Topic: "Test", Key: "Two"}, structs.Event{Topic: "Test", Key: "Two"}) + + req := &SubscribeRequest{ + Topics: map[structs.Topic][]string{ + "Test": {"extra-key"}, + }, + Namespace: "foo", + } + actual := filter(req, events) + expected := []structs.Event{ + {Topic: "Test", Key: "One", FilterKeys: []string{"extra-key"}}, + } + require.Equal(t, expected, actual) + + require.Equal(t, cap(actual), 1) +} diff --git a/nomad/structs/structs.go b/nomad/structs/structs.go index 75db4c35cab1..143670eee165 100644 --- a/nomad/structs/structs.go +++ b/nomad/structs/structs.go @@ -10708,8 +10708,7 @@ type ACLTokenUpsertResponse struct { WriteMeta } -// EEventStreamRequest is used to stream events from a servers -// EventPublisher +// EventStreamRequest is used to stream events from a servers EventBroker type EventStreamRequest struct { Topics map[Topic][]string Index int @@ -10719,7 +10718,7 @@ type EventStreamRequest struct { type EventStreamWrapper struct { Error *RpcError - Event *NDJson + Event *EventJson } // RpcError is used for serializing errors with a potential error code @@ -10741,28 +10740,54 @@ func (r *RpcError) Error() string { type Topic string +const ( + TopicDeployment Topic = "Deployment" + TopicEval Topic = "Eval" + TopicAlloc Topic = "Alloc" + TopicJob Topic = "Job" + TopicNode Topic = "Node" + TopicAll Topic = "*" +) + +// Event represents a change in Nomads state. type Event struct { - Topic Topic - Type string - Key string - Namespace string + // Topic represeents the primary object for the event + Topic Topic + + // Type is a short string representing the reason for the event + Type string + + // Key is the primary identifier of the Event, The involved objects ID + Key string + + // Namespace is the namespace of the object, If the object is not namespace + // aware (Node) it is left blank + Namespace string + + // FilterKeys are a set of additional related keys that are used to include + // events during filtering. FilterKeys []string - Index uint64 - Payload interface{} + + // Index is the raft index that corresponds to the event + Index uint64 + + // Payload is the Event itself see state/events.go for a list of events + Payload interface{} } +// Events is a wrapper that contains a set of events for a given index. type Events struct { Index uint64 Events []Event } -// NNDJson is a wrapper for a Newline Delimited JSON object -type NDJson struct { +// EventJson is a wrapper for a JSON object +type EventJson struct { Data []byte } -func (j *NDJson) Copy() *NDJson { - n := new(NDJson) +func (j *EventJson) Copy() *EventJson { + n := new(EventJson) *n = *j n.Data = make([]byte, len(j.Data)) copy(n.Data, j.Data) diff --git a/nomad/testing.go b/nomad/testing.go index edb2d1c8e49a..3b9d4bf4b8ba 100644 --- a/nomad/testing.go +++ b/nomad/testing.go @@ -47,7 +47,7 @@ func TestServer(t testing.T, cb func(*Config)) (*Server, func()) { config.Logger = testlog.HCLogger(t) config.Build = version.Version + "+unittest" config.DevMode = true - config.EnableEventPublisher = true + config.EnableEventBroker = true config.BootstrapExpect = 1 nodeNum := atomic.AddUint32(&nodeNumber, 1) config.NodeName = fmt.Sprintf("nomad-%03d", nodeNum) diff --git a/vendor/github.com/hashicorp/nomad/api/event.go b/vendor/github.com/hashicorp/nomad/api/event.go index dce3c265fe8c..f89e222848f7 100644 --- a/vendor/github.com/hashicorp/nomad/api/event.go +++ b/vendor/github.com/hashicorp/nomad/api/event.go @@ -6,7 +6,7 @@ import ( "fmt" ) -// Ebvents is a set of events for a corresponding index. Events returned for the +// Events is a set of events for a corresponding index. Events returned for the // index depend on which topics are subscribed to when a request is made. type Events struct { Index uint64 @@ -47,11 +47,11 @@ func (c *Client) EventStream() *EventStream { // Stream establishes a new subscription to Nomad's event stream and streams // results back to the returned channel. func (e *EventStream) Stream(ctx context.Context, topics map[Topic][]string, index uint64, q *QueryOptions) (<-chan *Events, error) { - r, err := e.client.newRequest("GET", "/v1/event/stream") if err != nil { return nil, err } + q = q.WithContext(ctx) r.setQueryOptions(q) // Build topic query params @@ -78,11 +78,11 @@ func (e *EventStream) Stream(ctx context.Context, topics map[Topic][]string, ind // Decode next newline delimited json of events var events Events if err := dec.Decode(&events); err != nil { + // set error and fallthrough to + // select eventsCh events = Events{Err: err} - eventsCh <- &events - return } - if events.IsHeartbeat() { + if events.Err == nil && events.IsHeartbeat() { continue } From 8c5f3a042d6793a352ca0bbc8ce9266bfa70b455 Mon Sep 17 00:00:00 2001 From: Drew Bailey <2614075+drewbailey@users.noreply.github.com> Date: Wed, 14 Oct 2020 14:21:37 -0400 Subject: [PATCH 19/19] event stream changelog --- CHANGELOG.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/CHANGELOG.md b/CHANGELOG.md index 9d31e3ffb353..6f4149c44027 100644 --- a/CHANGELOG.md +++ b/CHANGELOG.md @@ -1,5 +1,9 @@ ## 0.13.0 (Unreleased) +FEATURES: + +* **Event Stream**: Subscribe to change events as they occur in real time. [[GH-9013](https://github.com/hashicorp/nomad/issues/9013)] + IMPROVEMENTS: * core: Improved job deregistration error logging. [[GH-8745](https://github.com/hashicorp/nomad/issues/8745)] * api: Added support for cancellation contexts to HTTP API. [[GH-8836](https://github.com/hashicorp/nomad/issues/8836)]