Merge pull request #9013 from hashicorp/event-stream

Event stream
This commit is contained in:
Drew Bailey 2020-10-14 14:45:34 -04:00 committed by GitHub
commit cce13771e3
No known key found for this signature in database
GPG key ID: 4AEE18F83AFDEB23
66 changed files with 5729 additions and 344 deletions

View file

@ -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)]

98
api/event.go Normal file
View file

@ -0,0 +1,98 @@
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
Err error
}
// 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 map[string]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, 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
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 {
return nil, err
}
eventsCh := make(chan *Events, 10)
go func() {
defer resp.Body.Close()
defer close(eventsCh)
dec := json.NewDecoder(resp.Body)
for ctx.Err() == nil {
// 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}
}
if events.Err == nil && events.IsHeartbeat() {
continue
}
select {
case <-ctx.Done():
return
case eventsCh <- &events:
}
}
}()
return eventsCh, nil
}

113
api/event_test.go Normal file
View file

@ -0,0 +1,113 @@
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, err := events.Stream(ctx, topics, 0, q)
require.NoError(t, err)
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")
}
}
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)
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")
}
}

View file

@ -243,6 +243,15 @@ func convertServerConfig(agentConfig *Config) (*nomad.Config, error) {
if agentConfig.Server.UpgradeVersion != "" {
conf.UpgradeVersion = agentConfig.Server.UpgradeVersion
}
if agentConfig.Server.EnableEventBroker != nil {
conf.EnableEventBroker = *agentConfig.Server.EnableEventBroker
}
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 {
conf.AutopilotConfig.CleanupDeadServers = *agentConfig.Autopilot.CleanupDeadServers

View file

@ -57,6 +57,9 @@ func TestAgent_ServerConfig(t *testing.T) {
out, err := a.serverConfig()
require.NoError(t, err)
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)

View file

@ -484,6 +484,19 @@ type ServerConfig struct {
// This value is ignored.
DefaultSchedulerConfig *structs.SchedulerConfiguration `hcl:"default_scheduler_config"`
// EnableEventBroker configures whether this server's state store
// will generate events for its event stream.
EnableEventBroker *bool `hcl:"enable_event_broker"`
// EventBufferSize configure the amount of events to be held in memory.
// If EnableEventBroker 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.
DurableEventCount *int `hcl:"durable_event_count"`
// ExtraKeysHCL is used by hcl to surface unexpected keys
ExtraKeysHCL []string `hcl:",unusedKeys" json:"-"`
}
@ -874,8 +887,11 @@ func DefaultConfig() *Config {
BindWildcardDefaultHostNetwork: true,
},
Server: &ServerConfig{
Enabled: false,
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,
@ -1399,6 +1415,18 @@ func (a *ServerConfig) Merge(b *ServerConfig) *ServerConfig {
result.ServerJoin = result.ServerJoin.Merge(b.ServerJoin)
}
if b.EnableEventBroker != nil {
result.EnableEventBroker = b.EnableEventBroker
}
if b.EventBufferSize != nil {
result.EventBufferSize = b.EventBufferSize
}
if b.DurableEventCount != nil {
result.DurableEventCount = b.DurableEventCount
}
if b.DefaultSchedulerConfig != nil {
c := *b.DefaultSchedulerConfig
result.DefaultSchedulerConfig = &c

View file

@ -122,6 +122,9 @@ var basicConfig = &Config{
RedundancyZone: "foo",
UpgradeVersion: "0.8.0",
EncryptKey: "abc",
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,

View file

@ -138,6 +138,9 @@ func TestConfig_Merge(t *testing.T) {
MaxHeartbeatsPerSecond: 30.0,
RedundancyZone: "foo",
UpgradeVersion: "foo",
EnableEventBroker: helper.BoolToPtr(false),
EventBufferSize: helper.IntToPtr(0),
DurableEventCount: helper.IntToPtr(0),
},
ACL: &ACLConfig{
Enabled: true,
@ -328,6 +331,9 @@ func TestConfig_Merge(t *testing.T) {
NonVotingServer: true,
RedundancyZone: "bar",
UpgradeVersion: "bar",
EnableEventBroker: helper.BoolToPtr(true),
DurableEventCount: helper.IntToPtr(100),
EventBufferSize: helper.IntToPtr(100),
},
ACL: &ACLConfig{
Enabled: true,
@ -1163,3 +1169,58 @@ func TestTelemetry_Parse(t *testing.T) {
require.Exactly([]string{"+nomad.raft"}, config.Telemetry.PrefixFilter)
require.True(config.Telemetry.DisableDispatchedJobSummaryMetrics)
}
func TestEventBroker_Parse(t *testing.T) {
require := require.New(t)
{
a := &ServerConfig{
EnableEventBroker: helper.BoolToPtr(false),
EventBufferSize: helper.IntToPtr(0),
DurableEventCount: helper.IntToPtr(0),
}
b := DefaultConfig().Server
b.EnableEventBroker = nil
b.EventBufferSize = nil
b.DurableEventCount = nil
result := a.Merge(b)
require.Equal(false, *result.EnableEventBroker)
require.Equal(0, *result.EventBufferSize)
require.Equal(0, *result.DurableEventCount)
}
{
a := &ServerConfig{
EnableEventBroker: helper.BoolToPtr(true),
EventBufferSize: helper.IntToPtr(5000),
DurableEventCount: helper.IntToPtr(200),
}
b := DefaultConfig().Server
b.EnableEventBroker = nil
b.EventBufferSize = nil
b.DurableEventCount = nil
result := a.Merge(b)
require.Equal(true, *result.EnableEventBroker)
require.Equal(5000, *result.EventBufferSize)
require.Equal(200, *result.DurableEventCount)
}
{
a := &ServerConfig{
EnableEventBroker: helper.BoolToPtr(false),
EventBufferSize: helper.IntToPtr(0),
DurableEventCount: helper.IntToPtr(0),
}
b := DefaultConfig().Server
b.EnableEventBroker = helper.BoolToPtr(true)
b.EventBufferSize = helper.IntToPtr(20000)
b.DurableEventCount = helper.IntToPtr(1000)
result := a.Merge(b)
require.Equal(true, *result.EnableEventBroker)
require.Equal(20000, *result.EventBufferSize)
require.Equal(1000, *result.DurableEventCount)
}
}

View file

@ -0,0 +1,160 @@
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/structs"
"golang.org/x/sync/errgroup"
)
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")
// Set region, namespace and authtoken to args
s.parse(resp, req, &args.QueryOptions.Region, &args.QueryOptions)
// 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 {
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)
// 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 {
return CodedError(500, err.Error())
}
for {
select {
case <-errCtx.Done():
return nil
default:
}
// Decode the response
var res structs.EventStreamWrapper
if err := decoder.Decode(&res); err != nil {
return CodedError(500, err.Error())
}
decoder.Reset(httpPipe)
if err := res.Error; err != nil {
if err.Code != nil {
return CodedError(int(*err.Code), err.Error())
}
}
// Flush json entry to response
if _, err := io.Copy(output, bytes.NewReader(res.Event.Data)); err != nil {
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 := errs.Wait()
if codedErr != nil && strings.Contains(codedErr.Error(), io.ErrClosedPipe.Error()) {
codedErr = nil
}
return nil, codedErr
}
func parseEventTopics(query url.Values) (map[structs.Topic][]string, error) {
raw, ok := query["topic"]
if !ok {
return allTopics(), nil
}
topics := make(map[structs.Topic][]string)
for _, topic := range raw {
k, v, err := parseTopic(topic)
if err != nil {
return nil, fmt.Errorf("error parsing topics: %w", err)
}
topics[structs.Topic(k)] = append(topics[structs.Topic(k)], v)
}
return topics, nil
}
func parseTopic(topic string) (string, string, error) {
parts := strings.Split(topic, ":")
// 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
}
func allTopics() map[structs.Topic][]string {
return map[structs.Topic][]string{"*": {"*"}}
}

View file

@ -0,0 +1,204 @@
package agent
import (
"context"
"fmt"
"net/http"
"net/http/httptest"
"net/url"
"strings"
"testing"
"time"
"github.com/hashicorp/nomad/helper/uuid"
"github.com/hashicorp/nomad/nomad/structs"
"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().EventBroker()
require.NoError(t, err)
pub.Publish(&structs.Events{Index: 100, Events: []structs.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
cancel()
select {
case err := <-respErrCh:
require.Nil(t, err)
case <-time.After(1 * time.Second):
require.Fail(t, "waiting for request cancellation")
}
})
}
func TestEventStream_NamespaceQuery(t *testing.T) {
t.Parallel()
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()
respErrCh := make(chan error)
go func() {
_, err = s.Server.EventStream(resp, req)
respErrCh <- err
assert.NoError(t, err)
}()
pub, err := s.Agent.server.State().EventBroker()
require.NoError(t, err)
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"`
if strings.Contains(got, badID) {
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) {
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()
cases := []struct {
desc string
query string
want map[structs.Topic][]string
wantErr bool
}{
{
desc: "all topics and keys specified",
query: "?topic=*:*",
want: map[structs.Topic][]string{
"*": {"*"},
},
},
{
desc: "all topics and keys inferred",
query: "",
want: map[structs.Topic][]string{
"*": {"*"},
},
},
{
desc: "invalid key value formatting",
query: "?topic=NodeDrain:*:*",
wantErr: true,
},
{
desc: "Infer wildcard if absent",
query: "?topic=NodeDrain",
wantErr: false,
want: map[structs.Topic][]string{
"NodeDrain": {"*"},
},
},
{
desc: "single topic and key",
query: "?topic=NodeDrain:*",
want: map[structs.Topic][]string{
"NodeDrain": {"*"},
},
},
{
desc: "single topic multiple keys",
query: "?topic=NodeDrain:*&topic=NodeDrain:3caace09-f1f4-4d23-b37a-9ab5eb75069d",
want: map[structs.Topic][]string{
"NodeDrain": {
"*",
"3caace09-f1f4-4d23-b37a-9ab5eb75069d",
},
},
},
{
desc: "multiple topics",
query: "?topic=NodeRegister:*&topic=NodeDrain:3caace09-f1f4-4d23-b37a-9ab5eb75069d",
want: map[structs.Topic][]string{
"NodeDrain": {
"3caace09-f1f4-4d23-b37a-9ab5eb75069d",
},
"NodeRegister": {
"*",
},
},
},
}
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)
})
}
}

View file

@ -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 {

View file

@ -130,6 +130,9 @@ server {
upgrade_version = "0.8.0"
encrypt = "abc"
raft_multiplier = 4
enable_event_broker = false
event_buffer_size = 200
durable_event_count = 0
server_join {
retry_join = ["1.1.1.1", "2.2.2.2"]

View file

@ -261,6 +261,9 @@
"data_dir": "/tmp/data",
"deployment_gc_threshold": "12h",
"enabled": true,
"enable_event_broker": false,
"event_buffer_size": 200,
"durable_event_count": 0,
"enabled_schedulers": [
"test"
],

4
go.mod
View file

@ -62,8 +62,8 @@ 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-memdb v1.2.1
github.com/hashicorp/go-immutable-radix v1.3.0
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

6
go.sum
View file

@ -392,9 +392,11 @@ 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-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.2.1 h1:wI9btDjYUOJJHTCnRlAG/TkRyD/ij7meJMrLK9X31Cc=
github.com/hashicorp/go-memdb v1.2.1/go.mod h1:OSvLJ662Jim8hMM+gWGyhktyWk2xPCnWMc7DWIqtkGA=
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=

View file

@ -193,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(4, []*structs.Allocation{alloc2}); err != nil {
if err := state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 4, []*structs.Allocation{alloc2}); err != nil {
t.Fatalf("err: %v", err)
}
})

View file

@ -78,6 +78,17 @@ type Config struct {
// in the absence of ACLs
EnableDebug bool
// EnableEventBroker is used to enable or disable state store
// event publishing
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
@ -413,6 +424,9 @@ func DefaultConfig() *Config {
ReplicationBackoff: 30 * time.Second,
SentinelGCInterval: 30 * time.Second,
LicenseConfig: &LicenseConfig{},
EnableEventBroker: true,
EventBufferSize: 100,
DurableEventCount: 100,
AutopilotConfig: &structs.AutopilotConfig{
CleanupDeadServers: true,
LastContactThreshold: 200 * time.Millisecond,

View file

@ -1300,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(6, []*structs.Allocation{a2}), "updateAllocsFromClient")
assert.Nil(state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 6, []*structs.Allocation{a2}), "updateAllocsFromClient")
})
req.MinQueryIndex = 4

View file

@ -917,7 +917,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(structs.MsgTypeTestSetup, m.nextIndex(), req), "UpsertDeploymentAllocHealth")
}
// Wait for there to be one eval
@ -945,7 +945,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(structs.MsgTypeTestSetup, m.nextIndex(), req2), "UpsertDeploymentAllocHealth")
// Wait for there to be one eval
testutil.WaitForResult(func() (bool, error) {
@ -1024,7 +1024,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(structs.MsgTypeTestSetup, 100, []*structs.Allocation{a2}))
// Wait for the deployment to be failed
testutil.WaitForResult(func() (bool, error) {
@ -1208,7 +1208,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(structs.MsgTypeTestSetup, m.nextIndex(), []*structs.Allocation{a2}))
// Wait for the deployment to cross the deadline
dout, err := m.state.DeploymentByID(nil, d.ID)
@ -1381,7 +1381,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(structs.MsgTypeTestSetup, m.nextIndex(), []*structs.Allocation{a2}))
// Wait for the alloc's DesiredState to set reschedule
testutil.WaitForResult(func() (bool, error) {
@ -1453,7 +1453,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(structs.MsgTypeTestSetup, m.nextIndex(), req), "UpsertDeploymentAllocHealth")
}
// Wait for there to be one eval
@ -1481,7 +1481,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(structs.MsgTypeTestSetup, m.nextIndex(), req2), "UpsertDeploymentAllocHealth")
// Wait for there to be one eval
testutil.WaitForResult(func() (bool, error) {
@ -1562,7 +1562,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(structs.MsgTypeTestSetup, m.nextIndex(), req), "UpsertDeploymentAllocHealth")
req2 := &structs.ApplyDeploymentAllocHealthRequest{
DeploymentAllocHealthRequest: structs.DeploymentAllocHealthRequest{
@ -1570,7 +1570,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(structs.MsgTypeTestSetup, m.nextIndex(), req2), "UpsertDeploymentAllocHealth")
// Wait for there to be one eval for each job
testutil.WaitForResult(func() (bool, error) {

View file

@ -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
@ -95,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(i, u)
return i, m.state.UpdateDeploymentStatus(structs.MsgTypeTestSetup, i, u)
}
// matchDeploymentStatusUpdateConfig is used to configure the matching
@ -149,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(i, req)
return i, m.state.UpdateDeploymentPromotion(structs.MsgTypeTestSetup, i, req)
}
// matchDeploymentPromoteRequestConfig is used to configure the matching
@ -179,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(i, req)
return i, m.state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, i, req)
}
// matchDeploymentAllocHealthRequestConfig is used to configure the matching

View file

@ -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(structs.MsgTypeTestSetup, 1000, updates))
// Check that the node drain is removed
testutil.WaitForResult(func() (bool, error) {

View file

@ -374,7 +374,7 @@ func TestEvalEndpoint_Dequeue_UpdateWaitIndex(t *testing.T) {
EvalID: eval.ID,
}
assert := assert.New(t)
err := state.UpsertPlanResults(1000, &res)
err := state.UpsertPlanResults(structs.MsgTypeTestSetup, 1000, &res)
assert.Nil(err)
// Dequeue the eval

View file

@ -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) {}

242
nomad/event_endpoint.go Normal file
View file

@ -0,0 +1,242 @@
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/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
}
aclObj, err := e.srv.ResolveToken(args.AuthToken)
if err != nil {
handleJsonResultError(err, nil, encoder)
return
}
subReq := &stream.SubscribeRequest{
Token: args.AuthToken,
Topics: args.Topics,
Index: uint64(args.Index),
Namespace: args.Namespace,
}
// 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
}
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()
errCh := make(chan error)
jsonStream := stream.NewJsonStream(ctx, 30*time.Second)
// goroutine to detect remote side closing
go func() {
io.Copy(ioutil.Discard, conn)
cancel()
}()
go func() {
defer cancel()
for {
events, err := subscription.Next(ctx)
if err != nil {
select {
case errCh <- err:
case <-ctx.Done():
}
return
}
// Continue if there are no events
if len(events.Events) == 0 {
continue
}
if err := jsonStream.Send(events); err != nil {
select {
case errCh <- err:
case <-ctx.Done():
}
return
}
}
}()
var streamErr error
OUTER:
for {
select {
case streamErr = <-errCh:
break OUTER
case <-ctx.Done():
break OUTER
case eventJSON, ok := <-jsonStream.OutCh():
// 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),
})
}
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
}

View file

@ -0,0 +1,510 @@
package nomad
import (
"encoding/json"
"fmt"
"io"
"net"
"strings"
"testing"
"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"
"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.EnableEventBroker = true
})
defer cleanupS1()
// Create request for all topics and keys
req := structs.EventStreamRequest{
Topics: map[structs.Topic][]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)
// decode request responses
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().EventBroker()
require.NoError(t, err)
node := mock.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))
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 {
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.JsonHeartbeat {
continue
}
var event structs.Events
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.Events[0].Payload)
require.NoError(t, err)
require.Equal(t, node.ID, out.ID)
got++
if got == want {
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.EnableEventBroker = true
})
defer cleanupS1()
testutil.WaitForLeader(t, s1.RPC)
req := structs.EventStreamRequest{
Topics: map[structs.Topic][]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().EventBroker()
require.NoError(t, err)
node := mock.Node()
// send req
encoder := codec.NewEncoder(p1, structs.MsgpackHandle)
require.Nil(t, encoder.Encode(req))
// 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:
for {
select {
case <-timeout:
t.Fatal("timeout waiting for event stream")
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 {
// 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.EnableEventBroker = true
})
defer cleanupS1()
s2, cleanupS2 := TestServer(t, func(c *Config) {
c.EnableEventBroker = true
c.Region = "foo"
})
defer cleanupS2()
TestJoin(t, s1, s2)
// Create request targed for region foo
req := structs.EventStreamRequest{
Topics: map[structs.Topic][]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().EventBroker()
require.NoError(t, err)
node := mock.Node()
publisher.Publish(&structs.Events{Index: uint64(1), Events: []structs.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.JsonHeartbeat {
continue
}
var event structs.Events
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.Events[0].Payload)
require.NoError(t, err)
require.Equal(t, node.ID, out.ID)
break OUTER
}
}
}
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)
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: "",
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: "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: "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: tc.Topics,
QueryOptions: structs.QueryOptions{
Region: s.Region(),
Namespace: ns,
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().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:
t.Fatal("timeout waiting for events")
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 {
t.Fatalf("unexpected error %v", msg.Error)
}
}
}
})
}
}

View file

@ -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"
@ -52,6 +53,7 @@ const (
CSIPluginSnapshot
CSIVolumeSnapshot
ScalingEventsSnapshot
EventSnapshot
)
// LogApplier is the definition of a function that can apply a Raft log
@ -99,8 +101,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 int64
}
// snapshotHeader is the first entry in our snapshot
@ -125,14 +128,29 @@ type FSMConfig struct {
// Region is the region of the server embedding the FSM
Region string
// 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 int64
}
// 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.EnableEventBroker,
EventBufferSize: config.EventBufferSize,
DurableEventCount: config.DurableEventCount,
}
state, err := state.NewStateStore(sconfig)
if err != nil {
@ -162,6 +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.StopEventBroker()
return nil
}
@ -195,25 +214,25 @@ 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)
return n.applyStatusUpdate(msgType, 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)
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(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)
return n.applyAllocUpdate(msgType, 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:
@ -221,13 +240,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:
@ -245,15 +264,15 @@ 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)
return n.applyBatchDeregisterJob(msgType, buf[1:], log.Index)
case structs.AllocUpdateDesiredTransitionRequestType:
return n.applyAllocUpdateDesiredTransition(buf[1:], log.Index)
return n.applyAllocUpdateDesiredTransition(msgType, buf[1:], log.Index)
case structs.NodeUpdateEligibilityRequestType:
return n.applyNodeEligibilityUpdate(buf[1:], log.Index)
return n.applyNodeEligibilityUpdate(msgType, buf[1:], log.Index)
case structs.BatchNodeUpdateDrainRequestType:
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:
@ -310,7 +329,7 @@ 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 {
@ -320,7 +339,7 @@ func (n *nomadFSM) applyUpsertNode(buf []byte, index uint64) interface{} {
// Handle upgrade paths
req.Node.Canonicalize()
if err := n.state.UpsertNode(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
}
@ -334,14 +353,14 @@ 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 {
if err := n.state.DeleteNodeMsgType(reqType, index, []string{req.NodeID}); err != nil {
n.logger.Error("DeleteNode failed", "error", err)
return err
}
@ -364,14 +383,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
}
@ -393,7 +412,7 @@ 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 {
@ -414,28 +433,28 @@ 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.UpdateNodeDrainMsgType(reqType, index, req.NodeID, req.DrainStrategy, req.MarkEligible, req.UpdatedAt, req.NodeEvent); err != nil {
n.logger.Error("UpdateNodeDrain failed", "error", err)
return err
}
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 {
@ -449,7 +468,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
}
@ -465,7 +484,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 {
@ -481,7 +500,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
}
@ -559,7 +578,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(msgType, index, []*structs.Evaluation{req.Eval}); err != nil {
return err
}
}
@ -567,14 +586,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 {
@ -591,7 +610,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(msgType, index, []*structs.Evaluation{req.Eval}); err != nil {
return err
}
}
@ -603,7 +622,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 {
@ -613,7 +632,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)
@ -678,17 +697,18 @@ 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)
return n.upsertEvals(msgType, 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(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
}
@ -737,7 +757,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 {
@ -768,14 +788,14 @@ 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
}
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 {
@ -797,14 +817,14 @@ func (n *nomadFSM) applyAllocClientUpdate(buf []byte, index uint64) interface{}
}
// Update all the client allocations
if err := n.state.UpdateAllocsFromClient(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(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
}
@ -840,14 +860,14 @@ func (n *nomadFSM) applyAllocClientUpdate(buf []byte, index uint64) interface{}
// 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
}
@ -865,14 +885,14 @@ 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 {
if err := n.state.UpsertNodeEventsMsgType(msgType, index, req.NodeEvents); err != nil {
n.logger.Error("failed to add node events", "error", err)
return err
}
@ -944,14 +964,14 @@ 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 {
if err := n.state.UpsertPlanResults(msgType, index, &req); err != nil {
n.logger.Error("ApplyPlan failed", "error", err)
return err
}
@ -963,14 +983,14 @@ 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 {
if err := n.state.UpdateDeploymentStatus(msgType, index, &req); err != nil {
n.logger.Error("UpsertDeploymentStatusUpdate failed", "error", err)
return err
}
@ -980,14 +1000,14 @@ 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 {
if err := n.state.UpdateDeploymentPromotion(msgType, index, &req); err != nil {
n.logger.Error("UpsertDeploymentPromotion failed", "error", err)
return err
}
@ -998,14 +1018,14 @@ 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 {
if err := n.state.UpdateDeploymentAllocHealth(msgType, index, &req); err != nil {
n.logger.Error("UpsertDeploymentAllocHealth failed", "error", err)
return err
}
@ -1245,8 +1265,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
}
@ -1256,8 +1277,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.EnableEventBroker,
EventBufferSize: n.config.EventBufferSize,
DurableEventCount: n.config.DurableEventCount,
}
newState, err := state.NewStateStore(config)
if err != nil {
@ -1501,7 +1525,20 @@ func (n *nomadFSM) Restore(old io.ReadCloser) error {
if err := restore.CSIVolumeRestore(plugin); err != nil {
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
}
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]
@ -1516,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
@ -1536,6 +1575,43 @@ 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
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)
}
}
return nil
}
// 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
}
events, err := s.Events(nil)
if err != nil {
return err
}
count := 0
for {
raw := events.Next()
if raw == nil {
break
}
e := raw.(*structs.Events)
pub.Publish(e)
count++
}
l.Debug("finished hydrating event broker from snapshot", "events", count)
return nil
}
@ -1813,6 +1889,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
}
@ -2316,6 +2396,42 @@ func (s *nomadSnapshot) persistCSIVolumes(sink raft.SnapshotSink,
return nil
}
func (s *nomadSnapshot) persistEvents(sink raft.SnapshotSink, encoder *codec.Encoder) error {
if s.durableEventCount == 0 {
return nil
}
events, err := s.snap.LatestEventsReverse(nil)
if err != nil {
return err
}
var count int64
for {
// Get the next item
raw := events.Next()
if raw == nil {
break
}
// Prepare the request struct
event := raw.(*structs.Events)
// Write out a volume snapshot
sink.Write([]byte{byte(EventSnapshot)})
if err := encoder.Encode(event); err != nil {
return err
}
count += int64(len(event.Events))
// Only write to sink until durableCount has been reached
if count >= s.durableEventCount {
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.

View file

@ -50,11 +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",
EvalBroker: broker,
Periodic: dispatcher,
Blocked: NewBlockedEvals(broker, logger),
Logger: logger,
Region: "global",
EnableEventBroker: true,
EventBufferSize: 100,
}
fsm, err := NewFSM(fsmConfig)
if err != nil {
@ -3199,3 +3201,87 @@ 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)
fsm.config.EnableEventBroker = true
// DurableEventCount = 4 each mock events wrapper contains 2 events
fsm.config.DurableEventCount = 4
state := fsm.State()
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)
pub, err := state2.EventBroker()
require.NoError(t, err)
testutil.WaitForResult(func() (bool, error) {
plen := pub.Len()
if plen == 4 {
return true, nil
}
return false, fmt.Errorf("expected publisher to have len 2 got: %d", plen)
}, func(err error) {
require.Fail(t, err.Error())
})
}
func TestFSM_SnapshotRestore_Events_NoDurability(t *testing.T) {
t.Parallel()
fsm := testFSM(t)
// Enable event publisher with durable event count of zero
fsm.config.EnableEventBroker = true
fsm.config.DurableEventCount = 0
state := fsm.State()
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)
}

View file

@ -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(),
},
},
}
}

View file

@ -2048,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(200, []*structs.Allocation{allocUpdate})
err := state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 200, []*structs.Allocation{allocUpdate})
if err != nil {
t.Fatalf("err: %v", err)
}
@ -2747,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

View file

@ -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(structs.ApplyPlanResultsRequestType, nextIdx, &req); err != nil {
return future, err
}
}

View file

@ -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,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(),
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)

View file

@ -0,0 +1,116 @@
package state
import (
"context"
"testing"
"time"
"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.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 WaitForEvents(t *testing.T, s *StateStore, index uint64, minEvents int, timeout time.Duration) []structs.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) []structs.Event {
pub, err := s.EventBroker()
require.NoError(t, err)
sub, err := pub.Subscribe(&stream.SubscribeRequest{
Topics: map[structs.Topic][]string{
"*": {"*"},
},
Index: index,
StartExactlyAtIndex: true,
})
if err != nil {
return []structs.Event{}
}
defer sub.Unsubscribe()
require.NoError(t, err)
var events []structs.Event
for {
e, err := sub.NextNoBlock()
require.NoError(t, err)
if e == nil {
break
}
events = append(events, e...)
}
return events
}

221
nomad/state/events.go Normal file
View file

@ -0,0 +1,221 @@
package state
import (
"fmt"
"github.com/hashicorp/nomad/nomad/structs"
)
const (
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"
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
}
// 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
}
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) {
eventType, ok := MsgTypeEvents[changes.MsgType]
if !ok {
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 nil, fmt.Errorf("transaction change was not an Evaluation")
}
event := structs.Event{
Topic: structs.TopicEval,
Type: eventType,
Index: changes.Index,
Key: after.ID,
Namespace: after.Namespace,
Payload: &EvalEvent{
Eval: after,
},
}
events = append(events, event)
case "allocs":
if change.Deleted() {
return nil, nil
}
after, ok := change.After.(*structs.Allocation)
if !ok {
return nil, fmt.Errorf("transaction change was not an Allocation")
}
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: structs.TopicAlloc,
Type: eventType,
Index: changes.Index,
Key: after.ID,
FilterKeys: filterKeys,
Namespace: after.Namespace,
Payload: &AllocEvent{
Alloc: alloc,
},
}
events = append(events, event)
case "jobs":
if change.Deleted() {
return nil, nil
}
after, ok := change.After.(*structs.Job)
if !ok {
return nil, fmt.Errorf("transaction change was not an Allocation")
}
event := structs.Event{
Topic: structs.TopicJob,
Type: eventType,
Index: changes.Index,
Key: after.ID,
Namespace: after.Namespace,
Payload: &JobEvent{
Job: after,
},
}
events = append(events, event)
case "nodes":
if change.Deleted() {
return nil, nil
}
after, ok := change.After.(*structs.Node)
if !ok {
return nil, fmt.Errorf("transaction change was not a Node")
}
event := structs.Event{
Topic: structs.TopicNode,
Type: eventType,
Index: changes.Index,
Key: after.ID,
Payload: &NodeEvent{
Node: after,
},
}
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: structs.TopicDeployment,
Type: eventType,
Index: changes.Index,
Key: after.ID,
Namespace: after.Namespace,
FilterKeys: []string{after.JobID},
Payload: &DeploymentEvent{
Deployment: after,
},
}
events = append(events, event)
}
}
return &structs.Events{Index: changes.Index, Events: events}, nil
}

567
nomad/state/events_test.go Normal file
View file

@ -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()
}

View file

@ -0,0 +1,82 @@
package state
import (
"fmt"
"github.com/hashicorp/nomad/nomad/structs"
)
// NodeDeregisterEventFromChanges generates a NodeDeregistrationEvent from a set
// of transaction changes.
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")
}
event := structs.Event{
Topic: structs.TopicNode,
Type: TypeNodeDeregistration,
Index: changes.Index,
Key: before.ID,
Payload: &NodeEvent{
Node: before,
},
}
events = append(events, event)
}
}
return &structs.Events{Index: changes.Index, Events: events}, nil
}
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")
}
// 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 := structs.Event{
Topic: structs.TopicNode,
Type: TypeNodeDrain,
Index: changes.Index,
Key: after.ID,
Payload: &NodeDrainEvent{
Node: after,
JobAllocs: jobAllocs,
},
}
events = append(events, event)
}
}
return &structs.Events{Index: changes.Index, Events: events}, nil
}

View file

@ -0,0 +1,321 @@
package state
import (
"testing"
"time"
"github.com/hashicorp/nomad/nomad/mock"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/stretchr/testify/require"
)
func TestNodeEventsFromChanges(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 []structs.Event
WantErr bool
WantTopic structs.Topic
}{
{
MsgType: structs.NodeRegisterRequestType,
WantTopic: structs.TopicNode,
Name: "node registered",
Mutate: func(s *StateStore, tx *txn) error {
return upsertNodeTxn(tx, tx.Index, testNode())
},
WantEvents: []structs.Event{{
Topic: structs.TopicNode,
Type: TypeNodeRegistration,
Key: testNodeID(),
Index: 100,
Payload: &NodeEvent{
Node: testNode(),
},
}},
WantErr: false,
},
{
MsgType: structs.NodeRegisterRequestType,
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: structs.TopicNode,
Type: TypeNodeRegistration,
Key: testNodeID(),
Index: 100,
Payload: &NodeEvent{
Node: testNode(nodeNotReady),
},
}},
WantErr: false,
},
{
MsgType: structs.NodeDeregisterRequestType,
WantTopic: structs.TopicNode,
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: []structs.Event{{
Topic: structs.TopicNode,
Type: TypeNodeDeregistration,
Key: testNodeID(),
Index: 100,
Payload: &NodeEvent{
Node: testNode(),
},
}},
WantErr: false,
},
{
MsgType: structs.NodeDeregisterRequestType,
WantTopic: structs.TopicNode,
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: []structs.Event{
{
Topic: structs.TopicNode,
Type: TypeNodeDeregistration,
Key: testNodeID(),
Index: 100,
Payload: &NodeEvent{
Node: testNode(),
},
},
{
Topic: structs.TopicNode,
Type: TypeNodeDeregistration,
Key: testNodeIDTwo(),
Index: 100,
Payload: &NodeEvent{
Node: testNode(nodeIDTwo),
},
},
},
WantErr: false,
},
{
MsgType: structs.UpsertNodeEventsType,
WantTopic: structs.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: []structs.Event{
{
Topic: structs.TopicNode,
Type: TypeNodeEvent,
Key: testNodeID(),
Index: 100,
Payload: &NodeEvent{
Node: testNode(),
},
},
{
Topic: structs.TopicNode,
Type: TypeNodeEvent,
Key: testNodeIDTwo(),
Index: 100,
Payload: &NodeEvent{
Node: testNode(nodeIDTwo),
},
},
},
WantErr: false,
},
}
for _, tc := range cases {
t.Run(tc.Name, func(t *testing.T) {
s := TestStateStoreCfg(t, TestStateStorePublisher(t))
defer s.StopEventBroker()
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.WriteTxnMsgT(tc.MsgType, 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.NotNil(t, 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]
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.StopEventBroker()
// 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.Events, 1)
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)
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 structs.Event) {
t.Helper()
wantPayload := want.Payload.(*NodeEvent)
gotPayload := got.Payload.(*NodeEvent)
// Check payload equality for the fields that we can easily control
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 structs.Event) {
t.Helper()
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)
}
func requireNodeEventEqual(t *testing.T, want, got structs.Event) {
gotPayload := got.Payload.(*NodeEvent)
require.Len(t, gotPayload.Node.Events, 3)
}
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"
}

View file

@ -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",
},
},
},
}
}

View file

@ -1,6 +1,7 @@
package state
import (
"github.com/hashicorp/nomad/nomad/structs"
"testing"
memdb "github.com/hashicorp/go-memdb"
@ -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 *structs.Events) {
txn := db.Txn(true)
err := txn.Insert(eventsTable, e)
require.NoError(err)
txn.Commit()
}
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.(*structs.Events)
require.True(ok)
return s
}
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)
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)
}

View file

@ -4,7 +4,8 @@ import (
"fmt"
"github.com/hashicorp/go-memdb"
"github.com/hashicorp/nomad/nomad/event"
"github.com/hashicorp/nomad/nomad/stream"
"github.com/hashicorp/nomad/nomad/structs"
)
// ReadTxn is implemented by memdb.Txn to perform read operations.
@ -21,36 +22,31 @@ 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
// 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
publisher eventPublisher
processChanges func(ReadTxn, Changes) ([]event.Event, error)
memdb *memdb.MemDB
durableCount int64
publisher *stream.EventBroker
processChanges func(ReadTxn, Changes) (*structs.Events, error)
}
func NewChangeTrackerDB(db *memdb.MemDB, publisher eventPublisher, changesFn changeProcessor) *changeTrackerDB {
func NewChangeTrackerDB(db *memdb.MemDB, publisher *stream.EventBroker, changesFn changeProcessor, durableCount int64) *changeTrackerDB {
return &changeTrackerDB{
db: db,
publisher: event.NewPublisher(),
memdb: db,
publisher: publisher,
processChanges: changesFn,
durableCount: durableCount,
}
}
type changeProcessor func(ReadTxn, Changes) ([]event.Event, error)
type changeProcessor func(ReadTxn, Changes) (*structs.Events, error)
type eventPublisher interface {
Publish(events []event.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 noOpProcessChanges(ReadTxn, Changes) (*structs.Events, error) { return nil, nil }
// ReadTxn returns a read-only transaction which behaves exactly the same as
// memdb.Txn
@ -58,7 +54,7 @@ func noOpProcessChanges(ReadTxn, Changes) ([]event.Event, error) { return []even
// 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.
@ -73,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,
}
@ -81,16 +77,33 @@ func (c *changeTrackerDB) WriteTxn(idx uint64) *txn {
return t
}
func (c *changeTrackerDB) publish(changes Changes) error {
readOnlyTx := c.db.Txn(false)
func (c *changeTrackerDB) WriteTxnMsgT(msgType structs.MessageType, idx uint64) *txn {
persistChanges := c.durableCount > 0
t := &txn{
msgType: msgType,
Txn: c.memdb.Txn(true),
Index: idx,
publish: c.publish,
persistChanges: persistChanges,
}
t.Txn.TrackChanges()
return t
}
func (c *changeTrackerDB) publish(changes Changes) (*structs.Events, error) {
readOnlyTx := c.memdb.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 nil, fmt.Errorf("failed generating events from changes: %v", err)
}
c.publisher.Publish(events)
return nil
if events != nil {
c.publisher.Publish(events)
}
return events, nil
}
// WriteTxnRestore returns a wrapped RW transaction that does NOT have change
@ -101,28 +114,33 @@ func (c *changeTrackerDB) publish(changes Changes) 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
// 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 {
// 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
// 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
@ -136,17 +154,43 @@ func (tx *txn) Commit() error {
changes := Changes{
Index: tx.Index,
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 && events != nil {
// persist events after processing changes
err := tx.Txn.Insert("events", events)
if err != nil {
return err
}
}
}
tx.Txn.Commit()
return nil
}
func processDBChanges(tx ReadTxn, changes Changes) ([]event.Event, error) {
// TODO: add handlers here.
return []event.Event{}, 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 {
return tx.msgType
}
func processDBChanges(tx ReadTxn, changes Changes) (*structs.Events, error) {
switch changes.MsgType {
case structs.IgnoreUnknownTypeFlag:
// unknown event type
return nil, nil
case structs.NodeDeregisterRequestType:
return NodeDeregisterEventFromChanges(tx, changes)
case structs.NodeUpdateDrainRequestType:
return NodeDrainEventFromChanges(tx, changes)
default:
return GenericEventsFromChanges(tx, changes)
}
}

View file

@ -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"
)
@ -45,6 +45,16 @@ 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
// 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 int64
}
// The StateStore is responsible for maintaining all the Nomad
@ -64,6 +74,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.
stopEventBroker func()
}
// NewStateStore is used to create a new state store
@ -75,12 +89,25 @@ 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{}),
logger: config.Logger.Named("state_store"),
config: config,
abandonCh: make(chan struct{}),
stopEventBroker: cancel,
}
if config.EnablePublisher {
// Create new event publisher using provided config
broker := stream.NewEventBroker(ctx, stream.EventBrokerCfg{
EventBufferSize: config.EventBufferSize,
Logger: config.Logger,
OnEvict: s.eventBrokerEvict,
})
s.db = NewChangeTrackerDB(db, broker, processDBChanges, config.DurableEventCount)
} else {
s.db = NewChangeTrackerDB(db, nil, noOpProcessChanges, 0)
}
s.db = NewChangeTrackerDB(db, event.NewPublisher(), processDBChanges)
// Initialize the state store with required enterprise objects
if err := s.enterpriseInit(); err != nil {
@ -90,6 +117,37 @@ func NewStateStore(config *StateStoreConfig) (*StateStore, error) {
return s, nil
}
func (s *StateStore) EventBroker() (*stream.EventBroker, error) {
if s.db.publisher == nil {
return nil, fmt.Errorf("EventBroker not configured")
}
return s.db.publisher, nil
}
// eventBrokerEvict is used as a callback to delete an evicted events
// entry from go-memdb.
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)
} else {
s.logger.Error("Error deleting event from events table", "error", err)
}
}
}
func (s *StateStore) deleteEvent(events *structs.Events) error {
txn := s.db.memdb.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
@ -99,14 +157,15 @@ 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,
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, 0)
snap := &StateSnapshot{
StateStore: store,
@ -189,9 +248,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.StopEventBroker()
close(s.abandonCh)
}
// StopStopEventBroker calls the cancel func for the state stores event
// publisher. It should be called during server shutdown.
func (s *StateStore) StopEventBroker() {
s.stopEventBroker()
}
// 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)
@ -237,7 +303,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(msgType structs.MessageType, index uint64, results *structs.ApplyPlanResultsRequest) error {
snapshot, err := s.Snapshot()
if err != nil {
return err
@ -259,7 +325,7 @@ func (s *StateStore) UpsertPlanResults(index uint64, results *structs.ApplyPlanR
return err
}
txn := s.db.WriteTxn(index)
txn := s.db.WriteTxnMsgT(msgType, index)
defer txn.Abort()
// Upsert the newly created or updated deployment
@ -323,8 +389,7 @@ func (s *StateStore) UpsertPlanResults(index uint64, results *structs.ApplyPlanR
}
}
txn.Commit()
return nil
return txn.Commit()
}
// addComputedAllocAttrs adds the computed/derived attributes to the allocation.
@ -393,8 +458,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
@ -410,8 +474,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
@ -422,8 +485,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 {
@ -635,8 +697,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.
@ -687,8 +748,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
@ -722,6 +782,21 @@ func (s *StateStore) ScalingEventsByJob(ws memdb.WatchSet, namespace, jobID stri
return nil, 0, nil
}
// 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()
err := upsertNodeTxn(txn, index, node)
if err != nil {
return nil
}
return txn.Commit()
}
// 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.
@ -729,6 +804,14 @@ 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
}
return txn.Commit()
}
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 {
@ -777,19 +860,38 @@ func (s *StateStore) UpsertNode(index uint64, node *structs.Node) error {
return fmt.Errorf("csi plugin update failed: %v", err)
}
txn.Commit()
return nil
}
// DeleteNode deregisters a batch of nodes
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
}
return txn.Commit()
}
// 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
}
return txn.Commit()
}
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 {
@ -814,21 +916,19 @@ func (s *StateStore) DeleteNode(index uint64, nodes []string) error {
return fmt.Errorf("index update failed: %v", err)
}
txn.Commit()
return nil
}
// 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 {
@ -867,16 +967,27 @@ 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 {
txn := s.db.WriteTxn(index)
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.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 {
return err
}
}
txn.Commit()
return nil
return txn.Commit()
}
// UpdateNodeDrain is used to update the drain of a node
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.WriteTxnMsgT(msgType, index)
defer txn.Abort()
if err := s.updateNodeDrainImpl(txn, index, nodeID, drain, markEligible, updatedAt, event); err != nil {
return err
}
return txn.Commit()
}
// UpdateNodeDrain is used to update the drain of a node
@ -888,8 +999,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,
@ -937,9 +1047,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
@ -978,8 +1088,20 @@ func (s *StateStore) UpdateNodeEligibility(index uint64, nodeID string, eligibil
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 {
txn := s.db.WriteTxnMsgT(msgType, index)
defer txn.Abort()
for nodeID, events := range nodeEvents {
if err := s.upsertNodeEvents(index, nodeID, events, txn); err != nil {
return err
}
}
return txn.Commit()
}
// UpsertNodeEvents adds the node events to the nodes, rotating events as
@ -994,8 +1116,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
@ -1381,8 +1502,17 @@ 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
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,
@ -1483,7 +1613,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
}
@ -2006,8 +2136,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
@ -2196,8 +2325,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
@ -2239,8 +2367,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
@ -2451,8 +2578,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.
@ -2482,8 +2608,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.
@ -2514,8 +2639,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
@ -2525,7 +2649,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
}
@ -2593,7 +2717,19 @@ 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
}
// UpsertEvals is used to upsert a set of evaluations
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 {
return txn.Commit()
}
return err
}
@ -2791,8 +2927,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
@ -2910,8 +3045,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(msgType structs.MessageType, index uint64, allocs []*structs.Allocation) error {
txn := s.db.WriteTxnMsgT(msgType, index)
defer txn.Abort()
// Handle each of the updated allocations
@ -2926,8 +3061,7 @@ func (s *StateStore) UpdateAllocsFromClient(index uint64, allocs []*structs.Allo
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
@ -3026,8 +3160,18 @@ 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
// 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
@ -3153,10 +3297,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
@ -3177,8 +3321,7 @@ func (s *StateStore) UpdateAllocsDesiredTransitions(index uint64, allocs map[str
return fmt.Errorf("index update failed: %v", err)
}
txn.Commit()
return nil
return txn.Commit()
}
// nestedUpdateAllocDesiredTransition is used to nest an update of an
@ -3298,6 +3441,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)
@ -3483,8 +3630,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
@ -3504,8 +3650,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
@ -3615,8 +3760,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.
@ -3637,8 +3781,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.
@ -3719,8 +3862,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(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 {
@ -3741,8 +3884,7 @@ func (s *StateStore) UpdateDeploymentStatus(index uint64, req *structs.Deploymen
}
}
txn.Commit()
return nil
return txn.Commit()
}
// updateDeploymentStatusImpl is used to make deployment status updates
@ -3794,8 +3936,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
@ -3823,8 +3964,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(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
@ -3960,14 +4101,13 @@ func (s *StateStore) UpdateDeploymentPromotion(index uint64, req *structs.ApplyD
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(index uint64, req *structs.ApplyDeploymentAllocHealthRequest) error {
txn := s.db.WriteTxn(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
@ -4056,8 +4196,7 @@ func (s *StateStore) UpdateDeploymentAllocHealth(index uint64, req *structs.Appl
}
}
txn.Commit()
return nil
return txn.Commit()
}
// LastIndex returns the greatest index value for all indexes
@ -4268,8 +4407,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.
@ -4957,8 +5095,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
@ -4975,8 +5112,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
@ -5064,8 +5200,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
@ -5082,8 +5217,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
@ -5217,8 +5351,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.
@ -5247,8 +5380,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) {
@ -5277,20 +5409,19 @@ 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,
// 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
}
@ -5318,7 +5449,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
}
@ -5375,8 +5508,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.
@ -5445,7 +5577,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
@ -5621,6 +5753,48 @@ 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
}
return txn.Commit()
}
// StateSnapshot is used to provide a point-in-time snapshot
type StateSnapshot struct {
StateStore
@ -5722,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
@ -5868,6 +6042,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)

View file

@ -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.EventBroker()
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.EventBroker()
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)
})
}

View file

@ -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(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(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(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(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(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(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()
@ -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")
}
@ -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)
@ -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)
@ -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)
@ -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(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(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(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(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(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(1003, []*structs.Allocation{allocCopy1}); err != nil {
if err := state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1003, []*structs.Allocation{allocCopy1}); err != nil {
t.Fatalf("err: %v", err)
}
@ -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) {
@ -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(structs.MsgTypeTestSetup, 920, []*structs.Allocation{alloc})
alloc3 := alloc.Copy()
alloc3.ClientStatus = structs.AllocClientStatusRunning
alloc3.DesiredStatus = ""
state.UpdateAllocsFromClient(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(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(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(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(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(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(400, []*structs.Allocation{alloc1}); err != nil {
if err := state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 400, []*structs.Allocation{alloc1}); err != nil {
t.Fatalf("expect err: %v", err)
}
@ -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)
@ -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(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(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(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(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(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(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(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(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(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(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(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(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(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(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(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(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(3, req)
err := state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, 3, req)
if err != nil {
t.Fatalf("bad: %v", err)
}
@ -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,

View file

@ -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.

View file

@ -0,0 +1,301 @@
package stream
import (
"context"
"errors"
"fmt"
"sync/atomic"
"time"
"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
// 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 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
// 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 purne that mutate the head must be externally
// synchronized. This allows systems that already serialize writes to append
// without lock overhead.
type eventBuffer struct {
size *int64
head atomic.Value
tail atomic.Value
maxSize int64
onEvict EvictCallbackFn
}
// newEventBuffer creates an eventBuffer ready for use.
func newEventBuffer(size int64, onEvict EvictCallbackFn) *eventBuffer {
zero := int64(0)
b := &eventBuffer{
maxSize: size,
size: &zero,
onEvict: onEvict,
}
item := newBufferItem(&structs.Events{Index: 0, Events: 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 calls.
func (b *eventBuffer) Append(events *structs.Events) {
b.appendItem(newBufferItem(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
atomic.AddInt64(b.size, int64(len(item.Events.Events)))
// Advance Head until we are under allowable size
for atomic.LoadInt64(b.size) > b.maxSize {
b.advanceHead()
}
// notify waiters next event is available
close(oldTail.link.ch)
}
func newSentinelItem() *bufferItem {
return newBufferItem(&structs.Events{})
}
// 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)
// 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
// 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.Events.Index {
return item, int(item.Events.Index) - int(index)
}
if item.Events.Index == index {
return item, 0
}
for {
prev := item
item = item.NextNoBlock()
if item == nil {
return prev, int(index) - int(prev.Events.Index)
}
if index < item.Events.Index {
return item, int(item.Events.Index) - int(index)
}
if index == item.Events.Index {
return item, 0
}
}
}
// Len returns the current length of the buffer
func (b *eventBuffer) Len() int {
return int(atomic.LoadInt64(b.size))
}
// 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 *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
// 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(events *structs.Events) *bufferItem {
return &bufferItem{
link: &bufferLink{
ch: make(chan struct{}),
droppedCh: make(chan struct{}),
},
Events: events,
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
// 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")
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)
}

View file

@ -0,0 +1,245 @@
package stream
import (
"context"
"fmt"
"math/rand"
"testing"
"time"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
)
func TestEventBufferFuzz(t *testing.T) {
nReaders := 1000
nMessages := 1000
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
// 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 := structs.Event{
Index: uint64(i), // Indexes should be contiguous
}
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)
}
}()
// 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.Events[0].Index != expect {
errCh <- fmt.Errorf("subscriber %05d got bad event want=%d, got=%d", i,
expect, item.Events.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, nil)
for i := 0; i < 10; i++ {
e := structs.Event{
Index: uint64(i), // Indexes should be contiguous
}
b.Append(&structs.Events{Index: uint64(i), Events: []structs.Event{e}})
}
head := b.Head()
for i := 10; i < 15; i++ {
e := structs.Event{
Index: uint64(i), // Indexes should be contiguous
}
b.Append(&structs.Events{Index: uint64(i), Events: []structs.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, 5, int(newHead.Events.Index))
}
func TestEventBuffer_Size(t *testing.T) {
b := newEventBuffer(100, nil)
for i := 0; i < 10; i++ {
e := structs.Event{
Index: uint64(i), // Indexes should be contiguous
}
b.Append(&structs.Events{Index: uint64(i), Events: []structs.Event{e}})
}
require.Equal(t, 10, b.Len())
}
// 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{
Index: uint64(i), // Indexes should be contiguous
}
b.Append(&structs.Events{Index: uint64(i), Events: []structs.Event{e}})
}
require.Equal(t, 10, int(b.Len()))
// empty the buffer, which will bring the event buffer down
// to a single sentinel value
for i := 0; i < 11; i++ {
b.advanceHead()
}
// 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 TestEventBuffer_StartAt_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, nil)
for i := 11; i <= 100; i++ {
e := structs.Event{
Index: uint64(i), // Indexes should be contiguous
}
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.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, 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")
}
}

View file

@ -0,0 +1,228 @@
package stream
import (
"context"
"fmt"
"sync"
"time"
"github.com/armon/go-metrics"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/hashicorp/go-hclog"
)
const (
DefaultTTL = 1 * time.Hour
)
type EventBrokerCfg struct {
EventBufferSize int64
Logger hclog.Logger
OnEvict EvictCallbackFn
}
type EventBroker struct {
// mu protects the eventbuffer and subscriptions
mu sync.Mutex
// eventBuf stores a configurable amount of events in memory
eventBuf *eventBuffer
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
logger hclog.Logger
}
// 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()
}
// Set the event buffer size to a minimum
if cfg.EventBufferSize == 0 {
cfg.EventBufferSize = 100
}
buffer := newEventBuffer(cfg.EventBufferSize, cfg.OnEvict)
e := &EventBroker{
logger: cfg.Logger.Named("event_broker"),
eventBuf: buffer,
publishCh: make(chan *structs.Events, 64),
subscriptions: &subscriptions{
byToken: make(map[string]map[*SubscribeRequest]*Subscription),
},
}
go e.handleUpdates(ctx)
return e
}
// 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 *EventBroker) Publish(events *structs.Events) {
if len(events.Events) > 0 {
e.publishCh <- events
}
}
// 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
if req.Index != 0 {
head, offset = e.eventBuf.StartAtClosest(req.Index)
} else {
head = e.eventBuf.Head()
}
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
start := newBufferItem(&structs.Events{Index: req.Index})
start.link.next.Store(head)
close(start.link.ch)
sub := newSubscription(req, start, e.subscriptions.unsubscribeFn(req))
e.subscriptions.add(req, sub)
return sub, nil
}
// CloseAll closes all subscriptions
func (e *EventBroker) CloseAll() {
e.subscriptions.closeAll()
}
func (e *EventBroker) handleUpdates(ctx context.Context) {
for {
select {
case <-ctx.Done():
e.subscriptions.closeAll()
return
case update := <-e.publishCh:
e.sendEvents(update)
}
}
}
// sendEvents sends the given events to the publishers event buffer.
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.mu.Lock()
defer s.mu.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.mu.RLock()
defer s.mu.RUnlock()
for _, secretID := range tokenSecretIDs {
if subs, ok := s.byToken[secretID]; ok {
for _, sub := range subs {
sub.forceClose()
}
}
}
}
// 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 unsubscribeFn with the
// wrong pointer.
func (s *subscriptions) unsubscribeFn(req *SubscribeRequest) func() {
return func() {
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)
}
}
}
func (s *subscriptions) closeAll() {
s.mu.Lock()
defer s.mu.Unlock()
for _, byRequest := range s.byToken {
for _, sub := range byRequest {
sub.forceClose()
}
}
}

View file

@ -0,0 +1,163 @@
package stream
import (
"context"
"sync/atomic"
"testing"
"time"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/stretchr/testify/require"
)
func TestEventBroker_PublishChangesAndSubscribe(t *testing.T) {
subscription := &SubscribeRequest{
Topics: map[structs.Topic][]string{
"Test": {"sub-key"},
},
}
ctx, cancel := context.WithTimeout(context.Background(), 2*time.Second)
defer cancel()
publisher := NewEventBroker(ctx, EventBrokerCfg{EventBufferSize: 100})
sub, err := publisher.Subscribe(subscription)
require.NoError(t, err)
eventCh := consumeSubscription(ctx, sub)
// Now subscriber should block waiting for updates
assertNoResult(t, eventCh)
events := []structs.Event{{
Index: 1,
Topic: "Test",
Key: "sub-key",
Payload: "sample payload",
}}
publisher.Publish(&structs.Events{Index: 1, Events: events})
// Subscriber should see the published event
result := nextResult(t, eventCh)
require.NoError(t, result.Err)
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 = []structs.Event{{
Index: 2,
Topic: "Test",
Key: "sub-key",
Payload: "sample payload 2",
}}
publisher.Publish(&structs.Events{Index: 2, Events: events})
result = nextResult(t, eventCh)
require.NoError(t, result.Err)
expected = []structs.Event{{Payload: "sample payload 2", Key: "sub-key", Topic: "Test", Index: 2}}
require.Equal(t, expected, result.Events)
}
func TestEventBroker_ShutdownClosesSubscriptions(t *testing.T) {
ctx, cancel := context.WithCancel(context.Background())
t.Cleanup(cancel)
publisher := NewEventBroker(ctx, EventBrokerCfg{})
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)
}
// 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() {
for {
es, err := sub.Next(ctx)
eventCh <- subNextResult{
Events: es.Events,
Err: err,
}
if err != nil {
return
}
}
}()
return eventCh
}
type subNextResult struct {
Events []structs.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
}
}
}

86
nomad/stream/ndjson.go Normal file
View file

@ -0,0 +1,86 @@
package stream
import (
"context"
"encoding/json"
"fmt"
"time"
"github.com/hashicorp/nomad/nomad/structs"
)
var (
// JsonHeartbeat is an empty JSON object to send as a heartbeat
// Avoids creating many heartbeat instances
JsonHeartbeat = &structs.EventJson{Data: []byte("{}")}
)
// JsonStream is used to send new line delimited JSON and heartbeats
// to a destination (out channel)
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.
heartbeatTick *time.Ticker
}
// 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),
}
go s.heartbeat()
return s
}
func (n *JsonStream) OutCh() chan *structs.EventJson {
return n.outCh
}
func (n *JsonStream) heartbeat() {
for {
select {
case <-n.ctx.Done():
return
case <-n.heartbeatTick.C:
// Send a heartbeat frame
select {
case n.outCh <- JsonHeartbeat:
case <-n.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 *JsonStream) Send(v interface{}) error {
if n.ctx.Err() != nil {
return n.ctx.Err()
}
buf, err := json.Marshal(v)
if err != nil {
return fmt.Errorf("error marshaling json for stream: %w", err)
}
select {
case <-n.ctx.Done():
return fmt.Errorf("error stream is no longer running: %w", err)
case n.outCh <- &structs.EventJson{Data: buf}:
}
return nil
}

View file

@ -0,0 +1,76 @@
package stream
import (
"bytes"
"context"
"testing"
"time"
"github.com/stretchr/testify/require"
)
type testObj struct {
Name string `json:"name"`
}
func TestJsonStream(t *testing.T) {
t.Parallel()
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
s := NewJsonStream(ctx, 1*time.Second)
out := s.OutCh()
require.NoError(t, s.Send(testObj{Name: "test"}))
out1 := <-out
var expected bytes.Buffer
expected.Write([]byte(`{"name":"test"}`))
require.Equal(t, expected.Bytes(), out1.Data)
select {
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 TestJson_Send_After_Stop(t *testing.T) {
t.Parallel()
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
s := NewJsonStream(ctx, 1*time.Second)
// stop the stream
cancel()
time.Sleep(10 * time.Millisecond)
require.Error(t, s.Send(testObj{}))
}
func TestJson_HeartBeat(t *testing.T) {
t.Parallel()
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
s := NewJsonStream(ctx, 10*time.Millisecond)
out := s.OutCh()
heartbeat := <-out
require.Equal(t, JsonHeartbeat, heartbeat)
}

View file

@ -0,0 +1,191 @@
package stream
import (
"context"
"errors"
"sync/atomic"
"github.com/hashicorp/nomad/nomad/structs"
)
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 Subscription struct {
// state must be 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
// EventBroker to cancel Next().
forceClosed chan struct{}
// 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.
unsub func()
}
type SubscribeRequest struct {
Token string
Index uint64
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 {
return &Subscription{
forceClosed: make(chan struct{}),
req: req,
currentItem: item,
unsub: unsub,
}
}
func (s *Subscription) Next(ctx context.Context) (structs.Events, error) {
if atomic.LoadUint32(&s.state) == subscriptionStateClosed {
return structs.Events{}, ErrSubscriptionClosed
}
for {
next, err := s.currentItem.Next(ctx, s.forceClosed)
switch {
case err != nil && atomic.LoadUint32(&s.state) == subscriptionStateClosed:
return structs.Events{}, ErrSubscriptionClosed
case err != nil:
return structs.Events{}, err
}
s.currentItem = next
events := filter(s.req, next.Events.Events)
if len(events) == 0 {
continue
}
return structs.Events{Index: next.Events.Index, Events: events}, nil
}
}
func (s *Subscription) NextNoBlock() ([]structs.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
events := filter(s.req, next.Events.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/namespace
func filter(req *SubscribeRequest, events []structs.Event) []structs.Event {
if len(events) == 0 {
return events
}
var count int
for _, e := range events {
_, allTopics := req.Topics[structs.TopicAll]
if _, ok := req.Topics[e.Topic]; ok || allTopics {
var keys []string
if allTopics {
keys = req.Topics[structs.TopicAll]
} else {
keys = req.Topics[e.Topic]
}
if req.Namespace != "" && e.Namespace != "" && e.Namespace != req.Namespace {
continue
}
for _, k := range keys {
if e.Key == k || k == string(structs.TopicAll) || filterKeyContains(e.FilterKeys, k) {
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([]structs.Event, 0, count)
for _, e := range events {
_, allTopics := req.Topics[structs.TopicAll]
if _, ok := req.Topics[e.Topic]; ok || allTopics {
var keys []string
if allTopics {
keys = req.Topics[structs.TopicAll]
} 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 == string(structs.TopicAll) || filterKeyContains(e.FilterKeys, k) {
result = append(result, e)
}
}
}
}
return result
}
func filterKeyContains(filterKeys []string, key string) bool {
for _, fk := range filterKeys {
if fk == key {
return true
}
}
return false
}

View file

@ -0,0 +1,129 @@
package stream
import (
"testing"
"github.com/hashicorp/nomad/nomad/structs"
"github.com/stretchr/testify/require"
)
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"})
req := &SubscribeRequest{
Topics: map[structs.Topic][]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([]structs.Event, 0, 5)
events = append(events, structs.Event{Topic: "Test", Key: "One"}, structs.Event{Topic: "Test", Key: "Two"})
req := &SubscribeRequest{
Topics: map[structs.Topic][]string{
"Test": {"*"},
},
}
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([]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[structs.Topic][]string{
"Test": {"*"},
},
}
actual := filter(req, events)
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([]structs.Event, 0, 5)
events = append(events, structs.Event{Topic: "Test", Key: "One"}, structs.Event{Topic: "Test", Key: "Two"})
req := &SubscribeRequest{
Topics: map[structs.Topic][]string{
"Test": {"One"},
},
}
actual := filter(req, events)
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([]structs.Event, 0, 5)
events = append(events, structs.Event{Topic: "Test", Key: "One"}, structs.Event{Topic: "Test", Key: "Two"})
req := &SubscribeRequest{
Topics: map[structs.Topic][]string{
"NodeEvents": {"*"},
"Test": {"Highly-Specific-Key"},
},
}
actual := filter(req, events)
var expected []structs.Event
require.Equal(t, expected, actual)
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)
}
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)
}

View file

@ -106,6 +106,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.
@ -8988,6 +8992,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) {
@ -10694,3 +10707,89 @@ type ACLTokenUpsertResponse struct {
Tokens []*ACLToken
WriteMeta
}
// EventStreamRequest is used to stream events from a servers EventBroker
type EventStreamRequest struct {
Topics map[Topic][]string
Index int
QueryOptions
}
type EventStreamWrapper struct {
Error *RpcError
Event *EventJson
}
// 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
}
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 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 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
}
// EventJson is a wrapper for a JSON object
type EventJson struct {
Data []byte
}
func (j *EventJson) Copy() *EventJson {
n := new(EventJson)
*n = *j
n.Data = make([]byte, len(j.Data))
copy(n.Data, j.Data)
return n
}

View file

@ -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.EnableEventBroker = true
config.BootstrapExpect = 1
nodeNum := atomic.AddUint32(&nodeNumber, 1)
config.NodeName = fmt.Sprintf("nomad-%03d", nodeNum)

View file

@ -2957,7 +2957,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(structs.MsgTypeTestSetup, h.NextIndex(), []*structs.Allocation{out}))
}
// Create a mock evaluation which won't trigger any new placements
@ -3107,7 +3107,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(structs.MsgTypeTestSetup, h.NextIndex(), running))
// Mark some of the allocations as complete
var complete []*structs.Allocation
@ -3126,7 +3126,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(structs.MsgTypeTestSetup, h.NextIndex(), complete))
// Create a mock evaluation to deal with the node update
eval := &structs.Evaluation{

View file

@ -132,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(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{

View file

@ -170,7 +170,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(structs.MsgTypeTestSetup, index, &req)
return result, nil, err
}

View file

@ -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

View file

@ -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},
},
}

View file

@ -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
}

View file

@ -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},
},

View file

@ -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
}

View file

@ -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
)

View file

@ -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=

View file

@ -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.

98
vendor/github.com/hashicorp/nomad/api/event.go generated vendored Normal file
View file

@ -0,0 +1,98 @@
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
Err error
}
// 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 map[string]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, 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
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 {
return nil, err
}
eventsCh := make(chan *Events, 10)
go func() {
defer resp.Body.Close()
defer close(eventsCh)
dec := json.NewDecoder(resp.Body)
for ctx.Err() == nil {
// 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}
}
if events.Err == nil && events.IsHeartbeat() {
continue
}
select {
case <-ctx.Done():
return
case eventsCh <- &events:
}
}
}()
return eventsCh, nil
}

4
vendor/modules.txt vendored
View file

@ -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.3.0
## explicit
github.com/hashicorp/go-memdb
# github.com/hashicorp/go-msgpack v1.1.5