diff options
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) +} |