writetxn can return error, add alloc and job generic events. Add events
table for durability
This commit is contained in:
parent
559517455a
commit
9d48818eb8
|
@ -1,7 +1,6 @@
|
|||
package nomad
|
||||
|
||||
import (
|
||||
"context"
|
||||
"reflect"
|
||||
"testing"
|
||||
"time"
|
||||
|
@ -194,7 +193,7 @@ func TestAllocEndpoint_List_Blocking(t *testing.T) {
|
|||
alloc2.ClientStatus = structs.AllocClientStatusRunning
|
||||
time.AfterFunc(100*time.Millisecond, func() {
|
||||
state.UpsertJobSummary(3, mock.JobSummary(alloc2.JobID))
|
||||
if err := state.UpdateAllocsFromClient(context.Background(), 4, []*structs.Allocation{alloc2}); err != nil {
|
||||
if err := state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 4, []*structs.Allocation{alloc2}); err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
})
|
||||
|
|
|
@ -1,7 +1,6 @@
|
|||
package nomad
|
||||
|
||||
import (
|
||||
"context"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
|
@ -1301,7 +1300,7 @@ func TestDeploymentEndpoint_Allocations_Blocking(t *testing.T) {
|
|||
a2.ClientStatus = structs.AllocClientStatusRunning
|
||||
time.AfterFunc(100*time.Millisecond, func() {
|
||||
assert.Nil(state.UpsertJobSummary(5, mock.JobSummary(a2.JobID)), "UpsertJobSummary")
|
||||
assert.Nil(state.UpdateAllocsFromClient(context.Background(), 6, []*structs.Allocation{a2}), "updateAllocsFromClient")
|
||||
assert.Nil(state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 6, []*structs.Allocation{a2}), "updateAllocsFromClient")
|
||||
})
|
||||
|
||||
req.MinQueryIndex = 4
|
||||
|
|
|
@ -1,7 +1,6 @@
|
|||
package deploymentwatcher
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"testing"
|
||||
"time"
|
||||
|
@ -918,7 +917,7 @@ func TestDeploymentWatcher_Watch_NoProgressDeadline(t *testing.T) {
|
|||
HealthyAllocationIDs: []string{a.ID},
|
||||
},
|
||||
}
|
||||
require.Nil(m.state.UpdateDeploymentAllocHealth(context.Background(), m.nextIndex(), req), "UpsertDeploymentAllocHealth")
|
||||
require.Nil(m.state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, m.nextIndex(), req), "UpsertDeploymentAllocHealth")
|
||||
}
|
||||
|
||||
// Wait for there to be one eval
|
||||
|
@ -946,7 +945,7 @@ func TestDeploymentWatcher_Watch_NoProgressDeadline(t *testing.T) {
|
|||
UnhealthyAllocationIDs: []string{a.ID},
|
||||
},
|
||||
}
|
||||
require.Nil(m.state.UpdateDeploymentAllocHealth(context.Background(), m.nextIndex(), req2), "UpsertDeploymentAllocHealth")
|
||||
require.Nil(m.state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, m.nextIndex(), req2), "UpsertDeploymentAllocHealth")
|
||||
|
||||
// Wait for there to be one eval
|
||||
testutil.WaitForResult(func() (bool, error) {
|
||||
|
@ -1025,7 +1024,7 @@ func TestDeploymentWatcher_Watch_ProgressDeadline(t *testing.T) {
|
|||
Healthy: helper.BoolToPtr(false),
|
||||
Timestamp: now,
|
||||
}
|
||||
require.Nil(m.state.UpdateAllocsFromClient(context.Background(), 100, []*structs.Allocation{a2}))
|
||||
require.Nil(m.state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 100, []*structs.Allocation{a2}))
|
||||
|
||||
// Wait for the deployment to be failed
|
||||
testutil.WaitForResult(func() (bool, error) {
|
||||
|
@ -1209,7 +1208,7 @@ func TestDeploymentWatcher_Watch_ProgressDeadline_Canaries(t *testing.T) {
|
|||
Healthy: helper.BoolToPtr(true),
|
||||
Timestamp: now,
|
||||
}
|
||||
require.Nil(m.state.UpdateAllocsFromClient(context.Background(), m.nextIndex(), []*structs.Allocation{a2}))
|
||||
require.Nil(m.state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, m.nextIndex(), []*structs.Allocation{a2}))
|
||||
|
||||
// Wait for the deployment to cross the deadline
|
||||
dout, err := m.state.DeploymentByID(nil, d.ID)
|
||||
|
@ -1382,7 +1381,7 @@ func TestDeploymentWatcher_Watch_StartWithoutProgressDeadline(t *testing.T) {
|
|||
Healthy: helper.BoolToPtr(false),
|
||||
Timestamp: time.Now(),
|
||||
}
|
||||
require.Nil(m.state.UpdateAllocsFromClient(context.Background(), m.nextIndex(), []*structs.Allocation{a2}))
|
||||
require.Nil(m.state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, m.nextIndex(), []*structs.Allocation{a2}))
|
||||
|
||||
// Wait for the alloc's DesiredState to set reschedule
|
||||
testutil.WaitForResult(func() (bool, error) {
|
||||
|
@ -1454,7 +1453,7 @@ func TestDeploymentWatcher_RollbackFailed(t *testing.T) {
|
|||
HealthyAllocationIDs: []string{a.ID},
|
||||
},
|
||||
}
|
||||
require.Nil(m.state.UpdateDeploymentAllocHealth(context.Background(), m.nextIndex(), req), "UpsertDeploymentAllocHealth")
|
||||
require.Nil(m.state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, m.nextIndex(), req), "UpsertDeploymentAllocHealth")
|
||||
}
|
||||
|
||||
// Wait for there to be one eval
|
||||
|
@ -1482,7 +1481,7 @@ func TestDeploymentWatcher_RollbackFailed(t *testing.T) {
|
|||
UnhealthyAllocationIDs: []string{a.ID},
|
||||
},
|
||||
}
|
||||
require.Nil(m.state.UpdateDeploymentAllocHealth(context.Background(), m.nextIndex(), req2), "UpsertDeploymentAllocHealth")
|
||||
require.Nil(m.state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, m.nextIndex(), req2), "UpsertDeploymentAllocHealth")
|
||||
|
||||
// Wait for there to be one eval
|
||||
testutil.WaitForResult(func() (bool, error) {
|
||||
|
@ -1563,7 +1562,7 @@ func TestWatcher_BatchAllocUpdates(t *testing.T) {
|
|||
HealthyAllocationIDs: []string{a1.ID},
|
||||
},
|
||||
}
|
||||
require.Nil(m.state.UpdateDeploymentAllocHealth(context.Background(), m.nextIndex(), req), "UpsertDeploymentAllocHealth")
|
||||
require.Nil(m.state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, m.nextIndex(), req), "UpsertDeploymentAllocHealth")
|
||||
|
||||
req2 := &structs.ApplyDeploymentAllocHealthRequest{
|
||||
DeploymentAllocHealthRequest: structs.DeploymentAllocHealthRequest{
|
||||
|
@ -1571,7 +1570,7 @@ func TestWatcher_BatchAllocUpdates(t *testing.T) {
|
|||
HealthyAllocationIDs: []string{a2.ID},
|
||||
},
|
||||
}
|
||||
require.Nil(m.state.UpdateDeploymentAllocHealth(context.Background(), m.nextIndex(), req2), "UpsertDeploymentAllocHealth")
|
||||
require.Nil(m.state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, m.nextIndex(), req2), "UpsertDeploymentAllocHealth")
|
||||
|
||||
// Wait for there to be one eval for each job
|
||||
testutil.WaitForResult(func() (bool, error) {
|
||||
|
|
|
@ -1,7 +1,6 @@
|
|||
package deploymentwatcher
|
||||
|
||||
import (
|
||||
"context"
|
||||
"reflect"
|
||||
"strings"
|
||||
"sync"
|
||||
|
@ -96,7 +95,7 @@ func (m *mockBackend) UpsertJob(job *structs.Job) (uint64, error) {
|
|||
func (m *mockBackend) UpdateDeploymentStatus(u *structs.DeploymentStatusUpdateRequest) (uint64, error) {
|
||||
m.Called(u)
|
||||
i := m.nextIndex()
|
||||
return i, m.state.UpdateDeploymentStatus(context.Background(), i, u)
|
||||
return i, m.state.UpdateDeploymentStatus(structs.MsgTypeTestSetup, i, u)
|
||||
}
|
||||
|
||||
// matchDeploymentStatusUpdateConfig is used to configure the matching
|
||||
|
@ -150,7 +149,7 @@ func matchDeploymentStatusUpdateRequest(c *matchDeploymentStatusUpdateConfig) fu
|
|||
func (m *mockBackend) UpdateDeploymentPromotion(req *structs.ApplyDeploymentPromoteRequest) (uint64, error) {
|
||||
m.Called(req)
|
||||
i := m.nextIndex()
|
||||
return i, m.state.UpdateDeploymentPromotion(context.Background(), i, req)
|
||||
return i, m.state.UpdateDeploymentPromotion(structs.MsgTypeTestSetup, i, req)
|
||||
}
|
||||
|
||||
// matchDeploymentPromoteRequestConfig is used to configure the matching
|
||||
|
@ -180,7 +179,7 @@ func matchDeploymentPromoteRequest(c *matchDeploymentPromoteRequestConfig) func(
|
|||
func (m *mockBackend) UpdateDeploymentAllocHealth(req *structs.ApplyDeploymentAllocHealthRequest) (uint64, error) {
|
||||
m.Called(req)
|
||||
i := m.nextIndex()
|
||||
return i, m.state.UpdateDeploymentAllocHealth(context.Background(), i, req)
|
||||
return i, m.state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, i, req)
|
||||
}
|
||||
|
||||
// matchDeploymentAllocHealthRequestConfig is used to configure the matching
|
||||
|
|
|
@ -673,7 +673,7 @@ func TestDrainer_AllTypes_NoDeadline(t *testing.T) {
|
|||
new.ClientStatus = structs.AllocClientStatusComplete
|
||||
updates = append(updates, new)
|
||||
}
|
||||
require.Nil(state.UpdateAllocsFromClient(context.Background(), 1000, updates))
|
||||
require.Nil(state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1000, updates))
|
||||
|
||||
// Check that the node drain is removed
|
||||
testutil.WaitForResult(func() (bool, error) {
|
||||
|
|
|
@ -1,7 +1,6 @@
|
|||
package nomad
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"reflect"
|
||||
"strings"
|
||||
|
@ -375,7 +374,7 @@ func TestEvalEndpoint_Dequeue_UpdateWaitIndex(t *testing.T) {
|
|||
EvalID: eval.ID,
|
||||
}
|
||||
assert := assert.New(t)
|
||||
err := state.UpsertPlanResults(context.Background(), 1000, &res)
|
||||
err := state.UpsertPlanResults(structs.MsgTypeTestSetup, 1000, &res)
|
||||
assert.Nil(err)
|
||||
|
||||
// Dequeue the eval
|
||||
|
|
85
nomad/fsm.go
85
nomad/fsm.go
|
@ -1,7 +1,6 @@
|
|||
package nomad
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"io"
|
||||
"reflect"
|
||||
|
@ -204,19 +203,19 @@ func (n *nomadFSM) Apply(log *raft.Log) interface{} {
|
|||
case structs.NodeDeregisterRequestType:
|
||||
return n.applyDeregisterNode(msgType, buf[1:], log.Index)
|
||||
case structs.NodeUpdateStatusRequestType:
|
||||
return n.applyStatusUpdate(buf[1:], log.Index)
|
||||
return n.applyStatusUpdate(msgType, buf[1:], log.Index)
|
||||
case structs.NodeUpdateDrainRequestType:
|
||||
return n.applyDrainUpdate(msgType, buf[1:], log.Index)
|
||||
case structs.JobRegisterRequestType:
|
||||
return n.applyUpsertJob(buf[1:], log.Index)
|
||||
return n.applyUpsertJob(msgType, buf[1:], log.Index)
|
||||
case structs.JobDeregisterRequestType:
|
||||
return n.applyDeregisterJob(buf[1:], log.Index)
|
||||
return n.applyDeregisterJob(msgType, buf[1:], log.Index)
|
||||
case structs.EvalUpdateRequestType:
|
||||
return n.applyUpdateEval(msgType, buf[1:], log.Index)
|
||||
case structs.EvalDeleteRequestType:
|
||||
return n.applyDeleteEval(buf[1:], log.Index)
|
||||
case structs.AllocUpdateRequestType:
|
||||
return n.applyAllocUpdate(buf[1:], log.Index)
|
||||
return n.applyAllocUpdate(msgType, buf[1:], log.Index)
|
||||
case structs.AllocClientUpdateRequestType:
|
||||
return n.applyAllocClientUpdate(msgType, buf[1:], log.Index)
|
||||
case structs.ReconcileJobSummariesRequestType:
|
||||
|
@ -252,16 +251,21 @@ func (n *nomadFSM) Apply(log *raft.Log) interface{} {
|
|||
case structs.UpsertNodeEventsType:
|
||||
return n.applyUpsertNodeEvent(msgType, buf[1:], log.Index)
|
||||
case structs.JobBatchDeregisterRequestType:
|
||||
return n.applyBatchDeregisterJob(buf[1:], log.Index)
|
||||
// TODO(drew) higher priority event
|
||||
return n.applyBatchDeregisterJob(msgType, buf[1:], log.Index)
|
||||
case structs.AllocUpdateDesiredTransitionRequestType:
|
||||
// TODO(drew) higher priority event
|
||||
return n.applyAllocUpdateDesiredTransition(buf[1:], log.Index)
|
||||
case structs.NodeUpdateEligibilityRequestType:
|
||||
// TODO(drew) higher priority event
|
||||
return n.applyNodeEligibilityUpdate(buf[1:], log.Index)
|
||||
case structs.BatchNodeUpdateDrainRequestType:
|
||||
// TODO(drew) higher priority event
|
||||
return n.applyBatchDrainUpdate(buf[1:], log.Index)
|
||||
case structs.SchedulerConfigRequestType:
|
||||
return n.applySchedulerConfigUpdate(buf[1:], log.Index)
|
||||
case structs.NodeBatchDeregisterRequestType:
|
||||
// TODO(drew) higher priority event
|
||||
return n.applyDeregisterNodeBatch(buf[1:], log.Index)
|
||||
case structs.ClusterMetadataRequestType:
|
||||
return n.applyClusterMetadata(buf[1:], log.Index)
|
||||
|
@ -322,12 +326,10 @@ func (n *nomadFSM) applyUpsertNode(reqType structs.MessageType, buf []byte, inde
|
|||
panic(fmt.Errorf("failed to decode request: %v", err))
|
||||
}
|
||||
|
||||
ctx := context.WithValue(context.Background(), state.CtxMsgType, reqType)
|
||||
|
||||
// Handle upgrade paths
|
||||
req.Node.Canonicalize()
|
||||
|
||||
if err := n.state.UpsertNodeCtx(ctx, index, req.Node); err != nil {
|
||||
if err := n.state.UpsertNodeMsgType(reqType, index, req.Node); err != nil {
|
||||
n.logger.Error("UpsertNode failed", "error", err)
|
||||
return err
|
||||
}
|
||||
|
@ -348,9 +350,7 @@ func (n *nomadFSM) applyDeregisterNode(reqType structs.MessageType, buf []byte,
|
|||
panic(fmt.Errorf("failed to decode request: %v", err))
|
||||
}
|
||||
|
||||
ctx := context.WithValue(context.Background(), state.CtxMsgType, reqType)
|
||||
|
||||
if err := n.state.DeleteNodeCtx(ctx, index, []string{req.NodeID}); err != nil {
|
||||
if err := n.state.DeleteNodeMsgType(reqType, index, []string{req.NodeID}); err != nil {
|
||||
n.logger.Error("DeleteNode failed", "error", err)
|
||||
return err
|
||||
}
|
||||
|
@ -373,14 +373,14 @@ func (n *nomadFSM) applyDeregisterNodeBatch(buf []byte, index uint64) interface{
|
|||
return nil
|
||||
}
|
||||
|
||||
func (n *nomadFSM) applyStatusUpdate(buf []byte, index uint64) interface{} {
|
||||
func (n *nomadFSM) applyStatusUpdate(msgType structs.MessageType, buf []byte, index uint64) interface{} {
|
||||
defer metrics.MeasureSince([]string{"nomad", "fsm", "node_status_update"}, time.Now())
|
||||
var req structs.NodeUpdateStatusRequest
|
||||
if err := structs.Decode(buf, &req); err != nil {
|
||||
panic(fmt.Errorf("failed to decode request: %v", err))
|
||||
}
|
||||
|
||||
if err := n.state.UpdateNodeStatus(index, req.NodeID, req.Status, req.UpdatedAt, req.NodeEvent); err != nil {
|
||||
if err := n.state.UpdateNodeStatus(msgType, index, req.NodeID, req.Status, req.UpdatedAt, req.NodeEvent); err != nil {
|
||||
n.logger.Error("UpdateNodeStatus failed", "error", err)
|
||||
return err
|
||||
}
|
||||
|
@ -409,8 +409,6 @@ func (n *nomadFSM) applyDrainUpdate(reqType structs.MessageType, buf []byte, ind
|
|||
panic(fmt.Errorf("failed to decode request: %v", err))
|
||||
}
|
||||
|
||||
ctx := context.WithValue(context.Background(), state.CtxMsgType, reqType)
|
||||
|
||||
// COMPAT Remove in version 0.10
|
||||
// As part of Nomad 0.8 we have deprecated the drain boolean in favor of a
|
||||
// drain strategy but we need to handle the upgrade path where the Raft log
|
||||
|
@ -425,7 +423,7 @@ func (n *nomadFSM) applyDrainUpdate(reqType structs.MessageType, buf []byte, ind
|
|||
}
|
||||
}
|
||||
|
||||
if err := n.state.UpdateNodeDrainCtx(ctx, index, req.NodeID, req.DrainStrategy, req.MarkEligible, req.UpdatedAt, req.NodeEvent); err != nil {
|
||||
if err := n.state.UpdateNodeDrainMsgType(reqType, index, req.NodeID, req.DrainStrategy, req.MarkEligible, req.UpdatedAt, req.NodeEvent); err != nil {
|
||||
n.logger.Error("UpdateNodeDrain failed", "error", err)
|
||||
return err
|
||||
}
|
||||
|
@ -476,7 +474,7 @@ func (n *nomadFSM) applyNodeEligibilityUpdate(buf []byte, index uint64) interfac
|
|||
return nil
|
||||
}
|
||||
|
||||
func (n *nomadFSM) applyUpsertJob(buf []byte, index uint64) interface{} {
|
||||
func (n *nomadFSM) applyUpsertJob(msgType structs.MessageType, buf []byte, index uint64) interface{} {
|
||||
defer metrics.MeasureSince([]string{"nomad", "fsm", "register_job"}, time.Now())
|
||||
var req structs.JobRegisterRequest
|
||||
if err := structs.Decode(buf, &req); err != nil {
|
||||
|
@ -492,7 +490,7 @@ func (n *nomadFSM) applyUpsertJob(buf []byte, index uint64) interface{} {
|
|||
*/
|
||||
req.Job.Canonicalize()
|
||||
|
||||
if err := n.state.UpsertJob(index, req.Job); err != nil {
|
||||
if err := n.state.UpsertJobMsgType(msgType, index, req.Job); err != nil {
|
||||
n.logger.Error("UpsertJob failed", "error", err)
|
||||
return err
|
||||
}
|
||||
|
@ -570,7 +568,7 @@ func (n *nomadFSM) applyUpsertJob(buf []byte, index uint64) interface{} {
|
|||
// so this may be nil during server upgrades.
|
||||
if req.Eval != nil {
|
||||
req.Eval.JobModifyIndex = index
|
||||
if err := n.upsertEvals(context.Background(), index, []*structs.Evaluation{req.Eval}); err != nil {
|
||||
if err := n.upsertEvals(msgType, index, []*structs.Evaluation{req.Eval}); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
@ -578,14 +576,14 @@ func (n *nomadFSM) applyUpsertJob(buf []byte, index uint64) interface{} {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (n *nomadFSM) applyDeregisterJob(buf []byte, index uint64) interface{} {
|
||||
func (n *nomadFSM) applyDeregisterJob(msgType structs.MessageType, buf []byte, index uint64) interface{} {
|
||||
defer metrics.MeasureSince([]string{"nomad", "fsm", "deregister_job"}, time.Now())
|
||||
var req structs.JobDeregisterRequest
|
||||
if err := structs.Decode(buf, &req); err != nil {
|
||||
panic(fmt.Errorf("failed to decode request: %v", err))
|
||||
}
|
||||
|
||||
err := n.state.WithWriteTransaction(index, func(tx state.Txn) error {
|
||||
err := n.state.WithWriteTransaction(msgType, index, func(tx state.Txn) error {
|
||||
err := n.handleJobDeregister(index, req.JobID, req.Namespace, req.Purge, tx)
|
||||
|
||||
if err != nil {
|
||||
|
@ -602,7 +600,7 @@ func (n *nomadFSM) applyDeregisterJob(buf []byte, index uint64) interface{} {
|
|||
// always attempt upsert eval even if job deregister fail
|
||||
if req.Eval != nil {
|
||||
req.Eval.JobModifyIndex = index
|
||||
if err := n.upsertEvals(context.Background(), index, []*structs.Evaluation{req.Eval}); err != nil {
|
||||
if err := n.upsertEvals(msgType, index, []*structs.Evaluation{req.Eval}); err != nil {
|
||||
return err
|
||||
}
|
||||
}
|
||||
|
@ -614,7 +612,7 @@ func (n *nomadFSM) applyDeregisterJob(buf []byte, index uint64) interface{} {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (n *nomadFSM) applyBatchDeregisterJob(buf []byte, index uint64) interface{} {
|
||||
func (n *nomadFSM) applyBatchDeregisterJob(msgType structs.MessageType, buf []byte, index uint64) interface{} {
|
||||
defer metrics.MeasureSince([]string{"nomad", "fsm", "batch_deregister_job"}, time.Now())
|
||||
var req structs.JobBatchDeregisterRequest
|
||||
if err := structs.Decode(buf, &req); err != nil {
|
||||
|
@ -624,7 +622,7 @@ func (n *nomadFSM) applyBatchDeregisterJob(buf []byte, index uint64) interface{}
|
|||
// Perform all store updates atomically to ensure a consistent view for store readers.
|
||||
// A partial update may increment the snapshot index, allowing eval brokers to process
|
||||
// evals for jobs whose deregistering didn't get committed yet.
|
||||
err := n.state.WithWriteTransaction(index, func(tx state.Txn) error {
|
||||
err := n.state.WithWriteTransaction(msgType, index, func(tx state.Txn) error {
|
||||
for jobNS, options := range req.Jobs {
|
||||
if err := n.handleJobDeregister(index, jobNS.ID, jobNS.Namespace, options.Purge, tx); err != nil {
|
||||
n.logger.Error("deregistering job failed", "job", jobNS.ID, "error", err)
|
||||
|
@ -696,13 +694,11 @@ func (n *nomadFSM) applyUpdateEval(msgType structs.MessageType, buf []byte, inde
|
|||
panic(fmt.Errorf("failed to decode request: %v", err))
|
||||
}
|
||||
|
||||
ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType)
|
||||
|
||||
return n.upsertEvals(ctx, index, req.Evals)
|
||||
return n.upsertEvals(msgType, index, req.Evals)
|
||||
}
|
||||
|
||||
func (n *nomadFSM) upsertEvals(ctx context.Context, index uint64, evals []*structs.Evaluation) error {
|
||||
if err := n.state.UpsertEvalsCtx(ctx, index, evals); err != nil {
|
||||
func (n *nomadFSM) upsertEvals(msgType structs.MessageType, index uint64, evals []*structs.Evaluation) error {
|
||||
if err := n.state.UpsertEvalsMsgType(msgType, index, evals); err != nil {
|
||||
n.logger.Error("UpsertEvals failed", "error", err)
|
||||
return err
|
||||
}
|
||||
|
@ -751,7 +747,7 @@ func (n *nomadFSM) applyDeleteEval(buf []byte, index uint64) interface{} {
|
|||
return nil
|
||||
}
|
||||
|
||||
func (n *nomadFSM) applyAllocUpdate(buf []byte, index uint64) interface{} {
|
||||
func (n *nomadFSM) applyAllocUpdate(msgType structs.MessageType, buf []byte, index uint64) interface{} {
|
||||
defer metrics.MeasureSince([]string{"nomad", "fsm", "alloc_update"}, time.Now())
|
||||
var req structs.AllocUpdateRequest
|
||||
if err := structs.Decode(buf, &req); err != nil {
|
||||
|
@ -782,7 +778,7 @@ func (n *nomadFSM) applyAllocUpdate(buf []byte, index uint64) interface{} {
|
|||
alloc.Canonicalize()
|
||||
}
|
||||
|
||||
if err := n.state.UpsertAllocs(index, req.Alloc); err != nil {
|
||||
if err := n.state.UpsertAllocsMsgType(msgType, index, req.Alloc); err != nil {
|
||||
n.logger.Error("UpsertAllocs failed", "error", err)
|
||||
return err
|
||||
}
|
||||
|
@ -810,16 +806,15 @@ func (n *nomadFSM) applyAllocClientUpdate(msgType structs.MessageType, buf []byt
|
|||
}
|
||||
}
|
||||
|
||||
ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType)
|
||||
// Update all the client allocations
|
||||
if err := n.state.UpdateAllocsFromClient(ctx, index, req.Alloc); err != nil {
|
||||
if err := n.state.UpdateAllocsFromClient(msgType, index, req.Alloc); err != nil {
|
||||
n.logger.Error("UpdateAllocFromClient failed", "error", err)
|
||||
return err
|
||||
}
|
||||
|
||||
// Update any evals
|
||||
if len(req.Evals) > 0 {
|
||||
if err := n.upsertEvals(ctx, index, req.Evals); err != nil {
|
||||
if err := n.upsertEvals(msgType, index, req.Evals); err != nil {
|
||||
n.logger.Error("applyAllocClientUpdate failed to update evaluations", "error", err)
|
||||
return err
|
||||
}
|
||||
|
@ -887,9 +882,7 @@ func (n *nomadFSM) applyUpsertNodeEvent(msgType structs.MessageType, buf []byte,
|
|||
panic(fmt.Errorf("failed to decode EmitNodeEventsRequest: %v", err))
|
||||
}
|
||||
|
||||
ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType)
|
||||
|
||||
if err := n.state.UpsertNodeEventsCtx(ctx, index, req.NodeEvents); err != nil {
|
||||
if err := n.state.UpsertNodeEventsMsgType(msgType, index, req.NodeEvents); err != nil {
|
||||
n.logger.Error("failed to add node events", "error", err)
|
||||
return err
|
||||
}
|
||||
|
@ -968,9 +961,7 @@ func (n *nomadFSM) applyPlanResults(msgType structs.MessageType, buf []byte, ind
|
|||
panic(fmt.Errorf("failed to decode request: %v", err))
|
||||
}
|
||||
|
||||
ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType)
|
||||
|
||||
if err := n.state.UpsertPlanResults(ctx, index, &req); err != nil {
|
||||
if err := n.state.UpsertPlanResults(msgType, index, &req); err != nil {
|
||||
n.logger.Error("ApplyPlan failed", "error", err)
|
||||
return err
|
||||
}
|
||||
|
@ -989,9 +980,7 @@ func (n *nomadFSM) applyDeploymentStatusUpdate(msgType structs.MessageType, buf
|
|||
panic(fmt.Errorf("failed to decode request: %v", err))
|
||||
}
|
||||
|
||||
ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType)
|
||||
|
||||
if err := n.state.UpdateDeploymentStatus(ctx, index, &req); err != nil {
|
||||
if err := n.state.UpdateDeploymentStatus(msgType, index, &req); err != nil {
|
||||
n.logger.Error("UpsertDeploymentStatusUpdate failed", "error", err)
|
||||
return err
|
||||
}
|
||||
|
@ -1008,9 +997,7 @@ func (n *nomadFSM) applyDeploymentPromotion(msgType structs.MessageType, buf []b
|
|||
panic(fmt.Errorf("failed to decode request: %v", err))
|
||||
}
|
||||
|
||||
ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType)
|
||||
|
||||
if err := n.state.UpdateDeploymentPromotion(ctx, index, &req); err != nil {
|
||||
if err := n.state.UpdateDeploymentPromotion(msgType, index, &req); err != nil {
|
||||
n.logger.Error("UpsertDeploymentPromotion failed", "error", err)
|
||||
return err
|
||||
}
|
||||
|
@ -1028,9 +1015,7 @@ func (n *nomadFSM) applyDeploymentAllocHealth(msgType structs.MessageType, buf [
|
|||
panic(fmt.Errorf("failed to decode request: %v", err))
|
||||
}
|
||||
|
||||
ctx := context.WithValue(context.Background(), state.CtxMsgType, msgType)
|
||||
|
||||
if err := n.state.UpdateDeploymentAllocHealth(ctx, index, &req); err != nil {
|
||||
if err := n.state.UpdateDeploymentAllocHealth(msgType, index, &req); err != nil {
|
||||
n.logger.Error("UpsertDeploymentAllocHealth failed", "error", err)
|
||||
return err
|
||||
}
|
||||
|
|
|
@ -1,7 +1,6 @@
|
|||
package nomad
|
||||
|
||||
import (
|
||||
"context"
|
||||
"errors"
|
||||
"fmt"
|
||||
"net"
|
||||
|
@ -2049,7 +2048,7 @@ func TestClientEndpoint_GetAllocs_Blocking(t *testing.T) {
|
|||
allocUpdate.ID = alloc.ID
|
||||
allocUpdate.ClientStatus = structs.AllocClientStatusRunning
|
||||
state.UpsertJobSummary(199, mock.JobSummary(allocUpdate.JobID))
|
||||
err := state.UpdateAllocsFromClient(context.Background(), 200, []*structs.Allocation{allocUpdate})
|
||||
err := state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 200, []*structs.Allocation{allocUpdate})
|
||||
if err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
|
@ -2748,7 +2747,7 @@ func TestClientEndpoint_ListNodes_Blocking(t *testing.T) {
|
|||
|
||||
// Node status update triggers watches
|
||||
time.AfterFunc(100*time.Millisecond, func() {
|
||||
errCh <- state.UpdateNodeStatus(40, node.ID, structs.NodeStatusDown, 0, nil)
|
||||
errCh <- state.UpdateNodeStatus(structs.MsgTypeTestSetup, 40, node.ID, structs.NodeStatusDown, 0, nil)
|
||||
})
|
||||
|
||||
req.MinQueryIndex = 38
|
||||
|
|
|
@ -310,7 +310,7 @@ func (p *planner) applyPlan(plan *structs.Plan, result *structs.PlanResult, snap
|
|||
// Optimistically apply to our state view
|
||||
if snap != nil {
|
||||
nextIdx := p.raft.AppliedIndex() + 1
|
||||
if err := snap.UpsertPlanResults(context.Background(), nextIdx, &req); err != nil {
|
||||
if err := snap.UpsertPlanResults(structs.ApplyPlanResultsRequestType, nextIdx, &req); err != nil {
|
||||
return future, err
|
||||
}
|
||||
}
|
||||
|
|
|
@ -32,7 +32,7 @@ func TestDeploymentEventFromChanges(t *testing.T) {
|
|||
|
||||
setupTx.Txn.Commit()
|
||||
|
||||
ctx := context.WithValue(context.Background(), CtxMsgType, structs.DeploymentStatusUpdateRequestType)
|
||||
msgType := structs.DeploymentStatusUpdateRequestType
|
||||
|
||||
req := &structs.DeploymentStatusUpdateRequest{
|
||||
DeploymentUpdate: &structs.DeploymentStatusUpdate{
|
||||
|
@ -44,7 +44,7 @@ func TestDeploymentEventFromChanges(t *testing.T) {
|
|||
// Exlude Job and assert its added
|
||||
}
|
||||
|
||||
require.NoError(t, s.UpdateDeploymentStatus(ctx, 100, req))
|
||||
require.NoError(t, s.UpdateDeploymentStatus(msgType, 100, req))
|
||||
|
||||
events := WaitForEvents(t, s, 100, 1, 1*time.Second)
|
||||
require.Len(t, events, 2)
|
||||
|
@ -113,7 +113,7 @@ func TestDeploymentEventFromChanges_Promotion(t *testing.T) {
|
|||
|
||||
e := mock.Eval()
|
||||
// Request to promote canaries
|
||||
ctx := context.WithValue(context.Background(), CtxMsgType, structs.DeploymentPromoteRequestType)
|
||||
msgType := structs.DeploymentPromoteRequestType
|
||||
req := &structs.ApplyDeploymentPromoteRequest{
|
||||
DeploymentPromoteRequest: structs.DeploymentPromoteRequest{
|
||||
DeploymentID: d.ID,
|
||||
|
@ -122,7 +122,7 @@ func TestDeploymentEventFromChanges_Promotion(t *testing.T) {
|
|||
Eval: e,
|
||||
}
|
||||
|
||||
require.NoError(t, s.UpdateDeploymentPromotion(ctx, 100, req))
|
||||
require.NoError(t, s.UpdateDeploymentPromotion(msgType, 100, req))
|
||||
|
||||
events := WaitForEvents(t, s, 100, 1, 1*time.Second)
|
||||
require.Len(t, events, 2)
|
||||
|
|
|
@ -31,6 +31,8 @@ const (
|
|||
TypeAllocUpdated = "AllocUpdated"
|
||||
|
||||
TypeEvalUpdated = "EvalUpdated"
|
||||
|
||||
TypeJobRegistered = "JobRegistered"
|
||||
)
|
||||
|
||||
type JobEvent struct {
|
||||
|
@ -78,6 +80,12 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, erro
|
|||
eventType = TypeEvalUpdated
|
||||
case structs.AllocClientUpdateRequestType:
|
||||
eventType = TypeAllocUpdated
|
||||
case structs.JobRegisterRequestType:
|
||||
eventType = TypeJobRegistered
|
||||
case structs.AllocUpdateRequestType:
|
||||
eventType = TypeAllocUpdated
|
||||
case structs.NodeUpdateStatusRequestType:
|
||||
eventType = TypeNodeEvent
|
||||
}
|
||||
|
||||
var events []stream.Event
|
||||
|
@ -117,6 +125,39 @@ func GenericEventsFromChanges(tx ReadTxn, changes Changes) ([]stream.Event, erro
|
|||
},
|
||||
}
|
||||
|
||||
events = append(events, event)
|
||||
case "jobs":
|
||||
after, ok := change.After.(*structs.Job)
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("transaction change was not an Allocation")
|
||||
}
|
||||
|
||||
event := stream.Event{
|
||||
Topic: TopicAlloc,
|
||||
Type: eventType,
|
||||
Index: changes.Index,
|
||||
Key: after.ID,
|
||||
Payload: &JobEvent{
|
||||
Job: after,
|
||||
},
|
||||
}
|
||||
|
||||
events = append(events, event)
|
||||
case "nodes":
|
||||
after, ok := change.After.(*structs.Node)
|
||||
if !ok {
|
||||
return nil, fmt.Errorf("transaction change was not a Node")
|
||||
}
|
||||
|
||||
event := stream.Event{
|
||||
Topic: TopicNode,
|
||||
Type: eventType,
|
||||
Index: changes.Index,
|
||||
Key: after.ID,
|
||||
Payload: &NodeEvent{
|
||||
Node: after,
|
||||
},
|
||||
}
|
||||
events = append(events, event)
|
||||
}
|
||||
}
|
||||
|
|
|
@ -52,6 +52,7 @@ func init() {
|
|||
csiPluginTableSchema,
|
||||
scalingPolicyTableSchema,
|
||||
scalingEventTableSchema,
|
||||
eventTableSchema,
|
||||
}...)
|
||||
}
|
||||
|
||||
|
@ -900,3 +901,19 @@ func scalingEventTableSchema() *memdb.TableSchema {
|
|||
},
|
||||
}
|
||||
}
|
||||
|
||||
func eventTableSchema() *memdb.TableSchema {
|
||||
return &memdb.TableSchema{
|
||||
Name: "events",
|
||||
Indexes: map[string]*memdb.IndexSchema{
|
||||
"id": {
|
||||
Name: "id",
|
||||
AllowMissing: true,
|
||||
Unique: true,
|
||||
Indexer: &memdb.UintFieldIndex{
|
||||
Field: "Index",
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
|
|
@ -5,6 +5,7 @@ import (
|
|||
|
||||
memdb "github.com/hashicorp/go-memdb"
|
||||
"github.com/hashicorp/nomad/nomad/mock"
|
||||
"github.com/hashicorp/nomad/nomad/stream"
|
||||
"github.com/stretchr/testify/require"
|
||||
)
|
||||
|
||||
|
@ -144,3 +145,70 @@ func TestState_ScalingPolicyTargetFieldIndex_FromObject(t *testing.T) {
|
|||
require.Error(err)
|
||||
require.Equal("", string(val))
|
||||
}
|
||||
|
||||
func TestEventTableUintIndex(t *testing.T) {
|
||||
|
||||
require := require.New(t)
|
||||
|
||||
const (
|
||||
eventsTable = "events"
|
||||
uintIDIdx = "id"
|
||||
)
|
||||
|
||||
db, err := memdb.NewMemDB(&memdb.DBSchema{
|
||||
Tables: map[string]*memdb.TableSchema{
|
||||
eventsTable: eventTableSchema(),
|
||||
},
|
||||
})
|
||||
require.NoError(err)
|
||||
|
||||
// numRecords in table counts all the items in the table, which is expected
|
||||
// to always be 1 since that's the point of the singletonRecord Indexer.
|
||||
numRecordsInTable := func() int {
|
||||
txn := db.Txn(false)
|
||||
defer txn.Abort()
|
||||
|
||||
iter, err := txn.Get(eventsTable, uintIDIdx)
|
||||
require.NoError(err)
|
||||
|
||||
num := 0
|
||||
for item := iter.Next(); item != nil; item = iter.Next() {
|
||||
num++
|
||||
}
|
||||
return num
|
||||
}
|
||||
|
||||
insertEvents := func(e *stream.Events) {
|
||||
txn := db.Txn(true)
|
||||
err := txn.Insert(eventsTable, e)
|
||||
require.NoError(err)
|
||||
txn.Commit()
|
||||
}
|
||||
|
||||
get := func(idx uint64) *stream.Events {
|
||||
txn := db.Txn(false)
|
||||
defer txn.Abort()
|
||||
record, err := txn.First("events", "id", idx)
|
||||
require.NoError(err)
|
||||
s, ok := record.(*stream.Events)
|
||||
require.True(ok)
|
||||
return s
|
||||
}
|
||||
|
||||
firstEvent := &stream.Events{Index: 10, Events: []stream.Event{{Index: 10}, {Index: 10}}}
|
||||
secondEvent := &stream.Events{Index: 11, Events: []stream.Event{{Index: 11}, {Index: 11}}}
|
||||
thirdEvent := &stream.Events{Index: 202, Events: []stream.Event{{Index: 202}, {Index: 202}}}
|
||||
insertEvents(firstEvent)
|
||||
insertEvents(secondEvent)
|
||||
insertEvents(thirdEvent)
|
||||
require.Equal(3, numRecordsInTable())
|
||||
|
||||
gotFirst := get(10)
|
||||
require.Equal(firstEvent, gotFirst)
|
||||
|
||||
gotSecond := get(11)
|
||||
require.Equal(secondEvent, gotSecond)
|
||||
|
||||
gotThird := get(202)
|
||||
require.Equal(thirdEvent, gotThird)
|
||||
}
|
||||
|
|
|
@ -1,7 +1,6 @@
|
|||
package state
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
|
||||
"github.com/hashicorp/go-memdb"
|
||||
|
@ -80,10 +79,9 @@ func (c *changeTrackerDB) WriteTxn(idx uint64) *txn {
|
|||
return t
|
||||
}
|
||||
|
||||
// WriteTxnCtx is identical to WriteTxn but takes a ctx used for event sourcing
|
||||
func (c *changeTrackerDB) WriteTxnCtx(ctx context.Context, idx uint64) *txn {
|
||||
func (c *changeTrackerDB) WriteTxnMsgT(msgType structs.MessageType, idx uint64) *txn {
|
||||
t := &txn{
|
||||
ctx: ctx,
|
||||
msgType: msgType,
|
||||
Txn: c.db.Txn(true),
|
||||
Index: idx,
|
||||
publish: c.publish,
|
||||
|
@ -124,8 +122,8 @@ func (c *changeTrackerDB) WriteTxnRestore() *txn {
|
|||
// error. Any errors from the callback would be lost, which would result in a
|
||||
// missing change event, even though the state store had changed.
|
||||
type txn struct {
|
||||
// ctx is used to hold message type information from an FSM request
|
||||
ctx context.Context
|
||||
// msgType is used to inform event sourcing which type of event to create
|
||||
msgType structs.MessageType
|
||||
|
||||
*memdb.Txn
|
||||
// Index in raft where the write is occurring. The value is zero for a
|
||||
|
@ -165,20 +163,7 @@ func (tx *txn) Commit() error {
|
|||
// If the context is empty or the value isn't set IgnoreUnknownTypeFlag will
|
||||
// be returned to signal that the MsgType is unknown.
|
||||
func (tx *txn) MsgType() structs.MessageType {
|
||||
if tx.ctx == nil {
|
||||
return structs.IgnoreUnknownTypeFlag
|
||||
}
|
||||
|
||||
raw := tx.ctx.Value(CtxMsgType)
|
||||
if raw == nil {
|
||||
return structs.IgnoreUnknownTypeFlag
|
||||
}
|
||||
|
||||
msgType, ok := raw.(structs.MessageType)
|
||||
if !ok {
|
||||
return structs.IgnoreUnknownTypeFlag
|
||||
}
|
||||
return msgType
|
||||
return tx.msgType
|
||||
}
|
||||
|
||||
func processDBChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) {
|
||||
|
@ -188,6 +173,9 @@ func processDBChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) {
|
|||
return []stream.Event{}, nil
|
||||
case structs.NodeRegisterRequestType:
|
||||
return NodeRegisterEventFromChanges(tx, changes)
|
||||
case structs.NodeUpdateStatusRequestType:
|
||||
// TODO(drew) test
|
||||
return GenericEventsFromChanges(tx, changes)
|
||||
case structs.NodeDeregisterRequestType:
|
||||
return NodeDeregisterEventFromChanges(tx, changes)
|
||||
case structs.NodeUpdateDrainRequestType:
|
||||
|
@ -206,6 +194,12 @@ func processDBChanges(tx ReadTxn, changes Changes) ([]stream.Event, error) {
|
|||
return GenericEventsFromChanges(tx, changes)
|
||||
case structs.AllocClientUpdateRequestType:
|
||||
return GenericEventsFromChanges(tx, changes)
|
||||
case structs.JobRegisterRequestType:
|
||||
// TODO(drew) test
|
||||
return GenericEventsFromChanges(tx, changes)
|
||||
case structs.AllocUpdateRequestType:
|
||||
// TODO(drew) test
|
||||
return GenericEventsFromChanges(tx, changes)
|
||||
}
|
||||
return []stream.Event{}, nil
|
||||
}
|
||||
|
|
|
@ -270,7 +270,7 @@ RUN_QUERY:
|
|||
}
|
||||
|
||||
// UpsertPlanResults is used to upsert the results of a plan.
|
||||
func (s *StateStore) UpsertPlanResults(ctx context.Context, index uint64, results *structs.ApplyPlanResultsRequest) error {
|
||||
func (s *StateStore) UpsertPlanResults(msgType structs.MessageType, index uint64, results *structs.ApplyPlanResultsRequest) error {
|
||||
snapshot, err := s.Snapshot()
|
||||
if err != nil {
|
||||
return err
|
||||
|
@ -292,7 +292,7 @@ func (s *StateStore) UpsertPlanResults(ctx context.Context, index uint64, result
|
|||
return err
|
||||
}
|
||||
|
||||
txn := s.db.WriteTxnCtx(ctx, index)
|
||||
txn := s.db.WriteTxnMsgT(msgType, index)
|
||||
defer txn.Abort()
|
||||
|
||||
// Upsert the newly created or updated deployment
|
||||
|
@ -302,7 +302,7 @@ func (s *StateStore) UpsertPlanResults(ctx context.Context, index uint64, result
|
|||
}
|
||||
}
|
||||
|
||||
// Update the status of deployments effected by the plan.
|
||||
// Update the status of dmsgType structs.MessageType by the plan.
|
||||
if len(results.DeploymentUpdates) != 0 {
|
||||
s.upsertDeploymentUpdates(index, results.DeploymentUpdates, txn)
|
||||
}
|
||||
|
@ -356,11 +356,7 @@ func (s *StateStore) UpsertPlanResults(ctx context.Context, index uint64, result
|
|||
}
|
||||
}
|
||||
|
||||
if err := txn.Commit(); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return nil
|
||||
return txn.Commit()
|
||||
}
|
||||
|
||||
// addComputedAllocAttrs adds the computed/derived attributes to the allocation.
|
||||
|
@ -758,19 +754,18 @@ func (s *StateStore) ScalingEventsByJob(ws memdb.WatchSet, namespace, jobID stri
|
|||
return nil, 0, nil
|
||||
}
|
||||
|
||||
// UpsertNodeCtx is used to register a node or update a node definition
|
||||
// UpsertNodeMsgType is used to register a node or update a node definition
|
||||
// This is assumed to be triggered by the client, so we retain the value
|
||||
// of drain/eligibility which is set by the scheduler.
|
||||
func (s *StateStore) UpsertNodeCtx(ctx context.Context, index uint64, node *structs.Node) error {
|
||||
txn := s.db.WriteTxnCtx(ctx, index)
|
||||
func (s *StateStore) UpsertNodeMsgType(msgType structs.MessageType, index uint64, node *structs.Node) error {
|
||||
txn := s.db.WriteTxnMsgT(msgType, index)
|
||||
defer txn.Abort()
|
||||
|
||||
err := upsertNodeTxn(txn, index, node)
|
||||
if err != nil {
|
||||
return nil
|
||||
}
|
||||
txn.Commit()
|
||||
return nil
|
||||
return txn.Commit()
|
||||
}
|
||||
|
||||
// UpsertNode is used to register a node or update a node definition
|
||||
|
@ -841,16 +836,15 @@ func upsertNodeTxn(txn *txn, index uint64, node *structs.Node) error {
|
|||
}
|
||||
|
||||
// DeleteNode deregisters a batch of nodes
|
||||
func (s *StateStore) DeleteNodeCtx(ctx context.Context, index uint64, nodes []string) error {
|
||||
txn := s.db.WriteTxnCtx(ctx, index)
|
||||
func (s *StateStore) DeleteNodeMsgType(msgType structs.MessageType, index uint64, nodes []string) error {
|
||||
txn := s.db.WriteTxnMsgT(msgType, index)
|
||||
defer txn.Abort()
|
||||
|
||||
err := deleteNodeTxn(txn, index, nodes)
|
||||
if err != nil {
|
||||
return nil
|
||||
}
|
||||
txn.Commit()
|
||||
return nil
|
||||
return txn.Commit()
|
||||
}
|
||||
|
||||
// DeleteNode deregisters a batch of nodes
|
||||
|
@ -899,16 +893,15 @@ func deleteNodeTxn(txn *txn, index uint64, nodes []string) error {
|
|||
}
|
||||
|
||||
// UpdateNodeStatus is used to update the status of a node
|
||||
func (s *StateStore) UpdateNodeStatus(index uint64, nodeID, status string, updatedAt int64, event *structs.NodeEvent) error {
|
||||
txn := s.db.WriteTxn(index)
|
||||
func (s *StateStore) UpdateNodeStatus(msgType structs.MessageType, index uint64, nodeID, status string, updatedAt int64, event *structs.NodeEvent) error {
|
||||
txn := s.db.WriteTxnMsgT(msgType, index)
|
||||
defer txn.Abort()
|
||||
|
||||
if err := s.updateNodeStatusTxn(txn, nodeID, status, updatedAt, event); err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
txn.Commit()
|
||||
return nil
|
||||
return txn.Commit()
|
||||
}
|
||||
|
||||
func (s *StateStore) updateNodeStatusTxn(txn *txn, nodeID, status string, updatedAt int64, event *structs.NodeEvent) error {
|
||||
|
@ -960,16 +953,15 @@ func (s *StateStore) BatchUpdateNodeDrain(index uint64, updatedAt int64, updates
|
|||
}
|
||||
|
||||
// UpdateNodeDrain is used to update the drain of a node
|
||||
func (s *StateStore) UpdateNodeDrainCtx(ctx context.Context, index uint64, nodeID string,
|
||||
func (s *StateStore) UpdateNodeDrainMsgType(msgType structs.MessageType, index uint64, nodeID string,
|
||||
drain *structs.DrainStrategy, markEligible bool, updatedAt int64, event *structs.NodeEvent) error {
|
||||
|
||||
txn := s.db.WriteTxnCtx(ctx, index)
|
||||
txn := s.db.WriteTxnMsgT(msgType, index)
|
||||
defer txn.Abort()
|
||||
if err := s.updateNodeDrainImpl(txn, index, nodeID, drain, markEligible, updatedAt, event); err != nil {
|
||||
return err
|
||||
}
|
||||
txn.Commit()
|
||||
return nil
|
||||
return txn.Commit()
|
||||
}
|
||||
|
||||
// UpdateNodeDrain is used to update the drain of a node
|
||||
|
@ -1075,8 +1067,8 @@ func (s *StateStore) UpdateNodeEligibility(index uint64, nodeID string, eligibil
|
|||
return nil
|
||||
}
|
||||
|
||||
func (s *StateStore) UpsertNodeEventsCtx(ctx context.Context, index uint64, nodeEvents map[string][]*structs.NodeEvent) error {
|
||||
txn := s.db.WriteTxnCtx(ctx, index)
|
||||
func (s *StateStore) UpsertNodeEventsMsgType(msgType structs.MessageType, index uint64, nodeEvents map[string][]*structs.NodeEvent) error {
|
||||
txn := s.db.WriteTxnMsgT(msgType, index)
|
||||
defer txn.Abort()
|
||||
|
||||
for nodeID, events := range nodeEvents {
|
||||
|
@ -1085,8 +1077,7 @@ func (s *StateStore) UpsertNodeEventsCtx(ctx context.Context, index uint64, node
|
|||
}
|
||||
}
|
||||
|
||||
txn.Commit()
|
||||
return nil
|
||||
return txn.Commit()
|
||||
}
|
||||
|
||||
// UpsertNodeEvents adds the node events to the nodes, rotating events as
|
||||
|
@ -1492,6 +1483,16 @@ func (s *StateStore) UpsertJob(index uint64, job *structs.Job) error {
|
|||
return nil
|
||||
}
|
||||
|
||||
// UpsertJob is used to register a job or update a job definition
|
||||
func (s *StateStore) UpsertJobMsgType(msgType structs.MessageType, index uint64, job *structs.Job) error {
|
||||
txn := s.db.WriteTxnMsgT(msgType, index)
|
||||
defer txn.Abort()
|
||||
if err := s.upsertJobImpl(index, job, false, txn); err != nil {
|
||||
return err
|
||||
}
|
||||
return txn.Commit()
|
||||
}
|
||||
|
||||
// UpsertJobTxn is used to register a job or update a job definition, like UpsertJob,
|
||||
// but in a transaction. Useful for when making multiple modifications atomically
|
||||
func (s *StateStore) UpsertJobTxn(index uint64, job *structs.Job, txn Txn) error {
|
||||
|
@ -2706,13 +2707,13 @@ func (s *StateStore) UpsertEvals(index uint64, evals []*structs.Evaluation) erro
|
|||
}
|
||||
|
||||
// UpsertEvals is used to upsert a set of evaluations
|
||||
func (s *StateStore) UpsertEvalsCtx(ctx context.Context, index uint64, evals []*structs.Evaluation) error {
|
||||
txn := s.db.WriteTxnCtx(ctx, index)
|
||||
func (s *StateStore) UpsertEvalsMsgType(msgType structs.MessageType, index uint64, evals []*structs.Evaluation) error {
|
||||
txn := s.db.WriteTxnMsgT(msgType, index)
|
||||
defer txn.Abort()
|
||||
|
||||
err := s.UpsertEvalsTxn(index, evals, txn)
|
||||
if err == nil {
|
||||
txn.Commit()
|
||||
return txn.Commit()
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
@ -3029,8 +3030,8 @@ func (s *StateStore) EvalsByNamespace(ws memdb.WatchSet, namespace string) (memd
|
|||
// most things, some updates are authoritative from the client. Specifically,
|
||||
// the desired state comes from the schedulers, while the actual state comes
|
||||
// from clients.
|
||||
func (s *StateStore) UpdateAllocsFromClient(ctx context.Context, index uint64, allocs []*structs.Allocation) error {
|
||||
txn := s.db.WriteTxnCtx(ctx, index)
|
||||
func (s *StateStore) UpdateAllocsFromClient(msgType structs.MessageType, index uint64, allocs []*structs.Allocation) error {
|
||||
txn := s.db.WriteTxnMsgT(msgType, index)
|
||||
defer txn.Abort()
|
||||
|
||||
// Handle each of the updated allocations
|
||||
|
@ -3045,8 +3046,7 @@ func (s *StateStore) UpdateAllocsFromClient(ctx context.Context, index uint64, a
|
|||
return fmt.Errorf("index update failed: %v", err)
|
||||
}
|
||||
|
||||
txn.Commit()
|
||||
return nil
|
||||
return txn.Commit()
|
||||
}
|
||||
|
||||
// nestedUpdateAllocFromClient is used to nest an update of an allocation with client status
|
||||
|
@ -3149,6 +3149,17 @@ func (s *StateStore) UpsertAllocs(index uint64, allocs []*structs.Allocation) er
|
|||
return nil
|
||||
}
|
||||
|
||||
// UpsertAllocsMsgType is used to evict a set of allocations and allocate new ones at
|
||||
// the same time.
|
||||
func (s *StateStore) UpsertAllocsMsgType(msgType structs.MessageType, index uint64, allocs []*structs.Allocation) error {
|
||||
txn := s.db.WriteTxnMsgT(msgType, index)
|
||||
defer txn.Abort()
|
||||
if err := s.upsertAllocsImpl(index, allocs, txn); err != nil {
|
||||
return err
|
||||
}
|
||||
return txn.Commit()
|
||||
}
|
||||
|
||||
// upsertAllocs is the actual implementation of UpsertAllocs so that it may be
|
||||
// used with an existing transaction.
|
||||
func (s *StateStore) upsertAllocsImpl(index uint64, allocs []*structs.Allocation, txn *txn) error {
|
||||
|
@ -3842,8 +3853,8 @@ func (s *StateStore) SITokenAccessorsByNode(ws memdb.WatchSet, nodeID string) ([
|
|||
|
||||
// UpdateDeploymentStatus is used to make deployment status updates and
|
||||
// potentially make a evaluation
|
||||
func (s *StateStore) UpdateDeploymentStatus(ctx context.Context, index uint64, req *structs.DeploymentStatusUpdateRequest) error {
|
||||
txn := s.db.WriteTxnCtx(ctx, index)
|
||||
func (s *StateStore) UpdateDeploymentStatus(msgType structs.MessageType, index uint64, req *structs.DeploymentStatusUpdateRequest) error {
|
||||
txn := s.db.WriteTxnMsgT(msgType, index)
|
||||
defer txn.Abort()
|
||||
|
||||
if err := s.updateDeploymentStatusImpl(index, req.DeploymentUpdate, txn); err != nil {
|
||||
|
@ -3864,8 +3875,7 @@ func (s *StateStore) UpdateDeploymentStatus(ctx context.Context, index uint64, r
|
|||
}
|
||||
}
|
||||
|
||||
txn.Commit()
|
||||
return nil
|
||||
return txn.Commit()
|
||||
}
|
||||
|
||||
// updateDeploymentStatusImpl is used to make deployment status updates
|
||||
|
@ -3946,8 +3956,8 @@ func (s *StateStore) updateJobStabilityImpl(index uint64, namespace, jobID strin
|
|||
|
||||
// UpdateDeploymentPromotion is used to promote canaries in a deployment and
|
||||
// potentially make a evaluation
|
||||
func (s *StateStore) UpdateDeploymentPromotion(ctx context.Context, index uint64, req *structs.ApplyDeploymentPromoteRequest) error {
|
||||
txn := s.db.WriteTxnCtx(ctx, index)
|
||||
func (s *StateStore) UpdateDeploymentPromotion(msgType structs.MessageType, index uint64, req *structs.ApplyDeploymentPromoteRequest) error {
|
||||
txn := s.db.WriteTxnMsgT(msgType, index)
|
||||
defer txn.Abort()
|
||||
|
||||
// Retrieve deployment and ensure it is not terminal and is active
|
||||
|
@ -4083,14 +4093,13 @@ func (s *StateStore) UpdateDeploymentPromotion(ctx context.Context, index uint64
|
|||
return fmt.Errorf("index update failed: %v", err)
|
||||
}
|
||||
|
||||
txn.Commit()
|
||||
return nil
|
||||
return txn.Commit()
|
||||
}
|
||||
|
||||
// UpdateDeploymentAllocHealth is used to update the health of allocations as
|
||||
// part of the deployment and potentially make a evaluation
|
||||
func (s *StateStore) UpdateDeploymentAllocHealth(ctx context.Context, index uint64, req *structs.ApplyDeploymentAllocHealthRequest) error {
|
||||
txn := s.db.WriteTxnCtx(ctx, index)
|
||||
func (s *StateStore) UpdateDeploymentAllocHealth(msgType structs.MessageType, index uint64, req *structs.ApplyDeploymentAllocHealthRequest) error {
|
||||
txn := s.db.WriteTxnMsgT(msgType, index)
|
||||
defer txn.Abort()
|
||||
|
||||
// Retrieve deployment and ensure it is not terminal and is active
|
||||
|
@ -4179,8 +4188,7 @@ func (s *StateStore) UpdateDeploymentAllocHealth(ctx context.Context, index uint
|
|||
}
|
||||
}
|
||||
|
||||
txn.Commit()
|
||||
return nil
|
||||
return txn.Commit()
|
||||
}
|
||||
|
||||
// LastIndex returns the greatest index value for all indexes
|
||||
|
@ -5407,13 +5415,13 @@ func (s *StateStore) ClusterSetMetadata(index uint64, meta *structs.ClusterMetad
|
|||
// WithWriteTransaction executes the passed function within a write transaction,
|
||||
// and returns its result. If the invocation returns no error, the transaction
|
||||
// is committed; otherwise, it's aborted.
|
||||
func (s *StateStore) WithWriteTransaction(index uint64, fn func(Txn) error) error {
|
||||
tx := s.db.WriteTxn(index)
|
||||
func (s *StateStore) WithWriteTransaction(msgType structs.MessageType, index uint64, fn func(Txn) error) error {
|
||||
tx := s.db.WriteTxnMsgT(msgType, index)
|
||||
defer tx.Abort()
|
||||
|
||||
err := fn(tx)
|
||||
if err == nil {
|
||||
tx.Commit()
|
||||
return tx.Commit()
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
|
|
@ -127,7 +127,7 @@ func TestStateStore_UpsertPlanResults_AllocationsCreated_Denormalized(t *testing
|
|||
EvalID: eval.ID,
|
||||
}
|
||||
assert := assert.New(t)
|
||||
err := state.UpsertPlanResults(context.Background(), 1000, &res)
|
||||
err := state.UpsertPlanResults(structs.MsgTypeTestSetup, 1000, &res)
|
||||
assert.Nil(err)
|
||||
|
||||
ws := memdb.NewWatchSet()
|
||||
|
@ -203,7 +203,7 @@ func TestStateStore_UpsertPlanResults_AllocationsDenormalized(t *testing.T) {
|
|||
}
|
||||
assert := assert.New(t)
|
||||
planModifyIndex := uint64(1000)
|
||||
err := state.UpsertPlanResults(context.Background(), planModifyIndex, &res)
|
||||
err := state.UpsertPlanResults(structs.MsgTypeTestSetup, planModifyIndex, &res)
|
||||
require.NoError(err)
|
||||
|
||||
ws := memdb.NewWatchSet()
|
||||
|
@ -284,7 +284,7 @@ func TestStateStore_UpsertPlanResults_Deployment(t *testing.T) {
|
|||
EvalID: eval.ID,
|
||||
}
|
||||
|
||||
err := state.UpsertPlanResults(context.Background(), 1000, &res)
|
||||
err := state.UpsertPlanResults(structs.MsgTypeTestSetup, 1000, &res)
|
||||
if err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
|
@ -332,7 +332,7 @@ func TestStateStore_UpsertPlanResults_Deployment(t *testing.T) {
|
|||
EvalID: eval.ID,
|
||||
}
|
||||
|
||||
err = state.UpsertPlanResults(context.Background(), 1001, &res)
|
||||
err = state.UpsertPlanResults(structs.MsgTypeTestSetup, 1001, &res)
|
||||
if err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
|
@ -400,7 +400,7 @@ func TestStateStore_UpsertPlanResults_PreemptedAllocs(t *testing.T) {
|
|||
PreemptionEvals: []*structs.Evaluation{eval2},
|
||||
}
|
||||
|
||||
err = state.UpsertPlanResults(context.Background(), 1000, &res)
|
||||
err = state.UpsertPlanResults(structs.MsgTypeTestSetup, 1000, &res)
|
||||
require.NoError(err)
|
||||
|
||||
ws := memdb.NewWatchSet()
|
||||
|
@ -486,7 +486,7 @@ func TestStateStore_UpsertPlanResults_DeploymentUpdates(t *testing.T) {
|
|||
EvalID: eval.ID,
|
||||
}
|
||||
|
||||
err := state.UpsertPlanResults(context.Background(), 1000, &res)
|
||||
err := state.UpsertPlanResults(structs.MsgTypeTestSetup, 1000, &res)
|
||||
if err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
|
@ -898,7 +898,7 @@ func TestStateStore_UpdateNodeStatus_Node(t *testing.T) {
|
|||
Timestamp: time.Now(),
|
||||
}
|
||||
|
||||
require.NoError(state.UpdateNodeStatus(801, node.ID, structs.NodeStatusReady, 70, event))
|
||||
require.NoError(state.UpdateNodeStatus(structs.MsgTypeTestSetup, 801, node.ID, structs.NodeStatusReady, 70, event))
|
||||
require.True(watchFired(ws))
|
||||
|
||||
ws = memdb.NewWatchSet()
|
||||
|
@ -1903,7 +1903,7 @@ func TestStateStore_DeleteJobTxn_BatchDeletes(t *testing.T) {
|
|||
|
||||
// Actually delete
|
||||
const deletionIndex = uint64(10001)
|
||||
err = state.WithWriteTransaction(deletionIndex, func(txn Txn) error {
|
||||
err = state.WithWriteTransaction(structs.MsgTypeTestSetup, deletionIndex, func(txn Txn) error {
|
||||
for i, job := range jobs {
|
||||
err := state.DeleteJobTxn(deletionIndex, job.Namespace, job.ID, txn)
|
||||
require.NoError(t, err, "failed at %d %e", i, err)
|
||||
|
@ -4467,7 +4467,7 @@ func TestStateStore_UpdateAllocsFromClient(t *testing.T) {
|
|||
JobID: alloc.JobID,
|
||||
TaskGroup: alloc.TaskGroup,
|
||||
}
|
||||
err = state.UpdateAllocsFromClient(context.Background(), 1001, []*structs.Allocation{update})
|
||||
err = state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1001, []*structs.Allocation{update})
|
||||
if err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
|
@ -4565,7 +4565,7 @@ func TestStateStore_UpdateAllocsFromClient_ChildJob(t *testing.T) {
|
|||
TaskGroup: alloc2.TaskGroup,
|
||||
}
|
||||
|
||||
err = state.UpdateAllocsFromClient(context.Background(), 1001, []*structs.Allocation{update, update2})
|
||||
err = state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1001, []*structs.Allocation{update, update2})
|
||||
if err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
|
@ -4666,7 +4666,7 @@ func TestStateStore_UpdateMultipleAllocsFromClient(t *testing.T) {
|
|||
TaskGroup: alloc.TaskGroup,
|
||||
}
|
||||
|
||||
err = state.UpdateAllocsFromClient(context.Background(), 1001, []*structs.Allocation{update, update2})
|
||||
err = state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1001, []*structs.Allocation{update, update2})
|
||||
if err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
|
@ -4735,7 +4735,7 @@ func TestStateStore_UpdateAllocsFromClient_Deployment(t *testing.T) {
|
|||
Timestamp: healthy,
|
||||
},
|
||||
}
|
||||
require.Nil(state.UpdateAllocsFromClient(context.Background(), 1001, []*structs.Allocation{update}))
|
||||
require.Nil(state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1001, []*structs.Allocation{update}))
|
||||
|
||||
// Check that the deployment state was updated because the healthy
|
||||
// deployment
|
||||
|
@ -4780,7 +4780,7 @@ func TestStateStore_UpdateAllocsFromClient_DeploymentStateMerges(t *testing.T) {
|
|||
Canary: false,
|
||||
},
|
||||
}
|
||||
require.Nil(state.UpdateAllocsFromClient(context.Background(), 1001, []*structs.Allocation{update}))
|
||||
require.Nil(state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1001, []*structs.Allocation{update}))
|
||||
|
||||
// Check that the merging of the deployment status was correct
|
||||
out, err := state.AllocByID(nil, alloc.ID)
|
||||
|
@ -5161,7 +5161,7 @@ func TestStateStore_UpdateAlloc_NoJob(t *testing.T) {
|
|||
// Update the client state of the allocation to complete
|
||||
allocCopy1 := allocCopy.Copy()
|
||||
allocCopy1.ClientStatus = structs.AllocClientStatusComplete
|
||||
if err := state.UpdateAllocsFromClient(context.Background(), 1003, []*structs.Allocation{allocCopy1}); err != nil {
|
||||
if err := state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1003, []*structs.Allocation{allocCopy1}); err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
|
||||
|
@ -5272,12 +5272,12 @@ func TestStateStore_JobSummary(t *testing.T) {
|
|||
alloc1 := alloc.Copy()
|
||||
alloc1.ClientStatus = structs.AllocClientStatusPending
|
||||
alloc1.DesiredStatus = ""
|
||||
state.UpdateAllocsFromClient(context.Background(), 920, []*structs.Allocation{alloc})
|
||||
state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 920, []*structs.Allocation{alloc})
|
||||
|
||||
alloc3 := alloc.Copy()
|
||||
alloc3.ClientStatus = structs.AllocClientStatusRunning
|
||||
alloc3.DesiredStatus = ""
|
||||
state.UpdateAllocsFromClient(context.Background(), 930, []*structs.Allocation{alloc3})
|
||||
state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 930, []*structs.Allocation{alloc3})
|
||||
|
||||
// Upsert the alloc
|
||||
alloc4 := alloc.Copy()
|
||||
|
@ -5320,7 +5320,7 @@ func TestStateStore_JobSummary(t *testing.T) {
|
|||
alloc6 := alloc.Copy()
|
||||
alloc6.ClientStatus = structs.AllocClientStatusRunning
|
||||
alloc6.DesiredStatus = ""
|
||||
state.UpdateAllocsFromClient(context.Background(), 990, []*structs.Allocation{alloc6})
|
||||
state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 990, []*structs.Allocation{alloc6})
|
||||
|
||||
// We shouldn't have any summary at this point
|
||||
summary, _ = state.JobSummaryByID(ws, job.Namespace, job.ID)
|
||||
|
@ -5347,7 +5347,7 @@ func TestStateStore_JobSummary(t *testing.T) {
|
|||
alloc7.Job = outJob
|
||||
alloc7.ClientStatus = structs.AllocClientStatusComplete
|
||||
alloc7.DesiredStatus = structs.AllocDesiredStatusRun
|
||||
state.UpdateAllocsFromClient(context.Background(), 1020, []*structs.Allocation{alloc7})
|
||||
state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1020, []*structs.Allocation{alloc7})
|
||||
|
||||
expectedSummary = structs.JobSummary{
|
||||
JobID: job.ID,
|
||||
|
@ -5392,7 +5392,7 @@ func TestStateStore_ReconcileJobSummary(t *testing.T) {
|
|||
// Change the state of the first alloc to running
|
||||
alloc3 := alloc.Copy()
|
||||
alloc3.ClientStatus = structs.AllocClientStatusRunning
|
||||
state.UpdateAllocsFromClient(context.Background(), 120, []*structs.Allocation{alloc3})
|
||||
state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 120, []*structs.Allocation{alloc3})
|
||||
|
||||
//Add some more allocs to the second tg
|
||||
alloc4 := mock.Alloc()
|
||||
|
@ -5425,7 +5425,7 @@ func TestStateStore_ReconcileJobSummary(t *testing.T) {
|
|||
|
||||
state.UpsertAllocs(130, []*structs.Allocation{alloc4, alloc6, alloc8, alloc10})
|
||||
|
||||
state.UpdateAllocsFromClient(context.Background(), 150, []*structs.Allocation{alloc5, alloc7, alloc9, alloc11})
|
||||
state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 150, []*structs.Allocation{alloc5, alloc7, alloc9, alloc11})
|
||||
|
||||
// DeleteJobSummary is a helper method and doesn't modify the indexes table
|
||||
state.DeleteJobSummary(130, alloc.Namespace, alloc.Job.ID)
|
||||
|
@ -5564,7 +5564,7 @@ func TestStateStore_UpdateAlloc_JobNotPresent(t *testing.T) {
|
|||
alloc1.ClientStatus = structs.AllocClientStatusRunning
|
||||
|
||||
// Updating allocation should not throw any error
|
||||
if err := state.UpdateAllocsFromClient(context.Background(), 400, []*structs.Allocation{alloc1}); err != nil {
|
||||
if err := state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 400, []*structs.Allocation{alloc1}); err != nil {
|
||||
t.Fatalf("expect err: %v", err)
|
||||
}
|
||||
|
||||
|
@ -5574,7 +5574,7 @@ func TestStateStore_UpdateAlloc_JobNotPresent(t *testing.T) {
|
|||
// Update the alloc again
|
||||
alloc2 := alloc.Copy()
|
||||
alloc2.ClientStatus = structs.AllocClientStatusComplete
|
||||
if err := state.UpdateAllocsFromClient(context.Background(), 400, []*structs.Allocation{alloc1}); err != nil {
|
||||
if err := state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 400, []*structs.Allocation{alloc1}); err != nil {
|
||||
t.Fatalf("expect err: %v", err)
|
||||
}
|
||||
|
||||
|
@ -6484,7 +6484,7 @@ func TestStateJobSummary_UpdateJobCount(t *testing.T) {
|
|||
alloc5.JobID = alloc3.JobID
|
||||
alloc5.ClientStatus = structs.AllocClientStatusComplete
|
||||
|
||||
if err := state.UpdateAllocsFromClient(context.Background(), 1004, []*structs.Allocation{alloc4, alloc5}); err != nil {
|
||||
if err := state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1004, []*structs.Allocation{alloc4, alloc5}); err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
|
||||
|
@ -6561,7 +6561,7 @@ func TestJobSummary_UpdateClientStatus(t *testing.T) {
|
|||
alloc6.JobID = alloc.JobID
|
||||
alloc6.ClientStatus = structs.AllocClientStatusRunning
|
||||
|
||||
if err := state.UpdateAllocsFromClient(context.Background(), 1002, []*structs.Allocation{alloc4, alloc5, alloc6}); err != nil {
|
||||
if err := state.UpdateAllocsFromClient(structs.MsgTypeTestSetup, 1002, []*structs.Allocation{alloc4, alloc5, alloc6}); err != nil {
|
||||
t.Fatalf("err: %v", err)
|
||||
}
|
||||
|
||||
|
@ -6600,7 +6600,7 @@ func TestStateStore_UpsertDeploymentStatusUpdate_Nonexistent(t *testing.T) {
|
|||
Status: structs.DeploymentStatusRunning,
|
||||
},
|
||||
}
|
||||
err := state.UpdateDeploymentStatus(context.Background(), 2, req)
|
||||
err := state.UpdateDeploymentStatus(structs.MsgTypeTestSetup, 2, req)
|
||||
if err == nil || !strings.Contains(err.Error(), "does not exist") {
|
||||
t.Fatalf("expected error updating the status because the deployment doesn't exist")
|
||||
}
|
||||
|
@ -6627,7 +6627,7 @@ func TestStateStore_UpsertDeploymentStatusUpdate_Terminal(t *testing.T) {
|
|||
Status: structs.DeploymentStatusRunning,
|
||||
},
|
||||
}
|
||||
err := state.UpdateDeploymentStatus(context.Background(), 2, req)
|
||||
err := state.UpdateDeploymentStatus(structs.MsgTypeTestSetup, 2, req)
|
||||
if err == nil || !strings.Contains(err.Error(), "has terminal status") {
|
||||
t.Fatalf("expected error updating the status because the deployment is terminal")
|
||||
}
|
||||
|
@ -6661,7 +6661,7 @@ func TestStateStore_UpsertDeploymentStatusUpdate_NonTerminal(t *testing.T) {
|
|||
Job: j,
|
||||
Eval: e,
|
||||
}
|
||||
err := state.UpdateDeploymentStatus(context.Background(), 2, req)
|
||||
err := state.UpdateDeploymentStatus(structs.MsgTypeTestSetup, 2, req)
|
||||
if err != nil {
|
||||
t.Fatalf("bad: %v", err)
|
||||
}
|
||||
|
@ -6722,7 +6722,7 @@ func TestStateStore_UpsertDeploymentStatusUpdate_Successful(t *testing.T) {
|
|||
StatusDescription: structs.DeploymentStatusDescriptionSuccessful,
|
||||
},
|
||||
}
|
||||
err := state.UpdateDeploymentStatus(context.Background(), 3, req)
|
||||
err := state.UpdateDeploymentStatus(structs.MsgTypeTestSetup, 3, req)
|
||||
if err != nil {
|
||||
t.Fatalf("bad: %v", err)
|
||||
}
|
||||
|
@ -6820,7 +6820,7 @@ func TestStateStore_UpsertDeploymentPromotion_Nonexistent(t *testing.T) {
|
|||
All: true,
|
||||
},
|
||||
}
|
||||
err := state.UpdateDeploymentPromotion(context.Background(), 2, req)
|
||||
err := state.UpdateDeploymentPromotion(structs.MsgTypeTestSetup, 2, req)
|
||||
if err == nil || !strings.Contains(err.Error(), "does not exist") {
|
||||
t.Fatalf("expected error promoting because the deployment doesn't exist")
|
||||
}
|
||||
|
@ -6847,7 +6847,7 @@ func TestStateStore_UpsertDeploymentPromotion_Terminal(t *testing.T) {
|
|||
All: true,
|
||||
},
|
||||
}
|
||||
err := state.UpdateDeploymentPromotion(context.Background(), 2, req)
|
||||
err := state.UpdateDeploymentPromotion(structs.MsgTypeTestSetup, 2, req)
|
||||
if err == nil || !strings.Contains(err.Error(), "has terminal status") {
|
||||
t.Fatalf("expected error updating the status because the deployment is terminal: %v", err)
|
||||
}
|
||||
|
@ -6897,7 +6897,7 @@ func TestStateStore_UpsertDeploymentPromotion_Unhealthy(t *testing.T) {
|
|||
All: true,
|
||||
},
|
||||
}
|
||||
err := state.UpdateDeploymentPromotion(context.Background(), 4, req)
|
||||
err := state.UpdateDeploymentPromotion(structs.MsgTypeTestSetup, 4, req)
|
||||
require.NotNil(err)
|
||||
require.Contains(err.Error(), `Task group "web" has 0/2 healthy allocations`)
|
||||
}
|
||||
|
@ -6926,7 +6926,7 @@ func TestStateStore_UpsertDeploymentPromotion_NoCanaries(t *testing.T) {
|
|||
All: true,
|
||||
},
|
||||
}
|
||||
err := state.UpdateDeploymentPromotion(context.Background(), 4, req)
|
||||
err := state.UpdateDeploymentPromotion(structs.MsgTypeTestSetup, 4, req)
|
||||
require.NotNil(err)
|
||||
require.Contains(err.Error(), `Task group "web" has 0/2 healthy allocations`)
|
||||
}
|
||||
|
@ -6997,7 +6997,7 @@ func TestStateStore_UpsertDeploymentPromotion_All(t *testing.T) {
|
|||
},
|
||||
Eval: e,
|
||||
}
|
||||
err := state.UpdateDeploymentPromotion(context.Background(), 4, req)
|
||||
err := state.UpdateDeploymentPromotion(structs.MsgTypeTestSetup, 4, req)
|
||||
if err != nil {
|
||||
t.Fatalf("bad: %v", err)
|
||||
}
|
||||
|
@ -7103,7 +7103,7 @@ func TestStateStore_UpsertDeploymentPromotion_Subset(t *testing.T) {
|
|||
},
|
||||
Eval: e,
|
||||
}
|
||||
require.Nil(state.UpdateDeploymentPromotion(context.Background(), 4, req))
|
||||
require.Nil(state.UpdateDeploymentPromotion(structs.MsgTypeTestSetup, 4, req))
|
||||
|
||||
// Check that the status per task group was updated properly
|
||||
ws := memdb.NewWatchSet()
|
||||
|
@ -7146,7 +7146,7 @@ func TestStateStore_UpsertDeploymentAllocHealth_Nonexistent(t *testing.T) {
|
|||
HealthyAllocationIDs: []string{uuid.Generate()},
|
||||
},
|
||||
}
|
||||
err := state.UpdateDeploymentAllocHealth(context.Background(), 2, req)
|
||||
err := state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, 2, req)
|
||||
if err == nil || !strings.Contains(err.Error(), "does not exist") {
|
||||
t.Fatalf("expected error because the deployment doesn't exist: %v", err)
|
||||
}
|
||||
|
@ -7173,7 +7173,7 @@ func TestStateStore_UpsertDeploymentAllocHealth_Terminal(t *testing.T) {
|
|||
HealthyAllocationIDs: []string{uuid.Generate()},
|
||||
},
|
||||
}
|
||||
err := state.UpdateDeploymentAllocHealth(context.Background(), 2, req)
|
||||
err := state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, 2, req)
|
||||
if err == nil || !strings.Contains(err.Error(), "has terminal status") {
|
||||
t.Fatalf("expected error because the deployment is terminal: %v", err)
|
||||
}
|
||||
|
@ -7198,7 +7198,7 @@ func TestStateStore_UpsertDeploymentAllocHealth_BadAlloc_Nonexistent(t *testing.
|
|||
HealthyAllocationIDs: []string{uuid.Generate()},
|
||||
},
|
||||
}
|
||||
err := state.UpdateDeploymentAllocHealth(context.Background(), 2, req)
|
||||
err := state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, 2, req)
|
||||
if err == nil || !strings.Contains(err.Error(), "unknown alloc") {
|
||||
t.Fatalf("expected error because the alloc doesn't exist: %v", err)
|
||||
}
|
||||
|
@ -7338,7 +7338,7 @@ func TestStateStore_UpsertDeploymentAllocHealth_BadAlloc_MismatchDeployment(t *t
|
|||
HealthyAllocationIDs: []string{a.ID},
|
||||
},
|
||||
}
|
||||
err := state.UpdateDeploymentAllocHealth(context.Background(), 4, req)
|
||||
err := state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, 4, req)
|
||||
if err == nil || !strings.Contains(err.Error(), "not part of deployment") {
|
||||
t.Fatalf("expected error because the alloc isn't part of the deployment: %v", err)
|
||||
}
|
||||
|
@ -7395,7 +7395,7 @@ func TestStateStore_UpsertDeploymentAllocHealth(t *testing.T) {
|
|||
DeploymentUpdate: u,
|
||||
Timestamp: ts,
|
||||
}
|
||||
err := state.UpdateDeploymentAllocHealth(context.Background(), 3, req)
|
||||
err := state.UpdateDeploymentAllocHealth(structs.MsgTypeTestSetup, 3, req)
|
||||
if err != nil {
|
||||
t.Fatalf("bad: %v", err)
|
||||
}
|
||||
|
|
|
@ -107,6 +107,10 @@ const (
|
|||
// old servers to crash when the FSM attempts to process them.
|
||||
IgnoreUnknownTypeFlag MessageType = 128
|
||||
|
||||
// MsgTypeTestSetup is used during testing when calling state store
|
||||
// methods directly that require an FSM MessageType
|
||||
MsgTypeTestSetup MessageType = IgnoreUnknownTypeFlag
|
||||
|
||||
// ApiMajorVersion is returned as part of the Status.Version request.
|
||||
// It should be incremented anytime the APIs are changed in a way
|
||||
// that would break clients for sane client versioning.
|
||||
|
|
|
@ -2958,7 +2958,7 @@ func TestServiceSched_NodeUpdate(t *testing.T) {
|
|||
for i := 0; i < 4; i++ {
|
||||
out, _ := h.State.AllocByID(ws, allocs[i].ID)
|
||||
out.ClientStatus = structs.AllocClientStatusRunning
|
||||
require.NoError(t, h.State.UpdateAllocsFromClient(context.Background(), h.NextIndex(), []*structs.Allocation{out}))
|
||||
require.NoError(t, h.State.UpdateAllocsFromClient(structs.MsgTypeTestSetup, h.NextIndex(), []*structs.Allocation{out}))
|
||||
}
|
||||
|
||||
// Create a mock evaluation which won't trigger any new placements
|
||||
|
|
|
@ -1,7 +1,6 @@
|
|||
package scheduler
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"reflect"
|
||||
"sort"
|
||||
|
@ -133,7 +132,7 @@ func TestSystemSched_JobRegister_StickyAllocs(t *testing.T) {
|
|||
// Get an allocation and mark it as failed
|
||||
alloc := planned[4].Copy()
|
||||
alloc.ClientStatus = structs.AllocClientStatusFailed
|
||||
require.NoError(t, h.State.UpdateAllocsFromClient(context.Background(), h.NextIndex(), []*structs.Allocation{alloc}))
|
||||
require.NoError(t, h.State.UpdateAllocsFromClient(structs.MsgTypeTestSetup, h.NextIndex(), []*structs.Allocation{alloc}))
|
||||
|
||||
// Create a mock evaluation to handle the update
|
||||
eval = &structs.Evaluation{
|
||||
|
|
|
@ -1,7 +1,6 @@
|
|||
package scheduler
|
||||
|
||||
import (
|
||||
"context"
|
||||
"fmt"
|
||||
"sync"
|
||||
"time"
|
||||
|
@ -171,7 +170,7 @@ func (h *Harness) SubmitPlan(plan *structs.Plan) (*structs.PlanResult, State, er
|
|||
}
|
||||
|
||||
// Apply the full plan
|
||||
err := h.State.UpsertPlanResults(context.Background(), index, &req)
|
||||
err := h.State.UpsertPlanResults(structs.MsgTypeTestSetup, index, &req)
|
||||
return result, nil, err
|
||||
}
|
||||
|
||||
|
|
Loading…
Reference in New Issue