├── .gitignore ├── registry ├── init.go ├── dependency.go ├── raw.go ├── mock_dependency_for_test.go ├── job.go ├── doc.go ├── registry.go ├── formats.go └── job_test.go ├── management ├── doc.go ├── util.go ├── output.go ├── filter_test.go ├── mongodb_test.go └── interface.go ├── .golangci.yml ├── cli ├── cli_test.go ├── cli.go └── abortable_pool_management.go ├── .github └── dependabot.yml ├── retry.go ├── queue ├── doc.go ├── util.go ├── util_for_test.go ├── driver.go ├── limited_test.go ├── scope.go ├── scope_test.go ├── group_local.go └── group_util_test.go ├── dependency ├── state_string.go ├── mock.go ├── state.go ├── always.go ├── state_test.go ├── creates.go ├── manager.go ├── always_test.go ├── callback.go ├── edges_test.go ├── creates_test.go ├── edges.go ├── registry.go ├── local.go └── local_test.go ├── job ├── counter.go ├── counter_test.go ├── init.go ├── shell.go ├── shell_test.go ├── group_test.go └── group.go ├── pool ├── doc.go ├── noop.go ├── local.go ├── runner_test.go ├── local_test.go └── mock_queue_test.go ├── scripts └── waitForMongo.js ├── rest ├── service_queue_fetch.go ├── service_queue_test.go ├── service_queue_stats_test.go ├── service_queue_stats.go ├── service_queue_create.go ├── service_queue_job_test.go ├── service_abortable_pool.go ├── client_abortable_pool.go ├── service_queue_job.go ├── service_group_management.go ├── service_queue_create_test.go ├── service_queue.go └── service_management.go ├── formats.go ├── stats_test.go ├── logger ├── doc.go ├── job_single.go ├── job_test.go ├── sender_single.go └── sender_test.go ├── periodic_test.go ├── cmd ├── verify-mod-tidy │ └── verify-mod-tidy.go └── run-linter │ └── run-linter.go ├── errors_test.go ├── meta.go ├── periodic_group.go ├── stats.go ├── doc.go ├── go.mod └── wait.go /.gitignore: -------------------------------------------------------------------------------- 1 | build 2 | \#*\# 3 | *~ 4 | *.out 5 | -------------------------------------------------------------------------------- /registry/init.go: -------------------------------------------------------------------------------- 1 | package registry 2 | 3 | // private initialization of the package-level registries. 4 | -------------------------------------------------------------------------------- /management/doc.go: -------------------------------------------------------------------------------- 1 | // Package management provides increased observability and control of the state 2 | // of amboy queues. 3 | package management 4 | 5 | // this file is intentionally blank. 6 | -------------------------------------------------------------------------------- /management/util.go: -------------------------------------------------------------------------------- 1 | package management 2 | 3 | func addJobsSuffix(s string) string { 4 | return s + ".jobs" 5 | } 6 | 7 | func addGroupSuffix(s string) string { 8 | return s + ".group" 9 | } 10 | -------------------------------------------------------------------------------- /.golangci.yml: -------------------------------------------------------------------------------- 1 | --- 2 | linters: 3 | disable-all: true 4 | enable: 5 | - errcheck 6 | - gofmt 7 | - goimports 8 | - govet 9 | - ineffassign 10 | - misspell 11 | - unconvert 12 | -------------------------------------------------------------------------------- /cli/cli_test.go: -------------------------------------------------------------------------------- 1 | package cli 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/stretchr/testify/assert" 7 | ) 8 | 9 | func TestCLI(t *testing.T) { 10 | opts := &ServiceOptions{} 11 | cmd := Amboy(opts) 12 | assert.True(t, cmd.HasName("amboy")) 13 | assert.False(t, cmd.HasName("cli")) 14 | } 15 | -------------------------------------------------------------------------------- /.github/dependabot.yml: -------------------------------------------------------------------------------- 1 | --- 2 | version: 2 3 | updates: 4 | - package-ecosystem: gomod 5 | directory: "/" 6 | schedule: 7 | interval: monthly 8 | time: "06:00" 9 | timezone: "America/New_York" 10 | open-pull-requests-limit: 99 11 | commit-message: 12 | prefix: "CHORE: " 13 | reviewers: 14 | - evg-plt 15 | -------------------------------------------------------------------------------- /retry.go: -------------------------------------------------------------------------------- 1 | package amboy 2 | 3 | // WithRetryableQueue is a convenience function to perform an operation if the 4 | // Queue is a RetryableQueue; otherwise, it is a no-op. Returns whether ot not 5 | // the queue was a RetryableQueue. 6 | func WithRetryableQueue(q Queue, op func(RetryableQueue)) bool { 7 | rq, ok := q.(RetryableQueue) 8 | if !ok { 9 | return false 10 | } 11 | 12 | op(rq) 13 | 14 | return true 15 | } 16 | -------------------------------------------------------------------------------- /queue/doc.go: -------------------------------------------------------------------------------- 1 | /* 2 | Package queue provides several implementations of the amboy.Queue and 3 | amboy.RemoteQueue interfaces capable of processing amboy.Job implementations. 4 | It also provides implementations of ancillary queue components, such as the 5 | ScopeManager for supporting scopes and the amboy.RetryHandler for supporting 6 | retryable queues. 7 | */ 8 | package queue 9 | 10 | // This file is intentionally documentation only. 11 | -------------------------------------------------------------------------------- /dependency/state_string.go: -------------------------------------------------------------------------------- 1 | // Code generated by "stringer -type=State"; DO NOT EDIT 2 | 3 | package dependency 4 | 5 | import "fmt" 6 | 7 | const stateName = "ReadyPassedBlockedUnresolved" 8 | 9 | var stateIndex = [...]uint8{0, 5, 11, 18, 28} 10 | 11 | func (i State) String() string { 12 | if i < 0 || i >= State(len(stateIndex)-1) { 13 | return fmt.Sprintf("State(%d)", i) 14 | } 15 | return stateName[stateIndex[i]:stateIndex[i+1]] 16 | } 17 | -------------------------------------------------------------------------------- /job/counter.go: -------------------------------------------------------------------------------- 1 | package job 2 | 3 | var jobIDSource <-chan int 4 | 5 | func init() { 6 | jobIDSource = func() <-chan int { 7 | out := make(chan int, 10) 8 | go func() { 9 | var jobID int 10 | for { 11 | jobID++ 12 | out <- jobID 13 | } 14 | }() 15 | return out 16 | }() 17 | } 18 | 19 | // GetNumber is a source of safe monotonically increasing integers 20 | // for use in Job IDs. 21 | func GetNumber() int { 22 | return <-jobIDSource 23 | } 24 | -------------------------------------------------------------------------------- /job/counter_test.go: -------------------------------------------------------------------------------- 1 | package job 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/stretchr/testify/assert" 7 | ) 8 | 9 | // Counter is simple enough that a single simple test seems like 10 | // sufficient coverage here. 11 | 12 | func TestIDsAreIncreasing(t *testing.T) { 13 | assert := assert.New(t) 14 | 15 | for range [100]int{} { 16 | first := GetNumber() 17 | second := GetNumber() 18 | 19 | assert.True(first < second) 20 | assert.Equal(first+1, second) 21 | } 22 | } 23 | -------------------------------------------------------------------------------- /management/output.go: -------------------------------------------------------------------------------- 1 | package management 2 | 3 | // JobTypeCount holds data for counts of jobs by job type and group. 4 | type JobTypeCount struct { 5 | Type string 6 | Group string 7 | Count int 8 | } 9 | 10 | // GroupedID represents a job's ID and the group that the job belongs to, if 11 | // it's in a queue group. 12 | type GroupedID struct { 13 | ID string `bson:"_id" bson:"_id" yaml:"_id"` 14 | Group string `bson:"group,omitempty" json:"group,omitempty" yaml:"group,omitempty"` 15 | } 16 | -------------------------------------------------------------------------------- /management/filter_test.go: -------------------------------------------------------------------------------- 1 | package management 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/stretchr/testify/assert" 7 | ) 8 | 9 | func TestFilterValidation(t *testing.T) { 10 | t.Run("Status", func(t *testing.T) { 11 | for _, f := range []StatusFilter{Pending, InProgress, Stale, Completed, Retrying, All} { 12 | assert.Nil(t, f.Validate()) 13 | } 14 | }) 15 | 16 | t.Run("InvalidValues", func(t *testing.T) { 17 | for _, f := range []string{"", "foo", "bleh", "0"} { 18 | assert.Error(t, StatusFilter(f).Validate()) 19 | } 20 | }) 21 | } 22 | -------------------------------------------------------------------------------- /pool/doc.go: -------------------------------------------------------------------------------- 1 | /* 2 | Package pool provides specific implementations of the amboy.Runner interface 3 | that serve as the worker pools for jobs in work queues. 4 | 5 | Intentionally, most of the important logic about job execution and dispatching 6 | happens in the Queue implementation, and the Runner implementations are 7 | simplistic. 8 | 9 | Rate Limiting Pools Amboy includes a rate limiting pool to control the flow of 10 | jobs processed by the queue. The averaged pool uses an exponential weighted 11 | moving average and a targeted number of jobs to complete over an interval to 12 | achieve a reasonable flow of jobs through the runner. 13 | */ 14 | package pool 15 | 16 | // this file is intentional documentation only. 17 | -------------------------------------------------------------------------------- /scripts/waitForMongo.js: -------------------------------------------------------------------------------- 1 | // 2 | // Command to be run by mongosh to ensure that a local MongoDB instance is up 3 | // 4 | 5 | const timeout = 30 * 1000; // 30 seconds 6 | const start = new Date(); 7 | let connection = null; 8 | 9 | while (true) { 10 | let lastError = null; 11 | try { 12 | connection = new Mongo("localhost:27017"); 13 | } catch(error) { 14 | lastError = error; 15 | } 16 | 17 | const diff = (new Date()).getTime() - start.getTime(); 18 | 19 | if (connection) { 20 | console.log("waitForMongo.js: Successfully connected to local MongoDB instance:") 21 | break; 22 | } 23 | 24 | if (diff > timeout) { 25 | console.error("waitForMongo.js: Could not connect to local MongoDB instance:") 26 | console.error(lastError); 27 | exit(1) 28 | } 29 | sleep(100); 30 | } 31 | 32 | -------------------------------------------------------------------------------- /dependency/mock.go: -------------------------------------------------------------------------------- 1 | package dependency 2 | 3 | // MockDependency implements the dependency.Manager interface, but 4 | // provides the capability to mock out the behavior of the State() 5 | // method. 6 | type MockDependency struct { 7 | Response State 8 | T TypeInfo 9 | JobEdges 10 | } 11 | 12 | // NewMock constructs a new mocked dependency object. 13 | func NewMock() *MockDependency { 14 | return &MockDependency{ 15 | T: TypeInfo{ 16 | Name: "mock", 17 | Version: 0, 18 | }, 19 | JobEdges: NewJobEdges(), 20 | } 21 | } 22 | 23 | // State returns a state value derived from the Response field in the 24 | // MockDependency struct. 25 | func (d *MockDependency) State() State { return d.Response } 26 | 27 | // Type returns the TypeInfo value for this dependency implementation. 28 | func (d *MockDependency) Type() TypeInfo { return d.T } 29 | -------------------------------------------------------------------------------- /job/init.go: -------------------------------------------------------------------------------- 1 | package job 2 | 3 | import ( 4 | "github.com/mongodb/amboy" 5 | "github.com/mongodb/amboy/registry" 6 | ) 7 | 8 | // RegisterDefaultJobs registers all default job types in the amboy 9 | // Job registry which permits their use in contexts that require 10 | // serializing jobs to or from a common format (e.g. queues that 11 | // persist pending and completed jobs outside of the process,) or the 12 | // REST interface. 13 | // 14 | // In most applications these registrations happen automatically in 15 | // the context of package init() functions, but for the 16 | // default/generic jobs, users must explicitly load them into the 17 | // registry. 18 | func RegisterDefaultJobs() { 19 | registry.AddJobType("shell", func() amboy.Job { 20 | return NewShellJobInstance() 21 | }) 22 | registry.AddJobType("group", func() amboy.Job { 23 | return newGroupInstance() 24 | }) 25 | } 26 | -------------------------------------------------------------------------------- /rest/service_queue_fetch.go: -------------------------------------------------------------------------------- 1 | package rest 2 | 3 | import ( 4 | "net/http" 5 | 6 | "github.com/evergreen-ci/gimlet" 7 | "github.com/mongodb/amboy" 8 | "github.com/mongodb/amboy/registry" 9 | "github.com/mongodb/grip" 10 | ) 11 | 12 | // Fetch is an http handler that writes a job interchange object to a 13 | // the response, and allows clients to retrieve jobs from the service. 14 | func (s *QueueService) Fetch(w http.ResponseWriter, r *http.Request) { 15 | name := gimlet.GetVars(r)["name"] 16 | 17 | job, ok := s.queue.Get(r.Context(), name) 18 | if !ok { 19 | grip.Infof("job named %s does not exist in the queue", name) 20 | gimlet.WriteJSONResponse(w, http.StatusNotFound, nil) 21 | return 22 | } 23 | 24 | resp, err := registry.MakeJobInterchange(job, amboy.JSON) 25 | if err != nil { 26 | grip.Warningf("problem converting job %s to interchange format", name) 27 | gimlet.WriteJSONResponse(w, http.StatusInternalServerError, resp) 28 | return 29 | } 30 | 31 | gimlet.WriteJSON(w, resp) 32 | } 33 | -------------------------------------------------------------------------------- /formats.go: -------------------------------------------------------------------------------- 1 | package amboy 2 | 3 | // Format defines a sequence of constants used to distinguish between 4 | // different serialization formats for job objects used in the 5 | // amboy.ConvertTo and amboy.ConvertFrom functions, which support the 6 | // functionality of the Export and Import methods in the job 7 | // interface. 8 | type Format int 9 | 10 | // Supported values of the Format type, which represent different 11 | // supported serialization methods. 12 | const ( 13 | BSON Format = iota 14 | JSON 15 | BSON2 16 | ) 17 | 18 | // String implements fmt.Stringer and pretty prints the format name. 19 | func (f Format) String() string { 20 | switch f { 21 | case JSON: 22 | return "json" 23 | case BSON, BSON2: 24 | return "bson" 25 | default: 26 | return "INVALID" 27 | } 28 | } 29 | 30 | // IsValid returns true if when a valid format is specified, and false otherwise 31 | func (f Format) IsValid() bool { 32 | switch f { 33 | case JSON, BSON, BSON2: 34 | return true 35 | default: 36 | return false 37 | } 38 | } 39 | -------------------------------------------------------------------------------- /stats_test.go: -------------------------------------------------------------------------------- 1 | package amboy 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/mongodb/grip/message" 7 | "github.com/stretchr/testify/assert" 8 | ) 9 | 10 | func TestQueueStats(t *testing.T) { 11 | assert := assert.New(t) 12 | stat := &QueueStats{} 13 | 14 | assert.Implements((*message.Composer)(nil), stat) 15 | 16 | assert.True(stat.IsComplete()) 17 | 18 | assert.Contains(stat.String(), "running='0'") 19 | assert.Contains(stat.String(), "pending='0'") 20 | 21 | stat.Total++ 22 | assert.False(stat.IsComplete()) 23 | assert.Contains(stat.String(), "total='1'") 24 | 25 | stat.Completed++ 26 | assert.True(stat.IsComplete()) 27 | assert.Contains(stat.String(), "completed='1'") 28 | 29 | stat.Blocked++ 30 | assert.True(stat.IsComplete()) 31 | assert.Contains(stat.String(), "blocked='1'") 32 | 33 | stat.Total++ 34 | assert.True(stat.IsComplete()) 35 | assert.Contains(stat.String(), "total='2'") 36 | 37 | stat.Retrying++ 38 | assert.False(stat.IsComplete()) 39 | assert.Contains(stat.String(), "retrying='1'") 40 | 41 | } 42 | -------------------------------------------------------------------------------- /dependency/state.go: -------------------------------------------------------------------------------- 1 | package dependency 2 | 3 | // State provides a consistent set of values for 4 | // DependencyManager implementations to use to report their state, and 5 | // provide Queues and Jobs with a common set of terms to describe the 6 | // state of a job's dependencies 7 | type State int 8 | 9 | //go:generate stringer -type=State 10 | 11 | const ( 12 | // Ready indicates that a job is safe to execute from the 13 | // perspective of the Dependency Manager. 14 | Ready State = iota 15 | 16 | // Passed indicates that there is no work to be done for this 17 | // dependency. 18 | Passed 19 | 20 | // Blocked job are waiting for their dependencies to be 21 | // resolved. 22 | Blocked 23 | 24 | // Unresolved states are for cyclic dependencies or cases 25 | // where jobs depend on resources that cannot be built. 26 | Unresolved 27 | ) 28 | 29 | // IsValidState checks states and ensures that a state is valid. 30 | func IsValidState(s State) bool { 31 | if s >= 0 && s <= 3 { 32 | return true 33 | } 34 | 35 | return false 36 | } 37 | -------------------------------------------------------------------------------- /logger/doc.go: -------------------------------------------------------------------------------- 1 | // Package logger is a set of implementations to support amboy.Queue 2 | // backed grip/send.Senders for asynchronous and (generally) 3 | // non-blocking log message delivery. 4 | // 5 | // You can use amboy.Queue backed senders as an extension of an existing 6 | // queue or use constructors that use the queue.NewLimitedSizeLocalQueue to 7 | // deliver messages. 8 | // 9 | // These implementations do not guarantee delivery of log messages to 10 | // senders in any particular order. 11 | // 12 | // These senders do not provide any batching or group message sending: 13 | // messages are dispatched to queues immediately upon receipt. The 14 | // grip/send.NewBufferedSender implementation has these properties. 15 | // 16 | // The multi-sender implementation provided by this method creates a 17 | // single Job for every message. If you want to have a single Job for 18 | // every message, use the grip/send.MakeMultiSender in combination 19 | // with the single sender. 20 | package logger 21 | 22 | // this file is intentional documentation-only 23 | -------------------------------------------------------------------------------- /registry/dependency.go: -------------------------------------------------------------------------------- 1 | package registry 2 | 3 | import "github.com/mongodb/amboy/dependency" 4 | 5 | // AddDependencyType registers a new dependency.Manager factories. 6 | func AddDependencyType(name string, f dependency.ManagerFactory) { 7 | dependency.RegisterManager(name, f) 8 | } 9 | 10 | // GetDependencyFactory returns a dependency.Manager factory function 11 | // from the registry based on the name produced. If the name does not 12 | // exist, then the error value is non-nil. 13 | func GetDependencyFactory(name string) (dependency.ManagerFactory, error) { 14 | return dependency.GetManagerFactory(name) 15 | } 16 | 17 | // AddCheckType registers a callback function used in the 18 | // production of some dependencies 19 | func AddCheckType(name string, f dependency.CheckFactory) { 20 | dependency.RegisterCheck(name, f) 21 | } 22 | 23 | // GetCheckFactory returns a callback function factory for use in 24 | // dependencies 25 | func GetCheckFactory(name string) (dependency.CheckFactory, error) { 26 | return dependency.GetCheckFactory(name) 27 | } 28 | -------------------------------------------------------------------------------- /registry/raw.go: -------------------------------------------------------------------------------- 1 | package registry 2 | 3 | import mgobson "gopkg.in/mgo.v2/bson" 4 | 5 | type rawJob []byte 6 | 7 | func (j rawJob) MarshalJSON() ([]byte, error) { return j, nil } 8 | func (j *rawJob) UnmarshalBSON(in []byte) error { *j = in; return nil } 9 | func (j rawJob) MarshalBSON() ([]byte, error) { return j, nil } 10 | func (j *rawJob) UnmarshalJSON(in []byte) error { *j = in; return nil } 11 | func (j *rawJob) SetBSON(r mgobson.Raw) error { *j = r.Data; return nil } 12 | func (j *rawJob) GetBSON() (interface{}, error) { return *j, nil } 13 | 14 | type rawDependency []byte 15 | 16 | func (d rawDependency) MarshalJSON() ([]byte, error) { return d, nil } 17 | func (d *rawDependency) UnmarshalJSON(in []byte) error { *d = in; return nil } 18 | func (d rawDependency) MarshalBSON() ([]byte, error) { return d, nil } 19 | func (d *rawDependency) UnmarshalBSON(in []byte) error { *d = in; return nil } 20 | func (d *rawDependency) SetBSON(r mgobson.Raw) error { *d = r.Data; return nil } 21 | func (d *rawDependency) GetBSON() (interface{}, error) { return d, nil } 22 | -------------------------------------------------------------------------------- /dependency/always.go: -------------------------------------------------------------------------------- 1 | /* 2 | Package dependency contains the Manager interface, along with several 3 | implementations for different kinds of dependency checks. 4 | */ 5 | package dependency 6 | 7 | const alwaysRunName = "always" 8 | 9 | // Always is a DependencyManager implementation that always reports 10 | // that the job is ready to run. 11 | type alwaysManager struct { 12 | T TypeInfo `json:"type" bson:"type" yaml:"type"` 13 | JobEdges 14 | } 15 | 16 | // NewAlways creates a DependencyManager object that always 17 | // returns the "Ready" indicating that all dependency requirements 18 | // are met and that the target is required. 19 | func NewAlways() Manager { 20 | return &alwaysManager{ 21 | JobEdges: NewJobEdges(), 22 | T: TypeInfo{ 23 | Name: alwaysRunName, 24 | Version: 0, 25 | }, 26 | } 27 | } 28 | 29 | // State always returns, for Always, the "Ready" state. 30 | func (d *alwaysManager) State() State { 31 | return Ready 32 | } 33 | 34 | // Type returns a DependencyInterchange object to assist in 35 | // unmarshalling dependency objects. 36 | func (d *alwaysManager) Type() TypeInfo { 37 | return d.T 38 | } 39 | -------------------------------------------------------------------------------- /registry/mock_dependency_for_test.go: -------------------------------------------------------------------------------- 1 | package registry 2 | 3 | import "github.com/mongodb/amboy/dependency" 4 | 5 | // this is is a mock dependency implementation that's here 6 | 7 | type CheckTest struct { 8 | T dependency.TypeInfo `json:"type" bson:"type" yaml:"type"` 9 | dependency.JobEdges 10 | } 11 | 12 | func init() { 13 | AddDependencyType("test", func() dependency.Manager { 14 | return NewCheckTestDependency() 15 | }) 16 | } 17 | 18 | // NewCheckTestDependency creates a DependencyManager object that always 19 | // returns the "Passed" indicating that all dependency requirements 20 | // are met and that ing the target is required. 21 | func NewCheckTestDependency() *CheckTest { 22 | return &CheckTest{ 23 | JobEdges: dependency.NewJobEdges(), 24 | T: dependency.TypeInfo{ 25 | Name: "test", 26 | Version: 0, 27 | }, 28 | } 29 | } 30 | 31 | // State always returns, for CheckTest, the "Ready" state. 32 | func (d *CheckTest) State() dependency.State { 33 | return dependency.Ready 34 | } 35 | 36 | // Type returns a DependencyInterchange object to assist in 37 | // unmarshalling dependency objects. 38 | func (d *CheckTest) Type() dependency.TypeInfo { 39 | return d.T 40 | } 41 | -------------------------------------------------------------------------------- /registry/job.go: -------------------------------------------------------------------------------- 1 | package registry 2 | 3 | import "github.com/mongodb/amboy" 4 | 5 | // JobFactory is an alias for a function that returns a Job 6 | // interface. All Job implementation should have a factory function 7 | // with this signature to use with the amboy.RegisterJobType and 8 | // amboy.JobFactory functions that use an internal registry of jobs to 9 | // handle correct serialization and de-serialization of job objects. 10 | type JobFactory func() amboy.Job 11 | 12 | // AddJobType adds a job type to the amboy package's internal 13 | // registry of job types. This registry is used to support 14 | // serialization and de-serialization of between persistence layers. 15 | func AddJobType(name string, f JobFactory) { 16 | amboyRegistry.registerJobType(name, f) 17 | } 18 | 19 | // GetJobFactory produces Job objects of specific implementations 20 | // based on the type name, used in RegisterJobType and in the 21 | // JobType.Name field. 22 | func GetJobFactory(name string) (JobFactory, error) { 23 | return amboyRegistry.getJobFactory(name) 24 | } 25 | 26 | // JobTypeNames returns an iterator of all registered Job types 27 | func JobTypeNames() <-chan string { 28 | return amboyRegistry.jobTypeNames() 29 | } 30 | -------------------------------------------------------------------------------- /pool/noop.go: -------------------------------------------------------------------------------- 1 | package pool 2 | 3 | import ( 4 | "context" 5 | 6 | "github.com/mongodb/amboy" 7 | "github.com/pkg/errors" 8 | ) 9 | 10 | type noopPool struct { 11 | isStarted bool 12 | queue amboy.Queue 13 | } 14 | 15 | // NewNoop creates a runner implementation that has no workers, but 16 | // satisfies the workers and semantics of the Runner interface to 17 | // support queues deployments that have insert only queues. 18 | func NewNoop() amboy.Runner { return new(noopPool) } 19 | 20 | func (p *noopPool) Started() bool { return p.isStarted } 21 | 22 | func (p *noopPool) Start(ctx context.Context) error { 23 | if err := ctx.Err(); err != nil { 24 | return errors.WithStack(err) 25 | } 26 | 27 | if p.queue == nil { 28 | return errors.New("cannot start pool without setting a queue") 29 | } 30 | 31 | p.isStarted = true 32 | return nil 33 | } 34 | 35 | func (p *noopPool) SetQueue(q amboy.Queue) error { 36 | if q == nil { 37 | return errors.New("cannot set a nil queue") 38 | } 39 | 40 | if p.queue != nil { 41 | return errors.New("cannot override existing queue") 42 | } 43 | 44 | p.queue = q 45 | return nil 46 | } 47 | 48 | func (p *noopPool) Close(ctx context.Context) { 49 | p.isStarted = false 50 | } 51 | -------------------------------------------------------------------------------- /registry/doc.go: -------------------------------------------------------------------------------- 1 | /* 2 | Package registry contains infrastructure to support the persistence of 3 | Job definitions. 4 | 5 | # Job and Dependency Registries 6 | 7 | Systems need to be able to create and access Jobs and Dependency 8 | instances potentially from other implementations, and the registry 9 | provides a way to register new Job types both internal and external to 10 | the amboy package, and ensures that Jobs can be persisted and 11 | handled generically as needed. 12 | 13 | When you implement a new amboy/dependency.Manager or amboy.Job type, 14 | be sure to write a simple factory function for the type and register 15 | the factory in an init() function. Consider the following example: 16 | 17 | func init() { 18 | RegisterJobType("noop", noopJobFactory) 19 | } 20 | 21 | func noopJobFactory() amboy.Job { 22 | return &NoopJob{} 23 | } 24 | 25 | The dependency and job registers have similar interfaces. 26 | 27 | The registry package also provides functions for converting between an 28 | "interchange" format for persisting Job objects of mixed types 29 | consistently. Typically only authors of Queue implementations will 30 | need to use these operations. 31 | */ 32 | package registry 33 | 34 | // This file is intentionally documentation only. 35 | -------------------------------------------------------------------------------- /logger/job_single.go: -------------------------------------------------------------------------------- 1 | package logger 2 | 3 | import ( 4 | "context" 5 | "errors" 6 | "fmt" 7 | 8 | "github.com/mongodb/amboy" 9 | "github.com/mongodb/amboy/job" 10 | "github.com/mongodb/grip/message" 11 | "github.com/mongodb/grip/send" 12 | ) 13 | 14 | type sendMessageJob struct { 15 | message message.Composer 16 | sender send.Sender 17 | job.Base 18 | } 19 | 20 | // NewSendMessageJob creates an amboy.Job instance that sends the 21 | // specified message to the specified sender. 22 | // 23 | // This job is not compatible with remote-storage backed queues. 24 | func NewSendMessageJob(m message.Composer, s send.Sender) amboy.Job { 25 | j := &sendMessageJob{ 26 | message: m, 27 | sender: s, 28 | Base: job.Base{ 29 | JobType: amboy.JobType{ 30 | Name: "send-grip-message", 31 | Version: -1, 32 | }, 33 | }, 34 | } 35 | j.SetID(fmt.Sprintf("queued-message-%d--", job.GetNumber(), m, s)) 36 | return j 37 | } 38 | 39 | func (j *sendMessageJob) Run(_ context.Context) { 40 | defer j.MarkComplete() 41 | 42 | if j.message == nil { 43 | j.AddError(errors.New("message cannot be nil")) 44 | return 45 | } 46 | 47 | if j.sender == nil { 48 | j.AddError(errors.New("sender cannot be nil")) 49 | return 50 | } 51 | 52 | j.sender.Send(j.message) 53 | } 54 | -------------------------------------------------------------------------------- /dependency/state_test.go: -------------------------------------------------------------------------------- 1 | package dependency 2 | 3 | import ( 4 | "fmt" 5 | "strings" 6 | "testing" 7 | 8 | "github.com/stretchr/testify/suite" 9 | ) 10 | 11 | // StatesSuites checks the States types, which are the constants that 12 | // define if a job needs to run or can be a noop. 13 | type StatesSuite struct { 14 | suite.Suite 15 | } 16 | 17 | func TestStatesSuite(t *testing.T) { 18 | suite.Run(t, new(StatesSuite)) 19 | } 20 | 21 | func (s *StatesSuite) TestStatValidatorReturnsFalseForInvalidValues() { 22 | s.False(IsValidState(State(5))) 23 | s.False(IsValidState(State(-1))) 24 | } 25 | 26 | func (s *StatesSuite) TestStateValidatorReturnsTrueForValidValues() { 27 | for i := 0; i < 4; i++ { 28 | s.True(IsValidState(State(i))) 29 | } 30 | } 31 | 32 | func (s *StatesSuite) TestStringerInterfaceSatisfied() { 33 | s.Implements((*fmt.Stringer)(nil), State(0)) 34 | } 35 | 36 | func (s *StatesSuite) TestStringerProducesValidStrings() { 37 | for i := 0; i < 4; i++ { 38 | state := State(i) 39 | 40 | s.False(strings.HasPrefix(state.String(), "%s")) 41 | s.False(strings.HasPrefix(state.String(), "State(")) 42 | } 43 | 44 | } 45 | 46 | func (s *StatesSuite) TestStringerResturnsDefaultValueForOutOfBoundsStates() { 47 | s.True(strings.HasPrefix(State(-1).String(), "State(")) 48 | s.True(strings.HasPrefix(State(5).String(), "State(")) 49 | } 50 | -------------------------------------------------------------------------------- /queue/util.go: -------------------------------------------------------------------------------- 1 | package queue 2 | 3 | import ( 4 | "strings" 5 | "time" 6 | 7 | "github.com/mongodb/amboy" 8 | ) 9 | 10 | // addJobsSuffix adds the expected collection suffix for the non-grouped queue 11 | // if it doesn't already have the suffix. 12 | func addJobsSuffix(s string) string { 13 | if strings.HasSuffix(s, ".jobs") { 14 | return s 15 | } 16 | return s + ".jobs" 17 | } 18 | 19 | func trimJobsSuffix(s string) string { 20 | return strings.TrimSuffix(s, ".jobs") 21 | } 22 | 23 | // addGroupSuffix adds the expected collection suffix for a queue group if it 24 | // doesn't already have the suffix. 25 | func addGroupSuffix(s string) string { 26 | if strings.HasSuffix(s, ".group") { 27 | return s 28 | } 29 | return s + ".group" 30 | } 31 | 32 | func isDispatchable(stat amboy.JobStatusInfo, ti amboy.JobTimeInfo, lockTimeout time.Duration) bool { 33 | if isStaleInProgressJob(stat, lockTimeout) { 34 | return true 35 | } 36 | if ti.IsStale() { 37 | return false 38 | } 39 | if !ti.IsDispatchable() { 40 | return false 41 | } 42 | if stat.Completed { 43 | return false 44 | } 45 | if stat.InProgress { 46 | return false 47 | } 48 | 49 | return true 50 | } 51 | 52 | func isStaleInProgressJob(stat amboy.JobStatusInfo, lockTimeout time.Duration) bool { 53 | return stat.InProgress && time.Since(stat.ModificationTime) > lockTimeout 54 | } 55 | -------------------------------------------------------------------------------- /dependency/creates.go: -------------------------------------------------------------------------------- 1 | package dependency 2 | 3 | import "os" 4 | 5 | const createTypeName = "create-file" 6 | 7 | type createsFile struct { 8 | FileName string `bson:"file_name" json:"file_name" yaml:"file_name"` 9 | T TypeInfo `bson:"type" json:"type" yaml:"type"` 10 | JobEdges 11 | } 12 | 13 | func makeCreatesFile() *createsFile { 14 | return &createsFile{ 15 | T: TypeInfo{ 16 | Name: createTypeName, 17 | Version: 0, 18 | }, 19 | JobEdges: NewJobEdges(), 20 | } 21 | } 22 | 23 | // NewCreatesFile constructs a dependency manager object to support 24 | // jobs that are ready to run if a specific file doesn't 25 | // exist. 26 | func NewCreatesFile(name string) Manager { 27 | c := makeCreatesFile() 28 | c.FileName = name 29 | 30 | return c 31 | } 32 | 33 | // State returns Ready if the dependent file does not exist or is not 34 | // specified, and Passed if the file *does* exist. Jobs with Ready 35 | // states should be executed, while those with Passed states should be 36 | // a no-op. 37 | func (d *createsFile) State() State { 38 | if d.FileName == "" { 39 | return Ready 40 | } 41 | 42 | if _, err := os.Stat(d.FileName); os.IsNotExist(err) { 43 | return Ready 44 | } 45 | 46 | return Passed 47 | } 48 | 49 | // Type returns the type information on the "creates-file" Manager 50 | // implementation. 51 | func (d *createsFile) Type() TypeInfo { 52 | return d.T 53 | } 54 | -------------------------------------------------------------------------------- /dependency/manager.go: -------------------------------------------------------------------------------- 1 | package dependency 2 | 3 | // TypeInfo describes the type information that every dependency 4 | // implementation should provide in its Type() implementation. 5 | type TypeInfo struct { 6 | Name string `json:"name" bson:"name" yaml:"name"` 7 | Version int `json:"version" bson:"version" yaml:"version"` 8 | } 9 | 10 | // Manager objects provide a way for Jobs and queues to communicate 11 | // about dependencies between multiple Jobs. While some, indeed many 12 | // Job implementations, will have dependencies that *always* trigger 13 | // rebuilds, others will be able to specify a dependency that a queue 14 | // implementation can use to order Jobs. 15 | type Manager interface { 16 | // Reports the state of the dependency, and allows calling 17 | // jobs to determine if the dependencies for a Job have been 18 | // satisfied. 19 | State() State 20 | 21 | // Computes and returns a list of Job IDs that this job 22 | // depends on. While the State() method is ultimately 23 | // responsible for determining if a Dependency is resolved, 24 | // the Edges() function provides Queue implementations with a 25 | // way of (potentially) dependencies. 26 | Edges() []string 27 | 28 | // Adds new edges to the dependency manager. 29 | AddEdge(string) error 30 | 31 | // Returns a pointer to a DependencyType object, which is used 32 | // for serializing Dependency objects, when needed. 33 | Type() TypeInfo 34 | } 35 | -------------------------------------------------------------------------------- /dependency/always_test.go: -------------------------------------------------------------------------------- 1 | package dependency 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/stretchr/testify/suite" 7 | ) 8 | 9 | // AlwaysRebuildSuite tests the Always dependency implementation which 10 | // always returns the Ready dependency state. Does contain support for 11 | // dependency graph resolution, but the job will always run. 12 | type AlwaysRebuildSuite struct { 13 | dep *alwaysManager 14 | suite.Suite 15 | } 16 | 17 | func TestAlwaysRebuildSuite(t *testing.T) { 18 | suite.Run(t, new(AlwaysRebuildSuite)) 19 | } 20 | 21 | func (s *AlwaysRebuildSuite) SetupTest() { 22 | s.dep = NewAlways().(*alwaysManager) 23 | } 24 | 25 | func (s *AlwaysRebuildSuite) TestAlwaysImplementsDependencyManagerInterface() { 26 | s.Implements((*Manager)(nil), s.dep) 27 | } 28 | 29 | func (s *AlwaysRebuildSuite) TestConstructorCreatesObjectWithExpectedValues() { 30 | s.Equal("always", s.dep.T.Name) 31 | s.Equal(0, s.dep.T.Version) 32 | } 33 | 34 | func (s *AlwaysRebuildSuite) TestHasComposedJobEdgesInstance() { 35 | s.IsType(s.dep.JobEdges, JobEdges{}) 36 | 37 | var ok bool 38 | var dep interface{} = s.dep 39 | 40 | _, ok = dep.(interface { 41 | Edges() []string // nolint 42 | }) 43 | 44 | s.True(ok) 45 | 46 | _, ok = dep.(interface { 47 | AddEdge(string) error // nolint 48 | }) 49 | 50 | s.True(ok) 51 | } 52 | 53 | func (s *AlwaysRebuildSuite) TestTypeAccessorProvidesAccessToTheCorrectTypeInfo() { 54 | s.Equal("always", s.dep.Type().Name) 55 | s.Equal(0, s.dep.Type().Version) 56 | } 57 | -------------------------------------------------------------------------------- /registry/registry.go: -------------------------------------------------------------------------------- 1 | package registry 2 | 3 | import ( 4 | "sync" 5 | 6 | "github.com/mongodb/grip" 7 | "github.com/pkg/errors" 8 | ) 9 | 10 | var amboyRegistry *typeRegistry 11 | 12 | func init() { 13 | amboyRegistry = newTypeRegistry() 14 | } 15 | 16 | // Private implementation of the types registry. Public methods for 17 | // accessing the registry instance are in other files. 18 | 19 | type typeRegistry struct { 20 | job *jobs 21 | } 22 | 23 | type jobs struct { 24 | m map[string]JobFactory 25 | l *sync.RWMutex 26 | } 27 | 28 | func newTypeRegistry() *typeRegistry { 29 | return &typeRegistry{ 30 | job: &jobs{ 31 | m: make(map[string]JobFactory), 32 | l: &sync.RWMutex{}, 33 | }, 34 | } 35 | } 36 | 37 | func (r *typeRegistry) registerJobType(name string, f JobFactory) { 38 | r.job.l.Lock() 39 | defer r.job.l.Unlock() 40 | 41 | if _, exists := r.job.m[name]; exists { 42 | grip.Warningf("job named '%s' is already registered. Overwriting existing value.", name) 43 | } 44 | 45 | r.job.m[name] = f 46 | } 47 | 48 | func (r *typeRegistry) getJobFactory(name string) (JobFactory, error) { 49 | r.job.l.RLock() 50 | defer r.job.l.RUnlock() 51 | 52 | factory, ok := r.job.m[name] 53 | if !ok { 54 | return nil, errors.Errorf("there is no job type named '%s' registered", name) 55 | } 56 | 57 | return factory, nil 58 | } 59 | 60 | func (r *typeRegistry) jobTypeNames() <-chan string { 61 | output := make(chan string) 62 | 63 | go func() { 64 | r.job.l.RLock() 65 | defer r.job.l.RUnlock() 66 | for j := range r.job.m { 67 | output <- j 68 | } 69 | close(output) 70 | }() 71 | 72 | return output 73 | } 74 | -------------------------------------------------------------------------------- /dependency/callback.go: -------------------------------------------------------------------------------- 1 | package dependency 2 | 3 | const checkTypeName = "check" 4 | 5 | // CheckFunc describes a function type that can be registered and used 6 | // with the CheckManager. These functions are called by the dependency 7 | // manager and passed a list of edges for this job, and should 8 | // return. 9 | // 10 | // In effect this makes it easy to write many plugable custom dependency 11 | // manager, without needing to implement a large number of types. 12 | type CheckFunc func([]string) State 13 | 14 | type checkManager struct { 15 | CheckName string `bson:"function_name" json:"function_name" yaml:"function_name"` 16 | T TypeInfo `bson:"type" json:"type" yaml:"type"` 17 | JobEdges `bson:"edges" json:"edges" yaml:"edges"` 18 | } 19 | 20 | // NewCheckManager creates a new check manager that will call the 21 | // registered Check function matching that name. If no such function 22 | // exists, then the manager is Unresolved. 23 | func NewCheckManager(name string) Manager { 24 | m := makeCheckManager() 25 | m.CheckName = name 26 | return m 27 | } 28 | 29 | func makeCheckManager() *checkManager { 30 | return &checkManager{ 31 | JobEdges: NewJobEdges(), 32 | T: TypeInfo{ 33 | Version: 1, 34 | Name: checkTypeName, 35 | }, 36 | } 37 | 38 | } 39 | 40 | // Type returns the TypeInfo structure to satisfy the Manager 41 | // interface. 42 | func (d *checkManager) Type() TypeInfo { return d.T } 43 | 44 | // State returns a state constant that can be used to determine if a 45 | // dependency is satisfied. 46 | func (d *checkManager) State() State { 47 | factory, err := GetCheckFactory(d.CheckName) 48 | if err != nil { 49 | return Unresolved 50 | } 51 | 52 | check := factory() 53 | 54 | return check(d.Edges()) 55 | } 56 | -------------------------------------------------------------------------------- /registry/formats.go: -------------------------------------------------------------------------------- 1 | package registry 2 | 3 | import ( 4 | "encoding/json" 5 | 6 | "github.com/mongodb/amboy" 7 | "github.com/pkg/errors" 8 | "go.mongodb.org/mongo-driver/bson" 9 | mgobson "gopkg.in/mgo.v2/bson" 10 | ) 11 | 12 | // ConvertTo takes a Format specification and interface and returns a 13 | // serialized byte sequence according to that Format value. If there 14 | // is an issue with the serialization, or the Format value is not 15 | // supported, then this method returns an error. 16 | func convertTo(f amboy.Format, v interface{}) ([]byte, error) { 17 | var output []byte 18 | var err error 19 | 20 | switch f { 21 | case amboy.JSON: 22 | output, err = json.Marshal(v) 23 | case amboy.BSON: 24 | output, err = mgobson.Marshal(v) 25 | case amboy.BSON2: 26 | output, err = bson.Marshal(v) 27 | default: 28 | return nil, errors.New("no support for specified serialization format") 29 | } 30 | 31 | if err != nil { 32 | return nil, errors.Wrap(err, "problem serializing data") 33 | } 34 | 35 | return output, nil 36 | 37 | } 38 | 39 | // ConvertFrom takes a Format type, a byte sequence, and an interface 40 | // and attempts to deserialize that data into the interface object as 41 | // indicated by the Format specifier. 42 | func convertFrom(f amboy.Format, data []byte, v interface{}) error { 43 | switch f { 44 | case amboy.JSON: 45 | return errors.Wrap(json.Unmarshal(data, v), "problem serializing data from json") 46 | case amboy.BSON: 47 | return errors.Wrap(mgobson.Unmarshal(data, v), "problem serializing data from bson") 48 | case amboy.BSON2: 49 | return errors.Wrap(bson.Unmarshal(data, v), "problem serializing data from bson (new)") 50 | default: 51 | return errors.New("no support for specified serialization format") 52 | } 53 | } 54 | -------------------------------------------------------------------------------- /logger/job_test.go: -------------------------------------------------------------------------------- 1 | package logger 2 | 3 | import ( 4 | "context" 5 | "testing" 6 | 7 | "github.com/mongodb/amboy" 8 | "github.com/mongodb/grip/level" 9 | "github.com/mongodb/grip/message" 10 | "github.com/mongodb/grip/send" 11 | "github.com/stretchr/testify/suite" 12 | ) 13 | 14 | type MsgJobSuite struct { 15 | constructor func(m message.Composer, s send.Sender) amboy.Job 16 | suite.Suite 17 | } 18 | 19 | func TestSingleMsgJobSuite(t *testing.T) { 20 | s := new(MsgJobSuite) 21 | s.constructor = NewSendMessageJob 22 | suite.Run(t, s) 23 | } 24 | 25 | func (s *MsgJobSuite) TestIsNotSerializable() { 26 | job := s.constructor(nil, nil) 27 | s.True(0 > job.Type().Version) 28 | } 29 | 30 | func (s *MsgJobSuite) TestWithNilOpts() { 31 | job := s.constructor(nil, nil) 32 | s.NoError(job.Error()) 33 | job.Run(context.Background()) 34 | s.Error(job.Error()) 35 | } 36 | 37 | func (s *MsgJobSuite) TestWithNilMessage() { 38 | sender := send.MakeNative() 39 | sender.SetName("test") 40 | job := s.constructor(nil, sender) 41 | 42 | s.NoError(job.Error()) 43 | job.Run(context.Background()) 44 | s.Error(job.Error()) 45 | } 46 | 47 | func (s *MsgJobSuite) TestWithNilSender() { 48 | job := s.constructor(message.NewString("foo"), nil) 49 | 50 | s.NoError(job.Error()) 51 | job.Run(context.Background()) 52 | s.Error(job.Error()) 53 | } 54 | 55 | func (s *MsgJobSuite) TestMessgeSends() { 56 | sender, err := send.NewInternalLogger("test", send.LevelInfo{Default: level.Debug, Threshold: level.Info}) 57 | s.NoError(err) 58 | job := s.constructor(message.NewDefaultMessage(level.Alert, "foo"), sender) 59 | 60 | s.False(sender.HasMessage()) 61 | s.NoError(job.Error()) 62 | job.Run(context.Background()) 63 | s.NoError(job.Error()) 64 | s.True(sender.HasMessage()) 65 | } 66 | -------------------------------------------------------------------------------- /rest/service_queue_test.go: -------------------------------------------------------------------------------- 1 | package rest 2 | 3 | import ( 4 | "context" 5 | "testing" 6 | 7 | "github.com/mongodb/amboy" 8 | "github.com/mongodb/amboy/registry" 9 | "github.com/mongodb/grip" 10 | "github.com/mongodb/grip/level" 11 | "github.com/mongodb/grip/send" 12 | "github.com/stretchr/testify/require" 13 | "github.com/stretchr/testify/suite" 14 | ) 15 | 16 | func init() { 17 | grip.SetName("amboy.rest.tests") 18 | grip.Error(grip.SetSender(send.MakeNative())) 19 | 20 | lvl := grip.GetSender().Level() 21 | lvl.Threshold = level.Warning 22 | _ = grip.GetSender().SetLevel(lvl) 23 | } 24 | 25 | type RestServiceSuite struct { 26 | service *QueueService 27 | require *require.Assertions 28 | suite.Suite 29 | } 30 | 31 | func TestRestServiceSuite(t *testing.T) { 32 | suite.Run(t, new(RestServiceSuite)) 33 | } 34 | 35 | func (s *RestServiceSuite) SetupSuite() { 36 | s.require = s.Require() 37 | } 38 | 39 | func (s *RestServiceSuite) SetupTest() { 40 | s.service = NewQueueService() 41 | } 42 | 43 | func (s *RestServiceSuite) TearDownTest() { 44 | s.service.Close() 45 | } 46 | 47 | func (s *RestServiceSuite) TestInitialListOfRegisteredJobs() { 48 | defaultJob := &QueueService{} 49 | s.Len(defaultJob.registeredTypes, 0) 50 | 51 | count := 0 52 | for _, jType := range s.service.registeredTypes { 53 | factory, err := registry.GetJobFactory(jType) 54 | if s.NoError(err) { 55 | count++ 56 | s.Implements((*amboy.Job)(nil), factory()) 57 | } 58 | } 59 | 60 | s.Len(s.service.registeredTypes, count) 61 | } 62 | 63 | func (s *RestServiceSuite) TestServiceOpenMethodInitializesResources() { 64 | s.Nil(s.service.closer) 65 | s.Nil(s.service.queue) 66 | 67 | ctx := context.Background() 68 | s.NoError(s.service.Open(ctx)) 69 | 70 | s.NotNil(s.service.queue) 71 | s.NotNil(s.service.closer) 72 | } 73 | -------------------------------------------------------------------------------- /periodic_test.go: -------------------------------------------------------------------------------- 1 | package amboy 2 | 3 | import ( 4 | "context" 5 | "testing" 6 | "time" 7 | 8 | "github.com/stretchr/testify/assert" 9 | ) 10 | 11 | func TestWaitUntil(t *testing.T) { 12 | ctx, cancel := context.WithCancel(context.Background()) 13 | defer cancel() 14 | 15 | const interval = 5 * time.Second 16 | 17 | t.Run("PastStartAt", func(t *testing.T) { 18 | tsa := time.Now().Round(time.Second) 19 | waitUntilInterval(ctx, time.Now().Round(time.Second).Add(-interval), interval) 20 | assert.WithinDuration(t, tsa.Add(interval), time.Now(), interval) 21 | assert.True(t, tsa.Before(time.Now())) 22 | }) 23 | t.Run("FutureStartAt", func(t *testing.T) { 24 | tsa := time.Now().Round(time.Second) 25 | waitUntilInterval(ctx, time.Now().Round(time.Second).Add(interval), interval) 26 | assert.WithinDuration(t, tsa.Add(interval), time.Now(), interval) 27 | assert.True(t, tsa.Before(time.Now())) 28 | }) 29 | t.Run("Cancelable", func(t *testing.T) { 30 | ctx, cancel = context.WithCancel(ctx) 31 | cancel() 32 | tsa := time.Now().Round(time.Second) 33 | waitUntilInterval(ctx, time.Now().Round(time.Second).Add(interval), interval) 34 | assert.Equal(t, time.Now().Round(time.Second), tsa) 35 | }) 36 | t.Run("DuplicateJobError", func(t *testing.T) { 37 | t.Run("WithReportingDisabled", func(t *testing.T) { 38 | err := scheduleOp(ctx, nil, func(_ context.Context, q Queue) error { 39 | return NewDuplicateJobError("err") 40 | }, QueueOperationConfig{}) 41 | assert.NoError(t, err) 42 | }) 43 | t.Run("WithReportingEnabled", func(t *testing.T) { 44 | err := scheduleOp(ctx, nil, func(_ context.Context, q Queue) error { 45 | return NewDuplicateJobError("err") 46 | }, QueueOperationConfig{EnableDuplicateJobReporting: true}) 47 | assert.Error(t, err) 48 | assert.True(t, IsDuplicateJobError(err)) 49 | }) 50 | }) 51 | } 52 | -------------------------------------------------------------------------------- /rest/service_queue_stats_test.go: -------------------------------------------------------------------------------- 1 | package rest 2 | 3 | import ( 4 | "context" 5 | "encoding/json" 6 | "net/http/httptest" 7 | "testing" 8 | 9 | "github.com/stretchr/testify/assert" 10 | ) 11 | 12 | func TestStatusOuputGenerator(t *testing.T) { 13 | assert := assert.New(t) 14 | service := NewQueueService() 15 | 16 | ctx, cancel := context.WithCancel(context.Background()) 17 | defer cancel() 18 | 19 | st := service.getStatus(ctx) 20 | assert.Equal("degraded", st.Status) 21 | assert.False(st.QueueRunning) 22 | assert.True(len(st.SupportedJobTypes) > 0) 23 | 24 | // Now open the the service, and thus the queue, and watch the response change: 25 | assert.NoError(service.Open(ctx)) 26 | defer service.Close() 27 | 28 | st = service.getStatus(ctx) 29 | assert.Equal("ok", st.Status) 30 | assert.True(st.QueueRunning) 31 | assert.True(len(st.SupportedJobTypes) > 0) 32 | } 33 | 34 | func TestStatusMethod(t *testing.T) { 35 | assert := assert.New(t) // nolint 36 | ctx := context.Background() 37 | service := NewQueueService() 38 | 39 | w := httptest.NewRecorder() 40 | 41 | service.Status(w, httptest.NewRequest("GET", "http://example.com/status", nil)) 42 | assert.Equal(200, w.Code) 43 | 44 | st := status{} 45 | assert.NoError(json.Unmarshal(w.Body.Bytes(), &st)) 46 | assert.Equal("degraded", st.Status) 47 | assert.False(st.QueueRunning) 48 | assert.True(len(st.SupportedJobTypes) > 0) 49 | 50 | // Now open the the service, and thus the queue, and watch the response change: 51 | assert.NoError(service.Open(ctx)) 52 | defer service.Close() 53 | 54 | w = httptest.NewRecorder() 55 | 56 | service.Status(w, httptest.NewRequest("GET", "http://example.com/status", nil)) 57 | assert.Equal(200, w.Code) 58 | 59 | st = status{} 60 | assert.NoError(json.Unmarshal(w.Body.Bytes(), &st)) 61 | assert.Equal("ok", st.Status) 62 | assert.True(st.QueueRunning) 63 | assert.True(len(st.SupportedJobTypes) > 0) 64 | } 65 | -------------------------------------------------------------------------------- /dependency/edges_test.go: -------------------------------------------------------------------------------- 1 | package dependency 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/stretchr/testify/suite" 7 | ) 8 | 9 | type EdgeHandlerSuite struct { 10 | edges JobEdges 11 | suite.Suite 12 | } 13 | 14 | func TestEdgeHandlerSuite(t *testing.T) { 15 | suite.Run(t, new(EdgeHandlerSuite)) 16 | } 17 | 18 | func (s *EdgeHandlerSuite) SetupTest() { 19 | s.edges = NewJobEdges() 20 | } 21 | 22 | func (s *EdgeHandlerSuite) TestAddEdgePersistsInternallyAsExpected() { 23 | s.Len(s.edges.Edges(), 0) 24 | 25 | name := "test-one" 26 | s.NoError(s.edges.AddEdge(name)) 27 | edges := s.edges.Edges() 28 | s.Equal(edges[0], name) 29 | 30 | s.Equal(s.edges.TaskEdges[0], name) 31 | 32 | edge, ok := s.edges.edgesSet[name] 33 | s.True(ok) 34 | s.True(edge) 35 | } 36 | 37 | func (s *EdgeHandlerSuite) TestInternalEdgesSetIsMaintained() { 38 | s.Len(s.edges.Edges(), 0) 39 | 40 | name := "test-one" 41 | s.NoError(s.edges.AddEdge(name)) 42 | s.Len(s.edges.edgesSet, 1) 43 | s.Len(s.edges.Edges(), 1) 44 | 45 | // replace the internal set tracker 46 | s.edges.edgesSet = make(map[string]bool) 47 | s.Len(s.edges.edgesSet, 0) 48 | 49 | // It should be an error to add an edge more than once. 50 | s.Error(s.edges.AddEdge(name)) 51 | s.Len(s.edges.edgesSet, 1) 52 | s.Len(s.edges.TaskEdges, 1) 53 | s.Len(s.edges.Edges(), 1) 54 | 55 | // add another edge, shouldn't reduce a duplicate edge in the 56 | // set or the list 57 | s.NoError(s.edges.AddEdge("test-two")) 58 | s.Len(s.edges.edgesSet, 2) 59 | s.Len(s.edges.TaskEdges, 2) 60 | s.Len(s.edges.Edges(), 2) 61 | } 62 | 63 | func (s *EdgeHandlerSuite) TestTaskEdgeTracking() { 64 | // edge defaults to empty 65 | s.Len(s.edges.Edges(), 0) 66 | 67 | s.NoError(s.edges.AddEdge("foo")) 68 | s.Len(s.edges.Edges(), 1) 69 | 70 | // make sure the internals look like we expect. 71 | s.Len(s.edges.edgesSet, 1) 72 | exists, ok := s.edges.edgesSet["foo"] 73 | s.True(exists) 74 | s.True(ok) 75 | } 76 | -------------------------------------------------------------------------------- /queue/util_for_test.go: -------------------------------------------------------------------------------- 1 | package queue 2 | 3 | import ( 4 | "github.com/evergreen-ci/utility" 5 | "github.com/mongodb/amboy" 6 | ) 7 | 8 | // defaultMongoDBQueueGroupTestOptions returns default MongoDB queue group 9 | // options for testing purposes only. 10 | func defaultMongoDBQueueGroupTestOptions() MongoDBQueueGroupOptions { 11 | queueOpts := defaultMongoDBQueueTestOptions() 12 | return MongoDBQueueGroupOptions{ 13 | DefaultQueue: queueOpts, 14 | } 15 | } 16 | 17 | // defaultMongoDBQueueTestOptions returns default MongoDB queue options for 18 | // testing purposes only. 19 | func defaultMongoDBQueueTestOptions() MongoDBQueueOptions { 20 | dbOpts := defaultMongoDBTestOptions() 21 | return MongoDBQueueOptions{ 22 | DB: &dbOpts, 23 | NumWorkers: utility.ToIntPtr(1), 24 | } 25 | } 26 | 27 | // defaultMongoDBTestOptions returns default MongoDB options for testing 28 | // purposes only. 29 | func defaultMongoDBTestOptions() MongoDBOptions { 30 | opts := DefaultMongoDBOptions() 31 | opts.DB = "amboy_test" 32 | opts.Collection = newDriverID() 33 | return opts 34 | } 35 | 36 | // bsonJobTimeInfo converts all amboy.JobTimeInfo time fields into BSON time. 37 | func bsonJobTimeInfo(i amboy.JobTimeInfo) amboy.JobTimeInfo { 38 | i.Created = utility.BSONTime(i.Created) 39 | i.Start = utility.BSONTime(i.Start) 40 | i.End = utility.BSONTime(i.End) 41 | i.WaitUntil = utility.BSONTime(i.WaitUntil) 42 | i.DispatchBy = utility.BSONTime(i.DispatchBy) 43 | return i 44 | } 45 | 46 | // bsonJobStatusInfo converts all amboy.JobStatusInfo time fields into BSON 47 | // time. 48 | func bsonJobStatusInfo(i amboy.JobStatusInfo) amboy.JobStatusInfo { 49 | i.ModificationTime = utility.BSONTime(i.ModificationTime) 50 | return i 51 | } 52 | 53 | // bsonJobRetryInfo converts all amboy.JobRetryInfo time fields into BSON time. 54 | func bsonJobRetryInfo(i amboy.JobRetryInfo) amboy.JobRetryInfo { 55 | i.Start = utility.BSONTime(i.Start) 56 | i.End = utility.BSONTime(i.End) 57 | return i 58 | } 59 | -------------------------------------------------------------------------------- /rest/service_queue_stats.go: -------------------------------------------------------------------------------- 1 | package rest 2 | 3 | import ( 4 | "context" 5 | "net/http" 6 | "time" 7 | 8 | "github.com/evergreen-ci/gimlet" 9 | "github.com/mongodb/amboy" 10 | "github.com/mongodb/grip" 11 | ) 12 | 13 | type status struct { 14 | Status string `bson:"string" json:"string" yaml:"string"` 15 | QueueRunning bool `bson:"queue_running" json:"queue_running" yaml:"queue_running"` 16 | PendingJobs int `bson:"pending_jobs,omitempty" json:"pending_jobs,omitempty" yaml:"pending_jobs,omitempty"` 17 | SupportedJobTypes []string `bson:"supported_job_types" json:"supported_job_types" yaml:"supported_job_types"` 18 | } 19 | 20 | func (s *QueueService) getStatus(ctx context.Context) status { 21 | output := status{ 22 | SupportedJobTypes: s.registeredTypes, 23 | } 24 | 25 | if s.queue != nil && s.queue.Info().Started { 26 | output.Status = "ok" 27 | output.QueueRunning = true 28 | output.PendingJobs = s.queue.Stats(ctx).Pending 29 | } else { 30 | output.Status = "degraded" 31 | } 32 | 33 | return output 34 | } 35 | 36 | // Status defines an http.HandlerFunc that returns health check and 37 | // current staus status information for the entire service. 38 | func (s *QueueService) Status(w http.ResponseWriter, r *http.Request) { 39 | gimlet.WriteJSON(w, s.getStatus(r.Context())) 40 | } 41 | 42 | // WaitAll blocks waiting for all pending jobs in the queue to 43 | // stop. Has a default timeout of 10 seconds, and returns 408 (request 44 | // timeout) when the timeout succeeds. 45 | func (s *QueueService) WaitAll(w http.ResponseWriter, r *http.Request) { 46 | timeout, err := parseTimeout(r) 47 | if err != nil { 48 | grip.Infof("problem parsing timeout for wait-all operation: %v", err) 49 | } 50 | ctx, cancel := context.WithTimeout(r.Context(), timeout) 51 | defer cancel() 52 | 53 | ok := amboy.WaitInterval(ctx, s.queue, 100*time.Millisecond) 54 | st := s.getStatus(ctx) 55 | if !ok { 56 | gimlet.WriteJSONResponse(w, http.StatusRequestTimeout, st) 57 | return 58 | } 59 | 60 | gimlet.WriteJSON(w, st) 61 | } 62 | -------------------------------------------------------------------------------- /rest/service_queue_create.go: -------------------------------------------------------------------------------- 1 | package rest 2 | 3 | import ( 4 | "context" 5 | "net/http" 6 | 7 | "github.com/evergreen-ci/gimlet" 8 | "github.com/mongodb/amboy" 9 | "github.com/mongodb/amboy/registry" 10 | "github.com/mongodb/grip" 11 | ) 12 | 13 | type createResponse struct { 14 | Registered bool `bson:"registered" json:"registered" yaml:"registered"` 15 | QueueDepth int `bson:"queue_depth,omitempty" json:"queue_depth,omitempty" yaml:"queue_depth,omitempty"` 16 | ID string `bson:"id" json:"id" yaml:"id"` 17 | Error string `bson:"error,omitempty" json:"error,omitempty" yaml:"error,omitempty"` 18 | Status status `bson:"status,omitempty" json:"status,omitempty" yaml:"status,omitempty"` 19 | } 20 | 21 | func (s *QueueService) createJobResponseBase(ctx context.Context) *createResponse { 22 | return &createResponse{ 23 | QueueDepth: s.queue.Stats(ctx).Pending, 24 | Status: s.getStatus(ctx), 25 | } 26 | } 27 | 28 | func (s *QueueService) createJob(ctx context.Context, payload *registry.JobInterchange) (*createResponse, error) { 29 | resp := s.createJobResponseBase(ctx) 30 | j, err := payload.Resolve(amboy.JSON) 31 | 32 | if err != nil { 33 | resp.Error = err.Error() 34 | return resp, err 35 | } 36 | 37 | resp.ID = j.ID() 38 | 39 | err = s.queue.Put(ctx, j) 40 | if err != nil { 41 | resp.Error = err.Error() 42 | return resp, err 43 | } 44 | 45 | resp.Registered = true 46 | return resp, nil 47 | } 48 | 49 | // Create provides an interface for REST clients to create jobs in the 50 | // local queue that backs the service. 51 | func (s *QueueService) Create(w http.ResponseWriter, r *http.Request) { 52 | jobPayload := ®istry.JobInterchange{} 53 | ctx := r.Context() 54 | resp := s.createJobResponseBase(ctx) 55 | 56 | err := gimlet.GetJSON(r.Body, jobPayload) 57 | if err != nil { 58 | resp.Error = err.Error() 59 | grip.Error(err) 60 | gimlet.WriteJSONError(w, resp) 61 | return 62 | } 63 | 64 | resp, err = s.createJob(ctx, jobPayload) 65 | if err != nil { 66 | resp.Error = err.Error() 67 | grip.Error(err) 68 | gimlet.WriteJSONError(w, resp) 69 | return 70 | } 71 | 72 | gimlet.WriteJSON(w, resp) 73 | } 74 | -------------------------------------------------------------------------------- /dependency/creates_test.go: -------------------------------------------------------------------------------- 1 | package dependency 2 | 3 | import ( 4 | "fmt" 5 | "path/filepath" 6 | "runtime" 7 | "testing" 8 | 9 | "github.com/stretchr/testify/suite" 10 | ) 11 | 12 | func GetDirectoryOfFile() string { 13 | _, file, _, _ := runtime.Caller(1) 14 | 15 | return filepath.Dir(file) 16 | } 17 | 18 | // CreatesFileSuite tests the dependency.Manager 19 | // implementation that checks for the existence of a file. If the file 20 | // exist the dependency becomes a noop. 21 | type CreatesFileSuite struct { 22 | dep *createsFile 23 | packages []string 24 | suite.Suite 25 | } 26 | 27 | func TestCreatesFileSuite(t *testing.T) { 28 | suite.Run(t, new(CreatesFileSuite)) 29 | } 30 | 31 | func (s *CreatesFileSuite) SetupSuite() { 32 | s.packages = []string{"job", "dependency", "queue", "pool", "build", "registry"} 33 | } 34 | 35 | func (s *CreatesFileSuite) SetupTest() { 36 | s.dep = makeCreatesFile() 37 | } 38 | 39 | func (s *CreatesFileSuite) TestInstanceImplementsManagerInterface() { 40 | s.Implements((*Manager)(nil), s.dep) 41 | } 42 | 43 | func (s *CreatesFileSuite) TestConstructorCreatesObjectWithFileNameSet() { 44 | for _, dir := range s.packages { 45 | dep, ok := NewCreatesFile(dir).(*createsFile) 46 | s.True(ok) 47 | s.Equal(dir, dep.FileName) 48 | } 49 | } 50 | 51 | func (s *CreatesFileSuite) TestDependencyWithoutFileSetReportsReady() { 52 | s.Equal(s.dep.FileName, "") 53 | s.Equal(s.dep.State(), Ready) 54 | 55 | s.dep.FileName = " \\[ ]" 56 | s.Equal(s.dep.State(), Ready) 57 | 58 | s.dep.FileName = "foo" 59 | s.Equal(s.dep.State(), Ready) 60 | 61 | s.dep.FileName = " " 62 | s.Equal(s.dep.State(), Ready) 63 | 64 | s.dep.FileName = "" 65 | s.Equal(s.dep.State(), Ready) 66 | } 67 | 68 | func (s *CreatesFileSuite) TestAmboyPackageDirectoriesExistAndReportPassedState() { 69 | cwd := filepath.Dir(GetDirectoryOfFile()) 70 | 71 | for _, dir := range s.packages { 72 | p := filepath.Join(cwd, dir) 73 | dep := NewCreatesFile(p) 74 | s.Equal(dep.State(), Passed, fmt.Sprintln(dep.State(), p)) 75 | } 76 | } 77 | 78 | func (s *CreatesFileSuite) TestCreatesDependencyTestReportsExpectedType() { 79 | t := s.dep.Type() 80 | s.Equal(t.Name, "create-file") 81 | s.Equal(t.Version, 0) 82 | } 83 | -------------------------------------------------------------------------------- /dependency/edges.go: -------------------------------------------------------------------------------- 1 | /* 2 | Edges 3 | 4 | Dependencies have methods to add or access Edges of the job. These 5 | allow Jobs, by way of their dependencies to express relationships 6 | between Jobs. Fundamentally managing Job ordering is a property of a 7 | Queue implementation, but these methods allow jobs to express their 8 | dependencies on other jobs as a hint to Queue 9 | implementations. Separately, queue implementation checks the 10 | environment to ensure that all prerequisites are satisfied. 11 | */ 12 | package dependency 13 | 14 | import "github.com/pkg/errors" 15 | 16 | // JobEdges provides a common subset of a non-trivial Manager 17 | // implementation. These objects provide two methods of the Manager 18 | // interface, and keep track of the relationships between Jobs in a 19 | // job queue. 20 | type JobEdges struct { 21 | TaskEdges []string `bson:"edges" json:"edges" yaml:"edges"` 22 | edgesSet map[string]bool 23 | } 24 | 25 | // NewJobEdges returns an initialized JobEdges object. 26 | func NewJobEdges() JobEdges { 27 | return JobEdges{ 28 | TaskEdges: []string{}, 29 | edgesSet: make(map[string]bool), 30 | } 31 | } 32 | 33 | // Edges returns a copy of JobEdges.Edges list of edges for this 34 | // slice. As a result, adding or removing items from this slice does 35 | // not affect other readers, and this object *only* reflects changes to 36 | // the dependencies made after calling this method. 37 | func (e *JobEdges) Edges() []string { 38 | output := make([]string, len(e.TaskEdges)) 39 | copy(output, e.TaskEdges) 40 | 41 | return output 42 | } 43 | 44 | // AddEdge adds an edge to the dependency tracker. If the edge already 45 | // exists, this operation returns an error. 46 | func (e *JobEdges) AddEdge(name string) error { 47 | if len(e.TaskEdges) != len(e.edgesSet) { 48 | // this is probably the case when we're re-reading an 49 | // instance from the DB. we'll just rehydrate things 50 | // here. 51 | 52 | for _, edge := range e.TaskEdges { 53 | e.edgesSet[edge] = true 54 | } 55 | } 56 | 57 | if _, ok := e.edgesSet[name]; ok { 58 | return errors.Errorf("edge '%s' already exists", name) 59 | } 60 | 61 | // TODO we probably need a lock here, but maybe not? 62 | e.edgesSet[name] = true 63 | e.TaskEdges = append(e.TaskEdges, name) 64 | 65 | return nil 66 | } 67 | -------------------------------------------------------------------------------- /management/mongodb_test.go: -------------------------------------------------------------------------------- 1 | package management 2 | 3 | import ( 4 | "context" 5 | "testing" 6 | "time" 7 | 8 | "github.com/evergreen-ci/utility" 9 | "github.com/mongodb/amboy/queue" 10 | "github.com/stretchr/testify/assert" 11 | "github.com/stretchr/testify/require" 12 | "go.mongodb.org/mongo-driver/mongo" 13 | "go.mongodb.org/mongo-driver/mongo/options" 14 | ) 15 | 16 | func defaultMongoDBTestOptions() queue.MongoDBOptions { 17 | opts := queue.DefaultMongoDBOptions() 18 | opts.DB = "amboy_test" 19 | opts.Collection = "test." + utility.RandomString() 20 | return opts 21 | } 22 | 23 | func TestMongoDBConstructors(t *testing.T) { 24 | ctx, cancel := context.WithCancel(context.Background()) 25 | defer cancel() 26 | 27 | client, err := mongo.Connect(ctx, options.Client().ApplyURI(defaultMongoDBTestOptions().URI).SetConnectTimeout(time.Second)) 28 | require.NoError(t, err) 29 | 30 | t.Run("MissingClientShouldError", func(t *testing.T) { 31 | opts := defaultMongoDBTestOptions() 32 | conf := DBQueueManagerOptions{Options: opts} 33 | 34 | db, err := MakeDBQueueManager(ctx, conf) 35 | assert.Error(t, err) 36 | assert.Zero(t, db) 37 | }) 38 | t.Run("InvalidDBOptionsShouldError", func(t *testing.T) { 39 | opts := defaultMongoDBTestOptions() 40 | opts.Collection = "" 41 | conf := DBQueueManagerOptions{Options: opts} 42 | 43 | db, err := MakeDBQueueManager(ctx, conf) 44 | assert.Error(t, err) 45 | assert.Zero(t, db) 46 | }) 47 | t.Run("BuildNewConnector", func(t *testing.T) { 48 | opts := defaultMongoDBTestOptions() 49 | opts.Client = client 50 | opts.Collection = t.Name() 51 | conf := DBQueueManagerOptions{Options: opts} 52 | 53 | db, err := MakeDBQueueManager(ctx, conf) 54 | assert.NoError(t, err) 55 | assert.NotNil(t, db) 56 | 57 | r, ok := db.(*dbQueueManager) 58 | require.True(t, ok) 59 | require.NotNil(t, r) 60 | assert.NotZero(t, r.collection) 61 | }) 62 | t.Run("DialWithNewConstructor", func(t *testing.T) { 63 | opts := defaultMongoDBTestOptions() 64 | opts.Collection = t.Name() 65 | conf := DBQueueManagerOptions{Options: opts} 66 | 67 | r, err := NewDBQueueManager(ctx, conf) 68 | assert.NoError(t, err) 69 | assert.NotNil(t, r) 70 | }) 71 | t.Run("DialWithBadURI", func(t *testing.T) { 72 | opts := defaultMongoDBTestOptions() 73 | opts.Collection = t.Name() 74 | opts.URI = "mongodb://lochost:26016" 75 | conf := DBQueueManagerOptions{Options: opts} 76 | 77 | r, err := NewDBQueueManager(ctx, conf) 78 | assert.Error(t, err) 79 | assert.Nil(t, r) 80 | }) 81 | } 82 | -------------------------------------------------------------------------------- /cmd/verify-mod-tidy/verify-mod-tidy.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "bytes" 5 | "context" 6 | "flag" 7 | "fmt" 8 | "os" 9 | "os/exec" 10 | "time" 11 | 12 | "github.com/pkg/errors" 13 | ) 14 | 15 | const ( 16 | goModFile = "go.mod" 17 | goSumFile = "go.sum" 18 | ) 19 | 20 | // verify-mod-tidy verifies that `go mod tidy` has been run to clean up the 21 | // go.mod and go.sum files. 22 | func main() { 23 | var ( 24 | goBin string 25 | timeout time.Duration 26 | ) 27 | 28 | flag.DurationVar(&timeout, "timeout", 0, "timeout for verifying modules are tidy") 29 | flag.StringVar(&goBin, "goBin", "go", "path to go binary to use for mod tidy check") 30 | flag.Parse() 31 | 32 | ctx := context.Background() 33 | if timeout != 0 { 34 | var cancel context.CancelFunc 35 | ctx, cancel = context.WithTimeout(ctx, timeout) 36 | defer cancel() 37 | } 38 | 39 | oldGoMod, oldGoSum, err := readModuleFiles() 40 | if err != nil { 41 | fmt.Fprintln(os.Stderr, err) 42 | os.Exit(1) 43 | } 44 | 45 | if err := runModTidy(ctx, goBin); err != nil { 46 | fmt.Fprintln(os.Stderr, err) 47 | os.Exit(1) 48 | } 49 | 50 | newGoMod, newGoSum, err := readModuleFiles() 51 | if err != nil { 52 | fmt.Fprintln(os.Stderr, err) 53 | os.Exit(1) 54 | } 55 | 56 | if !bytes.Equal(oldGoMod, newGoMod) || !bytes.Equal(oldGoSum, newGoSum) { 57 | fmt.Fprintf(os.Stderr, "%s and/or %s are not tidy - please run `go mod tidy`.\n", goModFile, goSumFile) 58 | writeModuleFiles(oldGoMod, oldGoSum) 59 | os.Exit(1) 60 | } 61 | } 62 | 63 | // readModuleFiles reads the contents of the go module files. 64 | func readModuleFiles() (goMod []byte, goSum []byte, err error) { 65 | goMod, err = os.ReadFile(goModFile) 66 | if err != nil { 67 | return nil, nil, errors.Wrapf(err, "reading file '%s'", goModFile) 68 | } 69 | goSum, err = os.ReadFile(goSumFile) 70 | if err != nil { 71 | return nil, nil, errors.Wrapf(err, "reading file '%s'", goSumFile) 72 | } 73 | return goMod, goSum, nil 74 | } 75 | 76 | // writeModuleFiles writes the contents of the go module files. 77 | func writeModuleFiles(goMod, goSum []byte) { 78 | if err := os.WriteFile(goModFile, goMod, 0600); err != nil { 79 | fmt.Fprintln(os.Stderr, err) 80 | } 81 | if err := os.WriteFile(goSumFile, goSum, 0600); err != nil { 82 | fmt.Fprintln(os.Stderr, err) 83 | } 84 | } 85 | 86 | // runModTidy runs the `go mod tidy` command with the given go binary. 87 | func runModTidy(ctx context.Context, goBin string) error { 88 | cmd := exec.CommandContext(ctx, goBin, "mod", "tidy") 89 | cmd.Stdout = os.Stdout 90 | cmd.Stderr = os.Stderr 91 | return errors.Wrap(cmd.Run(), "mod tidy") 92 | } 93 | -------------------------------------------------------------------------------- /logger/sender_single.go: -------------------------------------------------------------------------------- 1 | package logger 2 | 3 | import ( 4 | "context" 5 | "sync" 6 | 7 | "github.com/mongodb/amboy" 8 | "github.com/mongodb/amboy/queue" 9 | "github.com/mongodb/grip/message" 10 | "github.com/mongodb/grip/send" 11 | ) 12 | 13 | type queueSender struct { 14 | mu sync.RWMutex 15 | ctx context.Context 16 | queue amboy.Queue 17 | canceler context.CancelFunc 18 | send.Sender 19 | } 20 | 21 | func newSender(ctx context.Context, q amboy.Queue, sender send.Sender) *queueSender { 22 | return &queueSender{ 23 | Sender: sender, 24 | queue: q, 25 | ctx: ctx, 26 | } 27 | } 28 | 29 | // MakeQueueSender wraps the sender with a queue-backed delivery 30 | // mechanism using the specified queue instance. 31 | // 32 | // These senders do not ensure that logged messages are propagated to 33 | // the underlying sender implementation in any order, and may result 34 | // in out-of-order logging. 35 | // 36 | // The close method does not close the underlying sender. 37 | // 38 | // In the event that the sender's Put method returns an error, the 39 | // message (and its error) will be logged directly (and synchronously) 40 | func MakeQueueSender(ctx context.Context, q amboy.Queue, sender send.Sender) send.Sender { 41 | return newSender(ctx, q, sender) 42 | } 43 | 44 | // NewQueueBackedSender creates a new LimitedSize queue, and creates a 45 | // sender implementation wrapping this sender. The queue is not shared. 46 | // 47 | // This sender returns an error if there is a problem starting the 48 | // queue, and cancels the queue upon closing, without waiting for the 49 | // queue to empty. 50 | func NewQueueBackedSender(ctx context.Context, sender send.Sender, workers, capacity int) (send.Sender, error) { 51 | q := queue.NewLocalLimitedSize(workers, capacity) 52 | s := newSender(ctx, q, sender) 53 | 54 | s.ctx, s.canceler = context.WithCancel(s.ctx) 55 | if err := q.Start(s.ctx); err != nil { 56 | return nil, err 57 | } 58 | 59 | return s, nil 60 | } 61 | 62 | func (s *queueSender) Send(m message.Composer) { 63 | if s.Level().ShouldLog(m) { 64 | s.mu.RLock() 65 | defer s.mu.RUnlock() 66 | 67 | err := s.queue.Put(s.ctx, NewSendMessageJob(m, s.Sender)) 68 | if err != nil { 69 | s.Send(message.NewErrorWrap(err, m.String())) 70 | } 71 | } 72 | } 73 | 74 | func (s *queueSender) Flush(ctx context.Context) error { 75 | s.mu.Lock() 76 | defer s.mu.Unlock() 77 | 78 | if !amboy.Wait(ctx, s.queue) { 79 | return ctx.Err() 80 | } 81 | 82 | return s.Sender.Flush(ctx) 83 | } 84 | 85 | func (s *queueSender) Close() error { 86 | if s.canceler != nil { 87 | s.canceler() 88 | } 89 | 90 | return nil 91 | } 92 | -------------------------------------------------------------------------------- /errors_test.go: -------------------------------------------------------------------------------- 1 | package amboy 2 | 3 | import ( 4 | "testing" 5 | 6 | "github.com/pkg/errors" 7 | "github.com/stretchr/testify/assert" 8 | ) 9 | 10 | func TestDuplicateError(t *testing.T) { 11 | t.Run("RegularErrorIsNotDuplicate", func(t *testing.T) { 12 | err := errors.New("err") 13 | assert.False(t, IsDuplicateJobError(err)) 14 | assert.False(t, IsDuplicateJobScopeError(err)) 15 | }) 16 | t.Run("NilErrorIsNotDuplicate", func(t *testing.T) { 17 | assert.False(t, IsDuplicateJobError(nil)) 18 | assert.False(t, IsDuplicateJobScopeError(nil)) 19 | }) 20 | t.Run("NewDuplicateJobError", func(t *testing.T) { 21 | err := NewDuplicateJobError("err") 22 | assert.True(t, IsDuplicateJobError(err)) 23 | assert.False(t, IsDuplicateJobScopeError(err)) 24 | }) 25 | t.Run("NewDuplicateJobErrorf", func(t *testing.T) { 26 | err := NewDuplicateJobErrorf("err %s", "err") 27 | assert.True(t, IsDuplicateJobError(err)) 28 | assert.False(t, IsDuplicateJobScopeError(err)) 29 | }) 30 | t.Run("MakeDuplicateJobError", func(t *testing.T) { 31 | err := MakeDuplicateJobError(errors.New("err")) 32 | assert.True(t, IsDuplicateJobError(err)) 33 | assert.False(t, IsDuplicateJobScopeError(err)) 34 | }) 35 | t.Run("NewDuplicateJobScopeError", func(t *testing.T) { 36 | err := NewDuplicateJobScopeError("err") 37 | assert.True(t, IsDuplicateJobError(err)) 38 | assert.True(t, IsDuplicateJobScopeError(err)) 39 | }) 40 | t.Run("NewDuplicateScopeJobErrorf", func(t *testing.T) { 41 | err := NewDuplicateJobScopeErrorf("err %s", "err") 42 | assert.True(t, IsDuplicateJobError(err)) 43 | assert.True(t, IsDuplicateJobScopeError(err)) 44 | }) 45 | t.Run("MakeDuplicateScopeJobError", func(t *testing.T) { 46 | err := MakeDuplicateJobScopeError(errors.New("err")) 47 | assert.True(t, IsDuplicateJobError(err)) 48 | assert.True(t, IsDuplicateJobScopeError(err)) 49 | }) 50 | } 51 | 52 | func TestJobNotFoundError(t *testing.T) { 53 | t.Run("RegularErrorIsNotJobNotFound", func(t *testing.T) { 54 | err := errors.New("err") 55 | assert.False(t, IsJobNotFoundError(err)) 56 | }) 57 | t.Run("NilErrorIsNotDuplicate", func(t *testing.T) { 58 | assert.False(t, IsJobNotFoundError(nil)) 59 | }) 60 | t.Run("NewJobNotFoundError", func(t *testing.T) { 61 | err := NewJobNotFoundError("err") 62 | assert.True(t, IsJobNotFoundError(err)) 63 | }) 64 | t.Run("NewJobNotFoundErrorf", func(t *testing.T) { 65 | err := NewJobNotFoundErrorf("err %s", "err") 66 | assert.True(t, IsJobNotFoundError(err)) 67 | }) 68 | t.Run("MakeJobNotFoundError", func(t *testing.T) { 69 | err := MakeJobNotFoundError(errors.New("err")) 70 | assert.True(t, IsJobNotFoundError(err)) 71 | }) 72 | } 73 | -------------------------------------------------------------------------------- /cli/cli.go: -------------------------------------------------------------------------------- 1 | package cli 2 | 3 | import ( 4 | "context" 5 | "net/http" 6 | "strings" 7 | 8 | "github.com/mongodb/amboy/management" 9 | "github.com/mongodb/amboy/rest" 10 | "github.com/pkg/errors" 11 | "github.com/urfave/cli" 12 | ) 13 | 14 | // Amboy provides a reusable CLI component for interacting with rest 15 | // management and reporting services. 16 | func Amboy(opts *ServiceOptions) cli.Command { 17 | return cli.Command{ 18 | Name: "amboy", 19 | Usage: "access administrative rest interfaces for an amboy service", 20 | Subcommands: []cli.Command{ 21 | queueManagement(opts), 22 | abortablePoolManagement(opts), 23 | }, 24 | } 25 | } 26 | 27 | // ServiceOptions makes it possible for users of Amboy to create a cli 28 | // tool with reasonable defaults and with client. 29 | type ServiceOptions struct { 30 | BaseURL string 31 | QueueManagementPrefix string 32 | AbortablePoolManagementPrefix string 33 | Client *http.Client 34 | } 35 | 36 | const ( 37 | serviceURLFlagName = "service" 38 | prefixFlagName = "prefix" 39 | ) 40 | 41 | func (o *ServiceOptions) managementReportFlags(base ...cli.Flag) []cli.Flag { 42 | return append(base, 43 | cli.StringFlag{ 44 | Name: serviceURLFlagName, 45 | Usage: "Specify the base URL of the service", 46 | Value: o.BaseURL, 47 | }, 48 | cli.StringFlag{ 49 | Name: prefixFlagName, 50 | Usage: "Specify the service prefix for the reporting service.", 51 | Value: o.QueueManagementPrefix, 52 | }, 53 | ) 54 | } 55 | 56 | func (o *ServiceOptions) abortablePoolManagementFlags(base ...cli.Flag) []cli.Flag { 57 | return append(base, 58 | cli.StringFlag{ 59 | Name: serviceURLFlagName, 60 | Usage: "Specify the base URL of the service.", 61 | Value: o.BaseURL, 62 | }, 63 | cli.StringFlag{ 64 | Name: prefixFlagName, 65 | Usage: "Specify the service prefix for the management service.", 66 | Value: o.AbortablePoolManagementPrefix, 67 | }, 68 | ) 69 | } 70 | 71 | func (o *ServiceOptions) withManagementClient(ctx context.Context, c *cli.Context, op func(client management.Manager) error) error { 72 | if o.Client == nil { 73 | o.Client = http.DefaultClient 74 | } 75 | 76 | client := rest.NewManagementClientFromExisting(o.Client, getCLIPath(c)) 77 | 78 | return errors.WithStack(op(client)) 79 | } 80 | 81 | func (o *ServiceOptions) withAbortablePoolManagementClient(ctx context.Context, c *cli.Context, op func(client *rest.AbortablePoolManagementClient) error) error { 82 | if o.Client == nil { 83 | o.Client = http.DefaultClient 84 | } 85 | 86 | client := rest.NewAbortablePoolManagementClientFromExisting(o.Client, getCLIPath(c)) 87 | 88 | return errors.WithStack(op(client)) 89 | } 90 | 91 | func getCLIPath(c *cli.Context) string { 92 | return strings.TrimRight(c.String(serviceURLFlagName), "/") + "/" + strings.TrimLeft(c.String(prefixFlagName), "/") 93 | } 94 | -------------------------------------------------------------------------------- /queue/driver.go: -------------------------------------------------------------------------------- 1 | package queue 2 | 3 | import ( 4 | "context" 5 | "time" 6 | 7 | "github.com/mongodb/amboy" 8 | ) 9 | 10 | // remoteQueueDriver describes the interface between a queue and an out of 11 | // process persistence layer, like a database. 12 | type remoteQueueDriver interface { 13 | ID() string 14 | Open(context.Context) error 15 | Close(context.Context) error 16 | 17 | // Get finds a job by job ID. For retryable jobs, this returns the latest 18 | // job attempt. 19 | Get(context.Context, string) (amboy.Job, error) 20 | // GetAttempt returns a retryable job by job ID and attempt number. If used 21 | // to find a non-retryable job, this should return nil job and an error. 22 | GetAttempt(ctx context.Context, id string, attempt int) (amboy.Job, error) 23 | // GetAllAttempts returns all attempts of a retryable job by job ID. If used 24 | // to find a non-retryable job, this should return no jobs and an error. 25 | GetAllAttempts(ctx context.Context, id string) ([]amboy.Job, error) 26 | // Put inserts a new job in the backing storage. 27 | Put(context.Context, amboy.Job) error 28 | // PutMany inserts new jobs in the backing storage. Each implementation can 29 | // decide how to handle a mixture of errors. 30 | PutMany(context.Context, []amboy.Job) error 31 | // Save updates an existing job in the backing storage. Implementations may 32 | // not allow calls to Save to run concurrently. 33 | Save(context.Context, amboy.Job) error 34 | // CompleteAndPut updates an existing job toComplete and inserts a new job 35 | // toPut atomically. Implementations may not allow calls to CompleteAndPut 36 | // to run concurrently. 37 | CompleteAndPut(ctx context.Context, toComplete amboy.Job, toPut amboy.Job) error 38 | 39 | Jobs(context.Context) <-chan amboy.Job 40 | // RetryableJobs returns retryable jobs, subject to a filter. 41 | RetryableJobs(context.Context, retryableJobFilter) <-chan amboy.Job 42 | Next(context.Context) amboy.Job 43 | 44 | Stats(context.Context) amboy.QueueStats 45 | JobInfo(context.Context) <-chan amboy.JobInfo 46 | Complete(context.Context, amboy.Job) error 47 | 48 | LockTimeout() time.Duration 49 | 50 | SetDispatcher(Dispatcher) 51 | Dispatcher() Dispatcher 52 | } 53 | 54 | // retryableJobFilter represents a query filter on retryable jobs. 55 | type retryableJobFilter string 56 | 57 | const ( 58 | // RetryableJobAll refers to all retryable jobs. 59 | retryableJobAll retryableJobFilter = "all-retryable" 60 | // RetryableJobAllRetrying refers to all retryable jobs that are currently 61 | // waiting to retry. 62 | retryableJobAllRetrying retryableJobFilter = "all-retrying" 63 | // RetryableJobActiveRetrying refers to retryable jobs that have recently 64 | // retried. 65 | retryableJobActiveRetrying retryableJobFilter = "active-retrying" 66 | // RetryableJobStaleRetrying refers to retryable jobs that should be 67 | // retrying but have not done so recently. 68 | retryableJobStaleRetrying retryableJobFilter = "stale-retrying" 69 | ) 70 | -------------------------------------------------------------------------------- /job/shell.go: -------------------------------------------------------------------------------- 1 | package job 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "os/exec" 7 | "strings" 8 | 9 | "github.com/mongodb/amboy" 10 | "github.com/mongodb/amboy/dependency" 11 | "github.com/mongodb/grip" 12 | ) 13 | 14 | // ShellJob is an amboy.Job implementation that runs shell commands in 15 | // the context of an amboy.Job object. 16 | type ShellJob struct { 17 | Command string `bson:"command" json:"command" yaml:"command"` 18 | Output string `bson:"output" json:"output" yaml:"output"` 19 | WorkingDir string `bson:"working_dir" json:"working_dir" yaml:"working_dir"` 20 | Env map[string]string `bson:"env" json:"env" yaml:"env"` 21 | 22 | Base `bson:"job_base" json:"job_base" yaml:"job_base"` 23 | } 24 | 25 | // NewShellJob takes the command, as a string along with the name of a 26 | // file that the command would create, and returns a pointer to a 27 | // ShellJob object. If the "creates" argument is an empty string then 28 | // the command always runs, otherwise only if the file specified does 29 | // not exist. You can change the dependency with the SetDependency 30 | // argument. 31 | func NewShellJob(cmd string, creates string) *ShellJob { 32 | j := NewShellJobInstance() 33 | j.Command = cmd 34 | 35 | if creates != "" { 36 | j.SetDependency(dependency.NewCreatesFile(creates)) 37 | } 38 | 39 | j.SetID(fmt.Sprintf("shell-job-%d-%s", GetNumber(), strings.Split(cmd, " ")[0])) 40 | 41 | return j 42 | } 43 | 44 | // NewShellJobInstance returns a pointer to an initialized ShellJob 45 | // instance, but does not set the command or the name. Use when the 46 | // command is not known at creation time. 47 | func NewShellJobInstance() *ShellJob { 48 | j := &ShellJob{ 49 | Env: make(map[string]string), 50 | Base: Base{ 51 | JobType: amboy.JobType{ 52 | Name: "shell", 53 | Version: 1, 54 | }, 55 | }, 56 | } 57 | return j 58 | } 59 | 60 | // Run executes the shell commands. Add keys to the Env map to modify 61 | // the environment, or change the value of the WorkingDir property to 62 | // set the working directory for this command. Captures output into 63 | // the Output attribute, and returns the error value of the command. 64 | func (j *ShellJob) Run(ctx context.Context) { 65 | defer j.MarkComplete() 66 | grip.Debugf("running %s", j.Command) 67 | 68 | args := strings.Split(j.Command, " ") 69 | 70 | j.mutex.RLock() 71 | cmd := exec.CommandContext(ctx, args[0], args[1:]...) // nolint 72 | j.mutex.RUnlock() 73 | 74 | cmd.Dir = j.WorkingDir 75 | cmd.Env = j.getEnvVars() 76 | 77 | output, err := cmd.CombinedOutput() 78 | j.AddError(err) 79 | 80 | j.mutex.Lock() 81 | defer j.mutex.Unlock() 82 | 83 | j.Output = strings.TrimSpace(string(output)) 84 | } 85 | 86 | func (j *ShellJob) getEnvVars() []string { 87 | if len(j.Env) == 0 { 88 | return []string{} 89 | } 90 | 91 | output := []string{} 92 | for k, v := range j.Env { 93 | output = append(output, strings.Join([]string{k, v}, "=")) 94 | } 95 | 96 | return output 97 | } 98 | -------------------------------------------------------------------------------- /dependency/registry.go: -------------------------------------------------------------------------------- 1 | package dependency 2 | 3 | import ( 4 | "sync" 5 | 6 | "github.com/mongodb/grip" 7 | "github.com/pkg/errors" 8 | ) 9 | 10 | var registry *registryCache 11 | 12 | func init() { 13 | registry = ®istryCache{ 14 | m: map[string]ManagerFactory{}, 15 | c: map[string]CheckFactory{}, 16 | } 17 | 18 | RegisterManager(alwaysRunName, NewAlways) 19 | RegisterManager(checkTypeName, func() Manager { return makeCheckManager() }) 20 | RegisterManager(createTypeName, func() Manager { return makeCreatesFile() }) 21 | RegisterManager(localFileTypeName, func() Manager { return MakeLocalFile() }) 22 | } 23 | 24 | // RegisterManager stores a dependency Manager factory in the global 25 | // Manager registry. 26 | func RegisterManager(name string, f ManagerFactory) { registry.addManager(name, f) } 27 | 28 | // GetManagerFactory returns a globally registered manager factory by name. 29 | func GetManagerFactory(name string) (ManagerFactory, error) { return registry.getManager(name) } 30 | 31 | // RegisterCheck stores a CheckFactory in the global check registry. 32 | func RegisterCheck(name string, f CheckFactory) { registry.addCheck(name, f) } 33 | 34 | // GetCheckFactory returns a globally registered check factory by name. 35 | func GetCheckFactory(name string) (CheckFactory, error) { return registry.getCheck(name) } 36 | 37 | // ManagerFactory is a function that takes no arguments and returns 38 | // a dependency.Manager interface. When implementing a new dependency 39 | // type, also register a factory function with the DependencyFactory 40 | // signature to facilitate serialization. 41 | type ManagerFactory func() Manager 42 | 43 | // CheckFactory is a function that takes no arguments and returns a 44 | // dependency callback for use in callback-style dependencies. 45 | type CheckFactory func() CheckFunc 46 | 47 | type registryCache struct { 48 | m map[string]ManagerFactory 49 | mmu sync.RWMutex 50 | c map[string]CheckFactory 51 | cmu sync.RWMutex 52 | } 53 | 54 | func (r *registryCache) addManager(name string, factory ManagerFactory) { 55 | r.mmu.Lock() 56 | defer r.mmu.Unlock() 57 | 58 | if _, ok := r.m[name]; ok { 59 | grip.Warningf("overriding cached dependency manager '%s'", name) 60 | } 61 | 62 | r.m[name] = factory 63 | } 64 | 65 | func (r *registryCache) getManager(name string) (ManagerFactory, error) { 66 | r.mmu.RLock() 67 | defer r.mmu.RUnlock() 68 | f, ok := r.m[name] 69 | if !ok { 70 | return nil, errors.Errorf("no factory named '%s' is registered", name) 71 | } 72 | return f, nil 73 | } 74 | 75 | func (r *registryCache) addCheck(name string, factory CheckFactory) { 76 | r.cmu.Lock() 77 | defer r.cmu.Unlock() 78 | 79 | if _, ok := r.c[name]; ok { 80 | grip.Warningf("overriding cached dependency callback '%s'", name) 81 | } 82 | 83 | r.c[name] = factory 84 | } 85 | 86 | func (r *registryCache) getCheck(name string) (CheckFactory, error) { 87 | r.cmu.RLock() 88 | defer r.cmu.RUnlock() 89 | 90 | f, ok := r.c[name] 91 | if !ok { 92 | return nil, errors.Errorf("no factory named '%s' is registered", name) 93 | } 94 | 95 | return f, nil 96 | } 97 | -------------------------------------------------------------------------------- /job/shell_test.go: -------------------------------------------------------------------------------- 1 | package job 2 | 3 | import ( 4 | "context" 5 | "runtime" 6 | "strings" 7 | "testing" 8 | 9 | "github.com/mongodb/amboy" 10 | "github.com/mongodb/amboy/dependency" 11 | "github.com/stretchr/testify/require" 12 | "github.com/stretchr/testify/suite" 13 | ) 14 | 15 | // ShellJobSuite collects tests of the generic shell command running 16 | // amboy.Job implementation. The actual implementation of the command 17 | // execution is straightforward, and so this test mostly checks the 18 | // constructor and the environment variable construction. 19 | type ShellJobSuite struct { 20 | job *ShellJob 21 | require *require.Assertions 22 | suite.Suite 23 | } 24 | 25 | func TestShellJobSuite(t *testing.T) { 26 | suite.Run(t, new(ShellJobSuite)) 27 | } 28 | 29 | func (s *ShellJobSuite) SetupSuite() { 30 | s.require = s.Require() 31 | } 32 | 33 | func (s *ShellJobSuite) SetupTest() { 34 | s.job = NewShellJobInstance() 35 | } 36 | 37 | func (s *ShellJobSuite) TestShellJobProducesObjectsThatImplementJobInterface() { 38 | s.Implements((*amboy.Job)(nil), s.job) 39 | s.Implements((*amboy.Job)(nil), NewShellJobInstance()) 40 | } 41 | 42 | func (s *ShellJobSuite) TestShellJobFactoryImplementsInterfaceWithCorrectTypeInfo() { 43 | sj := NewShellJobInstance() 44 | 45 | s.IsType(sj, s.job) 46 | s.Equal(sj.Type(), s.job.Type()) 47 | 48 | s.Equal(sj.Type().Name, "shell") 49 | s.Equal(sj.Type().Version, 1) 50 | } 51 | 52 | func (s *ShellJobSuite) TestShellJobDefaultsToAlwaysDependency() { 53 | s.Equal(s.job.Dependency().Type().Name, "always") 54 | } 55 | 56 | func (s *ShellJobSuite) TestShellJobConstructorHasCreatesFileDependency() { 57 | job := NewShellJob("foo", "bar") 58 | s.Equal(job.Dependency().Type().Name, "create-file") 59 | } 60 | 61 | func (s *ShellJobSuite) TestSetDependencyChangesDependencyStrategy() { 62 | s.job.SetDependency(dependency.NewCreatesFile("foo")) 63 | s.Equal(s.job.Dependency().Type().Name, "create-file") 64 | } 65 | 66 | func (s *ShellJobSuite) TestShellJobNameConstructedFromCommandNames() { 67 | job := NewShellJob("foo", "bar") 68 | s.Equal(job.ID(), job.Base.Name) 69 | 70 | s.True(strings.HasSuffix(job.ID(), "foo"), job.ID()) 71 | 72 | job = NewShellJob("touch foo bar", "baz") 73 | s.True(strings.HasSuffix(job.ID(), "touch"), job.ID()) 74 | } 75 | 76 | func (s *ShellJobSuite) TestRunTrivialCommandReturnsWithoutError() { 77 | s.job = NewShellJob("true", "") 78 | 79 | s.False(s.job.Status().Completed) 80 | s.job.Run(context.Background()) 81 | s.NoError(s.job.Error()) 82 | s.True(s.job.Status().Completed) 83 | } 84 | 85 | func (s *ShellJobSuite) TestRunWithErroneousCommandReturnsError() { 86 | s.job = NewShellJob("foo", "") 87 | 88 | s.False(s.job.Status().Completed) 89 | s.job.Run(context.Background()) 90 | s.Error(s.job.Error()) 91 | s.True(s.job.Status().Completed) 92 | } 93 | 94 | func (s *ShellJobSuite) TestEnvironmentVariableIsPassedToCommand() { 95 | s.job = NewShellJob("env", "") 96 | s.job.Env["MSG"] = "foo" 97 | s.job.Run(context.Background()) 98 | s.NoError(s.job.Error()) 99 | 100 | if runtime.GOOS == "windows" { 101 | s.True(len(s.job.Output) > 0) 102 | } else { 103 | s.Equal("MSG=foo", s.job.Output) 104 | } 105 | 106 | } 107 | -------------------------------------------------------------------------------- /management/interface.go: -------------------------------------------------------------------------------- 1 | package management 2 | 3 | import ( 4 | "context" 5 | 6 | "github.com/pkg/errors" 7 | ) 8 | 9 | // Manager is an interface that describes queue introspection tools and 10 | // utility for queue management that make it possible to get more details about 11 | // the running jobs in an amboy queue and gives users broader capabilities than 12 | // the Queue interface itself. 13 | type Manager interface { 14 | // JobStatus returns statistics of the number of jobs of each job type 15 | // matching the status. 16 | JobStatus(context.Context, StatusFilter) ([]JobTypeCount, error) 17 | // JobIDsByState returns a report of job IDs filtered by a job type and 18 | // status filter. Depending on the implementation, the returned job IDs can 19 | // be either logical job IDs or internally-stored job IDs. 20 | JobIDsByState(context.Context, string, StatusFilter) ([]GroupedID, error) 21 | 22 | // For all CompleteJob* methods, implementations should mark jobs as 23 | // completed. Furthermore, for implementations managing retryable queues, 24 | // they should complete the job's retrying phase (i.e. the job will not 25 | // retry). 26 | 27 | // CompleteJob marks a job complete by ID. Implementations may differ on 28 | // whether it matches the logical job ID (i.e. (amboy.Job).ID) or 29 | // internally-stored job IDs (which may differ from the user-visible job 30 | // ID). 31 | CompleteJob(context.Context, string) error 32 | // CompleteJobs marks all jobs complete that match the given status filter. 33 | CompleteJobs(context.Context, StatusFilter) error 34 | // CompleteJobsByType marks all jobs complete that match the given status 35 | // filter and job type. 36 | CompleteJobsByType(context.Context, StatusFilter, string) error 37 | // CompleteJobsByPattern marks all jobs complete that match the given status 38 | // filter and whose job ID matches the given regular expression. 39 | // Implementations may differ on whether the pattern matches the logical job 40 | // ID (i.e. (amboy.Job).ID) or internally-stored job IDs (which may differ 41 | // from the user-visible job ID). Furthermore, implementations may differ on 42 | // the accepted pattern-matching language. 43 | CompleteJobsByPattern(context.Context, StatusFilter, string) error 44 | } 45 | 46 | // StatusFilter defines a number of dimensions with which to filter 47 | // current jobs in a queue by status 48 | type StatusFilter string 49 | 50 | // Constants representing valid StatusFilters. 51 | const ( 52 | Pending StatusFilter = "pending" 53 | InProgress StatusFilter = "in-progress" 54 | Stale StatusFilter = "stale" 55 | Completed StatusFilter = "completed" 56 | Retrying StatusFilter = "retrying" 57 | StaleRetrying StatusFilter = "stale-retrying" 58 | All StatusFilter = "all" 59 | ) 60 | 61 | // Validate returns an error if a filter value is not valid. 62 | func (t StatusFilter) Validate() error { 63 | switch t { 64 | case InProgress, Pending, Stale, Completed, Retrying, All: 65 | return nil 66 | default: 67 | return errors.Errorf("invalid filter type '%s'", t) 68 | } 69 | } 70 | 71 | // ValidStatusFilters returns all valid status filters. 72 | func ValidStatusFilters() []StatusFilter { 73 | return []StatusFilter{Pending, InProgress, Stale, Completed, Retrying, All} 74 | } 75 | -------------------------------------------------------------------------------- /meta.go: -------------------------------------------------------------------------------- 1 | package amboy 2 | 3 | import ( 4 | "context" 5 | "time" 6 | 7 | "github.com/mongodb/grip" 8 | "github.com/mongodb/grip/message" 9 | "github.com/pkg/errors" 10 | ) 11 | 12 | // ResolveErrors takes a Queue and iterates over the completed Jobs' results, 13 | // returning a single aggregated error for all of the Queue's Jobs. 14 | func ResolveErrors(ctx context.Context, q Queue) error { 15 | catcher := grip.NewCatcher() 16 | 17 | for result := range q.Results(ctx) { 18 | if err := ctx.Err(); err != nil { 19 | catcher.Add(err) 20 | break 21 | } 22 | 23 | catcher.Add(result.Error()) 24 | } 25 | 26 | return catcher.Resolve() 27 | } 28 | 29 | // PopulateQueue adds Jobs from a channel to a Queue and returns an error with 30 | // the aggregated results of these operations. 31 | func PopulateQueue(ctx context.Context, q Queue, jobs <-chan Job) error { 32 | catcher := grip.NewCatcher() 33 | 34 | for j := range jobs { 35 | if err := ctx.Err(); err != nil { 36 | catcher.Add(err) 37 | break 38 | } 39 | 40 | catcher.Add(q.Put(ctx, j)) 41 | } 42 | 43 | return catcher.Resolve() 44 | } 45 | 46 | // QueueReport holds the IDs of Jobs in a Queue based on their current state. 47 | type QueueReport struct { 48 | Pending []string `json:"pending"` 49 | InProgress []string `json:"in_progress"` 50 | Completed []string `json:"completed"` 51 | Retrying []string `json:"retrying"` 52 | } 53 | 54 | // Report returns a QueueReport status for the state of a Queue. 55 | func Report(ctx context.Context, q Queue, limit int) QueueReport { 56 | var out QueueReport 57 | 58 | if limit == 0 { 59 | return out 60 | } 61 | 62 | var count int 63 | for info := range q.JobInfo(ctx) { 64 | switch { 65 | case info.Status.Completed: 66 | if info.Retry.ShouldRetry() { 67 | out.Retrying = append(out.Retrying, info.ID) 68 | } else { 69 | out.Completed = append(out.Completed, info.ID) 70 | } 71 | case info.Status.InProgress: 72 | out.InProgress = append(out.InProgress, info.ID) 73 | default: 74 | out.Pending = append(out.Pending, info.ID) 75 | } 76 | 77 | count++ 78 | if limit > 0 && count >= limit { 79 | break 80 | } 81 | 82 | } 83 | 84 | return out 85 | } 86 | 87 | // RunJob executes a single job directly, without a Queue, with similar 88 | // semantics as it would execute in a Queue: MaxTime is respected, and it uses 89 | // similar logging as is present in the queue, with errors propagated 90 | // functionally. 91 | func RunJob(ctx context.Context, job Job) error { 92 | var cancel context.CancelFunc 93 | ti := job.TimeInfo() 94 | ti.Start = time.Now() 95 | job.UpdateTimeInfo(ti) 96 | if ti.MaxTime > 0 { 97 | ctx, cancel = context.WithTimeout(ctx, ti.MaxTime) 98 | } else { 99 | ctx, cancel = context.WithCancel(ctx) 100 | } 101 | defer cancel() 102 | 103 | job.Run(ctx) 104 | ti.End = time.Now() 105 | msg := message.Fields{ 106 | "job_id": job.ID(), 107 | "job_type": job.Type().Name, 108 | "duration_secs": ti.Duration().Seconds(), 109 | } 110 | err := errors.WithStack(job.Error()) 111 | if err != nil { 112 | grip.Error(message.WrapError(err, msg)) 113 | } else { 114 | grip.Debug(msg) 115 | } 116 | 117 | return err 118 | } 119 | -------------------------------------------------------------------------------- /rest/service_queue_job_test.go: -------------------------------------------------------------------------------- 1 | package rest 2 | 3 | import ( 4 | "context" 5 | "encoding/json" 6 | "fmt" 7 | "net/http/httptest" 8 | "testing" 9 | 10 | "github.com/mongodb/amboy" 11 | "github.com/mongodb/amboy/job" 12 | "github.com/stretchr/testify/require" 13 | "github.com/stretchr/testify/suite" 14 | ) 15 | 16 | type JobStatusSuite struct { 17 | service *QueueService 18 | require *require.Assertions 19 | j amboy.Job 20 | jobName string 21 | closer context.CancelFunc 22 | suite.Suite 23 | } 24 | 25 | func TestJobStatusSuite(t *testing.T) { 26 | suite.Run(t, new(JobStatusSuite)) 27 | } 28 | 29 | func (s *JobStatusSuite) SetupSuite() { 30 | s.require = s.Require() 31 | s.service = NewQueueService() 32 | ctx, cancel := context.WithCancel(context.Background()) 33 | s.closer = cancel 34 | 35 | s.NoError(s.service.Open(ctx)) 36 | 37 | j := job.NewShellJob("echo foo", "") 38 | s.jobName = j.ID() 39 | s.NoError(s.service.queue.Put(ctx, j)) 40 | 41 | s.NoError(s.service.App().Resolve()) 42 | } 43 | 44 | func (s *JobStatusSuite) TearDownSuite() { 45 | s.closer() 46 | } 47 | 48 | func (s *JobStatusSuite) TestIncorrectOrInvalidJobNamesReturnExpectedResults() { 49 | ctx, cancel := context.WithCancel(context.Background()) 50 | defer cancel() 51 | 52 | for _, name := range []string{"", " ", "123", "DOES-NOT-EXIST", "foo"} { 53 | resp, err := s.service.getJobStatusResponse(ctx, name) 54 | s.Error(err) 55 | 56 | s.Equal(err.Error(), resp.Error) 57 | s.False(resp.Exists) 58 | s.False(resp.Completed) 59 | s.Equal(name, resp.ID) 60 | s.Nil(resp.Job) 61 | } 62 | } 63 | 64 | func (s *JobStatusSuite) TestJobNameReturnsSuccessfulResponse() { 65 | ctx, cancel := context.WithCancel(context.Background()) 66 | defer cancel() 67 | 68 | amboy.Wait(ctx, s.service.queue) 69 | 70 | resp, err := s.service.getJobStatusResponse(ctx, s.jobName) 71 | s.NoError(err) 72 | 73 | s.Equal("", resp.Error) 74 | s.True(resp.Exists) 75 | s.True(resp.Completed) 76 | s.Equal(s.jobName, resp.ID) 77 | } 78 | 79 | func (s *JobStatusSuite) TestRequestReturnsErrorInErrorConditions() { 80 | router, err := s.service.App().Handler() 81 | s.NoError(err) 82 | 83 | for _, name := range []string{"foo", "bar", "df-df"} { 84 | w := httptest.NewRecorder() 85 | req := httptest.NewRequest("GET", fmt.Sprintf("http://example.com/v1/job/status/%s", name), nil) 86 | 87 | router.ServeHTTP(w, req) 88 | 89 | s.Equal(400, w.Code) 90 | jst := jobStatusResponse{} 91 | err = json.Unmarshal(w.Body.Bytes(), &jst) 92 | s.NoError(err) 93 | 94 | s.Equal(name, jst.ID) 95 | s.False(jst.Exists) 96 | s.False(jst.Completed) 97 | s.True(jst.Error != "") 98 | } 99 | } 100 | 101 | func (s *JobStatusSuite) TestRequestValidJobStatus() { 102 | router, err := s.service.App().Handler() 103 | s.NoError(err) 104 | 105 | w := httptest.NewRecorder() 106 | req := httptest.NewRequest("GET", fmt.Sprintf("http://example.com/v1/job/status/%s", s.jobName), nil) 107 | 108 | router.ServeHTTP(w, req) 109 | 110 | s.Equal(200, w.Code) 111 | jst := jobStatusResponse{} 112 | s.NoError(json.Unmarshal(w.Body.Bytes(), &jst)) 113 | 114 | s.Equal(s.jobName, jst.ID) 115 | s.True(jst.Exists) 116 | s.True(jst.Completed) 117 | s.Equal("", jst.Error) 118 | } 119 | -------------------------------------------------------------------------------- /pool/local.go: -------------------------------------------------------------------------------- 1 | /* 2 | Local Workers Pool 3 | 4 | The LocalWorkers is a simple worker pool implementation that spawns a 5 | collection of (n) workers and dispatches jobs to worker threads, that 6 | consume work items from the Queue's Next() method. 7 | */ 8 | package pool 9 | 10 | import ( 11 | "context" 12 | "errors" 13 | "sync" 14 | 15 | "github.com/mongodb/amboy" 16 | "github.com/mongodb/grip" 17 | "github.com/mongodb/grip/recovery" 18 | ) 19 | 20 | // NewLocalWorkers is a constructor for pool of worker processes that 21 | // execute jobs from a queue locally, and takes arguments for 22 | // the number of worker processes and a amboy.Queue object. 23 | func NewLocalWorkers(numWorkers int, q amboy.Queue) amboy.Runner { 24 | r := &localWorkers{ 25 | queue: q, 26 | size: numWorkers, 27 | } 28 | 29 | if r.size <= 0 { 30 | grip.Infof("minimum pool size is 1, overriding invalid setting of %d", r.size) 31 | r.size = 1 32 | } 33 | 34 | return r 35 | } 36 | 37 | // localWorkers is a very minimal implementation of a worker pool, and 38 | // supports a configurable number of workers to process Jobs. 39 | type localWorkers struct { 40 | size int 41 | started bool 42 | wg sync.WaitGroup 43 | canceler context.CancelFunc 44 | queue amboy.Queue 45 | mu sync.RWMutex 46 | } 47 | 48 | // SetQueue allows callers to inject alternate amboy.Queue objects into 49 | // constructed Runner objects. Returns an error if the Runner has 50 | // started. 51 | func (r *localWorkers) SetQueue(q amboy.Queue) error { 52 | r.mu.Lock() 53 | defer r.mu.Unlock() 54 | if r.started { 55 | return errors.New("cannot change queue after starting a runner") 56 | } 57 | 58 | r.queue = q 59 | return nil 60 | } 61 | 62 | // Started returns true when the Runner has begun executing jobs. For 63 | // localWorkers this means that workers are running. 64 | func (r *localWorkers) Started() bool { 65 | r.mu.RLock() 66 | defer r.mu.RUnlock() 67 | return r.started 68 | } 69 | 70 | // Start initializes all worker process, and returns an error if the 71 | // Runner does not have a queue. 72 | func (r *localWorkers) Start(ctx context.Context) error { 73 | r.mu.Lock() 74 | defer r.mu.Unlock() 75 | 76 | if r.started { 77 | return nil 78 | } 79 | 80 | if r.queue == nil { 81 | return errors.New("runner must have an embedded queue") 82 | } 83 | 84 | workerCtx, cancel := context.WithCancel(ctx) 85 | r.canceler = cancel 86 | 87 | for w := 1; w <= r.size; w++ { 88 | go worker(workerCtx, "local", r.queue, &r.wg, &r.mu) 89 | grip.Debugf("started worker %d of %d waiting for jobs", w, r.size) 90 | } 91 | 92 | r.started = true 93 | grip.Debugf("running %d workers", r.size) 94 | 95 | return nil 96 | } 97 | 98 | // Close terminates all worker processes as soon as possible. 99 | func (r *localWorkers) Close(ctx context.Context) { 100 | r.mu.Lock() 101 | defer r.mu.Unlock() 102 | 103 | if r.canceler != nil { 104 | r.canceler() 105 | r.canceler = nil 106 | r.started = false 107 | } 108 | 109 | wait := make(chan struct{}) 110 | go func() { 111 | defer recovery.LogStackTraceAndContinue("waiting for close") 112 | defer close(wait) 113 | r.wg.Wait() 114 | }() 115 | 116 | select { 117 | case <-ctx.Done(): 118 | case <-wait: 119 | } 120 | } 121 | -------------------------------------------------------------------------------- /registry/job_test.go: -------------------------------------------------------------------------------- 1 | package registry 2 | 3 | import ( 4 | "fmt" 5 | "sync" 6 | "testing" 7 | 8 | "github.com/mongodb/amboy" 9 | "github.com/mongodb/grip" 10 | "github.com/mongodb/grip/level" 11 | "github.com/stretchr/testify/suite" 12 | ) 13 | 14 | // AmboyJobRegistrySuite tests the amboy job registry resource, which is 15 | // internal to the amboy package, but has an isolated interface for 16 | // client code that writes amboy jobs to support serialization and 17 | // de-serialization of jobs. 18 | type AmboyJobRegistrySuite struct { 19 | registry *typeRegistry 20 | suite.Suite 21 | } 22 | 23 | func TestAmboyJobRegistryResources(t *testing.T) { 24 | suite.Run(t, new(AmboyJobRegistrySuite)) 25 | } 26 | 27 | func (s *AmboyJobRegistrySuite) SetupSuite() { 28 | lvl := grip.GetSender().Level() 29 | lvl.Threshold = level.Emergency 30 | s.NoError(grip.GetSender().SetLevel(lvl)) 31 | } 32 | 33 | func (s *AmboyJobRegistrySuite) SetupTest() { 34 | s.registry = newTypeRegistry() 35 | s.Len(s.registry.job.m, 0) 36 | } 37 | 38 | func exampleJobFactory() amboy.Job { 39 | return (amboy.Job)(nil) 40 | } 41 | 42 | func groupJobFactory() amboy.Job { 43 | return (amboy.Job)(nil) 44 | } 45 | 46 | func (s *AmboyJobRegistrySuite) TestRegisterNewJobTypePersists() { 47 | s.registry.registerJobType("group_one", exampleJobFactory) 48 | s.Len(s.registry.job.m, 1) 49 | 50 | _, err := s.registry.getJobFactory("group_one") 51 | s.NoError(err) 52 | 53 | _, err = s.registry.getJobFactory("group") 54 | s.Error(err) 55 | 56 | s.Len(s.registry.job.m, 1) 57 | } 58 | 59 | func (s *AmboyJobRegistrySuite) TestJobsHaveUniqueNames() { 60 | s.registry.registerJobType("group", groupJobFactory) 61 | s.registry.registerJobType("group", groupJobFactory) 62 | s.registry.registerJobType("group", groupJobFactory) 63 | s.Len(s.registry.job.m, 1) 64 | } 65 | 66 | func (s *AmboyJobRegistrySuite) TestConcurrentAccess() { 67 | // this is a little simulation to test a moderate number 68 | // threads doing read and write access on a registry object. 69 | wg := &sync.WaitGroup{} 70 | 71 | num := 128 72 | for i := 0; i < num; i++ { 73 | wg.Add(1) 74 | go func(n int) { 75 | name := fmt.Sprintf("worker-%d", n) 76 | 77 | for f := 1; f < 12; f++ { 78 | s.registry.registerJobType(name, groupJobFactory) 79 | s.registry.registerJobType(name, groupJobFactory) 80 | } 81 | 82 | var err error 83 | 84 | for l := 1; l < 12; l++ { 85 | _, err = s.registry.getJobFactory(name) 86 | s.NoError(err) 87 | _, err = s.registry.getJobFactory(fmt.Sprintf("%s-%d-%d", name, n, l)) 88 | s.Error(err) 89 | // throw in another option to contend on the lock 90 | s.registry.registerJobType(name, groupJobFactory) 91 | } 92 | wg.Done() 93 | }(i) 94 | } 95 | wg.Wait() 96 | 97 | s.registry.job.l.RLock() 98 | defer s.registry.job.l.RUnlock() 99 | 100 | s.Len(s.registry.job.m, num, 101 | fmt.Sprintf("%d jobs in registry, %d expected", len(s.registry.job.m), num)) 102 | } 103 | 104 | func (s *AmboyJobRegistrySuite) TestRegistryFactoriesProduceTypesWithMatchingNames() { 105 | amboyRegistry.job.l.RLock() 106 | defer amboyRegistry.job.l.RUnlock() 107 | 108 | for name, factory := range amboyRegistry.job.m { 109 | job := factory() 110 | s.Equal(name, job.Type().Name) 111 | } 112 | } 113 | -------------------------------------------------------------------------------- /pool/runner_test.go: -------------------------------------------------------------------------------- 1 | package pool 2 | 3 | import ( 4 | "context" 5 | "testing" 6 | "time" 7 | 8 | "github.com/VividCortex/ewma" 9 | "github.com/mongodb/amboy" 10 | "github.com/mongodb/grip" 11 | "github.com/mongodb/grip/level" 12 | "github.com/mongodb/grip/send" 13 | "github.com/stretchr/testify/assert" 14 | ) 15 | 16 | func init() { 17 | sender := grip.GetSender() 18 | lvl := send.LevelInfo{ 19 | Threshold: level.Alert, 20 | Default: level.Warning, 21 | } 22 | grip.Warning(sender.SetLevel(lvl)) 23 | } 24 | 25 | type poolFactory func() amboy.Runner 26 | type testCaseFunc func(*testing.T) 27 | 28 | func makeTestQueue(pool amboy.Runner) amboy.Queue { 29 | return &QueueTester{ 30 | pool: pool, 31 | toProcess: make(chan amboy.Job), 32 | storage: make(map[string]amboy.Job), 33 | } 34 | 35 | } 36 | 37 | func TestRunnerImplementations(t *testing.T) { 38 | pools := map[string]func() amboy.Runner{ 39 | "Local": func() amboy.Runner { return new(localWorkers) }, 40 | "Noop": func() amboy.Runner { return new(noopPool) }, 41 | "RateLimitedAverage": func() amboy.Runner { 42 | return &ewmaRateLimiting{ 43 | size: 1, 44 | period: time.Second, 45 | target: 5, 46 | ewma: ewma.NewMovingAverage(), 47 | } 48 | }, 49 | "Abortable": func() amboy.Runner { 50 | return &abortablePool{ 51 | size: 1, 52 | jobs: make(map[string]context.CancelFunc), 53 | } 54 | }, 55 | } 56 | cases := map[string]func(poolFactory) testCaseFunc{ 57 | "NotStarted": func(factory poolFactory) testCaseFunc { 58 | return func(t *testing.T) { 59 | pool := factory() 60 | assert.False(t, pool.Started()) 61 | } 62 | }, 63 | "MutableQueue": func(factory poolFactory) testCaseFunc { 64 | return func(t *testing.T) { 65 | pool := factory() 66 | queue := makeTestQueue(pool) 67 | 68 | ctx, cancel := context.WithCancel(context.Background()) 69 | defer cancel() 70 | 71 | // it's an unconfigured runner without a queue, it should always error 72 | assert.Error(t, pool.Start(ctx)) 73 | 74 | // this should start the queue 75 | assert.NoError(t, pool.SetQueue(queue)) 76 | 77 | // it's cool to start the runner 78 | assert.NoError(t, pool.Start(ctx)) 79 | 80 | // once the runner starts you can't add pools 81 | assert.Error(t, pool.SetQueue(queue)) 82 | 83 | // subsequent calls to start should noop 84 | assert.NoError(t, pool.Start(ctx)) 85 | } 86 | }, 87 | "CloseImpactsStateAsExpected": func(factory poolFactory) testCaseFunc { 88 | return func(t *testing.T) { 89 | pool := factory() 90 | queue := makeTestQueue(pool) 91 | assert.False(t, pool.Started()) 92 | 93 | ctx, cancel := context.WithCancel(context.Background()) 94 | defer cancel() 95 | 96 | assert.False(t, pool.Started()) 97 | assert.NoError(t, pool.SetQueue(queue)) 98 | assert.NoError(t, pool.Start(ctx)) 99 | assert.True(t, pool.Started()) 100 | 101 | assert.NotPanics(t, func() { 102 | pool.Close(ctx) 103 | }) 104 | 105 | assert.False(t, pool.Started()) 106 | } 107 | }, 108 | } 109 | 110 | for poolName, factory := range pools { 111 | t.Run(poolName, func(t *testing.T) { 112 | for caseName, test := range cases { 113 | t.Run(poolName+caseName, test(factory)) 114 | } 115 | }) 116 | } 117 | } 118 | -------------------------------------------------------------------------------- /periodic_group.go: -------------------------------------------------------------------------------- 1 | package amboy 2 | 3 | import ( 4 | "context" 5 | "time" 6 | 7 | "github.com/mongodb/grip" 8 | "github.com/mongodb/grip/message" 9 | "github.com/mongodb/grip/recovery" 10 | "github.com/pkg/errors" 11 | ) 12 | 13 | // GroupQueueOperation describes a single queue population operation 14 | // for a group queue. 15 | type GroupQueueOperation struct { 16 | Operation QueueOperation 17 | Queue string 18 | Check func(context.Context) bool 19 | } 20 | 21 | // IntervalGroupQueueOperation schedules jobs on a queue group with 22 | // similar semantics as IntervalQueueOperation. 23 | // 24 | // Operations will continue to run as long as the context is not 25 | // canceled. If you do not pass any GroupQueueOperation items to this 26 | // function, it panics. 27 | func IntervalGroupQueueOperation(ctx context.Context, qg QueueGroup, interval time.Duration, startAt time.Time, conf QueueOperationConfig, ops ...GroupQueueOperation) { 28 | if len(ops) == 0 { 29 | grip.Error("queue group operation must contain at least one operation") 30 | return 31 | } 32 | 33 | go func() { 34 | var err error 35 | 36 | if interval <= time.Microsecond { 37 | grip.Criticalf("interval for queue group operation '%s' must be greater than a microsecond", interval) 38 | return 39 | } 40 | 41 | defer func() { 42 | err = recovery.HandlePanicWithError(recover(), err, "interval background job scheduler") 43 | 44 | if err != nil { 45 | if !conf.ContinueOnError { 46 | return 47 | } 48 | 49 | if ctx.Err() != nil { 50 | return 51 | } 52 | 53 | IntervalGroupQueueOperation(ctx, qg, interval, startAt, conf, ops...) 54 | } 55 | }() 56 | 57 | waitUntilInterval(ctx, startAt, interval) 58 | 59 | ticker := time.NewTicker(interval) 60 | defer ticker.Stop() 61 | 62 | if ctx.Err() != nil { 63 | return 64 | } 65 | count := 1 66 | for _, op := range ops { 67 | if op.Check == nil || op.Check(ctx) { 68 | if err = scheduleGroupOp(ctx, qg, op, conf); err != nil { 69 | return 70 | } 71 | } 72 | } 73 | 74 | for { 75 | select { 76 | case <-ctx.Done(): 77 | grip.InfoWhen(conf.DebugLogging, message.Fields{ 78 | "message": "exiting interval job scheduler", 79 | "queue": "group", 80 | "num_intervals": count, 81 | "reason": "operation canceled", 82 | "conf": conf, 83 | }) 84 | return 85 | case <-ticker.C: 86 | for _, op := range ops { 87 | if err := scheduleGroupOp(ctx, qg, op, conf); err != nil && !conf.ContinueOnError { 88 | return 89 | } 90 | } 91 | 92 | count++ 93 | } 94 | } 95 | }() 96 | } 97 | 98 | func scheduleGroupOp(ctx context.Context, group QueueGroup, op GroupQueueOperation, conf QueueOperationConfig) error { 99 | if op.Check == nil || op.Check(ctx) { 100 | q, err := group.Get(ctx, op.Queue) 101 | if err != nil { 102 | if conf.ContinueOnError { 103 | grip.WarningWhen(conf.LogErrors, err) 104 | return nil 105 | } 106 | 107 | grip.CriticalWhen(conf.LogErrors, err) 108 | return errors.Wrapf(err, "getting queue '%s' from group", op.Queue) 109 | } 110 | 111 | if err = scheduleOp(ctx, q, op.Operation, conf); err != nil { 112 | return errors.Wrapf(err, "scheduling job on group queue '%s'", op.Queue) 113 | } 114 | } 115 | return nil 116 | } 117 | -------------------------------------------------------------------------------- /queue/limited_test.go: -------------------------------------------------------------------------------- 1 | package queue 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "testing" 7 | "time" 8 | 9 | "github.com/mongodb/amboy/job" 10 | "github.com/mongodb/amboy/pool" 11 | "github.com/stretchr/testify/require" 12 | "github.com/stretchr/testify/suite" 13 | ) 14 | 15 | // limitedSizeQueueSuite suite tests the queue implementation that 16 | // uses the CappedResultStorage. These tests exercise the aspects of 17 | // this queue implementation that are not covered by the tests of the 18 | // storage object *or* exercised by the general queue functionality 19 | // tests, which test all implementations. 20 | type limitedSizeQueueSuite struct { 21 | queue *limitedSizeLocal 22 | numWorkers int 23 | numCapacity int 24 | require *require.Assertions 25 | suite.Suite 26 | } 27 | 28 | func TestLimitedSizeQueueSuite(t *testing.T) { 29 | suite.Run(t, new(limitedSizeQueueSuite)) 30 | } 31 | 32 | func (s *limitedSizeQueueSuite) SetupSuite() { 33 | s.numWorkers = 2 34 | s.numCapacity = 100 35 | s.require = s.Require() 36 | } 37 | 38 | func (s *limitedSizeQueueSuite) SetupTest() { 39 | s.queue = NewLocalLimitedSize(s.numWorkers, s.numCapacity).(*limitedSizeLocal) 40 | } 41 | 42 | func (s *limitedSizeQueueSuite) TestBufferForPendingWorkEqualToCapacityForResults() { 43 | ctx, cancel := context.WithTimeout(context.Background(), 100*time.Millisecond) 44 | defer cancel() 45 | 46 | s.False(s.queue.Info().Started) 47 | s.queue.Runner().Close(ctx) 48 | s.False(s.queue.Info().Started) 49 | s.Error(s.queue.Put(ctx, job.NewShellJob("sleep 10", ""))) 50 | 51 | s.NoError(s.queue.Start(ctx)) 52 | s.require.True(s.queue.Info().Started) 53 | for i := 0; i < 100*s.numCapacity*s.numWorkers; i++ { 54 | var outcome bool 55 | err := s.queue.Put(ctx, job.NewShellJob("sleep 10", "")) 56 | if i < s.numWorkers+s.numCapacity { 57 | outcome = s.NoError(err, "idx=%d stat=%+v", i, s.queue.Stats(ctx)) 58 | } else { 59 | outcome = s.Error(err, "idx=%d", i) 60 | } 61 | 62 | if !outcome { 63 | break 64 | } 65 | } 66 | 67 | s.Len(s.queue.channel, s.numCapacity) 68 | s.True(len(s.queue.storage) == s.numCapacity+s.numWorkers, fmt.Sprintf("storage=%d", len(s.queue.storage))) 69 | s.Error(s.queue.Put(ctx, job.NewShellJob("sleep 10", ""))) 70 | s.True(len(s.queue.storage) == s.numCapacity+s.numWorkers, fmt.Sprintf("storage=%d", len(s.queue.storage))) 71 | s.Len(s.queue.channel, s.numCapacity) 72 | } 73 | 74 | func (s *limitedSizeQueueSuite) TestCallingStartMultipleTimesDoesNotImpactState() { 75 | s.False(s.queue.Info().Started) 76 | ctx := context.Background() 77 | s.NoError(s.queue.Start(ctx)) 78 | s.True(s.queue.Info().Started) 79 | 80 | for i := 0; i < 100; i++ { 81 | s.Error(s.queue.Start(ctx)) 82 | } 83 | 84 | s.True(s.queue.Info().Started) 85 | } 86 | 87 | func (s *limitedSizeQueueSuite) TestCannotSetRunnerAfterQueueIsOpened() { 88 | secondRunner := pool.NewAbortablePool(1, s.queue) 89 | runner := s.queue.runner 90 | 91 | s.False(s.queue.Info().Started) 92 | for i := 0; i < 25; i++ { 93 | s.NoError(s.queue.SetRunner(secondRunner)) 94 | s.NoError(s.queue.SetRunner(runner)) 95 | } 96 | s.False(s.queue.Info().Started) 97 | 98 | ctx := context.Background() 99 | s.NoError(s.queue.Start(ctx)) 100 | 101 | s.True(s.queue.Info().Started) 102 | 103 | for i := 0; i < 30; i++ { 104 | s.Error(s.queue.SetRunner(secondRunner)) 105 | s.Error(s.queue.SetRunner(runner)) 106 | } 107 | } 108 | -------------------------------------------------------------------------------- /stats.go: -------------------------------------------------------------------------------- 1 | package amboy 2 | 3 | import ( 4 | "fmt" 5 | 6 | "github.com/mongodb/grip/level" 7 | "github.com/mongodb/grip/message" 8 | "github.com/pkg/errors" 9 | ) 10 | 11 | // QueueStats is a simple structure that the Stats() method in the 12 | // Queue interface returns and tracks the state of the queue, and 13 | // provides a common format for different Queue implementations to 14 | // report on their state. 15 | // 16 | // Implements grip's message.Composer interface when passed as a 17 | // pointer. 18 | type QueueStats struct { 19 | Running int `bson:"running" json:"running" yaml:"running"` 20 | Completed int `bson:"completed" json:"completed" yaml:"completed"` 21 | Retrying int `bson:"retrying" json:"retrying" yaml:"retrying"` 22 | Pending int `bson:"pending" json:"pending" yaml:"pending"` 23 | Blocked int `bson:"blocked" json:"blocked" yaml:"blocked"` 24 | Total int `bson:"total" json:"total" yaml:"total"` 25 | Context message.Fields `bson:"context,omitempty" json:"context,omitempty" yaml:"context,omitempty"` 26 | 27 | priority level.Priority 28 | } 29 | 30 | // String prints a long form report of the queue for human consumption. 31 | func (s QueueStats) String() string { 32 | return fmt.Sprintf("running='%d', completed='%d', retrying='%d' pending='%d', blocked='%d', total='%d'", 33 | s.Running, s.Completed, s.Retrying, s.Pending, s.Blocked, s.Total) 34 | } 35 | 36 | // IsComplete returns true when the total number of jobs are equal to 37 | // the number completed, or if the number of completed and blocked are 38 | // greater than or equal to total. For retryable queues, if any job is retrying, 39 | // it is not considered complete. This method is used by the Wait functions to 40 | // determine when a queue has completed all actionable work. 41 | func (s QueueStats) IsComplete() bool { 42 | if s.Retrying > 0 { 43 | return false 44 | } 45 | 46 | if s.Total == s.Completed { 47 | return true 48 | } 49 | 50 | if s.Total <= s.Completed+s.Blocked { 51 | return true 52 | } 53 | 54 | return false 55 | } 56 | 57 | // Loggable is part of the grip/message.Composer interface and only 58 | // returns true if the queue has at least one job. 59 | func (s QueueStats) Loggable() bool { return s.Total > 0 } 60 | 61 | // Raw is part of the grip/message.Composer interface and simply 62 | // returns the QueueStats object. 63 | func (s QueueStats) Raw() interface{} { return s } 64 | 65 | // Priority is part of the grip/message.Composer interface and returns 66 | // the priority of the message. 67 | func (s QueueStats) Priority() level.Priority { return s.priority } 68 | 69 | // SetPriority is part of the grip/message.Composer interface and 70 | // allows the caller to configure the piroity of the message. 71 | func (s *QueueStats) SetPriority(l level.Priority) error { 72 | if !l.IsValid() { 73 | return errors.Errorf("%s (%d) is not a valid level", l, l) 74 | } 75 | s.priority = l 76 | return nil 77 | } 78 | 79 | // Annotate is part of the grip/message.Composer interface and allows 80 | // the logging infrastructure to inject content and context into log 81 | // messages. 82 | func (s *QueueStats) Annotate(key string, value interface{}) error { 83 | if s.Context == nil { 84 | s.Context = message.Fields{ 85 | key: value, 86 | } 87 | 88 | return nil 89 | } 90 | 91 | if _, ok := s.Context[key]; ok { 92 | return fmt.Errorf("key '%s' already exists", key) 93 | } 94 | 95 | s.Context[key] = value 96 | 97 | return nil 98 | } 99 | -------------------------------------------------------------------------------- /rest/service_abortable_pool.go: -------------------------------------------------------------------------------- 1 | package rest 2 | 3 | import ( 4 | "net/http" 5 | 6 | "github.com/evergreen-ci/gimlet" 7 | "github.com/mongodb/amboy" 8 | "github.com/pkg/errors" 9 | ) 10 | 11 | // AbortablePoolManagementService defines a set of rest routes that make it 12 | // possible to remotely manage the jobs running in an abortable pool. 13 | type AbortablePoolManagementService struct { 14 | pool amboy.AbortableRunner 15 | } 16 | 17 | // NewAbortablePoolManagementService returns a service that defines REST routes 18 | // can manage an abortable pool. 19 | func NewAbortablePoolManagementService(p amboy.AbortableRunner) *AbortablePoolManagementService { 20 | return &AbortablePoolManagementService{ 21 | pool: p, 22 | } 23 | } 24 | 25 | // App returns a gimlet app with all of the routes registered. 26 | func (s *AbortablePoolManagementService) App() *gimlet.APIApp { 27 | app := gimlet.NewApp() 28 | 29 | app.AddRoute("/jobs/list").Version(1).Get().Handler(s.ListJobs) 30 | app.AddRoute("/jobs/abort").Version(1).Delete().Handler(s.AbortAllJobs) 31 | app.AddRoute("/job/{name}").Version(1).Get().Handler(s.GetJobStatus) 32 | app.AddRoute("/job/{name}").Version(1).Delete().Handler(s.AbortRunningJob) 33 | 34 | return app 35 | } 36 | 37 | // ListJobs is an http.HandlerFunc that returns a list of all running 38 | // jobs in the pool. 39 | func (s *AbortablePoolManagementService) ListJobs(rw http.ResponseWriter, r *http.Request) { 40 | jobs := s.pool.RunningJobs() 41 | 42 | gimlet.WriteJSON(rw, jobs) 43 | } 44 | 45 | // AbortAllJobs is an http.HandlerFunc that sends the signal to abort 46 | // all running jobs in the pool. May return a 408 (timeout) if the 47 | // calling context was canceled before the operation 48 | // returned. Otherwise, this handler returns 200. The body of the 49 | // response is always empty. 50 | func (s *AbortablePoolManagementService) AbortAllJobs(rw http.ResponseWriter, r *http.Request) { 51 | ctx := r.Context() 52 | 53 | if err := s.pool.AbortAll(ctx); err != nil { 54 | if ctx.Err() != nil { 55 | gimlet.WriteJSONResponse(rw, http.StatusRequestTimeout, struct{}{}) 56 | return 57 | } 58 | gimlet.WriteJSONInternalError(rw, err.Error()) 59 | return 60 | } 61 | 62 | gimlet.WriteJSON(rw, struct{}{}) 63 | } 64 | 65 | // GetJobStatus is an http.HandlerFunc reports on the status (running 66 | // or not running) of a specific job. 67 | func (s *AbortablePoolManagementService) GetJobStatus(rw http.ResponseWriter, r *http.Request) { 68 | name := gimlet.GetVars(r)["name"] 69 | 70 | if !s.pool.IsRunning(name) { 71 | gimlet.WriteJSONResponse(rw, http.StatusNotFound, 72 | map[string]string{ 73 | "name": name, 74 | "status": "not running", 75 | }) 76 | return 77 | } 78 | 79 | gimlet.WriteJSON(rw, map[string]string{ 80 | "name": name, 81 | "status": "running", 82 | }) 83 | } 84 | 85 | // AbortRunningJob is an http.HandlerFunc that terminates the 86 | // execution of a single running job, returning a 400 response when 87 | // the job doesn't exist. 88 | func (s *AbortablePoolManagementService) AbortRunningJob(rw http.ResponseWriter, r *http.Request) { 89 | name := gimlet.GetVars(r)["name"] 90 | ctx := r.Context() 91 | err := s.pool.Abort(ctx, name) 92 | if err != nil { 93 | gimlet.WriteResponse(rw, gimlet.MakeJSONErrorResponder(errors.Wrapf(err, 94 | "problem aborting job '%s'", name))) 95 | } 96 | 97 | gimlet.WriteJSON(rw, map[string]string{ 98 | "name": name, 99 | "status": "aborted", 100 | }) 101 | } 102 | -------------------------------------------------------------------------------- /queue/scope.go: -------------------------------------------------------------------------------- 1 | package queue 2 | 3 | import ( 4 | "sync" 5 | 6 | "github.com/mongodb/amboy" 7 | "github.com/pkg/errors" 8 | ) 9 | 10 | // ScopeManager provides a service to queue implementations to support 11 | // additional locking semantics for queues that cannot push that into 12 | // their backing storage. 13 | type ScopeManager interface { 14 | Acquire(owner string, scopes []string) error 15 | Release(owner string, scopes []string) error 16 | ReleaseAndAcquire(ownerToRelease string, scopesToRelease []string, ownerToAcquire string, scopesToAcquire []string) error 17 | } 18 | 19 | type scopeManagerImpl struct { 20 | mutex sync.Mutex 21 | scopes map[string]string 22 | } 23 | 24 | // NewLocalScopeManager constructs a ScopeManager implementation 25 | // suitable for use in most local (in memory) queue implementations. 26 | func NewLocalScopeManager() ScopeManager { 27 | return &scopeManagerImpl{ 28 | scopes: map[string]string{}, 29 | } 30 | } 31 | 32 | func (s *scopeManagerImpl) Acquire(id string, scopes []string) error { 33 | if len(scopes) == 0 { 34 | return nil 35 | } 36 | 37 | s.mutex.Lock() 38 | defer s.mutex.Unlock() 39 | 40 | var scopesToAcquire []string 41 | for _, sc := range scopes { 42 | holder, ok := s.scopes[sc] 43 | if !ok { 44 | scopesToAcquire = append(scopesToAcquire, sc) 45 | continue 46 | } 47 | 48 | if holder == id { 49 | continue 50 | } 51 | return amboy.NewDuplicateJobScopeErrorf("scope '%s' is already held by '%s', not '%s'", sc, holder, id) 52 | } 53 | 54 | for _, sc := range scopesToAcquire { 55 | s.scopes[sc] = id 56 | } 57 | 58 | return nil 59 | } 60 | 61 | func (s *scopeManagerImpl) Release(id string, scopes []string) error { 62 | if len(scopes) == 0 { 63 | return nil 64 | } 65 | 66 | s.mutex.Lock() 67 | defer s.mutex.Unlock() 68 | 69 | toRelease, err := s.getScopesToRelease(id, scopes) 70 | if err != nil { 71 | return errors.Wrap(err, "getting scopes to release") 72 | } 73 | 74 | for _, sc := range toRelease { 75 | delete(s.scopes, sc) 76 | } 77 | 78 | return nil 79 | } 80 | 81 | func (s *scopeManagerImpl) getScopesToRelease(id string, scopes []string) ([]string, error) { 82 | var scopesToRelease []string 83 | for _, sc := range scopes { 84 | holder, ok := s.scopes[sc] 85 | if !ok { 86 | continue 87 | } 88 | if holder == id { 89 | scopesToRelease = append(scopesToRelease, sc) 90 | continue 91 | } 92 | return nil, errors.Errorf("'%s' cannot release scope '%s' because it is owned by '%s'", id, sc, holder) 93 | } 94 | return scopesToRelease, nil 95 | } 96 | 97 | func (s *scopeManagerImpl) ReleaseAndAcquire(ownerToRelease string, scopesToRelease []string, ownerToAcquire string, scopesToAcquire []string) error { 98 | if len(scopesToRelease) == 0 && len(scopesToAcquire) == 0 { 99 | return nil 100 | } 101 | 102 | s.mutex.Lock() 103 | defer s.mutex.Unlock() 104 | 105 | toRelease, err := s.getScopesToRelease(ownerToRelease, scopesToRelease) 106 | if err != nil { 107 | return errors.Wrap(err, "getting scopes to release") 108 | } 109 | 110 | var toAcquire []string 111 | for _, sc := range scopesToAcquire { 112 | holder, ok := s.scopes[sc] 113 | if !ok || holder == ownerToRelease { 114 | toAcquire = append(toAcquire, sc) 115 | continue 116 | } 117 | if holder == ownerToAcquire { 118 | continue 119 | } 120 | return amboy.NewDuplicateJobScopeErrorf("scope '%s' is already held by '%s', which is neither '%s' nor '%s'", sc, holder, ownerToAcquire, ownerToRelease) 121 | } 122 | 123 | for _, sc := range toRelease { 124 | delete(s.scopes, sc) 125 | } 126 | 127 | for _, sc := range toAcquire { 128 | s.scopes[sc] = ownerToAcquire 129 | } 130 | 131 | return nil 132 | } 133 | -------------------------------------------------------------------------------- /job/group_test.go: -------------------------------------------------------------------------------- 1 | package job 2 | 3 | import ( 4 | "context" 5 | "strings" 6 | "testing" 7 | 8 | "github.com/mongodb/amboy" 9 | "github.com/mongodb/amboy/dependency" 10 | "github.com/mongodb/amboy/registry" 11 | "github.com/stretchr/testify/suite" 12 | ) 13 | 14 | func init() { 15 | RegisterDefaultJobs() 16 | } 17 | 18 | // JobGroupSuite exercises the Job implementation that allows you to 19 | // run multiple jobs in a worker pool as part of a single isolated 20 | // job. This is good exercise for the JobInterchange code and 21 | // requires some type fidelity of the interchange system. 22 | type JobGroupSuite struct { 23 | job *Group 24 | suite.Suite 25 | } 26 | 27 | func TestJobGroupSuite(t *testing.T) { 28 | // t.Skip("problems with interchange conversion in the context of group jobs") 29 | suite.Run(t, new(JobGroupSuite)) 30 | } 31 | 32 | func (s *JobGroupSuite) SetupTest() { 33 | s.job = NewGroup("group") 34 | } 35 | 36 | func (s *JobGroupSuite) TestJobFactoryAndConstructorHaveIdenticalTypeInformation() { 37 | fj, err := registry.GetJobFactory("group") 38 | s.NoError(err) 39 | 40 | j := fj() 41 | s.Equal(s.job.Type().Name, j.Type().Name) 42 | s.Equal(s.job.Type().Version, j.Type().Version) 43 | 44 | s.IsType(s.job, j) 45 | } 46 | 47 | func (s *JobGroupSuite) TestGroupAddMethodRequiresUniqueNames() { 48 | job := NewShellJob("touch foo", "foo") 49 | 50 | s.NoError(s.job.Add(job)) 51 | s.Error(s.job.Add(job)) 52 | 53 | job = NewShellJob("touch bar", "bar") 54 | s.NoError(s.job.Add(job)) 55 | 56 | s.Len(s.job.Jobs, 2) 57 | } 58 | 59 | func (s *JobGroupSuite) TestAllJobsAreCompleteAfterRunningGroup() { 60 | names := []string{"a", "b", "c", "d", "e", "f"} 61 | for _, name := range names { 62 | s.NoError(s.job.Add(NewShellJob("echo "+name, ""))) 63 | } 64 | s.Len(s.job.Jobs, len(names)) 65 | 66 | s.job.Run(context.Background()) 67 | s.True(s.job.Status().Completed) 68 | s.NoError(s.job.Error()) 69 | 70 | for _, interchange := range s.job.Jobs { 71 | s.True(interchange.Status.Completed) 72 | 73 | job, err := interchange.Resolve(amboy.JSON) 74 | s.NoError(err) 75 | s.True(job.Status().Completed) 76 | s.IsType(&ShellJob{}, job) 77 | } 78 | } 79 | 80 | func (s *JobGroupSuite) TestJobResultsPersistAfterGroupRuns() { 81 | // this tests runs two jobs, one that will fail and produce 82 | // errors, and one that runs normally, and we want to be able 83 | // to see that we can retrieve the results reasonably. 84 | 85 | s.NoError(s.job.Add(NewShellJob("true", ""))) 86 | fail := NewShellJob("false", "") 87 | fail.Env["name"] = "fail" 88 | 89 | s.NoError(s.job.Add(fail)) 90 | s.Len(s.job.Jobs, 2) 91 | 92 | s.job.Run(context.Background()) 93 | s.True(s.job.Status().Completed) 94 | s.Error(s.job.Error()) 95 | 96 | s.False(fail.Status().Completed) 97 | failEnvName, ok := fail.Env["name"] 98 | s.True(ok) 99 | s.Equal("fail", failEnvName) 100 | 101 | interchange, exists := s.job.Jobs[fail.ID()] 102 | s.True(exists) 103 | 104 | job, err := interchange.Resolve(amboy.JSON) 105 | s.NoError(err) 106 | s.False(job.Status().Completed) 107 | s.IsType(&ShellJob{}, job) 108 | } 109 | 110 | func (s *JobGroupSuite) TestJobIdReturnsUniqueString() { 111 | name := "foo" 112 | for i := 0; i < 20; i++ { 113 | job := NewGroup(name) 114 | 115 | id := job.ID() 116 | s.True(strings.HasPrefix(id, name), id) 117 | } 118 | } 119 | 120 | func (s *JobGroupSuite) TestJobGroupReturnsAlwaysDependency() { 121 | s.Equal(s.job.Dependency().Type().Name, "always") 122 | s.Equal(s.job.Dependency(), dependency.NewAlways()) 123 | } 124 | 125 | func (s *JobGroupSuite) TestJobGroupSetIsANoOp() { 126 | s.Equal(s.job.Dependency().Type().Name, "always") 127 | s.job.SetDependency(dependency.MakeLocalFile()) 128 | s.Equal(s.job.Dependency().Type().Name, "always") 129 | } 130 | -------------------------------------------------------------------------------- /doc.go: -------------------------------------------------------------------------------- 1 | /* 2 | Package amboy provides basic infrastructure for running and describing 3 | jobs and job workflows with, potentially, minimal overhead and 4 | additional complexity. 5 | 6 | # Overview and Motivation 7 | 8 | Amboy works with 4 basic logical objects: jobs representing work; 9 | runners, which are responsible for executing jobs; queues, 10 | that represent pipelines and offline workflows of jobs (i.e. not real 11 | time, processes that run outside of the primary execution path of a 12 | program); and dependencies that represent relationships between jobs. 13 | 14 | The inspiration for amboy was to be able to provide a unified way to 15 | define and run jobs, that would feel equally "native" for distributed 16 | applications and distributed web application, and move easily between 17 | different architectures. 18 | 19 | While amboy users will generally implement their own Job and 20 | dependency implementations, Amboy itself provides several example 21 | Queue implementations, as well as several generic examples and 22 | prototypes of Job and dependency.Manager objects. 23 | 24 | Generally speaking you should be able to use included amboy components 25 | to provide the queue and runner components, in conjunction with custom 26 | and generic job and dependency variations. 27 | 28 | Consider the following example: 29 | 30 | queue := queue.NewLocalLimitedSize(12, 50) // pass the number of workers and max capacity 31 | job := job.NewShellJob("make compile") 32 | 33 | err := queue.Put(job) 34 | if err != nil { 35 | // handle error case 36 | } 37 | 38 | err = queue.Start(ctx) // the queue starts a SimpleRunner object and 39 | // creates required channels. 40 | if err != nil { 41 | // handle error case 42 | } 43 | defer queue.Close() // Waits for all jobs to finish and releases all resources. 44 | 45 | amboy.Wait(ctx, queue) // Waits for all jobs to finish. 46 | */ 47 | package amboy 48 | 49 | // This file is intentionally documentation only. 50 | 51 | // The following content is intentionally excluded from godoc, but is 52 | // a reference for maintainers. 53 | 54 | /* 55 | Code Organization 56 | 57 | For the most part, the amboy package itself contains a few basic types 58 | and interfaces, and then several sub-packages are responsible for 59 | providing implementations and infrastructure to support these systems 60 | and interactions. The sub-package are quite small and intentionally 61 | isolated to make it easier to test and also avoid unintentional 62 | dependencies between the implementations of various components. 63 | 64 | Consider the following component packages: 65 | 66 | Registry 67 | 68 | The registry provides a way to declare job and dependency types so 69 | that Queue implementations, as well as the job.Group implementation, 70 | can persist job object generically. 71 | 72 | Pool 73 | 74 | Contains implementations of a Queue-compatible worker pool 75 | (i.e. Runners). Intentionally, runner implementations are naive and 76 | simple so there's less useful variation. 77 | 78 | Job 79 | 80 | Provides several generically useful Job implementations, for executing groups of 81 | sub-jobs or running shell commands in jobs. Additionally the package also 82 | contains tools used in writing specific job implementations, including a type 83 | used to interchange jobs, and a a monotonically increasing JobId generator. 84 | 85 | Queue 86 | 87 | Queue provides implementations of the Queue interface, which provide different 88 | job dispatching and distribution strategies. In addition, it provides 89 | implementations for queue-adjacent components such as job scope managers and 90 | retry handlers for retryable queues. 91 | 92 | Dependency 93 | 94 | The Dependency package contains the interface that describes how jobs 95 | and queues communicate about the dependency between jobs 96 | (dependecy.Manager), as well as several generic dependency 97 | implementations. 98 | */ 99 | -------------------------------------------------------------------------------- /cli/abortable_pool_management.go: -------------------------------------------------------------------------------- 1 | package cli 2 | 3 | import ( 4 | "context" 5 | 6 | "github.com/cheynewallace/tabby" 7 | "github.com/mongodb/amboy/rest" 8 | "github.com/pkg/errors" 9 | "github.com/urfave/cli" 10 | ) 11 | 12 | func abortablePoolManagement(opts *ServiceOptions) cli.Command { 13 | return cli.Command{ 14 | Name: "abortable_pool_management", 15 | Subcommands: []cli.Command{ 16 | manageListJobs(opts), 17 | manageAbortAllJobs(opts), 18 | manageCheckJob(opts), 19 | manageAbortJob(opts), 20 | }, 21 | } 22 | } 23 | 24 | func manageListJobs(opts *ServiceOptions) cli.Command { 25 | return cli.Command{ 26 | Name: "list", 27 | Flags: opts.abortablePoolManagementFlags(), 28 | Action: func(c *cli.Context) error { 29 | ctx, cancel := context.WithCancel(context.Background()) 30 | defer cancel() 31 | 32 | return opts.withAbortablePoolManagementClient(ctx, c, func(client *rest.AbortablePoolManagementClient) error { 33 | jobs, err := client.ListJobs(ctx) 34 | if err != nil { 35 | return errors.WithStack(err) 36 | } 37 | 38 | t := tabby.New() 39 | t.AddHeader("Job ID") 40 | for _, j := range jobs { 41 | t.AddLine(j) 42 | } 43 | t.Print() 44 | 45 | return nil 46 | }) 47 | 48 | }, 49 | } 50 | } 51 | 52 | func manageAbortAllJobs(opts *ServiceOptions) cli.Command { 53 | return cli.Command{ 54 | Name: "abort-all", 55 | Flags: opts.abortablePoolManagementFlags(), 56 | Action: func(c *cli.Context) error { 57 | ctx, cancel := context.WithCancel(context.Background()) 58 | defer cancel() 59 | 60 | return opts.withAbortablePoolManagementClient(ctx, c, func(client *rest.AbortablePoolManagementClient) error { 61 | return errors.WithStack(client.AbortAllJobs(ctx)) 62 | }) 63 | 64 | }, 65 | } 66 | } 67 | 68 | func manageCheckJob(opts *ServiceOptions) cli.Command { 69 | const jobIDFlagName = "id" 70 | 71 | return cli.Command{ 72 | Name: "check", 73 | Flags: opts.abortablePoolManagementFlags( 74 | cli.StringSliceFlag{ 75 | Name: jobIDFlagName, 76 | Usage: "specify the name of the job to check. May specify more than once.", 77 | }, 78 | ), 79 | Action: func(c *cli.Context) error { 80 | ctx, cancel := context.WithCancel(context.Background()) 81 | defer cancel() 82 | 83 | return opts.withAbortablePoolManagementClient(ctx, c, func(client *rest.AbortablePoolManagementClient) error { 84 | t := tabby.New() 85 | t.AddHeader("Job ID", "Is Running") 86 | for _, j := range c.StringSlice(jobIDFlagName) { 87 | isRunning, err := client.IsRunning(ctx, j) 88 | if err != nil { 89 | return errors.WithStack(err) 90 | } 91 | 92 | t.AddLine(j, isRunning) 93 | } 94 | t.Print() 95 | 96 | return nil 97 | }) 98 | 99 | }, 100 | } 101 | } 102 | 103 | func manageAbortJob(opts *ServiceOptions) cli.Command { 104 | const jobIDFlagName = "id" 105 | 106 | return cli.Command{ 107 | Name: "abort", 108 | Flags: opts.abortablePoolManagementFlags( 109 | cli.StringSliceFlag{ 110 | Name: jobIDFlagName, 111 | Usage: "specify the name of the job to abort. May specify more than once.", 112 | }, 113 | ), 114 | Action: func(c *cli.Context) error { 115 | ctx, cancel := context.WithCancel(context.Background()) 116 | defer cancel() 117 | 118 | return opts.withAbortablePoolManagementClient(ctx, c, func(client *rest.AbortablePoolManagementClient) error { 119 | var hasErrors bool 120 | t := tabby.New() 121 | t.AddHeader("Job ID", "Aborted", "Error") 122 | for _, j := range c.StringSlice(jobIDFlagName) { 123 | err := client.AbortJob(ctx, j) 124 | if err == nil { 125 | t.AddLine(j, true, "") 126 | } else { 127 | hasErrors = true 128 | t.AddLine(j, false, err.Error()) 129 | } 130 | } 131 | t.Print() 132 | 133 | if hasErrors { 134 | return errors.New("encountered errors while running some jobs") 135 | } 136 | 137 | return nil 138 | }) 139 | 140 | }, 141 | } 142 | } 143 | -------------------------------------------------------------------------------- /cmd/run-linter/run-linter.go: -------------------------------------------------------------------------------- 1 | package main 2 | 3 | import ( 4 | "bytes" 5 | "flag" 6 | "fmt" 7 | "os" 8 | "os/exec" 9 | "path/filepath" 10 | "strings" 11 | "time" 12 | ) 13 | 14 | type result struct { 15 | name string 16 | cmd string 17 | passed bool 18 | duration time.Duration 19 | output []string 20 | } 21 | 22 | // String prints the results of a linter run in gotest format. 23 | func (r *result) String() string { 24 | buf := &bytes.Buffer{} 25 | 26 | fmt.Fprintln(buf, "=== RUN", r.name) 27 | if r.passed { 28 | fmt.Fprintf(buf, "--- PASS: %s (%s)", r.name, r.duration) 29 | } else { 30 | fmt.Fprintf(buf, strings.Join(r.output, "\n")) 31 | fmt.Fprintf(buf, "--- FAIL: %s (%s)", r.name, r.duration) 32 | } 33 | 34 | return buf.String() 35 | } 36 | 37 | // fixup goes through the output and improves the output generated by 38 | // specific linters so that all output includes the relative path to the 39 | // error, instead of mixing relative and absolute paths. 40 | func (r *result) fixup(dirname string) { 41 | for idx, ln := range r.output { 42 | if strings.HasPrefix(ln, dirname) { 43 | r.output[idx] = ln[len(dirname)+1:] 44 | } 45 | } 46 | } 47 | 48 | // runs the golangci-lint on a list of packages; integrating with the "make lint" target. 49 | func main() { 50 | var ( 51 | lintArgs string 52 | lintBin string 53 | customLintersFlag string 54 | customLinters []string 55 | packageList string 56 | output string 57 | packages []string 58 | results []*result 59 | hasFailingTest bool 60 | ) 61 | 62 | flag.StringVar(&lintArgs, "lintArgs", "", "args to pass to golangci-lint") 63 | flag.StringVar(&lintBin, "lintBin", "", "path to golangci-lint") 64 | flag.StringVar(&packageList, "packages", "", "list of space separated packages") 65 | flag.StringVar(&customLintersFlag, "customLinters", "", "list of comma-separated custom linter commands") 66 | flag.StringVar(&output, "output", "", "output file for to write results.") 67 | flag.Parse() 68 | 69 | if len(customLintersFlag) != 0 { 70 | customLinters = strings.Split(customLintersFlag, ",") 71 | } 72 | packages = strings.Split(strings.Replace(packageList, "-", "/", -1), " ") 73 | dirname, _ := os.Getwd() 74 | cwd := filepath.Base(dirname) 75 | 76 | for _, pkg := range packages { 77 | pkgDir := "./" 78 | if cwd != pkg { 79 | pkgDir += pkg 80 | } 81 | splitLintArgs := strings.Split(lintArgs, " ") 82 | args := []string{lintBin, "run"} 83 | args = append(args, splitLintArgs...) 84 | args = append(args, pkgDir) 85 | 86 | startAt := time.Now() 87 | cmd := exec.Command(args[0], args[1:]...) 88 | cmd.Dir = dirname 89 | out, err := cmd.CombinedOutput() 90 | r := &result{ 91 | cmd: strings.Join(args, " "), 92 | name: "lint-" + strings.Replace(pkg, "/", "-", -1), 93 | passed: err == nil, 94 | duration: time.Since(startAt), 95 | output: strings.Split(string(out), "\n"), 96 | } 97 | 98 | for _, linter := range customLinters { 99 | customLinterStart := time.Now() 100 | linterArgs := strings.Split(linter, " ") 101 | linterArgs = append(linterArgs, pkgDir) 102 | cmd := exec.Command(linterArgs[0], linterArgs[1:]...) 103 | cmd.Dir = dirname 104 | out, err := cmd.CombinedOutput() 105 | r.passed = r.passed && err == nil 106 | r.duration += time.Since(customLinterStart) 107 | r.output = append(r.output, strings.Split(string(out), "\n")...) 108 | } 109 | r.fixup(dirname) 110 | 111 | if !r.passed { 112 | hasFailingTest = true 113 | } 114 | 115 | results = append(results, r) 116 | fmt.Println(r) 117 | } 118 | 119 | if output != "" { 120 | f, err := os.Create(output) 121 | if err != nil { 122 | os.Exit(1) 123 | } 124 | defer func() { 125 | if err != f.Close() { 126 | panic(err) 127 | } 128 | }() 129 | 130 | for _, r := range results { 131 | if _, err = f.WriteString(r.String() + "\n"); err != nil { 132 | fmt.Fprintf(os.Stderr, "%s", err) 133 | os.Exit(1) 134 | } 135 | } 136 | } 137 | 138 | if hasFailingTest { 139 | os.Exit(1) 140 | } 141 | } 142 | -------------------------------------------------------------------------------- /go.mod: -------------------------------------------------------------------------------- 1 | module github.com/mongodb/amboy 2 | 3 | go 1.24.0 4 | 5 | require ( 6 | github.com/VividCortex/ewma v1.2.0 7 | github.com/cheynewallace/tabby v1.1.1 8 | github.com/evergreen-ci/gimlet v0.0.0-20251205151908-163517996b82 9 | github.com/evergreen-ci/utility v0.0.0-20251203163234-8a1c0ea8b717 10 | github.com/fuyufjh/splunk-hec-go v0.3.4-0.20210909061418-feecd03924b7 // indirect 11 | github.com/google/uuid v1.6.0 12 | github.com/mongodb/grip v0.0.0-20251203205830-b5c5c666ab94 13 | github.com/pkg/errors v0.9.1 14 | github.com/stretchr/testify v1.10.0 15 | github.com/urfave/cli v1.22.10 16 | go.mongodb.org/mongo-driver v1.17.6 17 | go.opentelemetry.io/otel v1.16.0 18 | go.opentelemetry.io/otel/trace v1.16.0 19 | golang.org/x/oauth2 v0.33.0 // indirect 20 | gopkg.in/mgo.v2 v2.0.0-20190816093944-a6b53ec6cb22 21 | ) 22 | 23 | require ( 24 | github.com/PuerkitoBio/rehttp v1.1.0 // indirect 25 | github.com/andygrunwald/go-jira v1.16.0 // indirect 26 | github.com/aws/aws-sdk-go-v2 v1.30.3 // indirect 27 | github.com/aws/aws-sdk-go-v2/internal/configsources v1.3.15 // indirect 28 | github.com/aws/aws-sdk-go-v2/internal/endpoints/v2 v2.6.15 // indirect 29 | github.com/aws/aws-sdk-go-v2/service/ses v1.19.6 // indirect 30 | github.com/aws/smithy-go v1.20.3 // indirect 31 | github.com/coreos/go-oidc v2.2.1+incompatible // indirect 32 | github.com/coreos/go-systemd v0.0.0-20191104093116-d3cd4ed1dbcf // indirect 33 | github.com/cpuguy83/go-md2man/v2 v2.0.0-20190314233015-f79a8a8ca69d // indirect 34 | github.com/davecgh/go-spew v1.1.1 // indirect 35 | github.com/dghubble/oauth1 v0.7.2 // indirect 36 | github.com/evergreen-ci/negroni v1.0.1-0.20211028183800-67b6d7c2c035 // indirect 37 | github.com/fatih/structs v1.1.0 // indirect 38 | github.com/felixge/httpsnoop v1.0.3 // indirect 39 | github.com/go-logr/logr v1.2.4 // indirect 40 | github.com/go-logr/stdr v1.2.2 // indirect 41 | github.com/go-ole/go-ole v1.2.6 // indirect 42 | github.com/golang-jwt/jwt/v4 v4.5.2 // indirect 43 | github.com/golang/snappy v0.0.4 // indirect 44 | github.com/google/go-github/v79 v79.0.0 // indirect 45 | github.com/google/go-querystring v1.1.0 // indirect 46 | github.com/gorilla/mux v1.8.0 // indirect 47 | github.com/gorilla/websocket v1.4.2 // indirect 48 | github.com/jmespath/go-jmespath v0.4.0 // indirect 49 | github.com/jpillora/backoff v1.0.0 // indirect 50 | github.com/klauspost/compress v1.16.7 // indirect 51 | github.com/lufia/plan9stats v0.0.0-20211012122336-39d0f177ccd0 // indirect 52 | github.com/mattn/go-xmpp v0.0.0-20211029151415-912ba614897a // indirect 53 | github.com/montanaflynn/stats v0.7.1 // indirect 54 | github.com/peterhellberg/link v1.2.0 // indirect 55 | github.com/phyber/negroni-gzip v1.0.0 // indirect 56 | github.com/pmezard/go-difflib v1.0.0 // indirect 57 | github.com/power-devops/perfstat v0.0.0-20210106213030-5aafc221ea8c // indirect 58 | github.com/pquerna/cachecontrol v0.2.0 // indirect 59 | github.com/rogpeppe/go-internal v1.14.0 // indirect 60 | github.com/rs/cors v1.8.3 // indirect 61 | github.com/russross/blackfriday/v2 v2.0.1 // indirect 62 | github.com/sabhiram/go-gitignore v0.0.0-20210923224102-525f6e181f06 // indirect 63 | github.com/shirou/gopsutil/v3 v3.23.5 // indirect 64 | github.com/shoenig/go-m1cpu v0.1.6 // indirect 65 | github.com/shurcooL/sanitized_anchor_name v1.0.0 // indirect 66 | github.com/slack-go/slack v0.12.1 // indirect 67 | github.com/tklauser/go-sysconf v0.3.11 // indirect 68 | github.com/tklauser/numcpus v0.6.0 // indirect 69 | github.com/trivago/tgo v1.0.7 // indirect 70 | github.com/urfave/negroni v1.0.0 // indirect 71 | github.com/xdg-go/pbkdf2 v1.0.0 // indirect 72 | github.com/xdg-go/scram v1.1.2 // indirect 73 | github.com/xdg-go/stringprep v1.0.4 // indirect 74 | github.com/youmark/pkcs8 v0.0.0-20240726163527-a2c0da244d78 // indirect 75 | github.com/yusufpapurcu/wmi v1.2.3 // indirect 76 | go.opentelemetry.io/contrib/instrumentation/net/http/otelhttp v0.42.0 // indirect 77 | go.opentelemetry.io/otel/metric v1.16.0 // indirect 78 | go.opentelemetry.io/otel/sdk v1.15.1 // indirect 79 | golang.org/x/crypto v0.45.0 // indirect 80 | golang.org/x/sync v0.18.0 // indirect 81 | golang.org/x/sys v0.38.0 // indirect 82 | golang.org/x/text v0.31.0 // indirect 83 | gopkg.in/square/go-jose.v2 v2.6.0 // indirect 84 | gopkg.in/yaml.v2 v2.4.0 // indirect 85 | gopkg.in/yaml.v3 v3.0.1 // indirect 86 | ) 87 | -------------------------------------------------------------------------------- /logger/sender_test.go: -------------------------------------------------------------------------------- 1 | package logger 2 | 3 | import ( 4 | "context" 5 | "testing" 6 | "time" 7 | 8 | "github.com/evergreen-ci/utility" 9 | "github.com/mongodb/amboy" 10 | "github.com/mongodb/amboy/queue" 11 | "github.com/mongodb/grip" 12 | "github.com/mongodb/grip/level" 13 | "github.com/mongodb/grip/message" 14 | "github.com/mongodb/grip/send" 15 | "github.com/stretchr/testify/suite" 16 | ) 17 | 18 | // this suite is lifted directly from the grip sender's test suite 19 | type SenderSuite struct { 20 | senders map[string]send.Sender 21 | mock *send.InternalSender 22 | queue amboy.Queue 23 | canceler context.CancelFunc 24 | suite.Suite 25 | } 26 | 27 | func TestSenderSuite(t *testing.T) { 28 | suite.Run(t, new(SenderSuite)) 29 | } 30 | 31 | func (s *SenderSuite) SetupTest() { 32 | l := send.LevelInfo{Default: level.Info, Threshold: level.Notice} 33 | 34 | ctx := context.Background() 35 | 36 | ctx, s.canceler = context.WithCancel(ctx) 37 | var err error 38 | s.mock, err = send.NewInternalLogger("internal", l) 39 | s.Require().NoError(err) 40 | s.senders = map[string]send.Sender{} 41 | 42 | s.senders["single"], err = NewQueueBackedSender(ctx, s.mock, 2, 128) 43 | s.Require().NoError(err) 44 | 45 | s.queue = queue.NewLocalLimitedSize(4, 128) 46 | s.NoError(s.queue.Start(ctx)) 47 | s.Require().True(s.queue.Info().Started) 48 | 49 | s.senders["single-shared"] = MakeQueueSender(ctx, s.queue, s.mock) 50 | } 51 | 52 | func (s *SenderSuite) TearDownTest() { 53 | ctx, cancel := context.WithTimeout(context.Background(), 3*time.Second) 54 | defer cancel() 55 | amboy.WaitInterval(ctx, s.queue, 100*time.Millisecond) 56 | 57 | for _, sender := range s.senders { 58 | s.NoError(sender.Close()) 59 | } 60 | s.NoError(s.mock.Close()) 61 | if s.canceler != nil { 62 | s.canceler() 63 | } 64 | } 65 | 66 | func (s *SenderSuite) TearDownSuite() { 67 | for _, sender := range s.senders { 68 | s.NoError(sender.Close()) 69 | } 70 | } 71 | 72 | func (s *SenderSuite) TestSenderImplementsInterface() { 73 | // this actually won't catch the error; the compiler will in 74 | // the fixtures, but either way we need to make sure that the 75 | // tests actually enforce this. 76 | for name, sender := range s.senders { 77 | s.Implements((*send.Sender)(nil), sender, name) 78 | } 79 | } 80 | 81 | func (s *SenderSuite) TestNameSetterRoundTrip() { 82 | for n, sender := range s.senders { 83 | for i := 0; i < 100; i++ { 84 | name := utility.MakeRandomString(6) 85 | s.NotEqual(sender.Name(), name, n) 86 | sender.SetName(name) 87 | s.Equal(sender.Name(), name, n) 88 | } 89 | } 90 | } 91 | 92 | func (s *SenderSuite) TestLevelSetterRejectsInvalidSettings() { 93 | levels := []send.LevelInfo{ 94 | {Default: level.Invalid, Threshold: level.Invalid}, 95 | {Default: level.Priority(-10), Threshold: level.Priority(-1)}, 96 | {Default: level.Debug, Threshold: level.Priority(-1)}, 97 | {Default: level.Priority(800), Threshold: level.Priority(-2)}, 98 | } 99 | 100 | for n, sender := range s.senders { 101 | s.NoError(sender.SetLevel(send.LevelInfo{Default: level.Debug, Threshold: level.Alert})) 102 | for _, l := range levels { 103 | s.True(sender.Level().Valid(), n) 104 | s.False(l.Valid(), n) 105 | s.Error(sender.SetLevel(l), n) 106 | s.True(sender.Level().Valid(), n) 107 | s.NotEqual(sender.Level(), l, n) 108 | } 109 | 110 | } 111 | } 112 | 113 | func (s *SenderSuite) TestFlush() { 114 | for t, sender := range s.senders { 115 | for i := 0; i < 10; i++ { 116 | sender.Send(message.ConvertToComposer(level.Error, "message")) 117 | } 118 | s.Require().NoError(sender.Flush(context.Background()), t) 119 | for i := 0; i < 10; i++ { 120 | m, ok := s.mock.GetMessageSafe() 121 | s.Require().True(ok, t) 122 | s.Equal("message", m.Message.String(), t) 123 | } 124 | } 125 | } 126 | 127 | func (s *SenderSuite) TestCloserShouldUsusallyNoop() { 128 | for t, sender := range s.senders { 129 | s.NoError(sender.Close(), t) 130 | } 131 | } 132 | 133 | func (s *SenderSuite) TestBasicNoopSendTest() { 134 | for name, sender := range s.senders { 135 | grip.Info(name) 136 | for i := -10; i <= 110; i += 5 { 137 | m := message.NewDefaultMessage(level.Priority(i), "hello world! "+utility.MakeRandomString(5)) 138 | sender.Send(m) 139 | } 140 | } 141 | } 142 | -------------------------------------------------------------------------------- /wait.go: -------------------------------------------------------------------------------- 1 | /* 2 | Waiting for Jobs to Complete 3 | 4 | The amboy package proves a number of generic methods that, using the 5 | Queue.Stats() method, block until all jobs are complete. They provide different 6 | semantics, which may be useful in different circumstances. All of the Wait* 7 | functions wait until the total number of jobs submitted to the queue is equal to 8 | the number of completed jobs, and as a result these methods don't prevent other 9 | threads from adding jobs to the queue after beginning to wait. As a special 10 | case, retryable queues will also wait until there are no retrying jobs 11 | remaining. 12 | 13 | Additionally, there are a set of methods, WaitJob*, that allow callers to wait 14 | for a specific job to complete. 15 | */ 16 | package amboy 17 | 18 | import ( 19 | "context" 20 | "time" 21 | ) 22 | 23 | // Wait takes a queue and blocks until all job are completed or the context is 24 | // canceled. This operation runs in a tight-loop, which means that the Wait will 25 | // return *as soon* as all jobs are complete. Conversely, it's also possible 26 | // that frequent repeated calls to Stats() may contend with resources needed for 27 | // dispatching jobs or marking them complete. Retrying jobs are not considered 28 | // complete. 29 | func Wait(ctx context.Context, q Queue) bool { 30 | for { 31 | if ctx.Err() != nil { 32 | return false 33 | } 34 | 35 | stat := q.Stats(ctx) 36 | if stat.IsComplete() { 37 | return true 38 | } 39 | 40 | } 41 | } 42 | 43 | // WaitInterval provides the Wait operation and accepts a context for 44 | // cancellation while also waiting for an interval between stats calls. The 45 | // return value reports if the operation was canceled or if all jobs are 46 | // complete. Retrying jobs are not considered complete. 47 | func WaitInterval(ctx context.Context, q Queue, interval time.Duration) bool { 48 | timer := time.NewTimer(0) 49 | defer timer.Stop() 50 | 51 | for { 52 | select { 53 | case <-ctx.Done(): 54 | return false 55 | case <-timer.C: 56 | if q.Stats(ctx).IsComplete() { 57 | return true 58 | } 59 | 60 | timer.Reset(interval) 61 | } 62 | } 63 | } 64 | 65 | // WaitIntervalNum waits for a certain number of jobs to complete. Retrying jobs 66 | // are not considered complete. 67 | func WaitIntervalNum(ctx context.Context, q Queue, interval time.Duration, num int) bool { 68 | timer := time.NewTimer(0) 69 | defer timer.Stop() 70 | 71 | for { 72 | select { 73 | case <-ctx.Done(): 74 | return false 75 | case <-timer.C: 76 | if q.Stats(ctx).Completed-q.Stats(ctx).Retrying >= num { 77 | return true 78 | } 79 | } 80 | } 81 | } 82 | 83 | // WaitJob blocks until the job, based on its ID, is marked complete in the 84 | // queue, or the context is canceled. The return value is false if the job does 85 | // not exist (or is removed) and true when the job completes. A retrying job is 86 | // not considered complete. 87 | func WaitJob(ctx context.Context, j Job, q Queue) bool { 88 | var ok bool 89 | for { 90 | if ctx.Err() != nil { 91 | return false 92 | } 93 | 94 | j, ok = q.Get(ctx, j.ID()) 95 | if !ok { 96 | return false 97 | } 98 | 99 | if ctx.Err() != nil { 100 | return false 101 | } 102 | 103 | completed := j.Status().Completed && j.RetryInfo().ShouldRetry() 104 | if completed { 105 | return true 106 | } 107 | } 108 | } 109 | 110 | // WaitJobInterval takes a job and queue object and waits for the job to be 111 | // marked complete. The interval parameter controls how long the operation waits 112 | // between checks, and can be used to limit the impact of waiting on a busy 113 | // queue. The operation returns false if the job is not registered in the queue, 114 | // and true when the job completes. A retrying job is not considered complete. 115 | func WaitJobInterval(ctx context.Context, j Job, q Queue, interval time.Duration) bool { 116 | var ok bool 117 | 118 | timer := time.NewTimer(0) 119 | defer timer.Stop() 120 | 121 | for { 122 | select { 123 | case <-ctx.Done(): 124 | return false 125 | case <-timer.C: 126 | j, ok = q.Get(ctx, j.ID()) 127 | if !ok { 128 | return false 129 | } 130 | 131 | completed := j.Status().Completed && !j.RetryInfo().ShouldRetry() 132 | 133 | if completed { 134 | return true 135 | } 136 | 137 | timer.Reset(interval) 138 | } 139 | } 140 | } 141 | -------------------------------------------------------------------------------- /pool/local_test.go: -------------------------------------------------------------------------------- 1 | package pool 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "sync" 7 | "testing" 8 | "time" 9 | 10 | "github.com/mongodb/amboy" 11 | "github.com/mongodb/amboy/job" 12 | "github.com/stretchr/testify/assert" 13 | "github.com/stretchr/testify/suite" 14 | ) 15 | 16 | type LocalWorkersSuite struct { 17 | size int 18 | pool *localWorkers 19 | queue *QueueTester 20 | suite.Suite 21 | } 22 | 23 | func TestLocalWorkersSuiteSizeOne(t *testing.T) { 24 | s := new(LocalWorkersSuite) 25 | s.size = 1 26 | 27 | suite.Run(t, s) 28 | } 29 | 30 | func TestLocalWorkersSuiteSizeThree(t *testing.T) { 31 | s := new(LocalWorkersSuite) 32 | s.size = 3 33 | 34 | suite.Run(t, s) 35 | } 36 | 37 | func TestLocalWorkersSuiteSizeOneHundred(t *testing.T) { 38 | s := new(LocalWorkersSuite) 39 | s.size = 100 40 | 41 | suite.Run(t, s) 42 | } 43 | 44 | func (s *LocalWorkersSuite) SetupTest() { 45 | s.pool = NewLocalWorkers(s.size, nil).(*localWorkers) 46 | s.queue = NewQueueTester(s.pool) 47 | } 48 | 49 | func (s *LocalWorkersSuite) TestPanicJobsDoNotPanicHarness() { 50 | ctx, cancel := context.WithCancel(context.Background()) 51 | defer cancel() 52 | wg := &sync.WaitGroup{} 53 | 54 | s.queue.toProcess = jobsChanWithPanicingJobs(ctx, s.size) 55 | s.NotPanics(func() { worker(ctx, "test-local", s.queue, wg, &sync.Mutex{}) }) 56 | } 57 | 58 | func (s *LocalWorkersSuite) TestConstructedInstanceImplementsInterface() { 59 | s.Implements((*amboy.Runner)(nil), s.pool) 60 | } 61 | 62 | func (s *LocalWorkersSuite) TestPoolErrorsOnSuccessiveStarts() { 63 | s.False(s.pool.Started()) 64 | 65 | ctx, cancel := context.WithCancel(context.Background()) 66 | defer cancel() 67 | 68 | s.NoError(s.pool.Start(ctx)) 69 | s.True(s.pool.Started()) 70 | 71 | for i := 0; i < 20; i++ { 72 | s.NoError(s.pool.Start(ctx)) 73 | s.True(s.pool.Started()) 74 | } 75 | } 76 | 77 | func (s *LocalWorkersSuite) TestPoolStartsAndProcessesJobs() { 78 | const num int = 100 79 | var jobs []amboy.Job 80 | 81 | for i := 0; i < num; i++ { 82 | cmd := fmt.Sprintf("echo 'job=%d'", i) 83 | jobs = append(jobs, job.NewShellJob(cmd, "")) 84 | } 85 | 86 | s.False(s.pool.Started()) 87 | s.False(s.queue.Info().Started) 88 | 89 | ctx, cancel := context.WithCancel(context.Background()) 90 | defer cancel() 91 | 92 | s.NoError(s.queue.Start(ctx)) 93 | 94 | for _, job := range jobs { 95 | s.NoError(s.queue.Put(ctx, job)) 96 | } 97 | 98 | s.True(s.pool.Started()) 99 | s.True(s.queue.Info().Started) 100 | 101 | amboy.WaitInterval(ctx, s.queue, 100*time.Millisecond) 102 | 103 | counter := 0 104 | for j := range s.queue.Results(ctx) { 105 | s.True(j.Status().Completed) 106 | counter++ 107 | } 108 | s.Equal(counter, len(jobs)) 109 | 110 | for _, job := range jobs { 111 | s.True(job.Status().Completed) 112 | } 113 | } 114 | 115 | func (s *LocalWorkersSuite) TestQueueIsMutableBeforeStartingPool() { 116 | s.NotNil(s.pool.queue) 117 | s.False(s.pool.Started()) 118 | 119 | newQueue := NewQueueTester(s.pool) 120 | s.NoError(s.pool.SetQueue(newQueue)) 121 | 122 | s.Equal(newQueue, s.pool.queue) 123 | s.NotEqual(s.queue, s.pool.queue) 124 | } 125 | 126 | func (s *LocalWorkersSuite) TestQueueIsNotMutableAfterStartingPool() { 127 | s.NotNil(s.pool.queue) 128 | s.False(s.pool.Started()) 129 | 130 | ctx, cancel := context.WithCancel(context.Background()) 131 | defer cancel() 132 | 133 | s.NoError(s.pool.Start(ctx)) 134 | s.True(s.pool.Started()) 135 | 136 | newQueue := NewQueueTester(s.pool) 137 | s.Error(s.pool.SetQueue(newQueue)) 138 | 139 | s.Equal(s.queue, s.pool.queue) 140 | s.NotEqual(newQueue, s.pool.queue) 141 | } 142 | 143 | // This test makes sense to do without the fixtures in the suite 144 | 145 | func TestLocalWorkerPoolConstructorDoesNotAllowSizeValuesLessThanOne(t *testing.T) { 146 | assert := assert.New(t) 147 | var pool *localWorkers 148 | var runner amboy.Runner 149 | 150 | for _, size := range []int{-10, -1, 0} { 151 | runner = NewLocalWorkers(size, nil) 152 | pool = runner.(*localWorkers) 153 | 154 | assert.Equal(1, pool.size) 155 | } 156 | } 157 | 158 | func TestPanicJobPanics(t *testing.T) { 159 | assert := assert.New(t) // nolint 160 | ctx, cancel := context.WithCancel(context.Background()) 161 | defer cancel() 162 | 163 | for job := range jobsChanWithPanicingJobs(ctx, 8) { 164 | assert.Panics(func() { job.Run(ctx) }) 165 | } 166 | 167 | } 168 | -------------------------------------------------------------------------------- /rest/client_abortable_pool.go: -------------------------------------------------------------------------------- 1 | package rest 2 | 3 | import ( 4 | "context" 5 | "net/http" 6 | 7 | "github.com/evergreen-ci/gimlet" 8 | "github.com/pkg/errors" 9 | ) 10 | 11 | // AbortablePoolManagementClient provides a go wrapper to the 12 | // AbortablePoolManagement service. 13 | type AbortablePoolManagementClient struct { 14 | client *http.Client 15 | url string 16 | } 17 | 18 | // NewAbortablePoolManagementClient constructs a new 19 | // AbortablePoolManagementClient instance that constructs a new http.Client. 20 | func NewAbortablePoolManagementClient(url string) *AbortablePoolManagementClient { 21 | return NewAbortablePoolManagementClientFromExisting(&http.Client{}, url) 22 | } 23 | 24 | // NewAbortablePoolManagementClientFromExisting builds an 25 | // AbortablePoolManagementClient instance from an existing http.Client. 26 | func NewAbortablePoolManagementClientFromExisting(client *http.Client, url string) *AbortablePoolManagementClient { 27 | return &AbortablePoolManagementClient{ 28 | client: client, 29 | url: url, 30 | } 31 | } 32 | 33 | // ListJobs returns a full list of all running jobs managed by the 34 | // pool that the service reflects. 35 | func (c *AbortablePoolManagementClient) ListJobs(ctx context.Context) ([]string, error) { 36 | req, err := http.NewRequest(http.MethodGet, c.url+"/v1/jobs/list", nil) 37 | if err != nil { 38 | return nil, errors.Wrap(err, "problem building request") 39 | } 40 | 41 | req = req.WithContext(ctx) 42 | resp, err := c.client.Do(req) 43 | if err != nil { 44 | return nil, errors.Wrap(err, "error processing request") 45 | } 46 | defer resp.Body.Close() 47 | out := []string{} 48 | if err = gimlet.GetJSON(resp.Body, &out); err != nil { 49 | return nil, errors.Wrap(err, "problem reading response") 50 | } 51 | 52 | return out, nil 53 | } 54 | 55 | // AbortAllJobs issues the request to terminate all currently running 56 | // jobs managed by the pool that backs the request. 57 | func (c *AbortablePoolManagementClient) AbortAllJobs(ctx context.Context) error { 58 | req, err := http.NewRequest(http.MethodDelete, c.url+"/v1/jobs/abort", nil) 59 | if err != nil { 60 | return errors.Wrap(err, "problem building request") 61 | } 62 | 63 | req = req.WithContext(ctx) 64 | resp, err := c.client.Do(req) 65 | if err != nil { 66 | return errors.Wrap(err, "error processing request") 67 | } 68 | defer resp.Body.Close() 69 | 70 | if resp.StatusCode != http.StatusOK { 71 | return errors.New("failed to abort jobs") 72 | } 73 | 74 | return nil 75 | } 76 | 77 | // IsRunning checks if a job with a specified id is currently running 78 | // in the remote queue. Check the error value to identify if false 79 | // response is due to a communication problem with the service or is 80 | // legitimate. 81 | func (c *AbortablePoolManagementClient) IsRunning(ctx context.Context, job string) (bool, error) { 82 | req, err := http.NewRequest(http.MethodGet, c.url+"/v1/jobs/"+job, nil) 83 | if err != nil { 84 | return false, errors.Wrap(err, "problem building request") 85 | } 86 | 87 | req = req.WithContext(ctx) 88 | resp, err := c.client.Do(req) 89 | if err != nil { 90 | return false, errors.Wrap(err, "error processing request") 91 | } 92 | defer resp.Body.Close() 93 | 94 | if resp.StatusCode == http.StatusNotFound { 95 | return false, nil 96 | } 97 | 98 | return true, nil 99 | } 100 | 101 | // AbortJob sends the abort signal for a running job to the management 102 | // service, return any errors from the service. A nil response 103 | // indicates that the job has been successfully terminated. 104 | func (c *AbortablePoolManagementClient) AbortJob(ctx context.Context, job string) error { 105 | req, err := http.NewRequest(http.MethodDelete, c.url+"/v1/jobs/"+job, nil) 106 | if err != nil { 107 | return errors.Wrap(err, "problem building request") 108 | } 109 | 110 | req = req.WithContext(ctx) 111 | resp, err := c.client.Do(req) 112 | if err != nil { 113 | return errors.Wrap(err, "error processing request") 114 | } 115 | defer resp.Body.Close() 116 | 117 | if resp.StatusCode != 200 { 118 | rerr := &gimlet.ErrorResponse{} 119 | if err := gimlet.GetJSON(resp.Body, rerr); err != nil { 120 | return errors.Wrapf(err, "problem reading error response with %s", 121 | http.StatusText(resp.StatusCode)) 122 | 123 | } 124 | return errors.Wrap(rerr, "remove server returned error") 125 | } 126 | 127 | return nil 128 | } 129 | -------------------------------------------------------------------------------- /job/group.go: -------------------------------------------------------------------------------- 1 | package job 2 | 3 | import ( 4 | "context" 5 | "sync" 6 | 7 | "github.com/mongodb/amboy" 8 | "github.com/mongodb/amboy/dependency" 9 | "github.com/mongodb/amboy/registry" 10 | "github.com/mongodb/grip" 11 | "github.com/pkg/errors" 12 | ) 13 | 14 | // Group is a structure for running collections of Job objects at the 15 | // same time, as a single Job. Use Groups to isolate several Jobs from 16 | // other Jobs in the queue, and ensure that several Jobs run on a 17 | // single system. 18 | type Group struct { 19 | Jobs map[string]*registry.JobInterchange `bson:"jobs" json:"jobs" yaml:"jobs"` 20 | *Base `bson:"metadata" json:"metadata" yaml:"metadata"` 21 | 22 | mutex sync.RWMutex 23 | } 24 | 25 | // NewGroup creates a new, empty Group object. 26 | func NewGroup(name string) *Group { 27 | g := newGroupInstance() 28 | g.SetID(name) 29 | 30 | return g 31 | } 32 | 33 | // newGroupInstance is a common constructor for the public NewGroup 34 | // constructior and the registry.JobFactory constructor. 35 | func newGroupInstance() *Group { 36 | g := &Group{ 37 | Jobs: make(map[string]*registry.JobInterchange), 38 | Base: &Base{ 39 | JobType: amboy.JobType{ 40 | Name: "group", 41 | Version: 1, 42 | }, 43 | }, 44 | } 45 | return g 46 | } 47 | 48 | // Add is not part of the Job interface, but allows callers to append 49 | // jobs to the Group. Returns an error if a job with the same ID() 50 | // value already exists in the group. 51 | func (g *Group) Add(j amboy.Job) error { 52 | name := j.ID() 53 | 54 | g.mutex.Lock() 55 | defer g.mutex.Unlock() 56 | _, exists := g.Jobs[name] 57 | if exists { 58 | return errors.Errorf("job named '%s' already exists in group '%s'", 59 | name, g.ID()) 60 | } 61 | 62 | job, err := registry.MakeJobInterchange(j, amboy.JSON) 63 | if err != nil { 64 | return err 65 | } 66 | 67 | g.Jobs[name] = job 68 | return nil 69 | } 70 | 71 | // Run executes the jobs. Provides "continue on error" semantics for 72 | // Jobs in the Group. Returns an error if: the Group has already 73 | // run, or if any of the constituent Jobs produce an error *or* if 74 | // there are problems with the JobInterchange converters. 75 | func (g *Group) Run(ctx context.Context) { 76 | defer g.MarkComplete() 77 | 78 | if g.Status().Completed { 79 | g.AddError(errors.Errorf("group '%s' has already executed", g.ID())) 80 | return 81 | } 82 | 83 | wg := &sync.WaitGroup{} 84 | 85 | g.mutex.RLock() 86 | for _, job := range g.Jobs { 87 | if err := ctx.Err(); err != nil { 88 | g.AddError(err) 89 | break 90 | } 91 | 92 | runnableJob, err := job.Resolve(amboy.JSON) 93 | if err != nil { 94 | g.AddError(err) 95 | continue 96 | } 97 | 98 | depState := runnableJob.Dependency().State() 99 | if depState == dependency.Passed { 100 | grip.Infof("skipping job '%s' because of dependency", runnableJob.ID()) 101 | continue 102 | } else if depState == dependency.Blocked || depState == dependency.Unresolved { 103 | grip.Warningf("dispatching blocked/unresolved job '%s'", runnableJob.ID()) 104 | } 105 | 106 | wg.Add(1) 107 | go func(j amboy.Job, group *Group) { 108 | defer wg.Done() 109 | 110 | maxTime := j.TimeInfo().MaxTime 111 | if maxTime > 0 { 112 | var cancel context.CancelFunc 113 | ctx, cancel = context.WithTimeout(ctx, maxTime) 114 | defer cancel() 115 | } 116 | 117 | j.Run(ctx) 118 | 119 | // after the job completes, add the issue 120 | // back to Jobs map so that we preserve errors 121 | // idiomatically for Groups. 122 | jobErr := j.Error() 123 | g.AddError(jobErr) 124 | 125 | job, err := registry.MakeJobInterchange(j, amboy.JSON) 126 | if err != nil { 127 | g.AddError(err) 128 | return 129 | } 130 | 131 | if jobErr != nil { 132 | return 133 | } 134 | 135 | group.mutex.Lock() 136 | defer group.mutex.Unlock() 137 | group.Jobs[j.ID()] = job 138 | }(runnableJob, g) 139 | } 140 | g.mutex.RUnlock() 141 | wg.Wait() 142 | 143 | g.MarkComplete() 144 | } 145 | 146 | // SetDependency allows you to configure the dependency.Manager 147 | // instance for this object. If you want to swap different dependency 148 | // instances you can as long as the new instance is of the "Always" 149 | // type. 150 | func (g *Group) SetDependency(d dependency.Manager) { 151 | if d == nil || d.Type().Name != "always" { 152 | grip.Warningf("group job types must have 'always' dependency types, '%s' is invalid", d.Type().Name) 153 | return 154 | } 155 | 156 | g.Base.SetDependency(d) 157 | } 158 | -------------------------------------------------------------------------------- /rest/service_queue_job.go: -------------------------------------------------------------------------------- 1 | package rest 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "net/http" 7 | "time" 8 | 9 | "github.com/evergreen-ci/gimlet" 10 | "github.com/mongodb/amboy" 11 | "github.com/mongodb/grip" 12 | "github.com/mongodb/grip/message" 13 | "github.com/pkg/errors" 14 | ) 15 | 16 | type jobStatusResponse struct { 17 | Exists bool `bson:"job_exists" json:"job_exists" yaml:"job_exists"` 18 | Completed bool `bson:"completed" json:"completed" yaml:"completed"` 19 | ID string `bson:"id,omitempty" json:"id,omitempty" yaml:"id,omitempty"` 20 | JobsPending int `bson:"jobs_pending,omitempty" json:"jobs_pending,omitempty" yaml:"jobs_pending,omitempty"` 21 | Error string `bson:"error,omitempty" json:"error,omitempty" yaml:"error,omitempty"` 22 | Job interface{} `bson:"job,omitempty" json:"job,omitempty" yaml:"job,omitempty"` 23 | } 24 | 25 | func (s *QueueService) getJobStatusResponse(ctx context.Context, name string) (*jobStatusResponse, error) { 26 | var msg string 27 | var err error 28 | 29 | resp := &jobStatusResponse{} 30 | resp.JobsPending = s.queue.Stats(ctx).Pending 31 | resp.ID = name 32 | 33 | if name == "" { 34 | msg = fmt.Sprintf("did not specify job name: %s", name) 35 | err = errors.New(msg) 36 | resp.Error = msg 37 | 38 | return resp, err 39 | } 40 | 41 | j, exists := s.queue.Get(ctx, name) 42 | resp.Exists = exists 43 | 44 | if !exists { 45 | msg = fmt.Sprintf("could not recover job '%s'", name) 46 | err = errors.New(msg) 47 | resp.Error = msg 48 | 49 | return resp, err 50 | } 51 | 52 | completed := j.Status().Completed && (!j.RetryInfo().ShouldRetry()) 53 | 54 | resp.Exists = true 55 | resp.Completed = completed 56 | resp.Job = j 57 | 58 | return resp, nil 59 | } 60 | 61 | // JobStatus is a http.HandlerFunc that writes a job status document to the request. 62 | func (s *QueueService) JobStatus(w http.ResponseWriter, r *http.Request) { 63 | name := gimlet.GetVars(r)["name"] 64 | 65 | response, err := s.getJobStatusResponse(r.Context(), name) 66 | if err != nil { 67 | grip.Error(err) 68 | gimlet.WriteJSONError(w, response) 69 | return 70 | } 71 | 72 | gimlet.WriteJSON(w, response) 73 | } 74 | 75 | // WaitJob waits for a single job to be complete. It takes a timeout 76 | // argument, which defaults to 10 seconds, and returns 408 (request 77 | // timeout) if the timeout is reached before the job completes. 78 | func (s *QueueService) WaitJob(w http.ResponseWriter, r *http.Request) { 79 | ctx := r.Context() 80 | name := gimlet.GetVars(r)["name"] 81 | response, err := s.getJobStatusResponse(ctx, name) 82 | if err != nil { 83 | grip.Error(err) 84 | gimlet.WriteJSONError(w, response) 85 | } 86 | 87 | timeout, err := parseTimeout(r) 88 | if err != nil { 89 | grip.Info(message.WrapError(err, message.Fields{ 90 | "message": "problem parsing timeout", 91 | "name": name, 92 | })) 93 | } 94 | 95 | var cancel context.CancelFunc 96 | ctx, cancel = context.WithTimeout(ctx, timeout) 97 | defer cancel() 98 | 99 | response, code, err := s.waitForJob(ctx, name) 100 | grip.Error(err) 101 | gimlet.WriteJSONResponse(w, code, response) 102 | } 103 | 104 | func parseTimeout(r *http.Request) (time.Duration, error) { 105 | var err error 106 | 107 | timeout := 10 * time.Second 108 | 109 | timeoutInput, ok := r.URL.Query()["timeout"] 110 | 111 | if ok || len(timeoutInput) != 0 { 112 | timeout, err = time.ParseDuration(timeoutInput[0]) 113 | if err != nil { 114 | timeout = 10 * time.Second 115 | } 116 | } 117 | 118 | return timeout, errors.Wrapf(err, "problem parsing timeout from %s", timeoutInput) 119 | } 120 | 121 | func (s *QueueService) waitForJob(ctx context.Context, name string) (*jobStatusResponse, int, error) { 122 | job, ok := s.queue.Get(ctx, name) 123 | if !ok { 124 | response, err := s.getJobStatusResponse(ctx, name) 125 | grip.Error(err) 126 | return response, http.StatusNotFound, errors.Errorf( 127 | "problem finding job: %s", name) 128 | } 129 | 130 | ok = amboy.WaitJobInterval(ctx, job, s.queue, 100*time.Millisecond) 131 | 132 | response, err := s.getJobStatusResponse(ctx, name) 133 | if err != nil { 134 | return response, http.StatusInternalServerError, errors.Wrapf(err, 135 | "problem constructing response for while waiting for job %s", name) 136 | } 137 | 138 | if !ok { 139 | return response, http.StatusRequestTimeout, errors.Errorf( 140 | "reached timeout waiting for job: %s", name) 141 | } 142 | 143 | return response, http.StatusOK, nil 144 | } 145 | -------------------------------------------------------------------------------- /rest/service_group_management.go: -------------------------------------------------------------------------------- 1 | package rest 2 | 3 | import ( 4 | "net/http" 5 | 6 | "github.com/evergreen-ci/gimlet" 7 | "github.com/mongodb/amboy" 8 | "github.com/mongodb/grip" 9 | ) 10 | 11 | // ManagementGroupService provides the reporting service 12 | // impelementation for queue groups. 13 | type ManagementGroupService struct { 14 | group amboy.QueueGroup 15 | } 16 | 17 | // NewManagementGroupService returns a service that defines REST routes can 18 | // manage the abortable pools of a QueueGroup. 19 | func NewManagementGroupService(g amboy.QueueGroup) *ManagementGroupService { 20 | return &ManagementGroupService{ 21 | group: g, 22 | } 23 | } 24 | 25 | // App returns a gimlet app with all of the routes registered. 26 | func (s *ManagementGroupService) App() *gimlet.APIApp { 27 | app := gimlet.NewApp() 28 | 29 | app.AddRoute("/jobs/list").Version(1).Get().Handler(s.ListJobs) 30 | app.AddRoute("/jobs/abort").Version(1).Delete().Handler(s.AbortAllJobs) 31 | app.AddRoute("/job/{name}").Version(1).Get().Handler(s.GetJobStatus) 32 | app.AddRoute("/job/{name}").Version(1).Delete().Handler(s.AbortRunningJob) 33 | return app 34 | } 35 | 36 | // ListJobs is an http.HandlerFunc that returns a list of all running 37 | // jobs in all pools for the queue group. 38 | func (s *ManagementGroupService) ListJobs(rw http.ResponseWriter, r *http.Request) { 39 | ctx := r.Context() 40 | 41 | jobs := []string{} 42 | for _, group := range s.group.Queues(ctx) { 43 | if queue, err := s.group.Get(ctx, group); err == nil { 44 | if pool, ok := queue.Runner().(amboy.AbortableRunner); ok { 45 | jobs = append(jobs, pool.RunningJobs()...) 46 | } 47 | } 48 | } 49 | 50 | gimlet.WriteJSON(rw, jobs) 51 | } 52 | 53 | // AbortAllJobs is an http.HandlerFunc that sends the signal to abort 54 | // all running jobs in the pools of the group. May return a 408 (timeout) if the 55 | // calling context was canceled before the operation 56 | // returned. Otherwise, this handler returns 200. The body of the 57 | // response is always empty. 58 | func (s *ManagementGroupService) AbortAllJobs(rw http.ResponseWriter, r *http.Request) { 59 | ctx := r.Context() 60 | 61 | catcher := grip.NewBasicCatcher() 62 | for _, group := range s.group.Queues(ctx) { 63 | if queue, err := s.group.Get(ctx, group); err == nil { 64 | if pool, ok := queue.Runner().(amboy.AbortableRunner); ok { 65 | if err := pool.AbortAll(ctx); err != nil { 66 | if ctx.Err() != nil { 67 | gimlet.WriteJSONResponse(rw, http.StatusRequestTimeout, struct{}{}) 68 | return 69 | } 70 | catcher.Wrapf(err, "queue '%s'", queue.ID()) 71 | } 72 | } 73 | } 74 | } 75 | if catcher.HasErrors() { 76 | gimlet.WriteJSONInternalError(rw, catcher.Resolve().Error()) 77 | return 78 | } 79 | 80 | gimlet.WriteJSON(rw, struct{}{}) 81 | } 82 | 83 | // GetJobStatus is an http.HandlerFunc reports on the status (running 84 | // or not running) of a specific job. 85 | func (s *ManagementGroupService) GetJobStatus(rw http.ResponseWriter, r *http.Request) { 86 | name := gimlet.GetVars(r)["name"] 87 | 88 | ctx := r.Context() 89 | 90 | for _, group := range s.group.Queues(ctx) { 91 | if queue, err := s.group.Get(ctx, group); err == nil { 92 | if pool, ok := queue.Runner().(amboy.AbortableRunner); ok { 93 | if pool.IsRunning(name) { 94 | gimlet.WriteJSON(rw, map[string]string{ 95 | "name": name, 96 | "status": "running", 97 | "group": group, 98 | }) 99 | return 100 | } 101 | } 102 | } 103 | } 104 | 105 | gimlet.WriteJSONResponse(rw, http.StatusNotFound, 106 | map[string]string{ 107 | "name": name, 108 | "status": "not running", 109 | }) 110 | } 111 | 112 | // AbortRunningJob is an http.HandlerFunc that terminates the 113 | // execution of a single running job, returning a 400 response when 114 | // the job doesn't exist. 115 | func (s *ManagementGroupService) AbortRunningJob(rw http.ResponseWriter, r *http.Request) { 116 | name := gimlet.GetVars(r)["name"] 117 | 118 | ctx := r.Context() 119 | 120 | for _, group := range s.group.Queues(ctx) { 121 | if queue, err := s.group.Get(ctx, group); err == nil { 122 | if pool, ok := queue.Runner().(amboy.AbortableRunner); ok { 123 | if err = pool.Abort(ctx, name); err == nil { 124 | gimlet.WriteJSON(rw, map[string]string{ 125 | "name": name, 126 | "status": "aborted", 127 | "group": group, 128 | }) 129 | return 130 | } 131 | } 132 | } 133 | } 134 | 135 | gimlet.WriteJSONResponse(rw, http.StatusNotFound, 136 | map[string]string{ 137 | "name": name, 138 | "status": "unknown", 139 | }) 140 | } 141 | -------------------------------------------------------------------------------- /dependency/local.go: -------------------------------------------------------------------------------- 1 | package dependency 2 | 3 | import ( 4 | "os" 5 | 6 | "github.com/mongodb/grip" 7 | ) 8 | 9 | // LocalFile describes a dependency between a job and local 10 | // files. Has a notion of targets, and dependencies, and a la make 11 | // dependency resolution returns state Passed (e.g. noop) if the 12 | // target or targets are all newer than the dependency or 13 | // dependencies. LocalFile will return state Ready in 14 | // ambiguous cases where targets or dependencies are missing. 15 | // 16 | // Importantly, the edges, which amboy Jobs and Queue should always 17 | // assume are *not* files, are distinct from the Targets and 18 | // Dependencies in the context of this dependency.Manager. Add edges 19 | // (job names) for other amboy.Job IDs in the queue if you need to 20 | // express that kind of relationship. 21 | type LocalFile struct { 22 | // A list of file names that the job would in theory create. 23 | Targets []string `bson:"targets" json:"targets" yaml:"targets"` 24 | 25 | // A list of file names that represent dependencies that the 26 | // Target depends on. 27 | Dependencies []string `bson:"dependencies" json:"dependencies" yaml:"dependencies"` 28 | 29 | T TypeInfo `bson:"type" json:"type" yaml:"type"` 30 | JobEdges 31 | } 32 | 33 | const localFileTypeName = "local-file" 34 | 35 | // NewLocalFile creates a dependency object that checks if 36 | // dependencies on the local file system are created. This constructor 37 | // takes, as arguments, a target name and a variable number of 38 | // successive arguments that are dependencies. All arguments should be 39 | // file names, relative to the working directory of the 40 | // program. 41 | func NewLocalFile(target string, dependencies ...string) *LocalFile { 42 | d := MakeLocalFile() 43 | d.Targets = []string{target} 44 | d.Dependencies = dependencies 45 | 46 | return d 47 | } 48 | 49 | // MakeLocalFile constructs an empty local file instance. 50 | func MakeLocalFile() *LocalFile { 51 | return &LocalFile{ 52 | T: TypeInfo{ 53 | Name: localFileTypeName, 54 | Version: 0, 55 | }, 56 | JobEdges: NewJobEdges(), 57 | } 58 | } 59 | 60 | // State reports if the dependency is satisfied. If the targets or are 61 | // not specified *or* the file names of any target do not exist, then 62 | // State returns Ready. If a dependency does not exist, This call will 63 | // log a a warning. 64 | // 65 | // Otherwise, If any dependency has a modification time that is after 66 | // the earliest modification time of the targets, then State returns 67 | // Ready. When all targets were modified after all of the 68 | // dependencies, then State returns Passed. 69 | func (d *LocalFile) State() State { 70 | if len(d.Targets) == 0 { 71 | return Ready 72 | } 73 | // presumably it might make sense to do these checks in 74 | // parallel, but that seems premature, given that it could be 75 | // (potentially) a lot of IO, but that seems overkill at this 76 | // point. 77 | 78 | // first, find what the oldest target it, if any of the 79 | // dependencies are newer than it, we need to rebuild. 80 | var oldestTarget string 81 | var winningTargetStat os.FileInfo 82 | 83 | for _, target := range d.Targets { 84 | thisStat, err := os.Stat(target) 85 | if os.IsNotExist(err) { 86 | return Ready 87 | } 88 | 89 | // presumably this happens for the first target only, 90 | // but it's good to check. 91 | if oldestTarget == "" { 92 | oldestTarget = target 93 | winningTargetStat = thisStat 94 | continue 95 | } 96 | 97 | if thisStat.ModTime().Before(winningTargetStat.ModTime()) { 98 | oldestTarget = target 99 | winningTargetStat = thisStat 100 | } 101 | } 102 | 103 | // then, now find the newest dependency. 104 | var newestDependency string 105 | var winningDependencyStat os.FileInfo 106 | 107 | for _, dep := range d.Dependencies { 108 | thisStat, err := os.Stat(dep) 109 | if os.IsNotExist(err) { 110 | // this shouldn't trigger a rebuild. 111 | grip.Warningf("dependency %s does not exist", dep) 112 | } 113 | 114 | // presumably this happens for the first dependency 115 | // only, but it's good to check. 116 | if newestDependency == "" { 117 | newestDependency = dep 118 | winningDependencyStat = thisStat 119 | continue 120 | } 121 | 122 | if thisStat.ModTime().After(winningDependencyStat.ModTime()) { 123 | newestDependency = dep 124 | winningDependencyStat = thisStat 125 | } 126 | 127 | // this is a short circuit: if we find *one* 128 | // dependency that's newer than the oldest target, we 129 | // can return ready without stating a bunch of deps 130 | // unnecessarily. 131 | if thisStat.ModTime().After(winningTargetStat.ModTime()) { 132 | return Ready 133 | } 134 | } 135 | 136 | // if the last dependency check didn't return early, then the 137 | // job can be a noop. 138 | return Passed 139 | 140 | } 141 | 142 | // Type returns a TypeInfo object for the Dependency object. Used by 143 | // the registry and interchange systems. 144 | func (d *LocalFile) Type() TypeInfo { 145 | return d.T 146 | } 147 | -------------------------------------------------------------------------------- /rest/service_queue_create_test.go: -------------------------------------------------------------------------------- 1 | package rest 2 | 3 | import ( 4 | "bytes" 5 | "context" 6 | "encoding/json" 7 | "fmt" 8 | "net/http/httptest" 9 | "testing" 10 | 11 | "github.com/mongodb/amboy" 12 | "github.com/mongodb/amboy/job" 13 | "github.com/mongodb/amboy/registry" 14 | "github.com/stretchr/testify/require" 15 | "github.com/stretchr/testify/suite" 16 | ) 17 | 18 | type CreateJobSuite struct { 19 | service *QueueService 20 | require *require.Assertions 21 | closer context.CancelFunc 22 | suite.Suite 23 | } 24 | 25 | func TestCreateJobSuite(t *testing.T) { 26 | suite.Run(t, new(CreateJobSuite)) 27 | } 28 | 29 | func (s *CreateJobSuite) SetupSuite() { 30 | s.require = s.Require() 31 | s.service = NewQueueService() 32 | ctx, cancel := context.WithCancel(context.Background()) 33 | s.closer = cancel 34 | 35 | s.NoError(s.service.Open(ctx)) 36 | 37 | s.NoError(s.service.App().Resolve()) 38 | } 39 | 40 | func (s *CreateJobSuite) TearDownSuite() { 41 | s.closer() 42 | } 43 | 44 | func (s *CreateJobSuite) TestBaseResponseCreatorHasExpectedValues() { 45 | ctx, cancel := context.WithCancel(context.Background()) 46 | defer cancel() 47 | 48 | resp := s.service.createJobResponseBase(ctx) 49 | 50 | s.Equal(s.service.queue.Stats(ctx).Pending, resp.QueueDepth) 51 | s.Equal(s.service.getStatus(ctx), resp.Status) 52 | s.False(resp.Registered) 53 | s.Equal("", resp.ID) 54 | s.Equal("", resp.Error) 55 | } 56 | 57 | func (s *CreateJobSuite) TestNilJobPayloadResultsInError() { 58 | ctx, cancel := context.WithCancel(context.Background()) 59 | defer cancel() 60 | 61 | resp, err := s.service.createJob(ctx, ®istry.JobInterchange{}) 62 | s.Error(err) 63 | s.Equal(err.Error(), resp.Error) 64 | s.False(resp.Registered) 65 | } 66 | 67 | func (s *CreateJobSuite) TestAddingAJobThatAlreadyExistsResultsInError() { 68 | ctx, cancel := context.WithCancel(context.Background()) 69 | defer cancel() 70 | 71 | j := job.NewShellJob("true", "") 72 | payload, err := registry.MakeJobInterchange(j, amboy.JSON) 73 | s.NoError(err) 74 | 75 | s.NoError(s.service.queue.Put(ctx, j)) 76 | 77 | resp, err := s.service.createJob(ctx, payload) 78 | s.Error(err, fmt.Sprintf("%+v", resp)) 79 | 80 | s.Equal(err.Error(), resp.Error) 81 | s.Equal(j.ID(), resp.ID) 82 | } 83 | 84 | func (s *CreateJobSuite) TestAddingJobSuccessfuly() { 85 | ctx, cancel := context.WithCancel(context.Background()) 86 | defer cancel() 87 | 88 | j := job.NewShellJob("true", "") 89 | 90 | payload, err := registry.MakeJobInterchange(j, amboy.JSON) 91 | s.NoError(err) 92 | 93 | resp, err := s.service.createJob(ctx, payload) 94 | s.NoError(err) 95 | 96 | s.Equal(j.ID(), resp.ID) 97 | s.True(resp.Registered) 98 | s.Equal("", resp.Error) 99 | } 100 | 101 | func (s *CreateJobSuite) TestRequestWithNilPayload() { 102 | router, err := s.service.App().Handler() 103 | s.NoError(err) 104 | 105 | rb, err := json.Marshal(`{}`) 106 | s.NoError(err) 107 | w := httptest.NewRecorder() 108 | req := httptest.NewRequest("POST", "http://example.com/v1/job/create", bytes.NewBuffer(rb)) 109 | 110 | router.ServeHTTP(w, req) 111 | s.Equal(400, w.Code) 112 | 113 | resp := createResponse{} 114 | err = json.Unmarshal(w.Body.Bytes(), &resp) 115 | s.NoError(err) 116 | 117 | s.True(resp.Error != "") 118 | s.False(resp.Registered) 119 | } 120 | 121 | func (s *CreateJobSuite) TestRequestToAddJobThatAlreadyExists() { 122 | ctx, cancel := context.WithCancel(context.Background()) 123 | defer cancel() 124 | 125 | router, err := s.service.App().Handler() 126 | s.NoError(err) 127 | 128 | payload, err := registry.MakeJobInterchange(job.NewShellJob("true", ""), amboy.JSON) 129 | s.NoError(err) 130 | 131 | rb, err := json.Marshal(payload) 132 | s.NoError(err) 133 | 134 | j, err := payload.Resolve(amboy.JSON) 135 | s.NoError(err) 136 | 137 | s.NoError(s.service.queue.Put(ctx, j)) 138 | 139 | w := httptest.NewRecorder() 140 | req := httptest.NewRequest("POST", "http://example.com/v1/job/create", bytes.NewBuffer(rb)) 141 | router.ServeHTTP(w, req) 142 | s.Equal(400, w.Code) 143 | 144 | resp := createResponse{} 145 | err = json.Unmarshal(w.Body.Bytes(), &resp) 146 | s.NoError(err) 147 | 148 | s.True(resp.Error != "") 149 | s.Equal(j.ID(), resp.ID) 150 | s.False(resp.Registered) 151 | } 152 | 153 | func (s *CreateJobSuite) TestRequestToAddNewJobRegistersJob() { 154 | ctx, cancel := context.WithCancel(context.Background()) 155 | defer cancel() 156 | 157 | router, err := s.service.App().Handler() 158 | s.NoError(err) 159 | 160 | startingTotal := s.service.queue.Stats(ctx).Total 161 | j := job.NewShellJob("true", "") 162 | payload, err := registry.MakeJobInterchange(j, amboy.JSON) 163 | s.NoError(err) 164 | 165 | rb, err := json.Marshal(payload) 166 | s.NoError(err) 167 | 168 | w := httptest.NewRecorder() 169 | req := httptest.NewRequest("POST", "http://example.com/v1/job/create", bytes.NewBuffer(rb)) 170 | 171 | router.ServeHTTP(w, req) 172 | s.Equal(200, w.Code) 173 | 174 | resp := createResponse{} 175 | err = json.Unmarshal(w.Body.Bytes(), &resp) 176 | s.NoError(err) 177 | 178 | s.True(resp.Error == "") 179 | s.True(resp.Registered) 180 | s.Equal(j.ID(), resp.ID) 181 | s.Equal(s.service.queue.Stats(ctx).Total, startingTotal+1) 182 | } 183 | -------------------------------------------------------------------------------- /queue/scope_test.go: -------------------------------------------------------------------------------- 1 | package queue 2 | 3 | import ( 4 | "fmt" 5 | "testing" 6 | 7 | "github.com/stretchr/testify/require" 8 | ) 9 | 10 | func TestScopeManager(t *testing.T) { 11 | for acquireType, acquire := range map[string]func(mngr ScopeManager, id string, scopes []string) error{ 12 | "Acquire": func(mngr ScopeManager, id string, scopes []string) error { 13 | return mngr.Acquire(id, scopes) 14 | }, 15 | "ReleaseAndAcquire": func(mngr ScopeManager, id string, scopes []string) error { 16 | return mngr.ReleaseAndAcquire("", nil, id, scopes) 17 | }, 18 | } { 19 | t.Run(fmt.Sprintf("AcquireWith%sMethod", acquireType), func(t *testing.T) { 20 | for releaseType, release := range map[string]func(mngr ScopeManager, id string, scopes []string) error{ 21 | "Release": func(mngr ScopeManager, id string, scopes []string) error { 22 | return mngr.Release(id, scopes) 23 | }, 24 | "ReleaseAndAcquire": func(mngr ScopeManager, id string, scopes []string) error { 25 | return mngr.ReleaseAndAcquire(id, scopes, "", nil) 26 | }, 27 | } { 28 | t.Run(fmt.Sprintf("ReleaseWith%sMethod", releaseType), func(t *testing.T) { 29 | for testName, testCase := range map[string]func(t *testing.T, mngr ScopeManager){ 30 | "AcquireChecksEachScopeForUniqueness": func(t *testing.T, mngr ScopeManager) { 31 | require.NoError(t, acquire(mngr, "id1", []string{"foo"})) 32 | require.NoError(t, acquire(mngr, "id1", []string{"bar", "bat"})) 33 | require.Error(t, acquire(mngr, "id2", []string{"foo"})) 34 | require.Error(t, acquire(mngr, "id2", []string{"foo", "baz"})) 35 | }, 36 | "DoubleAcquireForSameIDWithSameScopesIsIdempotent": func(t *testing.T, mngr ScopeManager) { 37 | require.NoError(t, acquire(mngr, "id", []string{"foo"})) 38 | require.NoError(t, acquire(mngr, "id", []string{"foo"})) 39 | }, 40 | "DoubleAcquireForSameIDWithDifferentScopesSucceeds": func(t *testing.T, mngr ScopeManager) { 41 | require.NoError(t, acquire(mngr, "id", []string{"foo"})) 42 | require.NoError(t, acquire(mngr, "id", []string{"bar"})) 43 | }, 44 | "AcquireIsAtomic": func(t *testing.T, mngr ScopeManager) { 45 | require.NoError(t, acquire(mngr, "id1", []string{"foo"})) 46 | require.Error(t, acquire(mngr, "id2", []string{"bar", "foo"})) 47 | // If Acquire is atomic, "bar" should be unowned so this should not 48 | // error. 49 | require.NoError(t, release(mngr, "id1", []string{"bar"})) 50 | }, 51 | "ReleaseOfUnownedScopeForIDIsNoop": func(t *testing.T, mngr ScopeManager) { 52 | require.NoError(t, release(mngr, "id", []string{"foo"})) 53 | }, 54 | "ReleaseOfScopeOwnedByAnotherIDFails": func(t *testing.T, mngr ScopeManager) { 55 | require.NoError(t, acquire(mngr, "id1", []string{"foo"})) 56 | require.Error(t, release(mngr, "id2", []string{"foo"})) 57 | }, 58 | "DoubleReleaseOfSameScopeForIDIsIdempotent": func(t *testing.T, mngr ScopeManager) { 59 | require.NoError(t, acquire(mngr, "id", []string{"foo"})) 60 | require.NoError(t, release(mngr, "id", []string{"foo"})) 61 | require.NoError(t, release(mngr, "id", []string{"foo"})) 62 | }, 63 | "ReleaseIsAtomic": func(t *testing.T, mngr ScopeManager) { 64 | require.NoError(t, acquire(mngr, "id1", []string{"foo"})) 65 | require.NoError(t, acquire(mngr, "id2", []string{"bar", "bat"})) 66 | require.Error(t, release(mngr, "id2", []string{"bar", "foo"})) 67 | // If Release is atomic, "bar" should still be owned by "id2" so 68 | // this should error. 69 | require.Error(t, release(mngr, "id1", []string{"bar"})) 70 | }, 71 | } { 72 | t.Run(testName, func(t *testing.T) { 73 | testCase(t, NewLocalScopeManager()) 74 | }) 75 | } 76 | }) 77 | } 78 | }) 79 | } 80 | t.Run("ReleaseAndAcquire", func(t *testing.T) { 81 | for testName, testCase := range map[string]func(t *testing.T, mngr ScopeManager){ 82 | "AllowsJustRelease": func(t *testing.T, mngr ScopeManager) { 83 | require.NoError(t, mngr.ReleaseAndAcquire("id1", []string{"foo"}, "", nil)) 84 | }, 85 | "AllowsJustAcquire": func(t *testing.T, mngr ScopeManager) { 86 | require.NoError(t, mngr.ReleaseAndAcquire("", nil, "id2", []string{"bar"})) 87 | }, 88 | "IsReversible": func(t *testing.T, mngr ScopeManager) { 89 | require.NoError(t, mngr.ReleaseAndAcquire("id1", []string{"foo"}, "id2", []string{"bar"})) 90 | require.NoError(t, mngr.ReleaseAndAcquire("id2", []string{"bar"}, "id1", []string{"foo"})) 91 | }, 92 | "SwapsScopesFully": func(t *testing.T, mngr ScopeManager) { 93 | require.NoError(t, mngr.Acquire("id1", []string{"foo"})) 94 | require.NoError(t, mngr.ReleaseAndAcquire("id1", []string{"foo"}, "id2", []string{"foo"})) 95 | }, 96 | "SwapsScopesPartially": func(t *testing.T, mngr ScopeManager) { 97 | require.NoError(t, mngr.ReleaseAndAcquire("", nil, "id1", []string{"foo", "bar"})) 98 | require.NoError(t, mngr.ReleaseAndAcquire("id1", []string{"foo"}, "id2", []string{"foo"})) 99 | require.Error(t, mngr.Acquire("id2", []string{"bar"})) 100 | }, 101 | } { 102 | t.Run(testName, func(t *testing.T) { 103 | testCase(t, NewLocalScopeManager()) 104 | }) 105 | } 106 | }) 107 | } 108 | -------------------------------------------------------------------------------- /pool/mock_queue_test.go: -------------------------------------------------------------------------------- 1 | package pool 2 | 3 | import ( 4 | "context" 5 | "fmt" 6 | "sync" 7 | 8 | "github.com/google/uuid" 9 | "github.com/mongodb/amboy" 10 | "github.com/mongodb/amboy/job" 11 | "github.com/mongodb/grip" 12 | "github.com/pkg/errors" 13 | ) 14 | 15 | type QueueTester struct { 16 | started bool 17 | pool amboy.Runner 18 | retryHandler amboy.RetryHandler 19 | id string 20 | numComplete int 21 | toProcess chan amboy.Job 22 | storage map[string]amboy.Job 23 | 24 | mutex sync.RWMutex 25 | } 26 | 27 | func NewQueueTester(p amboy.Runner) *QueueTester { 28 | q := NewQueueTesterInstance() 29 | _ = p.SetQueue(q) 30 | q.pool = p 31 | 32 | return q 33 | } 34 | 35 | // Separate constructor for the object so we can avoid the side 36 | // effects of the extra SetQueue for tests where that doesn't make 37 | // sense. 38 | func NewQueueTesterInstance() *QueueTester { 39 | return &QueueTester{ 40 | toProcess: make(chan amboy.Job, 101), 41 | storage: make(map[string]amboy.Job), 42 | id: uuid.New().String(), 43 | } 44 | } 45 | 46 | func (q *QueueTester) Put(ctx context.Context, j amboy.Job) error { 47 | select { 48 | case <-ctx.Done(): 49 | return ctx.Err() 50 | case q.toProcess <- j: 51 | q.mutex.Lock() 52 | defer q.mutex.Unlock() 53 | 54 | q.storage[j.ID()] = j 55 | return nil 56 | } 57 | } 58 | 59 | func (q *QueueTester) PutMany(ctx context.Context, jobs []amboy.Job) error { 60 | catcher := grip.NewBasicCatcher() 61 | for _, j := range jobs { 62 | catcher.Wrapf(q.Put(ctx, j), "putting job '%s'", j.ID()) 63 | } 64 | return amboy.CollateWriteErrors(catcher.Errors()) 65 | } 66 | 67 | func (q *QueueTester) Save(ctx context.Context, j amboy.Job) error { 68 | q.mutex.Lock() 69 | defer q.mutex.Unlock() 70 | 71 | id := j.ID() 72 | if _, ok := q.storage[id]; !ok { 73 | return nil 74 | } 75 | q.storage[id] = j 76 | return nil 77 | } 78 | 79 | func (q *QueueTester) ID() string { return fmt.Sprintf("queue.tester.%s", q.id) } 80 | 81 | func (q *QueueTester) Get(ctx context.Context, name string) (amboy.Job, bool) { 82 | q.mutex.RLock() 83 | defer q.mutex.RUnlock() 84 | 85 | job, ok := q.storage[name] 86 | return job, ok 87 | } 88 | 89 | func (q *QueueTester) Info() amboy.QueueInfo { 90 | q.mutex.RLock() 91 | defer q.mutex.RUnlock() 92 | 93 | return q.info() 94 | } 95 | 96 | func (q *QueueTester) info() amboy.QueueInfo { 97 | return amboy.QueueInfo{ 98 | Started: q.started, 99 | LockTimeout: amboy.LockTimeout, 100 | } 101 | } 102 | 103 | func (q *QueueTester) Complete(ctx context.Context, j amboy.Job) error { 104 | q.mutex.Lock() 105 | defer q.mutex.Unlock() 106 | 107 | q.numComplete++ 108 | 109 | return nil 110 | } 111 | 112 | func (q *QueueTester) Stats(ctx context.Context) amboy.QueueStats { 113 | q.mutex.RLock() 114 | defer q.mutex.RUnlock() 115 | 116 | return amboy.QueueStats{ 117 | Running: len(q.storage) - len(q.toProcess), 118 | Completed: q.numComplete, 119 | Pending: len(q.toProcess), 120 | Total: len(q.storage), 121 | } 122 | } 123 | 124 | func (q *QueueTester) Runner() amboy.Runner { 125 | return q.pool 126 | } 127 | 128 | func (q *QueueTester) SetRunner(r amboy.Runner) error { 129 | if q.Info().Started { 130 | return errors.New("cannot set runner on active queue") 131 | } 132 | q.pool = r 133 | return nil 134 | } 135 | 136 | func (q *QueueTester) Next(ctx context.Context) amboy.Job { 137 | select { 138 | case <-ctx.Done(): 139 | return nil 140 | case job := <-q.toProcess: 141 | return job 142 | } 143 | } 144 | 145 | func (q *QueueTester) Start(ctx context.Context) error { 146 | if q.Info().Started { 147 | return nil 148 | } 149 | 150 | err := q.pool.Start(ctx) 151 | if err != nil { 152 | return errors.Wrap(err, "starting worker pool") 153 | } 154 | 155 | q.mutex.Lock() 156 | defer q.mutex.Unlock() 157 | 158 | q.started = true 159 | return nil 160 | } 161 | 162 | func (q *QueueTester) Results(ctx context.Context) <-chan amboy.Job { 163 | output := make(chan amboy.Job) 164 | 165 | go func() { 166 | defer close(output) 167 | for _, job := range q.storage { 168 | if ctx.Err() != nil { 169 | return 170 | } 171 | 172 | if job.Status().Completed { 173 | output <- job 174 | } 175 | } 176 | }() 177 | 178 | return output 179 | } 180 | 181 | func (q *QueueTester) JobInfo(ctx context.Context) <-chan amboy.JobInfo { 182 | infos := make(chan amboy.JobInfo) 183 | 184 | go func() { 185 | defer close(infos) 186 | for _, j := range q.storage { 187 | if ctx.Err() != nil { 188 | return 189 | 190 | } 191 | select { 192 | case <-ctx.Done(): 193 | return 194 | case infos <- amboy.NewJobInfo(j): 195 | } 196 | } 197 | }() 198 | 199 | return infos 200 | } 201 | 202 | func (q *QueueTester) Close(context.Context) {} 203 | 204 | type jobThatPanics struct { 205 | job.Base 206 | } 207 | 208 | func (j *jobThatPanics) Run(_ context.Context) { 209 | defer j.MarkComplete() 210 | 211 | panic("panic err") 212 | } 213 | 214 | func jobsChanWithPanicingJobs(ctx context.Context, num int) chan amboy.Job { 215 | out := make(chan amboy.Job) 216 | 217 | go func() { 218 | defer close(out) 219 | count := 0 220 | for { 221 | if count >= num { 222 | return 223 | } 224 | 225 | select { 226 | case <-ctx.Done(): 227 | return 228 | case out <- &jobThatPanics{}: 229 | count++ 230 | } 231 | } 232 | }() 233 | 234 | return out 235 | } 236 | -------------------------------------------------------------------------------- /rest/service_queue.go: -------------------------------------------------------------------------------- 1 | package rest 2 | 3 | import ( 4 | "context" 5 | "time" 6 | 7 | "github.com/evergreen-ci/gimlet" 8 | "github.com/mongodb/amboy" 9 | "github.com/mongodb/amboy/queue" 10 | "github.com/mongodb/amboy/registry" 11 | "github.com/mongodb/grip" 12 | "github.com/pkg/errors" 13 | ) 14 | 15 | // QueueService is used as a place holder for application state and configuration. 16 | type QueueService struct { 17 | queue amboy.Queue 18 | closer context.CancelFunc 19 | registeredTypes []string 20 | } 21 | 22 | // NewQueueService constructs a new service object. Use the Open() method 23 | // to initialize the service. The Open and OpenWithOptions methods 24 | // configure an embedded amboy service. If you use SetQueue you do not 25 | // need to call an open method. 26 | // 27 | // Use the App() method to get a gimlet Application that you can use 28 | // to run the service. 29 | func NewQueueService() *QueueService { 30 | service := &QueueService{} 31 | 32 | for name := range registry.JobTypeNames() { 33 | service.registeredTypes = append(service.registeredTypes, name) 34 | } 35 | 36 | return service 37 | } 38 | 39 | // App provides access to the gimplet.APIApp instance which builds and 40 | // orchestrates the REST API. Use this method if you want to combine 41 | // the routes in this QueueService with another service, or add additional 42 | // routes to support other application functionality. 43 | func (s *QueueService) App() *gimlet.APIApp { 44 | app := gimlet.NewApp() 45 | 46 | app.AddRoute("/").Version(0).Get().Handler(s.Status) 47 | app.AddRoute("/status").Version(1).Get().Handler(s.Status) 48 | app.AddRoute("/status/wait").Version(1).Get().Handler(s.WaitAll) 49 | app.AddRoute("/job/create").Version(1).Post().Handler(s.Create) 50 | app.AddRoute("/job/{name}").Version(1).Get().Handler(s.Fetch) 51 | app.AddRoute("/job/status/{name}").Version(1).Get().Handler(s.JobStatus) 52 | app.AddRoute("/job/wait/{name}").Version(1).Get().Handler(s.WaitJob) 53 | 54 | return app 55 | } 56 | 57 | // Open populates the application and starts the underlying 58 | // queue. This method sets and initializes a LocalLimitedSize queue 59 | // implementation, with 2 workers and storage for 256 jobs. Use 60 | // OpenInfo to have more control over the embedded queue. Use the 61 | // Close() method on the service to terminate the queue. 62 | func (s *QueueService) Open(ctx context.Context) error { 63 | opts := QueueServiceOptions{ 64 | ForceTimeout: time.Duration(0), 65 | QueueSize: 256, 66 | NumWorkers: 2, 67 | } 68 | 69 | if err := s.OpenWithOptions(ctx, opts); err != nil { 70 | return errors.Wrap(err, "could not open queue.") 71 | } 72 | 73 | return nil 74 | } 75 | 76 | // QueueServiceOptions provides a way to configure resources allocated by a service. 77 | type QueueServiceOptions struct { 78 | // ForceTimeout makes it possible to control how long to wait 79 | // for pending jobs to complete before canceling existing 80 | // work. If this value is zeroed, the Open operation will 81 | // close the previous queue and start a new queue, otherwise 82 | // it will attempt to wait for the specified time before closing 83 | // the previous queue. 84 | ForceTimeout time.Duration `bson:"force_timeout,omitempty" json:"force_timeout,omitempty" yaml:"force_timeout,omitempty"` 85 | 86 | // The default queue constructed by Open/OpenWith retains a 87 | // limited number of completed jobs to avoid unbounded memory 88 | // growth. This value *must* be specified. 89 | QueueSize int `bson:"queue_size" json:"queue_size" yaml:"queue_size"` 90 | 91 | // Controls the maximum number of go routines that can service 92 | // jobs in a queue. 93 | NumWorkers int `bson:"num_workers" json:"num_workers" yaml:"num_workers"` 94 | } 95 | 96 | // OpenWithOptions makes it possible to configure the underlying queue in a 97 | // service. Use the Close() method on the service to terminate the queue. 98 | func (s *QueueService) OpenWithOptions(ctx context.Context, opts QueueServiceOptions) error { 99 | if opts.NumWorkers == 0 || opts.QueueSize == 0 { 100 | return errors.Errorf("cannot build service with specified options: %+v", opts) 101 | } 102 | 103 | if s.closer != nil { 104 | if opts.ForceTimeout != 0 { 105 | waiterCtx, cancel := context.WithTimeout(ctx, opts.ForceTimeout) 106 | grip.Info("waiting for jobs to complete") 107 | amboy.Wait(waiterCtx, s.queue) 108 | cancel() 109 | } 110 | grip.Info("releasing remaining queue resources") 111 | s.closer() 112 | } 113 | 114 | ctx, cancel := context.WithCancel(ctx) 115 | s.closer = cancel 116 | 117 | s.queue = queue.NewLocalLimitedSize(opts.NumWorkers, opts.QueueSize) 118 | grip.Alert(s.queue.Start(ctx)) 119 | 120 | return nil 121 | } 122 | 123 | // Close releases resources (i.e. the queue) associated with the 124 | // service. If you've used SetQueue to define the embedded queue, 125 | // rather than Open/OpenWithOptions, 126 | func (s *QueueService) Close() { 127 | if s.closer != nil { 128 | s.closer() 129 | } 130 | } 131 | 132 | // Queue provides access to the underlying queue object for the service. 133 | func (s *QueueService) Queue() amboy.Queue { 134 | return s.queue 135 | } 136 | 137 | // SetQueue allows callers to inject an alternate queue implementation. 138 | func (s *QueueService) SetQueue(q amboy.Queue) error { 139 | if s.closer != nil { 140 | return errors.New("cannot set a new queue, QueueService is already open") 141 | } 142 | 143 | s.queue = q 144 | return nil 145 | } 146 | -------------------------------------------------------------------------------- /dependency/local_test.go: -------------------------------------------------------------------------------- 1 | package dependency 2 | 3 | import ( 4 | "fmt" 5 | "io/ioutil" 6 | "os" 7 | "path/filepath" 8 | "testing" 9 | "time" 10 | 11 | "github.com/google/uuid" 12 | "github.com/stretchr/testify/suite" 13 | ) 14 | 15 | // LocalFileSuite contains a group of tests for the 16 | // LocalFileDependency implementation of the dependency.Manager 17 | // interface, which provides make-style dependency checking. 18 | type LocalFileSuite struct { 19 | dep *LocalFile 20 | tmpDir string 21 | suite.Suite 22 | } 23 | 24 | func TestLocalFileSuite(t *testing.T) { 25 | suite.Run(t, new(LocalFileSuite)) 26 | } 27 | 28 | func (s *LocalFileSuite) SetupTest() { 29 | s.dep = MakeLocalFile() 30 | } 31 | 32 | func (s *LocalFileSuite) SetupSuite() { 33 | tmpDir, err := ioutil.TempDir("", uuid.New().String()) 34 | s.NoError(err) 35 | s.tmpDir = tmpDir 36 | } 37 | 38 | func (s *LocalFileSuite) TearDownSuite() { 39 | // s.NoError(os.RemoveAll(s.tmpDir)) 40 | } 41 | 42 | func (s *LocalFileSuite) TestTypeImplementsManagerInterface() { 43 | s.Implements((*Manager)(nil), s.dep) 44 | } 45 | 46 | func (s *LocalFileSuite) TestDefaultConstructorStoresExpectedValues() { 47 | s.dep = NewLocalFile("foo", "bar", "baz") 48 | s.Implements((*Manager)(nil), s.dep) 49 | 50 | s.Equal("foo", s.dep.Targets[0]) 51 | s.Len(s.dep.Targets, 1) 52 | 53 | s.Equal("bar", s.dep.Dependencies[0]) 54 | s.Equal("baz", s.dep.Dependencies[1]) 55 | s.Len(s.dep.Dependencies, 2) 56 | } 57 | 58 | func (s *LocalFileSuite) TestTaskEdgeTracking() { 59 | // edge defaults to empty 60 | s.Len(s.dep.Edges(), 0) 61 | 62 | s.NoError(s.dep.AddEdge("foo")) 63 | s.Len(s.dep.Edges(), 1) 64 | 65 | // make sure the internals look like we expect. 66 | s.Len(s.dep.edgesSet, 1) 67 | exists, ok := s.dep.edgesSet["foo"] 68 | s.True(exists) 69 | s.True(ok) 70 | } 71 | 72 | func (s *LocalFileSuite) TestLocalDependencyTestReportsExpectedType() { 73 | t := s.dep.Type() 74 | s.Equal(t.Name, "local-file") 75 | s.Equal(t.Version, 0) 76 | } 77 | 78 | func (s *LocalFileSuite) TestDependencyStateIsReadyWhenThereAreNoTargetsOrDependencies() { 79 | s.Len(s.dep.Targets, 0) 80 | s.Len(s.dep.Dependencies, 0) 81 | 82 | s.Equal(s.dep.State(), Ready) 83 | 84 | s.dep.Targets = append(s.dep.Targets, "foo") 85 | s.Equal(s.dep.State(), Ready) 86 | 87 | s.dep.Targets = []string{} 88 | s.dep.Dependencies = append(s.dep.Dependencies, "foo") 89 | s.Equal(s.dep.State(), Ready) 90 | } 91 | 92 | func (s *LocalFileSuite) TestDependencyStateIsReadyWhenTargetsAreOlderThanDependencies() { 93 | // first create 94 | for i := 0; i < 20; i++ { 95 | name := filepath.Join(s.tmpDir, fmt.Sprintf("target-%d", i)) 96 | s.dep.Targets = append(s.dep.Targets, name) 97 | file, err := os.Create(name) 98 | s.NoError(err) 99 | s.NoError(file.Close()) 100 | _, err = os.Stat(name) 101 | s.False(os.IsNotExist(err)) 102 | } 103 | 104 | // now sleep for a bit so the deps are newer 105 | time.Sleep(1 * time.Second) 106 | 107 | // then set things up to create dependencies 108 | for i := 0; i < 20; i++ { 109 | name := filepath.Join(s.tmpDir, fmt.Sprintf("dep-%d", i)) 110 | s.dep.Dependencies = append(s.dep.Dependencies, name) 111 | file, err := os.Create(name) 112 | s.NoError(err) 113 | s.NoError(file.Close()) 114 | _, err = os.Stat(name) 115 | s.False(os.IsNotExist(err)) 116 | } 117 | 118 | s.Equal(s.dep.State(), Ready) 119 | } 120 | 121 | func (s *LocalFileSuite) TestDependencyStateIsPassedWhenDependenciesAreOlderThanTargets() { 122 | // first set things up to create dependencies 123 | for i := 0; i < 20; i++ { 124 | name := filepath.Join(s.tmpDir, fmt.Sprintf("dep-%d", i)) 125 | s.dep.Dependencies = append(s.dep.Dependencies, name) 126 | file, err := os.Create(name) 127 | s.NoError(err) 128 | s.NoError(file.Close()) 129 | _, err = os.Stat(name) 130 | s.False(os.IsNotExist(err), name) 131 | } 132 | 133 | // now sleep for a bit so the targets are newer 134 | time.Sleep(1 * time.Second) 135 | 136 | // then create a bunch of targets 137 | for i := 0; i < 20; i++ { 138 | name := filepath.Join(s.tmpDir, fmt.Sprintf("target-%d", i)) 139 | s.dep.Targets = append(s.dep.Targets, name) 140 | file, err := os.Create(name) 141 | s.NoError(err) 142 | s.NoError(file.Close()) 143 | _, err = os.Stat(name) 144 | s.False(os.IsNotExist(err)) 145 | } 146 | 147 | s.Equal(s.dep.State(), Passed) 148 | } 149 | 150 | func (s *LocalFileSuite) TestDependencyStateIsReadyWhenOneDependencyIsNewerThanTargets() { 151 | // first set things up to create dependencies 152 | for i := 0; i < 20; i++ { 153 | name := filepath.Join(s.tmpDir, fmt.Sprintf("dep-%d", i)) 154 | s.dep.Dependencies = append(s.dep.Dependencies, name) 155 | file, err := os.Create(name) 156 | s.NoError(err) 157 | s.NoError(file.Close()) 158 | _, err = os.Stat(name) 159 | s.False(os.IsNotExist(err), name) 160 | } 161 | 162 | // now sleep for a bit so the targets are newer 163 | time.Sleep(1 * time.Second) 164 | 165 | // then create a bunch of targets 166 | for i := 0; i < 20; i++ { 167 | name := filepath.Join(s.tmpDir, fmt.Sprintf("target-%d", i)) 168 | s.dep.Targets = append(s.dep.Targets, name) 169 | file, err := os.Create(name) 170 | s.NoError(err) 171 | s.NoError(file.Close()) 172 | _, err = os.Stat(name) 173 | s.False(os.IsNotExist(err)) 174 | } 175 | 176 | // now sleep for a bit so the last dependency is newer 177 | time.Sleep(1 * time.Second) 178 | 179 | name := filepath.Join(s.tmpDir, "dep-newer") 180 | s.dep.Dependencies = append(s.dep.Dependencies, name) 181 | file, err := os.Create(name) 182 | s.NoError(err) 183 | s.NoError(file.Close()) 184 | _, err = os.Stat(name) 185 | s.False(os.IsNotExist(err), name) 186 | 187 | s.Equal(s.dep.State(), Ready) 188 | } 189 | -------------------------------------------------------------------------------- /queue/group_local.go: -------------------------------------------------------------------------------- 1 | package queue 2 | 3 | import ( 4 | "context" 5 | "time" 6 | 7 | "github.com/mongodb/amboy" 8 | "github.com/mongodb/grip" 9 | "github.com/mongodb/grip/message" 10 | "github.com/mongodb/grip/recovery" 11 | "github.com/pkg/errors" 12 | ) 13 | 14 | // localQueueGroup is a group of in-memory queues. 15 | type localQueueGroup struct { 16 | canceler context.CancelFunc 17 | opts LocalQueueGroupOptions 18 | cache GroupCache 19 | } 20 | 21 | // LocalQueueGroupOptions describe options passed to NewLocalQueueGroup. 22 | type LocalQueueGroupOptions struct { 23 | DefaultQueue LocalQueueOptions 24 | TTL time.Duration 25 | } 26 | 27 | func (o *LocalQueueGroupOptions) Validate() error { 28 | catcher := grip.NewBasicCatcher() 29 | catcher.NewWhen(o.TTL < 0, "TTL cannot be negative") 30 | catcher.NewWhen(o.TTL > 0 && o.TTL < time.Second, "TTL cannot be less than 1 second, unless it is 0") 31 | catcher.Wrap(o.DefaultQueue.Validate(), "invalid queue options") 32 | return catcher.Resolve() 33 | } 34 | 35 | // LocalQueueOptions represent options to construct a local queue. 36 | type LocalQueueOptions struct { 37 | Constructor func(ctx context.Context) (amboy.Queue, error) 38 | } 39 | 40 | func (o *LocalQueueOptions) BuildQueue(ctx context.Context) (amboy.Queue, error) { 41 | return o.Constructor(ctx) 42 | } 43 | 44 | func (o *LocalQueueOptions) Validate() error { 45 | if o.Constructor == nil { 46 | return errors.New("must specify a queue constructor") 47 | } 48 | return nil 49 | } 50 | 51 | func getLocalQueueOptions(opts ...amboy.QueueOptions) ([]LocalQueueOptions, error) { 52 | var localOpts []LocalQueueOptions 53 | 54 | for _, o := range opts { 55 | switch opt := o.(type) { 56 | case *LocalQueueOptions: 57 | if opt != nil { 58 | localOpts = append(localOpts, *opt) 59 | } 60 | default: 61 | return nil, errors.Errorf("found queue options of type '%T', but they must be local queue options", opt) 62 | } 63 | } 64 | 65 | return localOpts, nil 66 | } 67 | 68 | // mergeLocalQueueOptions merges all the given LocalQueueOptions into a single 69 | // set of options. Options are applied in the order they're specified and 70 | // conflicting options are overwritten. 71 | func mergeLocalQueueOptions(opts ...LocalQueueOptions) LocalQueueOptions { 72 | var merged LocalQueueOptions 73 | for _, o := range opts { 74 | if o.Constructor != nil { 75 | merged.Constructor = o.Constructor 76 | } 77 | } 78 | return merged 79 | } 80 | 81 | // NewLocalQueueGroup constructs a new local queue group. If ttl is 0, the queues will not be 82 | // TTLed except when the client explicitly calls Prune. 83 | func NewLocalQueueGroup(ctx context.Context, opts LocalQueueGroupOptions) (amboy.QueueGroup, error) { 84 | if err := opts.Validate(); err != nil { 85 | return nil, errors.Wrap(err, "invalid options") 86 | } 87 | g := &localQueueGroup{ 88 | opts: opts, 89 | cache: NewGroupCache(opts.TTL), 90 | } 91 | ctx, g.canceler = context.WithCancel(ctx) 92 | 93 | if opts.TTL > 0 { 94 | go func() { 95 | defer recovery.LogStackTraceAndContinue("panic in local queue group ticker") 96 | ticker := time.NewTicker(opts.TTL) 97 | defer ticker.Stop() 98 | for { 99 | select { 100 | case <-ctx.Done(): 101 | return 102 | case <-ticker.C: 103 | grip.Error(message.WrapError(g.Prune(ctx), 104 | message.Fields{ 105 | "group": "local queue group background pruning", 106 | "ttl": opts.TTL, 107 | })) 108 | } 109 | } 110 | }() 111 | } 112 | return g, nil 113 | } 114 | 115 | func (g *localQueueGroup) Len() int { return g.cache.Len() } 116 | 117 | func (g *localQueueGroup) Queues(_ context.Context) []string { 118 | return g.cache.Names() 119 | } 120 | 121 | // Get a queue with the given id. Get sets the last accessed time to now. Note 122 | // that this means that the time between when the queue is retrieved and when 123 | // the caller actually performs an operation on the queue (e.g. add a job) must 124 | // be within the TTL; otherwise, the queue might be closed before the operation 125 | // is done. 126 | func (g *localQueueGroup) Get(ctx context.Context, id string, opts ...amboy.QueueOptions) (amboy.Queue, error) { 127 | q := g.cache.Get(id) 128 | if q != nil { 129 | return q, nil 130 | } 131 | 132 | localQueueOpts, err := getLocalQueueOptions(opts...) 133 | if err != nil { 134 | return nil, errors.Wrap(err, "invalid queue options") 135 | } 136 | queueOpts := mergeLocalQueueOptions(append([]LocalQueueOptions{g.opts.DefaultQueue}, localQueueOpts...)...) 137 | if err := queueOpts.Validate(); err != nil { 138 | return nil, errors.Wrap(err, "invalid queue options") 139 | } 140 | queue, err := queueOpts.BuildQueue(ctx) 141 | if err != nil { 142 | return nil, errors.Wrap(err, "starting queue") 143 | } 144 | 145 | if err = g.cache.Set(id, queue, g.opts.TTL); err != nil { 146 | // It should be safe to throw away the queue here because another thread 147 | // already created it and we haven't started the workers. 148 | if q := g.cache.Get(id); q != nil { 149 | return q, nil 150 | } 151 | 152 | return nil, errors.Wrap(err, "caching queue") 153 | } 154 | 155 | if err = queue.Start(ctx); err != nil { 156 | return nil, errors.WithStack(err) 157 | } 158 | 159 | return queue, nil 160 | } 161 | 162 | // Put a queue at the given index. 163 | func (g *localQueueGroup) Put(ctx context.Context, id string, queue amboy.Queue) error { 164 | return errors.WithStack(g.cache.Set(id, queue, g.opts.TTL)) 165 | } 166 | 167 | // Prune old queues. 168 | func (g *localQueueGroup) Prune(ctx context.Context) error { return g.cache.Prune(ctx) } 169 | 170 | // Close the queues. 171 | func (g *localQueueGroup) Close(ctx context.Context) error { return g.cache.Close(ctx) } 172 | -------------------------------------------------------------------------------- /rest/service_management.go: -------------------------------------------------------------------------------- 1 | package rest 2 | 3 | import ( 4 | "net/http" 5 | 6 | "github.com/evergreen-ci/gimlet" 7 | "github.com/mongodb/amboy/management" 8 | "github.com/pkg/errors" 9 | ) 10 | 11 | // ManagementService wraps a manager instance as described in the management 12 | // package and provides an HTTP interface for all of the methods provided by 13 | // methods provided by the manager. 14 | type ManagementService struct { 15 | manager management.Manager 16 | } 17 | 18 | // NewManagementService constructs a management service from the manager 19 | // provided. 20 | func NewManagementService(m management.Manager) *ManagementService { 21 | return &ManagementService{ 22 | manager: m, 23 | } 24 | } 25 | 26 | // App returns a gimlet application with all of the routes 27 | // configured. 28 | func (s *ManagementService) App() *gimlet.APIApp { 29 | app := gimlet.NewApp() 30 | 31 | app.AddRoute("/status/{filter}").Version(1).Get().Handler(s.GetJobStatus) 32 | app.AddRoute("/id/status/{filter}/type/{type}").Version(1).Get().Handler(s.GetJobIDs) 33 | app.AddRoute("/jobs/mark_complete/id/{name}").Version(1).Post().Handler(s.MarkComplete) 34 | app.AddRoute("/jobs/mark_complete/status/{filter}").Version(1).Post().Handler(s.MarkManyComplete) 35 | app.AddRoute("/jobs/mark_complete/status/{filter}/type/{type}").Version(1).Post().Handler(s.MarkCompleteByType) 36 | app.AddRoute("/jobs/mark_complete/status/{filter}/pattern/{pattern}").Version(1).Post().Handler(s.MarkCompleteByPattern) 37 | 38 | return app 39 | } 40 | 41 | // GetJobStatus is an http.HandlerFunc that counts all jobs that match a status 42 | // filter. 43 | func (s *ManagementService) GetJobStatus(rw http.ResponseWriter, r *http.Request) { 44 | filter := management.StatusFilter(gimlet.GetVars(r)["filter"]) 45 | ctx := r.Context() 46 | 47 | err := filter.Validate() 48 | if err != nil { 49 | gimlet.WriteResponse(rw, gimlet.MakeJSONErrorResponder(err)) 50 | return 51 | } 52 | 53 | data, err := s.manager.JobStatus(ctx, filter) 54 | if err != nil { 55 | gimlet.WriteResponse(rw, gimlet.MakeJSONInternalErrorResponder(err)) 56 | return 57 | } 58 | 59 | gimlet.WriteJSON(rw, data) 60 | } 61 | 62 | // GetJobIDs is an http.HandlerFunc that produces a list of job IDs for jobs 63 | // that match a status filter and job type. 64 | func (s *ManagementService) GetJobIDs(rw http.ResponseWriter, r *http.Request) { 65 | vars := gimlet.GetVars(r) 66 | filter := management.StatusFilter(vars["filter"]) 67 | jobType := vars["type"] 68 | 69 | if err := filter.Validate(); err != nil { 70 | gimlet.WriteResponse(rw, gimlet.MakeJSONErrorResponder(err)) 71 | return 72 | } 73 | 74 | ctx := r.Context() 75 | data, err := s.manager.JobIDsByState(ctx, jobType, filter) 76 | if err != nil { 77 | gimlet.WriteResponse(rw, gimlet.MakeJSONInternalErrorResponder(err)) 78 | return 79 | } 80 | 81 | gimlet.WriteJSON(rw, data) 82 | } 83 | 84 | // MarkComplete is an http.Handlerfunc marks the given job complete. 85 | func (s *ManagementService) MarkComplete(rw http.ResponseWriter, r *http.Request) { 86 | vars := gimlet.GetVars(r) 87 | name := vars["name"] 88 | 89 | ctx := r.Context() 90 | if err := s.manager.CompleteJob(ctx, name); err != nil { 91 | gimlet.WriteResponse(rw, gimlet.MakeTextInternalErrorResponder(errors.Wrapf(err, "completing job '%s'", name))) 92 | return 93 | } 94 | 95 | gimlet.WriteJSON(rw, struct { 96 | Message string `json:"message"` 97 | JobName string `json:"job_name"` 98 | }{ 99 | Message: "mark job complete successful", 100 | JobName: name, 101 | }) 102 | } 103 | 104 | // MarkCompleteByType is an http.Handlerfunc marks all jobs of the given type 105 | // complete. 106 | func (s *ManagementService) MarkCompleteByType(rw http.ResponseWriter, r *http.Request) { 107 | vars := gimlet.GetVars(r) 108 | jobType := vars["type"] 109 | filter := vars["filter"] 110 | 111 | ctx := r.Context() 112 | if err := s.manager.CompleteJobsByType(ctx, management.StatusFilter(filter), jobType); err != nil { 113 | gimlet.WriteResponse(rw, gimlet.MakeTextInternalErrorResponder(errors.Wrapf(err, "completing jobs by type '%s'", jobType))) 114 | return 115 | } 116 | 117 | gimlet.WriteJSON(rw, struct { 118 | Message string `json:"message"` 119 | JobType string `json:"job_type"` 120 | }{ 121 | Message: "mark jobs complete by type successful", 122 | JobType: jobType, 123 | }) 124 | } 125 | 126 | // MarkManyComplete is an http.Handlerfunc marks all jobs of the 127 | // specified status complete. 128 | func (s *ManagementService) MarkManyComplete(rw http.ResponseWriter, r *http.Request) { 129 | vars := gimlet.GetVars(r) 130 | filter := vars["filter"] 131 | 132 | ctx := r.Context() 133 | if err := s.manager.CompleteJobs(ctx, management.StatusFilter(filter)); err != nil { 134 | gimlet.WriteResponse(rw, gimlet.MakeTextErrorResponder(errors.Wrapf(err, "completing jobs with filter '%s'", filter))) 135 | return 136 | } 137 | 138 | gimlet.WriteJSON(rw, struct { 139 | Message string `json:"message"` 140 | }{ 141 | Message: "mark jobs complete by filter successful", 142 | }) 143 | } 144 | 145 | // MarkCompleteByPattern is an http.Handlerfunc marks all jobs with the 146 | // specified pattern and status complete. 147 | func (s *ManagementService) MarkCompleteByPattern(rw http.ResponseWriter, r *http.Request) { 148 | vars := gimlet.GetVars(r) 149 | pattern := vars["pattern"] 150 | filter := vars["filter"] 151 | 152 | ctx := r.Context() 153 | if err := s.manager.CompleteJobsByPattern(ctx, management.StatusFilter(filter), pattern); err != nil { 154 | gimlet.WriteResponse(rw, gimlet.MakeTextInternalErrorResponder(errors.Wrapf(err, "completing jobs by pattern '%s' with filter '%s'", pattern, filter))) 155 | return 156 | } 157 | 158 | gimlet.WriteJSON(rw, struct { 159 | Message string `json:"message"` 160 | }{ 161 | Message: "mark jobs complete by pattern successful", 162 | }) 163 | } 164 | -------------------------------------------------------------------------------- /queue/group_util_test.go: -------------------------------------------------------------------------------- 1 | package queue 2 | 3 | import ( 4 | "context" 5 | "testing" 6 | "time" 7 | 8 | "github.com/stretchr/testify/require" 9 | ) 10 | 11 | func TestGroupCache(t *testing.T) { 12 | ctx, cancel := context.WithCancel(context.Background()) 13 | defer cancel() 14 | 15 | queue := NewLocalLimitedSize(2, 128) 16 | require.NotNil(t, queue) 17 | 18 | for _, impl := range []struct { 19 | name string 20 | factory func() GroupCache 21 | }{ 22 | { 23 | name: "BaseMinute", 24 | factory: func() GroupCache { return NewGroupCache(time.Minute) }, 25 | }, 26 | { 27 | name: "BaseZero", 28 | factory: func() GroupCache { return NewGroupCache(0) }, 29 | }, 30 | { 31 | name: "BaseHour", 32 | factory: func() GroupCache { return NewGroupCache(time.Hour) }, 33 | }, 34 | } { 35 | t.Run(impl.name, func(t *testing.T) { 36 | for _, test := range []struct { 37 | name string 38 | test func(*testing.T, GroupCache) 39 | }{ 40 | { 41 | name: "ValidateFixture", 42 | test: func(t *testing.T, cache GroupCache) { 43 | require.Len(t, cache.Names(), 0) 44 | require.Zero(t, cache.Len()) 45 | }, 46 | }, 47 | { 48 | name: "GetNilCase", 49 | test: func(t *testing.T, cache GroupCache) { 50 | require.Nil(t, cache.Get("foo")) 51 | }, 52 | }, 53 | { 54 | name: "SetNilCase", 55 | test: func(t *testing.T, cache GroupCache) { 56 | require.Error(t, cache.Set("foo", nil, 0)) 57 | }, 58 | }, 59 | { 60 | name: "SetZero", 61 | test: func(t *testing.T, cache GroupCache) { 62 | require.NoError(t, cache.Set("foo", queue, 0)) 63 | 64 | require.Len(t, cache.Names(), 1) 65 | require.Equal(t, 1, cache.Len()) 66 | }, 67 | }, 68 | { 69 | name: "DoubleSet", 70 | test: func(t *testing.T, cache GroupCache) { 71 | require.NoError(t, cache.Set("foo", queue, 0)) 72 | require.Error(t, cache.Set("foo", queue, 0)) 73 | }, 74 | }, 75 | { 76 | name: "RoundTrip", 77 | test: func(t *testing.T, cache GroupCache) { 78 | require.NoError(t, cache.Set("foo", queue, 0)) 79 | require.Equal(t, queue, cache.Get("foo")) 80 | }, 81 | }, 82 | { 83 | name: "RemoveNonExistantQueue", 84 | test: func(t *testing.T, cache GroupCache) { 85 | require.NoError(t, cache.Remove(ctx, "foo")) 86 | }, 87 | }, 88 | { 89 | name: "RemoveSteadyQueue", 90 | test: func(t *testing.T, cache GroupCache) { 91 | require.NoError(t, cache.Set("foo", queue, 0)) 92 | require.Equal(t, 1, cache.Len()) 93 | require.NoError(t, cache.Remove(ctx, "foo")) 94 | require.Equal(t, 0, cache.Len()) 95 | }, 96 | }, 97 | { 98 | name: "RemoveQueueWithWork", 99 | test: func(t *testing.T, cache GroupCache) { 100 | q := NewLocalLimitedSize(1, 128) 101 | require.NoError(t, q.Start(ctx)) 102 | j := newSleepJob() 103 | j.Sleep = time.Minute 104 | require.NoError(t, q.Put(ctx, j)) 105 | 106 | require.NoError(t, cache.Set("foo", q, 1)) 107 | require.Equal(t, 1, cache.Len()) 108 | require.Error(t, cache.Remove(ctx, "foo")) 109 | require.Equal(t, 1, cache.Len()) 110 | }, 111 | }, 112 | { 113 | name: "PruneNil", 114 | test: func(t *testing.T, cache GroupCache) { 115 | require.NoError(t, cache.Prune(ctx)) 116 | }, 117 | }, 118 | { 119 | name: "PruneOne", 120 | test: func(t *testing.T, cache GroupCache) { 121 | require.NoError(t, cache.Set("foo", queue, time.Millisecond)) 122 | time.Sleep(2 * time.Millisecond) 123 | require.NoError(t, cache.Prune(ctx)) 124 | require.Zero(t, cache.Len()) 125 | }, 126 | }, 127 | { 128 | name: "PruneWithCanceledContext", 129 | test: func(t *testing.T, cache GroupCache) { 130 | tctx, cancel := context.WithCancel(ctx) 131 | cancel() 132 | 133 | require.NoError(t, cache.Set("foo", queue, time.Hour)) 134 | require.Equal(t, 1, cache.Len()) 135 | require.Error(t, cache.Prune(tctx)) 136 | require.Equal(t, 1, cache.Len()) 137 | }, 138 | }, 139 | { 140 | name: "PruneWithUnexpiredTTL", 141 | test: func(t *testing.T, cache GroupCache) { 142 | require.NoError(t, cache.Set("foo", queue, time.Hour)) 143 | require.Equal(t, 1, cache.Len()) 144 | require.NoError(t, cache.Prune(ctx)) 145 | require.Equal(t, 1, cache.Len()) 146 | }, 147 | }, 148 | { 149 | name: "CloseNoop", 150 | test: func(t *testing.T, cache GroupCache) { 151 | require.NoError(t, cache.Close(ctx)) 152 | }, 153 | }, 154 | { 155 | name: "CloseErrorsCtxCancel", 156 | test: func(t *testing.T, cache GroupCache) { 157 | tctx, cancel := context.WithCancel(ctx) 158 | cancel() 159 | 160 | require.NoError(t, cache.Set("foo", queue, time.Hour)) 161 | require.Equal(t, 1, cache.Len()) 162 | require.Error(t, cache.Close(tctx)) 163 | require.Equal(t, 1, cache.Len()) 164 | }, 165 | }, 166 | { 167 | name: "CloseEmptyErrorsCtxCancel", 168 | test: func(t *testing.T, cache GroupCache) { 169 | tctx, cancel := context.WithCancel(ctx) 170 | cancel() 171 | 172 | require.Error(t, cache.Close(tctx)) 173 | }, 174 | }, 175 | { 176 | name: "ClosingClearsQueue", 177 | test: func(t *testing.T, cache GroupCache) { 178 | require.NoError(t, cache.Set("foo", queue, time.Hour)) 179 | require.Equal(t, 1, cache.Len()) 180 | require.NoError(t, cache.Close(ctx)) 181 | require.Equal(t, 0, cache.Len()) 182 | 183 | }, 184 | }, 185 | } { 186 | t.Run(test.name, func(t *testing.T) { 187 | cache := impl.factory() 188 | require.NotNil(t, cache) 189 | 190 | test.test(t, cache) 191 | }) 192 | } 193 | }) 194 | } 195 | 196 | } 197 | --------------------------------------------------------------------------------