summaryrefslogtreecommitdiff
path: root/plugins/jobs/brokers/amqp
diff options
context:
space:
mode:
authorValery Piashchynski <[email protected]>2021-07-12 12:45:53 +0300
committerValery Piashchynski <[email protected]>2021-07-12 12:45:53 +0300
commitaa1437d24ac215bec7fe053b06fa4773c9b1b1ad (patch)
tree7a6868867877f34ac5e2d490bfb589b3dce02917 /plugins/jobs/brokers/amqp
parent87971c4d310fe3d353197fc96b9b6f9106f01e57 (diff)
Update JOBS interface, remove List() method, implemented on the root RPC
level. AMQP consumer replace sync.Map with atomic.Value, because we associate only 1 pipeline with a driver. So, we can store pipeline in the atomic.Value. Implement events handler, add job events. Use job events to push information to the logger. Signed-off-by: Valery Piashchynski <[email protected]>
Diffstat (limited to 'plugins/jobs/brokers/amqp')
-rw-r--r--plugins/jobs/brokers/amqp/consumer.go67
-rw-r--r--plugins/jobs/brokers/amqp/plugin.go9
-rw-r--r--plugins/jobs/brokers/amqp/rabbit_init.go10
-rw-r--r--plugins/jobs/brokers/amqp/redial.go22
4 files changed, 63 insertions, 45 deletions
diff --git a/plugins/jobs/brokers/amqp/consumer.go b/plugins/jobs/brokers/amqp/consumer.go
index a7916f7e..481e102a 100644
--- a/plugins/jobs/brokers/amqp/consumer.go
+++ b/plugins/jobs/brokers/amqp/consumer.go
@@ -3,10 +3,12 @@ package amqp
import (
"fmt"
"sync"
+ "sync/atomic"
"time"
"github.com/google/uuid"
"github.com/spiral/errors"
+ "github.com/spiral/roadrunner/v2/pkg/events"
priorityqueue "github.com/spiral/roadrunner/v2/pkg/priority_queue"
"github.com/spiral/roadrunner/v2/plugins/config"
"github.com/spiral/roadrunner/v2/plugins/jobs/pipeline"
@@ -48,8 +50,9 @@ type JobsConsumer struct {
sync.RWMutex
log logger.Logger
pq priorityqueue.Queue
+ eh events.Handler
- pipelines sync.Map
+ pipeline atomic.Value
// amqp connection
conn *amqp.Connection
@@ -71,7 +74,7 @@ type JobsConsumer struct {
}
// NewAMQPConsumer initializes rabbitmq pipeline
-func NewAMQPConsumer(configKey string, log logger.Logger, cfg config.Configurer, pq priorityqueue.Queue) (*JobsConsumer, error) {
+func NewAMQPConsumer(configKey string, log logger.Logger, cfg config.Configurer, e events.Handler, pq priorityqueue.Queue) (*JobsConsumer, error) {
const op = errors.Op("new_amqp_consumer")
// we need to obtain two parts of the amqp information here.
// firs part - address to connect, it is located in the global section under the amqp pluginName
@@ -79,6 +82,7 @@ func NewAMQPConsumer(configKey string, log logger.Logger, cfg config.Configurer,
jb := &JobsConsumer{
log: log,
pq: pq,
+ eh: e,
consumeID: uuid.NewString(),
stopCh: make(chan struct{}),
// TODO to config
@@ -146,13 +150,14 @@ func NewAMQPConsumer(configKey string, log logger.Logger, cfg config.Configurer,
return jb, nil
}
-func FromPipeline(pipeline *pipeline.Pipeline, log logger.Logger, cfg config.Configurer, pq priorityqueue.Queue) (*JobsConsumer, error) {
+func FromPipeline(pipeline *pipeline.Pipeline, log logger.Logger, cfg config.Configurer, e events.Handler, pq priorityqueue.Queue) (*JobsConsumer, error) {
const op = errors.Op("new_amqp_consumer_from_pipeline")
// we need to obtain two parts of the amqp information here.
// firs part - address to connect, it is located in the global section under the amqp pluginName
// second part - queues and other pipeline information
jb := &JobsConsumer{
log: log,
+ eh: e,
pq: pq,
consumeID: uuid.NewString(),
stopCh: make(chan struct{}),
@@ -210,8 +215,11 @@ func FromPipeline(pipeline *pipeline.Pipeline, log logger.Logger, cfg config.Con
func (j *JobsConsumer) Push(job *structs.Job) error {
const op = errors.Op("rabbitmq_push")
// check if the pipeline registered
- if _, ok := j.pipelines.Load(job.Options.Pipeline); !ok {
- return errors.E(op, errors.Errorf("no such pipeline: %s", job.Options.Pipeline))
+
+ // load atomic value
+ pipe := j.pipeline.Load().(*pipeline.Pipeline)
+ if pipe.Name() != job.Options.Pipeline {
+ return errors.E(op, errors.Errorf("no such pipeline: %s, actual: %s", job.Options.Pipeline, pipe.Name()))
}
// lock needed here to protect redial concurrent operation
@@ -303,20 +311,16 @@ func (j *JobsConsumer) Push(job *structs.Job) error {
}
func (j *JobsConsumer) Register(pipeline *pipeline.Pipeline) error {
- const op = errors.Op("rabbitmq_register")
- if _, ok := j.pipelines.Load(pipeline.Name()); ok {
- return errors.E(op, errors.Errorf("queue %s has already been registered", pipeline))
- }
-
- j.pipelines.Store(pipeline.Name(), struct{}{})
-
+ j.pipeline.Store(pipeline)
return nil
}
-func (j *JobsConsumer) Run(pipeline *pipeline.Pipeline) error {
+func (j *JobsConsumer) Run(p *pipeline.Pipeline) error {
const op = errors.Op("rabbit_consume")
- if _, ok := j.pipelines.Load(pipeline.Name()); !ok {
- return errors.E(op, errors.Errorf("no such pipeline registered: %s", pipeline.Name()))
+
+ pipe := j.pipeline.Load().(*pipeline.Pipeline)
+ if pipe.Name() != p.Name() {
+ return errors.E(op, errors.Errorf("no such pipeline registered: %s", pipe.Name()))
}
// protect connection (redial)
@@ -354,22 +358,10 @@ func (j *JobsConsumer) Run(pipeline *pipeline.Pipeline) error {
return nil
}
-func (j *JobsConsumer) List() []string {
- out := make([]string, 0, 2)
-
- j.pipelines.Range(func(key, value interface{}) bool {
- pipe := key.(string)
- out = append(out, pipe)
- return true
- })
-
- return out
-}
-
-func (j *JobsConsumer) Pause(pipeline string) {
- if _, ok := j.pipelines.Load(pipeline); !ok {
- j.log.Error("no such pipeline", "requested pause on", pipeline)
- return
+func (j *JobsConsumer) Pause(p string) {
+ pipe := j.pipeline.Load().(*pipeline.Pipeline)
+ if pipe.Name() != p {
+ j.log.Error("no such pipeline", "requested pause on: ", p)
}
// protect connection (redial)
@@ -386,10 +378,10 @@ func (j *JobsConsumer) Pause(pipeline string) {
}
}
-func (j *JobsConsumer) Resume(pipeline string) {
- if _, ok := j.pipelines.Load(pipeline); !ok {
- j.log.Error("no such pipeline", "requested pause on", pipeline)
- return
+func (j *JobsConsumer) Resume(p string) {
+ pipe := j.pipeline.Load().(*pipeline.Pipeline)
+ if pipe.Name() != p {
+ j.log.Error("no such pipeline", "requested resume on: ", p)
}
// protect connection (redial)
@@ -430,11 +422,6 @@ func (j *JobsConsumer) Resume(pipeline string) {
func (j *JobsConsumer) Stop() error {
j.stopCh <- struct{}{}
- j.pipelines.Range(func(key, _ interface{}) bool {
- j.pipelines.Delete(key)
- return true
- })
-
return nil
}
diff --git a/plugins/jobs/brokers/amqp/plugin.go b/plugins/jobs/brokers/amqp/plugin.go
index ca972c5b..624f4405 100644
--- a/plugins/jobs/brokers/amqp/plugin.go
+++ b/plugins/jobs/brokers/amqp/plugin.go
@@ -2,6 +2,7 @@ package amqp
import (
"github.com/spiral/roadrunner/v2/common/jobs"
+ "github.com/spiral/roadrunner/v2/pkg/events"
priorityqueue "github.com/spiral/roadrunner/v2/pkg/priority_queue"
"github.com/spiral/roadrunner/v2/plugins/config"
"github.com/spiral/roadrunner/v2/plugins/jobs/pipeline"
@@ -29,11 +30,11 @@ func (p *Plugin) Name() string {
func (p *Plugin) Available() {}
-func (p *Plugin) JobsConstruct(configKey string, pq priorityqueue.Queue) (jobs.Consumer, error) {
- return NewAMQPConsumer(configKey, p.log, p.cfg, pq)
+func (p *Plugin) JobsConstruct(configKey string, e events.Handler, pq priorityqueue.Queue) (jobs.Consumer, error) {
+ return NewAMQPConsumer(configKey, p.log, p.cfg, e, pq)
}
// FromPipeline constructs AMQP driver from pipeline
-func (p *Plugin) FromPipeline(pipe *pipeline.Pipeline, pq priorityqueue.Queue) (jobs.Consumer, error) {
- return FromPipeline(pipe, p.log, p.cfg, pq)
+func (p *Plugin) FromPipeline(pipe *pipeline.Pipeline, e events.Handler, pq priorityqueue.Queue) (jobs.Consumer, error) {
+ return FromPipeline(pipe, p.log, p.cfg, e, pq)
}
diff --git a/plugins/jobs/brokers/amqp/rabbit_init.go b/plugins/jobs/brokers/amqp/rabbit_init.go
index cb9f2dc4..e3e5f8da 100644
--- a/plugins/jobs/brokers/amqp/rabbit_init.go
+++ b/plugins/jobs/brokers/amqp/rabbit_init.go
@@ -1,11 +1,14 @@
package amqp
import (
+ "time"
+
"github.com/spiral/errors"
+ "github.com/spiral/roadrunner/v2/pkg/events"
)
func (j *JobsConsumer) initRabbitMQ() error {
- const op = errors.Op("rabbit_initmq")
+ const op = errors.Op("jobs_plugin_rmq_init")
// Channel opens a unique, concurrent server channel to process the bulk of AMQP
// messages. Any error from methods on this receiver will render the receiver
// invalid and a new Channel should be opened.
@@ -53,5 +56,10 @@ func (j *JobsConsumer) initRabbitMQ() error {
return errors.E(op, err)
}
+ j.eh.Push(events.JobEvent{
+ Event: events.EventInitialized,
+ Driver: "amqp",
+ Start: time.Now(),
+ })
return channel.Close()
}
diff --git a/plugins/jobs/brokers/amqp/redial.go b/plugins/jobs/brokers/amqp/redial.go
index 277e75b7..571ee548 100644
--- a/plugins/jobs/brokers/amqp/redial.go
+++ b/plugins/jobs/brokers/amqp/redial.go
@@ -2,9 +2,12 @@ package amqp
import (
"fmt"
+ "time"
"github.com/cenkalti/backoff/v4"
"github.com/spiral/errors"
+ "github.com/spiral/roadrunner/v2/pkg/events"
+ "github.com/spiral/roadrunner/v2/plugins/jobs/pipeline"
"github.com/streadway/amqp"
)
@@ -22,6 +25,17 @@ func (j *JobsConsumer) redialer() { //nolint:gocognit
j.Lock()
+ t := time.Now()
+ pipe := j.pipeline.Load().(*pipeline.Pipeline)
+ j.eh.Push(events.JobEvent{
+ Event: events.EventPipeError,
+ Pipeline: pipe.Name(),
+ Driver: pipe.Driver(),
+ Error: err,
+ Start: time.Now(),
+ Elapsed: 0,
+ })
+
j.log.Error("connection closed, reconnecting", "error", err)
expb := backoff.NewExponentialBackOff()
// set the retry timeout (minutes)
@@ -85,6 +99,14 @@ func (j *JobsConsumer) redialer() { //nolint:gocognit
return
}
+ j.eh.Push(events.JobEvent{
+ Event: events.EventPipeActive,
+ Pipeline: pipe.Name(),
+ Driver: pipe.Driver(),
+ Start: t,
+ Elapsed: time.Since(t),
+ })
+
j.Unlock()
case <-j.stopCh: