From 1c5768784faa2b514b0b363e51d692df25091c44 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Mon, 13 Jun 2022 19:50:00 +0200 Subject: [PATCH 01/25] implement ticket queue --- go.mod | 1 + go.sum | 2 + pkg/foundation/ticketqueue/ticketqueue.go | 170 +++++++++++ .../ticketqueue/ticketqueue_test.go | 266 ++++++++++++++++++ 4 files changed, 439 insertions(+) create mode 100644 pkg/foundation/ticketqueue/ticketqueue.go create mode 100644 pkg/foundation/ticketqueue/ticketqueue_test.go diff --git a/go.mod b/go.mod index f00630e92..132eeb53e 100644 --- a/go.mod +++ b/go.mod @@ -15,6 +15,7 @@ require ( github.com/conduitio/conduit-connector-sdk v0.2.0 github.com/dgraph-io/badger/v3 v3.2103.2 github.com/dop251/goja v0.0.0-20210225094849-f3cfc97811c0 + github.com/gammazero/deque v0.2.0 github.com/golang/mock v1.6.0 github.com/google/go-cmp v0.5.8 github.com/google/uuid v1.3.0 diff --git a/go.sum b/go.sum index be1df8af7..2f695f1ce 100644 --- a/go.sum +++ b/go.sum @@ -205,6 +205,8 @@ github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMo github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= github.com/fsnotify/fsnotify v1.5.1 h1:mZcQUHVQUQWoPXXtuf9yuEXKudkV2sx1E06UadKWpgI= github.com/fsnotify/fsnotify v1.5.1/go.mod h1:T3375wBYaZdLLcVNkcVbzGHY7f1l/uK5T5Ai1i3InKU= +github.com/gammazero/deque v0.2.0 h1:SkieyNB4bg2/uZZLxvya0Pq6diUlwx7m2TeT7GAIWaA= +github.com/gammazero/deque v0.2.0/go.mod h1:LFroj8x4cMYCukHJDbxFCkT+r9AndaJnFMuZDV34tuU= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-fonts/dejavu v0.1.0/go.mod h1:4Wt4I4OU2Nq9asgDCteaAaWZOV24E+0/Pwo0gppep4g= github.com/go-fonts/latin-modern v0.2.0/go.mod h1:rQVLdDMK+mK1xscDwsqM5J8U2jrRa3T0ecnM9pNujks= diff --git a/pkg/foundation/ticketqueue/ticketqueue.go b/pkg/foundation/ticketqueue/ticketqueue.go new file mode 100644 index 000000000..d738ce607 --- /dev/null +++ b/pkg/foundation/ticketqueue/ticketqueue.go @@ -0,0 +1,170 @@ +// Copyright © 2022 Meroxa, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package ticketqueue + +import ( + "context" + + "github.com/conduitio/conduit/pkg/foundation/cerrors" + "github.com/gammazero/deque" +) + +// TicketQueue dispenses tickets and keeps track of their order. Tickets can be +// "cashed-in" for channels that let the caller communicate with a worker that +// is supposed to handle the ticket. TicketQueue ensures that tickets are +// cashed-in in the exact same order as they were dispensed. +// +// Essentially TicketQueue simulates a "take a number" system where the number +// on the ticket is monotonically increasing with each dispensed ticket. When +// the monitor displays the number on the ticket, the person holding the ticket +// can approach the counter. +// +// TicketQueue contains an unbounded buffer for tickets. A goroutine is pushing +// tickets to workers calling Next. To stop this goroutine the TicketQueue needs +// to be closed and all tickets need to be drained through Next until it returns +// an error. +type TicketQueue[REQ, RES any] struct { + // in is the channel where incoming tickets are sent into (see Take) + in chan Ticket[REQ, RES] + // out is the channel where outgoing tickets are sent into (see Next) + out chan Ticket[REQ, RES] +} + +// NewTicketQueue returns an initialized TicketQueue. +func NewTicketQueue[REQ, RES any]() *TicketQueue[REQ, RES] { + tq := &TicketQueue[REQ, RES]{ + in: make(chan Ticket[REQ, RES]), + out: make(chan Ticket[REQ, RES]), + } + tq.run() + return tq +} + +// Ticket is dispensed by TicketQueue. Once TicketQueue.Wait is called with a +// Ticket it should be discarded. +type Ticket[REQ, RES any] struct { + ctrl chan struct{} + req chan REQ + res chan RES +} + +// run launches a goroutine that fetches tickets from the channel in and buffers +// them in an unbounded queue. It also pushes tickets from the queue into the +// channel out. +func (tq *TicketQueue[REQ, RES]) run() { + in := tq.in + + // Deque is used as a normal queue and holds references to all open tickets + var q deque.Deque[Ticket[REQ, RES]] + outOrNil := func() chan Ticket[REQ, RES] { + if q.Len() == 0 { + return nil + } + return tq.out + } + nextTicket := func() Ticket[REQ, RES] { + if q.Len() == 0 { + return Ticket[REQ, RES]{} + } + return q.Front() + } + + go func() { + defer close(tq.out) + for q.Len() > 0 || in != nil { + select { + case v, ok := <-in: + if !ok { + in = nil + continue + } + q.PushBack(v) + case outOrNil() <- nextTicket(): + q.PopFront() // remove ticket from queue + } + } + }() +} + +// Take creates a ticket. The ticket can be used to call Wait. If TicketQueue +// is already closed, the call panics. +func (tq *TicketQueue[REQ, RES]) Take() Ticket[REQ, RES] { + t := Ticket[REQ, RES]{ + ctrl: make(chan struct{}), + req: make(chan REQ), + res: make(chan RES), + } + tq.in <- t + return t +} + +// Wait will block until all tickets before this ticket were already processed. +// Essentially this method means the caller wants to enqueue and wait for their +// turn. The function returns two channels that can be used to communicate with +// the processor of the ticket. The caller determines what messages are sent +// through those channels (if any). After Wait returns the ticket should be +// discarded. +// +// If ctx gets cancelled before the ticket is redeemed, the function returns the +// context error. If Wait is called a second time with the same ticket, the call +// returns an error. +func (tq *TicketQueue[REQ, RES]) Wait(ctx context.Context, t Ticket[REQ, RES]) (chan<- REQ, <-chan RES, error) { + select { + case <-ctx.Done(): + return nil, nil, ctx.Err() + case _, ok := <-t.ctrl: + if !ok { + return nil, nil, cerrors.New("ticket already used") + } + } + return t.req, t.res, nil +} + +// Next can be used to fetch the channels to communicate with the next ticket +// holder in line. If there is no next ticket holder or if the next ticket +// holder did not call Wait, the call will block. +// +// If ctx gets cancelled before the next ticket holder is ready, the function +// returns the context error. If TicketQueue is closed and there are no more +// open tickets, the call returns an error. +func (tq *TicketQueue[REQ, RES]) Next(ctx context.Context) (<-chan REQ, chan<- RES, error) { + var t Ticket[REQ, RES] + var ok bool + + select { + case <-ctx.Done(): + return nil, nil, ctx.Err() + case t, ok = <-tq.out: + if !ok { + return nil, nil, cerrors.New("TicketQueue is closed") + } + } + + select { + case <-ctx.Done(): + // BUG: the ticket is lost at this point + return nil, nil, ctx.Err() + case t.ctrl <- struct{}{}: // signal that Next is ready to proceed + close(t.ctrl) // ticket is used + } + + return t.req, t.res, nil +} + +// Close the ticket queue, no more new tickets can be dispensed after this. +// Calls to Wait and Next are still allowed until all open tickets are redeemed. +func (tq *TicketQueue[REQ, RES]) Close() { + close(tq.in) +} diff --git a/pkg/foundation/ticketqueue/ticketqueue_test.go b/pkg/foundation/ticketqueue/ticketqueue_test.go new file mode 100644 index 000000000..76f350f46 --- /dev/null +++ b/pkg/foundation/ticketqueue/ticketqueue_test.go @@ -0,0 +1,266 @@ +// Copyright © 2022 Meroxa, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package ticketqueue + +import ( + "context" + "fmt" + "math/rand" + "strings" + "sync" + "testing" + "time" + + "github.com/conduitio/conduit/pkg/foundation/cerrors" + "github.com/matryer/is" +) + +func TestTicketQueue_Next_ContextCanceled(t *testing.T) { + is := is.New(t) + ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*10) + defer cancel() + + tq := NewTicketQueue[int, float64]() + defer tq.Close() + + req, res, err := tq.Next(ctx) + is.Equal(req, nil) + is.Equal(res, nil) + is.Equal(err, context.DeadlineExceeded) +} + +func TestTicketQueue_Next_Closed(t *testing.T) { + is := is.New(t) + ctx := context.Background() + + tq := NewTicketQueue[int, float64]() + tq.Close() // close ticket queue + + req, res, err := tq.Next(ctx) + is.Equal(req, nil) + is.Equal(res, nil) + is.True(err != nil) +} + +func TestTicketQueue_Take_Closed(t *testing.T) { + is := is.New(t) + + tq := NewTicketQueue[int, float64]() + tq.Close() // close ticket queue, taking a ticket after this is not permitted + + defer func() { + is.True(recover() != nil) // expected Take to panic + }() + + tq.Take() +} + +func TestTicketQueue_Wait_ContextCanceled(t *testing.T) { + is := is.New(t) + ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*10) + defer cancel() + + tq := NewTicketQueue[int, float64]() + defer tq.Close() + + ticket := tq.Take() + req, res, err := tq.Wait(ctx, ticket) + is.Equal(req, nil) + is.Equal(res, nil) + is.Equal(err, context.DeadlineExceeded) +} + +func TestTicketQueue_Wait_ReuseTicket(t *testing.T) { + is := is.New(t) + ctx := context.Background() + + tq := NewTicketQueue[int, float64]() + defer tq.Close() + + var wg sync.WaitGroup + wg.Add(1) + go func() { + defer wg.Done() + + ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*10) + defer cancel() + _, _, err := tq.Next(ctx) + is.NoErr(err) + _, _, err = tq.Next(ctx) + is.Equal(err, context.DeadlineExceeded) + }() + + ticket := tq.Take() + _, _, err := tq.Wait(ctx, ticket) + is.NoErr(err) + + _, _, err = tq.Wait(ctx, ticket) + is.True(err != nil) // expected error for ticket that was already cashed-in + wg.Wait() +} + +func TestTicketQueue_Next_NoTicketWaiting(t *testing.T) { + is := is.New(t) + ctx := context.Background() + + tq := NewTicketQueue[int, float64]() + defer tq.Close() + + tq.Take() // take ticket, but don't cash it in + + ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*10) + defer cancel() + _, _, err := tq.Next(ctx) + is.Equal(err, context.DeadlineExceeded) +} + +func TestTicketQueue_Take_Buffer(t *testing.T) { + is := is.New(t) + ctx := context.Background() + + tq := NewTicketQueue[int, float64]() + defer tq.Close() + + // TicketQueue supports an unbounded amount of tickets, keep taking tickets + // for one second and take as many tickets as possible + testDuration := time.Second + + var wg sync.WaitGroup + var numTickets int + start := time.Now() + for time.Since(start) < testDuration { + numTickets += 1 + ticket := tq.Take() + go func() { + defer wg.Done() + _, _, err := tq.Wait(ctx, ticket) + is.NoErr(err) + }() + } + wg.Add(numTickets) + t.Logf("took %d tickets in %s", numTickets, testDuration) + + for i := 0; i < numTickets; i++ { + _, _, err := tq.Next(ctx) + is.NoErr(err) + } + + wg.Wait() // wait for all ticket goroutines to finish + + // try fetching next in line, but there is none + ctx, cancel := context.WithTimeout(ctx, time.Millisecond*10) + defer cancel() + _, _, err := tq.Next(ctx) + is.Equal(err, context.DeadlineExceeded) +} + +func TestTicketQueue_HandOff(t *testing.T) { + is := is.New(t) + ctx := context.Background() + + tq := NewTicketQueue[int, float64]() + defer tq.Close() + + wantInt := 123 + wantFloat := 1.23 + + done := make(chan struct{}) + go func() { + defer close(done) + ticket := tq.Take() + req, res, err := tq.Wait(ctx, ticket) + is.NoErr(err) + req <- wantInt + gotFloat := <-res + is.Equal(wantFloat, gotFloat) + }() + + req, res, err := tq.Next(ctx) + is.NoErr(err) + + gotInt := <-req + is.Equal(wantInt, gotInt) + + res <- wantFloat + <-done +} + +func ExampleTicketQueue() { + ctx := context.Background() + + tq := NewTicketQueue[string, error]() + defer tq.Close() + + sentence := []string{ + "Each", "word", "will", "be", "sent", "to", "the", "collector", "in", + "a", "separate", "goroutine", "and", "even", "though", "they", "will", + "sleep", "for", "a", "random", "amount", "of", "time,", "all", "words", + "will", "be", "processed", "in", "the", "right", "order.", + } + + r := rand.New(rand.NewSource(time.Now().UnixMilli())) + var wg sync.WaitGroup + for _, word := range sentence { + t := tq.Take() + wg.Add(1) + go func(word string) { + defer wg.Done() + // sleep for a random amount of time to simulate work being done + time.Sleep(time.Millisecond * time.Duration(r.Intn(100))) + // try to cash in ticket + req, res, err := tq.Wait(ctx, t) + if err != nil { + panic(cerrors.Errorf("unexpected error: %w", err)) + } + req <- word // send word to collector + err = <-res // receive error back + if err != nil { + panic(cerrors.Errorf("unexpected error: %w", err)) + } + }(word) + } + + // collect all tickets + var builder strings.Builder + for { + ctx, cancel := context.WithTimeout(ctx, time.Millisecond*200) + defer cancel() + + req, res, err := tq.Next(ctx) + if err != nil { + if err == context.DeadlineExceeded { + break + } + panic(cerrors.Errorf("unexpected error: %w", err)) + } + + word := <-req + _, err = builder.WriteRune(' ') + if err != nil { + res <- err + } + _, err = builder.WriteString(word) + if err != nil { + res <- err + } + close(res) + } + wg.Wait() + + fmt.Println(builder.String()) + + // Output: + // Each word will be sent to the collector in a separate goroutine and even though they will sleep for a random amount of time, all words will be processed in the right order. +} From 8bf3f66066559c8ca48d6fc7fba14147ba9275e0 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Tue, 14 Jun 2022 19:47:00 +0200 Subject: [PATCH 02/25] experiment with ordered semaphore --- pkg/foundation/semaphore/semaphore.go | 170 +++++++++++++++ .../semaphore/semaphore_bench_test.go | 90 ++++++++ pkg/foundation/semaphore/semaphore_test.go | 195 ++++++++++++++++++ .../ticketqueue/ticketqueue_test.go | 6 +- 4 files changed, 458 insertions(+), 3 deletions(-) create mode 100644 pkg/foundation/semaphore/semaphore.go create mode 100644 pkg/foundation/semaphore/semaphore_bench_test.go create mode 100644 pkg/foundation/semaphore/semaphore_test.go diff --git a/pkg/foundation/semaphore/semaphore.go b/pkg/foundation/semaphore/semaphore.go new file mode 100644 index 000000000..ad74ff126 --- /dev/null +++ b/pkg/foundation/semaphore/semaphore.go @@ -0,0 +1,170 @@ +// Copyright © 2022 Meroxa, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package semaphore provides a weighted semaphore implementation. +package semaphore + +import ( + "container/list" + "context" + "sync" +) + +// NewWeighted creates a new weighted semaphore with the given +// maximum combined weight for concurrent access. +func NewWeighted(n int64) *Weighted { + w := &Weighted{size: n} + return w +} + +// Weighted provides a way to bound concurrent access to a resource. +// The callers can request access with a given weight. +type Weighted struct { + size int64 + cur int64 + mu sync.Mutex + waiters list.List +} + +type waiter struct { + acquired bool + released bool + n int64 + ready chan struct{} // Closed when semaphore acquired. +} + +type Ticket struct { + elem *list.Element +} + +func (s *Weighted) Enqueue(n int64) Ticket { + if n > s.size { + panic("semaphore: tried to enqueue more than size of semaphore") + } + + s.mu.Lock() + w := waiter{n: n, ready: make(chan struct{})} + e := s.waiters.PushBack(w) + s.mu.Unlock() + return Ticket{elem: e} +} + +// Acquire acquires the semaphore with a weight of n, blocking until resources +// are available or ctx is done. On success, returns nil. On failure, returns +// ctx.Err() and leaves the semaphore unchanged. +// +// If ctx is already done, Acquire may still succeed without blocking. +func (s *Weighted) Acquire(ctx context.Context, t Ticket) error { + w := t.elem.Value.(waiter) + + s.mu.Lock() + if s.waiters.Front() == t.elem && s.size-s.cur >= w.n { + s.cur += w.n + s.waiters.Remove(t.elem) + w.acquired = true + t.elem.Value = w + // If there are extra tokens left, notify other waiters. + if s.size > s.cur { + s.notifyWaiters() + } + s.mu.Unlock() + return nil + } + if w.n > s.size { + // Don't make other Acquire calls block on one that's doomed to fail. + s.mu.Unlock() + <-ctx.Done() + return ctx.Err() + } + s.mu.Unlock() + + select { + case <-ctx.Done(): + err := ctx.Err() + s.mu.Lock() + select { + case <-w.ready: + // Acquired the semaphore after we were canceled. Rather than trying to + // fix up the queue, just pretend we didn't notice the cancelation. + err = nil + default: + isFront := s.waiters.Front() == t.elem + s.waiters.Remove(t.elem) + // If we're at the front and there are extra tokens left, notify other waiters. + if isFront && s.size > s.cur { + s.notifyWaiters() + } + } + s.mu.Unlock() + return err + + case <-w.ready: + return nil + } +} + +// Release releases the semaphore with a weight of n. +func (s *Weighted) Release(t Ticket) { + w := t.elem.Value.(waiter) + s.mu.Lock() + if !w.acquired { + s.mu.Unlock() + panic("semaphore: can't release ticket that was not acquired") + } + if w.released { + s.mu.Unlock() + panic("semaphore: ticket released twice") + } + + s.cur -= t.elem.Value.(waiter).n + w.released = true + t.elem.Value = w + if s.cur < 0 { + s.mu.Unlock() + panic("semaphore: released more than held") + } + s.notifyWaiters() + s.mu.Unlock() +} + +func (s *Weighted) notifyWaiters() { + for { + next := s.waiters.Front() + if next == nil { + break // No more waiters blocked. + } + + w := next.Value.(waiter) + if s.size-s.cur < w.n { + // Not enough tokens for the next waiter. We could keep going (to try to + // find a waiter with a smaller request), but under load that could cause + // starvation for large requests; instead, we leave all remaining waiters + // blocked. + // + // Consider a semaphore used as a read-write lock, with N tokens, N + // readers, and one writer. Each reader can Acquire(1) to obtain a read + // lock. The writer can Acquire(N) to obtain a write lock, excluding all + // of the readers. If we allow the readers to jump ahead in the queue, + // the writer will starve — there is always one token available for every + // reader. + break + } + + w.acquired = true + next.Value = w + s.cur += w.n + s.waiters.Remove(next) + close(w.ready) + } +} diff --git a/pkg/foundation/semaphore/semaphore_bench_test.go b/pkg/foundation/semaphore/semaphore_bench_test.go new file mode 100644 index 000000000..0dc1857ad --- /dev/null +++ b/pkg/foundation/semaphore/semaphore_bench_test.go @@ -0,0 +1,90 @@ +// Copyright © 2022 Meroxa, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package semaphore_test + +import ( + "container/list" + "context" + "fmt" + "testing" + + "github.com/conduitio/conduit/pkg/foundation/semaphore" +) + +// weighted is an interface matching a subset of *Weighted. It allows +// alternate implementations for testing and benchmarking. +type weighted interface { + Enqueue(int64) semaphore.Ticket + Acquire(context.Context, semaphore.Ticket) error + Release(semaphore.Ticket) +} + +// acquireN calls Acquire(size) on sem N times and then calls Release(size) N times. +func acquireN(b *testing.B, sem weighted, size int64, N int) { + b.ResetTimer() + tickets := list.New() + for i := 0; i < b.N; i++ { + tickets.Init() + for j := 0; j < N; j++ { + ticket := sem.Enqueue(size) + tickets.PushBack(ticket) + sem.Acquire(context.Background(), ticket) + } + ticket := tickets.Front() + for ticket != nil { + sem.Release(ticket.Value.(semaphore.Ticket)) + ticket = ticket.Next() + } + } +} + +func BenchmarkNewSeq(b *testing.B) { + for _, cap := range []int64{1, 128} { + b.Run(fmt.Sprintf("Weighted-%d", cap), func(b *testing.B) { + for i := 0; i < b.N; i++ { + _ = semaphore.NewWeighted(cap) + } + }) + } +} + +func BenchmarkAcquireSeq(b *testing.B) { + for _, c := range []struct { + cap, size int64 + N int + }{ + {1, 1, 1}, + {2, 1, 1}, + {16, 1, 1}, + {128, 1, 1}, + {2, 2, 1}, + {16, 2, 8}, + {128, 2, 64}, + {2, 1, 2}, + {16, 8, 2}, + {128, 64, 2}, + } { + for _, w := range []struct { + name string + w weighted + }{ + {"Weighted", semaphore.NewWeighted(c.cap)}, + } { + b.Run(fmt.Sprintf("%s-acquire-%d-%d-%d", w.name, c.cap, c.size, c.N), func(b *testing.B) { + acquireN(b, w.w, c.size, c.N) + }) + } + } +} diff --git a/pkg/foundation/semaphore/semaphore_test.go b/pkg/foundation/semaphore/semaphore_test.go new file mode 100644 index 000000000..89a2b5428 --- /dev/null +++ b/pkg/foundation/semaphore/semaphore_test.go @@ -0,0 +1,195 @@ +// Copyright © 2022 Meroxa, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package semaphore_test + +import ( + "context" + "math/rand" + "runtime" + "sync" + "testing" + "time" + + "github.com/conduitio/conduit/pkg/foundation/semaphore" +) + +const maxSleep = 1 * time.Millisecond + +func HammerWeighted(sem *semaphore.Weighted, n int64, loops int) { + for i := 0; i < loops; i++ { + tkn := sem.Enqueue(n) + err := sem.Acquire(context.Background(), tkn) + if err != nil { + panic(err) + } + time.Sleep(time.Duration(rand.Int63n(int64(maxSleep/time.Nanosecond))) * time.Nanosecond) + sem.Release(tkn) + } +} + +func TestWeighted(t *testing.T) { + t.Parallel() + + n := runtime.GOMAXPROCS(0) + loops := 10000 / n + sem := semaphore.NewWeighted(int64(n)) + + var wg sync.WaitGroup + wg.Add(n) + for i := 0; i < n; i++ { + i := i + go func() { + defer wg.Done() + HammerWeighted(sem, int64(i), loops) + }() + } + wg.Wait() +} + +func TestWeightedPanicReleaseUnacquired(t *testing.T) { + t.Parallel() + + defer func() { + if recover() == nil { + t.Fatal("release of an unacquired weighted semaphore did not panic") + } + }() + w := semaphore.NewWeighted(1) + tkn := w.Enqueue(1) + w.Release(tkn) +} + +func TestWeightedPanicEnqueueTooBig(t *testing.T) { + t.Parallel() + + defer func() { + if recover() == nil { + t.Fatal("enqueue of size bigger than weighted semaphore did not panic") + } + }() + const n = 5 + sem := semaphore.NewWeighted(n) + sem.Enqueue(n + 1) +} + +func TestWeightedAcquire(t *testing.T) { + t.Parallel() + + ctx := context.Background() + sem := semaphore.NewWeighted(2) + tryAcquire := func(n int64) bool { + ctx, cancel := context.WithTimeout(ctx, 10*time.Millisecond) + defer cancel() + + tkn := sem.Enqueue(1) + return sem.Acquire(ctx, tkn) == nil + } + + tries := []bool{} + tkn := sem.Enqueue(1) + sem.Acquire(ctx, tkn) + tries = append(tries, tryAcquire(1)) + tries = append(tries, tryAcquire(1)) + + sem.Release(tkn) + + tkn = sem.Enqueue(1) + sem.Acquire(ctx, tkn) + tries = append(tries, tryAcquire(1)) + + want := []bool{true, false, false} + for i := range tries { + if tries[i] != want[i] { + t.Errorf("tries[%d]: got %t, want %t", i, tries[i], want[i]) + } + } +} + +// TestLargeAcquireDoesntStarve times out if a large call to Acquire starves. +// Merely returning from the test function indicates success. +func TestLargeAcquireDoesntStarve(t *testing.T) { + t.Parallel() + + ctx := context.Background() + n := int64(runtime.GOMAXPROCS(0)) + sem := semaphore.NewWeighted(n) + running := true + + var wg sync.WaitGroup + wg.Add(int(n)) + for i := n; i > 0; i-- { + tkn := sem.Enqueue(1) + sem.Acquire(ctx, tkn) + go func() { + defer func() { + sem.Release(tkn) + wg.Done() + }() + for running { + time.Sleep(1 * time.Millisecond) + sem.Release(tkn) + tkn = sem.Enqueue(1) + sem.Acquire(ctx, tkn) + } + }() + } + + tkn := sem.Enqueue(n) + sem.Acquire(ctx, tkn) + running = false + sem.Release(tkn) + wg.Wait() +} + +// translated from https://github.com/zhiqiangxu/util/blob/master/mutex/crwmutex_test.go#L43 +func TestAllocCancelDoesntStarve(t *testing.T) { + sem := semaphore.NewWeighted(10) + + // Block off a portion of the semaphore so that Acquire(_, 10) can eventually succeed. + tkn := sem.Enqueue(1) + sem.Acquire(context.Background(), tkn) + + // In the background, Acquire(_, 10). + ctx, cancel := context.WithCancel(context.Background()) + defer cancel() + go func() { + tkn := sem.Enqueue(10) + sem.Acquire(ctx, tkn) + }() + + // Wait until the Acquire(_, 10) call blocks. + for { + ctx, cancel := context.WithTimeout(ctx, time.Millisecond) + tkn := sem.Enqueue(1) + err := sem.Acquire(ctx, tkn) + cancel() + if err != nil { + break + } + sem.Release(tkn) + runtime.Gosched() + } + + // Now try to grab a read lock, and simultaneously unblock the Acquire(_, 10) call. + // Both Acquire calls should unblock and return, in either order. + go cancel() + + tkn = sem.Enqueue(1) + err := sem.Acquire(context.Background(), tkn) + if err != nil { + t.Fatalf("Acquire(_, 1) failed unexpectedly: %v", err) + } + sem.Release(tkn) +} diff --git a/pkg/foundation/ticketqueue/ticketqueue_test.go b/pkg/foundation/ticketqueue/ticketqueue_test.go index 76f350f46..18a5adedb 100644 --- a/pkg/foundation/ticketqueue/ticketqueue_test.go +++ b/pkg/foundation/ticketqueue/ticketqueue_test.go @@ -215,10 +215,10 @@ func ExampleTicketQueue() { for _, word := range sentence { t := tq.Take() wg.Add(1) - go func(word string) { + go func(word string, delay time.Duration) { defer wg.Done() // sleep for a random amount of time to simulate work being done - time.Sleep(time.Millisecond * time.Duration(r.Intn(100))) + time.Sleep(delay) // try to cash in ticket req, res, err := tq.Wait(ctx, t) if err != nil { @@ -229,7 +229,7 @@ func ExampleTicketQueue() { if err != nil { panic(cerrors.Errorf("unexpected error: %w", err)) } - }(word) + }(word, time.Millisecond*time.Duration(r.Intn(100))) } // collect all tickets From 424d88956e53111dd512a9ef8e72b325cf360147 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Tue, 14 Jun 2022 20:01:01 +0200 Subject: [PATCH 03/25] ticketqueue benchmarks --- .../semaphore/semaphore_bench_test.go | 4 +- .../ticketqueue/ticketqueue_bench_test.go | 60 +++++++++++++++++++ 2 files changed, 62 insertions(+), 2 deletions(-) create mode 100644 pkg/foundation/ticketqueue/ticketqueue_bench_test.go diff --git a/pkg/foundation/semaphore/semaphore_bench_test.go b/pkg/foundation/semaphore/semaphore_bench_test.go index 0dc1857ad..565b5ac94 100644 --- a/pkg/foundation/semaphore/semaphore_bench_test.go +++ b/pkg/foundation/semaphore/semaphore_bench_test.go @@ -50,7 +50,7 @@ func acquireN(b *testing.B, sem weighted, size int64, N int) { } } -func BenchmarkNewSeq(b *testing.B) { +func BenchmarkNewSem(b *testing.B) { for _, cap := range []int64{1, 128} { b.Run(fmt.Sprintf("Weighted-%d", cap), func(b *testing.B) { for i := 0; i < b.N; i++ { @@ -60,7 +60,7 @@ func BenchmarkNewSeq(b *testing.B) { } } -func BenchmarkAcquireSeq(b *testing.B) { +func BenchmarkAcquireSem(b *testing.B) { for _, c := range []struct { cap, size int64 N int diff --git a/pkg/foundation/ticketqueue/ticketqueue_bench_test.go b/pkg/foundation/ticketqueue/ticketqueue_bench_test.go new file mode 100644 index 000000000..844136c15 --- /dev/null +++ b/pkg/foundation/ticketqueue/ticketqueue_bench_test.go @@ -0,0 +1,60 @@ +// Copyright © 2022 Meroxa, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package ticketqueue + +import ( + "context" + "fmt" + "testing" + + "github.com/gammazero/deque" +) + +func BenchmarkNewTicketQueue(b *testing.B) { + for i := 0; i < b.N; i++ { + tq := NewTicketQueue[int, int]() + defer tq.Close() + } + b.StopTimer() // don't measure Close +} + +func BenchmarkTicketQueueTake(b *testing.B) { + for _, N := range []int{1, 2, 8, 64, 128} { + b.Run(fmt.Sprintf("TicketQueue-%d", N), func(b *testing.B) { + ctx, cancel := context.WithCancel(context.Background()) + tq := NewTicketQueue[int, int]() + go func() { + for { + _, _, err := tq.Next(ctx) + if err == context.Canceled { + return + } + } + }() + + for i := 0; i < b.N; i++ { + tickets := deque.Deque[Ticket[int, int]]{} + for j := 0; j < N; j++ { + ticket := tq.Take() + tickets.PushBack(ticket) + } + for tickets.Len() > 0 { + tq.Wait(ctx, tickets.PopFront()) + } + } + cancel() + }) + } +} From 8a852db3ca4c92f66173c2d13544ae69d9dd2a09 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Fri, 17 Jun 2022 20:45:01 +0200 Subject: [PATCH 04/25] reduce allocations --- pkg/foundation/semaphore/semaphore.go | 143 +++++++++--------- .../semaphore/semaphore_bench_test.go | 14 +- pkg/foundation/semaphore/semaphore_test.go | 140 ++++++++--------- 3 files changed, 138 insertions(+), 159 deletions(-) diff --git a/pkg/foundation/semaphore/semaphore.go b/pkg/foundation/semaphore/semaphore.go index ad74ff126..e1f06af5d 100644 --- a/pkg/foundation/semaphore/semaphore.go +++ b/pkg/foundation/semaphore/semaphore.go @@ -16,9 +16,9 @@ package semaphore import ( - "container/list" - "context" "sync" + + "github.com/conduitio/conduit/pkg/foundation/cerrors" ) // NewWeighted creates a new weighted semaphore with the given @@ -31,21 +31,28 @@ func NewWeighted(n int64) *Weighted { // Weighted provides a way to bound concurrent access to a resource. // The callers can request access with a given weight. type Weighted struct { - size int64 - cur int64 - mu sync.Mutex - waiters list.List + size int64 + cur int64 + released int + mu sync.Mutex + + waiters []waiter + front int + batch int64 } type waiter struct { - acquired bool + index int + n int64 + ready chan struct{} // Closed when semaphore acquired. + released bool - n int64 - ready chan struct{} // Closed when semaphore acquired. + acquired bool } type Ticket struct { - elem *list.Element + index int + batch int64 } func (s *Weighted) Enqueue(n int64) Ticket { @@ -54,10 +61,16 @@ func (s *Weighted) Enqueue(n int64) Ticket { } s.mu.Lock() - w := waiter{n: n, ready: make(chan struct{})} - e := s.waiters.PushBack(w) - s.mu.Unlock() - return Ticket{elem: e} + defer s.mu.Unlock() + + index := len(s.waiters) + w := waiter{index: index, n: n, ready: make(chan struct{})} + s.waiters = append(s.waiters, w) + + return Ticket{ + index: index, + batch: s.batch, + } } // Acquire acquires the semaphore with a weight of n, blocking until resources @@ -65,15 +78,24 @@ func (s *Weighted) Enqueue(n int64) Ticket { // ctx.Err() and leaves the semaphore unchanged. // // If ctx is already done, Acquire may still succeed without blocking. -func (s *Weighted) Acquire(ctx context.Context, t Ticket) error { - w := t.elem.Value.(waiter) - +func (s *Weighted) Acquire(t Ticket) error { s.mu.Lock() - if s.waiters.Front() == t.elem && s.size-s.cur >= w.n { + if s.batch != t.batch { + s.mu.Unlock() + return cerrors.Errorf("semaphore: invalid batch") + } + + w := s.waiters[t.index] + if w.acquired { + return cerrors.New("semaphore: can't acquire ticket that was already acquired") + } + + w.acquired = true // mark that Acquire was already called for this Ticket + s.waiters[t.index] = w + + if s.front == t.index && s.size-s.cur >= w.n { s.cur += w.n - s.waiters.Remove(t.elem) - w.acquired = true - t.elem.Value = w + s.front++ // If there are extra tokens left, notify other waiters. if s.size > s.cur { s.notifyWaiters() @@ -81,71 +103,42 @@ func (s *Weighted) Acquire(ctx context.Context, t Ticket) error { s.mu.Unlock() return nil } - if w.n > s.size { - // Don't make other Acquire calls block on one that's doomed to fail. - s.mu.Unlock() - <-ctx.Done() - return ctx.Err() - } s.mu.Unlock() - select { - case <-ctx.Done(): - err := ctx.Err() - s.mu.Lock() - select { - case <-w.ready: - // Acquired the semaphore after we were canceled. Rather than trying to - // fix up the queue, just pretend we didn't notice the cancelation. - err = nil - default: - isFront := s.waiters.Front() == t.elem - s.waiters.Remove(t.elem) - // If we're at the front and there are extra tokens left, notify other waiters. - if isFront && s.size > s.cur { - s.notifyWaiters() - } - } - s.mu.Unlock() - return err - - case <-w.ready: - return nil - } + <-w.ready + return nil } // Release releases the semaphore with a weight of n. -func (s *Weighted) Release(t Ticket) { - w := t.elem.Value.(waiter) +func (s *Weighted) Release(t Ticket) error { s.mu.Lock() + defer s.mu.Unlock() + + if s.batch != t.batch { + return cerrors.Errorf("semaphore: invalid batch") + } + w := s.waiters[t.index] if !w.acquired { - s.mu.Unlock() - panic("semaphore: can't release ticket that was not acquired") + return cerrors.New("semaphore: can't release ticket that was not acquired") } if w.released { - s.mu.Unlock() - panic("semaphore: ticket released twice") + return cerrors.New("semaphore: ticket already released") } - s.cur -= t.elem.Value.(waiter).n + s.cur -= w.n w.released = true - t.elem.Value = w - if s.cur < 0 { - s.mu.Unlock() - panic("semaphore: released more than held") - } + s.waiters[t.index] = w + s.released++ s.notifyWaiters() - s.mu.Unlock() + if s.released == len(s.waiters) { + s.increaseBatch() + } + return nil } func (s *Weighted) notifyWaiters() { - for { - next := s.waiters.Front() - if next == nil { - break // No more waiters blocked. - } - - w := next.Value.(waiter) + for len(s.waiters) > s.front { + w := s.waiters[s.front] if s.size-s.cur < w.n { // Not enough tokens for the next waiter. We could keep going (to try to // find a waiter with a smaller request), but under load that could cause @@ -161,10 +154,14 @@ func (s *Weighted) notifyWaiters() { break } - w.acquired = true - next.Value = w s.cur += w.n - s.waiters.Remove(next) + s.front++ close(w.ready) } } + +func (s *Weighted) increaseBatch() { + s.waiters = s.waiters[:0] + s.batch += 1 + s.front = 0 +} diff --git a/pkg/foundation/semaphore/semaphore_bench_test.go b/pkg/foundation/semaphore/semaphore_bench_test.go index 565b5ac94..73dcfb686 100644 --- a/pkg/foundation/semaphore/semaphore_bench_test.go +++ b/pkg/foundation/semaphore/semaphore_bench_test.go @@ -16,7 +16,6 @@ package semaphore_test import ( "container/list" - "context" "fmt" "testing" @@ -27,8 +26,8 @@ import ( // alternate implementations for testing and benchmarking. type weighted interface { Enqueue(int64) semaphore.Ticket - Acquire(context.Context, semaphore.Ticket) error - Release(semaphore.Ticket) + Acquire(semaphore.Ticket) error + Release(semaphore.Ticket) error } // acquireN calls Acquire(size) on sem N times and then calls Release(size) N times. @@ -38,14 +37,7 @@ func acquireN(b *testing.B, sem weighted, size int64, N int) { for i := 0; i < b.N; i++ { tickets.Init() for j := 0; j < N; j++ { - ticket := sem.Enqueue(size) - tickets.PushBack(ticket) - sem.Acquire(context.Background(), ticket) - } - ticket := tickets.Front() - for ticket != nil { - sem.Release(ticket.Value.(semaphore.Ticket)) - ticket = ticket.Next() + _ = sem.Enqueue(size) } } } diff --git a/pkg/foundation/semaphore/semaphore_test.go b/pkg/foundation/semaphore/semaphore_test.go index 89a2b5428..9845f58fc 100644 --- a/pkg/foundation/semaphore/semaphore_test.go +++ b/pkg/foundation/semaphore/semaphore_test.go @@ -15,7 +15,6 @@ package semaphore_test import ( - "context" "math/rand" "runtime" "sync" @@ -30,7 +29,7 @@ const maxSleep = 1 * time.Millisecond func HammerWeighted(sem *semaphore.Weighted, n int64, loops int) { for i := 0; i < loops; i++ { tkn := sem.Enqueue(n) - err := sem.Acquire(context.Background(), tkn) + err := sem.Acquire(tkn) if err != nil { panic(err) } @@ -58,17 +57,48 @@ func TestWeighted(t *testing.T) { wg.Wait() } -func TestWeightedPanicReleaseUnacquired(t *testing.T) { +func TestWeightedReleaseUnacquired(t *testing.T) { t.Parallel() - defer func() { - if recover() == nil { - t.Fatal("release of an unacquired weighted semaphore did not panic") - } - }() w := semaphore.NewWeighted(1) tkn := w.Enqueue(1) - w.Release(tkn) + err := w.Release(tkn) + if err == nil { + t.Errorf("release of an unacquired ticket did not return an error") + } +} + +func TestWeightedReleaseTwice(t *testing.T) { + t.Parallel() + + w := semaphore.NewWeighted(1) + tkn := w.Enqueue(1) + w.Acquire(tkn) + err := w.Release(tkn) + if err != nil { + t.Errorf("release of an acquired ticket errored out: %v", err) + } + + err = w.Release(tkn) + if err == nil { + t.Errorf("release of an already released ticket did not return an error") + } +} + +func TestWeightedAcquireTwice(t *testing.T) { + t.Parallel() + + w := semaphore.NewWeighted(1) + tkn := w.Enqueue(1) + err := w.Acquire(tkn) + if err != nil { + t.Errorf("acquire of a ticket errored out: %v", err) + } + + err = w.Acquire(tkn) + if err == nil { + t.Errorf("acquire of an already acquired ticket did not return an error") + } } func TestWeightedPanicEnqueueTooBig(t *testing.T) { @@ -87,33 +117,35 @@ func TestWeightedPanicEnqueueTooBig(t *testing.T) { func TestWeightedAcquire(t *testing.T) { t.Parallel() - ctx := context.Background() sem := semaphore.NewWeighted(2) - tryAcquire := func(n int64) bool { - ctx, cancel := context.WithTimeout(ctx, 10*time.Millisecond) - defer cancel() - tkn := sem.Enqueue(1) - return sem.Acquire(ctx, tkn) == nil - } + tkn1 := sem.Enqueue(1) + sem.Acquire(tkn1) - tries := []bool{} - tkn := sem.Enqueue(1) - sem.Acquire(ctx, tkn) - tries = append(tries, tryAcquire(1)) - tries = append(tries, tryAcquire(1)) + tkn2 := sem.Enqueue(1) + sem.Acquire(tkn2) - sem.Release(tkn) + tkn3done := make(chan struct{}) + go func() { + defer close(tkn3done) + tkn3 := sem.Enqueue(1) + sem.Acquire(tkn3) + }() - tkn = sem.Enqueue(1) - sem.Acquire(ctx, tkn) - tries = append(tries, tryAcquire(1)) + select { + case <-tkn3done: + t.Errorf("tkn3done closed prematurely") + case <-time.After(time.Millisecond * 10): + // tkn3 Acquire is blocking as expected + } - want := []bool{true, false, false} - for i := range tries { - if tries[i] != want[i] { - t.Errorf("tries[%d]: got %t, want %t", i, tries[i], want[i]) - } + sem.Release(tkn1) + + select { + case <-tkn3done: + // tkn3 successfully acquired the semaphore + case <-time.After(time.Millisecond * 10): + t.Errorf("tkn3done didn't get closed") } } @@ -122,7 +154,6 @@ func TestWeightedAcquire(t *testing.T) { func TestLargeAcquireDoesntStarve(t *testing.T) { t.Parallel() - ctx := context.Background() n := int64(runtime.GOMAXPROCS(0)) sem := semaphore.NewWeighted(n) running := true @@ -131,7 +162,7 @@ func TestLargeAcquireDoesntStarve(t *testing.T) { wg.Add(int(n)) for i := n; i > 0; i-- { tkn := sem.Enqueue(1) - sem.Acquire(ctx, tkn) + sem.Acquire(tkn) go func() { defer func() { sem.Release(tkn) @@ -141,55 +172,14 @@ func TestLargeAcquireDoesntStarve(t *testing.T) { time.Sleep(1 * time.Millisecond) sem.Release(tkn) tkn = sem.Enqueue(1) - sem.Acquire(ctx, tkn) + sem.Acquire(tkn) } }() } tkn := sem.Enqueue(n) - sem.Acquire(ctx, tkn) + sem.Acquire(tkn) running = false sem.Release(tkn) wg.Wait() } - -// translated from https://github.com/zhiqiangxu/util/blob/master/mutex/crwmutex_test.go#L43 -func TestAllocCancelDoesntStarve(t *testing.T) { - sem := semaphore.NewWeighted(10) - - // Block off a portion of the semaphore so that Acquire(_, 10) can eventually succeed. - tkn := sem.Enqueue(1) - sem.Acquire(context.Background(), tkn) - - // In the background, Acquire(_, 10). - ctx, cancel := context.WithCancel(context.Background()) - defer cancel() - go func() { - tkn := sem.Enqueue(10) - sem.Acquire(ctx, tkn) - }() - - // Wait until the Acquire(_, 10) call blocks. - for { - ctx, cancel := context.WithTimeout(ctx, time.Millisecond) - tkn := sem.Enqueue(1) - err := sem.Acquire(ctx, tkn) - cancel() - if err != nil { - break - } - sem.Release(tkn) - runtime.Gosched() - } - - // Now try to grab a read lock, and simultaneously unblock the Acquire(_, 10) call. - // Both Acquire calls should unblock and return, in either order. - go cancel() - - tkn = sem.Enqueue(1) - err := sem.Acquire(context.Background(), tkn) - if err != nil { - t.Fatalf("Acquire(_, 1) failed unexpectedly: %v", err) - } - sem.Release(tkn) -} From ec7249ebfc0bd247fa4de9c98d00c7cc0f968ffd Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Tue, 21 Jun 2022 17:24:27 +0200 Subject: [PATCH 05/25] remove ticketqueue (semaphore implementation is more performant) --- pkg/foundation/ticketqueue/ticketqueue.go | 170 ----------- .../ticketqueue/ticketqueue_bench_test.go | 60 ---- .../ticketqueue/ticketqueue_test.go | 266 ------------------ 3 files changed, 496 deletions(-) delete mode 100644 pkg/foundation/ticketqueue/ticketqueue.go delete mode 100644 pkg/foundation/ticketqueue/ticketqueue_bench_test.go delete mode 100644 pkg/foundation/ticketqueue/ticketqueue_test.go diff --git a/pkg/foundation/ticketqueue/ticketqueue.go b/pkg/foundation/ticketqueue/ticketqueue.go deleted file mode 100644 index d738ce607..000000000 --- a/pkg/foundation/ticketqueue/ticketqueue.go +++ /dev/null @@ -1,170 +0,0 @@ -// Copyright © 2022 Meroxa, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package ticketqueue - -import ( - "context" - - "github.com/conduitio/conduit/pkg/foundation/cerrors" - "github.com/gammazero/deque" -) - -// TicketQueue dispenses tickets and keeps track of their order. Tickets can be -// "cashed-in" for channels that let the caller communicate with a worker that -// is supposed to handle the ticket. TicketQueue ensures that tickets are -// cashed-in in the exact same order as they were dispensed. -// -// Essentially TicketQueue simulates a "take a number" system where the number -// on the ticket is monotonically increasing with each dispensed ticket. When -// the monitor displays the number on the ticket, the person holding the ticket -// can approach the counter. -// -// TicketQueue contains an unbounded buffer for tickets. A goroutine is pushing -// tickets to workers calling Next. To stop this goroutine the TicketQueue needs -// to be closed and all tickets need to be drained through Next until it returns -// an error. -type TicketQueue[REQ, RES any] struct { - // in is the channel where incoming tickets are sent into (see Take) - in chan Ticket[REQ, RES] - // out is the channel where outgoing tickets are sent into (see Next) - out chan Ticket[REQ, RES] -} - -// NewTicketQueue returns an initialized TicketQueue. -func NewTicketQueue[REQ, RES any]() *TicketQueue[REQ, RES] { - tq := &TicketQueue[REQ, RES]{ - in: make(chan Ticket[REQ, RES]), - out: make(chan Ticket[REQ, RES]), - } - tq.run() - return tq -} - -// Ticket is dispensed by TicketQueue. Once TicketQueue.Wait is called with a -// Ticket it should be discarded. -type Ticket[REQ, RES any] struct { - ctrl chan struct{} - req chan REQ - res chan RES -} - -// run launches a goroutine that fetches tickets from the channel in and buffers -// them in an unbounded queue. It also pushes tickets from the queue into the -// channel out. -func (tq *TicketQueue[REQ, RES]) run() { - in := tq.in - - // Deque is used as a normal queue and holds references to all open tickets - var q deque.Deque[Ticket[REQ, RES]] - outOrNil := func() chan Ticket[REQ, RES] { - if q.Len() == 0 { - return nil - } - return tq.out - } - nextTicket := func() Ticket[REQ, RES] { - if q.Len() == 0 { - return Ticket[REQ, RES]{} - } - return q.Front() - } - - go func() { - defer close(tq.out) - for q.Len() > 0 || in != nil { - select { - case v, ok := <-in: - if !ok { - in = nil - continue - } - q.PushBack(v) - case outOrNil() <- nextTicket(): - q.PopFront() // remove ticket from queue - } - } - }() -} - -// Take creates a ticket. The ticket can be used to call Wait. If TicketQueue -// is already closed, the call panics. -func (tq *TicketQueue[REQ, RES]) Take() Ticket[REQ, RES] { - t := Ticket[REQ, RES]{ - ctrl: make(chan struct{}), - req: make(chan REQ), - res: make(chan RES), - } - tq.in <- t - return t -} - -// Wait will block until all tickets before this ticket were already processed. -// Essentially this method means the caller wants to enqueue and wait for their -// turn. The function returns two channels that can be used to communicate with -// the processor of the ticket. The caller determines what messages are sent -// through those channels (if any). After Wait returns the ticket should be -// discarded. -// -// If ctx gets cancelled before the ticket is redeemed, the function returns the -// context error. If Wait is called a second time with the same ticket, the call -// returns an error. -func (tq *TicketQueue[REQ, RES]) Wait(ctx context.Context, t Ticket[REQ, RES]) (chan<- REQ, <-chan RES, error) { - select { - case <-ctx.Done(): - return nil, nil, ctx.Err() - case _, ok := <-t.ctrl: - if !ok { - return nil, nil, cerrors.New("ticket already used") - } - } - return t.req, t.res, nil -} - -// Next can be used to fetch the channels to communicate with the next ticket -// holder in line. If there is no next ticket holder or if the next ticket -// holder did not call Wait, the call will block. -// -// If ctx gets cancelled before the next ticket holder is ready, the function -// returns the context error. If TicketQueue is closed and there are no more -// open tickets, the call returns an error. -func (tq *TicketQueue[REQ, RES]) Next(ctx context.Context) (<-chan REQ, chan<- RES, error) { - var t Ticket[REQ, RES] - var ok bool - - select { - case <-ctx.Done(): - return nil, nil, ctx.Err() - case t, ok = <-tq.out: - if !ok { - return nil, nil, cerrors.New("TicketQueue is closed") - } - } - - select { - case <-ctx.Done(): - // BUG: the ticket is lost at this point - return nil, nil, ctx.Err() - case t.ctrl <- struct{}{}: // signal that Next is ready to proceed - close(t.ctrl) // ticket is used - } - - return t.req, t.res, nil -} - -// Close the ticket queue, no more new tickets can be dispensed after this. -// Calls to Wait and Next are still allowed until all open tickets are redeemed. -func (tq *TicketQueue[REQ, RES]) Close() { - close(tq.in) -} diff --git a/pkg/foundation/ticketqueue/ticketqueue_bench_test.go b/pkg/foundation/ticketqueue/ticketqueue_bench_test.go deleted file mode 100644 index 844136c15..000000000 --- a/pkg/foundation/ticketqueue/ticketqueue_bench_test.go +++ /dev/null @@ -1,60 +0,0 @@ -// Copyright © 2022 Meroxa, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package ticketqueue - -import ( - "context" - "fmt" - "testing" - - "github.com/gammazero/deque" -) - -func BenchmarkNewTicketQueue(b *testing.B) { - for i := 0; i < b.N; i++ { - tq := NewTicketQueue[int, int]() - defer tq.Close() - } - b.StopTimer() // don't measure Close -} - -func BenchmarkTicketQueueTake(b *testing.B) { - for _, N := range []int{1, 2, 8, 64, 128} { - b.Run(fmt.Sprintf("TicketQueue-%d", N), func(b *testing.B) { - ctx, cancel := context.WithCancel(context.Background()) - tq := NewTicketQueue[int, int]() - go func() { - for { - _, _, err := tq.Next(ctx) - if err == context.Canceled { - return - } - } - }() - - for i := 0; i < b.N; i++ { - tickets := deque.Deque[Ticket[int, int]]{} - for j := 0; j < N; j++ { - ticket := tq.Take() - tickets.PushBack(ticket) - } - for tickets.Len() > 0 { - tq.Wait(ctx, tickets.PopFront()) - } - } - cancel() - }) - } -} diff --git a/pkg/foundation/ticketqueue/ticketqueue_test.go b/pkg/foundation/ticketqueue/ticketqueue_test.go deleted file mode 100644 index 18a5adedb..000000000 --- a/pkg/foundation/ticketqueue/ticketqueue_test.go +++ /dev/null @@ -1,266 +0,0 @@ -// Copyright © 2022 Meroxa, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -// See the License for the specific language governing permissions and -// limitations under the License. - -package ticketqueue - -import ( - "context" - "fmt" - "math/rand" - "strings" - "sync" - "testing" - "time" - - "github.com/conduitio/conduit/pkg/foundation/cerrors" - "github.com/matryer/is" -) - -func TestTicketQueue_Next_ContextCanceled(t *testing.T) { - is := is.New(t) - ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*10) - defer cancel() - - tq := NewTicketQueue[int, float64]() - defer tq.Close() - - req, res, err := tq.Next(ctx) - is.Equal(req, nil) - is.Equal(res, nil) - is.Equal(err, context.DeadlineExceeded) -} - -func TestTicketQueue_Next_Closed(t *testing.T) { - is := is.New(t) - ctx := context.Background() - - tq := NewTicketQueue[int, float64]() - tq.Close() // close ticket queue - - req, res, err := tq.Next(ctx) - is.Equal(req, nil) - is.Equal(res, nil) - is.True(err != nil) -} - -func TestTicketQueue_Take_Closed(t *testing.T) { - is := is.New(t) - - tq := NewTicketQueue[int, float64]() - tq.Close() // close ticket queue, taking a ticket after this is not permitted - - defer func() { - is.True(recover() != nil) // expected Take to panic - }() - - tq.Take() -} - -func TestTicketQueue_Wait_ContextCanceled(t *testing.T) { - is := is.New(t) - ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*10) - defer cancel() - - tq := NewTicketQueue[int, float64]() - defer tq.Close() - - ticket := tq.Take() - req, res, err := tq.Wait(ctx, ticket) - is.Equal(req, nil) - is.Equal(res, nil) - is.Equal(err, context.DeadlineExceeded) -} - -func TestTicketQueue_Wait_ReuseTicket(t *testing.T) { - is := is.New(t) - ctx := context.Background() - - tq := NewTicketQueue[int, float64]() - defer tq.Close() - - var wg sync.WaitGroup - wg.Add(1) - go func() { - defer wg.Done() - - ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*10) - defer cancel() - _, _, err := tq.Next(ctx) - is.NoErr(err) - _, _, err = tq.Next(ctx) - is.Equal(err, context.DeadlineExceeded) - }() - - ticket := tq.Take() - _, _, err := tq.Wait(ctx, ticket) - is.NoErr(err) - - _, _, err = tq.Wait(ctx, ticket) - is.True(err != nil) // expected error for ticket that was already cashed-in - wg.Wait() -} - -func TestTicketQueue_Next_NoTicketWaiting(t *testing.T) { - is := is.New(t) - ctx := context.Background() - - tq := NewTicketQueue[int, float64]() - defer tq.Close() - - tq.Take() // take ticket, but don't cash it in - - ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*10) - defer cancel() - _, _, err := tq.Next(ctx) - is.Equal(err, context.DeadlineExceeded) -} - -func TestTicketQueue_Take_Buffer(t *testing.T) { - is := is.New(t) - ctx := context.Background() - - tq := NewTicketQueue[int, float64]() - defer tq.Close() - - // TicketQueue supports an unbounded amount of tickets, keep taking tickets - // for one second and take as many tickets as possible - testDuration := time.Second - - var wg sync.WaitGroup - var numTickets int - start := time.Now() - for time.Since(start) < testDuration { - numTickets += 1 - ticket := tq.Take() - go func() { - defer wg.Done() - _, _, err := tq.Wait(ctx, ticket) - is.NoErr(err) - }() - } - wg.Add(numTickets) - t.Logf("took %d tickets in %s", numTickets, testDuration) - - for i := 0; i < numTickets; i++ { - _, _, err := tq.Next(ctx) - is.NoErr(err) - } - - wg.Wait() // wait for all ticket goroutines to finish - - // try fetching next in line, but there is none - ctx, cancel := context.WithTimeout(ctx, time.Millisecond*10) - defer cancel() - _, _, err := tq.Next(ctx) - is.Equal(err, context.DeadlineExceeded) -} - -func TestTicketQueue_HandOff(t *testing.T) { - is := is.New(t) - ctx := context.Background() - - tq := NewTicketQueue[int, float64]() - defer tq.Close() - - wantInt := 123 - wantFloat := 1.23 - - done := make(chan struct{}) - go func() { - defer close(done) - ticket := tq.Take() - req, res, err := tq.Wait(ctx, ticket) - is.NoErr(err) - req <- wantInt - gotFloat := <-res - is.Equal(wantFloat, gotFloat) - }() - - req, res, err := tq.Next(ctx) - is.NoErr(err) - - gotInt := <-req - is.Equal(wantInt, gotInt) - - res <- wantFloat - <-done -} - -func ExampleTicketQueue() { - ctx := context.Background() - - tq := NewTicketQueue[string, error]() - defer tq.Close() - - sentence := []string{ - "Each", "word", "will", "be", "sent", "to", "the", "collector", "in", - "a", "separate", "goroutine", "and", "even", "though", "they", "will", - "sleep", "for", "a", "random", "amount", "of", "time,", "all", "words", - "will", "be", "processed", "in", "the", "right", "order.", - } - - r := rand.New(rand.NewSource(time.Now().UnixMilli())) - var wg sync.WaitGroup - for _, word := range sentence { - t := tq.Take() - wg.Add(1) - go func(word string, delay time.Duration) { - defer wg.Done() - // sleep for a random amount of time to simulate work being done - time.Sleep(delay) - // try to cash in ticket - req, res, err := tq.Wait(ctx, t) - if err != nil { - panic(cerrors.Errorf("unexpected error: %w", err)) - } - req <- word // send word to collector - err = <-res // receive error back - if err != nil { - panic(cerrors.Errorf("unexpected error: %w", err)) - } - }(word, time.Millisecond*time.Duration(r.Intn(100))) - } - - // collect all tickets - var builder strings.Builder - for { - ctx, cancel := context.WithTimeout(ctx, time.Millisecond*200) - defer cancel() - - req, res, err := tq.Next(ctx) - if err != nil { - if err == context.DeadlineExceeded { - break - } - panic(cerrors.Errorf("unexpected error: %w", err)) - } - - word := <-req - _, err = builder.WriteRune(' ') - if err != nil { - res <- err - } - _, err = builder.WriteString(word) - if err != nil { - res <- err - } - close(res) - } - wg.Wait() - - fmt.Println(builder.String()) - - // Output: - // Each word will be sent to the collector in a separate goroutine and even though they will sleep for a random amount of time, all words will be processed in the right order. -} From b288e216e7a1b6154a86822ceb4cc52c84adaf54 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Tue, 21 Jun 2022 17:24:39 +0200 Subject: [PATCH 06/25] optimize semaphore for our use case --- pkg/foundation/semaphore/semaphore.go | 84 ++++++------------- .../semaphore/semaphore_bench_test.go | 46 +++++----- pkg/foundation/semaphore/semaphore_test.go | 79 +++++++---------- 3 files changed, 76 insertions(+), 133 deletions(-) diff --git a/pkg/foundation/semaphore/semaphore.go b/pkg/foundation/semaphore/semaphore.go index e1f06af5d..cb5c7c8a3 100644 --- a/pkg/foundation/semaphore/semaphore.go +++ b/pkg/foundation/semaphore/semaphore.go @@ -21,29 +21,19 @@ import ( "github.com/conduitio/conduit/pkg/foundation/cerrors" ) -// NewWeighted creates a new weighted semaphore with the given -// maximum combined weight for concurrent access. -func NewWeighted(n int64) *Weighted { - w := &Weighted{size: n} - return w -} - -// Weighted provides a way to bound concurrent access to a resource. -// The callers can request access with a given weight. -type Weighted struct { - size int64 - cur int64 +// Simple provides a way to bound concurrent access to a resource. It only +// allows one caller to gain access at a time. +type Simple struct { + waiters []waiter + front int + batch int64 + acquired bool released int mu sync.Mutex - - waiters []waiter - front int - batch int64 } type waiter struct { index int - n int64 ready chan struct{} // Closed when semaphore acquired. released bool @@ -55,16 +45,12 @@ type Ticket struct { batch int64 } -func (s *Weighted) Enqueue(n int64) Ticket { - if n > s.size { - panic("semaphore: tried to enqueue more than size of semaphore") - } - +func (s *Simple) Enqueue() Ticket { s.mu.Lock() defer s.mu.Unlock() index := len(s.waiters) - w := waiter{index: index, n: n, ready: make(chan struct{})} + w := waiter{index: index, ready: make(chan struct{})} s.waiters = append(s.waiters, w) return Ticket{ @@ -73,12 +59,10 @@ func (s *Weighted) Enqueue(n int64) Ticket { } } -// Acquire acquires the semaphore with a weight of n, blocking until resources -// are available or ctx is done. On success, returns nil. On failure, returns -// ctx.Err() and leaves the semaphore unchanged. -// -// If ctx is already done, Acquire may still succeed without blocking. -func (s *Weighted) Acquire(t Ticket) error { +// Acquire acquires the semaphore, blocking until resources are available. On +// success, returns nil. On failure, returns an error and leaves the semaphore +// unchanged. +func (s *Simple) Acquire(t Ticket) error { s.mu.Lock() if s.batch != t.batch { s.mu.Unlock() @@ -93,13 +77,9 @@ func (s *Weighted) Acquire(t Ticket) error { w.acquired = true // mark that Acquire was already called for this Ticket s.waiters[t.index] = w - if s.front == t.index && s.size-s.cur >= w.n { - s.cur += w.n + if s.front == t.index && !s.acquired { s.front++ - // If there are extra tokens left, notify other waiters. - if s.size > s.cur { - s.notifyWaiters() - } + s.acquired = true s.mu.Unlock() return nil } @@ -109,8 +89,10 @@ func (s *Weighted) Acquire(t Ticket) error { return nil } -// Release releases the semaphore with a weight of n. -func (s *Weighted) Release(t Ticket) error { +// Release releases the semaphore and notifies the next in line if any. +// If the ticket is not holding the lock on the semaphore the function returns +// an error. +func (s *Simple) Release(t Ticket) error { s.mu.Lock() defer s.mu.Unlock() @@ -125,43 +107,29 @@ func (s *Weighted) Release(t Ticket) error { return cerrors.New("semaphore: ticket already released") } - s.cur -= w.n w.released = true s.waiters[t.index] = w + s.acquired = false s.released++ - s.notifyWaiters() + s.notifyWaiter() if s.released == len(s.waiters) { s.increaseBatch() } return nil } -func (s *Weighted) notifyWaiters() { - for len(s.waiters) > s.front { +func (s *Simple) notifyWaiter() { + if len(s.waiters) > s.front { w := s.waiters[s.front] - if s.size-s.cur < w.n { - // Not enough tokens for the next waiter. We could keep going (to try to - // find a waiter with a smaller request), but under load that could cause - // starvation for large requests; instead, we leave all remaining waiters - // blocked. - // - // Consider a semaphore used as a read-write lock, with N tokens, N - // readers, and one writer. Each reader can Acquire(1) to obtain a read - // lock. The writer can Acquire(N) to obtain a write lock, excluding all - // of the readers. If we allow the readers to jump ahead in the queue, - // the writer will starve — there is always one token available for every - // reader. - break - } - - s.cur += w.n + s.acquired = true s.front++ close(w.ready) } } -func (s *Weighted) increaseBatch() { +func (s *Simple) increaseBatch() { s.waiters = s.waiters[:0] s.batch += 1 s.front = 0 + s.released = 0 } diff --git a/pkg/foundation/semaphore/semaphore_bench_test.go b/pkg/foundation/semaphore/semaphore_bench_test.go index 73dcfb686..8a72c4826 100644 --- a/pkg/foundation/semaphore/semaphore_bench_test.go +++ b/pkg/foundation/semaphore/semaphore_bench_test.go @@ -22,22 +22,14 @@ import ( "github.com/conduitio/conduit/pkg/foundation/semaphore" ) -// weighted is an interface matching a subset of *Weighted. It allows -// alternate implementations for testing and benchmarking. -type weighted interface { - Enqueue(int64) semaphore.Ticket - Acquire(semaphore.Ticket) error - Release(semaphore.Ticket) error -} - // acquireN calls Acquire(size) on sem N times and then calls Release(size) N times. -func acquireN(b *testing.B, sem weighted, size int64, N int) { +func acquireN(b *testing.B, sem *semaphore.Simple, N int) { b.ResetTimer() tickets := list.New() for i := 0; i < b.N; i++ { tickets.Init() for j := 0; j < N; j++ { - _ = sem.Enqueue(size) + _ = sem.Enqueue() } } } @@ -46,7 +38,7 @@ func BenchmarkNewSem(b *testing.B) { for _, cap := range []int64{1, 128} { b.Run(fmt.Sprintf("Weighted-%d", cap), func(b *testing.B) { for i := 0; i < b.N; i++ { - _ = semaphore.NewWeighted(cap) + _ = &semaphore.Simple{} } }) } @@ -54,28 +46,28 @@ func BenchmarkNewSem(b *testing.B) { func BenchmarkAcquireSem(b *testing.B) { for _, c := range []struct { - cap, size int64 - N int + cap int64 + N int }{ - {1, 1, 1}, - {2, 1, 1}, - {16, 1, 1}, - {128, 1, 1}, - {2, 2, 1}, - {16, 2, 8}, - {128, 2, 64}, - {2, 1, 2}, - {16, 8, 2}, - {128, 64, 2}, + {1, 1}, + {2, 1}, + {16, 1}, + {128, 1}, + {2, 1}, + {16, 8}, + {128, 64}, + {2, 2}, + {16, 2}, + {128, 2}, } { for _, w := range []struct { name string - w weighted + w *semaphore.Simple }{ - {"Weighted", semaphore.NewWeighted(c.cap)}, + {"Simple", &semaphore.Simple{}}, } { - b.Run(fmt.Sprintf("%s-acquire-%d-%d-%d", w.name, c.cap, c.size, c.N), func(b *testing.B) { - acquireN(b, w.w, c.size, c.N) + b.Run(fmt.Sprintf("%s-acquire-%d-%d", w.name, c.cap, c.N), func(b *testing.B) { + acquireN(b, w.w, c.N) }) } } diff --git a/pkg/foundation/semaphore/semaphore_test.go b/pkg/foundation/semaphore/semaphore_test.go index 9845f58fc..0e72eb7da 100644 --- a/pkg/foundation/semaphore/semaphore_test.go +++ b/pkg/foundation/semaphore/semaphore_test.go @@ -26,9 +26,9 @@ import ( const maxSleep = 1 * time.Millisecond -func HammerWeighted(sem *semaphore.Weighted, n int64, loops int) { +func HammerSimple(sem *semaphore.Simple, loops int) { for i := 0; i < loops; i++ { - tkn := sem.Enqueue(n) + tkn := sem.Enqueue() err := sem.Acquire(tkn) if err != nil { panic(err) @@ -38,41 +38,40 @@ func HammerWeighted(sem *semaphore.Weighted, n int64, loops int) { } } -func TestWeighted(t *testing.T) { +func TestSimple(t *testing.T) { t.Parallel() n := runtime.GOMAXPROCS(0) - loops := 10000 / n - sem := semaphore.NewWeighted(int64(n)) + loops := 5000 / n + sem := &semaphore.Simple{} var wg sync.WaitGroup wg.Add(n) for i := 0; i < n; i++ { - i := i go func() { defer wg.Done() - HammerWeighted(sem, int64(i), loops) + HammerSimple(sem, loops) }() } wg.Wait() } -func TestWeightedReleaseUnacquired(t *testing.T) { +func TestSimpleReleaseUnacquired(t *testing.T) { t.Parallel() - w := semaphore.NewWeighted(1) - tkn := w.Enqueue(1) + w := &semaphore.Simple{} + tkn := w.Enqueue() err := w.Release(tkn) if err == nil { t.Errorf("release of an unacquired ticket did not return an error") } } -func TestWeightedReleaseTwice(t *testing.T) { +func TestSimpleReleaseTwice(t *testing.T) { t.Parallel() - w := semaphore.NewWeighted(1) - tkn := w.Enqueue(1) + w := &semaphore.Simple{} + tkn := w.Enqueue() w.Acquire(tkn) err := w.Release(tkn) if err != nil { @@ -85,11 +84,11 @@ func TestWeightedReleaseTwice(t *testing.T) { } } -func TestWeightedAcquireTwice(t *testing.T) { +func TestSimpleAcquireTwice(t *testing.T) { t.Parallel() - w := semaphore.NewWeighted(1) - tkn := w.Enqueue(1) + w := &semaphore.Simple{} + tkn := w.Enqueue() err := w.Acquire(tkn) if err != nil { t.Errorf("acquire of a ticket errored out: %v", err) @@ -101,51 +100,35 @@ func TestWeightedAcquireTwice(t *testing.T) { } } -func TestWeightedPanicEnqueueTooBig(t *testing.T) { +func TestSimpleAcquire(t *testing.T) { t.Parallel() - defer func() { - if recover() == nil { - t.Fatal("enqueue of size bigger than weighted semaphore did not panic") - } - }() - const n = 5 - sem := semaphore.NewWeighted(n) - sem.Enqueue(n + 1) -} + sem := &semaphore.Simple{} -func TestWeightedAcquire(t *testing.T) { - t.Parallel() - - sem := semaphore.NewWeighted(2) - - tkn1 := sem.Enqueue(1) + tkn1 := sem.Enqueue() sem.Acquire(tkn1) - tkn2 := sem.Enqueue(1) - sem.Acquire(tkn2) - - tkn3done := make(chan struct{}) + tkn2done := make(chan struct{}) go func() { - defer close(tkn3done) - tkn3 := sem.Enqueue(1) - sem.Acquire(tkn3) + defer close(tkn2done) + tkn2 := sem.Enqueue() + sem.Acquire(tkn2) }() select { - case <-tkn3done: - t.Errorf("tkn3done closed prematurely") + case <-tkn2done: + t.Errorf("tkn2done closed prematurely") case <-time.After(time.Millisecond * 10): - // tkn3 Acquire is blocking as expected + // tkn2 Acquire is blocking as expected } sem.Release(tkn1) select { - case <-tkn3done: + case <-tkn2done: // tkn3 successfully acquired the semaphore case <-time.After(time.Millisecond * 10): - t.Errorf("tkn3done didn't get closed") + t.Errorf("tkn2done didn't get closed") } } @@ -155,13 +138,13 @@ func TestLargeAcquireDoesntStarve(t *testing.T) { t.Parallel() n := int64(runtime.GOMAXPROCS(0)) - sem := semaphore.NewWeighted(n) + sem := &semaphore.Simple{} running := true var wg sync.WaitGroup wg.Add(int(n)) for i := n; i > 0; i-- { - tkn := sem.Enqueue(1) + tkn := sem.Enqueue() sem.Acquire(tkn) go func() { defer func() { @@ -171,13 +154,13 @@ func TestLargeAcquireDoesntStarve(t *testing.T) { for running { time.Sleep(1 * time.Millisecond) sem.Release(tkn) - tkn = sem.Enqueue(1) + tkn = sem.Enqueue() sem.Acquire(tkn) } }() } - tkn := sem.Enqueue(n) + tkn := sem.Enqueue() sem.Acquire(tkn) running = false sem.Release(tkn) From 0471fbe2500123ffb06ca388eb9267a47d37fef8 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Tue, 21 Jun 2022 18:45:20 +0200 Subject: [PATCH 07/25] fix linter warnings, better benchmarks --- pkg/foundation/semaphore/semaphore.go | 3 +- .../semaphore/semaphore_bench_test.go | 69 +++++++++---------- pkg/foundation/semaphore/semaphore_test.go | 61 ++++++++++++---- 3 files changed, 81 insertions(+), 52 deletions(-) diff --git a/pkg/foundation/semaphore/semaphore.go b/pkg/foundation/semaphore/semaphore.go index cb5c7c8a3..86f2279ae 100644 --- a/pkg/foundation/semaphore/semaphore.go +++ b/pkg/foundation/semaphore/semaphore.go @@ -12,7 +12,6 @@ // See the License for the specific language governing permissions and // limitations under the License. -// Package semaphore provides a weighted semaphore implementation. package semaphore import ( @@ -129,7 +128,7 @@ func (s *Simple) notifyWaiter() { func (s *Simple) increaseBatch() { s.waiters = s.waiters[:0] - s.batch += 1 + s.batch++ s.front = 0 s.released = 0 } diff --git a/pkg/foundation/semaphore/semaphore_bench_test.go b/pkg/foundation/semaphore/semaphore_bench_test.go index 8a72c4826..8858428ed 100644 --- a/pkg/foundation/semaphore/semaphore_bench_test.go +++ b/pkg/foundation/semaphore/semaphore_bench_test.go @@ -22,53 +22,46 @@ import ( "github.com/conduitio/conduit/pkg/foundation/semaphore" ) -// acquireN calls Acquire(size) on sem N times and then calls Release(size) N times. -func acquireN(b *testing.B, sem *semaphore.Simple, N int) { - b.ResetTimer() - tickets := list.New() +func BenchmarkNewSem(b *testing.B) { for i := 0; i < b.N; i++ { - tickets.Init() - for j := 0; j < N; j++ { - _ = sem.Enqueue() - } + _ = &semaphore.Simple{} } } -func BenchmarkNewSem(b *testing.B) { - for _, cap := range []int64{1, 128} { - b.Run(fmt.Sprintf("Weighted-%d", cap), func(b *testing.B) { +func BenchmarkAcquireSem(b *testing.B) { + for _, N := range []int{1, 2, 8, 64, 128} { + b.Run(fmt.Sprintf("acquire-%d", N), func(b *testing.B) { + b.ResetTimer() + sem := &semaphore.Simple{} for i := 0; i < b.N; i++ { - _ = &semaphore.Simple{} + for j := 0; j < N; j++ { + t := sem.Enqueue() + _ = sem.Acquire(t) + _ = sem.Release(t) + } } }) } } -func BenchmarkAcquireSem(b *testing.B) { - for _, c := range []struct { - cap int64 - N int - }{ - {1, 1}, - {2, 1}, - {16, 1}, - {128, 1}, - {2, 1}, - {16, 8}, - {128, 64}, - {2, 2}, - {16, 2}, - {128, 2}, - } { - for _, w := range []struct { - name string - w *semaphore.Simple - }{ - {"Simple", &semaphore.Simple{}}, - } { - b.Run(fmt.Sprintf("%s-acquire-%d-%d", w.name, c.cap, c.N), func(b *testing.B) { - acquireN(b, w.w, c.N) - }) - } +func BenchmarkEnqueueReleaseSem(b *testing.B) { + for _, N := range []int{1, 2, 8, 64, 128} { + b.Run(fmt.Sprintf("enqueue/release-%d", N), func(b *testing.B) { + b.ResetTimer() + sem := &semaphore.Simple{} + tickets := list.New() + for i := 0; i < b.N; i++ { + tickets.Init() + for j := 0; j < N; j++ { + t := sem.Enqueue() + tickets.PushBack(t) + } + ticket := tickets.Front() + for ticket != nil { + _ = sem.Release(ticket.Value.(semaphore.Ticket)) + ticket = ticket.Next() + } + } + }) } } diff --git a/pkg/foundation/semaphore/semaphore_test.go b/pkg/foundation/semaphore/semaphore_test.go index 0e72eb7da..021887c6c 100644 --- a/pkg/foundation/semaphore/semaphore_test.go +++ b/pkg/foundation/semaphore/semaphore_test.go @@ -33,8 +33,12 @@ func HammerSimple(sem *semaphore.Simple, loops int) { if err != nil { panic(err) } + //nolint:gosec // math/rand is good enough for a test time.Sleep(time.Duration(rand.Int63n(int64(maxSleep/time.Nanosecond))) * time.Nanosecond) - sem.Release(tkn) + err = sem.Release(tkn) + if err != nil { + panic(err) + } } } @@ -72,8 +76,11 @@ func TestSimpleReleaseTwice(t *testing.T) { w := &semaphore.Simple{} tkn := w.Enqueue() - w.Acquire(tkn) - err := w.Release(tkn) + err := w.Acquire(tkn) + if err != nil { + t.Errorf("unexpected error: %v", err) + } + err = w.Release(tkn) if err != nil { t.Errorf("release of an acquired ticket errored out: %v", err) } @@ -106,13 +113,19 @@ func TestSimpleAcquire(t *testing.T) { sem := &semaphore.Simple{} tkn1 := sem.Enqueue() - sem.Acquire(tkn1) + err := sem.Acquire(tkn1) + if err != nil { + t.Errorf("unexpected error: %v", err) + } tkn2done := make(chan struct{}) go func() { defer close(tkn2done) tkn2 := sem.Enqueue() - sem.Acquire(tkn2) + err := sem.Acquire(tkn2) + if err != nil { + t.Errorf("unexpected error: %v", err) + } }() select { @@ -122,7 +135,10 @@ func TestSimpleAcquire(t *testing.T) { // tkn2 Acquire is blocking as expected } - sem.Release(tkn1) + err = sem.Release(tkn1) + if err != nil { + t.Errorf("unexpected error: %v", err) + } select { case <-tkn2done: @@ -145,24 +161,45 @@ func TestLargeAcquireDoesntStarve(t *testing.T) { wg.Add(int(n)) for i := n; i > 0; i-- { tkn := sem.Enqueue() - sem.Acquire(tkn) + err := sem.Acquire(tkn) + if err != nil { + t.Errorf("unexpected error: %v", err) + } + go func() { defer func() { - sem.Release(tkn) + err := sem.Release(tkn) + if err != nil { + t.Errorf("unexpected error: %v", err) + } wg.Done() }() for running { time.Sleep(1 * time.Millisecond) - sem.Release(tkn) + err := sem.Release(tkn) + if err != nil { + t.Errorf("unexpected error: %v", err) + } tkn = sem.Enqueue() - sem.Acquire(tkn) + err = sem.Acquire(tkn) + if err != nil { + t.Errorf("unexpected error: %v", err) + } } }() } tkn := sem.Enqueue() - sem.Acquire(tkn) + err := sem.Acquire(tkn) + if err != nil { + t.Errorf("unexpected error: %v", err) + } + running = false - sem.Release(tkn) + err = sem.Release(tkn) + if err != nil { + t.Errorf("unexpected error: %v", err) + } + wg.Wait() } From 83f818429749ec18459980335fd55194546e10a9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Tue, 21 Jun 2022 18:56:03 +0200 Subject: [PATCH 08/25] better docs --- pkg/foundation/semaphore/semaphore.go | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/pkg/foundation/semaphore/semaphore.go b/pkg/foundation/semaphore/semaphore.go index 86f2279ae..9a267eba4 100644 --- a/pkg/foundation/semaphore/semaphore.go +++ b/pkg/foundation/semaphore/semaphore.go @@ -39,11 +39,16 @@ type waiter struct { acquired bool } +// Ticket reserves a place in the queue and can be used to acquire access to a +// resource. type Ticket struct { index int batch int64 } +// Enqueue reserves the next place in the queue and returns a Ticket used to +// acquire access to the resource when it's the callers turn. The Ticket has to +// be supplied to Release before discarding. func (s *Simple) Enqueue() Ticket { s.mu.Lock() defer s.mu.Unlock() @@ -89,8 +94,8 @@ func (s *Simple) Acquire(t Ticket) error { } // Release releases the semaphore and notifies the next in line if any. -// If the ticket is not holding the lock on the semaphore the function returns -// an error. +// If the ticket was already released the function returns an error. After the +// ticket is released it should be discarded. func (s *Simple) Release(t Ticket) error { s.mu.Lock() defer s.mu.Unlock() From 83c97e0120d36225b2df7eaa6540fa70365156ae Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Tue, 21 Jun 2022 18:58:30 +0200 Subject: [PATCH 09/25] go mod tidy --- go.mod | 1 - go.sum | 2 -- 2 files changed, 3 deletions(-) diff --git a/go.mod b/go.mod index 44b4bb415..81cf5535c 100644 --- a/go.mod +++ b/go.mod @@ -15,7 +15,6 @@ require ( github.com/conduitio/conduit-connector-sdk v0.2.0 github.com/dgraph-io/badger/v3 v3.2103.2 github.com/dop251/goja v0.0.0-20210225094849-f3cfc97811c0 - github.com/gammazero/deque v0.2.0 github.com/golang/mock v1.6.0 github.com/google/go-cmp v0.5.8 github.com/google/uuid v1.3.0 diff --git a/go.sum b/go.sum index 7d881de12..e4a9d71b3 100644 --- a/go.sum +++ b/go.sum @@ -205,8 +205,6 @@ github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMo github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ= github.com/fsnotify/fsnotify v1.5.1 h1:mZcQUHVQUQWoPXXtuf9yuEXKudkV2sx1E06UadKWpgI= github.com/fsnotify/fsnotify v1.5.1/go.mod h1:T3375wBYaZdLLcVNkcVbzGHY7f1l/uK5T5Ai1i3InKU= -github.com/gammazero/deque v0.2.0 h1:SkieyNB4bg2/uZZLxvya0Pq6diUlwx7m2TeT7GAIWaA= -github.com/gammazero/deque v0.2.0/go.mod h1:LFroj8x4cMYCukHJDbxFCkT+r9AndaJnFMuZDV34tuU= github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04= github.com/go-fonts/dejavu v0.1.0/go.mod h1:4Wt4I4OU2Nq9asgDCteaAaWZOV24E+0/Pwo0gppep4g= github.com/go-fonts/latin-modern v0.2.0/go.mod h1:rQVLdDMK+mK1xscDwsqM5J8U2jrRa3T0ecnM9pNujks= From c0ded08951cb2019997cd7511d54006b5bfa704e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Fri, 10 Jun 2022 19:02:08 +0200 Subject: [PATCH 10/25] rename AckerNode to DestinationAckerNode --- pkg/pipeline/lifecycle.go | 4 +-- pkg/pipeline/stream/destination.go | 2 +- .../stream/{acker.go => destination_acker.go} | 29 ++++++++++--------- ...cker_test.go => destination_acker_test.go} | 4 +-- pkg/pipeline/stream/stream_test.go | 6 ++-- 5 files changed, 23 insertions(+), 22 deletions(-) rename pkg/pipeline/stream/{acker.go => destination_acker.go} (90%) rename pkg/pipeline/stream/{acker_test.go => destination_acker_test.go} (97%) diff --git a/pkg/pipeline/lifecycle.go b/pkg/pipeline/lifecycle.go index e7b8789c0..954166516 100644 --- a/pkg/pipeline/lifecycle.go +++ b/pkg/pipeline/lifecycle.go @@ -290,8 +290,8 @@ func (s *Service) buildMetricsNode( func (s *Service) buildAckerNode( dest connector.Destination, -) *stream.AckerNode { - return &stream.AckerNode{ +) *stream.DestinationAckerNode { + return &stream.DestinationAckerNode{ Name: dest.ID() + "-acker", Destination: dest, } diff --git a/pkg/pipeline/stream/destination.go b/pkg/pipeline/stream/destination.go index 467b97f7e..0c75f2179 100644 --- a/pkg/pipeline/stream/destination.go +++ b/pkg/pipeline/stream/destination.go @@ -30,7 +30,7 @@ type DestinationNode struct { Destination connector.Destination ConnectorTimer metrics.Timer // AckerNode is responsible for handling acks - AckerNode *AckerNode + AckerNode *DestinationAckerNode base subNodeBase logger log.CtxLogger diff --git a/pkg/pipeline/stream/acker.go b/pkg/pipeline/stream/destination_acker.go similarity index 90% rename from pkg/pipeline/stream/acker.go rename to pkg/pipeline/stream/destination_acker.go index 3f819605b..f2fa04861 100644 --- a/pkg/pipeline/stream/acker.go +++ b/pkg/pipeline/stream/destination_acker.go @@ -27,9 +27,9 @@ import ( "github.com/conduitio/conduit/pkg/record" ) -// AckerNode is responsible for handling acknowledgments received from the -// destination and forwarding them to the correct message. -type AckerNode struct { +// DestinationAckerNode is responsible for handling acknowledgments received +// from the destination and forwarding them to the correct message. +type DestinationAckerNode struct { Name string Destination connector.Destination @@ -49,8 +49,8 @@ type AckerNode struct { stopOnce sync.Once } -// init initializes AckerNode internal fields. -func (n *AckerNode) init() { +// init initializes DestinationAckerNode internal fields. +func (n *DestinationAckerNode) init() { n.initOnce.Do(func() { n.cache = &positionMessageMap{} n.start = make(chan struct{}) @@ -58,13 +58,13 @@ func (n *AckerNode) init() { }) } -func (n *AckerNode) ID() string { +func (n *DestinationAckerNode) ID() string { return n.Name } // Run continuously fetches acks from the destination and forwards them to the // correct message by calling Ack or Nack on that message. -func (n *AckerNode) Run(ctx context.Context) (err error) { +func (n *DestinationAckerNode) Run(ctx context.Context) (err error) { n.logger.Trace(ctx).Msg("starting acker node") defer n.logger.Trace(ctx).Msg("acker node stopped") @@ -122,7 +122,7 @@ func (n *AckerNode) Run(ctx context.Context) (err error) { // teardown will drop all messages still in the cache and return an error in // case there were still unprocessed messages in the cache. -func (n *AckerNode) teardown() error { +func (n *DestinationAckerNode) teardown() error { var dropped int n.cache.Range(func(pos record.Position, msg *Message) bool { msg.Drop() @@ -138,7 +138,7 @@ func (n *AckerNode) teardown() error { // handleAck either acks or nacks the message, depending on the supplied error. // If the nacking or acking fails, the message is dropped and the error is // returned. -func (n *AckerNode) handleAck(msg *Message, err error) error { +func (n *DestinationAckerNode) handleAck(msg *Message, err error) error { switch { case err != nil: n.logger.Trace(msg.Ctx).Err(err).Msg("nacking message") @@ -160,7 +160,7 @@ func (n *AckerNode) handleAck(msg *Message, err error) error { // ExpectAck makes the handler aware of the message and signals to it that an // ack for this message might be received at some point. -func (n *AckerNode) ExpectAck(msg *Message) error { +func (n *DestinationAckerNode) ExpectAck(msg *Message) error { // happens only once to signal Run that the destination is ready to be used. n.startOnce.Do(func() { n.init() @@ -185,7 +185,7 @@ func (n *AckerNode) ExpectAck(msg *Message) error { // ForgetAndDrop signals the handler that an ack for this message won't be // received, and it should remove it from its cache. In case an ack for this // message wasn't yet received it drops the message, otherwise it does nothing. -func (n *AckerNode) ForgetAndDrop(msg *Message) { +func (n *DestinationAckerNode) ForgetAndDrop(msg *Message) { _, ok := n.cache.LoadAndDelete(msg.Record.Position) if !ok { // message wasn't found in the cache, looks like the message was already @@ -197,8 +197,9 @@ func (n *AckerNode) ForgetAndDrop(msg *Message) { // Wait can be used to wait for the count of outstanding acks to drop to 0 or // the context gets canceled. Wait is expected to be the last function called on -// AckerNode, after Wait returns AckerNode will soon stop running. -func (n *AckerNode) Wait(ctx context.Context) { +// DestinationAckerNode, after Wait returns DestinationAckerNode will soon stop +// running. +func (n *DestinationAckerNode) Wait(ctx context.Context) { // happens only once to signal that the destination is stopping n.stopOnce.Do(func() { n.init() @@ -227,7 +228,7 @@ func (n *AckerNode) Wait(ctx context.Context) { } // SetLogger sets the logger. -func (n *AckerNode) SetLogger(logger log.CtxLogger) { +func (n *DestinationAckerNode) SetLogger(logger log.CtxLogger) { n.logger = logger } diff --git a/pkg/pipeline/stream/acker_test.go b/pkg/pipeline/stream/destination_acker_test.go similarity index 97% rename from pkg/pipeline/stream/acker_test.go rename to pkg/pipeline/stream/destination_acker_test.go index cc33e04c9..0a2a447af 100644 --- a/pkg/pipeline/stream/acker_test.go +++ b/pkg/pipeline/stream/destination_acker_test.go @@ -32,7 +32,7 @@ func TestAckerNode_Run_StopAfterWait(t *testing.T) { ctrl := gomock.NewController(t) dest := mock.NewDestination(ctrl) - node := &AckerNode{ + node := &DestinationAckerNode{ Name: "acker-node", Destination: dest, } @@ -69,7 +69,7 @@ func TestAckerNode_Run_StopAfterExpectAck(t *testing.T) { ctrl := gomock.NewController(t) dest := mock.NewDestination(ctrl) - node := &AckerNode{ + node := &DestinationAckerNode{ Name: "acker-node", Destination: dest, } diff --git a/pkg/pipeline/stream/stream_test.go b/pkg/pipeline/stream/stream_test.go index 8e0a1d909..57b1b2cb7 100644 --- a/pkg/pipeline/stream/stream_test.go +++ b/pkg/pipeline/stream/stream_test.go @@ -52,7 +52,7 @@ func Example_simpleStream() { Destination: printerDestination(ctrl, logger, "printer"), ConnectorTimer: noop.Timer{}, } - node3 := &stream.AckerNode{ + node3 := &stream.DestinationAckerNode{ Name: "printer-acker", Destination: node2.Destination, } @@ -144,12 +144,12 @@ func Example_complexStream() { Destination: printerDestination(ctrl, logger, "printer2"), ConnectorTimer: noop.Timer{}, } - node8 := &stream.AckerNode{ + node8 := &stream.DestinationAckerNode{ Name: "printer1-acker", Destination: node6.Destination, } node6.AckerNode = node8 - node9 := &stream.AckerNode{ + node9 := &stream.DestinationAckerNode{ Name: "printer2-acker", Destination: node7.Destination, } From c66c2739a1803373b8a179a88d84c2b027ac55e7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Wed, 22 Jun 2022 19:29:25 +0200 Subject: [PATCH 11/25] remove message status change middleware to ensure all message handlers are called --- pkg/pipeline/lifecycle.go | 8 +-- pkg/pipeline/stream/fanout.go | 27 +++------ pkg/pipeline/stream/message.go | 71 +++++++---------------- pkg/pipeline/stream/message_test.go | 83 +++++++++++++-------------- pkg/pipeline/stream/metrics.go | 4 +- pkg/pipeline/stream/processor_test.go | 8 +-- pkg/pipeline/stream/source.go | 12 ++-- 7 files changed, 83 insertions(+), 130 deletions(-) diff --git a/pkg/pipeline/lifecycle.go b/pkg/pipeline/lifecycle.go index 954166516..9b867ab51 100644 --- a/pkg/pipeline/lifecycle.go +++ b/pkg/pipeline/lifecycle.go @@ -288,7 +288,7 @@ func (s *Service) buildMetricsNode( } } -func (s *Service) buildAckerNode( +func (s *Service) buildDestinationAckerNode( dest connector.Destination, ) *stream.DestinationAckerNode { return &stream.DestinationAckerNode{ @@ -316,7 +316,7 @@ func (s *Service) buildDestinationNodes( continue // skip any connector that's not a destination } - ackerNode := s.buildAckerNode(instance.(connector.Destination)) + ackerNode := s.buildDestinationAckerNode(instance.(connector.Destination)) destinationNode := stream.DestinationNode{ Name: instance.ID(), Destination: instance.(connector.Destination), @@ -358,7 +358,7 @@ func (s *Service) runPipeline(ctx context.Context, pl *Instance) error { // If any of the nodes stops, the nodesTomb will be put into a dying state // and ctx will be cancelled. // This way, the other nodes will be notified that they need to stop too. - //nolint: staticcheck // nil used to use the default (parent provided via WithContext) + // nolint: staticcheck // nil used to use the default (parent provided via WithContext) ctx := nodesTomb.Context(nil) s.logger.Trace(ctx).Str(log.NodeIDField, node.ID()).Msg("running node") defer func() { @@ -406,7 +406,7 @@ func (s *Service) runPipeline(ctx context.Context, pl *Instance) error { // before declaring the pipeline as stopped. pl.t = &tomb.Tomb{} pl.t.Go(func() error { - //nolint: staticcheck // nil used to use the default (parent provided via WithContext) + // nolint: staticcheck // nil used to use the default (parent provided via WithContext) ctx := pl.t.Context(nil) err := nodesTomb.Wait() diff --git a/pkg/pipeline/stream/fanout.go b/pkg/pipeline/stream/fanout.go index 6e71aed26..c6c2aa9ec 100644 --- a/pkg/pipeline/stream/fanout.go +++ b/pkg/pipeline/stream/fanout.go @@ -146,38 +146,27 @@ func (n *FanoutNode) Run(ctx context.Context) error { // wrapAckHandler modifies the ack handler, so it's called with the original // message received by FanoutNode instead of the new message created by // FanoutNode. -func (n *FanoutNode) wrapAckHandler(origMsg *Message, f AckHandler) AckMiddleware { - return func(newMsg *Message, next AckHandler) error { - err := f(origMsg) - if err != nil { - return err - } - // next handler is called again with new message - return next(newMsg) +func (n *FanoutNode) wrapAckHandler(origMsg *Message, f AckHandler) AckHandler { + return func(_ *Message) error { + return f(origMsg) } } // wrapNackHandler modifies the nack handler, so it's called with the original // message received by FanoutNode instead of the new message created by // FanoutNode. -func (n *FanoutNode) wrapNackHandler(origMsg *Message, f NackHandler) NackMiddleware { - return func(newMsg *Message, reason error, next NackHandler) error { - err := f(origMsg, reason) - if err != nil { - return err - } - // next handler is called again with new message - return next(newMsg, err) +func (n *FanoutNode) wrapNackHandler(origMsg *Message, f NackHandler) NackHandler { + return func(_ *Message, reason error) error { + return f(origMsg, reason) } } // wrapDropHandler modifies the drop handler, so it's called with the original // message received by FanoutNode instead of the new message created by // FanoutNode. -func (n *FanoutNode) wrapDropHandler(origMsg *Message, f DropHandler) DropMiddleware { - return func(newMsg *Message, reason error, next DropHandler) { +func (n *FanoutNode) wrapDropHandler(origMsg *Message, f DropHandler) DropHandler { + return func(_ *Message, reason error) { f(origMsg, reason) - next(newMsg, reason) } } diff --git a/pkg/pipeline/stream/message.go b/pkg/pipeline/stream/message.go index f8d833297..5a2c761e3 100644 --- a/pkg/pipeline/stream/message.go +++ b/pkg/pipeline/stream/message.go @@ -22,6 +22,7 @@ import ( "sync" "github.com/conduitio/conduit/pkg/foundation/cerrors" + "github.com/conduitio/conduit/pkg/foundation/multierror" "github.com/conduitio/conduit/pkg/record" ) @@ -84,45 +85,21 @@ type ( // a nack or drop. StatusChangeHandler func(*Message, StatusChange) error - // StatusChangeMiddleware can be registered on a message and will be executed in - // case of a status change (see StatusChangeHandler). Middlewares are called in - // the reverse order of how they were registered. - // The middleware has two options when processing a message status change: - // - If it successfully processed the status change it should call the next - // handler and return its error. The handler may inspect the error and act - // accordingly, but it must return that error (or another error that - // contains it). It must not return an error if the next handler was called - // and it returned nil. - // - If it failed to process the status change successfully it must not call - // the next handler but instead return an error right away. - // Applying these rules means each middleware can be sure that all middlewares - // before it processed the status change successfully. - StatusChangeMiddleware func(*Message, StatusChange, StatusChangeHandler) error - // AckHandler is a variation of the StatusChangeHandler that is only called // when a message is acked. For more info see StatusChangeHandler. AckHandler func(*Message) error - // AckMiddleware is a variation of the StatusChangeMiddleware that is only - // called when a message is acked. For more info see StatusChangeMiddleware. - AckMiddleware func(*Message, AckHandler) error // NackHandler is a variation of the StatusChangeHandler that is only called // when a message is nacked. For more info see StatusChangeHandler. NackHandler func(*Message, error) error - // NackMiddleware is a variation of the StatusChangeMiddleware that is only - // called when a message is nacked. For more info see StatusChangeMiddleware. - NackMiddleware func(*Message, error, NackHandler) error // DropHandler is a variation of the StatusChangeHandler that is only called // when a message is dropped. For more info see StatusChangeHandler. DropHandler func(*Message, error) - // DropMiddleware is a variation of the StatusChangeMiddleware that is only - // called when a message is dropped. For more info see StatusChangeMiddleware. - DropMiddleware func(*Message, error, DropHandler) ) -// StatusChange is passed to StatusChangeMiddleware and StatusChangeHandler when -// the status of a message changes. +// StatusChange is passed to StatusChangeHandler when the status of a message +// changes. type StatusChange struct { Old MessageStatus New MessageStatus @@ -150,9 +127,9 @@ func (m *Message) ID() string { // RegisterStatusHandler is used to register a function that will be called on // any status change of the message. This function can only be called if the -// message status is open, otherwise it panics. Middlewares are called in the +// message status is open, otherwise it panics. Handlers are called in the // reverse order of how they were registered. -func (m *Message) RegisterStatusHandler(mw StatusChangeMiddleware) { +func (m *Message) RegisterStatusHandler(mw StatusChangeHandler) { m.init() m.handlerGuard.Lock() defer m.handlerGuard.Unlock() @@ -163,35 +140,34 @@ func (m *Message) RegisterStatusHandler(mw StatusChangeMiddleware) { next := m.handler m.handler = func(msg *Message, change StatusChange) error { - return mw(msg, change, next) + // all handlers are called and errors collected + err1 := mw(msg, change) + err2 := next(msg, change) + return multierror.Append(err1, err2) } } // RegisterAckHandler is used to register a function that will be called when // the message is acked. This function can only be called if the message status // is open, otherwise it panics. -func (m *Message) RegisterAckHandler(mw AckMiddleware) { - m.RegisterStatusHandler(func(msg *Message, change StatusChange, next StatusChangeHandler) error { +func (m *Message) RegisterAckHandler(mw AckHandler) { + m.RegisterStatusHandler(func(msg *Message, change StatusChange) error { if change.New != MessageStatusAcked { - return next(msg, change) + return nil // skip } - return mw(msg, func(msg *Message) error { - return next(msg, change) - }) + return mw(msg) }) } // RegisterNackHandler is used to register a function that will be called when // the message is nacked. This function can only be called if the message status // is open, otherwise it panics. -func (m *Message) RegisterNackHandler(mw NackMiddleware) { - m.RegisterStatusHandler(func(msg *Message, change StatusChange, next StatusChangeHandler) error { +func (m *Message) RegisterNackHandler(mw NackHandler) { + m.RegisterStatusHandler(func(msg *Message, change StatusChange) error { if change.New != MessageStatusNacked { - return next(msg, change) + return nil // skip } - return mw(msg, change.Reason, func(msg *Message, reason error) error { - return next(msg, change) - }) + return mw(msg, change.Reason) }) m.hasNackHandler = true } @@ -199,17 +175,12 @@ func (m *Message) RegisterNackHandler(mw NackMiddleware) { // RegisterDropHandler is used to register a function that will be called when // the message is dropped. This function can only be called if the message // status is open, otherwise it panics. -func (m *Message) RegisterDropHandler(mw DropMiddleware) { - m.RegisterStatusHandler(func(msg *Message, change StatusChange, next StatusChangeHandler) error { +func (m *Message) RegisterDropHandler(mw DropHandler) { + m.RegisterStatusHandler(func(msg *Message, change StatusChange) error { if change.New != MessageStatusDropped { - return next(msg, change) + return nil } - mw(msg, change.Reason, func(msg *Message, reason error) { - err := next(msg, change) - if err != nil { - panic(cerrors.Errorf("BUG: drop handlers should never return an error (message %s): %w", msg.ID(), err)) - } - }) + mw(msg, change.Reason) return nil }) } diff --git a/pkg/pipeline/stream/message_test.go b/pkg/pipeline/stream/message_test.go index 322009c9e..12c35b2f5 100644 --- a/pkg/pipeline/stream/message_test.go +++ b/pkg/pipeline/stream/message_test.go @@ -47,7 +47,7 @@ func TestMessage_Ack_WithHandler(t *testing.T) { ackedMessageHandlerCallCount int ) - msg.RegisterAckHandler(func(*Message, AckHandler) error { + msg.RegisterAckHandler(func(*Message) error { ackedMessageHandlerCallCount++ return nil }) @@ -90,35 +90,34 @@ func TestMessage_Ack_WithFailingHandler(t *testing.T) { ) { - // first handler should never be called - msg.RegisterAckHandler(func(*Message, AckHandler) error { - t.Fatalf("did not expect first handler to be called") + // first handler should still be called + msg.RegisterAckHandler(func(*Message) error { + ackedMessageHandlerCallCount++ return nil }) // second handler fails - msg.RegisterAckHandler(func(*Message, AckHandler) error { + msg.RegisterAckHandler(func(*Message) error { return wantErr }) // third handler should work as expected - msg.RegisterAckHandler(func(msg *Message, next AckHandler) error { + msg.RegisterAckHandler(func(msg *Message) error { ackedMessageHandlerCallCount++ - return next(msg) + return nil }) // fourth handler should be called twice, once for ack, once for drop - msg.RegisterStatusHandler(func(msg *Message, change StatusChange, next StatusChangeHandler) error { + msg.RegisterStatusHandler(func(msg *Message, change StatusChange) error { statusMessageHandlerCallCount++ - return next(msg, change) + return nil }) // drop handler should be called after the ack fails - msg.RegisterDropHandler(func(msg *Message, reason error, next DropHandler) { - if ackedMessageHandlerCallCount != 1 { - t.Fatal("expected acked message handler to already be called") + msg.RegisterDropHandler(func(msg *Message, reason error) { + if ackedMessageHandlerCallCount != 2 { + t.Fatal("expected acked message handlers to already be called") } droppedMessageHandlerCallCount++ - next(msg, reason) }) // nack handler should not be called - msg.RegisterNackHandler(func(*Message, error, NackHandler) error { + msg.RegisterNackHandler(func(*Message, error) error { t.Fatalf("did not expect nack handler to be called") return nil }) @@ -131,8 +130,8 @@ func TestMessage_Ack_WithFailingHandler(t *testing.T) { t.Fatalf("ack expected error %v, got: %v", wantErr, err) } assertMessageIsDropped(t, &msg) - if ackedMessageHandlerCallCount != 1 { - t.Fatalf("expected acked message handler to be called once, got %d calls", ackedMessageHandlerCallCount) + if ackedMessageHandlerCallCount != 2 { + t.Fatalf("expected acked message handler to be called twice, got %d calls", ackedMessageHandlerCallCount) } if droppedMessageHandlerCallCount != 1 { t.Fatalf("expected dropped message handler to be called once, got %d calls", droppedMessageHandlerCallCount) @@ -186,12 +185,12 @@ func TestMessage_Nack_WithHandler(t *testing.T) { nackedMessageHandlerCallCount int ) - msg.RegisterNackHandler(func(msg *Message, err error, next NackHandler) error { + msg.RegisterNackHandler(func(msg *Message, err error) error { nackedMessageHandlerCallCount++ if err != wantErr { t.Fatalf("nacked message handler, expected err %v, got %v", wantErr, err) } - return next(msg, err) + return nil }) err := msg.Nack(wantErr) @@ -225,35 +224,34 @@ func TestMessage_Nack_WithFailingHandler(t *testing.T) { ) { - // first handler should never be called - msg.RegisterNackHandler(func(*Message, error, NackHandler) error { - t.Fatalf("did not expect first handler to be called") + // first handler should still be called + msg.RegisterNackHandler(func(*Message, error) error { + nackedMessageHandlerCallCount++ return nil }) // second handler fails - msg.RegisterNackHandler(func(*Message, error, NackHandler) error { + msg.RegisterNackHandler(func(*Message, error) error { return wantErr }) // third handler should work as expected - msg.RegisterNackHandler(func(msg *Message, reason error, next NackHandler) error { + msg.RegisterNackHandler(func(msg *Message, reason error) error { nackedMessageHandlerCallCount++ - return next(msg, reason) + return nil }) // fourth handler should be called twice, once for ack, once for drop - msg.RegisterStatusHandler(func(msg *Message, change StatusChange, next StatusChangeHandler) error { + msg.RegisterStatusHandler(func(msg *Message, change StatusChange) error { statusMessageHandlerCallCount++ - return next(msg, change) + return nil }) // drop handler should be called after the nack fails - msg.RegisterDropHandler(func(msg *Message, reason error, next DropHandler) { - if nackedMessageHandlerCallCount != 1 { - t.Fatal("expected nacked message handler to already be called") + msg.RegisterDropHandler(func(msg *Message, reason error) { + if nackedMessageHandlerCallCount != 2 { + t.Fatal("expected nacked message handlers to already be called") } droppedMessageHandlerCallCount++ - next(msg, reason) }) // ack handler should not be called - msg.RegisterAckHandler(func(*Message, AckHandler) error { + msg.RegisterAckHandler(func(*Message) error { t.Fatalf("did not expect ack handler to be called") return nil }) @@ -266,8 +264,8 @@ func TestMessage_Nack_WithFailingHandler(t *testing.T) { t.Fatalf("nack expected error %v, got: %v", wantErr, err) } assertMessageIsDropped(t, &msg) - if nackedMessageHandlerCallCount != 1 { - t.Fatalf("expected nacked message handler to be called once, got %d calls", nackedMessageHandlerCallCount) + if nackedMessageHandlerCallCount != 2 { + t.Fatalf("expected nacked message handler to be called twice, got %d calls", nackedMessageHandlerCallCount) } if droppedMessageHandlerCallCount != 1 { t.Fatalf("expected dropped message handler to be called once, got %d calls", droppedMessageHandlerCallCount) @@ -309,14 +307,13 @@ func TestMessage_Drop_WithHandler(t *testing.T) { ) { - msg.RegisterDropHandler(func(msg *Message, reason error, next DropHandler) { + msg.RegisterDropHandler(func(msg *Message, reason error) { droppedMessageHandlerCallCount++ - next(msg, reason) }) // second handler should be called once for drop - msg.RegisterStatusHandler(func(msg *Message, change StatusChange, next StatusChangeHandler) error { + msg.RegisterStatusHandler(func(msg *Message, change StatusChange) error { statusMessageHandlerCallCount++ - return next(msg, change) + return nil }) } @@ -337,7 +334,7 @@ func TestMessage_Drop_WithFailingHandler(t *testing.T) { var msg Message // handler return error for drop - msg.RegisterStatusHandler(func(msg *Message, change StatusChange, next StatusChangeHandler) error { + msg.RegisterStatusHandler(func(msg *Message, change StatusChange) error { return cerrors.New("oops") }) @@ -391,7 +388,7 @@ func TestMessage_StatusChangeTwice(t *testing.T) { t.Run("nacked message", func(t *testing.T) { var msg Message // need to register a nack handler for message to be nacked - msg.RegisterNackHandler(func(*Message, error, NackHandler) error { return nil }) + msg.RegisterNackHandler(func(*Message, error) error { return nil }) err := msg.Nack(nil) if err != nil { t.Fatalf("ack did not expect error, got %v", err) @@ -424,7 +421,7 @@ func TestMessage_RegisterHandlerFail(t *testing.T) { t.Fatalf("expected msg.RegisterAckHandler to panic") } }() - msg.RegisterAckHandler(func(*Message, AckHandler) error { return nil }) + msg.RegisterAckHandler(func(*Message) error { return nil }) } assertRegisterNackHandlerPanics := func(msg *Message) { defer func() { @@ -432,7 +429,7 @@ func TestMessage_RegisterHandlerFail(t *testing.T) { t.Fatalf("expected msg.RegisterNackHandler to panic") } }() - msg.RegisterNackHandler(func(*Message, error, NackHandler) error { return nil }) + msg.RegisterNackHandler(func(*Message, error) error { return nil }) } assertRegisterDropHandlerPanics := func(msg *Message) { defer func() { @@ -440,7 +437,7 @@ func TestMessage_RegisterHandlerFail(t *testing.T) { t.Fatalf("expected msg.RegisterDropHandler to panic") } }() - msg.RegisterDropHandler(func(*Message, error, DropHandler) {}) + msg.RegisterDropHandler(func(*Message, error) {}) } // registering a handler after the message is acked should panic @@ -459,7 +456,7 @@ func TestMessage_RegisterHandlerFail(t *testing.T) { t.Run("nacked message", func(t *testing.T) { var msg Message // need to register a nack handler for message to be nacked - msg.RegisterNackHandler(func(*Message, error, NackHandler) error { return nil }) + msg.RegisterNackHandler(func(*Message, error) error { return nil }) err := msg.Nack(nil) if err != nil { t.Fatalf("ack did not expect error, got %v", err) diff --git a/pkg/pipeline/stream/metrics.go b/pkg/pipeline/stream/metrics.go index 406c5801d..b6dd3bb13 100644 --- a/pkg/pipeline/stream/metrics.go +++ b/pkg/pipeline/stream/metrics.go @@ -46,7 +46,7 @@ func (n *MetricsNode) Run(ctx context.Context) error { return err } - msg.RegisterAckHandler(func(msg *Message, next AckHandler) error { + msg.RegisterAckHandler(func(msg *Message) error { // TODO for now we call method Bytes() on key and payload to get the // bytes representation. In case of a structured payload or key it // is marshaled into JSON, which might not be the correct way to @@ -60,7 +60,7 @@ func (n *MetricsNode) Run(ctx context.Context) error { bytes += len(msg.Record.Payload.Bytes()) } n.BytesHistogram.Observe(float64(bytes)) - return next(msg) + return nil }) err = n.base.Send(ctx, n.logger, msg) diff --git a/pkg/pipeline/stream/processor_test.go b/pkg/pipeline/stream/processor_test.go index 1bb324425..43183f554 100644 --- a/pkg/pipeline/stream/processor_test.go +++ b/pkg/pipeline/stream/processor_test.go @@ -143,9 +143,9 @@ func TestProcessorNode_ErrorWithNackHandler(t *testing.T) { out := n.Pub() msg := &Message{Ctx: ctx} - msg.RegisterNackHandler(func(msg *Message, err error, next NackHandler) error { + msg.RegisterNackHandler(func(msg *Message, err error) error { assert.True(t, cerrors.Is(err, wantErr), "expected underlying error to be the transform error") - return next(msg, err) // the error should be regarded as handled + return nil // the error should be regarded as handled }) go func() { // publisher @@ -186,11 +186,11 @@ func TestProcessorNode_Skip(t *testing.T) { // register a dummy AckHandler and NackHandler for tests. counter := 0 - msg.RegisterAckHandler(func(msg *Message, next AckHandler) error { + msg.RegisterAckHandler(func(msg *Message) error { counter++ return nil }) - msg.RegisterNackHandler(func(msg *Message, err error, next NackHandler) error { + msg.RegisterNackHandler(func(msg *Message, err error) error { // Our NackHandler shouldn't ever be hit if we're correctly skipping // so fail the test if we get here at all. t.Fail() diff --git a/pkg/pipeline/stream/source.go b/pkg/pipeline/stream/source.go index 86bd8a788..16d8339fc 100644 --- a/pkg/pipeline/stream/source.go +++ b/pkg/pipeline/stream/source.go @@ -104,24 +104,20 @@ func (n *SourceNode) Run(ctx context.Context) (err error) { // register another open message wgOpenMessages.Add(1) msg.RegisterStatusHandler( - func(msg *Message, change StatusChange, next StatusChangeHandler) error { + func(msg *Message, change StatusChange) error { // this is the last handler to be executed, once this handler is // reached we know either the message was successfully acked, nacked // or dropped defer n.PipelineTimer.Update(time.Since(msg.Record.ReadAt)) defer wgOpenMessages.Done() - return next(msg, change) + return nil }, ) msg.RegisterAckHandler( - func(msg *Message, next AckHandler) error { + func(msg *Message) error { n.logger.Trace(msg.Ctx).Msg("forwarding ack to source connector") - err := n.Source.Ack(msg.Ctx, msg.Record.Position) - if err != nil { - return err - } - return next(msg) + return n.Source.Ack(msg.Ctx, msg.Record.Position) }, ) From 68cd7c7e18066500063db2e1fa247057e0077c3f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Wed, 22 Jun 2022 20:26:08 +0200 Subject: [PATCH 12/25] implement SourceAckerNode --- pkg/pipeline/lifecycle.go | 15 +++- pkg/pipeline/lifecycle_test.go | 1 + pkg/pipeline/stream/source.go | 7 -- pkg/pipeline/stream/source_acker.go | 129 ++++++++++++++++++++++++++++ pkg/pipeline/stream/stream_test.go | 80 ++++++++++------- 5 files changed, 190 insertions(+), 42 deletions(-) create mode 100644 pkg/pipeline/stream/source_acker.go diff --git a/pkg/pipeline/lifecycle.go b/pkg/pipeline/lifecycle.go index 9b867ab51..3c2e6f17f 100644 --- a/pkg/pipeline/lifecycle.go +++ b/pkg/pipeline/lifecycle.go @@ -233,6 +233,15 @@ func (s *Service) buildProcessorNodes( return nodes, nil } +func (s *Service) buildSourceAckerNode( + src connector.Source, +) *stream.SourceAckerNode { + return &stream.SourceAckerNode{ + Name: src.ID() + "-acker", + Source: src, + } +} + func (s *Service) buildSourceNodes( ctx context.Context, connFetcher ConnectorFetcher, @@ -259,15 +268,17 @@ func (s *Service) buildSourceNodes( pl.Config.Name, ), } + ackerNode := s.buildSourceAckerNode(instance.(connector.Source)) + ackerNode.Sub(sourceNode.Pub()) metricsNode := s.buildMetricsNode(pl, instance) - metricsNode.Sub(sourceNode.Pub()) + metricsNode.Sub(ackerNode.Pub()) procNodes, err := s.buildProcessorNodes(ctx, procFetcher, pl, instance.Config().ProcessorIDs, metricsNode, next) if err != nil { return nil, cerrors.Errorf("could not build processor nodes for connector %s: %w", instance.ID(), err) } - nodes = append(nodes, &sourceNode, metricsNode) + nodes = append(nodes, &sourceNode, ackerNode, metricsNode) nodes = append(nodes, procNodes...) } diff --git a/pkg/pipeline/lifecycle_test.go b/pkg/pipeline/lifecycle_test.go index c15b783fa..2424c3aa4 100644 --- a/pkg/pipeline/lifecycle_test.go +++ b/pkg/pipeline/lifecycle_test.go @@ -111,6 +111,7 @@ func TestServiceLifecycle_PipelineError(t *testing.T) { // wait for pipeline to finish err = pl.Wait() assert.Error(t, err) + t.Log(err) assert.Equal(t, StatusDegraded, pl.Status) // pipeline errors contain only string messages, so we can only compare the errors by the messages diff --git a/pkg/pipeline/stream/source.go b/pkg/pipeline/stream/source.go index 16d8339fc..9c827c5fc 100644 --- a/pkg/pipeline/stream/source.go +++ b/pkg/pipeline/stream/source.go @@ -114,13 +114,6 @@ func (n *SourceNode) Run(ctx context.Context) (err error) { }, ) - msg.RegisterAckHandler( - func(msg *Message) error { - n.logger.Trace(msg.Ctx).Msg("forwarding ack to source connector") - return n.Source.Ack(msg.Ctx, msg.Record.Position) - }, - ) - err = n.base.Send(ctx, n.logger, msg) if err != nil { msg.Drop() diff --git a/pkg/pipeline/stream/source_acker.go b/pkg/pipeline/stream/source_acker.go new file mode 100644 index 000000000..9dc6c759f --- /dev/null +++ b/pkg/pipeline/stream/source_acker.go @@ -0,0 +1,129 @@ +// Copyright © 2022 Meroxa, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package stream + +import ( + "context" + + "github.com/conduitio/conduit/pkg/connector" + "github.com/conduitio/conduit/pkg/foundation/cerrors" + "github.com/conduitio/conduit/pkg/foundation/log" + "github.com/conduitio/conduit/pkg/foundation/semaphore" +) + +// SourceAckerNode is responsible for handling acknowledgments for messages of +// a specific source and forwarding them to the source in the correct order. +type SourceAckerNode struct { + Name string + Source connector.Source + + base pubSubNodeBase + logger log.CtxLogger + + // sem ensures acks are sent to the source in the correct order and only one + // at a time + sem semaphore.Simple +} + +func (n *SourceAckerNode) ID() string { + return n.Name +} + +func (n *SourceAckerNode) Run(ctx context.Context) error { + trigger, cleanup, err := n.base.Trigger(ctx, n.logger) + if err != nil { + return err + } + + defer cleanup() + for { + msg, err := trigger() + if err != nil || msg == nil { + return err + } + + // enqueue message in semaphore + ticket := n.sem.Enqueue() + n.registerAckHandler(msg, ticket) + n.registerNackHandler(msg, ticket) + + err = n.base.Send(ctx, n.logger, msg) + if err != nil { + msg.Drop() + return err + } + } +} + +func (n *SourceAckerNode) registerAckHandler(msg *Message, ticket semaphore.Ticket) { + msg.RegisterAckHandler( + func(msg *Message) (err error) { + defer func() { + tmpErr := n.sem.Release(ticket) + if err != nil { + // we are already returning an error, log this one instead + n.logger.Err(msg.Ctx, tmpErr).Msg("error releasing semaphore ticket for ack") + return + } + err = tmpErr + }() + n.logger.Trace(msg.Ctx).Msg("acquiring semaphore for ack") + err = n.sem.Acquire(ticket) + if err != nil { + return cerrors.Errorf("could not acquire semaphore for ack: %w", err) + } + n.logger.Trace(msg.Ctx).Msg("forwarding ack to source connector") + return n.Source.Ack(msg.Ctx, msg.Record.Position) + }, + ) +} + +func (n *SourceAckerNode) registerNackHandler(msg *Message, ticket semaphore.Ticket) { + msg.RegisterNackHandler( + func(msg *Message, reason error) (err error) { + defer func() { + tmpErr := n.sem.Release(ticket) + if err != nil { + // we are already returning an error, log this one instead + n.logger.Err(msg.Ctx, tmpErr).Msg("error releasing semaphore ticket for nack") + return + } + err = tmpErr + }() + n.logger.Trace(msg.Ctx).Msg("acquiring semaphore for nack") + err = n.sem.Acquire(ticket) + if err != nil { + return cerrors.Errorf("could not acquire semaphore for nack: %w", err) + } + n.logger.Trace(msg.Ctx).Msg("forwarding nack to DLQ handler") + // TODO implement DLQ and call it here, right now any nacked message + // will just stop the pipeline because we don't support DLQs + // https://github.com/ConduitIO/conduit/issues/306 + return cerrors.New("no DLQ handler configured") + }, + ) +} + +func (n *SourceAckerNode) Sub(in <-chan *Message) { + n.base.Sub(in) +} + +func (n *SourceAckerNode) Pub() <-chan *Message { + return n.base.Pub() +} + +func (n *SourceAckerNode) SetLogger(logger log.CtxLogger) { + n.logger = logger +} diff --git a/pkg/pipeline/stream/stream_test.go b/pkg/pipeline/stream/stream_test.go index 57b1b2cb7..18366f8b4 100644 --- a/pkg/pipeline/stream/stream_test.go +++ b/pkg/pipeline/stream/stream_test.go @@ -47,27 +47,33 @@ func Example_simpleStream() { Source: generatorSource(ctrl, logger, "generator", 10, time.Millisecond*10), PipelineTimer: noop.Timer{}, } - node2 := &stream.DestinationNode{ + node2 := &stream.SourceAckerNode{ + Name: "generator-acker", + Source: node1.Source, + } + node3 := &stream.DestinationNode{ Name: "printer", Destination: printerDestination(ctrl, logger, "printer"), ConnectorTimer: noop.Timer{}, } - node3 := &stream.DestinationAckerNode{ + node4 := &stream.DestinationAckerNode{ Name: "printer-acker", - Destination: node2.Destination, + Destination: node3.Destination, } - node2.AckerNode = node3 + node3.AckerNode = node4 stream.SetLogger(node1, logger) stream.SetLogger(node2, logger) stream.SetLogger(node3, logger) + stream.SetLogger(node4, logger) // put everything together - out := node1.Pub() - node2.Sub(out) + node2.Sub(node1.Pub()) + node3.Sub(node2.Pub()) var wg sync.WaitGroup - wg.Add(3) + wg.Add(4) + go runNode(ctx, &wg, node4) go runNode(ctx, &wg, node3) go runNode(ctx, &wg, node2) go runNode(ctx, &wg, node1) @@ -104,6 +110,7 @@ func Example_simpleStream() { // DBG received ack message_id=p/generator-10 node_id=generator // INF stopping source connector component=SourceNode node_id=generator // DBG received error on error channel error="error reading from source: stream not open" component=SourceNode node_id=generator + // DBG incoming messages channel closed component=SourceAckerNode node_id=generator-acker // DBG incoming messages channel closed component=DestinationNode node_id=printer // INF finished successfully } @@ -122,57 +129,62 @@ func Example_complexStream() { Source: generatorSource(ctrl, logger, "generator1", 10, time.Millisecond*10), PipelineTimer: noop.Timer{}, } - node2 := &stream.SourceNode{ + node2 := &stream.SourceAckerNode{ + Name: "generator1-acker", + Source: node1.Source, + } + node3 := &stream.SourceNode{ Name: "generator2", Source: generatorSource(ctrl, logger, "generator2", 10, time.Millisecond*10), PipelineTimer: noop.Timer{}, } - node3 := &stream.FaninNode{Name: "fanin"} - node4 := &stream.ProcessorNode{ + node4 := &stream.SourceAckerNode{ + Name: "generator2-acker", + Source: node3.Source, + } + node5 := &stream.FaninNode{Name: "fanin"} + node6 := &stream.ProcessorNode{ Name: "counter", Processor: counterProcessor(ctrl, &count), ProcessorTimer: noop.Timer{}, } - node5 := &stream.FanoutNode{Name: "fanout"} - node6 := &stream.DestinationNode{ + node7 := &stream.FanoutNode{Name: "fanout"} + node8 := &stream.DestinationNode{ Name: "printer1", Destination: printerDestination(ctrl, logger, "printer1"), ConnectorTimer: noop.Timer{}, } - node7 := &stream.DestinationNode{ + node9 := &stream.DestinationNode{ Name: "printer2", Destination: printerDestination(ctrl, logger, "printer2"), ConnectorTimer: noop.Timer{}, } - node8 := &stream.DestinationAckerNode{ + node10 := &stream.DestinationAckerNode{ Name: "printer1-acker", - Destination: node6.Destination, + Destination: node8.Destination, } - node6.AckerNode = node8 - node9 := &stream.DestinationAckerNode{ + node8.AckerNode = node10 + node11 := &stream.DestinationAckerNode{ Name: "printer2-acker", - Destination: node7.Destination, + Destination: node9.Destination, } - node7.AckerNode = node9 + node9.AckerNode = node11 // put everything together - out := node1.Pub() - node3.Sub(out) - out = node2.Pub() - node3.Sub(out) + node2.Sub(node1.Pub()) + node4.Sub(node3.Pub()) + + node5.Sub(node2.Pub()) + node5.Sub(node4.Pub()) - out = node3.Pub() - node4.Sub(out) - out = node4.Pub() - node5.Sub(out) + node6.Sub(node5.Pub()) + node7.Sub(node6.Pub()) - out = node5.Pub() - node6.Sub(out) - out = node5.Pub() - node7.Sub(out) + node8.Sub(node7.Pub()) + node9.Sub(node7.Pub()) // run nodes - nodes := []stream.Node{node1, node2, node3, node4, node5, node6, node7, node8, node9} + nodes := []stream.Node{node1, node2, node3, node4, node5, node6, node7, node8, node9, node10, node11} var wg sync.WaitGroup wg.Add(len(nodes)) @@ -186,7 +198,7 @@ func Example_complexStream() { 250*time.Millisecond, func() { node1.Stop(nil) - node2.Stop(nil) + node3.Stop(nil) }, ) // give the nodes some time to process the records, plus a bit of time to stop @@ -260,6 +272,8 @@ func Example_complexStream() { // DBG received ack message_id=p/generator1-10 node_id=generator1 // INF stopping source connector component=SourceNode node_id=generator1 // INF stopping source connector component=SourceNode node_id=generator2 + // DBG incoming messages channel closed component=SourceAckerNode node_id=generator1-acker + // DBG incoming messages channel closed component=SourceAckerNode node_id=generator2-acker // DBG received error on error channel error="error reading from source: stream not open" component=SourceNode node_id=generator1 // DBG received error on error channel error="error reading from source: stream not open" component=SourceNode node_id=generator2 // DBG incoming messages channel closed component=ProcessorNode node_id=counter From 6bdc89377dcd29f6dd48da78ef8f2d33ba9774f4 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Thu, 23 Jun 2022 16:44:06 +0200 Subject: [PATCH 13/25] add todo note about possible deadlock --- pkg/pipeline/stream/destination_acker.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/pkg/pipeline/stream/destination_acker.go b/pkg/pipeline/stream/destination_acker.go index f2fa04861..2e437d89c 100644 --- a/pkg/pipeline/stream/destination_acker.go +++ b/pkg/pipeline/stream/destination_acker.go @@ -113,6 +113,11 @@ func (n *DestinationAckerNode) Run(ctx context.Context) (err error) { continue } + // TODO make sure acks are called in the right order or this will block + // forever. Right now we rely on connectors sending acks back in the + // correct order and this should generally be true, but we can't be + // completely sure and a badly written connector shouldn't provoke a + // deadlock. err = n.handleAck(msg, err) if err != nil { return err From 8b6dc73443a07d8c47cc8ecbbeb64b641d2845ac Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Thu, 23 Jun 2022 17:25:49 +0200 Subject: [PATCH 14/25] source acker node test --- pkg/pipeline/stream/destination_acker_test.go | 6 - pkg/pipeline/stream/source_acker_test.go | 155 ++++++++++++++++++ 2 files changed, 155 insertions(+), 6 deletions(-) create mode 100644 pkg/pipeline/stream/source_acker_test.go diff --git a/pkg/pipeline/stream/destination_acker_test.go b/pkg/pipeline/stream/destination_acker_test.go index 0a2a447af..66dfe759e 100644 --- a/pkg/pipeline/stream/destination_acker_test.go +++ b/pkg/pipeline/stream/destination_acker_test.go @@ -44,9 +44,6 @@ func TestAckerNode_Run_StopAfterWait(t *testing.T) { is.NoErr(err) }() - // give Go a chance to run the node - time.Sleep(time.Millisecond) - // note that there should be no calls to the destination at all if we didn't // receive any ExpectedAck call @@ -81,9 +78,6 @@ func TestAckerNode_Run_StopAfterExpectAck(t *testing.T) { is.NoErr(err) }() - // give Go a chance to run the node - time.Sleep(time.Millisecond) - // up to this point there should have been no calls to the destination // only after the call to ExpectAck should the node try to fetch any acks msg := &Message{ diff --git a/pkg/pipeline/stream/source_acker_test.go b/pkg/pipeline/stream/source_acker_test.go new file mode 100644 index 000000000..746e991ca --- /dev/null +++ b/pkg/pipeline/stream/source_acker_test.go @@ -0,0 +1,155 @@ +// Copyright © 2022 Meroxa, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package stream + +import ( + "context" + "math/rand" + "strconv" + "sync" + "testing" + "time" + + "github.com/conduitio/conduit/pkg/connector/mock" + "github.com/conduitio/conduit/pkg/record" + "github.com/golang/mock/gomock" + "github.com/matryer/is" +) + +func TestSourceAckerNode_ForwardAck(t *testing.T) { + is := is.New(t) + ctx := context.Background() + ctrl := gomock.NewController(t) + src := mock.NewSource(ctrl) + + node := &SourceAckerNode{ + Name: "acker-node", + Source: src, + } + in := make(chan *Message) + out := node.Pub() + node.Sub(in) + + go func() { + err := node.Run(ctx) + is.NoErr(err) + }() + + want := &Message{Ctx: ctx, Record: record.Record{Position: []byte("foo")}} + // expect to receive an ack in the source after the message is acked + src.EXPECT().Ack(want.Ctx, want.Record.Position).Return(nil) + + in <- want + got := <-out + is.Equal(got, want) + + // ack should be propagated to the source, the mock will do the assertion + err := got.Ack() + is.NoErr(err) + + // gracefully stop node and give the test 1 second to finish + close(in) + + waitCtx, cancel := context.WithTimeout(ctx, time.Second) + defer cancel() + + select { + case <-waitCtx.Done(): + is.Fail() // expected node to stop running + case <-out: + // all good + } +} + +func TestSourceAckerNode_AckOrder(t *testing.T) { + is := is.New(t) + ctx := context.Background() + ctrl := gomock.NewController(t) + src := mock.NewSource(ctrl) + + const count = 1000 + const maxSleep = 1 * time.Millisecond + + node := &SourceAckerNode{ + Name: "acker-node", + Source: src, + } + in := make(chan *Message) + out := node.Pub() + node.Sub(in) + + go func() { + err := node.Run(ctx) + is.NoErr(err) + }() + + // first send messages through the node in the correct order + messages := make([]*Message, count) + for i := 0; i < count; i++ { + m := &Message{ + Ctx: ctx, + Record: record.Record{ + Position: []byte(strconv.Itoa(i)), // position is monotonically increasing + }, + } + in <- m + <-out + messages[i] = m + } + + // expect to receive an acks in the same order as the order of the messages + expectedPosition := 0 + expectedCalls := make([]*gomock.Call, count) + for i := 0; i < count; i++ { + expectedCalls[i] = src.EXPECT(). + Ack(ctx, messages[i].Record.Position). + Do(func(context.Context, record.Position) { expectedPosition++ }). + Return(nil) + } + gomock.InOrder(expectedCalls...) // enforce order + + // ack messages concurrently in random order + var wg sync.WaitGroup + wg.Add(count) + for i := 0; i < count; i++ { + go func(msg *Message) { + defer wg.Done() + // sleep for a random amount of time and ack the message + //nolint:gosec // math/rand is good enough for a test + time.Sleep(time.Duration(rand.Int63n(int64(maxSleep/time.Nanosecond))) * time.Nanosecond) + err := msg.Ack() + is.NoErr(err) + }(messages[i]) + } + + // gracefully stop node and give the test 1 second to finish + close(in) + + wgDone := make(chan struct{}) + go func() { + defer close(wgDone) + wg.Wait() + }() + + waitCtx, cancel := context.WithTimeout(ctx, time.Second) + defer cancel() + + select { + case <-waitCtx.Done(): + is.Fail() // expected to receive all acks in time + case <-wgDone: + // all good + } +} From 94b4f437c20701932a4d6f89779326bbafaca418 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Tue, 28 Jun 2022 20:46:15 +0200 Subject: [PATCH 15/25] don't forward acks after a failed ack/nack --- pkg/pipeline/stream/source_acker.go | 49 ++++-- pkg/pipeline/stream/source_acker_test.go | 186 +++++++++++++++++++---- 2 files changed, 196 insertions(+), 39 deletions(-) diff --git a/pkg/pipeline/stream/source_acker.go b/pkg/pipeline/stream/source_acker.go index 9dc6c759f..c716f306e 100644 --- a/pkg/pipeline/stream/source_acker.go +++ b/pkg/pipeline/stream/source_acker.go @@ -35,6 +35,10 @@ type SourceAckerNode struct { // sem ensures acks are sent to the source in the correct order and only one // at a time sem semaphore.Simple + // fail is set to true once the first ack/nack fails and we can't guarantee + // that acks will be delivered in the correct order to the source anymore, + // at that point we completely stop processing acks/nacks + fail bool } func (n *SourceAckerNode) ID() string { @@ -71,19 +75,30 @@ func (n *SourceAckerNode) registerAckHandler(msg *Message, ticket semaphore.Tick msg.RegisterAckHandler( func(msg *Message) (err error) { defer func() { - tmpErr := n.sem.Release(ticket) if err != nil { - // we are already returning an error, log this one instead - n.logger.Err(msg.Ctx, tmpErr).Msg("error releasing semaphore ticket for ack") - return + n.fail = true + } + tmpErr := n.sem.Release(ticket) + if tmpErr != nil { + if err != nil { + // we are already returning an error, log this one instead + n.logger.Err(msg.Ctx, tmpErr).Msg("error releasing semaphore ticket for ack") + } else { + err = tmpErr + } } - err = tmpErr }() n.logger.Trace(msg.Ctx).Msg("acquiring semaphore for ack") err = n.sem.Acquire(ticket) if err != nil { return cerrors.Errorf("could not acquire semaphore for ack: %w", err) } + + if n.fail { + n.logger.Trace(msg.Ctx).Msg("blocking forwarding of ack to source connector, because another message failed to be acked/nacked") + return cerrors.Errorf("another message failed to be acked/nacked") + } + n.logger.Trace(msg.Ctx).Msg("forwarding ack to source connector") return n.Source.Ack(msg.Ctx, msg.Record.Position) }, @@ -94,22 +109,34 @@ func (n *SourceAckerNode) registerNackHandler(msg *Message, ticket semaphore.Tic msg.RegisterNackHandler( func(msg *Message, reason error) (err error) { defer func() { - tmpErr := n.sem.Release(ticket) if err != nil { - // we are already returning an error, log this one instead - n.logger.Err(msg.Ctx, tmpErr).Msg("error releasing semaphore ticket for nack") - return + n.fail = true + } + tmpErr := n.sem.Release(ticket) + if tmpErr != nil { + if err != nil { + // we are already returning an error, log this one instead + n.logger.Err(msg.Ctx, tmpErr).Msg("error releasing semaphore ticket for nack") + } else { + err = tmpErr + } } - err = tmpErr }() n.logger.Trace(msg.Ctx).Msg("acquiring semaphore for nack") err = n.sem.Acquire(ticket) if err != nil { return cerrors.Errorf("could not acquire semaphore for nack: %w", err) } + + if n.fail { + n.logger.Trace(msg.Ctx).Msg("blocking forwarding of nack to DLQ handler, because another message failed to be acked/nacked") + return cerrors.Errorf("another message failed to be acked/nacked") + } + n.logger.Trace(msg.Ctx).Msg("forwarding nack to DLQ handler") // TODO implement DLQ and call it here, right now any nacked message - // will just stop the pipeline because we don't support DLQs + // will just stop the pipeline because we don't support DLQs, + // don't forget to forward ack to source if the DLQ call succeeds // https://github.com/ConduitIO/conduit/issues/306 return cerrors.New("no DLQ handler configured") }, diff --git a/pkg/pipeline/stream/source_acker_test.go b/pkg/pipeline/stream/source_acker_test.go index 746e991ca..12307cbd6 100644 --- a/pkg/pipeline/stream/source_acker_test.go +++ b/pkg/pipeline/stream/source_acker_test.go @@ -16,12 +16,14 @@ package stream import ( "context" + "errors" "math/rand" "strconv" "sync" "testing" "time" + "github.com/conduitio/conduit/pkg/connector" "github.com/conduitio/conduit/pkg/connector/mock" "github.com/conduitio/conduit/pkg/record" "github.com/golang/mock/gomock" @@ -33,19 +35,9 @@ func TestSourceAckerNode_ForwardAck(t *testing.T) { ctx := context.Background() ctrl := gomock.NewController(t) src := mock.NewSource(ctrl) + helper := sourceAckerNodeTestHelper{} - node := &SourceAckerNode{ - Name: "acker-node", - Source: src, - } - in := make(chan *Message) - out := node.Pub() - node.Sub(in) - - go func() { - err := node.Run(ctx) - is.NoErr(err) - }() + _, in, out := helper.newSourceAckerNode(ctx, is, src) want := &Message{Ctx: ctx, Record: record.Record{Position: []byte("foo")}} // expect to receive an ack in the source after the message is acked @@ -78,10 +70,126 @@ func TestSourceAckerNode_AckOrder(t *testing.T) { ctx := context.Background() ctrl := gomock.NewController(t) src := mock.NewSource(ctrl) + helper := sourceAckerNodeTestHelper{} + + _, in, out := helper.newSourceAckerNode(ctx, is, src) + // send 1000 messages through the node + messages := helper.sendMessages(ctx, 1000, in, out) + // expect all messages to be acked + expectedCalls := helper.expectAcks(ctx, messages, src) + gomock.InOrder(expectedCalls...) // enforce order of acks - const count = 1000 - const maxSleep = 1 * time.Millisecond + // ack messages concurrently in random order, expect no errors + var wg sync.WaitGroup + helper.ackMessagesConcurrently( + &wg, + messages, + func(msg *Message, err error) { + is.NoErr(err) + }, + ) + + // gracefully stop node and give the test 1 second to finish + close(in) + + err := helper.wait(ctx, &wg, time.Second) + is.NoErr(err) // expected to receive acks in time +} + +func TestSourceAckerNode_FailedAck(t *testing.T) { + is := is.New(t) + ctx := context.Background() + ctrl := gomock.NewController(t) + src := mock.NewSource(ctrl) + helper := sourceAckerNodeTestHelper{} + + _, in, out := helper.newSourceAckerNode(ctx, is, src) + // send 1000 messages through the node + messages := helper.sendMessages(ctx, 1000, in, out) + // expect first 500 to be acked successfully + expectedCalls := helper.expectAcks(ctx, messages[:500], src) + gomock.InOrder(expectedCalls...) // enforce order of acks + // the 500th message should be acked unsuccessfully + wantErr := errors.New("test error") + src.EXPECT(). + Ack(ctx, messages[500].Record.Position). + Return(wantErr). + After(expectedCalls[len(expectedCalls)-1]) // should happen after last acked call + + // ack messages concurrently in random order, expect errors for second half + var wg sync.WaitGroup + helper.ackMessagesConcurrently(&wg, messages[:500], + func(msg *Message, err error) { + is.NoErr(err) // expected messages from the first half to be acked successfully + }, + ) + helper.ackMessagesConcurrently(&wg, messages[500:501], + func(msg *Message, err error) { + is.Equal(err, wantErr) // expected the middle message ack to fail with specific error + }, + ) + helper.ackMessagesConcurrently(&wg, messages[501:], + func(msg *Message, err error) { + is.True(err != nil) // expected messages from the second half to be acked unsuccessfully + is.True(err != wantErr) + }, + ) + + // gracefully stop node and give the test 1 second to finish + close(in) + + err := helper.wait(ctx, &wg, time.Second) + is.NoErr(err) // expected to receive acks in time +} +func TestSourceAckerNode_FailedNack(t *testing.T) { + is := is.New(t) + ctx := context.Background() + ctrl := gomock.NewController(t) + src := mock.NewSource(ctrl) + helper := sourceAckerNodeTestHelper{} + + _, in, out := helper.newSourceAckerNode(ctx, is, src) + // send 1000 messages through the node + messages := helper.sendMessages(ctx, 1000, in, out) + // expect first 500 to be acked successfully + expectedCalls := helper.expectAcks(ctx, messages[:500], src) + gomock.InOrder(expectedCalls...) // enforce order of acks + // the 500th message will be nacked unsuccessfully, no more acks should be received after that + + // ack messages concurrently in random order + var wg sync.WaitGroup + helper.ackMessagesConcurrently(&wg, messages[:500], + func(msg *Message, err error) { + is.NoErr(err) // expected messages from the first half to be acked successfully + }, + ) + helper.ackMessagesConcurrently(&wg, messages[501:], + func(msg *Message, err error) { + is.True(err != nil) // expected messages from the second half to be acked unsuccessfully + }, + ) + + wantErr := errors.New("test error") + err := messages[500].Nack(wantErr) + is.True(err != nil) // expected the 500th message nack to fail with specific error + + // gracefully stop node and give the test 1 second to finish + close(in) + + err = helper.wait(ctx, &wg, time.Second) + is.NoErr(err) // expected to receive acks in time +} + +// sourceAckerNodeTestHelper groups together helper functions for tests related +// to SourceAckerNode. +type sourceAckerNodeTestHelper struct{} + +func (sourceAckerNodeTestHelper) newSourceAckerNode( + ctx context.Context, + is *is.I, + src connector.Source, +) (*SourceAckerNode, chan<- *Message, <-chan *Message) { node := &SourceAckerNode{ Name: "acker-node", Source: src, @@ -95,7 +203,15 @@ func TestSourceAckerNode_AckOrder(t *testing.T) { is.NoErr(err) }() - // first send messages through the node in the correct order + return node, in, out +} + +func (sourceAckerNodeTestHelper) sendMessages( + ctx context.Context, + count int, + in chan<- *Message, + out <-chan *Message, +) []*Message { messages := make([]*Message, count) for i := 0; i < count; i++ { m := &Message{ @@ -108,20 +224,35 @@ func TestSourceAckerNode_AckOrder(t *testing.T) { <-out messages[i] = m } + return messages +} + +func (sourceAckerNodeTestHelper) expectAcks( + ctx context.Context, + messages []*Message, + src *mock.Source, +) []*gomock.Call { + count := len(messages) - // expect to receive an acks in the same order as the order of the messages - expectedPosition := 0 + // expect to receive acks successfully expectedCalls := make([]*gomock.Call, count) for i := 0; i < count; i++ { expectedCalls[i] = src.EXPECT(). Ack(ctx, messages[i].Record.Position). - Do(func(context.Context, record.Position) { expectedPosition++ }). Return(nil) } - gomock.InOrder(expectedCalls...) // enforce order - // ack messages concurrently in random order - var wg sync.WaitGroup + return expectedCalls +} + +func (sourceAckerNodeTestHelper) ackMessagesConcurrently( + wg *sync.WaitGroup, + messages []*Message, + assertAckErr func(*Message, error), +) { + const maxSleep = time.Millisecond + count := len(messages) + wg.Add(count) for i := 0; i < count; i++ { go func(msg *Message) { @@ -130,26 +261,25 @@ func TestSourceAckerNode_AckOrder(t *testing.T) { //nolint:gosec // math/rand is good enough for a test time.Sleep(time.Duration(rand.Int63n(int64(maxSleep/time.Nanosecond))) * time.Nanosecond) err := msg.Ack() - is.NoErr(err) + assertAckErr(msg, err) }(messages[i]) } +} - // gracefully stop node and give the test 1 second to finish - close(in) - +func (sourceAckerNodeTestHelper) wait(ctx context.Context, wg *sync.WaitGroup, timeout time.Duration) error { wgDone := make(chan struct{}) go func() { defer close(wgDone) wg.Wait() }() - waitCtx, cancel := context.WithTimeout(ctx, time.Second) + waitCtx, cancel := context.WithTimeout(ctx, timeout) defer cancel() select { case <-waitCtx.Done(): - is.Fail() // expected to receive all acks in time + return waitCtx.Err() case <-wgDone: - // all good + return nil } } From 3e283dd58df698410107e220d09ed981100d168d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Tue, 28 Jun 2022 20:49:54 +0200 Subject: [PATCH 16/25] use cerrors --- pkg/pipeline/stream/source_acker_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/pkg/pipeline/stream/source_acker_test.go b/pkg/pipeline/stream/source_acker_test.go index 12307cbd6..ad3d53385 100644 --- a/pkg/pipeline/stream/source_acker_test.go +++ b/pkg/pipeline/stream/source_acker_test.go @@ -16,7 +16,6 @@ package stream import ( "context" - "errors" "math/rand" "strconv" "sync" @@ -25,6 +24,7 @@ import ( "github.com/conduitio/conduit/pkg/connector" "github.com/conduitio/conduit/pkg/connector/mock" + "github.com/conduitio/conduit/pkg/foundation/cerrors" "github.com/conduitio/conduit/pkg/record" "github.com/golang/mock/gomock" "github.com/matryer/is" @@ -110,7 +110,7 @@ func TestSourceAckerNode_FailedAck(t *testing.T) { expectedCalls := helper.expectAcks(ctx, messages[:500], src) gomock.InOrder(expectedCalls...) // enforce order of acks // the 500th message should be acked unsuccessfully - wantErr := errors.New("test error") + wantErr := cerrors.New("test error") src.EXPECT(). Ack(ctx, messages[500].Record.Position). Return(wantErr). @@ -170,7 +170,7 @@ func TestSourceAckerNode_FailedNack(t *testing.T) { }, ) - wantErr := errors.New("test error") + wantErr := cerrors.New("test error") err := messages[500].Nack(wantErr) is.True(err != nil) // expected the 500th message nack to fail with specific error From f43fe35c5fd08179ae173038beebcaf8f159a18e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Thu, 7 Jul 2022 12:53:03 +0200 Subject: [PATCH 17/25] use cerrors.New --- pkg/foundation/semaphore/semaphore.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/foundation/semaphore/semaphore.go b/pkg/foundation/semaphore/semaphore.go index 9a267eba4..e68081a40 100644 --- a/pkg/foundation/semaphore/semaphore.go +++ b/pkg/foundation/semaphore/semaphore.go @@ -70,7 +70,7 @@ func (s *Simple) Acquire(t Ticket) error { s.mu.Lock() if s.batch != t.batch { s.mu.Unlock() - return cerrors.Errorf("semaphore: invalid batch") + return cerrors.New("semaphore: invalid batch") } w := s.waiters[t.index] @@ -101,7 +101,7 @@ func (s *Simple) Release(t Ticket) error { defer s.mu.Unlock() if s.batch != t.batch { - return cerrors.Errorf("semaphore: invalid batch") + return cerrors.New("semaphore: invalid batch") } w := s.waiters[t.index] if !w.acquired { From befaf4427cfdd9e662a62e15527e20639f6f066d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Thu, 7 Jul 2022 17:46:39 +0200 Subject: [PATCH 18/25] use LogOrReplace --- pkg/pipeline/stream/source_acker.go | 22 ++++++---------------- 1 file changed, 6 insertions(+), 16 deletions(-) diff --git a/pkg/pipeline/stream/source_acker.go b/pkg/pipeline/stream/source_acker.go index c716f306e..5c354b25f 100644 --- a/pkg/pipeline/stream/source_acker.go +++ b/pkg/pipeline/stream/source_acker.go @@ -79,14 +79,9 @@ func (n *SourceAckerNode) registerAckHandler(msg *Message, ticket semaphore.Tick n.fail = true } tmpErr := n.sem.Release(ticket) - if tmpErr != nil { - if err != nil { - // we are already returning an error, log this one instead - n.logger.Err(msg.Ctx, tmpErr).Msg("error releasing semaphore ticket for ack") - } else { - err = tmpErr - } - } + err = cerrors.LogOrReplace(err, tmpErr, func() { + n.logger.Err(msg.Ctx, tmpErr).Msg("error releasing semaphore ticket for ack") + }) }() n.logger.Trace(msg.Ctx).Msg("acquiring semaphore for ack") err = n.sem.Acquire(ticket) @@ -113,14 +108,9 @@ func (n *SourceAckerNode) registerNackHandler(msg *Message, ticket semaphore.Tic n.fail = true } tmpErr := n.sem.Release(ticket) - if tmpErr != nil { - if err != nil { - // we are already returning an error, log this one instead - n.logger.Err(msg.Ctx, tmpErr).Msg("error releasing semaphore ticket for nack") - } else { - err = tmpErr - } - } + err = cerrors.LogOrReplace(err, tmpErr, func() { + n.logger.Err(msg.Ctx, tmpErr).Msg("error releasing semaphore ticket for nack") + }) }() n.logger.Trace(msg.Ctx).Msg("acquiring semaphore for nack") err = n.sem.Acquire(ticket) From 64b7d2549c74e6b58dd179aff5188e90ba24114e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Mon, 11 Jul 2022 15:04:03 +0200 Subject: [PATCH 19/25] improve benchmarks --- .../semaphore/semaphore_bench_test.go | 24 ++++++++----------- 1 file changed, 10 insertions(+), 14 deletions(-) diff --git a/pkg/foundation/semaphore/semaphore_bench_test.go b/pkg/foundation/semaphore/semaphore_bench_test.go index 8858428ed..d4448f5d5 100644 --- a/pkg/foundation/semaphore/semaphore_bench_test.go +++ b/pkg/foundation/semaphore/semaphore_bench_test.go @@ -15,7 +15,6 @@ package semaphore_test import ( - "container/list" "fmt" "testing" @@ -28,7 +27,7 @@ func BenchmarkNewSem(b *testing.B) { } } -func BenchmarkAcquireSem(b *testing.B) { +func BenchmarkEnqueueOneByOne(b *testing.B) { for _, N := range []int{1, 2, 8, 64, 128} { b.Run(fmt.Sprintf("acquire-%d", N), func(b *testing.B) { b.ResetTimer() @@ -36,30 +35,27 @@ func BenchmarkAcquireSem(b *testing.B) { for i := 0; i < b.N; i++ { for j := 0; j < N; j++ { t := sem.Enqueue() - _ = sem.Acquire(t) - _ = sem.Release(t) + sem.Acquire(t) + sem.Release(t) } } }) } } -func BenchmarkEnqueueReleaseSem(b *testing.B) { +func BenchmarkEnqueueAll(b *testing.B) { for _, N := range []int{1, 2, 8, 64, 128} { b.Run(fmt.Sprintf("enqueue/release-%d", N), func(b *testing.B) { - b.ResetTimer() sem := &semaphore.Simple{} - tickets := list.New() + tickets := make([]semaphore.Ticket, N) + b.ResetTimer() for i := 0; i < b.N; i++ { - tickets.Init() for j := 0; j < N; j++ { - t := sem.Enqueue() - tickets.PushBack(t) + tickets[j] = sem.Enqueue() } - ticket := tickets.Front() - for ticket != nil { - _ = sem.Release(ticket.Value.(semaphore.Ticket)) - ticket = ticket.Next() + for j := 0; j < N; j++ { + _ = sem.Acquire(tickets[j]) + _ = sem.Release(tickets[j]) } } }) From 24c33860ad8fcfeb9f194b644c80b264e6dd4c44 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Mon, 11 Jul 2022 15:08:37 +0200 Subject: [PATCH 20/25] fix linter error --- pkg/foundation/semaphore/semaphore_bench_test.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/pkg/foundation/semaphore/semaphore_bench_test.go b/pkg/foundation/semaphore/semaphore_bench_test.go index d4448f5d5..4e63dd130 100644 --- a/pkg/foundation/semaphore/semaphore_bench_test.go +++ b/pkg/foundation/semaphore/semaphore_bench_test.go @@ -35,8 +35,8 @@ func BenchmarkEnqueueOneByOne(b *testing.B) { for i := 0; i < b.N; i++ { for j := 0; j < N; j++ { t := sem.Enqueue() - sem.Acquire(t) - sem.Release(t) + _ = sem.Acquire(t) + _ = sem.Release(t) } } }) From d4dd111fe999552cf24c1286cb333e161ab2fc5b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Mon, 11 Jul 2022 15:30:03 +0200 Subject: [PATCH 21/25] add comments --- pkg/foundation/semaphore/semaphore.go | 32 ++++++++++++++++++++------- 1 file changed, 24 insertions(+), 8 deletions(-) diff --git a/pkg/foundation/semaphore/semaphore.go b/pkg/foundation/semaphore/semaphore.go index e68081a40..2b535d9a5 100644 --- a/pkg/foundation/semaphore/semaphore.go +++ b/pkg/foundation/semaphore/semaphore.go @@ -23,26 +23,42 @@ import ( // Simple provides a way to bound concurrent access to a resource. It only // allows one caller to gain access at a time. type Simple struct { - waiters []waiter - front int - batch int64 + // waiters stores all waiters that have yet to acquire the semaphore. The + // slice will grow while new waiters are coming in, once tickets for all + // waiters are released the batch is incremented and the slice is reset. + waiters []waiter + // front is the index of the waiter that is next in line to acquire the + // semaphore. + front int + // batch is increased every time the batch is incremented. + batch int64 + // acquired is true if the semaphore is currently in the acquired state and + // needs to be released before it's acquired again. acquired bool + // released gets incremented every time a ticket is released. Once the count + // of waiters equals the number of released tickets the batch gets + // incremented and the waiters slice is reset. released int - mu sync.Mutex + // mu guards concurrent access to the fields above. + mu sync.Mutex } type waiter struct { - index int - ready chan struct{} // Closed when semaphore acquired. + // ready is closed when semaphore acquired. + ready chan struct{} - released bool + // acquired is set to true once the waiter acquires the semaphore. acquired bool + // released is set to true once the waiter releases the semaphore. + released bool } // Ticket reserves a place in the queue and can be used to acquire access to a // resource. type Ticket struct { + // index stores the index of the waiter in the semaphore. index int + // batch stores the batch in which this ticket was issued. batch int64 } @@ -54,7 +70,7 @@ func (s *Simple) Enqueue() Ticket { defer s.mu.Unlock() index := len(s.waiters) - w := waiter{index: index, ready: make(chan struct{})} + w := waiter{ready: make(chan struct{})} s.waiters = append(s.waiters, w) return Ticket{ From 3ebb744b777be31b9ec8d68ba3687243e6cc5630 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Mon, 11 Jul 2022 16:48:26 +0200 Subject: [PATCH 22/25] simplify implementation --- pkg/foundation/semaphore/semaphore.go | 129 +++++------------- .../semaphore/semaphore_bench_test.go | 13 +- pkg/foundation/semaphore/semaphore_test.go | 62 ++------- 3 files changed, 50 insertions(+), 154 deletions(-) diff --git a/pkg/foundation/semaphore/semaphore.go b/pkg/foundation/semaphore/semaphore.go index 2b535d9a5..b950dcd44 100644 --- a/pkg/foundation/semaphore/semaphore.go +++ b/pkg/foundation/semaphore/semaphore.go @@ -23,43 +23,19 @@ import ( // Simple provides a way to bound concurrent access to a resource. It only // allows one caller to gain access at a time. type Simple struct { - // waiters stores all waiters that have yet to acquire the semaphore. The - // slice will grow while new waiters are coming in, once tickets for all - // waiters are released the batch is incremented and the slice is reset. - waiters []waiter - // front is the index of the waiter that is next in line to acquire the - // semaphore. - front int - // batch is increased every time the batch is incremented. - batch int64 - // acquired is true if the semaphore is currently in the acquired state and - // needs to be released before it's acquired again. - acquired bool - // released gets incremented every time a ticket is released. Once the count - // of waiters equals the number of released tickets the batch gets - // incremented and the waiters slice is reset. - released int - // mu guards concurrent access to the fields above. + // lastTicket holds the last issued ticket. + lastTicket Ticket + // mu guards concurrent access to lastTicket. mu sync.Mutex } -type waiter struct { - // ready is closed when semaphore acquired. - ready chan struct{} - - // acquired is set to true once the waiter acquires the semaphore. - acquired bool - // released is set to true once the waiter releases the semaphore. - released bool -} - // Ticket reserves a place in the queue and can be used to acquire access to a // resource. type Ticket struct { - // index stores the index of the waiter in the semaphore. - index int - // batch stores the batch in which this ticket was issued. - batch int64 + // ready is closed when the ticket acquired the semaphore. + ready chan struct{} + // next is closed when the ticket is released. + next chan struct{} } // Enqueue reserves the next place in the queue and returns a Ticket used to @@ -69,87 +45,44 @@ func (s *Simple) Enqueue() Ticket { s.mu.Lock() defer s.mu.Unlock() - index := len(s.waiters) - w := waiter{ready: make(chan struct{})} - s.waiters = append(s.waiters, w) - - return Ticket{ - index: index, - batch: s.batch, + t := Ticket{ + ready: s.lastTicket.next, + next: make(chan struct{}), } -} - -// Acquire acquires the semaphore, blocking until resources are available. On -// success, returns nil. On failure, returns an error and leaves the semaphore -// unchanged. -func (s *Simple) Acquire(t Ticket) error { - s.mu.Lock() - if s.batch != t.batch { - s.mu.Unlock() - return cerrors.New("semaphore: invalid batch") - } - - w := s.waiters[t.index] - if w.acquired { - return cerrors.New("semaphore: can't acquire ticket that was already acquired") - } - - w.acquired = true // mark that Acquire was already called for this Ticket - s.waiters[t.index] = w - - if s.front == t.index && !s.acquired { - s.front++ - s.acquired = true - s.mu.Unlock() - return nil + if t.ready == nil { + // first time we create a ticket it will be already acquired + t.ready = make(chan struct{}) + close(t.ready) } - s.mu.Unlock() + s.lastTicket = t + return t +} - <-w.ready - return nil +// Acquire acquires the semaphore, blocking until resources are available. +// Returns nil if acquire was successful or ctx.Err if the context was cancelled +// in the meantime. +func (s *Simple) Acquire(t Ticket) { + <-t.ready } // Release releases the semaphore and notifies the next in line if any. // If the ticket was already released the function returns an error. After the // ticket is released it should be discarded. func (s *Simple) Release(t Ticket) error { - s.mu.Lock() - defer s.mu.Unlock() - - if s.batch != t.batch { - return cerrors.New("semaphore: invalid batch") - } - w := s.waiters[t.index] - if !w.acquired { + select { + case <-t.ready: + default: return cerrors.New("semaphore: can't release ticket that was not acquired") } - if w.released { + + select { + case <-t.next: return cerrors.New("semaphore: ticket already released") + default: } - w.released = true - s.waiters[t.index] = w - s.acquired = false - s.released++ - s.notifyWaiter() - if s.released == len(s.waiters) { - s.increaseBatch() + if t.next != nil { + close(t.next) } return nil } - -func (s *Simple) notifyWaiter() { - if len(s.waiters) > s.front { - w := s.waiters[s.front] - s.acquired = true - s.front++ - close(w.ready) - } -} - -func (s *Simple) increaseBatch() { - s.waiters = s.waiters[:0] - s.batch++ - s.front = 0 - s.released = 0 -} diff --git a/pkg/foundation/semaphore/semaphore_bench_test.go b/pkg/foundation/semaphore/semaphore_bench_test.go index 4e63dd130..67e1e0e14 100644 --- a/pkg/foundation/semaphore/semaphore_bench_test.go +++ b/pkg/foundation/semaphore/semaphore_bench_test.go @@ -28,14 +28,13 @@ func BenchmarkNewSem(b *testing.B) { } func BenchmarkEnqueueOneByOne(b *testing.B) { - for _, N := range []int{1, 2, 8, 64, 128} { - b.Run(fmt.Sprintf("acquire-%d", N), func(b *testing.B) { - b.ResetTimer() + for _, N := range []int{1, 2, 8, 64, 128, 1024} { + b.Run(fmt.Sprintf("ticket-count-%d", N), func(b *testing.B) { sem := &semaphore.Simple{} for i := 0; i < b.N; i++ { for j := 0; j < N; j++ { t := sem.Enqueue() - _ = sem.Acquire(t) + sem.Acquire(t) _ = sem.Release(t) } } @@ -44,8 +43,8 @@ func BenchmarkEnqueueOneByOne(b *testing.B) { } func BenchmarkEnqueueAll(b *testing.B) { - for _, N := range []int{1, 2, 8, 64, 128} { - b.Run(fmt.Sprintf("enqueue/release-%d", N), func(b *testing.B) { + for _, N := range []int{1, 2, 8, 64, 128, 1024} { + b.Run(fmt.Sprintf("ticket-count-%d", N), func(b *testing.B) { sem := &semaphore.Simple{} tickets := make([]semaphore.Ticket, N) b.ResetTimer() @@ -54,7 +53,7 @@ func BenchmarkEnqueueAll(b *testing.B) { tickets[j] = sem.Enqueue() } for j := 0; j < N; j++ { - _ = sem.Acquire(tickets[j]) + sem.Acquire(tickets[j]) _ = sem.Release(tickets[j]) } } diff --git a/pkg/foundation/semaphore/semaphore_test.go b/pkg/foundation/semaphore/semaphore_test.go index 021887c6c..42d3a84e8 100644 --- a/pkg/foundation/semaphore/semaphore_test.go +++ b/pkg/foundation/semaphore/semaphore_test.go @@ -29,13 +29,10 @@ const maxSleep = 1 * time.Millisecond func HammerSimple(sem *semaphore.Simple, loops int) { for i := 0; i < loops; i++ { tkn := sem.Enqueue() - err := sem.Acquire(tkn) - if err != nil { - panic(err) - } + sem.Acquire(tkn) //nolint:gosec // math/rand is good enough for a test time.Sleep(time.Duration(rand.Int63n(int64(maxSleep/time.Nanosecond))) * time.Nanosecond) - err = sem.Release(tkn) + err := sem.Release(tkn) if err != nil { panic(err) } @@ -64,7 +61,8 @@ func TestSimpleReleaseUnacquired(t *testing.T) { t.Parallel() w := &semaphore.Simple{} - tkn := w.Enqueue() + _ = w.Enqueue() // first ticket is automatically acquired + tkn := w.Enqueue() // next should be unacquired err := w.Release(tkn) if err == nil { t.Errorf("release of an unacquired ticket did not return an error") @@ -76,11 +74,8 @@ func TestSimpleReleaseTwice(t *testing.T) { w := &semaphore.Simple{} tkn := w.Enqueue() - err := w.Acquire(tkn) - if err != nil { - t.Errorf("unexpected error: %v", err) - } - err = w.Release(tkn) + w.Acquire(tkn) + err := w.Release(tkn) if err != nil { t.Errorf("release of an acquired ticket errored out: %v", err) } @@ -91,41 +86,19 @@ func TestSimpleReleaseTwice(t *testing.T) { } } -func TestSimpleAcquireTwice(t *testing.T) { - t.Parallel() - - w := &semaphore.Simple{} - tkn := w.Enqueue() - err := w.Acquire(tkn) - if err != nil { - t.Errorf("acquire of a ticket errored out: %v", err) - } - - err = w.Acquire(tkn) - if err == nil { - t.Errorf("acquire of an already acquired ticket did not return an error") - } -} - func TestSimpleAcquire(t *testing.T) { t.Parallel() sem := &semaphore.Simple{} tkn1 := sem.Enqueue() - err := sem.Acquire(tkn1) - if err != nil { - t.Errorf("unexpected error: %v", err) - } + sem.Acquire(tkn1) tkn2done := make(chan struct{}) go func() { defer close(tkn2done) tkn2 := sem.Enqueue() - err := sem.Acquire(tkn2) - if err != nil { - t.Errorf("unexpected error: %v", err) - } + sem.Acquire(tkn2) }() select { @@ -135,7 +108,7 @@ func TestSimpleAcquire(t *testing.T) { // tkn2 Acquire is blocking as expected } - err = sem.Release(tkn1) + err := sem.Release(tkn1) if err != nil { t.Errorf("unexpected error: %v", err) } @@ -161,10 +134,7 @@ func TestLargeAcquireDoesntStarve(t *testing.T) { wg.Add(int(n)) for i := n; i > 0; i-- { tkn := sem.Enqueue() - err := sem.Acquire(tkn) - if err != nil { - t.Errorf("unexpected error: %v", err) - } + sem.Acquire(tkn) go func() { defer func() { @@ -181,22 +151,16 @@ func TestLargeAcquireDoesntStarve(t *testing.T) { t.Errorf("unexpected error: %v", err) } tkn = sem.Enqueue() - err = sem.Acquire(tkn) - if err != nil { - t.Errorf("unexpected error: %v", err) - } + sem.Acquire(tkn) } }() } tkn := sem.Enqueue() - err := sem.Acquire(tkn) - if err != nil { - t.Errorf("unexpected error: %v", err) - } + sem.Acquire(tkn) running = false - err = sem.Release(tkn) + err := sem.Release(tkn) if err != nil { t.Errorf("unexpected error: %v", err) } From dc31319e60a59a99d4fa6f485fe4d3dae854188e Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Mon, 11 Jul 2022 17:06:19 +0200 Subject: [PATCH 23/25] update semaphore --- pkg/pipeline/stream/source_acker.go | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/pkg/pipeline/stream/source_acker.go b/pkg/pipeline/stream/source_acker.go index 5c354b25f..682151cf0 100644 --- a/pkg/pipeline/stream/source_acker.go +++ b/pkg/pipeline/stream/source_acker.go @@ -84,10 +84,7 @@ func (n *SourceAckerNode) registerAckHandler(msg *Message, ticket semaphore.Tick }) }() n.logger.Trace(msg.Ctx).Msg("acquiring semaphore for ack") - err = n.sem.Acquire(ticket) - if err != nil { - return cerrors.Errorf("could not acquire semaphore for ack: %w", err) - } + n.sem.Acquire(ticket) if n.fail { n.logger.Trace(msg.Ctx).Msg("blocking forwarding of ack to source connector, because another message failed to be acked/nacked") @@ -113,10 +110,7 @@ func (n *SourceAckerNode) registerNackHandler(msg *Message, ticket semaphore.Tic }) }() n.logger.Trace(msg.Ctx).Msg("acquiring semaphore for nack") - err = n.sem.Acquire(ticket) - if err != nil { - return cerrors.Errorf("could not acquire semaphore for nack: %w", err) - } + n.sem.Acquire(ticket) if n.fail { n.logger.Trace(msg.Ctx).Msg("blocking forwarding of nack to DLQ handler, because another message failed to be acked/nacked") From 5a6e8a3858931f2b111230981ea4a2ca8abce795 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Mon, 11 Jul 2022 17:07:14 +0200 Subject: [PATCH 24/25] update param name --- pkg/pipeline/stream/message.go | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/pkg/pipeline/stream/message.go b/pkg/pipeline/stream/message.go index 5a2c761e3..69be73212 100644 --- a/pkg/pipeline/stream/message.go +++ b/pkg/pipeline/stream/message.go @@ -129,7 +129,7 @@ func (m *Message) ID() string { // any status change of the message. This function can only be called if the // message status is open, otherwise it panics. Handlers are called in the // reverse order of how they were registered. -func (m *Message) RegisterStatusHandler(mw StatusChangeHandler) { +func (m *Message) RegisterStatusHandler(h StatusChangeHandler) { m.init() m.handlerGuard.Lock() defer m.handlerGuard.Unlock() @@ -141,7 +141,7 @@ func (m *Message) RegisterStatusHandler(mw StatusChangeHandler) { next := m.handler m.handler = func(msg *Message, change StatusChange) error { // all handlers are called and errors collected - err1 := mw(msg, change) + err1 := h(msg, change) err2 := next(msg, change) return multierror.Append(err1, err2) } @@ -150,24 +150,24 @@ func (m *Message) RegisterStatusHandler(mw StatusChangeHandler) { // RegisterAckHandler is used to register a function that will be called when // the message is acked. This function can only be called if the message status // is open, otherwise it panics. -func (m *Message) RegisterAckHandler(mw AckHandler) { +func (m *Message) RegisterAckHandler(h AckHandler) { m.RegisterStatusHandler(func(msg *Message, change StatusChange) error { if change.New != MessageStatusAcked { return nil // skip } - return mw(msg) + return h(msg) }) } // RegisterNackHandler is used to register a function that will be called when // the message is nacked. This function can only be called if the message status // is open, otherwise it panics. -func (m *Message) RegisterNackHandler(mw NackHandler) { +func (m *Message) RegisterNackHandler(h NackHandler) { m.RegisterStatusHandler(func(msg *Message, change StatusChange) error { if change.New != MessageStatusNacked { return nil // skip } - return mw(msg, change.Reason) + return h(msg, change.Reason) }) m.hasNackHandler = true } @@ -175,12 +175,12 @@ func (m *Message) RegisterNackHandler(mw NackHandler) { // RegisterDropHandler is used to register a function that will be called when // the message is dropped. This function can only be called if the message // status is open, otherwise it panics. -func (m *Message) RegisterDropHandler(mw DropHandler) { +func (m *Message) RegisterDropHandler(h DropHandler) { m.RegisterStatusHandler(func(msg *Message, change StatusChange) error { if change.New != MessageStatusDropped { return nil } - mw(msg, change.Reason) + h(msg, change.Reason) return nil }) } From 54a65d1779081576e20f313513250db365812e5f Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Lovro=20Ma=C5=BEgon?= Date: Mon, 11 Jul 2022 17:12:31 +0200 Subject: [PATCH 25/25] remove redundant if clause --- pkg/foundation/semaphore/semaphore.go | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/pkg/foundation/semaphore/semaphore.go b/pkg/foundation/semaphore/semaphore.go index b950dcd44..e51cfd6b4 100644 --- a/pkg/foundation/semaphore/semaphore.go +++ b/pkg/foundation/semaphore/semaphore.go @@ -81,8 +81,6 @@ func (s *Simple) Release(t Ticket) error { default: } - if t.next != nil { - close(t.next) - } + close(t.next) return nil }