summaryrefslogtreecommitdiff
path: root/plugins/memory/memoryjobs
diff options
context:
space:
mode:
Diffstat (limited to 'plugins/memory/memoryjobs')
-rw-r--r--plugins/memory/memoryjobs/consumer.go269
-rw-r--r--plugins/memory/memoryjobs/item.go133
2 files changed, 402 insertions, 0 deletions
diff --git a/plugins/memory/memoryjobs/consumer.go b/plugins/memory/memoryjobs/consumer.go
new file mode 100644
index 00000000..94abaadb
--- /dev/null
+++ b/plugins/memory/memoryjobs/consumer.go
@@ -0,0 +1,269 @@
+package memoryjobs
+
+import (
+ "context"
+ "sync/atomic"
+ "time"
+
+ "github.com/spiral/errors"
+ "github.com/spiral/roadrunner/v2/pkg/events"
+ priorityqueue "github.com/spiral/roadrunner/v2/pkg/priority_queue"
+ jobState "github.com/spiral/roadrunner/v2/pkg/state/job"
+ "github.com/spiral/roadrunner/v2/plugins/config"
+ "github.com/spiral/roadrunner/v2/plugins/jobs/job"
+ "github.com/spiral/roadrunner/v2/plugins/jobs/pipeline"
+ "github.com/spiral/roadrunner/v2/plugins/logger"
+ "github.com/spiral/roadrunner/v2/utils"
+)
+
+const (
+ prefetch string = "prefetch"
+ goroutinesMax uint64 = 1000
+)
+
+type Config struct {
+ Prefetch uint64 `mapstructure:"prefetch"`
+}
+
+type consumer struct {
+ cfg *Config
+ log logger.Logger
+ eh events.Handler
+ pipeline atomic.Value
+ pq priorityqueue.Queue
+ localPrefetch chan *Item
+
+ // time.sleep goroutines max number
+ goroutines uint64
+
+ delayed *int64
+ active *int64
+
+ listeners uint32
+ stopCh chan struct{}
+}
+
+func NewJobBroker(configKey string, log logger.Logger, cfg config.Configurer, eh events.Handler, pq priorityqueue.Queue) (*consumer, error) {
+ const op = errors.Op("new_ephemeral_pipeline")
+
+ jb := &consumer{
+ log: log,
+ pq: pq,
+ eh: eh,
+ goroutines: 0,
+ active: utils.Int64(0),
+ delayed: utils.Int64(0),
+ stopCh: make(chan struct{}, 1),
+ }
+
+ err := cfg.UnmarshalKey(configKey, &jb.cfg)
+ if err != nil {
+ return nil, errors.E(op, err)
+ }
+
+ if jb.cfg.Prefetch == 0 {
+ jb.cfg.Prefetch = 100_000
+ }
+
+ // initialize a local queue
+ jb.localPrefetch = make(chan *Item, jb.cfg.Prefetch)
+
+ return jb, nil
+}
+
+func FromPipeline(pipeline *pipeline.Pipeline, log logger.Logger, eh events.Handler, pq priorityqueue.Queue) (*consumer, error) {
+ jb := &consumer{
+ log: log,
+ pq: pq,
+ eh: eh,
+ goroutines: 0,
+ active: utils.Int64(0),
+ delayed: utils.Int64(0),
+ stopCh: make(chan struct{}, 1),
+ }
+
+ // initialize a local queue
+ jb.localPrefetch = make(chan *Item, pipeline.Int(prefetch, 100_000))
+
+ return jb, nil
+}
+
+func (c *consumer) Push(ctx context.Context, jb *job.Job) error {
+ const op = errors.Op("ephemeral_push")
+
+ // check if the pipeline registered
+ _, ok := c.pipeline.Load().(*pipeline.Pipeline)
+ if !ok {
+ return errors.E(op, errors.Errorf("no such pipeline: %s", jb.Options.Pipeline))
+ }
+
+ err := c.handleItem(ctx, fromJob(jb))
+ if err != nil {
+ return errors.E(op, err)
+ }
+
+ return nil
+}
+
+func (c *consumer) State(_ context.Context) (*jobState.State, error) {
+ pipe := c.pipeline.Load().(*pipeline.Pipeline)
+ return &jobState.State{
+ Pipeline: pipe.Name(),
+ Driver: pipe.Driver(),
+ Queue: pipe.Name(),
+ Active: atomic.LoadInt64(c.active),
+ Delayed: atomic.LoadInt64(c.delayed),
+ Ready: ready(atomic.LoadUint32(&c.listeners)),
+ }, nil
+}
+
+func (c *consumer) Register(_ context.Context, pipeline *pipeline.Pipeline) error {
+ c.pipeline.Store(pipeline)
+ return nil
+}
+
+func (c *consumer) Pause(_ context.Context, p string) {
+ pipe := c.pipeline.Load().(*pipeline.Pipeline)
+ if pipe.Name() != p {
+ c.log.Error("no such pipeline", "requested pause on: ", p)
+ }
+
+ l := atomic.LoadUint32(&c.listeners)
+ // no active listeners
+ if l == 0 {
+ c.log.Warn("no active listeners, nothing to pause")
+ return
+ }
+
+ atomic.AddUint32(&c.listeners, ^uint32(0))
+
+ // stop the consumer
+ c.stopCh <- struct{}{}
+
+ c.eh.Push(events.JobEvent{
+ Event: events.EventPipePaused,
+ Driver: pipe.Driver(),
+ Pipeline: pipe.Name(),
+ Start: time.Now(),
+ Elapsed: 0,
+ })
+}
+
+func (c *consumer) Resume(_ context.Context, p string) {
+ pipe := c.pipeline.Load().(*pipeline.Pipeline)
+ if pipe.Name() != p {
+ c.log.Error("no such pipeline", "requested resume on: ", p)
+ }
+
+ l := atomic.LoadUint32(&c.listeners)
+ // listener already active
+ if l == 1 {
+ c.log.Warn("listener already in the active state")
+ return
+ }
+
+ // resume the consumer on the same channel
+ c.consume()
+
+ atomic.StoreUint32(&c.listeners, 1)
+ c.eh.Push(events.JobEvent{
+ Event: events.EventPipeActive,
+ Pipeline: pipe.Name(),
+ Start: time.Now(),
+ Elapsed: 0,
+ })
+}
+
+// Run is no-op for the ephemeral
+func (c *consumer) Run(_ context.Context, pipe *pipeline.Pipeline) error {
+ c.eh.Push(events.JobEvent{
+ Event: events.EventPipeActive,
+ Driver: pipe.Driver(),
+ Pipeline: pipe.Name(),
+ Start: time.Now(),
+ })
+ return nil
+}
+
+func (c *consumer) Stop(_ context.Context) error {
+ pipe := c.pipeline.Load().(*pipeline.Pipeline)
+
+ if atomic.LoadUint32(&c.listeners) > 0 {
+ c.stopCh <- struct{}{}
+ }
+
+ c.eh.Push(events.JobEvent{
+ Event: events.EventPipeStopped,
+ Pipeline: pipe.Name(),
+ Start: time.Now(),
+ Elapsed: 0,
+ })
+
+ return nil
+}
+
+func (c *consumer) handleItem(ctx context.Context, msg *Item) error {
+ const op = errors.Op("ephemeral_handle_request")
+ // handle timeouts
+ // theoretically, some bad user may send millions requests with a delay and produce a billion (for example)
+ // goroutines here. We should limit goroutines here.
+ if msg.Options.Delay > 0 {
+ // if we have 1000 goroutines waiting on the delay - reject 1001
+ if atomic.LoadUint64(&c.goroutines) >= goroutinesMax {
+ return errors.E(op, errors.Str("max concurrency number reached"))
+ }
+
+ go func(jj *Item) {
+ atomic.AddUint64(&c.goroutines, 1)
+ atomic.AddInt64(c.delayed, 1)
+
+ time.Sleep(jj.Options.DelayDuration())
+
+ // send the item after timeout expired
+ c.localPrefetch <- jj
+
+ atomic.AddUint64(&c.goroutines, ^uint64(0))
+ }(msg)
+
+ return nil
+ }
+
+ // increase number of the active jobs
+ atomic.AddInt64(c.active, 1)
+
+ // insert to the local, limited pipeline
+ select {
+ case c.localPrefetch <- msg:
+ return nil
+ case <-ctx.Done():
+ return errors.E(op, errors.Errorf("local pipeline is full, consider to increase prefetch number, current limit: %d, context error: %v", c.cfg.Prefetch, ctx.Err()))
+ }
+}
+
+func (c *consumer) consume() {
+ go func() {
+ // redirect
+ for {
+ select {
+ case item, ok := <-c.localPrefetch:
+ if !ok {
+ c.log.Warn("ephemeral local prefetch queue was closed")
+ return
+ }
+
+ // set requeue channel
+ item.Options.requeueFn = c.handleItem
+ item.Options.active = c.active
+ item.Options.delayed = c.delayed
+
+ c.pq.Insert(item)
+ case <-c.stopCh:
+ return
+ }
+ }
+ }()
+}
+
+func ready(r uint32) bool {
+ return r > 0
+}
diff --git a/plugins/memory/memoryjobs/item.go b/plugins/memory/memoryjobs/item.go
new file mode 100644
index 00000000..8224c26b
--- /dev/null
+++ b/plugins/memory/memoryjobs/item.go
@@ -0,0 +1,133 @@
+package memoryjobs
+
+import (
+ "context"
+ "sync/atomic"
+ "time"
+
+ json "github.com/json-iterator/go"
+ "github.com/spiral/roadrunner/v2/plugins/jobs/job"
+ "github.com/spiral/roadrunner/v2/utils"
+)
+
+type Item struct {
+ // Job contains name of job broker (usually PHP class).
+ Job string `json:"job"`
+
+ // Ident is unique identifier of the job, should be provided from outside
+ Ident string `json:"id"`
+
+ // Payload is string data (usually JSON) passed to Job broker.
+ Payload string `json:"payload"`
+
+ // Headers with key-values pairs
+ Headers map[string][]string `json:"headers"`
+
+ // Options contains set of PipelineOptions specific to job execution. Can be empty.
+ Options *Options `json:"options,omitempty"`
+}
+
+// Options carry information about how to handle given job.
+type Options struct {
+ // Priority is job priority, default - 10
+ // pointer to distinguish 0 as a priority and nil as priority not set
+ Priority int64 `json:"priority"`
+
+ // Pipeline manually specified pipeline.
+ Pipeline string `json:"pipeline,omitempty"`
+
+ // Delay defines time duration to delay execution for. Defaults to none.
+ Delay int64 `json:"delay,omitempty"`
+
+ // private
+ requeueFn func(context.Context, *Item) error
+ active *int64
+ delayed *int64
+}
+
+// DelayDuration returns delay duration in a form of time.Duration.
+func (o *Options) DelayDuration() time.Duration {
+ return time.Second * time.Duration(o.Delay)
+}
+
+func (i *Item) ID() string {
+ return i.Ident
+}
+
+func (i *Item) Priority() int64 {
+ return i.Options.Priority
+}
+
+// Body packs job payload into binary payload.
+func (i *Item) Body() []byte {
+ return utils.AsBytes(i.Payload)
+}
+
+// Context packs job context (job, id) into binary payload.
+func (i *Item) Context() ([]byte, error) {
+ ctx, err := json.Marshal(
+ struct {
+ ID string `json:"id"`
+ Job string `json:"job"`
+ Headers map[string][]string `json:"headers"`
+ Pipeline string `json:"pipeline"`
+ }{ID: i.Ident, Job: i.Job, Headers: i.Headers, Pipeline: i.Options.Pipeline},
+ )
+
+ if err != nil {
+ return nil, err
+ }
+
+ return ctx, nil
+}
+
+func (i *Item) Ack() error {
+ i.atomicallyReduceCount()
+ return nil
+}
+
+func (i *Item) Nack() error {
+ i.atomicallyReduceCount()
+ return nil
+}
+
+func (i *Item) Requeue(headers map[string][]string, delay int64) error {
+ // overwrite the delay
+ i.Options.Delay = delay
+ i.Headers = headers
+
+ i.atomicallyReduceCount()
+
+ err := i.Options.requeueFn(context.Background(), i)
+ if err != nil {
+ return err
+ }
+
+ return nil
+}
+
+// atomicallyReduceCount reduces counter of active or delayed jobs
+func (i *Item) atomicallyReduceCount() {
+ // if job was delayed, reduce number of the delayed jobs
+ if i.Options.Delay > 0 {
+ atomic.AddInt64(i.Options.delayed, ^int64(0))
+ return
+ }
+
+ // otherwise, reduce number of the active jobs
+ atomic.AddInt64(i.Options.active, ^int64(0))
+ // noop for the in-memory
+}
+
+func fromJob(job *job.Job) *Item {
+ return &Item{
+ Job: job.Job,
+ Ident: job.Ident,
+ Payload: job.Payload,
+ Options: &Options{
+ Priority: job.Options.Priority,
+ Pipeline: job.Options.Pipeline,
+ Delay: job.Options.Delay,
+ },
+ }
+}