From ef99e3d16eeb08e8be2b552084137be28031a385 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Tue, 27 Oct 2015 14:36:32 -0700 Subject: [PATCH 01/26] nomad: initial pass at blocking queries for jobs --- nomad/job_endpoint.go | 66 +++++++++++++++++++++----------------- nomad/job_endpoint_test.go | 51 +++++++++++++++++++++++++++++ nomad/rpc.go | 7 ++++ nomad/state/state_store.go | 17 ++++++++++ 4 files changed, 112 insertions(+), 29 deletions(-) diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index 63d31eb3c6b..cfb92bc24d7 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -216,35 +216,43 @@ func (j *Job) List(args *structs.JobListRequest, } defer metrics.MeasureSince([]string{"nomad", "job", "list"}, time.Now()) - // Capture all the jobs - snap, err := j.srv.fsm.State().Snapshot() - if err != nil { - return err - } - iter, err := snap.Jobs() - if err != nil { - return err - } - - for { - raw := iter.Next() - if raw == nil { - break - } - job := raw.(*structs.Job) - reply.Jobs = append(reply.Jobs, job.Stub()) - } - - // Use the last index that affected the jobs table - index, err := snap.Index("jobs") - if err != nil { - return err - } - reply.Index = index - - // Set the query response - j.srv.setQueryMeta(&reply.QueryMeta) - return nil + // Setup the blocking query + opts := blockingOptions{ + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + jobsWatch: true, + run: func() error { + // Capture all the jobs + snap, err := j.srv.fsm.State().Snapshot() + if err != nil { + return err + } + iter, err := snap.Jobs() + if err != nil { + return err + } + + for { + raw := iter.Next() + if raw == nil { + break + } + job := raw.(*structs.Job) + reply.Jobs = append(reply.Jobs, job.Stub()) + } + + // Use the last index that affected the jobs table + index, err := snap.Index("jobs") + if err != nil { + return err + } + reply.Index = index + + // Set the query response + j.srv.setQueryMeta(&reply.QueryMeta) + return nil + }} + return j.srv.blockingRPC(&opts) } // Allocations is used to list the allocations for a job diff --git a/nomad/job_endpoint_test.go b/nomad/job_endpoint_test.go index e43ed3ba219..e922f31c335 100644 --- a/nomad/job_endpoint_test.go +++ b/nomad/job_endpoint_test.go @@ -3,6 +3,7 @@ package nomad import ( "reflect" "testing" + "time" "github.com/hashicorp/net-rpc-msgpackrpc" "github.com/hashicorp/nomad/nomad/mock" @@ -397,6 +398,56 @@ func TestJobEndpoint_ListJobs(t *testing.T) { } } +func TestJobEndpoint_ListJobs_blocking(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + + // Create the job + job := mock.Job() + + go func() { + // Wait a bit + time.Sleep(100 * time.Millisecond) + + // Send the register request + state := s1.fsm.State() + err := state.UpsertJob(2, job) + if err != nil { + t.Fatalf("err: %v", err) + } + }() + + // Lookup the jobs. Should block until the index is reached. + get := &structs.JobListRequest{ + QueryOptions: structs.QueryOptions{ + Region: "global", + MinQueryIndex: 1, + }, + } + start := time.Now() + var resp structs.JobListResponse + if err := msgpackrpc.CallWithCodec(codec, "Job.List", get, &resp); err != nil { + t.Fatalf("err: %v", err) + } + + // Check that we blocked + if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } + + if resp.Index != 2 { + t.Fatalf("Bad index: %d %d", resp.Index, 2) + } + if len(resp.Jobs) != 1 { + t.Fatalf("bad: %#v", resp.Jobs) + } + if resp.Jobs[0].ID != job.ID { + t.Fatalf("bad: %#v", resp.Jobs[0]) + } +} + func TestJobEndpoint_Allocations(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() diff --git a/nomad/rpc.go b/nomad/rpc.go index 074dec0d61e..dff77eafadb 100644 --- a/nomad/rpc.go +++ b/nomad/rpc.go @@ -271,6 +271,7 @@ type blockingOptions struct { queryOpts *structs.QueryOptions queryMeta *structs.QueryMeta allocWatch string + jobsWatch bool run func() error } @@ -309,6 +310,9 @@ func (s *Server) blockingRPC(opts *blockingOptions) error { if opts.allocWatch != "" { state.StopWatchAllocs(opts.allocWatch, notifyCh) } + if opts.jobsWatch { + state.StopWatchJobs(notifyCh) + } }() REGISTER_NOTIFY: @@ -317,6 +321,9 @@ REGISTER_NOTIFY: if opts.allocWatch != "" { state.WatchAllocs(opts.allocWatch, notifyCh) } + if opts.jobsWatch { + state.WatchJobs(notifyCh) + } RUN_QUERY: // Update the query meta data diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 22487234bc6..a24fe919546 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -58,8 +58,12 @@ type IndexEntry struct { // stateWatch holds shared state for watching updates. This is // outside of StateStore so it can be shared with snapshots. type stateWatch struct { + // Allocation watches by node allocs map[string]*NotifyGroup allocLock sync.Mutex + + // Full table job watches + jobs *NotifyGroup } // NewStateStore is used to create a new state store @@ -73,6 +77,7 @@ func NewStateStore(logOutput io.Writer) (*StateStore, error) { // Create the watch entry watch := &stateWatch{ allocs: make(map[string]*NotifyGroup), + jobs: &NotifyGroup{}, } // Create the state store @@ -155,6 +160,16 @@ func (w *stateWatch) notifyAllocs(nodes map[string]struct{}) { } } +// WatchJobs is used to start watching the jobs view for changes. +func (s *StateStore) WatchJobs(notify chan struct{}) { + s.watch.jobs.Wait(notify) +} + +// StopWatchJobs is used to cancel notification on the given channel. +func (s *StateStore) StopWatchJobs(notify chan struct{}) { + s.watch.jobs.Clear(notify) +} + // 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 which is set by the scheduler. @@ -342,6 +357,7 @@ func (s *StateStore) UpsertJob(index uint64, job *structs.Job) error { return fmt.Errorf("index update failed: %v", err) } + txn.Defer(func() { s.watch.jobs.Notify() }) txn.Commit() return nil } @@ -368,6 +384,7 @@ func (s *StateStore) DeleteJob(index uint64, jobID string) error { return fmt.Errorf("index update failed: %v", err) } + txn.Defer(func() { s.watch.jobs.Notify() }) txn.Commit() return nil } From 750be3892c66c84d00539f585729220cd53a09b0 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Tue, 27 Oct 2015 15:52:40 -0700 Subject: [PATCH 02/26] nomad: allow blocking on empty data views --- nomad/rpc.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/nomad/rpc.go b/nomad/rpc.go index dff77eafadb..a6b6595f319 100644 --- a/nomad/rpc.go +++ b/nomad/rpc.go @@ -334,7 +334,7 @@ RUN_QUERY: err := opts.run() // Check for minimum query time - if err == nil && opts.queryMeta.Index > 0 && opts.queryMeta.Index <= opts.queryOpts.MinQueryIndex { + if err == nil && opts.queryOpts.MinQueryIndex > 0 && opts.queryMeta.Index <= opts.queryOpts.MinQueryIndex { select { case <-notifyCh: goto REGISTER_NOTIFY From 1012a3e5ac9468b51cb5efe0c136dcf572b99572 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Wed, 28 Oct 2015 11:13:30 -0700 Subject: [PATCH 03/26] nomad: use a generic full-table watcher --- nomad/job_endpoint.go | 6 ++-- nomad/rpc.go | 18 ++++------- nomad/state/state_store.go | 65 +++++++++++++++++++++++++++++++------- 3 files changed, 64 insertions(+), 25 deletions(-) diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index cfb92bc24d7..8960a2e9daf 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -218,9 +218,9 @@ func (j *Job) List(args *structs.JobListRequest, // Setup the blocking query opts := blockingOptions{ - queryOpts: &args.QueryOptions, - queryMeta: &reply.QueryMeta, - jobsWatch: true, + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watchTables: []string{"jobs"}, run: func() error { // Capture all the jobs snap, err := j.srv.fsm.State().Snapshot() diff --git a/nomad/rpc.go b/nomad/rpc.go index a6b6595f319..dcb120cc87e 100644 --- a/nomad/rpc.go +++ b/nomad/rpc.go @@ -268,11 +268,11 @@ func (s *Server) setQueryMeta(m *structs.QueryMeta) { // blockingOptions is used to parameterize blockingRPC type blockingOptions struct { - queryOpts *structs.QueryOptions - queryMeta *structs.QueryMeta - allocWatch string - jobsWatch bool - run func() error + queryOpts *structs.QueryOptions + queryMeta *structs.QueryMeta + allocWatch string + watchTables []string + run func() error } // blockingRPC is used for queries that need to wait for a @@ -310,9 +310,7 @@ func (s *Server) blockingRPC(opts *blockingOptions) error { if opts.allocWatch != "" { state.StopWatchAllocs(opts.allocWatch, notifyCh) } - if opts.jobsWatch { - state.StopWatchJobs(notifyCh) - } + state.StopWatchTables(notifyCh, opts.watchTables...) }() REGISTER_NOTIFY: @@ -321,9 +319,7 @@ REGISTER_NOTIFY: if opts.allocWatch != "" { state.WatchAllocs(opts.allocWatch, notifyCh) } - if opts.jobsWatch { - state.WatchJobs(notifyCh) - } + state.WatchTables(notifyCh, opts.watchTables...) RUN_QUERY: // Update the query meta data diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index a24fe919546..ac16b2ead72 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -62,8 +62,47 @@ type stateWatch struct { allocs map[string]*NotifyGroup allocLock sync.Mutex - // Full table job watches - jobs *NotifyGroup + // Full table watches + tables map[string]*NotifyGroup + tableLock sync.Mutex +} + +// watchTable is used to subscribe a channel to a full table watch. +func (w *stateWatch) watchTable(table string, ch chan struct{}) { + w.tableLock.Lock() + defer w.tableLock.Unlock() + + tw, ok := w.tables[table] + if !ok { + tw = new(NotifyGroup) + w.tables[table] = tw + } + tw.Wait(ch) +} + +// stopWatchTable is used to unsubscribe a channel from a table watch. +func (w *stateWatch) stopWatchTable(table string, ch chan struct{}) { + w.tableLock.Lock() + defer w.tableLock.Unlock() + + if tw, ok := w.tables[table]; ok { + tw.Clear(ch) + if tw.Empty() { + delete(w.tables, table) + } + } +} + +// notifyTables is used to notify watchers of the given tables. +func (w *stateWatch) notifyTables(tables ...string) { + w.tableLock.Lock() + defer w.tableLock.Unlock() + + for _, table := range tables { + if tw, ok := w.tables[table]; ok { + tw.Notify() + } + } } // NewStateStore is used to create a new state store @@ -77,7 +116,7 @@ func NewStateStore(logOutput io.Writer) (*StateStore, error) { // Create the watch entry watch := &stateWatch{ allocs: make(map[string]*NotifyGroup), - jobs: &NotifyGroup{}, + tables: make(map[string]*NotifyGroup), } // Create the state store @@ -160,14 +199,18 @@ func (w *stateWatch) notifyAllocs(nodes map[string]struct{}) { } } -// WatchJobs is used to start watching the jobs view for changes. -func (s *StateStore) WatchJobs(notify chan struct{}) { - s.watch.jobs.Wait(notify) +// WatchTables is used to subscribe a channel to a set of tables. +func (s *StateStore) WatchTables(notify chan struct{}, tables ...string) { + for _, table := range tables { + s.watch.watchTable(table, notify) + } } -// StopWatchJobs is used to cancel notification on the given channel. -func (s *StateStore) StopWatchJobs(notify chan struct{}) { - s.watch.jobs.Clear(notify) +// StopWatchTables is used to unsubscribe a channel from table watches. +func (s *StateStore) StopWatchTables(notify chan struct{}, tables ...string) { + for _, table := range tables { + s.watch.stopWatchTable(table, notify) + } } // UpsertNode is used to register a node or update a node definition @@ -357,7 +400,7 @@ func (s *StateStore) UpsertJob(index uint64, job *structs.Job) error { return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.jobs.Notify() }) + txn.Defer(func() { s.watch.notifyTables("jobs") }) txn.Commit() return nil } @@ -384,7 +427,7 @@ func (s *StateStore) DeleteJob(index uint64, jobID string) error { return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.jobs.Notify() }) + txn.Defer(func() { s.watch.notifyTables("jobs") }) txn.Commit() return nil } From 75af87c2d4129425963a9963ae20a88bd0ce35d8 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Wed, 28 Oct 2015 11:21:39 -0700 Subject: [PATCH 04/26] nomad: support blocking queries on nodes --- nomad/node_endpoint.go | 61 +++++++++++++++++++++---------------- nomad/node_endpoint_test.go | 50 ++++++++++++++++++++++++++++++ nomad/state/state_store.go | 4 +++ 3 files changed, 89 insertions(+), 26 deletions(-) diff --git a/nomad/node_endpoint.go b/nomad/node_endpoint.go index 9ce14aadd4f..715b6a58de0 100644 --- a/nomad/node_endpoint.go +++ b/nomad/node_endpoint.go @@ -404,35 +404,44 @@ func (n *Node) List(args *structs.NodeListRequest, } defer metrics.MeasureSince([]string{"nomad", "client", "list"}, time.Now()) - // Capture all the nodes - snap, err := n.srv.fsm.State().Snapshot() - if err != nil { - return err - } - iter, err := snap.Nodes() - if err != nil { - return err - } + // Setup the blocking query + opts := blockingOptions{ + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watchTables: []string{"nodes"}, + run: func() error { - for { - raw := iter.Next() - if raw == nil { - break - } - node := raw.(*structs.Node) - reply.Nodes = append(reply.Nodes, node.Stub()) - } + // Capture all the nodes + snap, err := n.srv.fsm.State().Snapshot() + if err != nil { + return err + } + iter, err := snap.Nodes() + if err != nil { + return err + } - // Use the last index that affected the jobs table - index, err := snap.Index("nodes") - if err != nil { - return err - } - reply.Index = index + for { + raw := iter.Next() + if raw == nil { + break + } + node := raw.(*structs.Node) + reply.Nodes = append(reply.Nodes, node.Stub()) + } - // Set the query response - n.srv.setQueryMeta(&reply.QueryMeta) - return nil + // Use the last index that affected the jobs table + index, err := snap.Index("nodes") + if err != nil { + return err + } + reply.Index = index + + // Set the query response + n.srv.setQueryMeta(&reply.QueryMeta) + return nil + }} + return n.srv.blockingRPC(&opts) } // createNodeEvals is used to create evaluations for each alloc on a node. diff --git a/nomad/node_endpoint_test.go b/nomad/node_endpoint_test.go index 62f4a495964..c1a312d4829 100644 --- a/nomad/node_endpoint_test.go +++ b/nomad/node_endpoint_test.go @@ -752,3 +752,53 @@ func TestClientEndpoint_ListNodes(t *testing.T) { t.Fatalf("bad: %#v", resp2.Nodes[0]) } } + +func TestClientEndpoint_ListNodes_blocking(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + + // Create the node + node := mock.Node() + + go func() { + // Wait a bit + time.Sleep(100 * time.Millisecond) + + // Send the register request + state := s1.fsm.State() + err := state.UpsertNode(2, node) + if err != nil { + t.Fatalf("err: %v", err) + } + }() + + // List the nodes. Should block until the index is reached. + get := &structs.NodeListRequest{ + QueryOptions: structs.QueryOptions{ + Region: "global", + MinQueryIndex: 1, + }, + } + start := time.Now() + var resp structs.NodeListResponse + if err := msgpackrpc.CallWithCodec(codec, "Node.List", get, &resp); err != nil { + t.Fatalf("err: %v", err) + } + + // Check that we blocked + if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } + + if resp.Index != 2 { + t.Fatalf("Bad index: %d %d", resp.Index, 2) + } + if len(resp.Nodes) != 1 { + t.Fatalf("bad: %#v", resp.Nodes) + } + if resp.Nodes[0].ID != node.ID { + t.Fatalf("bad: %#v", resp.Nodes[0]) + } +} diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index ac16b2ead72..a244bb71ac5 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -245,6 +245,7 @@ func (s *StateStore) UpsertNode(index uint64, node *structs.Node) error { return fmt.Errorf("index update failed: %v", err) } + txn.Defer(func() { s.watch.notifyTables("nodes") }) txn.Commit() return nil } @@ -271,6 +272,7 @@ func (s *StateStore) DeleteNode(index uint64, nodeID string) error { return fmt.Errorf("index update failed: %v", err) } + txn.Defer(func() { s.watch.notifyTables("nodes") }) txn.Commit() return nil } @@ -306,6 +308,7 @@ func (s *StateStore) UpdateNodeStatus(index uint64, nodeID, status string) error return fmt.Errorf("index update failed: %v", err) } + txn.Defer(func() { s.watch.notifyTables("nodes") }) txn.Commit() return nil } @@ -341,6 +344,7 @@ func (s *StateStore) UpdateNodeDrain(index uint64, nodeID string, drain bool) er return fmt.Errorf("index update failed: %v", err) } + txn.Defer(func() { s.watch.notifyTables("nodes") }) txn.Commit() return nil } From 417b76a1ac5eba32fa21b4731f49a780c155fc66 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Wed, 28 Oct 2015 12:29:06 -0700 Subject: [PATCH 05/26] nomad: test all node watch triggers --- nomad/node_endpoint.go | 5 +- nomad/node_endpoint_test.go | 96 ++++++++++++++++++++++++++++++------- 2 files changed, 82 insertions(+), 19 deletions(-) diff --git a/nomad/node_endpoint.go b/nomad/node_endpoint.go index 715b6a58de0..23c50de5763 100644 --- a/nomad/node_endpoint.go +++ b/nomad/node_endpoint.go @@ -410,7 +410,6 @@ func (n *Node) List(args *structs.NodeListRequest, queryMeta: &reply.QueryMeta, watchTables: []string{"nodes"}, run: func() error { - // Capture all the nodes snap, err := n.srv.fsm.State().Snapshot() if err != nil { @@ -421,14 +420,16 @@ func (n *Node) List(args *structs.NodeListRequest, return err } + var nodes []*structs.NodeListStub for { raw := iter.Next() if raw == nil { break } node := raw.(*structs.Node) - reply.Nodes = append(reply.Nodes, node.Stub()) + nodes = append(nodes, node.Stub()) } + reply.Nodes = nodes // Use the last index that affected the jobs table index, err := snap.Index("nodes") diff --git a/nomad/node_endpoint_test.go b/nomad/node_endpoint_test.go index c1a312d4829..91ae5d4fc9e 100644 --- a/nomad/node_endpoint_test.go +++ b/nomad/node_endpoint_test.go @@ -756,26 +756,21 @@ func TestClientEndpoint_ListNodes(t *testing.T) { func TestClientEndpoint_ListNodes_blocking(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() + state := s1.fsm.State() codec := rpcClient(t, s1) testutil.WaitForLeader(t, s1.RPC) // Create the node node := mock.Node() - go func() { - // Wait a bit - time.Sleep(100 * time.Millisecond) - - // Send the register request - state := s1.fsm.State() - err := state.UpsertNode(2, node) - if err != nil { + // Node upsert triggers watches + time.AfterFunc(100*time.Millisecond, func() { + if err := state.UpsertNode(2, node); err != nil { t.Fatalf("err: %v", err) } - }() + }) - // List the nodes. Should block until the index is reached. - get := &structs.NodeListRequest{ + req := &structs.NodeListRequest{ QueryOptions: structs.QueryOptions{ Region: "global", MinQueryIndex: 1, @@ -783,22 +778,89 @@ func TestClientEndpoint_ListNodes_blocking(t *testing.T) { } start := time.Now() var resp structs.NodeListResponse - if err := msgpackrpc.CallWithCodec(codec, "Node.List", get, &resp); err != nil { + if err := msgpackrpc.CallWithCodec(codec, "Node.List", req, &resp); err != nil { t.Fatalf("err: %v", err) } - // Check that we blocked if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } - if resp.Index != 2 { t.Fatalf("Bad index: %d %d", resp.Index, 2) } - if len(resp.Nodes) != 1 { + if len(resp.Nodes) != 1 || resp.Nodes[0].ID != node.ID { t.Fatalf("bad: %#v", resp.Nodes) } - if resp.Nodes[0].ID != node.ID { - t.Fatalf("bad: %#v", resp.Nodes[0]) + + // Node drain updates trigger watches. + time.AfterFunc(100*time.Millisecond, func() { + if err := state.UpdateNodeDrain(3, node.ID, true); err != nil { + t.Fatalf("err: %v", err) + } + }) + + req.MinQueryIndex = 2 + var resp2 structs.NodeListResponse + start = time.Now() + if err := msgpackrpc.CallWithCodec(codec, "Node.List", req, &resp2); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } + if resp2.Index != 3 { + t.Fatalf("Bad index: %d %d", resp2.Index, 3) + } + if len(resp2.Nodes) != 1 || !resp2.Nodes[0].Drain { + t.Fatalf("bad: %#v", resp2.Nodes) + } + + // Node status update triggers watches + time.AfterFunc(100*time.Millisecond, func() { + if err := state.UpdateNodeStatus(4, node.ID, structs.NodeStatusDown); err != nil { + t.Fatalf("err: %v", err) + } + }) + + req.MinQueryIndex = 3 + var resp3 structs.NodeListResponse + start = time.Now() + if err := msgpackrpc.CallWithCodec(codec, "Node.List", req, &resp3); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } + if resp3.Index != 4 { + t.Fatalf("Bad index: %d %d", resp3.Index, 4) + } + if len(resp3.Nodes) != 1 || resp3.Nodes[0].Status != structs.NodeStatusDown { + t.Fatalf("bad: %#v", resp3.Nodes) + } + + // Node delete triggers watches. + time.AfterFunc(100*time.Millisecond, func() { + if err := state.DeleteNode(5, node.ID); err != nil { + t.Fatalf("err: %v", err) + } + }) + + req.MinQueryIndex = 4 + var resp4 structs.NodeListResponse + start = time.Now() + if err := msgpackrpc.CallWithCodec(codec, "Node.List", req, &resp4); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } + if resp4.Index != 5 { + t.Fatalf("Bad index: %d %d", resp4.Index, 5) + } + if len(resp4.Nodes) != 0 { + t.Fatalf("bad: %#v", resp4.Nodes) } } From 49a2bef922c4c6883aacd2acb6c76225bf9a8256 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Wed, 28 Oct 2015 12:43:00 -0700 Subject: [PATCH 06/26] nomad: job watches return correct response, add tests --- nomad/job_endpoint.go | 4 +++- nomad/job_endpoint_test.go | 48 ++++++++++++++++++++++++-------------- 2 files changed, 34 insertions(+), 18 deletions(-) diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index 8960a2e9daf..ca6d59e1a0a 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -232,14 +232,16 @@ func (j *Job) List(args *structs.JobListRequest, return err } + var jobs []*structs.JobListStub for { raw := iter.Next() if raw == nil { break } job := raw.(*structs.Job) - reply.Jobs = append(reply.Jobs, job.Stub()) + jobs = append(jobs, job.Stub()) } + reply.Jobs = jobs // Use the last index that affected the jobs table index, err := snap.Index("jobs") diff --git a/nomad/job_endpoint_test.go b/nomad/job_endpoint_test.go index e922f31c335..8a9e5a1eebb 100644 --- a/nomad/job_endpoint_test.go +++ b/nomad/job_endpoint_test.go @@ -401,26 +401,21 @@ func TestJobEndpoint_ListJobs(t *testing.T) { func TestJobEndpoint_ListJobs_blocking(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() + state := s1.fsm.State() codec := rpcClient(t, s1) testutil.WaitForLeader(t, s1.RPC) // Create the job job := mock.Job() - go func() { - // Wait a bit - time.Sleep(100 * time.Millisecond) - - // Send the register request - state := s1.fsm.State() - err := state.UpsertJob(2, job) - if err != nil { + // Upsert job triggers watches + time.AfterFunc(100*time.Millisecond, func() { + if err := state.UpsertJob(2, job); err != nil { t.Fatalf("err: %v", err) } - }() + }) - // Lookup the jobs. Should block until the index is reached. - get := &structs.JobListRequest{ + req := &structs.JobListRequest{ QueryOptions: structs.QueryOptions{ Region: "global", MinQueryIndex: 1, @@ -428,23 +423,42 @@ func TestJobEndpoint_ListJobs_blocking(t *testing.T) { } start := time.Now() var resp structs.JobListResponse - if err := msgpackrpc.CallWithCodec(codec, "Job.List", get, &resp); err != nil { + if err := msgpackrpc.CallWithCodec(codec, "Job.List", req, &resp); err != nil { t.Fatalf("err: %v", err) } - // Check that we blocked if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } - if resp.Index != 2 { t.Fatalf("Bad index: %d %d", resp.Index, 2) } - if len(resp.Jobs) != 1 { + if len(resp.Jobs) != 1 || resp.Jobs[0].ID != job.ID { t.Fatalf("bad: %#v", resp.Jobs) } - if resp.Jobs[0].ID != job.ID { - t.Fatalf("bad: %#v", resp.Jobs[0]) + + // Job deletion triggers watches + time.AfterFunc(100*time.Millisecond, func() { + if err := state.DeleteJob(3, job.ID); err != nil { + t.Fatalf("err: %v", err) + } + }) + + req.MinQueryIndex = 2 + start = time.Now() + var resp2 structs.JobListResponse + if err := msgpackrpc.CallWithCodec(codec, "Job.List", req, &resp2); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } + if resp2.Index != 3 { + t.Fatalf("Bad index: %d %d", resp2.Index, 3) + } + if len(resp2.Jobs) != 0 { + t.Fatalf("bad: %#v", resp2.Jobs) } } From c74a5b8c0a5f276f067c81c0dc540b08d251e264 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Wed, 28 Oct 2015 18:11:55 -0700 Subject: [PATCH 07/26] nomad/state: move methods so we can sanely find them --- nomad/state/state_store.go | 217 ++++++++++++++++++++----------------- 1 file changed, 116 insertions(+), 101 deletions(-) diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index a244bb71ac5..31dbd7bacf0 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -10,6 +10,13 @@ import ( "github.com/hashicorp/nomad/nomad/structs" ) +// IndexEntry is used with the "index" table +// for managing the latest Raft index affecting a table. +type IndexEntry struct { + Key string + Value uint64 +} + // The StateStore is responsible for maintaining all the Nomad // state. It is manipulated by the FSM which maintains consistency // through the use of Raft. The goals of the StateStore are to provide @@ -23,88 +30,6 @@ type StateStore struct { watch *stateWatch } -// StateSnapshot is used to provide a point-in-time snapshot -type StateSnapshot struct { - StateStore -} - -// StateRestore is used to optimize the performance when -// restoring state by only using a single large transaction -// instead of thousands of sub transactions -type StateRestore struct { - txn *memdb.Txn - watch *stateWatch - allocNodes map[string]struct{} -} - -// Abort is used to abort the restore operation -func (s *StateRestore) Abort() { - s.txn.Abort() -} - -// Commit is used to commit the restore operation -func (s *StateRestore) Commit() { - s.txn.Defer(func() { s.watch.notifyAllocs(s.allocNodes) }) - s.txn.Commit() -} - -// IndexEntry is used with the "index" table -// for managing the latest Raft index affecting a table. -type IndexEntry struct { - Key string - Value uint64 -} - -// stateWatch holds shared state for watching updates. This is -// outside of StateStore so it can be shared with snapshots. -type stateWatch struct { - // Allocation watches by node - allocs map[string]*NotifyGroup - allocLock sync.Mutex - - // Full table watches - tables map[string]*NotifyGroup - tableLock sync.Mutex -} - -// watchTable is used to subscribe a channel to a full table watch. -func (w *stateWatch) watchTable(table string, ch chan struct{}) { - w.tableLock.Lock() - defer w.tableLock.Unlock() - - tw, ok := w.tables[table] - if !ok { - tw = new(NotifyGroup) - w.tables[table] = tw - } - tw.Wait(ch) -} - -// stopWatchTable is used to unsubscribe a channel from a table watch. -func (w *stateWatch) stopWatchTable(table string, ch chan struct{}) { - w.tableLock.Lock() - defer w.tableLock.Unlock() - - if tw, ok := w.tables[table]; ok { - tw.Clear(ch) - if tw.Empty() { - delete(w.tables, table) - } - } -} - -// notifyTables is used to notify watchers of the given tables. -func (w *stateWatch) notifyTables(tables ...string) { - w.tableLock.Lock() - defer w.tableLock.Unlock() - - for _, table := range tables { - if tw, ok := w.tables[table]; ok { - tw.Notify() - } - } -} - // NewStateStore is used to create a new state store func NewStateStore(logOutput io.Writer) (*StateStore, error) { // Create the MemDB @@ -151,6 +76,7 @@ func (s *StateStore) Restore() (*StateRestore, error) { txn: txn, watch: s.watch, allocNodes: make(map[string]struct{}), + tables: make(map[string]struct{}), } return r, nil } @@ -186,19 +112,6 @@ func (s *StateStore) StopWatchAllocs(node string, notify chan struct{}) { } } -// notifyAllocs is used to notify any node alloc listeners of a change -func (w *stateWatch) notifyAllocs(nodes map[string]struct{}) { - w.allocLock.Lock() - defer w.allocLock.Unlock() - - for node := range nodes { - if grp, ok := w.allocs[node]; ok { - grp.Notify() - delete(w.allocs, node) - } - } -} - // WatchTables is used to subscribe a channel to a set of tables. func (s *StateStore) WatchTables(notify chan struct{}, tables ...string) { for _, table := range tables { @@ -245,7 +158,8 @@ func (s *StateStore) UpsertNode(index uint64, node *structs.Node) error { return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notifyTables("nodes") }) + tables := map[string]struct{}{"nodes": struct{}{}} + txn.Defer(func() { s.watch.notifyTables(tables) }) txn.Commit() return nil } @@ -272,7 +186,8 @@ func (s *StateStore) DeleteNode(index uint64, nodeID string) error { return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notifyTables("nodes") }) + tables := map[string]struct{}{"nodes": struct{}{}} + txn.Defer(func() { s.watch.notifyTables(tables) }) txn.Commit() return nil } @@ -308,7 +223,8 @@ func (s *StateStore) UpdateNodeStatus(index uint64, nodeID, status string) error return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notifyTables("nodes") }) + tables := map[string]struct{}{"nodes": struct{}{}} + txn.Defer(func() { s.watch.notifyTables(tables) }) txn.Commit() return nil } @@ -344,7 +260,8 @@ func (s *StateStore) UpdateNodeDrain(index uint64, nodeID string, drain bool) er return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notifyTables("nodes") }) + tables := map[string]struct{}{"nodes": struct{}{}} + txn.Defer(func() { s.watch.notifyTables(tables) }) txn.Commit() return nil } @@ -404,7 +321,8 @@ func (s *StateStore) UpsertJob(index uint64, job *structs.Job) error { return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notifyTables("jobs") }) + tables := map[string]struct{}{"jobs": struct{}{}} + txn.Defer(func() { s.watch.notifyTables(tables) }) txn.Commit() return nil } @@ -431,7 +349,8 @@ func (s *StateStore) DeleteJob(index uint64, jobID string) error { return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notifyTables("jobs") }) + tables := map[string]struct{}{"jobs": struct{}{}} + txn.Defer(func() { s.watch.notifyTables(tables) }) txn.Commit() return nil } @@ -817,8 +736,38 @@ func (s *StateStore) Indexes() (memdb.ResultIterator, error) { return iter, nil } +// StateSnapshot is used to provide a point-in-time snapshot +type StateSnapshot struct { + StateStore +} + +// StateRestore is used to optimize the performance when +// restoring state by only using a single large transaction +// instead of thousands of sub transactions +type StateRestore struct { + txn *memdb.Txn + watch *stateWatch + allocNodes map[string]struct{} + tables map[string]struct{} +} + +// Abort is used to abort the restore operation +func (s *StateRestore) Abort() { + s.txn.Abort() +} + +// Commit is used to commit the restore operation +func (s *StateRestore) Commit() { + s.txn.Defer(func() { + s.watch.notifyAllocs(s.allocNodes) + s.watch.notifyTables(s.tables) + }) + s.txn.Commit() +} + // NodeRestore is used to restore a node func (r *StateRestore) NodeRestore(node *structs.Node) error { + r.tables["nodes"] = struct{}{} if err := r.txn.Insert("nodes", node); err != nil { return fmt.Errorf("node insert failed: %v", err) } @@ -827,6 +776,7 @@ func (r *StateRestore) NodeRestore(node *structs.Node) error { // JobRestore is used to restore a job func (r *StateRestore) JobRestore(job *structs.Job) error { + r.tables["jobs"] = struct{}{} if err := r.txn.Insert("jobs", job); err != nil { return fmt.Errorf("job insert failed: %v", err) } @@ -835,6 +785,7 @@ func (r *StateRestore) JobRestore(job *structs.Job) error { // EvalRestore is used to restore an evaluation func (r *StateRestore) EvalRestore(eval *structs.Evaluation) error { + r.tables["evals"] = struct{}{} if err := r.txn.Insert("evals", eval); err != nil { return fmt.Errorf("eval insert failed: %v", err) } @@ -843,6 +794,7 @@ func (r *StateRestore) EvalRestore(eval *structs.Evaluation) error { // AllocRestore is used to restore an allocation func (r *StateRestore) AllocRestore(alloc *structs.Allocation) error { + r.tables["allocs"] = struct{}{} r.allocNodes[alloc.NodeID] = struct{}{} if err := r.txn.Insert("allocs", alloc); err != nil { return fmt.Errorf("alloc insert failed: %v", err) @@ -857,3 +809,66 @@ func (r *StateRestore) IndexRestore(idx *IndexEntry) error { } return nil } + +// stateWatch holds shared state for watching updates. This is +// outside of StateStore so it can be shared with snapshots. +type stateWatch struct { + // Allocation watches by node + allocs map[string]*NotifyGroup + allocLock sync.Mutex + + // Full table watches + tables map[string]*NotifyGroup + tableLock sync.Mutex +} + +// watchTable is used to subscribe a channel to a full table watch. +func (w *stateWatch) watchTable(table string, ch chan struct{}) { + w.tableLock.Lock() + defer w.tableLock.Unlock() + + tw, ok := w.tables[table] + if !ok { + tw = new(NotifyGroup) + w.tables[table] = tw + } + tw.Wait(ch) +} + +// stopWatchTable is used to unsubscribe a channel from a table watch. +func (w *stateWatch) stopWatchTable(table string, ch chan struct{}) { + w.tableLock.Lock() + defer w.tableLock.Unlock() + + if tw, ok := w.tables[table]; ok { + tw.Clear(ch) + if tw.Empty() { + delete(w.tables, table) + } + } +} + +// notifyTables is used to notify watchers of the given tables. +func (w *stateWatch) notifyTables(tables map[string]struct{}) { + w.tableLock.Lock() + defer w.tableLock.Unlock() + + for table, _ := range tables { + if tw, ok := w.tables[table]; ok { + tw.Notify() + } + } +} + +// notifyAllocs is used to notify any node alloc listeners of a change +func (w *stateWatch) notifyAllocs(nodes map[string]struct{}) { + w.allocLock.Lock() + defer w.allocLock.Unlock() + + for node := range nodes { + if grp, ok := w.allocs[node]; ok { + grp.Notify() + delete(w.allocs, node) + } + } +} From e23f547f2383c110498488450afa6009776cc051 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Wed, 28 Oct 2015 18:34:56 -0700 Subject: [PATCH 08/26] nomad: support blocking queries on eval list --- nomad/eval_endpoint.go | 68 +++++++++++++++++++++---------------- nomad/eval_endpoint_test.go | 64 ++++++++++++++++++++++++++++++++++ nomad/state/state_store.go | 9 ++++- 3 files changed, 111 insertions(+), 30 deletions(-) diff --git a/nomad/eval_endpoint.go b/nomad/eval_endpoint.go index 0dce98a52d8..6e8e6505439 100644 --- a/nomad/eval_endpoint.go +++ b/nomad/eval_endpoint.go @@ -219,35 +219,45 @@ func (e *Eval) List(args *structs.EvalListRequest, } defer metrics.MeasureSince([]string{"nomad", "eval", "list"}, time.Now()) - // Scan all the evaluations - snap, err := e.srv.fsm.State().Snapshot() - if err != nil { - return err - } - iter, err := snap.Evals() - if err != nil { - return err - } - - for { - raw := iter.Next() - if raw == nil { - break - } - eval := raw.(*structs.Evaluation) - reply.Evaluations = append(reply.Evaluations, eval) - } - - // Use the last index that affected the jobs table - index, err := snap.Index("evals") - if err != nil { - return err - } - reply.Index = index - - // Set the query response - e.srv.setQueryMeta(&reply.QueryMeta) - return nil + // Setup the blocking query + opts := blockingOptions{ + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watchTables: []string{"evals"}, + run: func() error { + // Scan all the evaluations + snap, err := e.srv.fsm.State().Snapshot() + if err != nil { + return err + } + iter, err := snap.Evals() + if err != nil { + return err + } + + var evals []*structs.Evaluation + for { + raw := iter.Next() + if raw == nil { + break + } + eval := raw.(*structs.Evaluation) + evals = append(evals, eval) + } + reply.Evaluations = evals + + // Use the last index that affected the jobs table + index, err := snap.Index("evals") + if err != nil { + return err + } + reply.Index = index + + // Set the query response + e.srv.setQueryMeta(&reply.QueryMeta) + return nil + }} + return e.srv.blockingRPC(&opts) } // Allocations is used to list the allocations for an evaluation diff --git a/nomad/eval_endpoint_test.go b/nomad/eval_endpoint_test.go index eb61ea3d074..3b9a62a8e8a 100644 --- a/nomad/eval_endpoint_test.go +++ b/nomad/eval_endpoint_test.go @@ -334,6 +334,70 @@ func TestEvalEndpoint_List(t *testing.T) { } } +func TestEvalEndpoint_List_blocking(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + state := s1.fsm.State() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + + // Create the ieval + eval := mock.Eval() + + // Upsert eval triggers watches + time.AfterFunc(100*time.Millisecond, func() { + if err := state.UpsertEvals(2, []*structs.Evaluation{eval}); err != nil { + t.Fatalf("err: %v", err) + } + }) + + req := &structs.EvalListRequest{ + QueryOptions: structs.QueryOptions{ + Region: "global", + MinQueryIndex: 1, + }, + } + start := time.Now() + var resp structs.EvalListResponse + if err := msgpackrpc.CallWithCodec(codec, "Eval.List", req, &resp); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } + if resp.Index != 2 { + t.Fatalf("Bad index: %d %d", resp.Index, 2) + } + if len(resp.Evaluations) != 1 || resp.Evaluations[0].ID != eval.ID { + t.Fatalf("bad: %#v", resp.Evaluations) + } + + // Eval deletion triggers watches + time.AfterFunc(100*time.Millisecond, func() { + if err := state.DeleteEval(3, []string{eval.ID}, nil); err != nil { + t.Fatalf("err: %v", err) + } + }) + + req.MinQueryIndex = 2 + start = time.Now() + var resp2 structs.EvalListResponse + if err := msgpackrpc.CallWithCodec(codec, "Eval.List", req, &resp2); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp2) + } + if resp2.Index != 3 { + t.Fatalf("Bad index: %d %d", resp2.Index, 3) + } + if len(resp2.Evaluations) != 0 { + t.Fatalf("bad: %#v", resp2.Evaluations) + } +} + func TestEvalEndpoint_Allocations(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 31dbd7bacf0..9a7a3327393 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -407,6 +407,8 @@ func (s *StateStore) UpsertEvals(index uint64, evals []*structs.Evaluation) erro } } + tables := map[string]struct{}{"evals": struct{}{}} + txn.Defer(func() { s.watch.notifyTables(tables) }) txn.Commit() return nil } @@ -478,7 +480,12 @@ func (s *StateStore) DeleteEval(index uint64, evals []string, allocs []string) e if err := txn.Insert("index", &IndexEntry{"allocs", index}); err != nil { return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notifyAllocs(nodes) }) + + tables := map[string]struct{}{"evals": struct{}{}} + txn.Defer(func() { + s.watch.notifyAllocs(nodes) + s.watch.notifyTables(tables) + }) txn.Commit() return nil } From b9fb0252007ebf669b46fb67170b9cbd0d239060 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Wed, 28 Oct 2015 18:35:48 -0700 Subject: [PATCH 09/26] nomad: fix node test output --- nomad/job_endpoint_test.go | 2 +- nomad/node_endpoint_test.go | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/nomad/job_endpoint_test.go b/nomad/job_endpoint_test.go index 8a9e5a1eebb..0591e73bf6c 100644 --- a/nomad/job_endpoint_test.go +++ b/nomad/job_endpoint_test.go @@ -452,7 +452,7 @@ func TestJobEndpoint_ListJobs_blocking(t *testing.T) { } if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { - t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + t.Fatalf("should block (returned in %s) %#v", elapsed, resp2) } if resp2.Index != 3 { t.Fatalf("Bad index: %d %d", resp2.Index, 3) diff --git a/nomad/node_endpoint_test.go b/nomad/node_endpoint_test.go index 91ae5d4fc9e..d06e6ea0f88 100644 --- a/nomad/node_endpoint_test.go +++ b/nomad/node_endpoint_test.go @@ -807,7 +807,7 @@ func TestClientEndpoint_ListNodes_blocking(t *testing.T) { } if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { - t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + t.Fatalf("should block (returned in %s) %#v", elapsed, resp2) } if resp2.Index != 3 { t.Fatalf("Bad index: %d %d", resp2.Index, 3) @@ -831,7 +831,7 @@ func TestClientEndpoint_ListNodes_blocking(t *testing.T) { } if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { - t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + t.Fatalf("should block (returned in %s) %#v", elapsed, resp3) } if resp3.Index != 4 { t.Fatalf("Bad index: %d %d", resp3.Index, 4) @@ -855,7 +855,7 @@ func TestClientEndpoint_ListNodes_blocking(t *testing.T) { } if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { - t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + t.Fatalf("should block (returned in %s) %#v", elapsed, resp4) } if resp4.Index != 5 { t.Fatalf("Bad index: %d %d", resp4.Index, 5) From b162c259d24ce6336c49477649829aef7b793dc1 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Wed, 28 Oct 2015 19:25:39 -0700 Subject: [PATCH 10/26] nomad: support full table watches for allocations --- nomad/alloc_endpoint.go | 62 ++++++++++++++++++-------------- nomad/alloc_endpoint_test.go | 69 ++++++++++++++++++++++++++++++++++++ nomad/state/state_store.go | 12 +++++-- 3 files changed, 115 insertions(+), 28 deletions(-) diff --git a/nomad/alloc_endpoint.go b/nomad/alloc_endpoint.go index 53b630480ae..09bd28727a0 100644 --- a/nomad/alloc_endpoint.go +++ b/nomad/alloc_endpoint.go @@ -19,35 +19,45 @@ func (a *Alloc) List(args *structs.AllocListRequest, reply *structs.AllocListRes } defer metrics.MeasureSince([]string{"nomad", "alloc", "list"}, time.Now()) - // Capture all the allocations - snap, err := a.srv.fsm.State().Snapshot() - if err != nil { - return err - } - iter, err := snap.Allocs() - if err != nil { - return err - } + // Setup the blocking query + opts := blockingOptions{ + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watchTables: []string{"allocs"}, + run: func() error { + // Capture all the allocations + snap, err := a.srv.fsm.State().Snapshot() + if err != nil { + return err + } + iter, err := snap.Allocs() + if err != nil { + return err + } - for { - raw := iter.Next() - if raw == nil { - break - } - alloc := raw.(*structs.Allocation) - reply.Allocations = append(reply.Allocations, alloc.Stub()) - } + var allocs []*structs.AllocListStub + for { + raw := iter.Next() + if raw == nil { + break + } + alloc := raw.(*structs.Allocation) + allocs = append(allocs, alloc.Stub()) + } + reply.Allocations = allocs - // Use the last index that affected the jobs table - index, err := snap.Index("allocs") - if err != nil { - return err - } - reply.Index = index + // Use the last index that affected the jobs table + index, err := snap.Index("allocs") + if err != nil { + return err + } + reply.Index = index - // Set the query response - a.srv.setQueryMeta(&reply.QueryMeta) - return nil + // Set the query response + a.srv.setQueryMeta(&reply.QueryMeta) + return nil + }} + return a.srv.blockingRPC(&opts) } // GetAlloc is used to lookup a particular allocation diff --git a/nomad/alloc_endpoint_test.go b/nomad/alloc_endpoint_test.go index 8076b64d6c3..01688da960e 100644 --- a/nomad/alloc_endpoint_test.go +++ b/nomad/alloc_endpoint_test.go @@ -3,6 +3,7 @@ package nomad import ( "reflect" "testing" + "time" "github.com/hashicorp/net-rpc-msgpackrpc" "github.com/hashicorp/nomad/nomad/mock" @@ -44,6 +45,74 @@ func TestAllocEndpoint_List(t *testing.T) { } } +func TestAllocEndpoint_List_blocking(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + state := s1.fsm.State() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + + // Create the alloc + alloc := mock.Alloc() + + // Upsert alloc triggers watches + time.AfterFunc(100*time.Millisecond, func() { + if err := state.UpsertAllocs(2, []*structs.Allocation{alloc}); err != nil { + t.Fatalf("err: %v", err) + } + }) + + req := &structs.AllocListRequest{ + QueryOptions: structs.QueryOptions{ + Region: "global", + MinQueryIndex: 1, + }, + } + start := time.Now() + var resp structs.AllocListResponse + if err := msgpackrpc.CallWithCodec(codec, "Alloc.List", req, &resp); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } + if resp.Index != 2 { + t.Fatalf("Bad index: %d %d", resp.Index, 2) + } + if len(resp.Allocations) != 1 || resp.Allocations[0].ID != alloc.ID { + t.Fatalf("bad: %#v", resp.Allocations) + } + + // Client updates trigger watches + alloc2 := mock.Alloc() + alloc2.ID = alloc.ID + alloc2.ClientStatus = structs.AllocClientStatusRunning + time.AfterFunc(100*time.Millisecond, func() { + if err := state.UpdateAllocFromClient(3, alloc2); err != nil { + t.Fatalf("err: %v", err) + } + }) + + req.MinQueryIndex = 2 + start = time.Now() + var resp2 structs.AllocListResponse + if err := msgpackrpc.CallWithCodec(codec, "Alloc.List", req, &resp2); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp2) + } + if resp2.Index != 3 { + t.Fatalf("Bad index: %d %d", resp2.Index, 3) + } + if len(resp2.Allocations) != 1 || resp.Allocations[0].ID != alloc.ID || + resp2.Allocations[0].ClientStatus != structs.AllocClientStatusRunning { + t.Fatalf("bad: %#v", resp2.Allocations) + } +} + func TestAllocEndpoint_GetAlloc(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 9a7a3327393..389991de9a1 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -580,8 +580,12 @@ func (s *StateStore) UpdateAllocFromClient(index uint64, alloc *structs.Allocati return fmt.Errorf("index update failed: %v", err) } + tables := map[string]struct{}{"allocs": struct{}{}} nodes := map[string]struct{}{alloc.NodeID: struct{}{}} - txn.Defer(func() { s.watch.notifyAllocs(nodes) }) + txn.Defer(func() { + s.watch.notifyAllocs(nodes) + s.watch.notifyTables(tables) + }) txn.Commit() return nil } @@ -621,7 +625,11 @@ func (s *StateStore) UpsertAllocs(index uint64, allocs []*structs.Allocation) er return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notifyAllocs(nodes) }) + tables := map[string]struct{}{"allocs": struct{}{}} + txn.Defer(func() { + s.watch.notifyAllocs(nodes) + s.watch.notifyTables(tables) + }) txn.Commit() return nil } From 2558ab3f31e989a49fe125f098cfb02423134f5b Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Thu, 29 Oct 2015 11:57:41 -0700 Subject: [PATCH 11/26] nomad: unify watcher inputs for reusability --- nomad/alloc_endpoint.go | 6 +- nomad/eval_endpoint.go | 6 +- nomad/job_endpoint.go | 6 +- nomad/node_endpoint.go | 12 +-- nomad/rpc.go | 22 ++-- nomad/state/state_store.go | 209 +++++++++++++------------------------ 6 files changed, 98 insertions(+), 163 deletions(-) diff --git a/nomad/alloc_endpoint.go b/nomad/alloc_endpoint.go index 09bd28727a0..a2ce6a09a80 100644 --- a/nomad/alloc_endpoint.go +++ b/nomad/alloc_endpoint.go @@ -21,9 +21,9 @@ func (a *Alloc) List(args *structs.AllocListRequest, reply *structs.AllocListRes // Setup the blocking query opts := blockingOptions{ - queryOpts: &args.QueryOptions, - queryMeta: &reply.QueryMeta, - watchTables: []string{"allocs"}, + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watchTable: "allocs", run: func() error { // Capture all the allocations snap, err := a.srv.fsm.State().Snapshot() diff --git a/nomad/eval_endpoint.go b/nomad/eval_endpoint.go index 6e8e6505439..5d87948aa27 100644 --- a/nomad/eval_endpoint.go +++ b/nomad/eval_endpoint.go @@ -221,9 +221,9 @@ func (e *Eval) List(args *structs.EvalListRequest, // Setup the blocking query opts := blockingOptions{ - queryOpts: &args.QueryOptions, - queryMeta: &reply.QueryMeta, - watchTables: []string{"evals"}, + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watchTable: "evals", run: func() error { // Scan all the evaluations snap, err := e.srv.fsm.State().Snapshot() diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index ca6d59e1a0a..30bc35563da 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -218,9 +218,9 @@ func (j *Job) List(args *structs.JobListRequest, // Setup the blocking query opts := blockingOptions{ - queryOpts: &args.QueryOptions, - queryMeta: &reply.QueryMeta, - watchTables: []string{"jobs"}, + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watchTable: "jobs", run: func() error { // Capture all the jobs snap, err := j.srv.fsm.State().Snapshot() diff --git a/nomad/node_endpoint.go b/nomad/node_endpoint.go index 23c50de5763..7e73329742a 100644 --- a/nomad/node_endpoint.go +++ b/nomad/node_endpoint.go @@ -330,9 +330,9 @@ func (n *Node) GetAllocs(args *structs.NodeSpecificRequest, // Setup the blocking query opts := blockingOptions{ - queryOpts: &args.QueryOptions, - queryMeta: &reply.QueryMeta, - allocWatch: args.NodeID, + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watchAllocNode: args.NodeID, run: func() error { // Look for the node snap, err := n.srv.fsm.State().Snapshot() @@ -406,9 +406,9 @@ func (n *Node) List(args *structs.NodeListRequest, // Setup the blocking query opts := blockingOptions{ - queryOpts: &args.QueryOptions, - queryMeta: &reply.QueryMeta, - watchTables: []string{"nodes"}, + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watchTable: "nodes", run: func() error { // Capture all the nodes snap, err := n.srv.fsm.State().Snapshot() diff --git a/nomad/rpc.go b/nomad/rpc.go index dcb120cc87e..f1977dbc72f 100644 --- a/nomad/rpc.go +++ b/nomad/rpc.go @@ -268,11 +268,11 @@ func (s *Server) setQueryMeta(m *structs.QueryMeta) { // blockingOptions is used to parameterize blockingRPC type blockingOptions struct { - queryOpts *structs.QueryOptions - queryMeta *structs.QueryMeta - allocWatch string - watchTables []string - run func() error + queryOpts *structs.QueryOptions + queryMeta *structs.QueryMeta + watchAllocNode string + watchTable string + run func() error } // blockingRPC is used for queries that need to wait for a @@ -307,19 +307,15 @@ func (s *Server) blockingRPC(opts *blockingOptions) error { state = s.fsm.State() defer func() { timeout.Stop() - if opts.allocWatch != "" { - state.StopWatchAllocs(opts.allocWatch, notifyCh) - } - state.StopWatchTables(notifyCh, opts.watchTables...) + state.StopWatchAllocNode(opts.watchAllocNode, notifyCh) + state.StopWatchTable(opts.watchTable, notifyCh) }() REGISTER_NOTIFY: // Register the notification channel. This may be done // multiple times if we have not reached the target wait index. - if opts.allocWatch != "" { - state.WatchAllocs(opts.allocWatch, notifyCh) - } - state.WatchTables(notifyCh, opts.watchTables...) + state.WatchAllocNode(opts.watchAllocNode, notifyCh) + state.WatchTable(opts.watchTable, notifyCh) RUN_QUERY: // Update the query meta data diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 389991de9a1..0a8adc0d01f 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -38,17 +38,11 @@ func NewStateStore(logOutput io.Writer) (*StateStore, error) { return nil, fmt.Errorf("state store setup failed: %v", err) } - // Create the watch entry - watch := &stateWatch{ - allocs: make(map[string]*NotifyGroup), - tables: make(map[string]*NotifyGroup), - } - // Create the state store s := &StateStore{ logger: log.New(logOutput, "", log.LstdFlags), db: db, - watch: watch, + watch: newStateWatch(), } return s, nil } @@ -73,57 +67,30 @@ func (s *StateStore) Snapshot() (*StateSnapshot, error) { func (s *StateStore) Restore() (*StateRestore, error) { txn := s.db.Txn(true) r := &StateRestore{ - txn: txn, - watch: s.watch, - allocNodes: make(map[string]struct{}), - tables: make(map[string]struct{}), + txn: txn, + watch: s.watch, } return r, nil } -// WatchAllocs is used to subscribe a channel to changes in allocations for a node -func (s *StateStore) WatchAllocs(node string, notify chan struct{}) { - s.watch.allocLock.Lock() - defer s.watch.allocLock.Unlock() - - // Check for an existing notify group - if grp, ok := s.watch.allocs[node]; ok { - grp.Wait(notify) - return - } - - // Create new notify group - grp := &NotifyGroup{} - grp.Wait(notify) - s.watch.allocs[node] = grp +// WatchTable is used to subscribe a channel to a full table watch. +func (s *StateStore) WatchTable(table string, notify chan struct{}) { + s.watch.watch(watchItem{table: table}, notify) } -// StopWatchAllocs is used to unsubscribe a channel from changes in allocations -func (s *StateStore) StopWatchAllocs(node string, notify chan struct{}) { - s.watch.allocLock.Lock() - defer s.watch.allocLock.Unlock() - - // Check for an existing notify group - if grp, ok := s.watch.allocs[node]; ok { - grp.Clear(notify) - if grp.Empty() { - delete(s.watch.allocs, node) - } - } +// StopWatchTable unsubscribes a channel from a full table watch. +func (s *StateStore) StopWatchTable(table string, notify chan struct{}) { + s.watch.stopWatch(watchItem{table: table}, notify) } -// WatchTables is used to subscribe a channel to a set of tables. -func (s *StateStore) WatchTables(notify chan struct{}, tables ...string) { - for _, table := range tables { - s.watch.watchTable(table, notify) - } +// WatchAllocNode is used to subscribe a channel to a node allocation watch. +func (s *StateStore) WatchAllocNode(nodeID string, notify chan struct{}) { + s.watch.watch(watchItem{allocNode: nodeID}, notify) } -// StopWatchTables is used to unsubscribe a channel from table watches. -func (s *StateStore) StopWatchTables(notify chan struct{}, tables ...string) { - for _, table := range tables { - s.watch.stopWatchTable(table, notify) - } +// StopWatchAllocNode unsubscribes a channel from a node allocation watch. +func (s *StateStore) StopWatchAllocNode(nodeID string, notify chan struct{}) { + s.watch.stopWatch(watchItem{allocNode: nodeID}, notify) } // UpsertNode is used to register a node or update a node definition @@ -158,8 +125,7 @@ func (s *StateStore) UpsertNode(index uint64, node *structs.Node) error { return fmt.Errorf("index update failed: %v", err) } - tables := map[string]struct{}{"nodes": struct{}{}} - txn.Defer(func() { s.watch.notifyTables(tables) }) + txn.Defer(func() { s.watch.notify(watchItem{table: "nodes"}) }) txn.Commit() return nil } @@ -186,8 +152,7 @@ func (s *StateStore) DeleteNode(index uint64, nodeID string) error { return fmt.Errorf("index update failed: %v", err) } - tables := map[string]struct{}{"nodes": struct{}{}} - txn.Defer(func() { s.watch.notifyTables(tables) }) + txn.Defer(func() { s.watch.notify(watchItem{table: "nodes"}) }) txn.Commit() return nil } @@ -223,8 +188,7 @@ func (s *StateStore) UpdateNodeStatus(index uint64, nodeID, status string) error return fmt.Errorf("index update failed: %v", err) } - tables := map[string]struct{}{"nodes": struct{}{}} - txn.Defer(func() { s.watch.notifyTables(tables) }) + txn.Defer(func() { s.watch.notify(watchItem{table: "nodes"}) }) txn.Commit() return nil } @@ -260,8 +224,7 @@ func (s *StateStore) UpdateNodeDrain(index uint64, nodeID string, drain bool) er return fmt.Errorf("index update failed: %v", err) } - tables := map[string]struct{}{"nodes": struct{}{}} - txn.Defer(func() { s.watch.notifyTables(tables) }) + txn.Defer(func() { s.watch.notify(watchItem{table: "nodes"}) }) txn.Commit() return nil } @@ -321,8 +284,7 @@ func (s *StateStore) UpsertJob(index uint64, job *structs.Job) error { return fmt.Errorf("index update failed: %v", err) } - tables := map[string]struct{}{"jobs": struct{}{}} - txn.Defer(func() { s.watch.notifyTables(tables) }) + txn.Defer(func() { s.watch.notify(watchItem{table: "jobs"}) }) txn.Commit() return nil } @@ -349,8 +311,7 @@ func (s *StateStore) DeleteJob(index uint64, jobID string) error { return fmt.Errorf("index update failed: %v", err) } - tables := map[string]struct{}{"jobs": struct{}{}} - txn.Defer(func() { s.watch.notifyTables(tables) }) + txn.Defer(func() { s.watch.notify(watchItem{table: "jobs"}) }) txn.Commit() return nil } @@ -407,8 +368,7 @@ func (s *StateStore) UpsertEvals(index uint64, evals []*structs.Evaluation) erro } } - tables := map[string]struct{}{"evals": struct{}{}} - txn.Defer(func() { s.watch.notifyTables(tables) }) + txn.Defer(func() { s.watch.notify(watchItem{table: "evals"}) }) txn.Commit() return nil } @@ -444,7 +404,6 @@ func (s *StateStore) nestedUpsertEval(txn *memdb.Txn, index uint64, eval *struct func (s *StateStore) DeleteEval(index uint64, evals []string, allocs []string) error { txn := s.db.Txn(true) defer txn.Abort() - nodes := make(map[string]struct{}) for _, eval := range evals { existing, err := txn.First("evals", "id", eval) @@ -467,7 +426,6 @@ func (s *StateStore) DeleteEval(index uint64, evals []string, allocs []string) e if existing == nil { continue } - nodes[existing.(*structs.Allocation).NodeID] = struct{}{} if err := txn.Delete("allocs", existing); err != nil { return fmt.Errorf("alloc delete failed: %v", err) } @@ -481,11 +439,7 @@ func (s *StateStore) DeleteEval(index uint64, evals []string, allocs []string) e return fmt.Errorf("index update failed: %v", err) } - tables := map[string]struct{}{"evals": struct{}{}} - txn.Defer(func() { - s.watch.notifyAllocs(nodes) - s.watch.notifyTables(tables) - }) + txn.Defer(func() { s.watch.notify(watchItem{table: "evals"}) }) txn.Commit() return nil } @@ -580,12 +534,7 @@ func (s *StateStore) UpdateAllocFromClient(index uint64, alloc *structs.Allocati return fmt.Errorf("index update failed: %v", err) } - tables := map[string]struct{}{"allocs": struct{}{}} - nodes := map[string]struct{}{alloc.NodeID: struct{}{}} - txn.Defer(func() { - s.watch.notifyAllocs(nodes) - s.watch.notifyTables(tables) - }) + txn.Defer(func() { s.watch.notify(watchItem{table: "allocs"}) }) txn.Commit() return nil } @@ -595,7 +544,6 @@ func (s *StateStore) UpdateAllocFromClient(index uint64, alloc *structs.Allocati func (s *StateStore) UpsertAllocs(index uint64, allocs []*structs.Allocation) error { txn := s.db.Txn(true) defer txn.Abort() - nodes := make(map[string]struct{}) // Handle the allocations for _, alloc := range allocs { @@ -614,7 +562,6 @@ func (s *StateStore) UpsertAllocs(index uint64, allocs []*structs.Allocation) er alloc.ClientStatus = exist.ClientStatus alloc.ClientDescription = exist.ClientDescription } - nodes[alloc.NodeID] = struct{}{} if err := txn.Insert("allocs", alloc); err != nil { return fmt.Errorf("alloc insert failed: %v", err) } @@ -625,11 +572,7 @@ func (s *StateStore) UpsertAllocs(index uint64, allocs []*structs.Allocation) er return fmt.Errorf("index update failed: %v", err) } - tables := map[string]struct{}{"allocs": struct{}{}} - txn.Defer(func() { - s.watch.notifyAllocs(nodes) - s.watch.notifyTables(tables) - }) + txn.Defer(func() { s.watch.notify(watchItem{table: "allocs"}) }) txn.Commit() return nil } @@ -760,10 +703,9 @@ type StateSnapshot struct { // restoring state by only using a single large transaction // instead of thousands of sub transactions type StateRestore struct { - txn *memdb.Txn - watch *stateWatch - allocNodes map[string]struct{} - tables map[string]struct{} + txn *memdb.Txn + watch *stateWatch + items []watchItem } // Abort is used to abort the restore operation @@ -773,16 +715,13 @@ func (s *StateRestore) Abort() { // Commit is used to commit the restore operation func (s *StateRestore) Commit() { - s.txn.Defer(func() { - s.watch.notifyAllocs(s.allocNodes) - s.watch.notifyTables(s.tables) - }) + s.txn.Defer(func() { s.watch.notify(s.items...) }) s.txn.Commit() } // NodeRestore is used to restore a node func (r *StateRestore) NodeRestore(node *structs.Node) error { - r.tables["nodes"] = struct{}{} + r.items = append(r.items, watchItem{table: "nodes"}) if err := r.txn.Insert("nodes", node); err != nil { return fmt.Errorf("node insert failed: %v", err) } @@ -791,7 +730,7 @@ func (r *StateRestore) NodeRestore(node *structs.Node) error { // JobRestore is used to restore a job func (r *StateRestore) JobRestore(job *structs.Job) error { - r.tables["jobs"] = struct{}{} + r.items = append(r.items, watchItem{table: "jobs"}) if err := r.txn.Insert("jobs", job); err != nil { return fmt.Errorf("job insert failed: %v", err) } @@ -800,7 +739,7 @@ func (r *StateRestore) JobRestore(job *structs.Job) error { // EvalRestore is used to restore an evaluation func (r *StateRestore) EvalRestore(eval *structs.Evaluation) error { - r.tables["evals"] = struct{}{} + r.items = append(r.items, watchItem{table: "evals"}) if err := r.txn.Insert("evals", eval); err != nil { return fmt.Errorf("eval insert failed: %v", err) } @@ -809,8 +748,8 @@ func (r *StateRestore) EvalRestore(eval *structs.Evaluation) error { // AllocRestore is used to restore an allocation func (r *StateRestore) AllocRestore(alloc *structs.Allocation) error { - r.tables["allocs"] = struct{}{} - r.allocNodes[alloc.NodeID] = struct{}{} + r.items = append(r.items, watchItem{table: "allocs"}) + r.items = append(r.items, watchItem{allocNode: alloc.NodeID}) if err := r.txn.Insert("allocs", alloc); err != nil { return fmt.Errorf("alloc insert failed: %v", err) } @@ -825,65 +764,65 @@ func (r *StateRestore) IndexRestore(idx *IndexEntry) error { return nil } +// watchItem describes the scope of a watch. It is used to provide a uniform +// input for subscribe/unsubscribe and notification firing. +type watchItem struct { + allocID string + allocNode string + evalID string + jobID string + nodeID string + table string +} + // stateWatch holds shared state for watching updates. This is // outside of StateStore so it can be shared with snapshots. type stateWatch struct { - // Allocation watches by node - allocs map[string]*NotifyGroup - allocLock sync.Mutex - - // Full table watches - tables map[string]*NotifyGroup - tableLock sync.Mutex + items map[watchItem]*NotifyGroup + l sync.Mutex } -// watchTable is used to subscribe a channel to a full table watch. -func (w *stateWatch) watchTable(table string, ch chan struct{}) { - w.tableLock.Lock() - defer w.tableLock.Unlock() - - tw, ok := w.tables[table] - if !ok { - tw = new(NotifyGroup) - w.tables[table] = tw +// newStateWatch creates a new stateWatch for change notification. +func newStateWatch() *stateWatch { + return &stateWatch{ + items: make(map[watchItem]*NotifyGroup), } - tw.Wait(ch) } -// stopWatchTable is used to unsubscribe a channel from a table watch. -func (w *stateWatch) stopWatchTable(table string, ch chan struct{}) { - w.tableLock.Lock() - defer w.tableLock.Unlock() +// watch subscribes a channel to the given watch item. +func (w *stateWatch) watch(wi watchItem, ch chan struct{}) { + w.l.Lock() + defer w.l.Unlock() - if tw, ok := w.tables[table]; ok { - tw.Clear(ch) - if tw.Empty() { - delete(w.tables, table) - } + grp, ok := w.items[wi] + if !ok { + grp = new(NotifyGroup) + w.items[wi] = grp } + grp.Wait(ch) } -// notifyTables is used to notify watchers of the given tables. -func (w *stateWatch) notifyTables(tables map[string]struct{}) { - w.tableLock.Lock() - defer w.tableLock.Unlock() +// stopWatch unsubscribes a channel from the given watch item. +func (w *stateWatch) stopWatch(wi watchItem, ch chan struct{}) { + w.l.Lock() + defer w.l.Unlock() - for table, _ := range tables { - if tw, ok := w.tables[table]; ok { - tw.Notify() + if grp, ok := w.items[wi]; ok { + grp.Clear(ch) + if grp.Empty() { + delete(w.items, wi) } } } -// notifyAllocs is used to notify any node alloc listeners of a change -func (w *stateWatch) notifyAllocs(nodes map[string]struct{}) { - w.allocLock.Lock() - defer w.allocLock.Unlock() +// notify is used to fire notifications on the given watch items. +func (w *stateWatch) notify(items ...watchItem) { + w.l.Lock() + defer w.l.Unlock() - for node := range nodes { - if grp, ok := w.allocs[node]; ok { + for _, wi := range items { + if grp, ok := w.items[wi]; ok { grp.Notify() - delete(w.allocs, node) } } } From d9e593a0ae06d726566db9ce4e63a72bd15e0229 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Thu, 29 Oct 2015 12:09:09 -0700 Subject: [PATCH 12/26] nomad: deduplicate watch items with a helper --- nomad/state/state_store.go | 33 ++++++++++++++++++++++++++------- 1 file changed, 26 insertions(+), 7 deletions(-) diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 0a8adc0d01f..60af402d374 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -69,6 +69,7 @@ func (s *StateStore) Restore() (*StateRestore, error) { r := &StateRestore{ txn: txn, watch: s.watch, + items: make(watchItems), } return r, nil } @@ -705,7 +706,7 @@ type StateSnapshot struct { type StateRestore struct { txn *memdb.Txn watch *stateWatch - items []watchItem + items watchItems } // Abort is used to abort the restore operation @@ -715,13 +716,13 @@ func (s *StateRestore) Abort() { // Commit is used to commit the restore operation func (s *StateRestore) Commit() { - s.txn.Defer(func() { s.watch.notify(s.items...) }) + s.txn.Defer(func() { s.watch.notify(s.items.items()...) }) s.txn.Commit() } // NodeRestore is used to restore a node func (r *StateRestore) NodeRestore(node *structs.Node) error { - r.items = append(r.items, watchItem{table: "nodes"}) + r.items.add(watchItem{table: "nodes"}) if err := r.txn.Insert("nodes", node); err != nil { return fmt.Errorf("node insert failed: %v", err) } @@ -730,7 +731,7 @@ func (r *StateRestore) NodeRestore(node *structs.Node) error { // JobRestore is used to restore a job func (r *StateRestore) JobRestore(job *structs.Job) error { - r.items = append(r.items, watchItem{table: "jobs"}) + r.items.add(watchItem{table: "jobs"}) if err := r.txn.Insert("jobs", job); err != nil { return fmt.Errorf("job insert failed: %v", err) } @@ -739,7 +740,7 @@ func (r *StateRestore) JobRestore(job *structs.Job) error { // EvalRestore is used to restore an evaluation func (r *StateRestore) EvalRestore(eval *structs.Evaluation) error { - r.items = append(r.items, watchItem{table: "evals"}) + r.items.add(watchItem{table: "evals"}) if err := r.txn.Insert("evals", eval); err != nil { return fmt.Errorf("eval insert failed: %v", err) } @@ -748,8 +749,8 @@ func (r *StateRestore) EvalRestore(eval *structs.Evaluation) error { // AllocRestore is used to restore an allocation func (r *StateRestore) AllocRestore(alloc *structs.Allocation) error { - r.items = append(r.items, watchItem{table: "allocs"}) - r.items = append(r.items, watchItem{allocNode: alloc.NodeID}) + r.items.add(watchItem{table: "allocs"}) + r.items.add(watchItem{allocNode: alloc.NodeID}) if err := r.txn.Insert("allocs", alloc); err != nil { return fmt.Errorf("alloc insert failed: %v", err) } @@ -775,6 +776,24 @@ type watchItem struct { table string } +// watchItems is a helper used to construct a set of watchItems. It deduplicates +// the items as they are added using map keys. +type watchItems map[watchItem]struct{} + +// add adds an item to the watch set. +func (w watchItems) add(wi watchItem) { + w[wi] = struct{}{} +} + +// items returns the items as a slice. +func (w watchItems) items() []watchItem { + items := make([]watchItem, 0, len(w)) + for wi, _ := range w { + items = append(items, wi) + } + return items +} + // stateWatch holds shared state for watching updates. This is // outside of StateStore so it can be shared with snapshots. type stateWatch struct { From 31abf97e06ae51edcdc7b09aa8cee8dadf087ea2 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Thu, 29 Oct 2015 13:21:25 -0700 Subject: [PATCH 13/26] nomad: move state watcher into its own file, add tests --- nomad/state/state_store.go | 92 +++------------------------------ nomad/state/state_store_test.go | 10 ++-- nomad/state/watch.go | 86 ++++++++++++++++++++++++++++++ nomad/state/watch_test.go | 64 +++++++++++++++++++++++ 4 files changed, 163 insertions(+), 89 deletions(-) create mode 100644 nomad/state/watch.go create mode 100644 nomad/state/watch_test.go diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 60af402d374..6852334477c 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -4,7 +4,6 @@ import ( "fmt" "io" "log" - "sync" "github.com/hashicorp/go-memdb" "github.com/hashicorp/nomad/nomad/structs" @@ -405,6 +404,8 @@ func (s *StateStore) nestedUpsertEval(txn *memdb.Txn, index uint64, eval *struct func (s *StateStore) DeleteEval(index uint64, evals []string, allocs []string) error { txn := s.db.Txn(true) defer txn.Abort() + watch := make(watchItems) + watch.add(watchItem{table: "evals"}) for _, eval := range evals { existing, err := txn.First("evals", "id", eval) @@ -427,6 +428,7 @@ func (s *StateStore) DeleteEval(index uint64, evals []string, allocs []string) e if existing == nil { continue } + watch.add(watchItem{allocNode: existing.(*structs.Allocation).NodeID}) if err := txn.Delete("allocs", existing); err != nil { return fmt.Errorf("alloc delete failed: %v", err) } @@ -440,7 +442,7 @@ func (s *StateStore) DeleteEval(index uint64, evals []string, allocs []string) e return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watchItem{table: "evals"}) }) + txn.Defer(func() { s.watch.notify(watch.items()...) }) txn.Commit() return nil } @@ -545,9 +547,12 @@ func (s *StateStore) UpdateAllocFromClient(index uint64, alloc *structs.Allocati func (s *StateStore) UpsertAllocs(index uint64, allocs []*structs.Allocation) error { txn := s.db.Txn(true) defer txn.Abort() + watch := make(watchItems) + watch.add(watchItem{table: "allocs"}) // Handle the allocations for _, alloc := range allocs { + watch.add(watchItem{allocNode: alloc.NodeID}) existing, err := txn.First("allocs", "id", alloc.ID) if err != nil { return fmt.Errorf("alloc lookup failed: %v", err) @@ -573,7 +578,7 @@ func (s *StateStore) UpsertAllocs(index uint64, allocs []*structs.Allocation) er return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watchItem{table: "allocs"}) }) + txn.Defer(func() { s.watch.notify(watch.items()...) }) txn.Commit() return nil } @@ -764,84 +769,3 @@ func (r *StateRestore) IndexRestore(idx *IndexEntry) error { } return nil } - -// watchItem describes the scope of a watch. It is used to provide a uniform -// input for subscribe/unsubscribe and notification firing. -type watchItem struct { - allocID string - allocNode string - evalID string - jobID string - nodeID string - table string -} - -// watchItems is a helper used to construct a set of watchItems. It deduplicates -// the items as they are added using map keys. -type watchItems map[watchItem]struct{} - -// add adds an item to the watch set. -func (w watchItems) add(wi watchItem) { - w[wi] = struct{}{} -} - -// items returns the items as a slice. -func (w watchItems) items() []watchItem { - items := make([]watchItem, 0, len(w)) - for wi, _ := range w { - items = append(items, wi) - } - return items -} - -// stateWatch holds shared state for watching updates. This is -// outside of StateStore so it can be shared with snapshots. -type stateWatch struct { - items map[watchItem]*NotifyGroup - l sync.Mutex -} - -// newStateWatch creates a new stateWatch for change notification. -func newStateWatch() *stateWatch { - return &stateWatch{ - items: make(map[watchItem]*NotifyGroup), - } -} - -// watch subscribes a channel to the given watch item. -func (w *stateWatch) watch(wi watchItem, ch chan struct{}) { - w.l.Lock() - defer w.l.Unlock() - - grp, ok := w.items[wi] - if !ok { - grp = new(NotifyGroup) - w.items[wi] = grp - } - grp.Wait(ch) -} - -// stopWatch unsubscribes a channel from the given watch item. -func (w *stateWatch) stopWatch(wi watchItem, ch chan struct{}) { - w.l.Lock() - defer w.l.Unlock() - - if grp, ok := w.items[wi]; ok { - grp.Clear(ch) - if grp.Empty() { - delete(w.items, wi) - } - } -} - -// notify is used to fire notifications on the given watch items. -func (w *stateWatch) notify(items ...watchItem) { - w.l.Lock() - defer w.l.Unlock() - - for _, wi := range items { - if grp, ok := w.items[wi]; ok { - grp.Notify() - } - } -} diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index 1c4b602381f..58f8093bfd8 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -585,7 +585,7 @@ func TestStateStore_DeleteEval_Eval(t *testing.T) { } notify1 := make(chan struct{}, 1) - state.WatchAllocs(alloc.NodeID, notify1) + state.WatchAllocNode(alloc.NodeID, notify1) err = state.DeleteEval(1002, []string{eval.ID, eval2.ID}, []string{alloc.ID, alloc2.ID}) if err != nil { @@ -808,14 +808,14 @@ func TestStateStore_UpsertAlloc_Alloc(t *testing.T) { } } -func TestStateStore_WatchAllocs(t *testing.T) { +func TestStateStore_WatchAllocNode(t *testing.T) { state := testStateStore(t) notify1 := make(chan struct{}, 1) notify2 := make(chan struct{}, 1) - state.WatchAllocs("foo", notify1) - state.WatchAllocs("foo", notify2) - state.StopWatchAllocs("foo", notify2) + state.WatchAllocNode("foo", notify1) + state.WatchAllocNode("foo", notify2) + state.StopWatchAllocNode("foo", notify2) alloc := mock.Alloc() alloc.NodeID = "foo" diff --git a/nomad/state/watch.go b/nomad/state/watch.go new file mode 100644 index 00000000000..c0e529b28b2 --- /dev/null +++ b/nomad/state/watch.go @@ -0,0 +1,86 @@ +package state + +import ( + "sync" +) + +// watchItem describes the scope of a watch. It is used to provide a uniform +// input for subscribe/unsubscribe and notification firing. +type watchItem struct { + allocID string + allocNode string + evalID string + jobID string + nodeID string + table string +} + +// watchItems is a helper used to construct a set of watchItems. It deduplicates +// the items as they are added using map keys. +type watchItems map[watchItem]struct{} + +// add adds an item to the watch set. +func (w watchItems) add(wi watchItem) { + w[wi] = struct{}{} +} + +// items returns the items as a slice. +func (w watchItems) items() []watchItem { + items := make([]watchItem, 0, len(w)) + for wi, _ := range w { + items = append(items, wi) + } + return items +} + +// stateWatch holds shared state for watching updates. This is +// outside of StateStore so it can be shared with snapshots. +type stateWatch struct { + items map[watchItem]*NotifyGroup + l sync.Mutex +} + +// newStateWatch creates a new stateWatch for change notification. +func newStateWatch() *stateWatch { + return &stateWatch{ + items: make(map[watchItem]*NotifyGroup), + } +} + +// watch subscribes a channel to the given watch item. +func (w *stateWatch) watch(wi watchItem, ch chan struct{}) { + w.l.Lock() + defer w.l.Unlock() + + grp, ok := w.items[wi] + if !ok { + grp = new(NotifyGroup) + w.items[wi] = grp + } + grp.Wait(ch) +} + +// stopWatch unsubscribes a channel from the given watch item. +func (w *stateWatch) stopWatch(wi watchItem, ch chan struct{}) { + w.l.Lock() + defer w.l.Unlock() + + if grp, ok := w.items[wi]; ok { + grp.Clear(ch) + if grp.Empty() { + delete(w.items, wi) + } + } +} + +// notify is used to fire notifications on the given watch items. +func (w *stateWatch) notify(items ...watchItem) { + w.l.Lock() + defer w.l.Unlock() + + for _, wi := range items { + if grp, ok := w.items[wi]; ok { + grp.Notify() + } + } +} diff --git a/nomad/state/watch_test.go b/nomad/state/watch_test.go new file mode 100644 index 00000000000..5992b65ee8a --- /dev/null +++ b/nomad/state/watch_test.go @@ -0,0 +1,64 @@ +package state + +import ( + "testing" +) + +func TestWatchItems(t *testing.T) { + // No items returns empty slice + wi := make(watchItems) + if items := wi.items(); len(items) != 0 { + t.Fatalf("expected empty, got: %#v", items) + } + + // Adding items works + wi.add(watchItem{table: "foo"}) + wi.add(watchItem{nodeID: "bar"}) + if items := wi.items(); len(items) != 2 { + t.Fatalf("expected 2 items, got: %#v", items) + } + + // Adding duplicates auto-dedupes + wi.add(watchItem{table: "foo"}) + if items := wi.items(); len(items) != 2 { + t.Fatalf("expected 2 items, got: %#v", items) + } +} + +func TestStateWatch_watch(t *testing.T) { + watch := newStateWatch() + notify1 := make(chan struct{}, 1) + notify2 := make(chan struct{}, 1) + notify3 := make(chan struct{}, 1) + + // Notifications trigger subscribed channels + watch.watch(watchItem{table: "foo"}, notify1) + watch.watch(watchItem{table: "bar"}, notify2) + watch.watch(watchItem{table: "baz"}, notify3) + + watch.notify(watchItem{table: "foo"}, watchItem{table: "bar"}) + if len(notify1) != 1 { + t.Fatalf("should notify") + } + if len(notify2) != 1 { + t.Fatalf("should notify") + } + if len(notify3) != 0 { + t.Fatalf("should not notify") + } +} + +func TestStateWatch_stopWatch(t *testing.T) { + watch := newStateWatch() + notify := make(chan struct{}) + + // First subscribe + watch.watch(watchItem{table: "foo"}, notify) + + // Unsubscribe stop notifications + watch.stopWatch(watchItem{table: "foo"}, notify) + watch.notify(watchItem{table: "foo"}) + if len(notify) != 0 { + t.Fatalf("should not notify") + } +} From d9a77e0257fbac9decd70c4d4bd3fbb1652d6b3f Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Thu, 29 Oct 2015 13:52:15 -0700 Subject: [PATCH 14/26] nomad: add triggering for more types of events --- nomad/state/state_store.go | 67 ++++++++++++++++++++++++++++++-------- nomad/state/watch.go | 29 ++++++++--------- nomad/state/watch_test.go | 25 +++++++------- 3 files changed, 81 insertions(+), 40 deletions(-) diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 6852334477c..0895b77e854 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -100,6 +100,10 @@ func (s *StateStore) UpsertNode(index uint64, node *structs.Node) error { txn := s.db.Txn(true) defer txn.Abort() + watch := make(watchItems) + watch.add(watchItem{table: "nodes"}) + watch.add(watchItem{node: node.ID}) + // Check if the node already exists existing, err := txn.First("nodes", "id", node.ID) if err != nil { @@ -125,7 +129,7 @@ func (s *StateStore) UpsertNode(index uint64, node *structs.Node) error { return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watchItem{table: "nodes"}) }) + txn.Defer(func() { s.watch.notify(watch) }) txn.Commit() return nil } @@ -135,6 +139,10 @@ func (s *StateStore) DeleteNode(index uint64, nodeID string) error { txn := s.db.Txn(true) defer txn.Abort() + watch := make(watchItems) + watch.add(watchItem{table: "nodes"}) + watch.add(watchItem{node: nodeID}) + // Lookup the node existing, err := txn.First("nodes", "id", nodeID) if err != nil { @@ -152,7 +160,7 @@ func (s *StateStore) DeleteNode(index uint64, nodeID string) error { return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watchItem{table: "nodes"}) }) + txn.Defer(func() { s.watch.notify(watch) }) txn.Commit() return nil } @@ -162,6 +170,10 @@ func (s *StateStore) UpdateNodeStatus(index uint64, nodeID, status string) error txn := s.db.Txn(true) defer txn.Abort() + watch := make(watchItems) + watch.add(watchItem{table: "nodes"}) + watch.add(watchItem{node: nodeID}) + // Lookup the node existing, err := txn.First("nodes", "id", nodeID) if err != nil { @@ -188,7 +200,7 @@ func (s *StateStore) UpdateNodeStatus(index uint64, nodeID, status string) error return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watchItem{table: "nodes"}) }) + txn.Defer(func() { s.watch.notify(watch) }) txn.Commit() return nil } @@ -198,6 +210,10 @@ func (s *StateStore) UpdateNodeDrain(index uint64, nodeID string, drain bool) er txn := s.db.Txn(true) defer txn.Abort() + watch := make(watchItems) + watch.add(watchItem{table: "nodes"}) + watch.add(watchItem{node: nodeID}) + // Lookup the node existing, err := txn.First("nodes", "id", nodeID) if err != nil { @@ -224,7 +240,7 @@ func (s *StateStore) UpdateNodeDrain(index uint64, nodeID string, drain bool) er return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watchItem{table: "nodes"}) }) + txn.Defer(func() { s.watch.notify(watch) }) txn.Commit() return nil } @@ -261,6 +277,10 @@ func (s *StateStore) UpsertJob(index uint64, job *structs.Job) error { txn := s.db.Txn(true) defer txn.Abort() + watch := make(watchItems) + watch.add(watchItem{table: "jobs"}) + watch.add(watchItem{job: job.ID}) + // Check if the job already exists existing, err := txn.First("jobs", "id", job.ID) if err != nil { @@ -284,7 +304,7 @@ func (s *StateStore) UpsertJob(index uint64, job *structs.Job) error { return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watchItem{table: "jobs"}) }) + txn.Defer(func() { s.watch.notify(watch) }) txn.Commit() return nil } @@ -294,6 +314,10 @@ func (s *StateStore) DeleteJob(index uint64, jobID string) error { txn := s.db.Txn(true) defer txn.Abort() + watch := make(watchItems) + watch.add(watchItem{table: "jobs"}) + watch.add(watchItem{job: jobID}) + // Lookup the node existing, err := txn.First("jobs", "id", jobID) if err != nil { @@ -311,7 +335,7 @@ func (s *StateStore) DeleteJob(index uint64, jobID string) error { return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watchItem{table: "jobs"}) }) + txn.Defer(func() { s.watch.notify(watch) }) txn.Commit() return nil } @@ -361,14 +385,18 @@ func (s *StateStore) UpsertEvals(index uint64, evals []*structs.Evaluation) erro txn := s.db.Txn(true) defer txn.Abort() + watch := make(watchItems) + watch.add(watchItem{table: "evals"}) + // Do a nested upsert for _, eval := range evals { + watch.add(watchItem{eval: eval.ID}) if err := s.nestedUpsertEval(txn, index, eval); err != nil { return err } } - txn.Defer(func() { s.watch.notify(watchItem{table: "evals"}) }) + txn.Defer(func() { s.watch.notify(watch) }) txn.Commit() return nil } @@ -405,7 +433,6 @@ func (s *StateStore) DeleteEval(index uint64, evals []string, allocs []string) e txn := s.db.Txn(true) defer txn.Abort() watch := make(watchItems) - watch.add(watchItem{table: "evals"}) for _, eval := range evals { existing, err := txn.First("evals", "id", eval) @@ -418,6 +445,8 @@ func (s *StateStore) DeleteEval(index uint64, evals []string, allocs []string) e if err := txn.Delete("evals", existing); err != nil { return fmt.Errorf("eval delete failed: %v", err) } + watch.add(watchItem{table: "evals"}) + watch.add(watchItem{eval: eval}) } for _, alloc := range allocs { @@ -428,10 +457,12 @@ func (s *StateStore) DeleteEval(index uint64, evals []string, allocs []string) e if existing == nil { continue } - watch.add(watchItem{allocNode: existing.(*structs.Allocation).NodeID}) if err := txn.Delete("allocs", existing); err != nil { return fmt.Errorf("alloc delete failed: %v", err) } + watch.add(watchItem{table: "allocs"}) + watch.add(watchItem{alloc: alloc}) + watch.add(watchItem{allocNode: existing.(*structs.Allocation).NodeID}) } // Update the indexes @@ -442,7 +473,7 @@ func (s *StateStore) DeleteEval(index uint64, evals []string, allocs []string) e return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watch.items()...) }) + txn.Defer(func() { s.watch.notify(watch) }) txn.Commit() return nil } @@ -504,6 +535,11 @@ func (s *StateStore) UpdateAllocFromClient(index uint64, alloc *structs.Allocati txn := s.db.Txn(true) defer txn.Abort() + watch := make(watchItems) + watch.add(watchItem{table: "allocs"}) + watch.add(watchItem{alloc: alloc.ID}) + watch.add(watchItem{allocNode: alloc.NodeID}) + // Look for existing alloc existing, err := txn.First("allocs", "id", alloc.ID) if err != nil { @@ -537,7 +573,7 @@ func (s *StateStore) UpdateAllocFromClient(index uint64, alloc *structs.Allocati return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watchItem{table: "allocs"}) }) + txn.Defer(func() { s.watch.notify(watch) }) txn.Commit() return nil } @@ -547,12 +583,12 @@ func (s *StateStore) UpdateAllocFromClient(index uint64, alloc *structs.Allocati func (s *StateStore) UpsertAllocs(index uint64, allocs []*structs.Allocation) error { txn := s.db.Txn(true) defer txn.Abort() + watch := make(watchItems) watch.add(watchItem{table: "allocs"}) // Handle the allocations for _, alloc := range allocs { - watch.add(watchItem{allocNode: alloc.NodeID}) existing, err := txn.First("allocs", "id", alloc.ID) if err != nil { return fmt.Errorf("alloc lookup failed: %v", err) @@ -571,6 +607,9 @@ func (s *StateStore) UpsertAllocs(index uint64, allocs []*structs.Allocation) er if err := txn.Insert("allocs", alloc); err != nil { return fmt.Errorf("alloc insert failed: %v", err) } + + watch.add(watchItem{alloc: alloc.ID}) + watch.add(watchItem{allocNode: alloc.NodeID}) } // Update the indexes @@ -578,7 +617,7 @@ func (s *StateStore) UpsertAllocs(index uint64, allocs []*structs.Allocation) er return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watch.items()...) }) + txn.Defer(func() { s.watch.notify(watch) }) txn.Commit() return nil } @@ -721,7 +760,7 @@ func (s *StateRestore) Abort() { // Commit is used to commit the restore operation func (s *StateRestore) Commit() { - s.txn.Defer(func() { s.watch.notify(s.items.items()...) }) + s.txn.Defer(func() { s.watch.notify(s.items) }) s.txn.Commit() } diff --git a/nomad/state/watch.go b/nomad/state/watch.go index c0e529b28b2..124eb061271 100644 --- a/nomad/state/watch.go +++ b/nomad/state/watch.go @@ -7,11 +7,11 @@ import ( // watchItem describes the scope of a watch. It is used to provide a uniform // input for subscribe/unsubscribe and notification firing. type watchItem struct { - allocID string + alloc string allocNode string - evalID string - jobID string - nodeID string + eval string + job string + node string table string } @@ -19,20 +19,19 @@ type watchItem struct { // the items as they are added using map keys. type watchItems map[watchItem]struct{} +func newWatchItems(items ...watchItem) watchItems { + wi := make(watchItems) + for _, item := range items { + wi.add(item) + } + return wi +} + // add adds an item to the watch set. func (w watchItems) add(wi watchItem) { w[wi] = struct{}{} } -// items returns the items as a slice. -func (w watchItems) items() []watchItem { - items := make([]watchItem, 0, len(w)) - for wi, _ := range w { - items = append(items, wi) - } - return items -} - // stateWatch holds shared state for watching updates. This is // outside of StateStore so it can be shared with snapshots. type stateWatch struct { @@ -74,11 +73,11 @@ func (w *stateWatch) stopWatch(wi watchItem, ch chan struct{}) { } // notify is used to fire notifications on the given watch items. -func (w *stateWatch) notify(items ...watchItem) { +func (w *stateWatch) notify(items watchItems) { w.l.Lock() defer w.l.Unlock() - for _, wi := range items { + for wi, _ := range items { if grp, ok := w.items[wi]; ok { grp.Notify() } diff --git a/nomad/state/watch_test.go b/nomad/state/watch_test.go index 5992b65ee8a..aad53d798e1 100644 --- a/nomad/state/watch_test.go +++ b/nomad/state/watch_test.go @@ -5,23 +5,19 @@ import ( ) func TestWatchItems(t *testing.T) { - // No items returns empty slice wi := make(watchItems) - if items := wi.items(); len(items) != 0 { - t.Fatalf("expected empty, got: %#v", items) - } // Adding items works wi.add(watchItem{table: "foo"}) - wi.add(watchItem{nodeID: "bar"}) - if items := wi.items(); len(items) != 2 { - t.Fatalf("expected 2 items, got: %#v", items) + wi.add(watchItem{node: "bar"}) + if len(wi) != 2 { + t.Fatalf("expected 2 items, got: %#v", wi) } // Adding duplicates auto-dedupes wi.add(watchItem{table: "foo"}) - if items := wi.items(); len(items) != 2 { - t.Fatalf("expected 2 items, got: %#v", items) + if len(wi) != 2 { + t.Fatalf("expected 2 items, got: %#v", wi) } } @@ -36,7 +32,11 @@ func TestStateWatch_watch(t *testing.T) { watch.watch(watchItem{table: "bar"}, notify2) watch.watch(watchItem{table: "baz"}, notify3) - watch.notify(watchItem{table: "foo"}, watchItem{table: "bar"}) + items := make(watchItems) + items.add(watchItem{table: "foo"}) + items.add(watchItem{table: "bar"}) + + watch.notify(items) if len(notify1) != 1 { t.Fatalf("should notify") } @@ -57,7 +57,10 @@ func TestStateWatch_stopWatch(t *testing.T) { // Unsubscribe stop notifications watch.stopWatch(watchItem{table: "foo"}, notify) - watch.notify(watchItem{table: "foo"}) + + items := make(watchItems) + items.add(watchItem{table: "foo"}) + watch.notify(items) if len(notify) != 0 { t.Fatalf("should not notify") } From 573e9dfb9abfb15c51602797a0c0ab05fca4ddf2 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Thu, 29 Oct 2015 14:47:39 -0700 Subject: [PATCH 15/26] nomad: export watcher to share between rpc and state store --- nomad/alloc_endpoint.go | 7 +- nomad/eval_endpoint.go | 7 +- nomad/job_endpoint.go | 7 +- nomad/node_endpoint.go | 13 +-- nomad/rpc.go | 16 ++- nomad/state/state_store.go | 182 ++++++++++++++++++++------------ nomad/state/state_store_test.go | 56 +++++++++- nomad/state/watch.go | 85 --------------- nomad/state/watch_test.go | 67 ------------ nomad/watch/watch.go | 33 ++++++ nomad/watch/watch_test.go | 31 ++++++ 11 files changed, 255 insertions(+), 249 deletions(-) delete mode 100644 nomad/state/watch.go delete mode 100644 nomad/state/watch_test.go create mode 100644 nomad/watch/watch.go create mode 100644 nomad/watch/watch_test.go diff --git a/nomad/alloc_endpoint.go b/nomad/alloc_endpoint.go index a2ce6a09a80..f3b9dbdc4c2 100644 --- a/nomad/alloc_endpoint.go +++ b/nomad/alloc_endpoint.go @@ -5,6 +5,7 @@ import ( "github.com/armon/go-metrics" "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/nomad/watch" ) // Alloc endpoint is used for manipulating allocations @@ -21,9 +22,9 @@ func (a *Alloc) List(args *structs.AllocListRequest, reply *structs.AllocListRes // Setup the blocking query opts := blockingOptions{ - queryOpts: &args.QueryOptions, - queryMeta: &reply.QueryMeta, - watchTable: "allocs", + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watch: watch.NewItems(watch.Item{Table: "allocs"}), run: func() error { // Capture all the allocations snap, err := a.srv.fsm.State().Snapshot() diff --git a/nomad/eval_endpoint.go b/nomad/eval_endpoint.go index 5d87948aa27..07dfc18fe70 100644 --- a/nomad/eval_endpoint.go +++ b/nomad/eval_endpoint.go @@ -6,6 +6,7 @@ import ( "github.com/armon/go-metrics" "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/nomad/watch" ) const ( @@ -221,9 +222,9 @@ func (e *Eval) List(args *structs.EvalListRequest, // Setup the blocking query opts := blockingOptions{ - queryOpts: &args.QueryOptions, - queryMeta: &reply.QueryMeta, - watchTable: "evals", + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watch: watch.NewItems(watch.Item{Table: "evals"}), run: func() error { // Scan all the evaluations snap, err := e.srv.fsm.State().Snapshot() diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index 30bc35563da..ff296a986d7 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -6,6 +6,7 @@ import ( "github.com/armon/go-metrics" "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/nomad/watch" ) // Job endpoint is used for job interactions @@ -218,9 +219,9 @@ func (j *Job) List(args *structs.JobListRequest, // Setup the blocking query opts := blockingOptions{ - queryOpts: &args.QueryOptions, - queryMeta: &reply.QueryMeta, - watchTable: "jobs", + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watch: watch.NewItems(watch.Item{Table: "jobs"}), run: func() error { // Capture all the jobs snap, err := j.srv.fsm.State().Snapshot() diff --git a/nomad/node_endpoint.go b/nomad/node_endpoint.go index 7e73329742a..65a83e1b122 100644 --- a/nomad/node_endpoint.go +++ b/nomad/node_endpoint.go @@ -6,6 +6,7 @@ import ( "github.com/armon/go-metrics" "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/nomad/watch" ) // Node endpoint is used for client interactions @@ -330,9 +331,9 @@ func (n *Node) GetAllocs(args *structs.NodeSpecificRequest, // Setup the blocking query opts := blockingOptions{ - queryOpts: &args.QueryOptions, - queryMeta: &reply.QueryMeta, - watchAllocNode: args.NodeID, + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watch: watch.NewItems(watch.Item{AllocNode: args.NodeID}), run: func() error { // Look for the node snap, err := n.srv.fsm.State().Snapshot() @@ -406,9 +407,9 @@ func (n *Node) List(args *structs.NodeListRequest, // Setup the blocking query opts := blockingOptions{ - queryOpts: &args.QueryOptions, - queryMeta: &reply.QueryMeta, - watchTable: "nodes", + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watch: watch.NewItems(watch.Item{Table: "nodes"}), run: func() error { // Capture all the nodes snap, err := n.srv.fsm.State().Snapshot() diff --git a/nomad/rpc.go b/nomad/rpc.go index f1977dbc72f..21f9c9dc671 100644 --- a/nomad/rpc.go +++ b/nomad/rpc.go @@ -13,6 +13,7 @@ import ( "github.com/hashicorp/net-rpc-msgpackrpc" "github.com/hashicorp/nomad/nomad/state" "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/nomad/watch" "github.com/hashicorp/raft" "github.com/hashicorp/yamux" ) @@ -268,11 +269,10 @@ func (s *Server) setQueryMeta(m *structs.QueryMeta) { // blockingOptions is used to parameterize blockingRPC type blockingOptions struct { - queryOpts *structs.QueryOptions - queryMeta *structs.QueryMeta - watchAllocNode string - watchTable string - run func() error + queryOpts *structs.QueryOptions + queryMeta *structs.QueryMeta + watch watch.Items + run func() error } // blockingRPC is used for queries that need to wait for a @@ -307,15 +307,13 @@ func (s *Server) blockingRPC(opts *blockingOptions) error { state = s.fsm.State() defer func() { timeout.Stop() - state.StopWatchAllocNode(opts.watchAllocNode, notifyCh) - state.StopWatchTable(opts.watchTable, notifyCh) + state.StopWatch(opts.watch, notifyCh) }() REGISTER_NOTIFY: // Register the notification channel. This may be done // multiple times if we have not reached the target wait index. - state.WatchAllocNode(opts.watchAllocNode, notifyCh) - state.WatchTable(opts.watchTable, notifyCh) + state.Watch(opts.watch, notifyCh) RUN_QUERY: // Update the query meta data diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 0895b77e854..17484f985c8 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -4,9 +4,11 @@ import ( "fmt" "io" "log" + "sync" "github.com/hashicorp/go-memdb" "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/nomad/watch" ) // IndexEntry is used with the "index" table @@ -68,29 +70,21 @@ func (s *StateStore) Restore() (*StateRestore, error) { r := &StateRestore{ txn: txn, watch: s.watch, - items: make(watchItems), + items: watch.NewItems(), } return r, nil } -// WatchTable is used to subscribe a channel to a full table watch. -func (s *StateStore) WatchTable(table string, notify chan struct{}) { - s.watch.watch(watchItem{table: table}, notify) -} - -// StopWatchTable unsubscribes a channel from a full table watch. -func (s *StateStore) StopWatchTable(table string, notify chan struct{}) { - s.watch.stopWatch(watchItem{table: table}, notify) -} - -// WatchAllocNode is used to subscribe a channel to a node allocation watch. -func (s *StateStore) WatchAllocNode(nodeID string, notify chan struct{}) { - s.watch.watch(watchItem{allocNode: nodeID}, notify) +func (s *StateStore) Watch(items watch.Items, notify chan struct{}) { + for wi, _ := range items { + s.watch.watch(wi, notify) + } } -// StopWatchAllocNode unsubscribes a channel from a node allocation watch. -func (s *StateStore) StopWatchAllocNode(nodeID string, notify chan struct{}) { - s.watch.stopWatch(watchItem{allocNode: nodeID}, notify) +func (s *StateStore) StopWatch(items watch.Items, notify chan struct{}) { + for wi, _ := range items { + s.watch.stopWatch(wi, notify) + } } // UpsertNode is used to register a node or update a node definition @@ -100,9 +94,9 @@ func (s *StateStore) UpsertNode(index uint64, node *structs.Node) error { txn := s.db.Txn(true) defer txn.Abort() - watch := make(watchItems) - watch.add(watchItem{table: "nodes"}) - watch.add(watchItem{node: node.ID}) + watcher := watch.NewItems() + watcher.Add(watch.Item{Table: "nodes"}) + watcher.Add(watch.Item{Node: node.ID}) // Check if the node already exists existing, err := txn.First("nodes", "id", node.ID) @@ -129,7 +123,7 @@ func (s *StateStore) UpsertNode(index uint64, node *structs.Node) error { return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watch) }) + txn.Defer(func() { s.watch.notify(watcher) }) txn.Commit() return nil } @@ -139,9 +133,9 @@ func (s *StateStore) DeleteNode(index uint64, nodeID string) error { txn := s.db.Txn(true) defer txn.Abort() - watch := make(watchItems) - watch.add(watchItem{table: "nodes"}) - watch.add(watchItem{node: nodeID}) + watcher := watch.NewItems() + watcher.Add(watch.Item{Table: "nodes"}) + watcher.Add(watch.Item{Node: nodeID}) // Lookup the node existing, err := txn.First("nodes", "id", nodeID) @@ -160,7 +154,7 @@ func (s *StateStore) DeleteNode(index uint64, nodeID string) error { return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watch) }) + txn.Defer(func() { s.watch.notify(watcher) }) txn.Commit() return nil } @@ -170,9 +164,9 @@ func (s *StateStore) UpdateNodeStatus(index uint64, nodeID, status string) error txn := s.db.Txn(true) defer txn.Abort() - watch := make(watchItems) - watch.add(watchItem{table: "nodes"}) - watch.add(watchItem{node: nodeID}) + watcher := watch.NewItems() + watcher.Add(watch.Item{Table: "nodes"}) + watcher.Add(watch.Item{Node: nodeID}) // Lookup the node existing, err := txn.First("nodes", "id", nodeID) @@ -200,7 +194,7 @@ func (s *StateStore) UpdateNodeStatus(index uint64, nodeID, status string) error return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watch) }) + txn.Defer(func() { s.watch.notify(watcher) }) txn.Commit() return nil } @@ -210,9 +204,9 @@ func (s *StateStore) UpdateNodeDrain(index uint64, nodeID string, drain bool) er txn := s.db.Txn(true) defer txn.Abort() - watch := make(watchItems) - watch.add(watchItem{table: "nodes"}) - watch.add(watchItem{node: nodeID}) + watcher := watch.NewItems() + watcher.Add(watch.Item{Table: "nodes"}) + watcher.Add(watch.Item{Node: nodeID}) // Lookup the node existing, err := txn.First("nodes", "id", nodeID) @@ -240,7 +234,7 @@ func (s *StateStore) UpdateNodeDrain(index uint64, nodeID string, drain bool) er return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watch) }) + txn.Defer(func() { s.watch.notify(watcher) }) txn.Commit() return nil } @@ -277,9 +271,9 @@ func (s *StateStore) UpsertJob(index uint64, job *structs.Job) error { txn := s.db.Txn(true) defer txn.Abort() - watch := make(watchItems) - watch.add(watchItem{table: "jobs"}) - watch.add(watchItem{job: job.ID}) + watcher := watch.NewItems() + watcher.Add(watch.Item{Table: "jobs"}) + watcher.Add(watch.Item{Job: job.ID}) // Check if the job already exists existing, err := txn.First("jobs", "id", job.ID) @@ -304,7 +298,7 @@ func (s *StateStore) UpsertJob(index uint64, job *structs.Job) error { return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watch) }) + txn.Defer(func() { s.watch.notify(watcher) }) txn.Commit() return nil } @@ -314,9 +308,9 @@ func (s *StateStore) DeleteJob(index uint64, jobID string) error { txn := s.db.Txn(true) defer txn.Abort() - watch := make(watchItems) - watch.add(watchItem{table: "jobs"}) - watch.add(watchItem{job: jobID}) + watcher := watch.NewItems() + watcher.Add(watch.Item{Table: "jobs"}) + watcher.Add(watch.Item{Job: jobID}) // Lookup the node existing, err := txn.First("jobs", "id", jobID) @@ -335,7 +329,7 @@ func (s *StateStore) DeleteJob(index uint64, jobID string) error { return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watch) }) + txn.Defer(func() { s.watch.notify(watcher) }) txn.Commit() return nil } @@ -385,18 +379,18 @@ func (s *StateStore) UpsertEvals(index uint64, evals []*structs.Evaluation) erro txn := s.db.Txn(true) defer txn.Abort() - watch := make(watchItems) - watch.add(watchItem{table: "evals"}) + watcher := watch.NewItems() + watcher.Add(watch.Item{Table: "evals"}) // Do a nested upsert for _, eval := range evals { - watch.add(watchItem{eval: eval.ID}) + watcher.Add(watch.Item{Eval: eval.ID}) if err := s.nestedUpsertEval(txn, index, eval); err != nil { return err } } - txn.Defer(func() { s.watch.notify(watch) }) + txn.Defer(func() { s.watch.notify(watcher) }) txn.Commit() return nil } @@ -432,7 +426,9 @@ func (s *StateStore) nestedUpsertEval(txn *memdb.Txn, index uint64, eval *struct func (s *StateStore) DeleteEval(index uint64, evals []string, allocs []string) error { txn := s.db.Txn(true) defer txn.Abort() - watch := make(watchItems) + watcher := watch.NewItems() + watcher.Add(watch.Item{Table: "evals"}) + watcher.Add(watch.Item{Table: "allocs"}) for _, eval := range evals { existing, err := txn.First("evals", "id", eval) @@ -445,8 +441,7 @@ func (s *StateStore) DeleteEval(index uint64, evals []string, allocs []string) e if err := txn.Delete("evals", existing); err != nil { return fmt.Errorf("eval delete failed: %v", err) } - watch.add(watchItem{table: "evals"}) - watch.add(watchItem{eval: eval}) + watcher.Add(watch.Item{Eval: eval}) } for _, alloc := range allocs { @@ -460,9 +455,8 @@ func (s *StateStore) DeleteEval(index uint64, evals []string, allocs []string) e if err := txn.Delete("allocs", existing); err != nil { return fmt.Errorf("alloc delete failed: %v", err) } - watch.add(watchItem{table: "allocs"}) - watch.add(watchItem{alloc: alloc}) - watch.add(watchItem{allocNode: existing.(*structs.Allocation).NodeID}) + watcher.Add(watch.Item{Alloc: alloc}) + watcher.Add(watch.Item{AllocNode: existing.(*structs.Allocation).NodeID}) } // Update the indexes @@ -473,7 +467,7 @@ func (s *StateStore) DeleteEval(index uint64, evals []string, allocs []string) e return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watch) }) + txn.Defer(func() { s.watch.notify(watcher) }) txn.Commit() return nil } @@ -535,10 +529,10 @@ func (s *StateStore) UpdateAllocFromClient(index uint64, alloc *structs.Allocati txn := s.db.Txn(true) defer txn.Abort() - watch := make(watchItems) - watch.add(watchItem{table: "allocs"}) - watch.add(watchItem{alloc: alloc.ID}) - watch.add(watchItem{allocNode: alloc.NodeID}) + watcher := watch.NewItems() + watcher.Add(watch.Item{Table: "allocs"}) + watcher.Add(watch.Item{Alloc: alloc.ID}) + watcher.Add(watch.Item{AllocNode: alloc.NodeID}) // Look for existing alloc existing, err := txn.First("allocs", "id", alloc.ID) @@ -573,7 +567,7 @@ func (s *StateStore) UpdateAllocFromClient(index uint64, alloc *structs.Allocati return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watch) }) + txn.Defer(func() { s.watch.notify(watcher) }) txn.Commit() return nil } @@ -584,8 +578,8 @@ func (s *StateStore) UpsertAllocs(index uint64, allocs []*structs.Allocation) er txn := s.db.Txn(true) defer txn.Abort() - watch := make(watchItems) - watch.add(watchItem{table: "allocs"}) + watcher := watch.NewItems() + watcher.Add(watch.Item{Table: "allocs"}) // Handle the allocations for _, alloc := range allocs { @@ -608,8 +602,8 @@ func (s *StateStore) UpsertAllocs(index uint64, allocs []*structs.Allocation) er return fmt.Errorf("alloc insert failed: %v", err) } - watch.add(watchItem{alloc: alloc.ID}) - watch.add(watchItem{allocNode: alloc.NodeID}) + watcher.Add(watch.Item{Alloc: alloc.ID}) + watcher.Add(watch.Item{AllocNode: alloc.NodeID}) } // Update the indexes @@ -617,7 +611,7 @@ func (s *StateStore) UpsertAllocs(index uint64, allocs []*structs.Allocation) er return fmt.Errorf("index update failed: %v", err) } - txn.Defer(func() { s.watch.notify(watch) }) + txn.Defer(func() { s.watch.notify(watcher) }) txn.Commit() return nil } @@ -750,7 +744,7 @@ type StateSnapshot struct { type StateRestore struct { txn *memdb.Txn watch *stateWatch - items watchItems + items watch.Items } // Abort is used to abort the restore operation @@ -766,7 +760,7 @@ func (s *StateRestore) Commit() { // NodeRestore is used to restore a node func (r *StateRestore) NodeRestore(node *structs.Node) error { - r.items.add(watchItem{table: "nodes"}) + r.items.Add(watch.Item{Table: "nodes"}) if err := r.txn.Insert("nodes", node); err != nil { return fmt.Errorf("node insert failed: %v", err) } @@ -775,7 +769,7 @@ func (r *StateRestore) NodeRestore(node *structs.Node) error { // JobRestore is used to restore a job func (r *StateRestore) JobRestore(job *structs.Job) error { - r.items.add(watchItem{table: "jobs"}) + r.items.Add(watch.Item{Table: "jobs"}) if err := r.txn.Insert("jobs", job); err != nil { return fmt.Errorf("job insert failed: %v", err) } @@ -784,7 +778,7 @@ func (r *StateRestore) JobRestore(job *structs.Job) error { // EvalRestore is used to restore an evaluation func (r *StateRestore) EvalRestore(eval *structs.Evaluation) error { - r.items.add(watchItem{table: "evals"}) + r.items.Add(watch.Item{Table: "evals"}) if err := r.txn.Insert("evals", eval); err != nil { return fmt.Errorf("eval insert failed: %v", err) } @@ -793,8 +787,8 @@ func (r *StateRestore) EvalRestore(eval *structs.Evaluation) error { // AllocRestore is used to restore an allocation func (r *StateRestore) AllocRestore(alloc *structs.Allocation) error { - r.items.add(watchItem{table: "allocs"}) - r.items.add(watchItem{allocNode: alloc.NodeID}) + r.items.Add(watch.Item{Table: "allocs"}) + r.items.Add(watch.Item{AllocNode: alloc.NodeID}) if err := r.txn.Insert("allocs", alloc); err != nil { return fmt.Errorf("alloc insert failed: %v", err) } @@ -808,3 +802,55 @@ func (r *StateRestore) IndexRestore(idx *IndexEntry) error { } return nil } + +// stateWatch holds shared state for watching updates. This is +// outside of StateStore so it can be shared with snapshots. +type stateWatch struct { + items map[watch.Item]*NotifyGroup + l sync.Mutex +} + +// newStateWatch creates a new stateWatch for change notification. +func newStateWatch() *stateWatch { + return &stateWatch{ + items: make(map[watch.Item]*NotifyGroup), + } +} + +// watch subscribes a channel to the given watch item. +func (w *stateWatch) watch(wi watch.Item, ch chan struct{}) { + w.l.Lock() + defer w.l.Unlock() + + grp, ok := w.items[wi] + if !ok { + grp = new(NotifyGroup) + w.items[wi] = grp + } + grp.Wait(ch) +} + +// stopWatch unsubscribes a channel from the given watch item. +func (w *stateWatch) stopWatch(wi watch.Item, ch chan struct{}) { + w.l.Lock() + defer w.l.Unlock() + + if grp, ok := w.items[wi]; ok { + grp.Clear(ch) + if grp.Empty() { + delete(w.items, wi) + } + } +} + +// notify is used to fire notifications on the given watch items. +func (w *stateWatch) notify(items watch.Items) { + w.l.Lock() + defer w.l.Unlock() + + for wi, _ := range items { + if grp, ok := w.items[wi]; ok { + grp.Notify() + } + } +} diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index 58f8093bfd8..2d1134d9c01 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -8,6 +8,7 @@ import ( "github.com/hashicorp/nomad/nomad/mock" "github.com/hashicorp/nomad/nomad/structs" + "github.com/hashicorp/nomad/nomad/watch" ) func testStateStore(t *testing.T) *StateStore { @@ -585,7 +586,7 @@ func TestStateStore_DeleteEval_Eval(t *testing.T) { } notify1 := make(chan struct{}, 1) - state.WatchAllocNode(alloc.NodeID, notify1) + state.Watch(watch.NewItems(watch.Item{AllocNode: alloc.NodeID}), notify1) err = state.DeleteEval(1002, []string{eval.ID, eval2.ID}, []string{alloc.ID, alloc2.ID}) if err != nil { @@ -808,14 +809,16 @@ func TestStateStore_UpsertAlloc_Alloc(t *testing.T) { } } -func TestStateStore_WatchAllocNode(t *testing.T) { +func TestStateStore_Watch(t *testing.T) { state := testStateStore(t) notify1 := make(chan struct{}, 1) notify2 := make(chan struct{}, 1) - state.WatchAllocNode("foo", notify1) - state.WatchAllocNode("foo", notify2) - state.StopWatchAllocNode("foo", notify2) + + items := watch.NewItems(watch.Item{AllocNode: "foo"}) + state.Watch(items, notify1) + state.Watch(items, notify2) + state.StopWatch(items, notify2) alloc := mock.Alloc() alloc.NodeID = "foo" @@ -1032,6 +1035,49 @@ func TestStateStore_RestoreAlloc(t *testing.T) { } } +func TestStateWatch_watch(t *testing.T) { + sw := newStateWatch() + notify1 := make(chan struct{}, 1) + notify2 := make(chan struct{}, 1) + notify3 := make(chan struct{}, 1) + + // Notifications trigger subscribed channels + sw.watch(watch.Item{Table: "foo"}, notify1) + sw.watch(watch.Item{Table: "bar"}, notify2) + sw.watch(watch.Item{Table: "baz"}, notify3) + + items := watch.NewItems() + items.Add(watch.Item{Table: "foo"}) + items.Add(watch.Item{Table: "bar"}) + + sw.notify(items) + if len(notify1) != 1 { + t.Fatalf("should notify") + } + if len(notify2) != 1 { + t.Fatalf("should notify") + } + if len(notify3) != 0 { + t.Fatalf("should not notify") + } +} + +func TestStateWatch_stopWatch(t *testing.T) { + sw := newStateWatch() + notify := make(chan struct{}) + + // First subscribe + sw.watch(watch.Item{Table: "foo"}, notify) + + // Unsubscribe stop notifications + sw.stopWatch(watch.Item{Table: "foo"}, notify) + + sw.notify(watch.NewItems(watch.Item{Table: "foo"})) + if len(notify) != 0 { + t.Fatalf("should not notify") + } +} + // NodeIDSort is used to sort nodes by ID type NodeIDSort []*structs.Node diff --git a/nomad/state/watch.go b/nomad/state/watch.go deleted file mode 100644 index 124eb061271..00000000000 --- a/nomad/state/watch.go +++ /dev/null @@ -1,85 +0,0 @@ -package state - -import ( - "sync" -) - -// watchItem describes the scope of a watch. It is used to provide a uniform -// input for subscribe/unsubscribe and notification firing. -type watchItem struct { - alloc string - allocNode string - eval string - job string - node string - table string -} - -// watchItems is a helper used to construct a set of watchItems. It deduplicates -// the items as they are added using map keys. -type watchItems map[watchItem]struct{} - -func newWatchItems(items ...watchItem) watchItems { - wi := make(watchItems) - for _, item := range items { - wi.add(item) - } - return wi -} - -// add adds an item to the watch set. -func (w watchItems) add(wi watchItem) { - w[wi] = struct{}{} -} - -// stateWatch holds shared state for watching updates. This is -// outside of StateStore so it can be shared with snapshots. -type stateWatch struct { - items map[watchItem]*NotifyGroup - l sync.Mutex -} - -// newStateWatch creates a new stateWatch for change notification. -func newStateWatch() *stateWatch { - return &stateWatch{ - items: make(map[watchItem]*NotifyGroup), - } -} - -// watch subscribes a channel to the given watch item. -func (w *stateWatch) watch(wi watchItem, ch chan struct{}) { - w.l.Lock() - defer w.l.Unlock() - - grp, ok := w.items[wi] - if !ok { - grp = new(NotifyGroup) - w.items[wi] = grp - } - grp.Wait(ch) -} - -// stopWatch unsubscribes a channel from the given watch item. -func (w *stateWatch) stopWatch(wi watchItem, ch chan struct{}) { - w.l.Lock() - defer w.l.Unlock() - - if grp, ok := w.items[wi]; ok { - grp.Clear(ch) - if grp.Empty() { - delete(w.items, wi) - } - } -} - -// notify is used to fire notifications on the given watch items. -func (w *stateWatch) notify(items watchItems) { - w.l.Lock() - defer w.l.Unlock() - - for wi, _ := range items { - if grp, ok := w.items[wi]; ok { - grp.Notify() - } - } -} diff --git a/nomad/state/watch_test.go b/nomad/state/watch_test.go deleted file mode 100644 index aad53d798e1..00000000000 --- a/nomad/state/watch_test.go +++ /dev/null @@ -1,67 +0,0 @@ -package state - -import ( - "testing" -) - -func TestWatchItems(t *testing.T) { - wi := make(watchItems) - - // Adding items works - wi.add(watchItem{table: "foo"}) - wi.add(watchItem{node: "bar"}) - if len(wi) != 2 { - t.Fatalf("expected 2 items, got: %#v", wi) - } - - // Adding duplicates auto-dedupes - wi.add(watchItem{table: "foo"}) - if len(wi) != 2 { - t.Fatalf("expected 2 items, got: %#v", wi) - } -} - -func TestStateWatch_watch(t *testing.T) { - watch := newStateWatch() - notify1 := make(chan struct{}, 1) - notify2 := make(chan struct{}, 1) - notify3 := make(chan struct{}, 1) - - // Notifications trigger subscribed channels - watch.watch(watchItem{table: "foo"}, notify1) - watch.watch(watchItem{table: "bar"}, notify2) - watch.watch(watchItem{table: "baz"}, notify3) - - items := make(watchItems) - items.add(watchItem{table: "foo"}) - items.add(watchItem{table: "bar"}) - - watch.notify(items) - if len(notify1) != 1 { - t.Fatalf("should notify") - } - if len(notify2) != 1 { - t.Fatalf("should notify") - } - if len(notify3) != 0 { - t.Fatalf("should not notify") - } -} - -func TestStateWatch_stopWatch(t *testing.T) { - watch := newStateWatch() - notify := make(chan struct{}) - - // First subscribe - watch.watch(watchItem{table: "foo"}, notify) - - // Unsubscribe stop notifications - watch.stopWatch(watchItem{table: "foo"}, notify) - - items := make(watchItems) - items.add(watchItem{table: "foo"}) - watch.notify(items) - if len(notify) != 0 { - t.Fatalf("should not notify") - } -} diff --git a/nomad/watch/watch.go b/nomad/watch/watch.go new file mode 100644 index 00000000000..e5cdce16f84 --- /dev/null +++ b/nomad/watch/watch.go @@ -0,0 +1,33 @@ +package watch + +// The watch package provides a means of describing a watch for a blocking +// query. It is exported so it may be shared between Nomad's RPC layer and +// the underlying state store. + +// Item describes the scope of a watch. It is used to provide a uniform +// input for subscribe/unsubscribe and notification firing. +type Item struct { + Alloc string + AllocNode string + Eval string + Job string + Node string + Table string +} + +// Items is a helper used to construct a set of watchItems. It deduplicates +// the items as they are added using map keys. +type Items map[Item]struct{} + +func NewItems(items ...Item) Items { + wi := make(Items) + for _, item := range items { + wi.Add(item) + } + return wi +} + +// Add adds an item to the watch set. +func (wi Items) Add(i Item) { + wi[i] = struct{}{} +} diff --git a/nomad/watch/watch_test.go b/nomad/watch/watch_test.go new file mode 100644 index 00000000000..9a8901aa818 --- /dev/null +++ b/nomad/watch/watch_test.go @@ -0,0 +1,31 @@ +package watch + +import ( + "testing" +) + +func TestWatchItems(t *testing.T) { + // Creates an empty set of items + wi := NewItems() + if len(wi) != 0 { + t.Fatalf("expect 0 items, got: %#v", wi) + } + + // Creates a new set of supplied items + wi = NewItems(Item{Table: "foo"}) + if len(wi) != 1 { + t.Fatalf("expected 1 item, got: %#v", wi) + } + + // Adding items works + wi.Add(Item{Node: "bar"}) + if len(wi) != 2 { + t.Fatalf("expected 2 items, got: %#v", wi) + } + + // Adding duplicates auto-dedupes + wi.Add(Item{Table: "foo"}) + if len(wi) != 2 { + t.Fatalf("expected 2 items, got: %#v", wi) + } +} From cd5bdd7c08d86ac39b880dc3bbffc298f29b7552 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Thu, 29 Oct 2015 15:01:29 -0700 Subject: [PATCH 16/26] nomad: support blocking queries on single jobs --- nomad/job_endpoint.go | 57 ++++++++++++++++++++++---------------- nomad/job_endpoint_test.go | 49 ++++++++++++++++++++++++++++++++ 2 files changed, 82 insertions(+), 24 deletions(-) diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index ff296a986d7..ad15d626d5c 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -181,32 +181,41 @@ func (j *Job) GetJob(args *structs.JobSpecificRequest, } defer metrics.MeasureSince([]string{"nomad", "job", "get_job"}, time.Now()) - // Look for the job - snap, err := j.srv.fsm.State().Snapshot() - if err != nil { - return err - } - out, err := snap.JobByID(args.JobID) - if err != nil { - return err - } + // Setup the blocking query + opts := blockingOptions{ + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watch: watch.NewItems(watch.Item{Job: args.JobID}), + run: func() error { - // Setup the output - if out != nil { - reply.Job = out - reply.Index = out.ModifyIndex - } else { - // Use the last index that affected the nodes table - index, err := snap.Index("jobs") - if err != nil { - return err - } - reply.Index = index - } + // Look for the job + snap, err := j.srv.fsm.State().Snapshot() + if err != nil { + return err + } + out, err := snap.JobByID(args.JobID) + if err != nil { + return err + } - // Set the query response - j.srv.setQueryMeta(&reply.QueryMeta) - return nil + // Setup the output + if out != nil { + reply.Job = out + reply.Index = out.ModifyIndex + } else { + // Use the last index that affected the nodes table + index, err := snap.Index("jobs") + if err != nil { + return err + } + reply.Index = index + } + + // Set the query response + j.srv.setQueryMeta(&reply.QueryMeta) + return nil + }} + return j.srv.blockingRPC(&opts) } // List is used to list the jobs registered in the system diff --git a/nomad/job_endpoint_test.go b/nomad/job_endpoint_test.go index 0591e73bf6c..5b4ba079bca 100644 --- a/nomad/job_endpoint_test.go +++ b/nomad/job_endpoint_test.go @@ -364,6 +364,55 @@ func TestJobEndpoint_GetJob(t *testing.T) { } } +func TestJobEndpoint_GetJob_blocking(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + state := s1.fsm.State() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + + // Create the jobs + job1 := mock.Job() + job2 := mock.Job() + + // Upsert a job we are not interested in first. + time.AfterFunc(100*time.Millisecond, func() { + if err := state.UpsertJob(2, job1); err != nil { + t.Fatalf("err: %v", err) + } + }) + + // Upsert another job later which should trigger the watch. + time.AfterFunc(200*time.Millisecond, func() { + if err := state.UpsertJob(2, job2); err != nil { + t.Fatalf("err: %v", err) + } + }) + + req := &structs.JobSpecificRequest{ + JobID: job2.ID, + QueryOptions: structs.QueryOptions{ + Region: "global", + MinQueryIndex: 1, + }, + } + start := time.Now() + var resp structs.SingleJobResponse + if err := msgpackrpc.CallWithCodec(codec, "Job.GetJob", req, &resp); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Now().Sub(start); elapsed < 200*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } + if resp.Index != 2 { + t.Fatalf("Bad index: %d %d", resp.Index, 2) + } + if resp.Job == nil || resp.Job.ID != job2.ID { + t.Fatalf("bad: %#v", resp.Job) + } +} + func TestJobEndpoint_ListJobs(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() From de495bfc3ccd5b234340396a9c39633412f6cd01 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Thu, 29 Oct 2015 15:26:14 -0700 Subject: [PATCH 17/26] nomad: support blocking queries on job allocations --- nomad/job_endpoint.go | 59 ++++++++++++++++++++---------------- nomad/job_endpoint_test.go | 61 +++++++++++++++++++++++++++++++++++--- nomad/state/state_store.go | 2 ++ nomad/watch/watch.go | 1 + 4 files changed, 94 insertions(+), 29 deletions(-) diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index ad15d626d5c..6d43e1c89a1 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -275,34 +275,43 @@ func (j *Job) Allocations(args *structs.JobSpecificRequest, } defer metrics.MeasureSince([]string{"nomad", "job", "allocations"}, time.Now()) - // Capture the allocations - snap, err := j.srv.fsm.State().Snapshot() - if err != nil { - return err - } - allocs, err := snap.AllocsByJob(args.JobID) - if err != nil { - return err - } + // Setup the blocking query + opts := blockingOptions{ + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watch: watch.NewItems(watch.Item{AllocJob: args.JobID}), + run: func() error { + // Capture the allocations + snap, err := j.srv.fsm.State().Snapshot() + if err != nil { + return err + } + allocs, err := snap.AllocsByJob(args.JobID) + if err != nil { + return err + } - // Convert to stubs - if len(allocs) > 0 { - reply.Allocations = make([]*structs.AllocListStub, 0, len(allocs)) - for _, alloc := range allocs { - reply.Allocations = append(reply.Allocations, alloc.Stub()) - } - } + // Convert to stubs + if len(allocs) > 0 { + reply.Allocations = make([]*structs.AllocListStub, 0, len(allocs)) + for _, alloc := range allocs { + reply.Allocations = append(reply.Allocations, alloc.Stub()) + } + } - // Use the last index that affected the allocs table - index, err := snap.Index("allocs") - if err != nil { - return err - } - reply.Index = index + // Use the last index that affected the allocs table + index, err := snap.Index("allocs") + if err != nil { + return err + } + reply.Index = index - // Set the query response - j.srv.setQueryMeta(&reply.QueryMeta) - return nil + // Set the query response + j.srv.setQueryMeta(&reply.QueryMeta) + return nil + + }} + return j.srv.blockingRPC(&opts) } // Evaluations is used to list the evaluations for a job diff --git a/nomad/job_endpoint_test.go b/nomad/job_endpoint_test.go index 5b4ba079bca..f42031a7a14 100644 --- a/nomad/job_endpoint_test.go +++ b/nomad/job_endpoint_test.go @@ -377,14 +377,14 @@ func TestJobEndpoint_GetJob_blocking(t *testing.T) { // Upsert a job we are not interested in first. time.AfterFunc(100*time.Millisecond, func() { - if err := state.UpsertJob(2, job1); err != nil { + if err := state.UpsertJob(1000, job1); err != nil { t.Fatalf("err: %v", err) } }) // Upsert another job later which should trigger the watch. time.AfterFunc(200*time.Millisecond, func() { - if err := state.UpsertJob(2, job2); err != nil { + if err := state.UpsertJob(2000, job2); err != nil { t.Fatalf("err: %v", err) } }) @@ -405,8 +405,8 @@ func TestJobEndpoint_GetJob_blocking(t *testing.T) { if elapsed := time.Now().Sub(start); elapsed < 200*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } - if resp.Index != 2 { - t.Fatalf("Bad index: %d %d", resp.Index, 2) + if resp.Index != 2000 { + t.Fatalf("Bad index: %d %d", resp.Index, 2000) } if resp.Job == nil || resp.Job.ID != job2.ID { t.Fatalf("bad: %#v", resp.Job) @@ -546,6 +546,59 @@ func TestJobEndpoint_Allocations(t *testing.T) { } } +func TestJobEndpoint_Allocations_blocking(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + + // Create the register request + alloc1 := mock.Alloc() + alloc2 := mock.Alloc() + alloc2.JobID = "job1" + state := s1.fsm.State() + + // First upsert an unrelated alloc + time.AfterFunc(100*time.Millisecond, func() { + err := state.UpsertAllocs(1000, []*structs.Allocation{alloc1}) + if err != nil { + t.Fatalf("err: %v", err) + } + }) + + // Upsert an alloc for the job we are interested in later + time.AfterFunc(200*time.Millisecond, func() { + err := state.UpsertAllocs(2000, []*structs.Allocation{alloc2}) + if err != nil { + t.Fatalf("err: %v", err) + } + }) + + // Lookup the jobs + get := &structs.JobSpecificRequest{ + JobID: "job1", + QueryOptions: structs.QueryOptions{ + Region: "global", + MinQueryIndex: 1, + }, + } + var resp structs.JobAllocationsResponse + start := time.Now() + if err := msgpackrpc.CallWithCodec(codec, "Job.Allocations", get, &resp); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Now().Sub(start); elapsed < 200*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } + if resp.Index != 2000 { + t.Fatalf("Bad index: %d %d", resp.Index, 2000) + } + if len(resp.Allocations) != 1 || resp.Allocations[0].JobID != "job1" { + t.Fatalf("bad: %#v", resp.Allocations) + } +} + func TestJobEndpoint_Evaluations(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 17484f985c8..2a31555f234 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -532,6 +532,7 @@ func (s *StateStore) UpdateAllocFromClient(index uint64, alloc *structs.Allocati watcher := watch.NewItems() watcher.Add(watch.Item{Table: "allocs"}) watcher.Add(watch.Item{Alloc: alloc.ID}) + watcher.Add(watch.Item{AllocJob: alloc.JobID}) watcher.Add(watch.Item{AllocNode: alloc.NodeID}) // Look for existing alloc @@ -603,6 +604,7 @@ func (s *StateStore) UpsertAllocs(index uint64, allocs []*structs.Allocation) er } watcher.Add(watch.Item{Alloc: alloc.ID}) + watcher.Add(watch.Item{AllocJob: alloc.JobID}) watcher.Add(watch.Item{AllocNode: alloc.NodeID}) } diff --git a/nomad/watch/watch.go b/nomad/watch/watch.go index e5cdce16f84..3973e562ba5 100644 --- a/nomad/watch/watch.go +++ b/nomad/watch/watch.go @@ -8,6 +8,7 @@ package watch // input for subscribe/unsubscribe and notification firing. type Item struct { Alloc string + AllocJob string AllocNode string Eval string Job string From 89a0af6306d95cf92d33cafeb903ea3bdf698327 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Thu, 29 Oct 2015 15:48:44 -0700 Subject: [PATCH 18/26] nomad: support blocking queries on single nodes --- nomad/node_endpoint.go | 64 +++++++++++++++++++++---------------- nomad/node_endpoint_test.go | 50 +++++++++++++++++++++++++++++ 2 files changed, 86 insertions(+), 28 deletions(-) diff --git a/nomad/node_endpoint.go b/nomad/node_endpoint.go index 65a83e1b122..f28cc3930c9 100644 --- a/nomad/node_endpoint.go +++ b/nomad/node_endpoint.go @@ -283,37 +283,45 @@ func (n *Node) GetNode(args *structs.NodeSpecificRequest, } defer metrics.MeasureSince([]string{"nomad", "client", "get_node"}, time.Now()) - // Verify the arguments - if args.NodeID == "" { - return fmt.Errorf("missing node ID") - } + // Setup the blocking query + opts := blockingOptions{ + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watch: watch.NewItems(watch.Item{Node: args.NodeID}), + run: func() error { + // Verify the arguments + if args.NodeID == "" { + return fmt.Errorf("missing node ID") + } - // Look for the node - snap, err := n.srv.fsm.State().Snapshot() - if err != nil { - return err - } - out, err := snap.NodeByID(args.NodeID) - if err != nil { - return err - } + // Look for the node + snap, err := n.srv.fsm.State().Snapshot() + if err != nil { + return err + } + out, err := snap.NodeByID(args.NodeID) + if err != nil { + return err + } - // Setup the output - if out != nil { - reply.Node = out - reply.Index = out.ModifyIndex - } else { - // Use the last index that affected the nodes table - index, err := snap.Index("nodes") - if err != nil { - return err - } - reply.Index = index - } + // Setup the output + if out != nil { + reply.Node = out + reply.Index = out.ModifyIndex + } else { + // Use the last index that affected the nodes table + index, err := snap.Index("nodes") + if err != nil { + return err + } + reply.Index = index + } - // Set the query response - n.srv.setQueryMeta(&reply.QueryMeta) - return nil + // Set the query response + n.srv.setQueryMeta(&reply.QueryMeta) + return nil + }} + return n.srv.blockingRPC(&opts) } // GetAllocs is used to request allocations for a specific node diff --git a/nomad/node_endpoint_test.go b/nomad/node_endpoint_test.go index d06e6ea0f88..8d53c0a6623 100644 --- a/nomad/node_endpoint_test.go +++ b/nomad/node_endpoint_test.go @@ -371,6 +371,56 @@ func TestClientEndpoint_GetNode(t *testing.T) { } } +func TestClientEndpoint_GetNode_blocking(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + state := s1.fsm.State() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + + // Create the node + node1 := mock.Node() + node2 := mock.Node() + + // First create an unrelated node. + time.AfterFunc(100*time.Millisecond, func() { + if err := state.UpsertNode(1000, node1); err != nil { + t.Fatalf("err: %v", err) + } + }) + + // Upsert the node we are watching later + time.AfterFunc(200*time.Millisecond, func() { + if err := state.UpsertNode(2000, node2); err != nil { + t.Fatalf("err: %v", err) + } + }) + + // Lookup the node + get := &structs.NodeSpecificRequest{ + NodeID: node2.ID, + QueryOptions: structs.QueryOptions{ + Region: "global", + MinQueryIndex: 1, + }, + } + var resp structs.SingleNodeResponse + start := time.Now() + if err := msgpackrpc.CallWithCodec(codec, "Node.GetNode", get, &resp); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Now().Sub(start); elapsed < 200*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } + if resp.Index != 2000 { + t.Fatalf("Bad index: %d %d", resp.Index, 2000) + } + if resp.Node == nil || resp.Node.ID != node2.ID { + t.Fatalf("bad: %#v", resp.Node) + } +} + func TestClientEndpoint_GetAllocs(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() From ff6e1fea49db9282042289d15745704de084e35b Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Thu, 29 Oct 2015 16:04:53 -0700 Subject: [PATCH 19/26] nomad: support blocking queries on single allocations --- nomad/alloc_endpoint.go | 56 ++++++++++++++++++++---------------- nomad/alloc_endpoint_test.go | 52 +++++++++++++++++++++++++++++++++ 2 files changed, 84 insertions(+), 24 deletions(-) diff --git a/nomad/alloc_endpoint.go b/nomad/alloc_endpoint.go index f3b9dbdc4c2..e8b6af63c85 100644 --- a/nomad/alloc_endpoint.go +++ b/nomad/alloc_endpoint.go @@ -69,30 +69,38 @@ func (a *Alloc) GetAlloc(args *structs.AllocSpecificRequest, } defer metrics.MeasureSince([]string{"nomad", "alloc", "get_alloc"}, time.Now()) - // Lookup the allocation - snap, err := a.srv.fsm.State().Snapshot() - if err != nil { - return err - } - out, err := snap.AllocByID(args.AllocID) - if err != nil { - return err - } + // Setup the blocking query + opts := blockingOptions{ + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watch: watch.NewItems(watch.Item{Alloc: args.AllocID}), + run: func() error { + // Lookup the allocation + snap, err := a.srv.fsm.State().Snapshot() + if err != nil { + return err + } + out, err := snap.AllocByID(args.AllocID) + if err != nil { + return err + } - // Setup the output - if out != nil { - reply.Alloc = out - reply.Index = out.ModifyIndex - } else { - // Use the last index that affected the nodes table - index, err := snap.Index("allocs") - if err != nil { - return err - } - reply.Index = index - } + // Setup the output + if out != nil { + reply.Alloc = out + reply.Index = out.ModifyIndex + } else { + // Use the last index that affected the nodes table + index, err := snap.Index("allocs") + if err != nil { + return err + } + reply.Index = index + } - // Set the query response - a.srv.setQueryMeta(&reply.QueryMeta) - return nil + // Set the query response + a.srv.setQueryMeta(&reply.QueryMeta) + return nil + }} + return a.srv.blockingRPC(&opts) } diff --git a/nomad/alloc_endpoint_test.go b/nomad/alloc_endpoint_test.go index 01688da960e..0f7e4c0c477 100644 --- a/nomad/alloc_endpoint_test.go +++ b/nomad/alloc_endpoint_test.go @@ -144,3 +144,55 @@ func TestAllocEndpoint_GetAlloc(t *testing.T) { t.Fatalf("bad: %#v", resp.Alloc) } } + +func TestAllocEndpoint_GetAlloc_blocking(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + state := s1.fsm.State() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + + // Create the allocs + alloc1 := mock.Alloc() + alloc2 := mock.Alloc() + + // First create an unrelated alloc + time.AfterFunc(100*time.Millisecond, func() { + err := state.UpsertAllocs(1000, []*structs.Allocation{alloc1}) + if err != nil { + t.Fatalf("err: %v", err) + } + }) + + // Create the alloc we are watching later + time.AfterFunc(200*time.Millisecond, func() { + err := state.UpsertAllocs(2000, []*structs.Allocation{alloc2}) + if err != nil { + t.Fatalf("err: %v", err) + } + }) + + // Lookup the jobs + get := &structs.AllocSpecificRequest{ + AllocID: alloc2.ID, + QueryOptions: structs.QueryOptions{ + Region: "global", + MinQueryIndex: 1, + }, + } + var resp structs.SingleAllocResponse + start := time.Now() + if err := msgpackrpc.CallWithCodec(codec, "Alloc.GetAlloc", get, &resp); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Now().Sub(start); elapsed < 200*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } + if resp.Index != 2000 { + t.Fatalf("Bad index: %d %d", resp.Index, 2000) + } + if resp.Alloc == nil || resp.Alloc.ID != alloc2.ID { + t.Fatalf("bad: %#v", resp.Alloc) + } +} From 1e4320e0331ef97436711b0802276d9687a7b9d0 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Thu, 29 Oct 2015 16:12:25 -0700 Subject: [PATCH 20/26] nomad: support blocking queries for single evals --- nomad/eval_endpoint.go | 56 +++++++++++++++++++++---------------- nomad/eval_endpoint_test.go | 52 ++++++++++++++++++++++++++++++++++ 2 files changed, 84 insertions(+), 24 deletions(-) diff --git a/nomad/eval_endpoint.go b/nomad/eval_endpoint.go index 07dfc18fe70..e473b5b1065 100644 --- a/nomad/eval_endpoint.go +++ b/nomad/eval_endpoint.go @@ -27,32 +27,40 @@ func (e *Eval) GetEval(args *structs.EvalSpecificRequest, } defer metrics.MeasureSince([]string{"nomad", "eval", "get_eval"}, time.Now()) - // Look for the job - snap, err := e.srv.fsm.State().Snapshot() - if err != nil { - return err - } - out, err := snap.EvalByID(args.EvalID) - if err != nil { - return err - } + // Setup the blocking query + opts := blockingOptions{ + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watch: watch.NewItems(watch.Item{Eval: args.EvalID}), + run: func() error { + // Look for the job + snap, err := e.srv.fsm.State().Snapshot() + if err != nil { + return err + } + out, err := snap.EvalByID(args.EvalID) + if err != nil { + return err + } - // Setup the output - if out != nil { - reply.Eval = out - reply.Index = out.ModifyIndex - } else { - // Use the last index that affected the nodes table - index, err := snap.Index("evals") - if err != nil { - return err - } - reply.Index = index - } + // Setup the output + if out != nil { + reply.Eval = out + reply.Index = out.ModifyIndex + } else { + // Use the last index that affected the nodes table + index, err := snap.Index("evals") + if err != nil { + return err + } + reply.Index = index + } - // Set the query response - e.srv.setQueryMeta(&reply.QueryMeta) - return nil + // Set the query response + e.srv.setQueryMeta(&reply.QueryMeta) + return nil + }} + return e.srv.blockingRPC(&opts) } // Dequeue is used to dequeue a pending evaluation diff --git a/nomad/eval_endpoint_test.go b/nomad/eval_endpoint_test.go index 3b9a62a8e8a..01ec27f46ef 100644 --- a/nomad/eval_endpoint_test.go +++ b/nomad/eval_endpoint_test.go @@ -51,6 +51,58 @@ func TestEvalEndpoint_GetEval(t *testing.T) { } } +func TestEvalEndpoint_GetEval_blocking(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + state := s1.fsm.State() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + + // Create the evals + eval1 := mock.Eval() + eval2 := mock.Eval() + + // First create an unrelated eval + time.AfterFunc(100*time.Millisecond, func() { + err := state.UpsertEvals(1000, []*structs.Evaluation{eval1}) + if err != nil { + t.Fatalf("err: %v", err) + } + }) + + // Upsert the eval we are watching later + time.AfterFunc(200*time.Millisecond, func() { + err := state.UpsertEvals(2000, []*structs.Evaluation{eval2}) + if err != nil { + t.Fatalf("err: %v", err) + } + }) + + // Lookup the eval + get := &structs.EvalSpecificRequest{ + EvalID: eval2.ID, + QueryOptions: structs.QueryOptions{ + Region: "global", + MinQueryIndex: 1, + }, + } + var resp structs.SingleEvalResponse + start := time.Now() + if err := msgpackrpc.CallWithCodec(codec, "Eval.GetEval", get, &resp); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Now().Sub(start); elapsed < 200*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } + if resp.Index != 2000 { + t.Fatalf("Bad index: %d %d", resp.Index, 2000) + } + if resp.Eval == nil || resp.Eval.ID != eval2.ID { + t.Fatalf("bad: %#v", resp.Eval) + } +} + func TestEvalEndpoint_Dequeue(t *testing.T) { s1 := testServer(t, func(c *Config) { c.NumSchedulers = 0 // Prevent automatic dequeue From a27e8bbe51496278074bcd26cdd5a27b40f3c10a Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Thu, 29 Oct 2015 16:20:57 -0700 Subject: [PATCH 21/26] nomad: support blocking queries on eval-specific allocations --- nomad/eval_endpoint.go | 58 +++++++++++++++++++++---------------- nomad/eval_endpoint_test.go | 52 +++++++++++++++++++++++++++++++++ nomad/state/state_store.go | 2 ++ nomad/watch/watch.go | 1 + 4 files changed, 88 insertions(+), 25 deletions(-) diff --git a/nomad/eval_endpoint.go b/nomad/eval_endpoint.go index e473b5b1065..5afa31dfcc2 100644 --- a/nomad/eval_endpoint.go +++ b/nomad/eval_endpoint.go @@ -277,32 +277,40 @@ func (e *Eval) Allocations(args *structs.EvalSpecificRequest, } defer metrics.MeasureSince([]string{"nomad", "eval", "allocations"}, time.Now()) - // Capture the allocations - snap, err := e.srv.fsm.State().Snapshot() - if err != nil { - return err - } - allocs, err := snap.AllocsByEval(args.EvalID) - if err != nil { - return err - } + // Setup the blocking query + opts := blockingOptions{ + queryOpts: &args.QueryOptions, + queryMeta: &reply.QueryMeta, + watch: watch.NewItems(watch.Item{AllocEval: args.EvalID}), + run: func() error { + // Capture the allocations + snap, err := e.srv.fsm.State().Snapshot() + if err != nil { + return err + } + allocs, err := snap.AllocsByEval(args.EvalID) + if err != nil { + return err + } - // Convert to a stub - if len(allocs) > 0 { - reply.Allocations = make([]*structs.AllocListStub, 0, len(allocs)) - for _, alloc := range allocs { - reply.Allocations = append(reply.Allocations, alloc.Stub()) - } - } + // Convert to a stub + if len(allocs) > 0 { + reply.Allocations = make([]*structs.AllocListStub, 0, len(allocs)) + for _, alloc := range allocs { + reply.Allocations = append(reply.Allocations, alloc.Stub()) + } + } - // Use the last index that affected the allocs table - index, err := snap.Index("allocs") - if err != nil { - return err - } - reply.Index = index + // Use the last index that affected the allocs table + index, err := snap.Index("allocs") + if err != nil { + return err + } + reply.Index = index - // Set the query response - e.srv.setQueryMeta(&reply.QueryMeta) - return nil + // Set the query response + e.srv.setQueryMeta(&reply.QueryMeta) + return nil + }} + return e.srv.blockingRPC(&opts) } diff --git a/nomad/eval_endpoint_test.go b/nomad/eval_endpoint_test.go index 01ec27f46ef..442a2c1aac2 100644 --- a/nomad/eval_endpoint_test.go +++ b/nomad/eval_endpoint_test.go @@ -484,3 +484,55 @@ func TestEvalEndpoint_Allocations(t *testing.T) { t.Fatalf("bad: %#v", resp.Allocations) } } + +func TestEvalEndpoint_Allocations_blocking(t *testing.T) { + s1 := testServer(t, nil) + defer s1.Shutdown() + state := s1.fsm.State() + codec := rpcClient(t, s1) + testutil.WaitForLeader(t, s1.RPC) + + // Create the allocs + alloc1 := mock.Alloc() + alloc2 := mock.Alloc() + + // Upsert an unrelated alloc first + time.AfterFunc(100*time.Millisecond, func() { + err := state.UpsertAllocs(1000, []*structs.Allocation{alloc1}) + if err != nil { + t.Fatalf("err: %v", err) + } + }) + + // Upsert an alloc which will trigger the watch later + time.AfterFunc(200*time.Millisecond, func() { + err := state.UpsertAllocs(2000, []*structs.Allocation{alloc2}) + if err != nil { + t.Fatalf("err: %v", err) + } + }) + + // Lookup the eval + get := &structs.EvalSpecificRequest{ + EvalID: alloc2.EvalID, + QueryOptions: structs.QueryOptions{ + Region: "global", + MinQueryIndex: 1, + }, + } + var resp structs.EvalAllocationsResponse + start := time.Now() + if err := msgpackrpc.CallWithCodec(codec, "Eval.Allocations", get, &resp); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Now().Sub(start); elapsed < 200*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } + if resp.Index != 2000 { + t.Fatalf("Bad index: %d %d", resp.Index, 2000) + } + if len(resp.Allocations) != 1 || resp.Allocations[0].ID != alloc2.ID { + t.Fatalf("bad: %#v", resp.Allocations) + } +} diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 2a31555f234..f4f97489af1 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -532,6 +532,7 @@ func (s *StateStore) UpdateAllocFromClient(index uint64, alloc *structs.Allocati watcher := watch.NewItems() watcher.Add(watch.Item{Table: "allocs"}) watcher.Add(watch.Item{Alloc: alloc.ID}) + watcher.Add(watch.Item{AllocEval: alloc.EvalID}) watcher.Add(watch.Item{AllocJob: alloc.JobID}) watcher.Add(watch.Item{AllocNode: alloc.NodeID}) @@ -604,6 +605,7 @@ func (s *StateStore) UpsertAllocs(index uint64, allocs []*structs.Allocation) er } watcher.Add(watch.Item{Alloc: alloc.ID}) + watcher.Add(watch.Item{AllocEval: alloc.EvalID}) watcher.Add(watch.Item{AllocJob: alloc.JobID}) watcher.Add(watch.Item{AllocNode: alloc.NodeID}) } diff --git a/nomad/watch/watch.go b/nomad/watch/watch.go index 3973e562ba5..c71fe508778 100644 --- a/nomad/watch/watch.go +++ b/nomad/watch/watch.go @@ -8,6 +8,7 @@ package watch // input for subscribe/unsubscribe and notification firing. type Item struct { Alloc string + AllocEval string AllocJob string AllocNode string Eval string From 6fb8a2d3e2481b829e4ba3a4ec87b90fac52b6a7 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Thu, 29 Oct 2015 19:00:02 -0700 Subject: [PATCH 22/26] nomad: more tests --- nomad/alloc_endpoint_test.go | 14 ++--- nomad/eval_endpoint.go | 2 +- nomad/eval_endpoint_test.go | 55 +++++++++++++----- nomad/job_endpoint.go | 2 +- nomad/job_endpoint_test.go | 67 +++++++++++++++------- nomad/node_endpoint.go | 2 +- nomad/node_endpoint_test.go | 106 ++++++++++++++++++++++++++++++----- 7 files changed, 188 insertions(+), 60 deletions(-) diff --git a/nomad/alloc_endpoint_test.go b/nomad/alloc_endpoint_test.go index 0f7e4c0c477..4147011ac95 100644 --- a/nomad/alloc_endpoint_test.go +++ b/nomad/alloc_endpoint_test.go @@ -45,7 +45,7 @@ func TestAllocEndpoint_List(t *testing.T) { } } -func TestAllocEndpoint_List_blocking(t *testing.T) { +func TestAllocEndpoint_List_Blocking(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() state := s1.fsm.State() @@ -145,7 +145,7 @@ func TestAllocEndpoint_GetAlloc(t *testing.T) { } } -func TestAllocEndpoint_GetAlloc_blocking(t *testing.T) { +func TestAllocEndpoint_GetAlloc_Blocking(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() state := s1.fsm.State() @@ -158,7 +158,7 @@ func TestAllocEndpoint_GetAlloc_blocking(t *testing.T) { // First create an unrelated alloc time.AfterFunc(100*time.Millisecond, func() { - err := state.UpsertAllocs(1000, []*structs.Allocation{alloc1}) + err := state.UpsertAllocs(100, []*structs.Allocation{alloc1}) if err != nil { t.Fatalf("err: %v", err) } @@ -166,7 +166,7 @@ func TestAllocEndpoint_GetAlloc_blocking(t *testing.T) { // Create the alloc we are watching later time.AfterFunc(200*time.Millisecond, func() { - err := state.UpsertAllocs(2000, []*structs.Allocation{alloc2}) + err := state.UpsertAllocs(200, []*structs.Allocation{alloc2}) if err != nil { t.Fatalf("err: %v", err) } @@ -177,7 +177,7 @@ func TestAllocEndpoint_GetAlloc_blocking(t *testing.T) { AllocID: alloc2.ID, QueryOptions: structs.QueryOptions{ Region: "global", - MinQueryIndex: 1, + MinQueryIndex: 50, }, } var resp structs.SingleAllocResponse @@ -189,8 +189,8 @@ func TestAllocEndpoint_GetAlloc_blocking(t *testing.T) { if elapsed := time.Now().Sub(start); elapsed < 200*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } - if resp.Index != 2000 { - t.Fatalf("Bad index: %d %d", resp.Index, 2000) + if resp.Index != 200 { + t.Fatalf("Bad index: %d %d", resp.Index, 200) } if resp.Alloc == nil || resp.Alloc.ID != alloc2.ID { t.Fatalf("bad: %#v", resp.Alloc) diff --git a/nomad/eval_endpoint.go b/nomad/eval_endpoint.go index 5afa31dfcc2..bc74e85f362 100644 --- a/nomad/eval_endpoint.go +++ b/nomad/eval_endpoint.go @@ -44,8 +44,8 @@ func (e *Eval) GetEval(args *structs.EvalSpecificRequest, } // Setup the output + reply.Eval = out if out != nil { - reply.Eval = out reply.Index = out.ModifyIndex } else { // Use the last index that affected the nodes table diff --git a/nomad/eval_endpoint_test.go b/nomad/eval_endpoint_test.go index 442a2c1aac2..6f3d154e534 100644 --- a/nomad/eval_endpoint_test.go +++ b/nomad/eval_endpoint_test.go @@ -51,7 +51,7 @@ func TestEvalEndpoint_GetEval(t *testing.T) { } } -func TestEvalEndpoint_GetEval_blocking(t *testing.T) { +func TestEvalEndpoint_GetEval_Blocking(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() state := s1.fsm.State() @@ -64,7 +64,7 @@ func TestEvalEndpoint_GetEval_blocking(t *testing.T) { // First create an unrelated eval time.AfterFunc(100*time.Millisecond, func() { - err := state.UpsertEvals(1000, []*structs.Evaluation{eval1}) + err := state.UpsertEvals(100, []*structs.Evaluation{eval1}) if err != nil { t.Fatalf("err: %v", err) } @@ -72,35 +72,60 @@ func TestEvalEndpoint_GetEval_blocking(t *testing.T) { // Upsert the eval we are watching later time.AfterFunc(200*time.Millisecond, func() { - err := state.UpsertEvals(2000, []*structs.Evaluation{eval2}) + err := state.UpsertEvals(200, []*structs.Evaluation{eval2}) if err != nil { t.Fatalf("err: %v", err) } }) // Lookup the eval - get := &structs.EvalSpecificRequest{ + req := &structs.EvalSpecificRequest{ EvalID: eval2.ID, QueryOptions: structs.QueryOptions{ Region: "global", - MinQueryIndex: 1, + MinQueryIndex: 50, }, } var resp structs.SingleEvalResponse start := time.Now() - if err := msgpackrpc.CallWithCodec(codec, "Eval.GetEval", get, &resp); err != nil { + if err := msgpackrpc.CallWithCodec(codec, "Eval.GetEval", req, &resp); err != nil { t.Fatalf("err: %v", err) } if elapsed := time.Now().Sub(start); elapsed < 200*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } - if resp.Index != 2000 { - t.Fatalf("Bad index: %d %d", resp.Index, 2000) + if resp.Index != 200 { + t.Fatalf("Bad index: %d %d", resp.Index, 200) } if resp.Eval == nil || resp.Eval.ID != eval2.ID { t.Fatalf("bad: %#v", resp.Eval) } + + // Eval delete triggers watches + time.AfterFunc(100*time.Millisecond, func() { + err := state.DeleteEval(300, []string{eval2.ID}, []string{}) + if err != nil { + t.Fatalf("err: %v", err) + } + }) + + req.QueryOptions.MinQueryIndex = 250 + var resp2 structs.SingleEvalResponse + start = time.Now() + if err := msgpackrpc.CallWithCodec(codec, "Eval.GetEval", req, &resp2); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp2) + } + if resp2.Index != 300 { + t.Fatalf("Bad index: %d %d", resp2.Index, 300) + } + if resp2.Eval != nil { + t.Fatalf("bad: %#v", resp2.Eval) + } } func TestEvalEndpoint_Dequeue(t *testing.T) { @@ -386,7 +411,7 @@ func TestEvalEndpoint_List(t *testing.T) { } } -func TestEvalEndpoint_List_blocking(t *testing.T) { +func TestEvalEndpoint_List_Blocking(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() state := s1.fsm.State() @@ -485,7 +510,7 @@ func TestEvalEndpoint_Allocations(t *testing.T) { } } -func TestEvalEndpoint_Allocations_blocking(t *testing.T) { +func TestEvalEndpoint_Allocations_Blocking(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() state := s1.fsm.State() @@ -498,7 +523,7 @@ func TestEvalEndpoint_Allocations_blocking(t *testing.T) { // Upsert an unrelated alloc first time.AfterFunc(100*time.Millisecond, func() { - err := state.UpsertAllocs(1000, []*structs.Allocation{alloc1}) + err := state.UpsertAllocs(100, []*structs.Allocation{alloc1}) if err != nil { t.Fatalf("err: %v", err) } @@ -506,7 +531,7 @@ func TestEvalEndpoint_Allocations_blocking(t *testing.T) { // Upsert an alloc which will trigger the watch later time.AfterFunc(200*time.Millisecond, func() { - err := state.UpsertAllocs(2000, []*structs.Allocation{alloc2}) + err := state.UpsertAllocs(200, []*structs.Allocation{alloc2}) if err != nil { t.Fatalf("err: %v", err) } @@ -517,7 +542,7 @@ func TestEvalEndpoint_Allocations_blocking(t *testing.T) { EvalID: alloc2.EvalID, QueryOptions: structs.QueryOptions{ Region: "global", - MinQueryIndex: 1, + MinQueryIndex: 50, }, } var resp structs.EvalAllocationsResponse @@ -529,8 +554,8 @@ func TestEvalEndpoint_Allocations_blocking(t *testing.T) { if elapsed := time.Now().Sub(start); elapsed < 200*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } - if resp.Index != 2000 { - t.Fatalf("Bad index: %d %d", resp.Index, 2000) + if resp.Index != 200 { + t.Fatalf("Bad index: %d %d", resp.Index, 200) } if len(resp.Allocations) != 1 || resp.Allocations[0].ID != alloc2.ID { t.Fatalf("bad: %#v", resp.Allocations) diff --git a/nomad/job_endpoint.go b/nomad/job_endpoint.go index 6d43e1c89a1..e961428e4ef 100644 --- a/nomad/job_endpoint.go +++ b/nomad/job_endpoint.go @@ -199,8 +199,8 @@ func (j *Job) GetJob(args *structs.JobSpecificRequest, } // Setup the output + reply.Job = out if out != nil { - reply.Job = out reply.Index = out.ModifyIndex } else { // Use the last index that affected the nodes table diff --git a/nomad/job_endpoint_test.go b/nomad/job_endpoint_test.go index f42031a7a14..9e09de53883 100644 --- a/nomad/job_endpoint_test.go +++ b/nomad/job_endpoint_test.go @@ -364,7 +364,7 @@ func TestJobEndpoint_GetJob(t *testing.T) { } } -func TestJobEndpoint_GetJob_blocking(t *testing.T) { +func TestJobEndpoint_GetJob_Blocking(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() state := s1.fsm.State() @@ -377,14 +377,14 @@ func TestJobEndpoint_GetJob_blocking(t *testing.T) { // Upsert a job we are not interested in first. time.AfterFunc(100*time.Millisecond, func() { - if err := state.UpsertJob(1000, job1); err != nil { + if err := state.UpsertJob(100, job1); err != nil { t.Fatalf("err: %v", err) } }) // Upsert another job later which should trigger the watch. time.AfterFunc(200*time.Millisecond, func() { - if err := state.UpsertJob(2000, job2); err != nil { + if err := state.UpsertJob(200, job2); err != nil { t.Fatalf("err: %v", err) } }) @@ -393,7 +393,7 @@ func TestJobEndpoint_GetJob_blocking(t *testing.T) { JobID: job2.ID, QueryOptions: structs.QueryOptions{ Region: "global", - MinQueryIndex: 1, + MinQueryIndex: 50, }, } start := time.Now() @@ -405,12 +405,37 @@ func TestJobEndpoint_GetJob_blocking(t *testing.T) { if elapsed := time.Now().Sub(start); elapsed < 200*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } - if resp.Index != 2000 { - t.Fatalf("Bad index: %d %d", resp.Index, 2000) + if resp.Index != 200 { + t.Fatalf("Bad index: %d %d", resp.Index, 200) } if resp.Job == nil || resp.Job.ID != job2.ID { t.Fatalf("bad: %#v", resp.Job) } + + // Job delete fires watches + time.AfterFunc(100*time.Millisecond, func() { + if err := state.DeleteJob(300, job2.ID); err != nil { + t.Fatalf("err: %v", err) + } + }) + + req.QueryOptions.MinQueryIndex = 250 + start = time.Now() + + var resp2 structs.SingleJobResponse + if err := msgpackrpc.CallWithCodec(codec, "Job.GetJob", req, &resp2); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp2) + } + if resp2.Index != 300 { + t.Fatalf("Bad index: %d %d", resp2.Index, 300) + } + if resp2.Job != nil { + t.Fatalf("bad: %#v", resp2.Job) + } } func TestJobEndpoint_ListJobs(t *testing.T) { @@ -447,7 +472,7 @@ func TestJobEndpoint_ListJobs(t *testing.T) { } } -func TestJobEndpoint_ListJobs_blocking(t *testing.T) { +func TestJobEndpoint_ListJobs_Blocking(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() state := s1.fsm.State() @@ -459,7 +484,7 @@ func TestJobEndpoint_ListJobs_blocking(t *testing.T) { // Upsert job triggers watches time.AfterFunc(100*time.Millisecond, func() { - if err := state.UpsertJob(2, job); err != nil { + if err := state.UpsertJob(100, job); err != nil { t.Fatalf("err: %v", err) } }) @@ -467,7 +492,7 @@ func TestJobEndpoint_ListJobs_blocking(t *testing.T) { req := &structs.JobListRequest{ QueryOptions: structs.QueryOptions{ Region: "global", - MinQueryIndex: 1, + MinQueryIndex: 50, }, } start := time.Now() @@ -479,8 +504,8 @@ func TestJobEndpoint_ListJobs_blocking(t *testing.T) { if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } - if resp.Index != 2 { - t.Fatalf("Bad index: %d %d", resp.Index, 2) + if resp.Index != 100 { + t.Fatalf("Bad index: %d %d", resp.Index, 100) } if len(resp.Jobs) != 1 || resp.Jobs[0].ID != job.ID { t.Fatalf("bad: %#v", resp.Jobs) @@ -488,12 +513,12 @@ func TestJobEndpoint_ListJobs_blocking(t *testing.T) { // Job deletion triggers watches time.AfterFunc(100*time.Millisecond, func() { - if err := state.DeleteJob(3, job.ID); err != nil { + if err := state.DeleteJob(200, job.ID); err != nil { t.Fatalf("err: %v", err) } }) - req.MinQueryIndex = 2 + req.MinQueryIndex = 150 start = time.Now() var resp2 structs.JobListResponse if err := msgpackrpc.CallWithCodec(codec, "Job.List", req, &resp2); err != nil { @@ -503,8 +528,8 @@ func TestJobEndpoint_ListJobs_blocking(t *testing.T) { if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp2) } - if resp2.Index != 3 { - t.Fatalf("Bad index: %d %d", resp2.Index, 3) + if resp2.Index != 200 { + t.Fatalf("Bad index: %d %d", resp2.Index, 200) } if len(resp2.Jobs) != 0 { t.Fatalf("bad: %#v", resp2.Jobs) @@ -546,7 +571,7 @@ func TestJobEndpoint_Allocations(t *testing.T) { } } -func TestJobEndpoint_Allocations_blocking(t *testing.T) { +func TestJobEndpoint_Allocations_Blocking(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() codec := rpcClient(t, s1) @@ -560,7 +585,7 @@ func TestJobEndpoint_Allocations_blocking(t *testing.T) { // First upsert an unrelated alloc time.AfterFunc(100*time.Millisecond, func() { - err := state.UpsertAllocs(1000, []*structs.Allocation{alloc1}) + err := state.UpsertAllocs(100, []*structs.Allocation{alloc1}) if err != nil { t.Fatalf("err: %v", err) } @@ -568,7 +593,7 @@ func TestJobEndpoint_Allocations_blocking(t *testing.T) { // Upsert an alloc for the job we are interested in later time.AfterFunc(200*time.Millisecond, func() { - err := state.UpsertAllocs(2000, []*structs.Allocation{alloc2}) + err := state.UpsertAllocs(200, []*structs.Allocation{alloc2}) if err != nil { t.Fatalf("err: %v", err) } @@ -579,7 +604,7 @@ func TestJobEndpoint_Allocations_blocking(t *testing.T) { JobID: "job1", QueryOptions: structs.QueryOptions{ Region: "global", - MinQueryIndex: 1, + MinQueryIndex: 50, }, } var resp structs.JobAllocationsResponse @@ -591,8 +616,8 @@ func TestJobEndpoint_Allocations_blocking(t *testing.T) { if elapsed := time.Now().Sub(start); elapsed < 200*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } - if resp.Index != 2000 { - t.Fatalf("Bad index: %d %d", resp.Index, 2000) + if resp.Index != 200 { + t.Fatalf("Bad index: %d %d", resp.Index, 200) } if len(resp.Allocations) != 1 || resp.Allocations[0].JobID != "job1" { t.Fatalf("bad: %#v", resp.Allocations) diff --git a/nomad/node_endpoint.go b/nomad/node_endpoint.go index f28cc3930c9..5bd600380f1 100644 --- a/nomad/node_endpoint.go +++ b/nomad/node_endpoint.go @@ -305,8 +305,8 @@ func (n *Node) GetNode(args *structs.NodeSpecificRequest, } // Setup the output + reply.Node = out if out != nil { - reply.Node = out reply.Index = out.ModifyIndex } else { // Use the last index that affected the nodes table diff --git a/nomad/node_endpoint_test.go b/nomad/node_endpoint_test.go index 8d53c0a6623..9a74316c7bd 100644 --- a/nomad/node_endpoint_test.go +++ b/nomad/node_endpoint_test.go @@ -371,7 +371,7 @@ func TestClientEndpoint_GetNode(t *testing.T) { } } -func TestClientEndpoint_GetNode_blocking(t *testing.T) { +func TestClientEndpoint_GetNode_Blocking(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() state := s1.fsm.State() @@ -384,41 +384,92 @@ func TestClientEndpoint_GetNode_blocking(t *testing.T) { // First create an unrelated node. time.AfterFunc(100*time.Millisecond, func() { - if err := state.UpsertNode(1000, node1); err != nil { + if err := state.UpsertNode(100, node1); err != nil { t.Fatalf("err: %v", err) } }) // Upsert the node we are watching later time.AfterFunc(200*time.Millisecond, func() { - if err := state.UpsertNode(2000, node2); err != nil { + if err := state.UpsertNode(200, node2); err != nil { t.Fatalf("err: %v", err) } }) // Lookup the node - get := &structs.NodeSpecificRequest{ + req := &structs.NodeSpecificRequest{ NodeID: node2.ID, QueryOptions: structs.QueryOptions{ Region: "global", - MinQueryIndex: 1, + MinQueryIndex: 50, }, } var resp structs.SingleNodeResponse start := time.Now() - if err := msgpackrpc.CallWithCodec(codec, "Node.GetNode", get, &resp); err != nil { + if err := msgpackrpc.CallWithCodec(codec, "Node.GetNode", req, &resp); err != nil { t.Fatalf("err: %v", err) } if elapsed := time.Now().Sub(start); elapsed < 200*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } - if resp.Index != 2000 { - t.Fatalf("Bad index: %d %d", resp.Index, 2000) + if resp.Index != 200 { + t.Fatalf("Bad index: %d %d", resp.Index, 200) } if resp.Node == nil || resp.Node.ID != node2.ID { t.Fatalf("bad: %#v", resp.Node) } + + // Node update triggers watches + time.AfterFunc(100*time.Millisecond, func() { + nodeUpdate := mock.Node() + nodeUpdate.ID = node2.ID + nodeUpdate.Status = structs.NodeStatusDown + if err := state.UpsertNode(300, nodeUpdate); err != nil { + t.Fatalf("err: %v", err) + } + }) + + req.QueryOptions.MinQueryIndex = 250 + var resp2 structs.SingleNodeResponse + start = time.Now() + if err := msgpackrpc.CallWithCodec(codec, "Node.GetNode", req, &resp2); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } + if resp2.Index != 300 { + t.Fatalf("Bad index: %d %d", resp2.Index, 300) + } + if resp2.Node == nil || resp2.Node.Status != structs.NodeStatusDown { + t.Fatalf("bad: %#v", resp2.Node) + } + + // Node delete triggers watches + time.AfterFunc(100*time.Millisecond, func() { + if err := state.DeleteNode(400, node2.ID); err != nil { + t.Fatalf("err: %v", err) + } + }) + + req.QueryOptions.MinQueryIndex = 350 + var resp3 structs.SingleNodeResponse + start = time.Now() + if err := msgpackrpc.CallWithCodec(codec, "Node.GetNode", req, &resp3); err != nil { + t.Fatalf("err: %v", err) + } + + if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + t.Fatalf("should block (returned in %s) %#v", elapsed, resp) + } + if resp3.Index != 400 { + t.Fatalf("Bad index: %d %d", resp2.Index, 400) + } + if resp3.Node != nil { + t.Fatalf("bad: %#v", resp3.Node) + } } func TestClientEndpoint_GetAllocs(t *testing.T) { @@ -507,16 +558,15 @@ func TestClientEndpoint_GetAllocs_Blocking(t *testing.T) { alloc.NodeID = node.ID state := s1.fsm.State() start := time.Now() - go func() { - time.Sleep(100 * time.Millisecond) + time.AfterFunc(100*time.Millisecond, func() { err := state.UpsertAllocs(100, []*structs.Allocation{alloc}) if err != nil { t.Fatalf("err: %v", err) } - }() + }) // Lookup the allocs in a blocking query - get := &structs.NodeSpecificRequest{ + req := &structs.NodeSpecificRequest{ NodeID: node.ID, QueryOptions: structs.QueryOptions{ Region: "global", @@ -525,7 +575,7 @@ func TestClientEndpoint_GetAllocs_Blocking(t *testing.T) { }, } var resp2 structs.NodeAllocsResponse - if err := msgpackrpc.CallWithCodec(codec, "Node.GetAllocs", get, &resp2); err != nil { + if err := msgpackrpc.CallWithCodec(codec, "Node.GetAllocs", req, &resp2); err != nil { t.Fatalf("err: %v", err) } @@ -541,6 +591,34 @@ func TestClientEndpoint_GetAllocs_Blocking(t *testing.T) { if len(resp2.Allocs) != 1 || resp2.Allocs[0].ID != alloc.ID { t.Fatalf("bad: %#v", resp2.Allocs) } + + // Alloc updates fire watches + time.AfterFunc(100*time.Millisecond, func() { + allocUpdate := mock.Alloc() + allocUpdate.NodeID = alloc.NodeID + allocUpdate.ID = alloc.ID + allocUpdate.ClientStatus = structs.AllocClientStatusRunning + err := state.UpdateAllocFromClient(200, allocUpdate) + if err != nil { + t.Fatalf("err: %v", err) + } + }) + + req.QueryOptions.MinQueryIndex = 150 + var resp3 structs.NodeAllocsResponse + if err := msgpackrpc.CallWithCodec(codec, "Node.GetAllocs", req, &resp3); err != nil { + t.Fatalf("err: %v", err) + } + + if time.Since(start) < 100*time.Millisecond { + t.Fatalf("too fast") + } + if resp3.Index != 200 { + t.Fatalf("Bad index: %d %d", resp3.Index, 200) + } + if len(resp3.Allocs) != 1 || resp3.Allocs[0].ClientStatus != structs.AllocClientStatusRunning { + t.Fatalf("bad: %#v", resp3.Allocs[0]) + } } func TestClientEndpoint_UpdateAlloc(t *testing.T) { @@ -803,7 +881,7 @@ func TestClientEndpoint_ListNodes(t *testing.T) { } } -func TestClientEndpoint_ListNodes_blocking(t *testing.T) { +func TestClientEndpoint_ListNodes_Blocking(t *testing.T) { s1 := testServer(t, nil) defer s1.Shutdown() state := s1.fsm.State() From ae4156d2b3f3d1c70c0db006c2cc8653f126a68e Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Thu, 29 Oct 2015 21:42:41 -0700 Subject: [PATCH 23/26] nomad/state: add watch trigger tests --- nomad/state/state_store.go | 3 + nomad/state/state_store_test.go | 249 ++++++++++++++++++++++++-------- 2 files changed, 191 insertions(+), 61 deletions(-) diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index f4f97489af1..47ead285e47 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -765,6 +765,7 @@ func (s *StateRestore) Commit() { // NodeRestore is used to restore a node func (r *StateRestore) NodeRestore(node *structs.Node) error { r.items.Add(watch.Item{Table: "nodes"}) + r.items.Add(watch.Item{Node: node.ID}) if err := r.txn.Insert("nodes", node); err != nil { return fmt.Errorf("node insert failed: %v", err) } @@ -774,6 +775,7 @@ func (r *StateRestore) NodeRestore(node *structs.Node) error { // JobRestore is used to restore a job func (r *StateRestore) JobRestore(job *structs.Job) error { r.items.Add(watch.Item{Table: "jobs"}) + r.items.Add(watch.Item{Job: job.ID}) if err := r.txn.Insert("jobs", job); err != nil { return fmt.Errorf("job insert failed: %v", err) } @@ -783,6 +785,7 @@ func (r *StateRestore) JobRestore(job *structs.Job) error { // EvalRestore is used to restore an evaluation func (r *StateRestore) EvalRestore(eval *structs.Evaluation) error { r.items.Add(watch.Item{Table: "evals"}) + r.items.Add(watch.Item{Eval: eval.ID}) if err := r.txn.Insert("evals", eval); err != nil { return fmt.Errorf("eval insert failed: %v", err) } diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index 2d1134d9c01..788b9f26a8c 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -26,6 +26,12 @@ func TestStateStore_UpsertNode_Node(t *testing.T) { state := testStateStore(t) node := mock.Node() + notify := notifyTest{ + {desc: "table", item: watch.Item{Table: "nodes"}}, + {desc: "node", item: watch.Item{Node: node.ID}}, + } + notify.start(state) + err := state.UpsertNode(1000, node) if err != nil { t.Fatalf("err: %v", err) @@ -47,12 +53,20 @@ func TestStateStore_UpsertNode_Node(t *testing.T) { if index != 1000 { t.Fatalf("bad: %d", index) } + + notify.verify(t) } func TestStateStore_DeleteNode_Node(t *testing.T) { state := testStateStore(t) node := mock.Node() + notify := notifyTest{ + {desc: "table", item: watch.Item{Table: "nodes"}}, + {desc: "node", item: watch.Item{Node: node.ID}}, + } + notify.start(state) + err := state.UpsertNode(1000, node) if err != nil { t.Fatalf("err: %v", err) @@ -79,12 +93,20 @@ func TestStateStore_DeleteNode_Node(t *testing.T) { if index != 1001 { t.Fatalf("bad: %d", index) } + + notify.verify(t) } func TestStateStore_UpdateNodeStatus_Node(t *testing.T) { state := testStateStore(t) node := mock.Node() + notify := notifyTest{ + {desc: "table", item: watch.Item{Table: "nodes"}}, + {desc: "node", item: watch.Item{Node: node.ID}}, + } + notify.start(state) + err := state.UpsertNode(1000, node) if err != nil { t.Fatalf("err: %v", err) @@ -114,12 +136,20 @@ func TestStateStore_UpdateNodeStatus_Node(t *testing.T) { if index != 1001 { t.Fatalf("bad: %d", index) } + + notify.verify(t) } func TestStateStore_UpdateNodeDrain_Node(t *testing.T) { state := testStateStore(t) node := mock.Node() + notify := notifyTest{ + {desc: "table", item: watch.Item{Table: "nodes"}}, + {desc: "node", item: watch.Item{Node: node.ID}}, + } + notify.start(state) + err := state.UpsertNode(1000, node) if err != nil { t.Fatalf("err: %v", err) @@ -149,6 +179,8 @@ func TestStateStore_UpdateNodeDrain_Node(t *testing.T) { if index != 1001 { t.Fatalf("bad: %d", index) } + + notify.verify(t) } func TestStateStore_Nodes(t *testing.T) { @@ -189,18 +221,23 @@ func TestStateStore_Nodes(t *testing.T) { func TestStateStore_RestoreNode(t *testing.T) { state := testStateStore(t) + node := mock.Node() + + notify := notifyTest{ + {desc: "table", item: watch.Item{Table: "nodes"}}, + {desc: "node", item: watch.Item{Node: node.ID}}, + } + notify.start(state) restore, err := state.Restore() if err != nil { t.Fatalf("err: %v", err) } - node := mock.Node() err = restore.NodeRestore(node) if err != nil { t.Fatalf("err: %v", err) } - restore.Commit() out, err := state.NodeByID(node.ID) @@ -211,12 +248,20 @@ func TestStateStore_RestoreNode(t *testing.T) { if !reflect.DeepEqual(out, node) { t.Fatalf("Bad: %#v %#v", out, node) } + + notify.verify(t) } func TestStateStore_UpsertJob_Job(t *testing.T) { state := testStateStore(t) job := mock.Job() + notify := notifyTest{ + {desc: "table", item: watch.Item{Table: "jobs"}}, + {desc: "job", item: watch.Item{Job: job.ID}}, + } + notify.start(state) + err := state.UpsertJob(1000, job) if err != nil { t.Fatalf("err: %v", err) @@ -238,12 +283,20 @@ func TestStateStore_UpsertJob_Job(t *testing.T) { if index != 1000 { t.Fatalf("bad: %d", index) } + + notify.verify(t) } func TestStateStore_UpdateUpsertJob_Job(t *testing.T) { state := testStateStore(t) job := mock.Job() + notify := notifyTest{ + {desc: "table", item: watch.Item{Table: "jobs"}}, + {desc: "job", item: watch.Item{Job: job.ID}}, + } + notify.start(state) + err := state.UpsertJob(1000, job) if err != nil { t.Fatalf("err: %v", err) @@ -279,12 +332,20 @@ func TestStateStore_UpdateUpsertJob_Job(t *testing.T) { if index != 1001 { t.Fatalf("bad: %d", index) } + + notify.verify(t) } func TestStateStore_DeleteJob_Job(t *testing.T) { state := testStateStore(t) job := mock.Job() + notify := notifyTest{ + {desc: "table", item: watch.Item{Table: "jobs"}}, + {desc: "job", item: watch.Item{Job: job.ID}}, + } + notify.start(state) + err := state.UpsertJob(1000, job) if err != nil { t.Fatalf("err: %v", err) @@ -311,6 +372,8 @@ func TestStateStore_DeleteJob_Job(t *testing.T) { if index != 1001 { t.Fatalf("bad: %d", index) } + + notify.verify(t) } func TestStateStore_Jobs(t *testing.T) { @@ -418,18 +481,23 @@ func TestStateStore_JobsByScheduler(t *testing.T) { func TestStateStore_RestoreJob(t *testing.T) { state := testStateStore(t) + job := mock.Job() + + notify := notifyTest{ + {desc: "table", item: watch.Item{Table: "jobs"}}, + {desc: "job", item: watch.Item{Job: job.ID}}, + } + notify.start(state) restore, err := state.Restore() if err != nil { t.Fatalf("err: %v", err) } - job := mock.Job() err = restore.JobRestore(job) if err != nil { t.Fatalf("err: %v", err) } - restore.Commit() out, err := state.JobByID(job.ID) @@ -440,6 +508,8 @@ func TestStateStore_RestoreJob(t *testing.T) { if !reflect.DeepEqual(out, job) { t.Fatalf("Bad: %#v %#v", out, job) } + + notify.verify(t) } func TestStateStore_Indexes(t *testing.T) { @@ -504,6 +574,12 @@ func TestStateStore_UpsertEvals_Eval(t *testing.T) { state := testStateStore(t) eval := mock.Eval() + notify := notifyTest{ + {desc: "table", item: watch.Item{Table: "evals"}}, + {desc: "eval", item: watch.Item{Eval: eval.ID}}, + } + notify.start(state) + err := state.UpsertEvals(1000, []*structs.Evaluation{eval}) if err != nil { t.Fatalf("err: %v", err) @@ -525,6 +601,8 @@ func TestStateStore_UpsertEvals_Eval(t *testing.T) { if index != 1000 { t.Fatalf("bad: %d", index) } + + notify.verify(t) } func TestStateStore_Update_UpsertEvals_Eval(t *testing.T) { @@ -536,6 +614,12 @@ func TestStateStore_Update_UpsertEvals_Eval(t *testing.T) { t.Fatalf("err: %v", err) } + notify := notifyTest{ + {desc: "table", item: watch.Item{Table: "evals"}}, + {desc: "eval", item: watch.Item{Eval: eval.ID}}, + } + notify.start(state) + eval2 := mock.Eval() eval2.ID = eval.ID err = state.UpsertEvals(1001, []*structs.Evaluation{eval2}) @@ -566,40 +650,50 @@ func TestStateStore_Update_UpsertEvals_Eval(t *testing.T) { if index != 1001 { t.Fatalf("bad: %d", index) } + + notify.verify(t) } func TestStateStore_DeleteEval_Eval(t *testing.T) { state := testStateStore(t) - eval := mock.Eval() + eval1 := mock.Eval() eval2 := mock.Eval() - alloc := mock.Alloc() + alloc1 := mock.Alloc() alloc2 := mock.Alloc() - err := state.UpsertEvals(1000, []*structs.Evaluation{eval, eval2}) + notify := notifyTest{ + {desc: "table", item: watch.Item{Table: "evals"}}, + {desc: "eval1", item: watch.Item{Eval: eval1.ID}}, + {desc: "eval2", item: watch.Item{Eval: eval2.ID}}, + {desc: "alloc1", item: watch.Item{Alloc: alloc1.ID}}, + {desc: "alloc2", item: watch.Item{Alloc: alloc2.ID}}, + {desc: "allocnode1", item: watch.Item{AllocNode: alloc1.NodeID}}, + {desc: "allocnode2", item: watch.Item{AllocNode: alloc2.NodeID}}, + } + notify.start(state) + + err := state.UpsertEvals(1000, []*structs.Evaluation{eval1, eval2}) if err != nil { t.Fatalf("err: %v", err) } - err = state.UpsertAllocs(1001, []*structs.Allocation{alloc, alloc2}) + err = state.UpsertAllocs(1001, []*structs.Allocation{alloc1, alloc2}) if err != nil { t.Fatalf("err: %v", err) } - notify1 := make(chan struct{}, 1) - state.Watch(watch.NewItems(watch.Item{AllocNode: alloc.NodeID}), notify1) - - err = state.DeleteEval(1002, []string{eval.ID, eval2.ID}, []string{alloc.ID, alloc2.ID}) + err = state.DeleteEval(1002, []string{eval1.ID, eval2.ID}, []string{alloc1.ID, alloc2.ID}) if err != nil { t.Fatalf("err: %v", err) } - out, err := state.EvalByID(eval.ID) + out, err := state.EvalByID(eval1.ID) if err != nil { t.Fatalf("err: %v", err) } if out != nil { - t.Fatalf("bad: %#v %#v", eval, out) + t.Fatalf("bad: %#v %#v", eval1, out) } out, err = state.EvalByID(eval2.ID) @@ -608,16 +702,16 @@ func TestStateStore_DeleteEval_Eval(t *testing.T) { } if out != nil { - t.Fatalf("bad: %#v %#v", eval, out) + t.Fatalf("bad: %#v %#v", eval1, out) } - outA, err := state.AllocByID(alloc.ID) + outA, err := state.AllocByID(alloc1.ID) if err != nil { t.Fatalf("err: %v", err) } if out != nil { - t.Fatalf("bad: %#v %#v", alloc, outA) + t.Fatalf("bad: %#v %#v", alloc1, outA) } outA, err = state.AllocByID(alloc2.ID) @@ -626,7 +720,7 @@ func TestStateStore_DeleteEval_Eval(t *testing.T) { } if out != nil { - t.Fatalf("bad: %#v %#v", alloc, outA) + t.Fatalf("bad: %#v %#v", alloc1, outA) } index, err := state.Index("evals") @@ -645,11 +739,7 @@ func TestStateStore_DeleteEval_Eval(t *testing.T) { t.Fatalf("bad: %d", index) } - select { - case <-notify1: - default: - t.Fatalf("should be notified") - } + notify.verify(t) } func TestStateStore_EvalsByJob(t *testing.T) { @@ -721,34 +811,50 @@ func TestStateStore_Evals(t *testing.T) { func TestStateStore_RestoreEval(t *testing.T) { state := testStateStore(t) + eval := mock.Eval() + + notify := notifyTest{ + {desc: "table", item: watch.Item{Table: "evals"}}, + {desc: "eval", item: watch.Item{Eval: eval.ID}}, + } + notify.start(state) restore, err := state.Restore() if err != nil { t.Fatalf("err: %v", err) } - job := mock.Eval() - err = restore.EvalRestore(job) + err = restore.EvalRestore(eval) if err != nil { t.Fatalf("err: %v", err) } - restore.Commit() - out, err := state.EvalByID(job.ID) + out, err := state.EvalByID(eval.ID) if err != nil { t.Fatalf("err: %v", err) } - if !reflect.DeepEqual(out, job) { - t.Fatalf("Bad: %#v %#v", out, job) + if !reflect.DeepEqual(out, eval) { + t.Fatalf("Bad: %#v %#v", out, eval) } + + notify.verify(t) } func TestStateStore_UpdateAllocFromClient(t *testing.T) { state := testStateStore(t) - alloc := mock.Alloc() + + notify := notifyTest{ + {desc: "table", item: watch.Item{Table: "allocs"}}, + {desc: "alloc", item: watch.Item{Alloc: alloc.ID}}, + {desc: "alloceval", item: watch.Item{AllocEval: alloc.EvalID}}, + {desc: "allocjob", item: watch.Item{AllocJob: alloc.JobID}}, + {desc: "allocnode", item: watch.Item{AllocNode: alloc.NodeID}}, + } + notify.start(state) + err := state.UpsertAllocs(1000, []*structs.Allocation{alloc}) if err != nil { t.Fatalf("err: %v", err) @@ -780,12 +886,23 @@ func TestStateStore_UpdateAllocFromClient(t *testing.T) { if index != 1001 { t.Fatalf("bad: %d", index) } + + notify.verify(t) } func TestStateStore_UpsertAlloc_Alloc(t *testing.T) { state := testStateStore(t) - alloc := mock.Alloc() + + notify := notifyTest{ + {desc: "table", item: watch.Item{Table: "allocs"}}, + {desc: "alloc", item: watch.Item{Alloc: alloc.ID}}, + {desc: "alloceval", item: watch.Item{AllocEval: alloc.EvalID}}, + {desc: "allocjob", item: watch.Item{AllocJob: alloc.JobID}}, + {desc: "allocnode", item: watch.Item{AllocNode: alloc.NodeID}}, + } + notify.start(state) + err := state.UpsertAllocs(1000, []*structs.Allocation{alloc}) if err != nil { t.Fatalf("err: %v", err) @@ -807,37 +924,8 @@ func TestStateStore_UpsertAlloc_Alloc(t *testing.T) { if index != 1000 { t.Fatalf("bad: %d", index) } -} - -func TestStateStore_Watch(t *testing.T) { - state := testStateStore(t) - - notify1 := make(chan struct{}, 1) - notify2 := make(chan struct{}, 1) - - items := watch.NewItems(watch.Item{AllocNode: "foo"}) - state.Watch(items, notify1) - state.Watch(items, notify2) - state.StopWatch(items, notify2) - - alloc := mock.Alloc() - alloc.NodeID = "foo" - err := state.UpsertAllocs(1000, []*structs.Allocation{alloc}) - if err != nil { - t.Fatalf("err: %v", err) - } - - select { - case <-notify1: - default: - t.Fatalf("should be notified") - } - select { - case <-notify2: - t.Fatalf("should not be notified") - default: - } + notify.verify(t) } func TestStateStore_UpdateAlloc_Alloc(t *testing.T) { @@ -852,6 +940,16 @@ func TestStateStore_UpdateAlloc_Alloc(t *testing.T) { alloc2 := mock.Alloc() alloc2.ID = alloc.ID alloc2.NodeID = alloc.NodeID + ".new" + + notify := notifyTest{ + {desc: "table", item: watch.Item{Table: "allocs"}}, + {desc: "alloc", item: watch.Item{Alloc: alloc2.ID}}, + {desc: "alloceval", item: watch.Item{AllocEval: alloc2.EvalID}}, + {desc: "allocjob", item: watch.Item{AllocJob: alloc2.JobID}}, + {desc: "allocnode", item: watch.Item{AllocNode: alloc2.NodeID}}, + } + notify.start(state) + err = state.UpsertAllocs(1001, []*structs.Allocation{alloc2}) if err != nil { t.Fatalf("err: %v", err) @@ -880,6 +978,8 @@ func TestStateStore_UpdateAlloc_Alloc(t *testing.T) { if index != 1001 { t.Fatalf("bad: %d", index) } + + notify.verify(t) } func TestStateStore_EvictAlloc_Alloc(t *testing.T) { @@ -1078,6 +1178,33 @@ func TestStateWatch_stopWatch(t *testing.T) { } } +// notifyTestCase is used to set up and verify watch triggers. +type notifyTestCase struct { + desc string + item watch.Item + ch chan struct{} +} + +// notifyTest is a suite of notifyTestCases. +type notifyTest []*notifyTestCase + +// start creates the notify channels and subscribes them. +func (n notifyTest) start(state *StateStore) { + for _, tcase := range n { + tcase.ch = make(chan struct{}, 1) + state.Watch(watch.NewItems(tcase.item), tcase.ch) + } +} + +// verify ensures that each channel received a notification. +func (n notifyTest) verify(t *testing.T) { + for _, tcase := range n { + if len(tcase.ch) != 1 { + t.Fatalf("should notify %s", tcase.desc) + } + } +} + // NodeIDSort is used to sort nodes by ID type NodeIDSort []*structs.Node From 284c2e2f2b07ed976422d877f821e77e887147fd Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Fri, 30 Oct 2015 08:27:47 -0700 Subject: [PATCH 24/26] nomad: cleanup and more tests --- nomad/alloc_endpoint.go | 2 +- nomad/alloc_endpoint_test.go | 6 +- nomad/eval_endpoint_test.go | 10 +- nomad/job_endpoint_test.go | 10 +- nomad/node_endpoint_test.go | 14 +-- nomad/state/state_store.go | 10 +- nomad/state/state_store_test.go | 215 ++++++++++++++++---------------- 7 files changed, 139 insertions(+), 128 deletions(-) diff --git a/nomad/alloc_endpoint.go b/nomad/alloc_endpoint.go index e8b6af63c85..c07d5549d73 100644 --- a/nomad/alloc_endpoint.go +++ b/nomad/alloc_endpoint.go @@ -86,8 +86,8 @@ func (a *Alloc) GetAlloc(args *structs.AllocSpecificRequest, } // Setup the output + reply.Alloc = out if out != nil { - reply.Alloc = out reply.Index = out.ModifyIndex } else { // Use the last index that affected the nodes table diff --git a/nomad/alloc_endpoint_test.go b/nomad/alloc_endpoint_test.go index 4147011ac95..bcab0a3876b 100644 --- a/nomad/alloc_endpoint_test.go +++ b/nomad/alloc_endpoint_test.go @@ -74,7 +74,7 @@ func TestAllocEndpoint_List_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + if elapsed := time.Since(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } if resp.Index != 2 { @@ -101,7 +101,7 @@ func TestAllocEndpoint_List_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + if elapsed := time.Since(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp2) } if resp2.Index != 3 { @@ -186,7 +186,7 @@ func TestAllocEndpoint_GetAlloc_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 200*time.Millisecond { + if elapsed := time.Since(start); elapsed < 200*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } if resp.Index != 200 { diff --git a/nomad/eval_endpoint_test.go b/nomad/eval_endpoint_test.go index 6f3d154e534..55782a031da 100644 --- a/nomad/eval_endpoint_test.go +++ b/nomad/eval_endpoint_test.go @@ -92,7 +92,7 @@ func TestEvalEndpoint_GetEval_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 200*time.Millisecond { + if elapsed := time.Since(start); elapsed < 200*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } if resp.Index != 200 { @@ -117,7 +117,7 @@ func TestEvalEndpoint_GetEval_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + if elapsed := time.Since(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp2) } if resp2.Index != 300 { @@ -440,7 +440,7 @@ func TestEvalEndpoint_List_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + if elapsed := time.Since(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } if resp.Index != 2 { @@ -464,7 +464,7 @@ func TestEvalEndpoint_List_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + if elapsed := time.Since(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp2) } if resp2.Index != 3 { @@ -551,7 +551,7 @@ func TestEvalEndpoint_Allocations_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 200*time.Millisecond { + if elapsed := time.Since(start); elapsed < 200*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } if resp.Index != 200 { diff --git a/nomad/job_endpoint_test.go b/nomad/job_endpoint_test.go index 9e09de53883..c12e5b4630f 100644 --- a/nomad/job_endpoint_test.go +++ b/nomad/job_endpoint_test.go @@ -402,7 +402,7 @@ func TestJobEndpoint_GetJob_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 200*time.Millisecond { + if elapsed := time.Since(start); elapsed < 200*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } if resp.Index != 200 { @@ -427,7 +427,7 @@ func TestJobEndpoint_GetJob_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + if elapsed := time.Since(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp2) } if resp2.Index != 300 { @@ -501,7 +501,7 @@ func TestJobEndpoint_ListJobs_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + if elapsed := time.Since(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } if resp.Index != 100 { @@ -525,7 +525,7 @@ func TestJobEndpoint_ListJobs_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + if elapsed := time.Since(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp2) } if resp2.Index != 200 { @@ -613,7 +613,7 @@ func TestJobEndpoint_Allocations_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 200*time.Millisecond { + if elapsed := time.Since(start); elapsed < 200*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } if resp.Index != 200 { diff --git a/nomad/node_endpoint_test.go b/nomad/node_endpoint_test.go index 9a74316c7bd..74b154655e7 100644 --- a/nomad/node_endpoint_test.go +++ b/nomad/node_endpoint_test.go @@ -410,7 +410,7 @@ func TestClientEndpoint_GetNode_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 200*time.Millisecond { + if elapsed := time.Since(start); elapsed < 200*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } if resp.Index != 200 { @@ -437,7 +437,7 @@ func TestClientEndpoint_GetNode_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + if elapsed := time.Since(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } if resp2.Index != 300 { @@ -461,7 +461,7 @@ func TestClientEndpoint_GetNode_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + if elapsed := time.Since(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } if resp3.Index != 400 { @@ -910,7 +910,7 @@ func TestClientEndpoint_ListNodes_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + if elapsed := time.Since(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp) } if resp.Index != 2 { @@ -934,7 +934,7 @@ func TestClientEndpoint_ListNodes_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + if elapsed := time.Since(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp2) } if resp2.Index != 3 { @@ -958,7 +958,7 @@ func TestClientEndpoint_ListNodes_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + if elapsed := time.Since(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp3) } if resp3.Index != 4 { @@ -982,7 +982,7 @@ func TestClientEndpoint_ListNodes_Blocking(t *testing.T) { t.Fatalf("err: %v", err) } - if elapsed := time.Now().Sub(start); elapsed < 100*time.Millisecond { + if elapsed := time.Since(start); elapsed < 100*time.Millisecond { t.Fatalf("should block (returned in %s) %#v", elapsed, resp4) } if resp4.Index != 5 { diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index 47ead285e47..ec5aef29f83 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -455,8 +455,11 @@ func (s *StateStore) DeleteEval(index uint64, evals []string, allocs []string) e if err := txn.Delete("allocs", existing); err != nil { return fmt.Errorf("alloc delete failed: %v", err) } - watcher.Add(watch.Item{Alloc: alloc}) - watcher.Add(watch.Item{AllocNode: existing.(*structs.Allocation).NodeID}) + realAlloc := existing.(*structs.Allocation) + watcher.Add(watch.Item{Alloc: realAlloc.ID}) + watcher.Add(watch.Item{AllocEval: realAlloc.EvalID}) + watcher.Add(watch.Item{AllocJob: realAlloc.JobID}) + watcher.Add(watch.Item{AllocNode: realAlloc.NodeID}) } // Update the indexes @@ -795,6 +798,9 @@ func (r *StateRestore) EvalRestore(eval *structs.Evaluation) error { // AllocRestore is used to restore an allocation func (r *StateRestore) AllocRestore(alloc *structs.Allocation) error { r.items.Add(watch.Item{Table: "allocs"}) + r.items.Add(watch.Item{Alloc: alloc.ID}) + r.items.Add(watch.Item{AllocEval: alloc.EvalID}) + r.items.Add(watch.Item{AllocJob: alloc.JobID}) r.items.Add(watch.Item{AllocNode: alloc.NodeID}) if err := r.txn.Insert("allocs", alloc); err != nil { return fmt.Errorf("alloc insert failed: %v", err) diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index 788b9f26a8c..2a596745071 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -26,11 +26,10 @@ func TestStateStore_UpsertNode_Node(t *testing.T) { state := testStateStore(t) node := mock.Node() - notify := notifyTest{ - {desc: "table", item: watch.Item{Table: "nodes"}}, - {desc: "node", item: watch.Item{Node: node.ID}}, - } - notify.start(state) + notify := setupNotifyTest( + state, + watch.Item{Table: "nodes"}, + watch.Item{Node: node.ID}) err := state.UpsertNode(1000, node) if err != nil { @@ -61,11 +60,10 @@ func TestStateStore_DeleteNode_Node(t *testing.T) { state := testStateStore(t) node := mock.Node() - notify := notifyTest{ - {desc: "table", item: watch.Item{Table: "nodes"}}, - {desc: "node", item: watch.Item{Node: node.ID}}, - } - notify.start(state) + notify := setupNotifyTest( + state, + watch.Item{Table: "nodes"}, + watch.Item{Node: node.ID}) err := state.UpsertNode(1000, node) if err != nil { @@ -101,11 +99,10 @@ func TestStateStore_UpdateNodeStatus_Node(t *testing.T) { state := testStateStore(t) node := mock.Node() - notify := notifyTest{ - {desc: "table", item: watch.Item{Table: "nodes"}}, - {desc: "node", item: watch.Item{Node: node.ID}}, - } - notify.start(state) + notify := setupNotifyTest( + state, + watch.Item{Table: "nodes"}, + watch.Item{Node: node.ID}) err := state.UpsertNode(1000, node) if err != nil { @@ -144,11 +141,10 @@ func TestStateStore_UpdateNodeDrain_Node(t *testing.T) { state := testStateStore(t) node := mock.Node() - notify := notifyTest{ - {desc: "table", item: watch.Item{Table: "nodes"}}, - {desc: "node", item: watch.Item{Node: node.ID}}, - } - notify.start(state) + notify := setupNotifyTest( + state, + watch.Item{Table: "nodes"}, + watch.Item{Node: node.ID}) err := state.UpsertNode(1000, node) if err != nil { @@ -223,11 +219,10 @@ func TestStateStore_RestoreNode(t *testing.T) { state := testStateStore(t) node := mock.Node() - notify := notifyTest{ - {desc: "table", item: watch.Item{Table: "nodes"}}, - {desc: "node", item: watch.Item{Node: node.ID}}, - } - notify.start(state) + notify := setupNotifyTest( + state, + watch.Item{Table: "nodes"}, + watch.Item{Node: node.ID}) restore, err := state.Restore() if err != nil { @@ -256,11 +251,10 @@ func TestStateStore_UpsertJob_Job(t *testing.T) { state := testStateStore(t) job := mock.Job() - notify := notifyTest{ - {desc: "table", item: watch.Item{Table: "jobs"}}, - {desc: "job", item: watch.Item{Job: job.ID}}, - } - notify.start(state) + notify := setupNotifyTest( + state, + watch.Item{Table: "jobs"}, + watch.Item{Job: job.ID}) err := state.UpsertJob(1000, job) if err != nil { @@ -291,11 +285,10 @@ func TestStateStore_UpdateUpsertJob_Job(t *testing.T) { state := testStateStore(t) job := mock.Job() - notify := notifyTest{ - {desc: "table", item: watch.Item{Table: "jobs"}}, - {desc: "job", item: watch.Item{Job: job.ID}}, - } - notify.start(state) + notify := setupNotifyTest( + state, + watch.Item{Table: "jobs"}, + watch.Item{Job: job.ID}) err := state.UpsertJob(1000, job) if err != nil { @@ -340,11 +333,10 @@ func TestStateStore_DeleteJob_Job(t *testing.T) { state := testStateStore(t) job := mock.Job() - notify := notifyTest{ - {desc: "table", item: watch.Item{Table: "jobs"}}, - {desc: "job", item: watch.Item{Job: job.ID}}, - } - notify.start(state) + notify := setupNotifyTest( + state, + watch.Item{Table: "jobs"}, + watch.Item{Job: job.ID}) err := state.UpsertJob(1000, job) if err != nil { @@ -483,11 +475,10 @@ func TestStateStore_RestoreJob(t *testing.T) { state := testStateStore(t) job := mock.Job() - notify := notifyTest{ - {desc: "table", item: watch.Item{Table: "jobs"}}, - {desc: "job", item: watch.Item{Job: job.ID}}, - } - notify.start(state) + notify := setupNotifyTest( + state, + watch.Item{Table: "jobs"}, + watch.Item{Job: job.ID}) restore, err := state.Restore() if err != nil { @@ -574,11 +565,10 @@ func TestStateStore_UpsertEvals_Eval(t *testing.T) { state := testStateStore(t) eval := mock.Eval() - notify := notifyTest{ - {desc: "table", item: watch.Item{Table: "evals"}}, - {desc: "eval", item: watch.Item{Eval: eval.ID}}, - } - notify.start(state) + notify := setupNotifyTest( + state, + watch.Item{Table: "evals"}, + watch.Item{Eval: eval.ID}) err := state.UpsertEvals(1000, []*structs.Evaluation{eval}) if err != nil { @@ -614,11 +604,10 @@ func TestStateStore_Update_UpsertEvals_Eval(t *testing.T) { t.Fatalf("err: %v", err) } - notify := notifyTest{ - {desc: "table", item: watch.Item{Table: "evals"}}, - {desc: "eval", item: watch.Item{Eval: eval.ID}}, - } - notify.start(state) + notify := setupNotifyTest( + state, + watch.Item{Table: "evals"}, + watch.Item{Eval: eval.ID}) eval2 := mock.Eval() eval2.ID = eval.ID @@ -661,16 +650,19 @@ func TestStateStore_DeleteEval_Eval(t *testing.T) { alloc1 := mock.Alloc() alloc2 := mock.Alloc() - notify := notifyTest{ - {desc: "table", item: watch.Item{Table: "evals"}}, - {desc: "eval1", item: watch.Item{Eval: eval1.ID}}, - {desc: "eval2", item: watch.Item{Eval: eval2.ID}}, - {desc: "alloc1", item: watch.Item{Alloc: alloc1.ID}}, - {desc: "alloc2", item: watch.Item{Alloc: alloc2.ID}}, - {desc: "allocnode1", item: watch.Item{AllocNode: alloc1.NodeID}}, - {desc: "allocnode2", item: watch.Item{AllocNode: alloc2.NodeID}}, - } - notify.start(state) + notify := setupNotifyTest( + state, + watch.Item{Table: "evals"}, + watch.Item{Eval: eval1.ID}, + watch.Item{Eval: eval2.ID}, + watch.Item{Alloc: alloc1.ID}, + watch.Item{Alloc: alloc2.ID}, + watch.Item{AllocEval: alloc1.EvalID}, + watch.Item{AllocEval: alloc2.EvalID}, + watch.Item{AllocJob: alloc1.JobID}, + watch.Item{AllocJob: alloc2.JobID}, + watch.Item{AllocNode: alloc1.NodeID}, + watch.Item{AllocNode: alloc2.NodeID}) err := state.UpsertEvals(1000, []*structs.Evaluation{eval1, eval2}) if err != nil { @@ -813,11 +805,10 @@ func TestStateStore_RestoreEval(t *testing.T) { state := testStateStore(t) eval := mock.Eval() - notify := notifyTest{ - {desc: "table", item: watch.Item{Table: "evals"}}, - {desc: "eval", item: watch.Item{Eval: eval.ID}}, - } - notify.start(state) + notify := setupNotifyTest( + state, + watch.Item{Table: "evals"}, + watch.Item{Eval: eval.ID}) restore, err := state.Restore() if err != nil { @@ -846,14 +837,13 @@ func TestStateStore_UpdateAllocFromClient(t *testing.T) { state := testStateStore(t) alloc := mock.Alloc() - notify := notifyTest{ - {desc: "table", item: watch.Item{Table: "allocs"}}, - {desc: "alloc", item: watch.Item{Alloc: alloc.ID}}, - {desc: "alloceval", item: watch.Item{AllocEval: alloc.EvalID}}, - {desc: "allocjob", item: watch.Item{AllocJob: alloc.JobID}}, - {desc: "allocnode", item: watch.Item{AllocNode: alloc.NodeID}}, - } - notify.start(state) + notify := setupNotifyTest( + state, + watch.Item{Table: "allocs"}, + watch.Item{Alloc: alloc.ID}, + watch.Item{AllocEval: alloc.EvalID}, + watch.Item{AllocJob: alloc.JobID}, + watch.Item{AllocNode: alloc.NodeID}) err := state.UpsertAllocs(1000, []*structs.Allocation{alloc}) if err != nil { @@ -894,14 +884,13 @@ func TestStateStore_UpsertAlloc_Alloc(t *testing.T) { state := testStateStore(t) alloc := mock.Alloc() - notify := notifyTest{ - {desc: "table", item: watch.Item{Table: "allocs"}}, - {desc: "alloc", item: watch.Item{Alloc: alloc.ID}}, - {desc: "alloceval", item: watch.Item{AllocEval: alloc.EvalID}}, - {desc: "allocjob", item: watch.Item{AllocJob: alloc.JobID}}, - {desc: "allocnode", item: watch.Item{AllocNode: alloc.NodeID}}, - } - notify.start(state) + notify := setupNotifyTest( + state, + watch.Item{Table: "allocs"}, + watch.Item{Alloc: alloc.ID}, + watch.Item{AllocEval: alloc.EvalID}, + watch.Item{AllocJob: alloc.JobID}, + watch.Item{AllocNode: alloc.NodeID}) err := state.UpsertAllocs(1000, []*structs.Allocation{alloc}) if err != nil { @@ -941,14 +930,13 @@ func TestStateStore_UpdateAlloc_Alloc(t *testing.T) { alloc2.ID = alloc.ID alloc2.NodeID = alloc.NodeID + ".new" - notify := notifyTest{ - {desc: "table", item: watch.Item{Table: "allocs"}}, - {desc: "alloc", item: watch.Item{Alloc: alloc2.ID}}, - {desc: "alloceval", item: watch.Item{AllocEval: alloc2.EvalID}}, - {desc: "allocjob", item: watch.Item{AllocJob: alloc2.JobID}}, - {desc: "allocnode", item: watch.Item{AllocNode: alloc2.NodeID}}, - } - notify.start(state) + notify := setupNotifyTest( + state, + watch.Item{Table: "allocs"}, + watch.Item{Alloc: alloc2.ID}, + watch.Item{AllocEval: alloc2.EvalID}, + watch.Item{AllocJob: alloc2.JobID}, + watch.Item{AllocNode: alloc2.NodeID}) err = state.UpsertAllocs(1001, []*structs.Allocation{alloc2}) if err != nil { @@ -1111,13 +1099,21 @@ func TestStateStore_Allocs(t *testing.T) { func TestStateStore_RestoreAlloc(t *testing.T) { state := testStateStore(t) + alloc := mock.Alloc() + + notify := setupNotifyTest( + state, + watch.Item{Table: "allocs"}, + watch.Item{Alloc: alloc.ID}, + watch.Item{AllocEval: alloc.EvalID}, + watch.Item{AllocJob: alloc.JobID}, + watch.Item{AllocNode: alloc.NodeID}) restore, err := state.Restore() if err != nil { t.Fatalf("err: %v", err) } - alloc := mock.Alloc() err = restore.AllocRestore(alloc) if err != nil { t.Fatalf("err: %v", err) @@ -1133,6 +1129,8 @@ func TestStateStore_RestoreAlloc(t *testing.T) { if !reflect.DeepEqual(out, alloc) { t.Fatalf("Bad: %#v %#v", out, alloc) } + + notify.verify(t) } func TestStateWatch_watch(t *testing.T) { @@ -1172,15 +1170,30 @@ func TestStateWatch_stopWatch(t *testing.T) { // Unsubscribe stop notifications sw.stopWatch(watch.Item{Table: "foo"}, notify) + // Check that the group was removed + if _, ok := sw.items[watch.Item{Table: "foo"}]; ok { + t.Fatalf("should remove group") + } + + // Check that we are not notified sw.notify(watch.NewItems(watch.Item{Table: "foo"})) if len(notify) != 0 { t.Fatalf("should not notify") } } +func setupNotifyTest(state *StateStore, items ...watch.Item) notifyTest { + var n notifyTest + for _, item := range items { + ch := make(chan struct{}, 1) + state.Watch(watch.NewItems(item), ch) + n = append(n, ¬ifyTestCase{item, ch}) + } + return n +} + // notifyTestCase is used to set up and verify watch triggers. type notifyTestCase struct { - desc string item watch.Item ch chan struct{} } @@ -1188,19 +1201,11 @@ type notifyTestCase struct { // notifyTest is a suite of notifyTestCases. type notifyTest []*notifyTestCase -// start creates the notify channels and subscribes them. -func (n notifyTest) start(state *StateStore) { - for _, tcase := range n { - tcase.ch = make(chan struct{}, 1) - state.Watch(watch.NewItems(tcase.item), tcase.ch) - } -} - // verify ensures that each channel received a notification. func (n notifyTest) verify(t *testing.T) { for _, tcase := range n { if len(tcase.ch) != 1 { - t.Fatalf("should notify %s", tcase.desc) + t.Fatalf("should notify %#v", tcase.item) } } } From 5f53478137ec01b71cda970f3cba1f1e4d5d2944 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Fri, 30 Oct 2015 08:42:23 -0700 Subject: [PATCH 25/26] nomad/state: subscribe/unsubscribe all watch items while holding the lock --- nomad/state/state_store.go | 40 +++++++++++++++++---------------- nomad/state/state_store_test.go | 13 ++++++----- nomad/watch/watch.go | 1 + 3 files changed, 30 insertions(+), 24 deletions(-) diff --git a/nomad/state/state_store.go b/nomad/state/state_store.go index ec5aef29f83..30ee8725982 100644 --- a/nomad/state/state_store.go +++ b/nomad/state/state_store.go @@ -75,16 +75,14 @@ func (s *StateStore) Restore() (*StateRestore, error) { return r, nil } +// Watch subscribes a channel to a set of watch items. func (s *StateStore) Watch(items watch.Items, notify chan struct{}) { - for wi, _ := range items { - s.watch.watch(wi, notify) - } + s.watch.watch(items, notify) } +// StopWatch unsubscribes a channel from a set of watch items. func (s *StateStore) StopWatch(items watch.Items, notify chan struct{}) { - for wi, _ := range items { - s.watch.stopWatch(wi, notify) - } + s.watch.stopWatch(items, notify) } // UpsertNode is used to register a node or update a node definition @@ -830,28 +828,32 @@ func newStateWatch() *stateWatch { } } -// watch subscribes a channel to the given watch item. -func (w *stateWatch) watch(wi watch.Item, ch chan struct{}) { +// watch subscribes a channel to the given watch items. +func (w *stateWatch) watch(items watch.Items, ch chan struct{}) { w.l.Lock() defer w.l.Unlock() - grp, ok := w.items[wi] - if !ok { - grp = new(NotifyGroup) - w.items[wi] = grp + for item, _ := range items { + grp, ok := w.items[item] + if !ok { + grp = new(NotifyGroup) + w.items[item] = grp + } + grp.Wait(ch) } - grp.Wait(ch) } -// stopWatch unsubscribes a channel from the given watch item. -func (w *stateWatch) stopWatch(wi watch.Item, ch chan struct{}) { +// stopWatch unsubscribes a channel from the given watch items. +func (w *stateWatch) stopWatch(items watch.Items, ch chan struct{}) { w.l.Lock() defer w.l.Unlock() - if grp, ok := w.items[wi]; ok { - grp.Clear(ch) - if grp.Empty() { - delete(w.items, wi) + for item, _ := range items { + if grp, ok := w.items[item]; ok { + grp.Clear(ch) + if grp.Empty() { + delete(w.items, item) + } } } } diff --git a/nomad/state/state_store_test.go b/nomad/state/state_store_test.go index 2a596745071..5e1021e557c 100644 --- a/nomad/state/state_store_test.go +++ b/nomad/state/state_store_test.go @@ -653,6 +653,7 @@ func TestStateStore_DeleteEval_Eval(t *testing.T) { notify := setupNotifyTest( state, watch.Item{Table: "evals"}, + watch.Item{Table: "allocs"}, watch.Item{Eval: eval1.ID}, watch.Item{Eval: eval2.ID}, watch.Item{Alloc: alloc1.ID}, @@ -1140,9 +1141,9 @@ func TestStateWatch_watch(t *testing.T) { notify3 := make(chan struct{}, 1) // Notifications trigger subscribed channels - sw.watch(watch.Item{Table: "foo"}, notify1) - sw.watch(watch.Item{Table: "bar"}, notify2) - sw.watch(watch.Item{Table: "baz"}, notify3) + sw.watch(watch.NewItems(watch.Item{Table: "foo"}), notify1) + sw.watch(watch.NewItems(watch.Item{Table: "bar"}), notify2) + sw.watch(watch.NewItems(watch.Item{Table: "baz"}), notify3) items := watch.NewItems() items.Add(watch.Item{Table: "foo"}) @@ -1165,10 +1166,10 @@ func TestStateWatch_stopWatch(t *testing.T) { notify := make(chan struct{}) // First subscribe - sw.watch(watch.Item{Table: "foo"}, notify) + sw.watch(watch.NewItems(watch.Item{Table: "foo"}), notify) // Unsubscribe stop notifications - sw.stopWatch(watch.Item{Table: "foo"}, notify) + sw.stopWatch(watch.NewItems(watch.Item{Table: "foo"}), notify) // Check that the group was removed if _, ok := sw.items[watch.Item{Table: "foo"}]; ok { @@ -1182,6 +1183,8 @@ func TestStateWatch_stopWatch(t *testing.T) { } } +// setupNotifyTest takes a state store and a set of watch items, then creates +// and subscribes a notification channel for each item. func setupNotifyTest(state *StateStore, items ...watch.Item) notifyTest { var n notifyTest for _, item := range items { diff --git a/nomad/watch/watch.go b/nomad/watch/watch.go index c71fe508778..102e535b2ed 100644 --- a/nomad/watch/watch.go +++ b/nomad/watch/watch.go @@ -21,6 +21,7 @@ type Item struct { // the items as they are added using map keys. type Items map[Item]struct{} +// NewItems creates a new Items set and adds the given items. func NewItems(items ...Item) Items { wi := make(Items) for _, item := range items { From 2a1577ec8823500d82cd9cdf9fbb8c6acb9d3e93 Mon Sep 17 00:00:00 2001 From: Ryan Uber Date: Wed, 4 Nov 2015 11:18:17 -0800 Subject: [PATCH 26/26] nomad/watch: add a note about the Item struct --- nomad/watch/watch.go | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/nomad/watch/watch.go b/nomad/watch/watch.go index 102e535b2ed..4e9bafbc903 100644 --- a/nomad/watch/watch.go +++ b/nomad/watch/watch.go @@ -5,7 +5,9 @@ package watch // the underlying state store. // Item describes the scope of a watch. It is used to provide a uniform -// input for subscribe/unsubscribe and notification firing. +// input for subscribe/unsubscribe and notification firing. Specifying +// multiple fields does not place a watch on multiple items. Each Item +// describes exactly one scoped watch. type Item struct { Alloc string AllocEval string