summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorValery Piashchynski <[email protected]>2021-08-12 13:25:36 +0300
committerValery Piashchynski <[email protected]>2021-08-12 13:25:36 +0300
commitecbfc5c5265a9895f4e371ce4388f64df8714e63 (patch)
treedf0749155487eae6bcdbb2456885131a21916f4d
parent4169e8374f581ba2213f8cd1833cc6b9b84438e8 (diff)
Remove unneeded options, complete tests for the ephemeral, update proto
Signed-off-by: Valery Piashchynski <[email protected]>
-rw-r--r--pkg/priority_queue/binary_heap_test.go2
-rw-r--r--pkg/priority_queue/interface.go3
-rw-r--r--plugins/jobs/drivers/amqp/item.go25
-rw-r--r--plugins/jobs/drivers/beanstalk/consumer.go2
-rw-r--r--plugins/jobs/drivers/beanstalk/encode_test.go2
-rw-r--r--plugins/jobs/drivers/beanstalk/item.go26
-rw-r--r--plugins/jobs/drivers/ephemeral/consumer.go34
-rw-r--r--plugins/jobs/drivers/ephemeral/item.go44
-rw-r--r--plugins/jobs/drivers/sqs/item.go28
-rw-r--r--plugins/jobs/job/general.go1
-rw-r--r--plugins/jobs/job/job_options.go28
-rw-r--r--plugins/jobs/job/job_options_test.go42
-rw-r--r--plugins/jobs/plugin.go7
-rw-r--r--plugins/jobs/rpc.go8
-rw-r--r--proto/jobs/v1beta/jobs.pb.go64
-rw-r--r--proto/jobs/v1beta/jobs.proto6
-rw-r--r--tests/plugins/jobs/ephemeral/.rr-ephemeral-declare.yaml2
-rw-r--r--tests/plugins/jobs/ephemeral/.rr-ephemeral-init.yaml4
-rw-r--r--tests/plugins/jobs/ephemeral/.rr-ephemeral-jobs-err.yaml21
-rw-r--r--tests/plugins/jobs/ephemeral/.rr-ephemeral-pause-resume.yaml4
-rw-r--r--tests/plugins/jobs/helpers.go18
-rw-r--r--tests/plugins/jobs/jobs_ephemeral_test.go193
22 files changed, 231 insertions, 333 deletions
diff --git a/pkg/priority_queue/binary_heap_test.go b/pkg/priority_queue/binary_heap_test.go
index 05ddf5ef..fb5b83de 100644
--- a/pkg/priority_queue/binary_heap_test.go
+++ b/pkg/priority_queue/binary_heap_test.go
@@ -40,8 +40,6 @@ func (t Test) Priority() int64 {
return int64(t)
}
-func (t Test) Recycle() {}
-
func TestBinHeap_Init(t *testing.T) {
a := []Item{Test(2), Test(23), Test(33), Test(44), Test(1), Test(2), Test(2), Test(2), Test(4), Test(6), Test(99)}
diff --git a/pkg/priority_queue/interface.go b/pkg/priority_queue/interface.go
index 0034cbd3..9efa4652 100644
--- a/pkg/priority_queue/interface.go
+++ b/pkg/priority_queue/interface.go
@@ -28,7 +28,4 @@ type Item interface {
// Requeue - put the message back to the queue with the optional delay
Requeue(headers map[string][]string, delay int64) error
-
- // Recycle frees resources allocated by the Item
- Recycle()
}
diff --git a/plugins/jobs/drivers/amqp/item.go b/plugins/jobs/drivers/amqp/item.go
index 9b9625b0..5990d137 100644
--- a/plugins/jobs/drivers/amqp/item.go
+++ b/plugins/jobs/drivers/amqp/item.go
@@ -41,9 +41,6 @@ type Options struct {
// Delay defines time duration to delay execution for. Defaults to none.
Delay int64 `json:"delay,omitempty"`
- // Reserve defines for how broker should wait until treating job are failed. Defaults to 30 min.
- Timeout int64 `json:"timeout,omitempty"`
-
// private
// Ack delegates an acknowledgement through the Acknowledger interface that the client or server has finished work on a delivery
ack func(multiply bool) error
@@ -66,15 +63,6 @@ func (o *Options) DelayDuration() time.Duration {
return time.Second * time.Duration(o.Delay)
}
-// TimeoutDuration returns timeout duration in a form of time.Duration.
-func (o *Options) TimeoutDuration() time.Duration {
- if o.Timeout == 0 {
- return 30 * time.Minute
- }
-
- return time.Second * time.Duration(o.Timeout)
-}
-
func (i *Item) ID() string {
return i.Ident
}
@@ -96,9 +84,8 @@ func (i *Item) Context() ([]byte, error) {
ID string `json:"id"`
Job string `json:"job"`
Headers map[string][]string `json:"headers"`
- Timeout int64 `json:"timeout"`
Pipeline string `json:"pipeline"`
- }{ID: i.Ident, Job: i.Job, Headers: i.Headers, Timeout: i.Options.Timeout, Pipeline: i.Options.Pipeline},
+ }{ID: i.Ident, Job: i.Job, Headers: i.Headers, Pipeline: i.Options.Pipeline},
)
if err != nil {
@@ -141,10 +128,6 @@ func (i *Item) Requeue(headers map[string][]string, delay int64) error {
return nil
}
-func (i *Item) Recycle() {
- i.Options = nil
-}
-
// fromDelivery converts amqp.Delivery into an Item which will be pushed to the PQ
func (j *JobConsumer) fromDelivery(d amqp.Delivery) (*Item, error) {
const op = errors.Op("from_delivery_convert")
@@ -179,7 +162,6 @@ func fromJob(job *job.Job) *Item {
Priority: job.Options.Priority,
Pipeline: job.Options.Pipeline,
Delay: job.Options.Delay,
- Timeout: job.Options.Timeout,
},
}
}
@@ -195,7 +177,6 @@ func pack(id string, j *Item) (amqp.Table, error) {
job.RRJob: j.Job,
job.RRPipeline: j.Options.Pipeline,
job.RRHeaders: headers,
- job.RRTimeout: j.Options.Timeout,
job.RRDelay: j.Options.Delay,
job.RRPriority: j.Options.Priority,
}, nil
@@ -232,10 +213,6 @@ func (j *JobConsumer) unpack(d amqp.Delivery) (*Item, error) {
}
}
- if _, ok := d.Headers[job.RRTimeout].(int64); ok {
- item.Options.Timeout = d.Headers[job.RRTimeout].(int64)
- }
-
if _, ok := d.Headers[job.RRDelay].(int64); ok {
item.Options.Delay = d.Headers[job.RRDelay].(int64)
}
diff --git a/plugins/jobs/drivers/beanstalk/consumer.go b/plugins/jobs/drivers/beanstalk/consumer.go
index f41a2c8a..eaf99be1 100644
--- a/plugins/jobs/drivers/beanstalk/consumer.go
+++ b/plugins/jobs/drivers/beanstalk/consumer.go
@@ -201,7 +201,7 @@ func (j *JobConsumer) handleItem(ctx context.Context, item *Item) error {
// <ttr> seconds, the job will time out and the server will release the job.
// The minimum ttr is 1. If the client sends 0, the server will silently
// increase the ttr to 1. Maximum ttr is 2**32-1.
- id, err := j.pool.Put(ctx, bb.Bytes(), *j.tubePriority, item.Options.DelayDuration(), item.Options.TimeoutDuration())
+ id, err := j.pool.Put(ctx, bb.Bytes(), *j.tubePriority, item.Options.DelayDuration(), j.tout)
if err != nil {
errD := j.pool.Delete(ctx, id)
if errD != nil {
diff --git a/plugins/jobs/drivers/beanstalk/encode_test.go b/plugins/jobs/drivers/beanstalk/encode_test.go
index 34f2342b..e43207eb 100644
--- a/plugins/jobs/drivers/beanstalk/encode_test.go
+++ b/plugins/jobs/drivers/beanstalk/encode_test.go
@@ -26,7 +26,6 @@ func BenchmarkEncodeGob(b *testing.B) {
Priority: 10,
Pipeline: "test-local-pipe",
Delay: 10,
- Timeout: 5,
},
}
@@ -60,7 +59,6 @@ func BenchmarkEncodeJsonIter(b *testing.B) {
Priority: 10,
Pipeline: "test-local-pipe",
Delay: 10,
- Timeout: 5,
},
}
diff --git a/plugins/jobs/drivers/beanstalk/item.go b/plugins/jobs/drivers/beanstalk/item.go
index 47336b43..f1d7ac76 100644
--- a/plugins/jobs/drivers/beanstalk/item.go
+++ b/plugins/jobs/drivers/beanstalk/item.go
@@ -41,15 +41,6 @@ type Options struct {
// Delay defines time duration to delay execution for. Defaults to none.
Delay int64 `json:"delay,omitempty"`
- // Reserve defines for how broker should wait until treating job are failed.
- // - <ttr> -- time to run -- is an integer number of seconds to allow a worker
- // to run this job. This time is counted from the moment a worker reserves
- // this job. If the worker does not delete, release, or bury the job within
- // <ttr> seconds, the job will time out and the server will release the job.
- // The minimum ttr is 1. If the client sends 0, the server will silently
- // increase the ttr to 1. Maximum ttr is 2**32-1.
- Timeout int64 `json:"timeout,omitempty"`
-
// Private ================
id uint64
conn *beanstalk.Conn
@@ -61,15 +52,6 @@ func (o *Options) DelayDuration() time.Duration {
return time.Second * time.Duration(o.Delay)
}
-// TimeoutDuration returns timeout duration in a form of time.Duration.
-func (o *Options) TimeoutDuration() time.Duration {
- if o.Timeout == 0 {
- return 30 * time.Minute
- }
-
- return time.Second * time.Duration(o.Timeout)
-}
-
func (i *Item) ID() string {
return i.Ident
}
@@ -91,9 +73,8 @@ func (i *Item) Context() ([]byte, error) {
ID string `json:"id"`
Job string `json:"job"`
Headers map[string][]string `json:"headers"`
- Timeout int64 `json:"timeout"`
Pipeline string `json:"pipeline"`
- }{ID: i.Ident, Job: i.Job, Headers: i.Headers, Timeout: i.Options.Timeout, Pipeline: i.Options.Pipeline},
+ }{ID: i.Ident, Job: i.Job, Headers: i.Headers, Pipeline: i.Options.Pipeline},
)
if err != nil {
@@ -130,10 +111,6 @@ func (i *Item) Requeue(headers map[string][]string, delay int64) error {
return nil
}
-func (i *Item) Recycle() {
- i.Options = nil
-}
-
func fromJob(job *job.Job) *Item {
return &Item{
Job: job.Job,
@@ -144,7 +121,6 @@ func fromJob(job *job.Job) *Item {
Priority: job.Options.Priority,
Pipeline: job.Options.Pipeline,
Delay: job.Options.Delay,
- Timeout: job.Options.Timeout,
},
}
}
diff --git a/plugins/jobs/drivers/ephemeral/consumer.go b/plugins/jobs/drivers/ephemeral/consumer.go
index 03959b49..95ad6ecd 100644
--- a/plugins/jobs/drivers/ephemeral/consumer.go
+++ b/plugins/jobs/drivers/ephemeral/consumer.go
@@ -16,7 +16,8 @@ import (
)
const (
- prefetch string = "prefetch"
+ prefetch string = "prefetch"
+ goroutinesMax uint64 = 1000
)
type Config struct {
@@ -32,7 +33,7 @@ type JobConsumer struct {
localPrefetch chan *Item
// time.sleep goroutines max number
- goroutinesMaxNum uint64
+ goroutines uint64
stopCh chan struct{}
}
@@ -41,11 +42,11 @@ func NewJobBroker(configKey string, log logger.Logger, cfg config.Configurer, eh
const op = errors.Op("new_ephemeral_pipeline")
jb := &JobConsumer{
- log: log,
- pq: pq,
- eh: eh,
- goroutinesMaxNum: 1000,
- stopCh: make(chan struct{}, 1),
+ log: log,
+ pq: pq,
+ eh: eh,
+ goroutines: 0,
+ stopCh: make(chan struct{}, 1),
}
err := cfg.UnmarshalKey(configKey, &jb.cfg)
@@ -68,11 +69,11 @@ func NewJobBroker(configKey string, log logger.Logger, cfg config.Configurer, eh
func FromPipeline(pipeline *pipeline.Pipeline, log logger.Logger, eh events.Handler, pq priorityqueue.Queue) (*JobConsumer, error) {
jb := &JobConsumer{
- log: log,
- pq: pq,
- eh: eh,
- goroutinesMaxNum: 1000,
- stopCh: make(chan struct{}, 1),
+ log: log,
+ pq: pq,
+ eh: eh,
+ goroutines: 0,
+ stopCh: make(chan struct{}, 1),
}
// initialize a local queue
@@ -112,18 +113,18 @@ func (j *JobConsumer) handleItem(ctx context.Context, msg *Item) error {
// 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(&j.goroutinesMaxNum) >= 1000 {
+ if atomic.LoadUint64(&j.goroutines) >= goroutinesMax {
return errors.E(op, errors.Str("max concurrency number reached"))
}
go func(jj *Item) {
- atomic.AddUint64(&j.goroutinesMaxNum, 1)
+ atomic.AddUint64(&j.goroutines, 1)
time.Sleep(jj.Options.DelayDuration())
// send the item after timeout expired
j.localPrefetch <- jj
- atomic.AddUint64(&j.goroutinesMaxNum, ^uint64(0))
+ atomic.AddUint64(&j.goroutines, ^uint64(0))
}(msg)
return nil
@@ -149,7 +150,8 @@ func (j *JobConsumer) consume() {
}
// set requeue channel
- item.Options.requeueCh = j.localPrefetch
+ item.Options.requeueFn = j.handleItem
+
j.pq.Insert(item)
case <-j.stopCh:
return
diff --git a/plugins/jobs/drivers/ephemeral/item.go b/plugins/jobs/drivers/ephemeral/item.go
index 9fab8d24..1a61d7e9 100644
--- a/plugins/jobs/drivers/ephemeral/item.go
+++ b/plugins/jobs/drivers/ephemeral/item.go
@@ -1,6 +1,7 @@
package ephemeral
import (
+ "context"
"time"
json "github.com/json-iterator/go"
@@ -37,11 +38,8 @@ type Options struct {
// Delay defines time duration to delay execution for. Defaults to none.
Delay int64 `json:"delay,omitempty"`
- // Timeout defines for how broker should wait until treating job are failed. Defaults to 30 min.
- Timeout int64 `json:"timeout,omitempty"`
-
// private
- requeueCh chan *Item
+ requeueFn func(context.Context, *Item) error
}
// DelayDuration returns delay duration in a form of time.Duration.
@@ -49,15 +47,6 @@ func (o *Options) DelayDuration() time.Duration {
return time.Second * time.Duration(o.Delay)
}
-// TimeoutDuration returns timeout duration in a form of time.Duration.
-func (o *Options) TimeoutDuration() time.Duration {
- if o.Timeout == 0 {
- return 30 * time.Minute
- }
-
- return time.Second * time.Duration(o.Timeout)
-}
-
func (i *Item) ID() string {
return i.Ident
}
@@ -78,9 +67,8 @@ func (i *Item) Context() ([]byte, error) {
ID string `json:"id"`
Job string `json:"job"`
Headers map[string][]string `json:"headers"`
- Timeout int64 `json:"timeout"`
Pipeline string `json:"pipeline"`
- }{ID: i.Ident, Job: i.Job, Headers: i.Headers, Timeout: i.Options.Timeout, Pipeline: i.Options.Pipeline},
+ }{ID: i.Ident, Job: i.Job, Headers: i.Headers, Pipeline: i.Options.Pipeline},
)
if err != nil {
@@ -101,25 +89,16 @@ func (i *Item) Nack() error {
}
func (i *Item) Requeue(headers map[string][]string, delay int64) error {
- go func() {
- time.Sleep(time.Second * time.Duration(delay))
- // overwrite the delay
- i.Options.Delay = delay
- i.Headers = headers
- select {
- case i.Options.requeueCh <- i:
- return
- default:
- // TODO(rustatian): logs?
- return
- }
- }()
+ // overwrite the delay
+ i.Options.Delay = delay
+ i.Headers = headers
- return nil
-}
+ err := i.Options.requeueFn(context.Background(), i)
+ if err != nil {
+ return err
+ }
-func (i *Item) Recycle() {
- i.Options = nil
+ return nil
}
func fromJob(job *job.Job) *Item {
@@ -131,7 +110,6 @@ func fromJob(job *job.Job) *Item {
Priority: job.Options.Priority,
Pipeline: job.Options.Pipeline,
Delay: job.Options.Delay,
- Timeout: job.Options.Timeout,
},
}
}
diff --git a/plugins/jobs/drivers/sqs/item.go b/plugins/jobs/drivers/sqs/item.go
index eac06731..df72b2e5 100644
--- a/plugins/jobs/drivers/sqs/item.go
+++ b/plugins/jobs/drivers/sqs/item.go
@@ -24,7 +24,6 @@ const (
var itemAttributes = []string{
job.RRJob,
job.RRDelay,
- job.RRTimeout,
job.RRPriority,
job.RRHeaders,
}
@@ -58,9 +57,6 @@ type Options struct {
// Delay defines time duration to delay execution for. Defaults to none.
Delay int64 `json:"delay,omitempty"`
- // Reserve defines for how broker should wait until treating job are failed. Defaults to 30 min.
- Timeout int64 `json:"timeout,omitempty"`
-
// Private ================
approxReceiveCount int64
queue *string
@@ -74,15 +70,6 @@ func (o *Options) DelayDuration() time.Duration {
return time.Second * time.Duration(o.Delay)
}
-// TimeoutDuration returns timeout duration in a form of time.Duration.
-func (o *Options) TimeoutDuration() time.Duration {
- if o.Timeout == 0 {
- return 30 * time.Minute
- }
-
- return time.Second * time.Duration(o.Timeout)
-}
-
func (i *Item) ID() string {
return i.Ident
}
@@ -104,9 +91,8 @@ func (i *Item) Context() ([]byte, error) {
ID string `json:"id"`
Job string `json:"job"`
Headers map[string][]string `json:"headers"`
- Timeout int64 `json:"timeout"`
Pipeline string `json:"pipeline"`
- }{ID: i.Ident, Job: i.Job, Headers: i.Headers, Timeout: i.Options.Timeout, Pipeline: i.Options.Pipeline},
+ }{ID: i.Ident, Job: i.Job, Headers: i.Headers, Pipeline: i.Options.Pipeline},
)
if err != nil {
@@ -172,10 +158,6 @@ func (i *Item) Requeue(headers map[string][]string, delay int64) error {
return nil
}
-func (i *Item) Recycle() {
- i.Options = nil
-}
-
func fromJob(job *job.Job) *Item {
return &Item{
Job: job.Job,
@@ -186,7 +168,6 @@ func fromJob(job *job.Job) *Item {
Priority: job.Options.Priority,
Pipeline: job.Options.Pipeline,
Delay: job.Options.Delay,
- Timeout: job.Options.Timeout,
},
}
}
@@ -205,7 +186,6 @@ func (i *Item) pack(queue *string) (*sqs.SendMessageInput, error) {
MessageAttributes: map[string]types.MessageAttributeValue{
job.RRJob: {DataType: aws.String(StringType), BinaryValue: nil, BinaryListValues: nil, StringListValues: nil, StringValue: aws.String(i.Job)},
job.RRDelay: {DataType: aws.String(StringType), BinaryValue: nil, BinaryListValues: nil, StringListValues: nil, StringValue: aws.String(strconv.Itoa(int(i.Options.Delay)))},
- job.RRTimeout: {DataType: aws.String(StringType), BinaryValue: nil, BinaryListValues: nil, StringListValues: nil, StringValue: aws.String(strconv.Itoa(int(i.Options.Timeout)))},
job.RRHeaders: {DataType: aws.String(BinaryType), BinaryValue: data, BinaryListValues: nil, StringListValues: nil, StringValue: nil},
job.RRPriority: {DataType: aws.String(NumberType), BinaryValue: nil, BinaryListValues: nil, StringListValues: nil, StringValue: aws.String(strconv.Itoa(int(i.Options.Priority)))},
},
@@ -236,11 +216,6 @@ func (j *JobConsumer) unpack(msg *types.Message) (*Item, error) {
return nil, errors.E(op, err)
}
- to, err := strconv.Atoi(*msg.MessageAttributes[job.RRTimeout].StringValue)
- if err != nil {
- return nil, errors.E(op, err)
- }
-
priority, err := strconv.Atoi(*msg.MessageAttributes[job.RRPriority].StringValue)
if err != nil {
return nil, errors.E(op, err)
@@ -257,7 +232,6 @@ func (j *JobConsumer) unpack(msg *types.Message) (*Item, error) {
Headers: h,
Options: &Options{
Delay: int64(delay),
- Timeout: int64(to),
Priority: int64(priority),
// private
diff --git a/plugins/jobs/job/general.go b/plugins/jobs/job/general.go
index 0a75f2e6..390f44b5 100644
--- a/plugins/jobs/job/general.go
+++ b/plugins/jobs/job/general.go
@@ -6,7 +6,6 @@ const (
RRJob string = "rr_job"
RRHeaders string = "rr_headers"
RRPipeline string = "rr_pipeline"
- RRTimeout string = "rr_timeout"
RRDelay string = "rr_delay"
RRPriority string = "rr_priority"
)
diff --git a/plugins/jobs/job/job_options.go b/plugins/jobs/job/job_options.go
index 133ae1a8..b7e4ed36 100644
--- a/plugins/jobs/job/job_options.go
+++ b/plugins/jobs/job/job_options.go
@@ -13,12 +13,6 @@ type Options struct {
// Delay defines time duration to delay execution for. Defaults to none.
Delay int64 `json:"delay,omitempty"`
-
- // RetryDelay defines for how long job should be waiting until next retry. Defaults to none.
- RetryDelay int64 `json:"retryDelay,omitempty"`
-
- // Reserve defines for how broker should wait until treating job are failed. Defaults to 30 min.
- Timeout int64 `json:"timeout,omitempty"`
}
// Merge merges job options.
@@ -27,34 +21,12 @@ func (o *Options) Merge(from *Options) {
o.Pipeline = from.Pipeline
}
- if o.Timeout == 0 {
- o.Timeout = from.Timeout
- }
-
- if o.RetryDelay == 0 {
- o.RetryDelay = from.RetryDelay
- }
-
if o.Delay == 0 {
o.Delay = from.Delay
}
}
-// RetryDuration returns retry delay duration in a form of time.Duration.
-func (o *Options) RetryDuration() time.Duration {
- return time.Second * time.Duration(o.RetryDelay)
-}
-
// DelayDuration returns delay duration in a form of time.Duration.
func (o *Options) DelayDuration() time.Duration {
return time.Second * time.Duration(o.Delay)
}
-
-// TimeoutDuration returns timeout duration in a form of time.Duration.
-func (o *Options) TimeoutDuration() time.Duration {
- if o.Timeout == 0 {
- return 30 * time.Minute
- }
-
- return time.Second * time.Duration(o.Timeout)
-}
diff --git a/plugins/jobs/job/job_options_test.go b/plugins/jobs/job/job_options_test.go
index 061e83cc..a47151a3 100644
--- a/plugins/jobs/job/job_options_test.go
+++ b/plugins/jobs/job/job_options_test.go
@@ -7,16 +7,6 @@ import (
"github.com/stretchr/testify/assert"
)
-func TestOptions_RetryDuration(t *testing.T) {
- opts := &Options{RetryDelay: 0}
- assert.Equal(t, time.Duration(0), opts.RetryDuration())
-}
-
-func TestOptions_RetryDuration2(t *testing.T) {
- opts := &Options{RetryDelay: 1}
- assert.Equal(t, time.Second, opts.RetryDuration())
-}
-
func TestOptions_DelayDuration(t *testing.T) {
opts := &Options{Delay: 0}
assert.Equal(t, time.Duration(0), opts.DelayDuration())
@@ -27,49 +17,29 @@ func TestOptions_DelayDuration2(t *testing.T) {
assert.Equal(t, time.Second, opts.DelayDuration())
}
-func TestOptions_TimeoutDuration(t *testing.T) {
- opts := &Options{Timeout: 0}
- assert.Equal(t, time.Minute*30, opts.TimeoutDuration())
-}
-
-func TestOptions_TimeoutDuration2(t *testing.T) {
- opts := &Options{Timeout: 1}
- assert.Equal(t, time.Second, opts.TimeoutDuration())
-}
-
func TestOptions_Merge(t *testing.T) {
opts := &Options{}
opts.Merge(&Options{
- Pipeline: "pipeline",
- Delay: 2,
- Timeout: 1,
- RetryDelay: 1,
+ Pipeline: "pipeline",
+ Delay: 2,
})
assert.Equal(t, "pipeline", opts.Pipeline)
assert.Equal(t, int64(2), opts.Delay)
- assert.Equal(t, int64(1), opts.Timeout)
- assert.Equal(t, int64(1), opts.RetryDelay)
}
func TestOptions_MergeKeepOriginal(t *testing.T) {
opts := &Options{
- Pipeline: "default",
- Delay: 10,
- Timeout: 10,
- RetryDelay: 10,
+ Pipeline: "default",
+ Delay: 10,
}
opts.Merge(&Options{
- Pipeline: "pipeline",
- Delay: 2,
- Timeout: 1,
- RetryDelay: 1,
+ Pipeline: "pipeline",
+ Delay: 2,
})
assert.Equal(t, "default", opts.Pipeline)
assert.Equal(t, int64(10), opts.Delay)
- assert.Equal(t, int64(10), opts.Timeout)
- assert.Equal(t, int64(10), opts.RetryDelay)
}
diff --git a/plugins/jobs/plugin.go b/plugins/jobs/plugin.go
index e2fffda7..26015516 100644
--- a/plugins/jobs/plugin.go
+++ b/plugins/jobs/plugin.go
@@ -246,13 +246,14 @@ func (p *Plugin) Serve() chan error { //nolint:gocognit
p.putPayload(exec)
errNack := jb.Nack()
if errNack != nil {
- p.log.Error("negatively acknowledge failed, job might be lost", "error", errNack)
+ p.log.Error("negatively acknowledge failed, job might be lost", "root error", err, "error nack", errNack)
+ continue
}
+
+ p.log.Error("job negatively acknowledged", "error", err)
continue
}
- // free the resources
- jb.Recycle()
// return payload
p.putPayload(exec)
}
diff --git a/plugins/jobs/rpc.go b/plugins/jobs/rpc.go
index af1e12c0..7f9859fb 100644
--- a/plugins/jobs/rpc.go
+++ b/plugins/jobs/rpc.go
@@ -126,11 +126,9 @@ func (r *rpc) from(j *jobsv1beta.Job) *job.Job {
Ident: j.GetId(),
Payload: j.GetPayload(),
Options: &job.Options{
- Priority: j.GetOptions().GetPriority(),
- Pipeline: j.GetOptions().GetPipeline(),
- Delay: j.GetOptions().GetDelay(),
- RetryDelay: j.GetOptions().GetRetryDelay(),
- Timeout: j.GetOptions().GetTimeout(),
+ Priority: j.GetOptions().GetPriority(),
+ Pipeline: j.GetOptions().GetPipeline(),
+ Delay: j.GetOptions().GetDelay(),
},
}
diff --git a/proto/jobs/v1beta/jobs.pb.go b/proto/jobs/v1beta/jobs.pb.go
index 9201f659..6a6f59af 100644
--- a/proto/jobs/v1beta/jobs.pb.go
+++ b/proto/jobs/v1beta/jobs.pb.go
@@ -259,8 +259,8 @@ type Job struct {
Job string `protobuf:"bytes,1,opt,name=job,proto3" json:"job,omitempty"`
Id string `protobuf:"bytes,2,opt,name=id,proto3" json:"id,omitempty"`
Payload string `protobuf:"bytes,3,opt,name=payload,proto3" json:"payload,omitempty"`
- Headers map[string]*HeaderValue `protobuf:"bytes,5,rep,name=headers,proto3" json:"headers,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
- Options *Options `protobuf:"bytes,4,opt,name=options,proto3" json:"options,omitempty"`
+ Headers map[string]*HeaderValue `protobuf:"bytes,4,rep,name=headers,proto3" json:"headers,omitempty" protobuf_key:"bytes,1,opt,name=key,proto3" protobuf_val:"bytes,2,opt,name=value,proto3"`
+ Options *Options `protobuf:"bytes,5,opt,name=options,proto3" json:"options,omitempty"`
}
func (x *Job) Reset() {
@@ -335,12 +335,9 @@ type Options struct {
sizeCache protoimpl.SizeCache
unknownFields protoimpl.UnknownFields
- Priority int64 `protobuf:"varint,1,opt,name=priority,proto3" json:"priority,omitempty"`
- Pipeline string `protobuf:"bytes,2,opt,name=pipeline,proto3" json:"pipeline,omitempty"`
- Delay int64 `protobuf:"varint,3,opt,name=delay,proto3" json:"delay,omitempty"`
- Attempts int64 `protobuf:"varint,4,opt,name=attempts,proto3" json:"attempts,omitempty"`
- RetryDelay int64 `protobuf:"varint,5,opt,name=retry_delay,json=retryDelay,proto3" json:"retry_delay,omitempty"`
- Timeout int64 `protobuf:"varint,6,opt,name=timeout,proto3" json:"timeout,omitempty"`
+ Priority int64 `protobuf:"varint,1,opt,name=priority,proto3" json:"priority,omitempty"`
+ Pipeline string `protobuf:"bytes,2,opt,name=pipeline,proto3" json:"pipeline,omitempty"`
+ Delay int64 `protobuf:"varint,3,opt,name=delay,proto3" json:"delay,omitempty"`
}
func (x *Options) Reset() {
@@ -396,27 +393,6 @@ func (x *Options) GetDelay() int64 {
return 0
}
-func (x *Options) GetAttempts() int64 {
- if x != nil {
- return x.Attempts
- }
- return 0
-}
-
-func (x *Options) GetRetryDelay() int64 {
- if x != nil {
- return x.RetryDelay
- }
- return 0
-}
-
-func (x *Options) GetTimeout() int64 {
- if x != nil {
- return x.Timeout
- }
- return 0
-}
-
type HeaderValue struct {
state protoimpl.MessageState
sizeCache protoimpl.SizeCache
@@ -493,10 +469,10 @@ var file_jobs_proto_rawDesc = []byte{
0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x02, 0x69, 0x64, 0x12, 0x18, 0x0a, 0x07, 0x70,
0x61, 0x79, 0x6c, 0x6f, 0x61, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x70, 0x61,
0x79, 0x6c, 0x6f, 0x61, 0x64, 0x12, 0x37, 0x0a, 0x07, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73,
- 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6a, 0x6f, 0x62, 0x73, 0x2e, 0x76, 0x31,
+ 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1d, 0x2e, 0x6a, 0x6f, 0x62, 0x73, 0x2e, 0x76, 0x31,
0x62, 0x65, 0x74, 0x61, 0x2e, 0x4a, 0x6f, 0x62, 0x2e, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73,
0x45, 0x6e, 0x74, 0x72, 0x79, 0x52, 0x07, 0x68, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x12, 0x2e,
- 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0b, 0x32,
+ 0x0a, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0b, 0x32,
0x14, 0x2e, 0x6a, 0x6f, 0x62, 0x73, 0x2e, 0x76, 0x31, 0x62, 0x65, 0x74, 0x61, 0x2e, 0x4f, 0x70,
0x74, 0x69, 0x6f, 0x6e, 0x73, 0x52, 0x07, 0x6f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x1a, 0x54,
0x0a, 0x0c, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x73, 0x45, 0x6e, 0x74, 0x72, 0x79, 0x12, 0x10,
@@ -504,22 +480,16 @@ var file_jobs_proto_rawDesc = []byte{
0x12, 0x2e, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32,
0x18, 0x2e, 0x6a, 0x6f, 0x62, 0x73, 0x2e, 0x76, 0x31, 0x62, 0x65, 0x74, 0x61, 0x2e, 0x48, 0x65,
0x61, 0x64, 0x65, 0x72, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65,
- 0x3a, 0x02, 0x38, 0x01, 0x22, 0xae, 0x01, 0x0a, 0x07, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73,
- 0x12, 0x1a, 0x0a, 0x08, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x18, 0x01, 0x20, 0x01,
- 0x28, 0x03, 0x52, 0x08, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x12, 0x1a, 0x0a, 0x08,
- 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08,
- 0x70, 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x64, 0x65, 0x6c, 0x61,
- 0x79, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x64, 0x65, 0x6c, 0x61, 0x79, 0x12, 0x1a,
- 0x0a, 0x08, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x03,
- 0x52, 0x08, 0x61, 0x74, 0x74, 0x65, 0x6d, 0x70, 0x74, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x72, 0x65,
- 0x74, 0x72, 0x79, 0x5f, 0x64, 0x65, 0x6c, 0x61, 0x79, 0x18, 0x05, 0x20, 0x01, 0x28, 0x03, 0x52,
- 0x0a, 0x72, 0x65, 0x74, 0x72, 0x79, 0x44, 0x65, 0x6c, 0x61, 0x79, 0x12, 0x18, 0x0a, 0x07, 0x74,
- 0x69, 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x18, 0x06, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x74, 0x69,
- 0x6d, 0x65, 0x6f, 0x75, 0x74, 0x22, 0x23, 0x0a, 0x0b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x56,
- 0x61, 0x6c, 0x75, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20,
- 0x03, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x42, 0x0f, 0x5a, 0x0d, 0x2e, 0x2f,
- 0x3b, 0x6a, 0x6f, 0x62, 0x73, 0x76, 0x31, 0x62, 0x65, 0x74, 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f,
- 0x74, 0x6f, 0x33,
+ 0x3a, 0x02, 0x38, 0x01, 0x22, 0x57, 0x0a, 0x07, 0x4f, 0x70, 0x74, 0x69, 0x6f, 0x6e, 0x73, 0x12,
+ 0x1a, 0x0a, 0x08, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x18, 0x01, 0x20, 0x01, 0x28,
+ 0x03, 0x52, 0x08, 0x70, 0x72, 0x69, 0x6f, 0x72, 0x69, 0x74, 0x79, 0x12, 0x1a, 0x0a, 0x08, 0x70,
+ 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x70,
+ 0x69, 0x70, 0x65, 0x6c, 0x69, 0x6e, 0x65, 0x12, 0x14, 0x0a, 0x05, 0x64, 0x65, 0x6c, 0x61, 0x79,
+ 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x05, 0x64, 0x65, 0x6c, 0x61, 0x79, 0x22, 0x23, 0x0a,
+ 0x0b, 0x48, 0x65, 0x61, 0x64, 0x65, 0x72, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x14, 0x0a, 0x05,
+ 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x01, 0x20, 0x03, 0x28, 0x09, 0x52, 0x05, 0x76, 0x61, 0x6c,
+ 0x75, 0x65, 0x42, 0x0f, 0x5a, 0x0d, 0x2e, 0x2f, 0x3b, 0x6a, 0x6f, 0x62, 0x73, 0x76, 0x31, 0x62,
+ 0x65, 0x74, 0x61, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33,
}
var (
diff --git a/proto/jobs/v1beta/jobs.proto b/proto/jobs/v1beta/jobs.proto
index 77d1fb51..68d2ed97 100644
--- a/proto/jobs/v1beta/jobs.proto
+++ b/proto/jobs/v1beta/jobs.proto
@@ -30,16 +30,14 @@ message Job {
string job = 1;
string id = 2;
string payload = 3;
- map<string, HeaderValue> headers = 5;
- Options options = 4;
+ map<string, HeaderValue> headers = 4;
+ Options options = 5;
}
message Options {
int64 priority = 1;
string pipeline = 2;
int64 delay = 3;
- int64 retry_delay = 5;
- int64 timeout = 6;
}
message HeaderValue {
diff --git a/tests/plugins/jobs/ephemeral/.rr-ephemeral-declare.yaml b/tests/plugins/jobs/ephemeral/.rr-ephemeral-declare.yaml
index ea5b61d2..726c24ac 100644
--- a/tests/plugins/jobs/ephemeral/.rr-ephemeral-declare.yaml
+++ b/tests/plugins/jobs/ephemeral/.rr-ephemeral-declare.yaml
@@ -2,7 +2,7 @@ rpc:
listen: tcp://127.0.0.1:6001
server:
- command: "php ../../client.php echo pipes"
+ command: "php ../../jobs_ok.php"
relay: "pipes"
relay_timeout: "20s"
diff --git a/tests/plugins/jobs/ephemeral/.rr-ephemeral-init.yaml b/tests/plugins/jobs/ephemeral/.rr-ephemeral-init.yaml
index ee628387..8914dfaa 100644
--- a/tests/plugins/jobs/ephemeral/.rr-ephemeral-init.yaml
+++ b/tests/plugins/jobs/ephemeral/.rr-ephemeral-init.yaml
@@ -2,12 +2,12 @@ rpc:
listen: tcp://127.0.0.1:6001
server:
- command: "php ../../client.php echo pipes"
+ command: "php ../../jobs_ok.php"
relay: "pipes"
relay_timeout: "20s"
logs:
- level: info
+ level: debug
encoding: console
mode: development
diff --git a/tests/plugins/jobs/ephemeral/.rr-ephemeral-jobs-err.yaml b/tests/plugins/jobs/ephemeral/.rr-ephemeral-jobs-err.yaml
new file mode 100644
index 00000000..05dc3ffa
--- /dev/null
+++ b/tests/plugins/jobs/ephemeral/.rr-ephemeral-jobs-err.yaml
@@ -0,0 +1,21 @@
+rpc:
+ listen: tcp://127.0.0.1:6001
+
+server:
+ command: "php ../../jobs_err.php"
+ relay: "pipes"
+ relay_timeout: "20s"
+
+logs:
+ level: debug
+ encoding: console
+ mode: development
+
+jobs:
+ num_pollers: 10
+ pipeline_size: 100000
+ pool:
+ num_workers: 10
+ max_jobs: 0
+ allocate_timeout: 60s
+ destroy_timeout: 60s
diff --git a/tests/plugins/jobs/ephemeral/.rr-ephemeral-pause-resume.yaml b/tests/plugins/jobs/ephemeral/.rr-ephemeral-pause-resume.yaml
index dc5bc3a1..e1b76263 100644
--- a/tests/plugins/jobs/ephemeral/.rr-ephemeral-pause-resume.yaml
+++ b/tests/plugins/jobs/ephemeral/.rr-ephemeral-pause-resume.yaml
@@ -2,12 +2,12 @@ rpc:
listen: tcp://127.0.0.1:6001
server:
- command: "php ../../client.php echo pipes"
+ command: "php ../../jobs_ok.php"
relay: "pipes"
relay_timeout: "20s"
logs:
- level: info
+ level: debug
mode: development
jobs:
diff --git a/tests/plugins/jobs/helpers.go b/tests/plugins/jobs/helpers.go
index 831c732c..4c2f2fea 100644
--- a/tests/plugins/jobs/helpers.go
+++ b/tests/plugins/jobs/helpers.go
@@ -73,12 +73,9 @@ func pushToPipe(pipeline string) func(t *testing.T) {
Payload: `{"hello":"world"}`,
Headers: map[string]*jobsv1beta.HeaderValue{"test": {Value: []string{"test2"}}},
Options: &jobsv1beta.Options{
- Priority: 1,
- Pipeline: pipeline,
- Delay: 0,
- Attempts: 0,
- RetryDelay: 0,
- Timeout: 0,
+ Priority: 1,
+ Pipeline: pipeline,
+ Delay: 0,
},
}}
@@ -100,12 +97,9 @@ func pushToPipeErr(pipeline string) func(t *testing.T) {
Payload: `{"hello":"world"}`,
Headers: map[string]*jobsv1beta.HeaderValue{"test": {Value: []string{"test2"}}},
Options: &jobsv1beta.Options{
- Priority: 1,
- Pipeline: pipeline,
- Delay: 0,
- Attempts: 0,
- RetryDelay: 0,
- Timeout: 0,
+ Priority: 1,
+ Pipeline: pipeline,
+ Delay: 0,
},
}}
diff --git a/tests/plugins/jobs/jobs_ephemeral_test.go b/tests/plugins/jobs/jobs_ephemeral_test.go
index 04d95506..0a882556 100644
--- a/tests/plugins/jobs/jobs_ephemeral_test.go
+++ b/tests/plugins/jobs/jobs_ephemeral_test.go
@@ -17,7 +17,6 @@ import (
"github.com/spiral/roadrunner/v2/plugins/informer"
"github.com/spiral/roadrunner/v2/plugins/jobs"
"github.com/spiral/roadrunner/v2/plugins/jobs/drivers/ephemeral"
- "github.com/spiral/roadrunner/v2/plugins/logger"
"github.com/spiral/roadrunner/v2/plugins/resetter"
rpcPlugin "github.com/spiral/roadrunner/v2/plugins/rpc"
"github.com/spiral/roadrunner/v2/plugins/server"
@@ -44,22 +43,17 @@ func TestEphemeralInit(t *testing.T) {
mockLogger.EXPECT().Debug("Started RPC service", "address", "tcp://127.0.0.1:6001", "services", gomock.Any()).Times(1)
mockLogger.EXPECT().Error(gomock.Any(), gomock.Any(), gomock.Any()).AnyTimes()
- mockLogger.EXPECT().Info("pipeline started", "pipeline", "test-1", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
- mockLogger.EXPECT().Info("pipeline started", "pipeline", "test-2", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
-
- mockLogger.EXPECT().Info("driver initialized", "driver", "amqp", "start", gomock.Any()).Times(2)
+ mockLogger.EXPECT().Info("pipeline active", "pipeline", "test-1", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
+ mockLogger.EXPECT().Info("pipeline active", "pipeline", "test-2", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
mockLogger.EXPECT().Warn("pipeline stopped", "pipeline", "test-1", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
mockLogger.EXPECT().Warn("pipeline stopped", "pipeline", "test-2", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
- mockLogger.EXPECT().Info("delivery channel closed, leaving the rabbit listener").Times(2)
-
err = cont.RegisterAll(
cfg,
&server.Plugin{},
&rpcPlugin.Plugin{},
- &logger.ZapLogger{},
- // mockLogger,
+ mockLogger,
&jobs.Plugin{},
&resetter.Plugin{},
&informer.Plugin{},
@@ -135,22 +129,13 @@ func TestEphemeralDeclare(t *testing.T) {
mockLogger.EXPECT().Debug("Started RPC service", "address", "tcp://127.0.0.1:6001", "services", gomock.Any()).Times(1)
mockLogger.EXPECT().Error(gomock.Any(), gomock.Any(), gomock.Any()).AnyTimes()
- mockLogger.EXPECT().Info("pipeline started", "pipeline", "test-1", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
- mockLogger.EXPECT().Info("pipeline started", "pipeline", "test-2", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
-
- mockLogger.EXPECT().Info("driver initialized", "driver", "amqp", "start", gomock.Any()).Times(2)
-
- mockLogger.EXPECT().Warn("pipeline stopped", "pipeline", "test-1", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
- mockLogger.EXPECT().Warn("pipeline stopped", "pipeline", "test-2", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
-
- mockLogger.EXPECT().Info("delivery channel closed, leaving the rabbit listener").Times(2)
+ mockLogger.EXPECT().Warn("pipeline stopped", "pipeline", "test-3", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
err = cont.RegisterAll(
cfg,
&server.Plugin{},
&rpcPlugin.Plugin{},
- &logger.ZapLogger{},
- // mockLogger,
+ mockLogger,
&jobs.Plugin{},
&resetter.Plugin{},
&informer.Plugin{},
@@ -208,41 +193,16 @@ func TestEphemeralDeclare(t *testing.T) {
t.Run("DeclareEphemeralPipeline", declareEphemeralPipe)
t.Run("ConsumeEphemeralPipeline", consumeEphemeralPipe)
t.Run("PushEphemeralPipeline", pushToPipe("test-3"))
+ time.Sleep(time.Second)
t.Run("PauseEphemeralPipeline", pausePipelines("test-3"))
+ time.Sleep(time.Second)
t.Run("DestroyEphemeralPipeline", destroyPipelines("test-3"))
time.Sleep(time.Second * 5)
stopCh <- struct{}{}
wg.Wait()
-}
-
-func declareEphemeralPipe(t *testing.T) {
- conn, err := net.Dial("tcp", "127.0.0.1:6001")
- assert.NoError(t, err)
- client := rpc.NewClientWithCodec(goridgeRpc.NewClientCodec(conn))
-
- pipe := &jobsv1beta.DeclareRequest{Pipeline: map[string]string{
- "driver": "ephemeral",
- "name": "test-3",
- "prefetch": "10000",
- }}
- er := &jobsv1beta.Empty{}
- err = client.Call("jobs.Declare", pipe, er)
- assert.NoError(t, err)
-}
-
-func consumeEphemeralPipe(t *testing.T) {
- conn, err := net.Dial("tcp", "127.0.0.1:6001")
- assert.NoError(t, err)
- client := rpc.NewClientWithCodec(goridgeRpc.NewClientCodec(conn))
-
- pipe := &jobsv1beta.Pipelines{Pipelines: make([]string, 1)}
- pipe.GetPipelines()[0] = "test-3"
-
- er := &jobsv1beta.Empty{}
- err = client.Call("jobs.Resume", pipe, er)
- assert.NoError(t, err)
+ time.Sleep(time.Second * 5)
}
func TestEphemeralPauseResume(t *testing.T) {
@@ -263,25 +223,18 @@ func TestEphemeralPauseResume(t *testing.T) {
mockLogger.EXPECT().Debug("Started RPC service", "address", "tcp://127.0.0.1:6001", "services", gomock.Any()).Times(1)
mockLogger.EXPECT().Error(gomock.Any(), gomock.Any(), gomock.Any()).AnyTimes()
- mockLogger.EXPECT().Info("pipeline started", "pipeline", "test-local-2", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
- mockLogger.EXPECT().Info("pipeline started", "pipeline", "test-local", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
+ mockLogger.EXPECT().Info("pipeline active", "pipeline", "test-local-2", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
+ mockLogger.EXPECT().Info("pipeline active", "pipeline", "test-local", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
mockLogger.EXPECT().Warn("pipeline stopped", "pipeline", "test-local-3", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
mockLogger.EXPECT().Warn("pipeline stopped", "pipeline", "test-local-2", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
- mockLogger.EXPECT().Warn("pipeline stopped", "pipeline", "test-local", "start", gomock.Any(), "elapsed", gomock.Any()).Times(2)
-
- mockLogger.EXPECT().Info("pipeline active", "pipeline", "test-local", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
-
- // TODO delete
- mockLogger.EXPECT().Debug("request", "body:", gomock.Any(), "context:", gomock.Any()).AnyTimes()
- mockLogger.EXPECT().Debug("response", "body:", gomock.Any(), "context:", gomock.Any()).AnyTimes()
+ mockLogger.EXPECT().Warn("pipeline stopped", "pipeline", "test-local", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
err = cont.RegisterAll(
cfg,
&server.Plugin{},
&rpcPlugin.Plugin{},
- &logger.ZapLogger{},
- // mockLogger,
+ mockLogger,
&jobs.Plugin{},
&resetter.Plugin{},
&informer.Plugin{},
@@ -347,3 +300,125 @@ func TestEphemeralPauseResume(t *testing.T) {
stopCh <- struct{}{}
wg.Wait()
}
+
+func TestEphemeralJobsError(t *testing.T) {
+ cont, err := endure.NewContainer(nil, endure.SetLogLevel(endure.ErrorLevel))
+ assert.NoError(t, err)
+
+ cfg := &config.Viper{
+ Path: "ephemeral/.rr-ephemeral-jobs-err.yaml",
+ Prefix: "rr",
+ }
+
+ controller := gomock.NewController(t)
+ mockLogger := mocks.NewMockLogger(controller)
+
+ // general
+ mockLogger.EXPECT().Debug("worker destructed", "pid", gomock.Any()).AnyTimes()
+ mockLogger.EXPECT().Debug("worker constructed", "pid", gomock.Any()).AnyTimes()
+ mockLogger.EXPECT().Debug("Started RPC service", "address", "tcp://127.0.0.1:6001", "services", gomock.Any()).Times(1)
+ mockLogger.EXPECT().Error(gomock.Any(), gomock.Any(), gomock.Any()).AnyTimes()
+
+ mockLogger.EXPECT().Error("jobs protocol error", "error", "error", "delay", gomock.Any(), "requeue", gomock.Any()).Times(3)
+ mockLogger.EXPECT().Warn("pipeline stopped", "pipeline", "test-3", "start", gomock.Any(), "elapsed", gomock.Any()).Times(1)
+
+ err = cont.RegisterAll(
+ cfg,
+ &server.Plugin{},
+ &rpcPlugin.Plugin{},
+ mockLogger,
+ &jobs.Plugin{},
+ &resetter.Plugin{},
+ &informer.Plugin{},
+ &ephemeral.Plugin{},
+ )
+ assert.NoError(t, err)
+
+ err = cont.Init()
+ if err != nil {
+ t.Fatal(err)
+ }
+
+ ch, err := cont.Serve()
+ if err != nil {
+ t.Fatal(err)
+ }
+
+ sig := make(chan os.Signal, 1)
+ signal.Notify(sig, os.Interrupt, syscall.SIGINT, syscall.SIGTERM)
+
+ wg := &sync.WaitGroup{}
+ wg.Add(1)
+
+ stopCh := make(chan struct{}, 1)
+
+ go func() {
+ defer wg.Done()
+ for {
+ select {
+ case e := <-ch:
+ assert.Fail(t, "error", e.Error.Error())
+ err = cont.Stop()
+ if err != nil {
+ assert.FailNow(t, "error", err.Error())
+ }
+ case <-sig:
+ err = cont.Stop()
+ if err != nil {
+ assert.FailNow(t, "error", err.Error())
+ }
+ return
+ case <-stopCh:
+ // timeout
+ err = cont.Stop()
+ if err != nil {
+ assert.FailNow(t, "error", err.Error())
+ }
+ return
+ }
+ }
+ }()
+
+ time.Sleep(time.Second * 3)
+
+ t.Run("DeclareEphemeralPipeline", declareEphemeralPipe)
+ t.Run("ConsumeEphemeralPipeline", resumePipes("test-3"))
+ t.Run("PushEphemeralPipeline", pushToPipe("test-3"))
+ time.Sleep(time.Second * 25)
+ t.Run("PauseEphemeralPipeline", pausePipelines("test-3"))
+ time.Sleep(time.Second)
+ t.Run("DestroyEphemeralPipeline", destroyPipelines("test-3"))
+
+ time.Sleep(time.Second * 5)
+ stopCh <- struct{}{}
+ wg.Wait()
+}
+
+func declareEphemeralPipe(t *testing.T) {
+ conn, err := net.Dial("tcp", "127.0.0.1:6001")
+ assert.NoError(t, err)
+ client := rpc.NewClientWithCodec(goridgeRpc.NewClientCodec(conn))
+
+ pipe := &jobsv1beta.DeclareRequest{Pipeline: map[string]string{
+ "driver": "ephemeral",
+ "name": "test-3",
+ "prefetch": "10000",
+ }}
+
+ er := &jobsv1beta.Empty{}
+ err = client.Call("jobs.Declare", pipe, er)
+ assert.NoError(t, err)
+}
+
+func consumeEphemeralPipe(t *testing.T) {
+ conn, err := net.Dial("tcp", "127.0.0.1:6001")
+ assert.NoError(t, err)
+ client := rpc.NewClientWithCodec(goridgeRpc.NewClientCodec(conn))
+
+ pipe := &jobsv1beta.Pipelines{Pipelines: make([]string, 1)}
+ pipe.GetPipelines()[0] = "test-3"
+
+ er := &jobsv1beta.Empty{}
+ err = client.Call("jobs.Resume", pipe, er)
+ assert.NoError(t, err)
+}