From 0ac34bcee3c27e82650e1fde32dcc3c0eefa71a4 Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Tue, 8 Feb 2022 13:53:23 -0500 Subject: [PATCH 01/23] Fix batch pin index calculation logic and improve logging Signed-off-by: Peter Broadhurst --- internal/batch/batch_manager.go | 9 ++-- internal/broadcast/message.go | 2 +- internal/events/aggregator.go | 45 +++++++++++-------- internal/events/aggregator_batch_state.go | 7 +-- .../events/aggregator_batch_state_test.go | 2 +- internal/events/aggregator_test.go | 2 + internal/privatemessaging/message.go | 2 +- 7 files changed, 41 insertions(+), 28 deletions(-) diff --git a/internal/batch/batch_manager.go b/internal/batch/batch_manager.go index d1dbdfe9a0..1e0c7f19b6 100644 --- a/internal/batch/batch_manager.go +++ b/internal/batch/batch_manager.go @@ -51,7 +51,7 @@ func NewBatchManager(ctx context.Context, ni sysmessaging.LocalNodeInfo, di data startupOffsetRetryAttempts: config.GetInt(config.OrchestratorStartupAttempts), dispatchers: make(map[fftypes.MessageType]*dispatcher), shoulderTap: make(chan bool, 1), - newMessages: make(chan int64, readPageSize), + newMessages: make(chan int64), sequencerClosed: make(chan struct{}), retry: &retry.Retry{ InitialDelay: config.GetDuration(config.BatchRetryInitDelay), @@ -219,7 +219,7 @@ func (bm *batchManager) assembleMessageData(msg *fftypes.Message) (data []*fftyp if !foundAll { return nil, i18n.NewError(bm.ctx, i18n.MsgDataNotFound, msg.Header.ID) } - log.L(bm.ctx).Infof("Detected new batch-pinned message %s", msg.Header.ID) + log.L(bm.ctx).Infof("Detected new batch-pinned message %s sequence=%d", msg.Header.ID, msg.Sequence) return data, nil } @@ -227,10 +227,12 @@ func (bm *batchManager) markRewind(rewindTo int64) { bm.rewindMux.Lock() // Make sure we only rewind backwards - as we might get multiple shoulder taps // for different message sequences during a single poll cycle. - if bm.rewindTo < 0 || rewindTo < bm.rewindTo { + previousRewind := bm.rewindTo + if previousRewind < 0 || rewindTo < previousRewind { bm.rewindTo = rewindTo } bm.rewindMux.Unlock() + log.L(bm.ctx).Debugf("Marking rewind to sequence=%d (previous=%d)", rewindTo, previousRewind) } func (bm *batchManager) popRewind() int64 { @@ -245,6 +247,7 @@ func (bm *batchManager) readPage() ([]*fftypes.Message, error) { rewindTo := bm.popRewind() if rewindTo >= 0 && rewindTo < bm.offset { + log.L(bm.ctx).Debugf("Rewinding to sequence=%d", rewindTo) if err := bm.updateOffset(true, rewindTo); err != nil { return nil, err } diff --git a/internal/broadcast/message.go b/internal/broadcast/message.go index 81a93eb919..673f91e0d4 100644 --- a/internal/broadcast/message.go +++ b/internal/broadcast/message.go @@ -166,7 +166,7 @@ func (s *broadcastSender) sendInternal(ctx context.Context, method sendMethod) ( if err := s.mgr.database.UpsertMessage(ctx, &s.msg.Message, database.UpsertOptimizationNew); err != nil { return err } - log.L(ctx).Infof("Sent broadcast message %s:%s", s.msg.Header.Namespace, s.msg.Header.ID) + log.L(ctx).Infof("Sent broadcast message %s:%s sequence=%d", s.msg.Header.Namespace, s.msg.Header.ID, s.msg.Sequence) return err } diff --git a/internal/events/aggregator.go b/internal/events/aggregator.go index 0e1c231954..8a0bce0146 100644 --- a/internal/events/aggregator.go +++ b/internal/events/aggregator.go @@ -183,6 +183,20 @@ func (ag *aggregator) getPins(ctx context.Context, filter database.Filter) ([]ff return ls, err } +func (ag *aggregator) extractBatchMessagePin(batch *fftypes.Batch, requiredIndex int64) (totalBatchPins int64, msg *fftypes.Message, msgBaseIndex int64) { + for _, batchMsg := range batch.Payload.Messages { + batchMsgBaseIdx := totalBatchPins + for i := 0; i < len(batchMsg.Header.Topics); i++ { + if totalBatchPins == requiredIndex { + msg = batchMsg + msgBaseIndex = batchMsgBaseIdx + } + totalBatchPins++ + } + } + return totalBatchPins, msg, msgBaseIndex +} + func (ag *aggregator) processPins(ctx context.Context, pins []*fftypes.Pin, state *batchState) (err error) { l := log.L(ctx) @@ -192,7 +206,6 @@ func (ag *aggregator) processPins(ctx context.Context, pins []*fftypes.Pin, stat // We must check all the contexts in the message, and mark them dispatched together. dupMsgCheck := make(map[fftypes.UUID]bool) for _, pin := range pins { - l.Debugf("Aggregating pin %.10d batch=%s hash=%s masked=%t", pin.Sequence, pin.Batch, pin.Hash, pin.Masked) if batch == nil || *batch.ID != *pin.Batch { batch, err = ag.database.GetBatchByID(ctx, pin.Batch) @@ -206,23 +219,14 @@ func (ag *aggregator) processPins(ctx context.Context, pins []*fftypes.Pin, stat } // Extract the message from the batch - where the index is of a topic within a message - var msg *fftypes.Message - var i int64 = -1 - var msgBaseIndex int64 - for iM := 0; i < pin.Index && iM < len(batch.Payload.Messages); iM++ { - msg = batch.Payload.Messages[iM] - msgBaseIndex = i - for iT := 0; i < pin.Index && iT < len(msg.Header.Topics); iT++ { - i++ - } - } - - if i < pin.Index { - l.Errorf("Batch %s does not have message-topic index %d - pin %s is invalid", pin.Batch, pin.Index, pin.Hash) + batchPinCount, msg, msgBaseIndex := ag.extractBatchMessagePin(batch, pin.Index) + if msg == nil { + l.Errorf("Pin %.10d outside of range: batch=%s pinCount=%d pinIndex=%d hash=%s masked=%t", pin.Sequence, pin.Batch, batchPinCount, pin.Index, pin.Hash, pin.Masked) continue } - l.Tracef("Batch %s message %d: %+v", batch.ID, pin.Index, msg) - if msg == nil || msg.Header.ID == nil { + + l.Debugf("Aggregating pin %.10d batch=%s msg=%s pinIndex=%d msgBaseIndex=%d hash=%s masked=%t", pin.Sequence, pin.Batch, msg.Header.ID, pin.Index, msgBaseIndex, pin.Hash, pin.Masked) + if msg.Header.ID == nil { l.Errorf("null message entry %d in batch '%s'", pin.Index, batch.ID) continue } @@ -243,6 +247,8 @@ func (ag *aggregator) processPins(ctx context.Context, pins []*fftypes.Pin, stat } func (ag *aggregator) processMessage(ctx context.Context, batch *fftypes.Batch, pin *fftypes.Pin, msgBaseIndex int64, msg *fftypes.Message, state *batchState) (err error) { + l := log.L(ctx) + // Check if it's ready to be processed unmaskedContexts := make([]*fftypes.Bytes32, 0, len(msg.Header.Topics)) nextPins := make([]*nextPinState, 0, len(msg.Header.Topics)) @@ -250,14 +256,14 @@ func (ag *aggregator) processMessage(ctx context.Context, batch *fftypes.Batch, // Private messages have one or more masked "pin" hashes that allow us to work // out if it's the next message in the sequence, given the previous messages if msg.Header.Group == nil || len(msg.Pins) == 0 || len(msg.Header.Topics) != len(msg.Pins) { - log.L(ctx).Errorf("Message '%s' in batch '%s' has invalid pin data pins=%v topics=%v", msg.Header.ID, batch.ID, msg.Pins, msg.Header.Topics) + l.Errorf("Message '%s' in batch '%s' has invalid pin data pins=%v topics=%v", msg.Header.ID, batch.ID, msg.Pins, msg.Header.Topics) return nil } for i, pinStr := range msg.Pins { var msgContext fftypes.Bytes32 err := msgContext.UnmarshalText([]byte(pinStr)) if err != nil { - log.L(ctx).Errorf("Message '%s' in batch '%s' has invalid pin at index %d: '%s'", msg.Header.ID, batch.ID, i, pinStr) + l.Errorf("Message '%s' in batch '%s' has invalid pin at index %d: '%s'", msg.Header.ID, batch.ID, i, pinStr) return nil } nextPin, err := state.CheckMaskedContextReady(ctx, msg, msg.Header.Topics[i], pin.Sequence, &msgContext) @@ -272,7 +278,7 @@ func (ag *aggregator) processMessage(ctx context.Context, batch *fftypes.Batch, h.Write([]byte(topic)) msgContext := fftypes.HashResult(h) unmaskedContexts = append(unmaskedContexts, msgContext) - ready, err := state.CheckUnmaskedContextReady(ctx, *msgContext, msg, msg.Header.Topics[i], pin.Sequence) + ready, err := state.CheckUnmaskedContextReady(ctx, msgContext, msg, msg.Header.Topics[i], pin.Sequence) if err != nil || !ready { return err } @@ -280,6 +286,7 @@ func (ag *aggregator) processMessage(ctx context.Context, batch *fftypes.Batch, } + l.Debugf("Attempt dispatch msg=%s broadcastContexts=%v privatePins=%v", msg.Header.ID, unmaskedContexts, msg.Pins) dispatched, err := ag.attemptMessageDispatch(ctx, msg, state) if err != nil { return err diff --git a/internal/events/aggregator_batch_state.go b/internal/events/aggregator_batch_state.go index 287c5cd561..917dd6c39e 100644 --- a/internal/events/aggregator_batch_state.go +++ b/internal/events/aggregator_batch_state.go @@ -137,12 +137,12 @@ func (bs *batchState) RunFinalize(ctx context.Context) error { return bs.flushPins(ctx) } -func (bs *batchState) CheckUnmaskedContextReady(ctx context.Context, contextUnmasked fftypes.Bytes32, msg *fftypes.Message, topic string, firstMsgPinSequence int64) (bool, error) { +func (bs *batchState) CheckUnmaskedContextReady(ctx context.Context, contextUnmasked *fftypes.Bytes32, msg *fftypes.Message, topic string, firstMsgPinSequence int64) (bool, error) { - ucs, found := bs.unmaskedContexts[contextUnmasked] + ucs, found := bs.unmaskedContexts[*contextUnmasked] if !found { ucs = &contextState{blockedBy: -1} - bs.unmaskedContexts[contextUnmasked] = ucs + bs.unmaskedContexts[*contextUnmasked] = ucs // We need to check there's no earlier sequences with the same unmasked context fb := database.PinQueryFactory.NewFilterLimit(ctx, 1) // only need the first one @@ -261,6 +261,7 @@ func (bs *batchState) flushPins(ctx context.Context) error { fb.Gte("index", dm.firstPinIndex), fb.Lte("index", dm.lastPinIndex), ) + log.L(ctx).Debugf("Marking message dispatched batch=%s msg=%s firstIndex=%d lastIndex=%d", dm.batchID, dm.msgID, dm.firstPinIndex, dm.lastPinIndex) update := database.PinQueryFactory.NewUpdate(ctx).Set("dispatched", true) if err := bs.database.UpdatePins(ctx, filter, update); err != nil { return err diff --git a/internal/events/aggregator_batch_state_test.go b/internal/events/aggregator_batch_state_test.go index e846082b7c..c35ec69783 100644 --- a/internal/events/aggregator_batch_state_test.go +++ b/internal/events/aggregator_batch_state_test.go @@ -52,7 +52,7 @@ func TestSetContextBlockedByNoState(t *testing.T) { unmaskedContext := fftypes.NewRandB32() bs.SetContextBlockedBy(ag.ctx, *unmaskedContext, 10) - ready, err := bs.CheckUnmaskedContextReady(ag.ctx, *unmaskedContext, &fftypes.Message{}, "topic1", 1) + ready, err := bs.CheckUnmaskedContextReady(ag.ctx, unmaskedContext, &fftypes.Message{}, "topic1", 1) assert.NoError(t, err) assert.False(t, ready) } diff --git a/internal/events/aggregator_test.go b/internal/events/aggregator_test.go index fd9c0197b5..5ea592574f 100644 --- a/internal/events/aggregator_test.go +++ b/internal/events/aggregator_test.go @@ -24,6 +24,7 @@ import ( "github.com/hyperledger/firefly/internal/config" "github.com/hyperledger/firefly/internal/definitions" + "github.com/hyperledger/firefly/internal/log" "github.com/hyperledger/firefly/mocks/databasemocks" "github.com/hyperledger/firefly/mocks/datamocks" "github.com/hyperledger/firefly/mocks/definitionsmocks" @@ -166,6 +167,7 @@ func TestAggregationMaskedZeroNonceMatch(t *testing.T) { } func TestAggregationMaskedNextSequenceMatch(t *testing.T) { + log.SetLevel("debug") ag, cancel := newTestAggregator() defer cancel() diff --git a/internal/privatemessaging/message.go b/internal/privatemessaging/message.go index 3376b1265d..2cb7bef6a4 100644 --- a/internal/privatemessaging/message.go +++ b/internal/privatemessaging/message.go @@ -185,7 +185,7 @@ func (s *messageSender) sendInternal(ctx context.Context, method sendMethod) err if err := s.mgr.database.UpsertMessage(ctx, &s.msg.Message, database.UpsertOptimizationNew); err != nil { return err } - log.L(ctx).Infof("Sent private message %s:%s", s.msg.Header.Namespace, s.msg.Header.ID) + log.L(ctx).Infof("Sent private message %s:%s sequence=%d", s.msg.Header.Namespace, s.msg.Header.ID, s.msg.Sequence) if method == methodSendImmediate { if err := s.sendUnpinned(ctx); err != nil { From 8bd5fd3d47b0920b9d91481fbe6d707570660268 Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Tue, 8 Feb 2022 19:59:25 -0500 Subject: [PATCH 02/23] Rewind offset to one before the message that popped Signed-off-by: Peter Broadhurst --- internal/batch/batch_manager.go | 2 +- internal/batch/batch_manager_test.go | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/internal/batch/batch_manager.go b/internal/batch/batch_manager.go index 1e0c7f19b6..a25520c4a5 100644 --- a/internal/batch/batch_manager.go +++ b/internal/batch/batch_manager.go @@ -338,7 +338,7 @@ func (bm *batchManager) newEventNotifications() { return } l.Debugf("New message sequence notification: %d", m) - bm.markRewind(m) + bm.markRewind(m - 1) case <-bm.ctx.Done(): l.Debugf("Exiting due to cancelled context") return diff --git a/internal/batch/batch_manager_test.go b/internal/batch/batch_manager_test.go index 88462cb74a..68f0adac50 100644 --- a/internal/batch/batch_manager_test.go +++ b/internal/batch/batch_manager_test.go @@ -512,8 +512,8 @@ func TestWaitConsumesMessagesAndDoesNotBlock(t *testing.T) { } // And should generate a shoulder tap <-bm.shoulderTap - // And a rewind - assert.Equal(t, int64(12345), bm.popRewind()) + // And a rewind to one before that sequence (as readPage is greater-than) + assert.Equal(t, int64(12344), bm.popRewind()) bm.Close() } From 5bdaad04e414f4f215eb08570e6c0780c3d6693c Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Wed, 9 Feb 2022 01:14:56 -0500 Subject: [PATCH 03/23] Re-work batch logic for simplicity, efficiency, and restart recovery Signed-off-by: Peter Broadhurst --- internal/batch/batch_manager.go | 125 ++-- internal/batch/batch_manager_test.go | 91 +-- internal/batch/batch_processor.go | 544 ++++++++++-------- internal/batch/batch_processor_test.go | 15 +- internal/database/sqlcommon/batch_sql.go | 4 +- internal/database/sqlcommon/batch_sql_test.go | 20 +- internal/events/persist_batch.go | 2 +- mocks/databasemocks/plugin.go | 10 +- pkg/database/plugin.go | 5 +- pkg/fftypes/message.go | 2 + 10 files changed, 365 insertions(+), 453 deletions(-) diff --git a/internal/batch/batch_manager.go b/internal/batch/batch_manager.go index a25520c4a5..c7552c02df 100644 --- a/internal/batch/batch_manager.go +++ b/internal/batch/batch_manager.go @@ -46,11 +46,11 @@ func NewBatchManager(ctx context.Context, ni sysmessaging.LocalNodeInfo, di data ni: ni, database: di, data: dm, + readOffset: -1, // On restart we trawl for all ready messages readPageSize: uint64(readPageSize), messagePollTimeout: config.GetDuration(config.BatchManagerReadPollTimeout), startupOffsetRetryAttempts: config.GetInt(config.OrchestratorStartupAttempts), dispatchers: make(map[fftypes.MessageType]*dispatcher), - shoulderTap: make(chan bool, 1), newMessages: make(chan int64), sequencerClosed: make(chan struct{}), retry: &retry.Retry{ @@ -76,18 +76,16 @@ type batchManager struct { database database.Plugin data data.Manager dispatchers map[fftypes.MessageType]*dispatcher - shoulderTap chan bool newMessages chan int64 sequencerClosed chan struct{} retry *retry.Retry offsetID int64 - offset int64 + recoveryOffset int64 + readOffset int64 closed bool readPageSize uint64 messagePollTimeout time.Duration startupOffsetRetryAttempts int - rewindMux sync.Mutex - rewindTo int64 } type DispatchHandler func(context.Context, *fftypes.Batch, []*fftypes.Bytes32) error @@ -118,12 +116,9 @@ func (bm *batchManager) RegisterDispatcher(msgTypes []fftypes.MessageType, handl } func (bm *batchManager) Start() error { - bm.markRewind(-1) - if err := bm.restoreOffset(); err != nil { return err } - go bm.newEventNotifications() go bm.messageSequencer() return nil } @@ -148,17 +143,12 @@ func (bm *batchManager) restoreOffset() (err error) { } } bm.offsetID = offset.RowID - bm.offset = offset.Current - log.L(bm.ctx).Infof("Batch manager restored offset %d", bm.offset) + bm.readOffset = offset.Current + bm.recoveryOffset = offset.Current + log.L(bm.ctx).Infof("Batch manager restored offset %d", offset.Current) return nil } -func (bm *batchManager) removeProcessor(dispatcher *dispatcher, key string) { - dispatcher.mux.Lock() - delete(dispatcher.processors, key) - dispatcher.mux.Unlock() -} - func (bm *batchManager) getProcessor(batchType fftypes.MessageType, group *fftypes.Bytes32, namespace string, identity *fftypes.Identity) (*batchProcessor, error) { dispatcher, ok := bm.dispatchers[batchType] if !ok { @@ -178,9 +168,6 @@ func (bm *batchManager) getProcessor(batchType fftypes.MessageType, group *fftyp identity: *identity, group: group, dispatch: dispatcher.handler, - processorQuiescing: func() { - bm.removeProcessor(dispatcher, key) - }, }, bm.retry, ) @@ -192,10 +179,12 @@ func (bm *batchManager) getProcessor(batchType fftypes.MessageType, group *fftyp } func (bm *batchManager) Close() { + var processors []*batchProcessor if bm != nil && !bm.closed { for _, d := range bm.dispatchers { d.mux.Lock() for _, p := range d.processors { + processors = append(processors, p) p.close() } d.mux.Unlock() @@ -204,6 +193,9 @@ func (bm *batchManager) Close() { close(bm.newMessages) } bm = nil + for _, p := range processors { + <-p.done + } } func (bm *batchManager) assembleMessageData(msg *fftypes.Message) (data []*fftypes.Data, err error) { @@ -223,43 +215,14 @@ func (bm *batchManager) assembleMessageData(msg *fftypes.Message) (data []*fftyp return data, nil } -func (bm *batchManager) markRewind(rewindTo int64) { - bm.rewindMux.Lock() - // Make sure we only rewind backwards - as we might get multiple shoulder taps - // for different message sequences during a single poll cycle. - previousRewind := bm.rewindTo - if previousRewind < 0 || rewindTo < previousRewind { - bm.rewindTo = rewindTo - } - bm.rewindMux.Unlock() - log.L(bm.ctx).Debugf("Marking rewind to sequence=%d (previous=%d)", rewindTo, previousRewind) -} - -func (bm *batchManager) popRewind() int64 { - bm.rewindMux.Lock() - rewindTo := bm.rewindTo - bm.rewindTo = -1 - bm.rewindMux.Unlock() - return rewindTo -} - func (bm *batchManager) readPage() ([]*fftypes.Message, error) { - rewindTo := bm.popRewind() - if rewindTo >= 0 && rewindTo < bm.offset { - log.L(bm.ctx).Debugf("Rewinding to sequence=%d", rewindTo) - if err := bm.updateOffset(true, rewindTo); err != nil { - return nil, err - } - } - var msgs []*fftypes.Message err := bm.retry.Do(bm.ctx, "retrieve messages", func(attempt int) (retry bool, err error) { fb := database.MessageQueryFactory.NewFilterLimit(bm.ctx, bm.readPageSize) msgs, _, err = bm.database.GetMessages(bm.ctx, fb.And( - fb.Gt("sequence", bm.offset), + fb.Gt("sequence", bm.readOffset), fb.Eq("state", fftypes.MessageStateReady), - fb.Eq("txtype", fftypes.TransactionTypeBatchPin), ).Sort("sequence").Limit(bm.readPageSize)) if err != nil { return !bm.closed, err // Retry indefinitely, until closed (or context cancelled) @@ -314,9 +277,8 @@ func (bm *batchManager) messageSequencer() { } } - if !bm.closed { - _ = bm.updateOffset(true, msgs[len(msgs)-1].Sequence) - } + // Next time round only read after the messages we just processed (unless we get a tap to rewind) + bm.readOffset = msgs[len(msgs)-1].Sequence } // Wait to be woken again @@ -326,39 +288,34 @@ func (bm *batchManager) messageSequencer() { } } -// newEventNotifications just consumes new messags, logs them, then ensures there's a shoulderTap -// in the channel - without blocking. This is important as we must not block the notifier -func (bm *batchManager) newEventNotifications() { - l := log.L(bm.ctx).WithField("role", "batch-newmessages") - for { +func (bm *batchManager) waitForShoulderTapOrPollTimeout() { + l := log.L(bm.ctx) + + // Drain any new message notifications, moving back our + // readOffset as required + newMessages := false + checkingMessages := true + for checkingMessages { select { - case m, ok := <-bm.newMessages: - if !ok { - l.Debugf("Exiting due to close") - return + case seq := <-bm.newMessages: + l.Debugf("Notification of message %d", seq) + if (seq - 1) < bm.readOffset { + bm.readOffset = seq - 1 } - l.Debugf("New message sequence notification: %d", m) - bm.markRewind(m - 1) - case <-bm.ctx.Done(): - l.Debugf("Exiting due to cancelled context") - return - } - // Do not block sending to the shoulderTap - as it can only contain one - select { - case bm.shoulderTap <- true: + newMessages = true default: + checkingMessages = false } } -} + if newMessages { + return + } -func (bm *batchManager) waitForShoulderTapOrPollTimeout() { - l := log.L(bm.ctx) + // Otherwise set a timeout timeout := time.NewTimer(bm.messagePollTimeout) select { case <-timeout.C: l.Debugf("Woken after poll timeout") - case <-bm.shoulderTap: - l.Debugf("Woken for trigger for messages") case <-bm.ctx.Done(): l.Debugf("Exiting due to cancelled context") bm.Close() @@ -366,22 +323,6 @@ func (bm *batchManager) waitForShoulderTapOrPollTimeout() { } } -func (bm *batchManager) updateOffset(infiniteRetry bool, newOffset int64) (err error) { - l := log.L(bm.ctx) - return bm.retry.Do(bm.ctx, "update offset", func(attempt int) (retry bool, err error) { - bm.offset = newOffset - u := database.OffsetQueryFactory.NewUpdate(bm.ctx).Set("current", bm.offset) - err = bm.database.UpdateOffset(bm.ctx, bm.offsetID, u) - if err != nil { - l.Errorf("Batch persist attempt %d failed: %s", attempt, err) - stillRetrying := infiniteRetry || (attempt <= bm.startupOffsetRetryAttempts) - return !bm.closed && stillRetrying, err - } - l.Infof("Batch manager committed offset %d", newOffset) - return false, nil - }) -} - func (bm *batchManager) dispatchMessage(dispatched chan *batchDispatch, msg *fftypes.Message, data ...*fftypes.Data) error { l := log.L(bm.ctx) processor, err := bm.getProcessor(msg.Header.Type, msg.Header.Group, msg.Header.Namespace, &msg.Header.Identity) @@ -409,6 +350,6 @@ func (bm *batchManager) WaitStop() { d.mux.Unlock() } for _, p := range processors { - p.waitClosed() + p.close() } } diff --git a/internal/batch/batch_manager_test.go b/internal/batch/batch_manager_test.go index 68f0adac50..fd2b800f88 100644 --- a/internal/batch/batch_manager_test.go +++ b/internal/batch/batch_manager_test.go @@ -22,7 +22,6 @@ import ( "testing" "time" - "github.com/hyperledger/firefly/internal/config" "github.com/hyperledger/firefly/internal/log" "github.com/hyperledger/firefly/mocks/databasemocks" "github.com/hyperledger/firefly/mocks/datamocks" @@ -271,7 +270,7 @@ func TestInitRestoreExistingOffset(t *testing.T) { defer bm.Close() err = bm.Start() assert.NoError(t, err) - assert.Equal(t, int64(12345), bm.(*batchManager).offset) + assert.Equal(t, int64(12345), bm.(*batchManager).readOffset) } func TestInitFailCannotRestoreOffset(t *testing.T) { @@ -498,74 +497,6 @@ func TestWaitForPollTimeout(t *testing.T) { bm.(*batchManager).waitForShoulderTapOrPollTimeout() } -func TestWaitConsumesMessagesAndDoesNotBlock(t *testing.T) { - config.Reset() - mdi := &databasemocks.Plugin{} - mdm := &datamocks.Manager{} - mni := &sysmessagingmocks.LocalNodeInfo{} - bmi, _ := NewBatchManager(context.Background(), mni, mdi, mdm) - bm := bmi.(*batchManager) - _ = bm.popRewind() - go bm.newEventNotifications() - for i := 0; i < int(bm.readPageSize); i++ { - bm.NewMessages() <- 12345 - } - // And should generate a shoulder tap - <-bm.shoulderTap - // And a rewind to one before that sequence (as readPage is greater-than) - assert.Equal(t, int64(12344), bm.popRewind()) - bm.Close() -} - -func TestReadPageWithRewindSuccess(t *testing.T) { - config.Reset() - mdi := &databasemocks.Plugin{} - mdm := &datamocks.Manager{} - mni := &sysmessagingmocks.LocalNodeInfo{} - - msg := &fftypes.Message{Header: fftypes.MessageHeader{ID: fftypes.NewUUID()}} - mdi.On("UpdateOffset", mock.Anything, int64(0), mock.Anything).Return(nil) - mdi.On("GetMessages", mock.Anything, mock.MatchedBy(func(filter database.Filter) bool { - f, _ := filter.Finalize() - assert.Contains(t, f.String(), "12345") - return true - })).Return([]*fftypes.Message{msg}, nil, nil) - - bmi, _ := NewBatchManager(context.Background(), mni, mdi, mdm) - bm := bmi.(*batchManager) - _ = bm.popRewind() - bm.offset = 44444 - bm.markRewind(22222) - bm.markRewind(12345) - bm.markRewind(33333) - msgs, err := bm.readPage() - assert.NoError(t, err) - assert.Len(t, msgs, 1) - bm.Close() - - mdi.AssertExpectations(t) -} - -func TestReadPageWithRewindFail(t *testing.T) { - config.Reset() - mdi := &databasemocks.Plugin{} - mdm := &datamocks.Manager{} - mni := &sysmessagingmocks.LocalNodeInfo{} - - mdi.On("UpdateOffset", mock.Anything, int64(0), mock.Anything).Return(fmt.Errorf("pop")) - - bmi, _ := NewBatchManager(context.Background(), mni, mdi, mdm) - bm := bmi.(*batchManager) - bm.Close() - - bm.offset = 22222 - bm.markRewind(12345) - _, err := bm.readPage() - assert.EqualError(t, err, "pop") - - mdi.AssertExpectations(t) -} - func TestAssembleMessageDataNilData(t *testing.T) { mdi := &databasemocks.Plugin{} mdm := &datamocks.Manager{} @@ -582,17 +513,6 @@ func TestAssembleMessageDataNilData(t *testing.T) { assert.Regexp(t, "FF10133", err) } -func TestAssembleMessageDataClosed(t *testing.T) { - mdi := &databasemocks.Plugin{} - mdm := &datamocks.Manager{} - mni := &sysmessagingmocks.LocalNodeInfo{} - bm, _ := NewBatchManager(context.Background(), mni, mdi, mdm) - bm.(*batchManager).retry.MaximumDelay = 1 * time.Microsecond - mdi.On("UpdateOffset", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(fmt.Errorf("pop")) - err := bm.(*batchManager).updateOffset(false, 10) - assert.EqualError(t, err, "pop") -} - func TestGetMessageDataFail(t *testing.T) { mdi := &databasemocks.Plugin{} mdm := &datamocks.Manager{} @@ -628,12 +548,3 @@ func TestGetMessageNotFound(t *testing.T) { }) assert.Regexp(t, "FF10133", err) } - -func TestWaitForShoulderTap(t *testing.T) { - mdi := &databasemocks.Plugin{} - mdm := &datamocks.Manager{} - mni := &sysmessagingmocks.LocalNodeInfo{} - bm, _ := NewBatchManager(context.Background(), mni, mdi, mdm) - bm.(*batchManager).shoulderTap <- true - bm.(*batchManager).waitForShoulderTapOrPollTimeout() -} diff --git a/internal/batch/batch_processor.go b/internal/batch/batch_processor.go index 5fa2c62235..671aa043ed 100644 --- a/internal/batch/batch_processor.go +++ b/internal/batch/batch_processor.go @@ -22,6 +22,8 @@ import ( "database/sql/driver" "encoding/binary" "fmt" + "math" + "sync" "time" "github.com/hyperledger/firefly/internal/log" @@ -36,7 +38,6 @@ type batchWork struct { msg *fftypes.Message data []*fftypes.Data dispatched chan *batchDispatch - abandoned bool } type batchDispatch struct { @@ -46,50 +47,75 @@ type batchDispatch struct { type batchProcessorConf struct { Options - namespace string - identity fftypes.Identity - group *fftypes.Bytes32 - dispatch DispatchHandler - processorQuiescing func() + namespace string + identity fftypes.Identity + group *fftypes.Bytes32 + dispatch DispatchHandler + requestQuiesce func() bool +} + +// FlushStatus is an object that can be returned on REST queries to understand the status +// of the batch processor +type FlushStatus struct { + LastFlushTime *fftypes.FFTime `json:"lastFlushStartTime"` + Flushing *fftypes.UUID `json:"flushing,omitempty"` + LastFlushError string `json:"lastFlushError,omitempty"` + LastFlushErrorTime *fftypes.FFTime `json:"lastFlushErrorTime,omitempty"` + AverageBatchBytes int64 `json:"averageBatchBytes"` + AverageBatchMessages float64 `json:"averageBatchMessages"` + AverageBatchData float64 `json:"averageBatchData"` + AverageFlushTimeMS int64 `json:"averageFlushTimeMS"` + + totalBatches int64 + totalBytesFlushed int64 + totalMessagesFlushed int64 + totalDataFlushed int64 + totalFlushDuration time.Duration } type batchProcessor struct { - ctx context.Context - ni sysmessaging.LocalNodeInfo - database database.Plugin - txHelper txcommon.Helper - name string - cancelCtx func() - closed bool - newWork chan *batchWork - persistWork chan *batchWork - sealBatch chan bool - batchSealed chan bool - retry *retry.Retry - conf *batchProcessorConf + ctx context.Context + ni sysmessaging.LocalNodeInfo + database database.Plugin + txHelper txcommon.Helper + name string + cancelCtx func() + done chan struct{} + newWork chan *batchWork + assemblyID *fftypes.UUID + assemblyQueue []*batchWork + assemblyQueueBytes int64 + flushedSequences []int64 + statusMux sync.Mutex + flushStatus FlushStatus + retry *retry.Retry + conf *batchProcessorConf } const batchSizeEstimateBase = int64(512) func newBatchProcessor(ctx context.Context, ni sysmessaging.LocalNodeInfo, di database.Plugin, conf *batchProcessorConf, retry *retry.Retry) *batchProcessor { - pCtx := log.WithLogField(ctx, "role", fmt.Sprintf("batchproc-%s:%s:%s", conf.namespace, conf.identity.Author, conf.identity.Key)) + name := fmt.Sprintf("%s:%s:%s", conf.namespace, conf.identity.Author, conf.identity.Key) + pCtx := log.WithLogField(ctx, "role", name) pCtx, cancelCtx := context.WithCancel(pCtx) bp := &batchProcessor{ - ctx: pCtx, - cancelCtx: cancelCtx, - ni: ni, - database: di, - txHelper: txcommon.NewTransactionHelper(di), - name: fmt.Sprintf("%s:%s:%s", conf.namespace, conf.identity.Author, conf.identity.Key), - newWork: make(chan *batchWork), - persistWork: make(chan *batchWork, conf.BatchMaxSize), - sealBatch: make(chan bool), - batchSealed: make(chan bool), - retry: retry, - conf: conf, + ctx: pCtx, + cancelCtx: cancelCtx, + ni: ni, + database: di, + txHelper: txcommon.NewTransactionHelper(di), + name: name, + newWork: make(chan *batchWork, conf.BatchMaxSize), + retry: retry, + conf: conf, + flushedSequences: []int64{}, + flushStatus: FlushStatus{ + LastFlushTime: fftypes.Now(), + }, } + bp.newAssembly() go bp.assemblyLoop() - go bp.persistenceLoop() + log.L(pCtx).Infof("Batch processor created") return bp } @@ -101,103 +127,225 @@ func (bw *batchWork) estimateSize() int64 { return sizeEstimate } -// The assemblyLoop accepts work into the pipe as quickly as possible. -// It dispatches work asynchronously to the persistenceLoop, which is responsible for -// calling back each piece of work once persisted into a batch -// (doesn't wait until that batch is sealed/dispatched). -// The assemblyLoop seals batches when they are full, or timeout. +func (bp *batchProcessor) Status() *FlushStatus { + bp.statusMux.Lock() + defer bp.statusMux.Unlock() + statusCopy := bp.flushStatus + return &statusCopy +} + +func (bp *batchProcessor) newAssembly(initalWork ...*batchWork) { + bp.assemblyID = fftypes.NewUUID() + bp.assemblyQueue = append([]*batchWork{}, initalWork...) + bp.assemblyQueueBytes = batchSizeEstimateBase +} + +// addWork adds the work to the assemblyQueue, and calculates if we have overflowed with this work. +// We check for duplicates, and add the work in sequence order. +// This helps in the case for parallel REST APIs all committing to the DB at a similar time. +// With a sufficient batch size and batch timeout, the batch will still dispatch the messages +// in DB sequence order (although this is not guaranteed). +func (bp *batchProcessor) addWork(newWork *batchWork) (overflow bool) { + newQueue := make([]*batchWork, 0, len(bp.assemblyQueue)+1) + added := false + skip := false + // Check it's not in the recently flushed lish + for _, flushedSequence := range bp.flushedSequences { + if newWork.msg.Sequence == flushedSequence { + log.L(bp.ctx).Debugf("Ignoring add of recently flushed message %s sequence=%d to in-flight batch assembly %s", newWork.msg.Header.ID, newWork.msg.Sequence, bp.assemblyID) + skip = true + break + } + } + // Build the new sorted work list, checking there for duplicates too + for _, work := range bp.assemblyQueue { + if newWork.msg.Sequence == work.msg.Sequence { + log.L(bp.ctx).Debugf("Ignoring duplicate add of message %s sequence=%d to in-flight batch assembly %s", newWork.msg.Header.ID, newWork.msg.Sequence, bp.assemblyID) + skip = true + } + if !added && !skip && newWork.msg.Sequence < work.msg.Sequence { + newQueue = append(newQueue, newWork) + added = true + } + newQueue = append(newQueue, work) + } + if !added && !skip { + newQueue = append(newQueue, newWork) + added = true + } + if added { + log.L(bp.ctx).Debugf("Added message %s sequence=%d to in-flight batch assembly %s", newWork.msg.Header.ID, newWork.msg.Sequence, bp.assemblyID) + bp.assemblyQueueBytes += newWork.estimateSize() + } + bp.assemblyQueue = newQueue + overflow = len(bp.assemblyQueue) > 1 && (bp.assemblyQueueBytes > bp.conf.BatchMaxBytes || len(bp.assemblyQueue) > int(bp.conf.BatchMaxSize)) + return overflow +} + +func (bp *batchProcessor) startFlush(overflow bool) (id *fftypes.UUID, flushAssembly []*batchWork, byteSize int64) { + bp.statusMux.Lock() + defer bp.statusMux.Lock() + // Star the clock + bp.flushStatus.LastFlushTime = fftypes.Now() + // Split the current work if required for overflow + overflowWork := make([]*batchWork, 0) + if overflow { + lastElem := len(bp.assemblyQueue) - 1 + flushAssembly = append(flushAssembly, bp.assemblyQueue[:lastElem]...) + overflowWork = append(overflowWork, bp.assemblyQueue[lastElem]) + } else { + flushAssembly = bp.assemblyQueue + } + // We need to keep track of the sequences we're flushing, because until we finish our flush + // the batch processor might be re-queuing the same messages to use due to rewinds. + // We keep all of the last batch, and up to twice the batch size over time (noting our channel + // size is our batch size - so the batch manager cannot get further than that ahead). + newFlushedSeqLen := len(flushAssembly) + len(bp.flushedSequences) + maxFlushedSeqLen := int(2 * bp.conf.BatchMaxSize) + if newFlushedSeqLen > maxFlushedSeqLen && maxFlushedSeqLen > len(bp.flushedSequences) { + newFlushedSeqLen = maxFlushedSeqLen + } + newFlushedSequnces := make([]int64, 0, newFlushedSeqLen) + for i, fs := range flushAssembly { + newFlushedSequnces[i] = fs.msg.Sequence + } + for i := 0; i < newFlushedSeqLen-len(flushAssembly); i++ { + newFlushedSequnces[i+len(flushAssembly)] = bp.flushedSequences[i] + } + // Cycle to the next assembly + id = bp.assemblyID + byteSize = bp.assemblyQueueBytes + bp.flushStatus.Flushing = id + bp.newAssembly(overflowWork...) + return id, flushAssembly, byteSize +} + +func (bp *batchProcessor) endFlush(batch *fftypes.Batch, byteSize int64) { + bp.statusMux.Lock() + defer bp.statusMux.Lock() + fs := &bp.flushStatus + + duration := time.Since(*fs.LastFlushTime.Time()) + fs.Flushing = nil + + fs.totalBatches++ + + fs.totalFlushDuration += duration + fs.AverageFlushTimeMS = (fs.totalFlushDuration / time.Duration(fs.totalBatches)).Milliseconds() + + fs.totalBytesFlushed += byteSize + fs.AverageBatchBytes = (fs.totalBytesFlushed / fs.totalBatches) + + fs.totalMessagesFlushed += int64(len(batch.Payload.Messages)) + fs.AverageBatchMessages = math.Round((float64(fs.totalMessagesFlushed)/float64(fs.totalBatches))*100) / 100 + + fs.totalDataFlushed += int64(len(batch.Payload.Data)) + fs.AverageBatchData = math.Round((float64(fs.totalDataFlushed)/float64(fs.totalBatches))*100) / 100 +} + +// The assemblyLoop receives new work, sorts it, and waits for the size/timer to pop before +// flushing the batch. The newWork channel has up to one batch of slots queue length, +// so that we can have one batch of work queuing for assembly, while we have one batch flushing. func (bp *batchProcessor) assemblyLoop() { - defer bp.close() - defer close(bp.sealBatch) // close persitenceLoop when we exit + defer close(bp.done) l := log.L(bp.ctx) - var batchSize uint - var batchPayloadEstimate = batchSizeEstimateBase - var lastBatchSealed = time.Now() - var quiescing bool - var overflowedWork *batchWork - for { - var timedOut, closed bool - if overflowedWork != nil { - // We overflowed the size cap when we took this message out the newWork - // queue last time round the lop - bp.persistWork <- overflowedWork - batchSize++ - batchPayloadEstimate += overflowedWork.estimateSize() - overflowedWork = nil - } else { - // We timeout waiting at the point we think we're ready for disposal, - // unless we've started a batch in which case we wait for what's left - // of the batch timeout - timeToWait := bp.conf.DisposeTimeout - if quiescing { - timeToWait = 100 * time.Millisecond - } else if batchSize > 0 { - timeToWait = bp.conf.BatchTimeout - time.Since(lastBatchSealed) + + overflow := false + var batchTimeout = time.NewTimer(bp.conf.DisposeTimeout) + idle := true + quescing := false + for !quescing { + + var timedout bool + select { + case <-bp.ctx.Done(): + l.Tracef("Batch processor shutting down") + _ = batchTimeout.Stop() + return + case <-batchTimeout.C: + l.Errorf("Batch timer popped") + if len(bp.assemblyQueue) == 0 { + // It probably makes sense to exit, but we don't know if more work is coming our way, + // so we just inform the manager we've hit our dispose timeout and see if they close + // the input work channel (which they can safely do on the dispatcher routine) + bp.conf.requestQuiesce() + } else { + // We need to flush + timedout = true } - timeout := time.NewTimer(timeToWait) - - // Wait for work, the timeout, or close - select { - case <-timeout.C: - timedOut = true - case work, ok := <-bp.newWork: - if ok && !work.abandoned { - workSize := work.estimateSize() - if batchSize > 0 && batchPayloadEstimate+workSize > bp.conf.BatchMaxBytes { - overflowedWork = work - } else { - batchSize++ - batchPayloadEstimate += workSize - bp.persistWork <- work - } - } else { - closed = true + case work, ok := <-bp.newWork: + if !ok { + quescing = true + } else { + overflow = bp.addWork(work) + if idle { + // We've hit a message while we were idle - we now need to wait for the batch to time out. + _ = batchTimeout.Stop() + batchTimeout = time.NewTimer(bp.conf.BatchTimeout) + idle = false } } - } + if overflow || timedout || quescing { + // Let Go GC the old timer + _ = batchTimeout.Stop() + + // If we are in overflow, start the clock for the next batch to start before we do the flush + // (even though we won't check it until after). + if overflow { + batchTimeout = time.NewTimer(bp.conf.BatchTimeout) + } - // Don't include the sealing time in the duration - batchFull := overflowedWork != nil || batchSize >= bp.conf.BatchMaxSize - l.Debugf("Assembly batch loop: Size=%d Full=%t Bytes=%.2fkb (est) Overflow=%t", batchSize, batchFull, float64(batchPayloadEstimate)/1024, overflowedWork != nil) + err := bp.flush(overflow) + if err != nil { + l.Tracef("Batch processor shutting down: %s", err) + _ = batchTimeout.Stop() + return + } - batchDuration := time.Since(lastBatchSealed) - if quiescing && batchSize == 0 { - l.Debugf("Batch assembler disposed after %.2fs of inactivity", float64(batchDuration)/float64(time.Second)) - return + // If we didn't overflow, then just go back to idle - we don't know if we have more work to come, so + // either we'll pop straight away (and move to the batch timeout) or wait for the dispose timeout + if !overflow && !quescing { + batchTimeout = time.NewTimer(bp.conf.DisposeTimeout) + idle = true + } } + } +} - if closed || batchDuration > bp.conf.DisposeTimeout { - bp.conf.processorQuiescing() - quiescing = true - } +func (bp *batchProcessor) flush(overflow bool) error { + id, flushWork, byteSize := bp.startFlush(overflow) + batch := bp.buildFlushBatch(id, flushWork) - if (quiescing || timedOut || batchFull) && batchSize > 0 { - bp.sealBatch <- true - <-bp.batchSealed - l.Debugf("Assembly batch sealed") - lastBatchSealed = time.Now() - batchSize = 0 - batchPayloadEstimate = batchSizeEstimateBase - } + pins, err := bp.persistBatch(batch) + if err != nil { + return err + } + err = bp.dispatchBatch(batch, pins) + if err != nil { + return err + } + + err = bp.markMessagesDispatched(batch) + if err != nil { + return err } + + bp.endFlush(batch, byteSize) + return nil } -func (bp *batchProcessor) createOrAddToBatch(batch *fftypes.Batch, newWork []*batchWork) *fftypes.Batch { - l := log.L(bp.ctx) - if batch == nil { - batchID := fftypes.NewUUID() - l.Debugf("New batch %s", batchID) - batch = &fftypes.Batch{ - ID: batchID, - Namespace: bp.conf.namespace, - Identity: bp.conf.identity, - Group: bp.conf.group, - Payload: fftypes.BatchPayload{}, - Created: fftypes.Now(), - Node: bp.ni.GetNodeUUID(bp.ctx), - } +func (bp *batchProcessor) buildFlushBatch(id *fftypes.UUID, newWork []*batchWork) *fftypes.Batch { + log.L(bp.ctx).Debugf("Flushing batch %s", id) + batch := &fftypes.Batch{ + ID: id, + Namespace: bp.conf.namespace, + Identity: bp.conf.identity, + Group: bp.conf.group, + Payload: fftypes.BatchPayload{}, + Created: fftypes.Now(), + Node: bp.ni.GetNodeUUID(bp.ctx), } for _, w := range newWork { if w.msg != nil { @@ -270,146 +418,56 @@ func (bp *batchProcessor) maskContexts(ctx context.Context, batch *fftypes.Batch return contextsOrPins, nil } -func (bp *batchProcessor) dispatchBatch(batch *fftypes.Batch, pins []*fftypes.Bytes32) { - // Call the dispatcher to do the heavy lifting - will only exit if we're closed - _ = bp.retry.Do(bp.ctx, "batch dispatch", func(attempt int) (retry bool, err error) { - err = bp.conf.dispatch(bp.ctx, batch, pins) - if err != nil { - return !bp.closed, err - } - return false, nil - }) -} - -func (bp *batchProcessor) persistBatch(batch *fftypes.Batch, newWork []*batchWork, seal bool) (contexts []*fftypes.Bytes32, err error) { +func (bp *batchProcessor) persistBatch(batch *fftypes.Batch) (contexts []*fftypes.Bytes32, err error) { err = bp.retry.Do(bp.ctx, "batch persist", func(attempt int) (retry bool, err error) { - err = bp.database.RunAsGroup(bp.ctx, func(ctx context.Context) (err error) { - // Update all the messages in the batch with the batch ID - if len(newWork) > 0 { - msgIDs := make([]driver.Value, 0, len(newWork)) - for _, w := range newWork { - if w.msg != nil { - msgIDs = append(msgIDs, w.msg.Header.ID) - } - } - filter := database.MessageQueryFactory.NewFilter(ctx).In("id", msgIDs) - update := database.MessageQueryFactory.NewUpdate(ctx). - Set("batch", batch.ID). - Set("group", batch.Group) - err = bp.database.UpdateMessages(ctx, filter, update) + return true, bp.database.RunAsGroup(bp.ctx, func(ctx context.Context) (err error) { + // Generate a new Transaction, which will be used to record status of the associated transaction as it happens + if contexts, err = bp.maskContexts(ctx, batch); err != nil { + return err } - if err == nil && seal { - // Generate a new Transaction, which will be used to record status of the associated transaction as it happens - contexts, err = bp.maskContexts(ctx, batch) - if err == nil { - batch.Payload.TX.Type = fftypes.TransactionTypeBatchPin - batch.Payload.TX.ID, err = bp.txHelper.SubmitNewTransaction(ctx, batch.Namespace, fftypes.TransactionTypeBatchPin) - batch.Hash = batch.Payload.Hash() - log.L(ctx).Debugf("Batch %s sealed. Hash=%s", batch.ID, batch.Hash) - } - } - if err == nil { - // Persist the batch itself - err = bp.database.UpsertBatch(ctx, batch, seal /* we set the hash as it seals */) + batch.Payload.TX.Type = fftypes.TransactionTypeBatchPin + if batch.Payload.TX.ID, err = bp.txHelper.SubmitNewTransaction(ctx, batch.Namespace, fftypes.TransactionTypeBatchPin); err != nil { + return err } - return err + + batch.Hash = batch.Payload.Hash() + log.L(ctx).Debugf("Batch %s sealed. Hash=%s", batch.ID, batch.Hash) + return bp.database.UpsertBatch(ctx, batch) }) - if err != nil { - return !bp.closed, err - } - return false, nil }) return contexts, err } -func (bp *batchProcessor) persistenceLoop() { - defer close(bp.batchSealed) - l := log.L(bp.ctx) - var currentBatch *fftypes.Batch - var batchSize = 0 - for !bp.closed { - var seal bool - newWork := make([]*batchWork, 0, bp.conf.BatchMaxSize) - - // Block waiting for work, or a batch sealing request - select { - case w := <-bp.persistWork: - newWork = append(newWork, w) - case <-bp.sealBatch: - seal = true - } - - // Drain everything currently in the pipe waiting for dispatch - // This means we batch the writing to the database, which has to happen before - // we can callback the work with a persisted batch ID. - // We drain both the message queue, and the seal, because there's no point - // going round the loop (persisting twice) if the batch has just filled - var drained bool - for !drained { - select { - case _, ok := <-bp.sealBatch: - seal = true - if !ok { - return // Closed by termination of assemblyLoop - } - case w := <-bp.persistWork: - newWork = append(newWork, w) - default: - drained = true - } - } - - batchSize += len(newWork) - currentBatch = bp.createOrAddToBatch(currentBatch, newWork) - l.Debugf("Adding %d entries to batch %s. Size=%d Seal=%t", len(newWork), currentBatch.ID, batchSize, seal) - - // Persist the batch - indefinite retry (unless we close, or context is cancelled) - contexts, err := bp.persistBatch(currentBatch, newWork, seal) - if err != nil { - return - } +func (bp *batchProcessor) dispatchBatch(batch *fftypes.Batch, pins []*fftypes.Bytes32) error { + // Call the dispatcher to do the heavy lifting - will only exit if we're closed + return bp.retry.Do(bp.ctx, "batch dispatch", func(attempt int) (retry bool, err error) { + return true, bp.conf.dispatch(bp.ctx, batch, pins) + }) +} - // Inform all the work in this batch of the batch they have been persisted - // into. At this point they can carry on processing, because we won't lose - // the work - it's tracked in a batch ready to go - for _, w := range newWork { - w.dispatched <- &batchDispatch{ - w.msg, - currentBatch.ID, +func (bp *batchProcessor) markMessagesDispatched(batch *fftypes.Batch) error { + return bp.retry.Do(bp.ctx, "batch persist", func(attempt int) (retry bool, err error) { + return true, bp.database.RunAsGroup(bp.ctx, func(ctx context.Context) (err error) { + // Update all the messages in the batch with the batch ID + msgIDs := make([]driver.Value, 0, len(batch.Payload.Messages)) + for i, msg := range batch.Payload.Messages { + msgIDs[i] = msg.Header.ID } - } - - if seal { - // At this point the batch is sealed, and the assember can start - // queing up the next batch. We only let them get one batch ahead - // (due to the size of the channel being the maxBatchSize) before - // they start blocking waiting for us to complete database of - // the current batch. - bp.batchSealed <- true - - // Synchronously dispatch the batch. Must be last thing we do in the loop, as we - // will break out of the retry in the case that we close - bp.dispatchBatch(currentBatch, contexts) - - // Move onto the next batch - currentBatch = nil - batchSize = 0 - } - - } + fb := database.MessageQueryFactory.NewFilter(ctx) + filter := fb.And( + fb.In("id", msgIDs), + fb.Eq("state", fftypes.MessageStateReady), // In the outside chance the next state transition happens first (which supersedes this) + ) + update := database.MessageQueryFactory.NewUpdate(ctx). + Set("batch", batch.ID). // Mark the batch they are in + Set("state", fftypes.MessageStateSent) // Set them sent, so they won't be picked up and re-sent after restart/rewind + return bp.database.UpdateMessages(ctx, filter, update) + }) + }) } func (bp *batchProcessor) close() { - if !bp.closed { - // We don't cancel the context here, as we use close during quiesce and don't want the - // persistence loop to have its context cancelled, and fail to perform DB operations - close(bp.newWork) - bp.closed = true - } -} - -func (bp *batchProcessor) waitClosed() { - <-bp.sealBatch - <-bp.batchSealed + bp.cancelCtx() + <-bp.done } diff --git a/internal/batch/batch_processor_test.go b/internal/batch/batch_processor_test.go index 3c0ec19765..4089b5140f 100644 --- a/internal/batch/batch_processor_test.go +++ b/internal/batch/batch_processor_test.go @@ -36,10 +36,9 @@ func newTestBatchProcessor(dispatch DispatchHandler) (*databasemocks.Plugin, *ba mni := &sysmessagingmocks.LocalNodeInfo{} mni.On("GetNodeUUID", mock.Anything).Return(fftypes.NewUUID()).Maybe() bp := newBatchProcessor(context.Background(), mni, mdi, &batchProcessorConf{ - namespace: "ns1", - identity: fftypes.Identity{Author: "did:firefly:org/abcd", Key: "0x12345"}, - dispatch: dispatch, - processorQuiescing: func() {}, + namespace: "ns1", + identity: fftypes.Identity{Author: "did:firefly:org/abcd", Key: "0x12345"}, + dispatch: dispatch, Options: Options{ BatchMaxSize: 10, BatchMaxBytes: 1024 * 1024, @@ -112,7 +111,7 @@ func TestUnfilledBatch(t *testing.T) { assert.Equal(t, len(dispatched[0].Payload.Messages), len(work)) bp.close() - bp.waitClosed() + bp.close() } @@ -168,7 +167,7 @@ func TestBatchSizeOverflow(t *testing.T) { assert.Equal(t, len(dispatched[1].Payload.Messages), 1) bp.close() - bp.waitClosed() + bp.close() } @@ -239,7 +238,7 @@ func TestFilledBatchSlowPersistence(t *testing.T) { assert.Equal(t, len(dispatched[0].Payload.Data), 5) bp.close() - bp.waitClosed() + bp.close() } @@ -286,7 +285,7 @@ func TestCloseToUnblockUpsertBatch(t *testing.T) { // Close to unblock bp.close() - bp.waitClosed() + bp.close() } diff --git a/internal/database/sqlcommon/batch_sql.go b/internal/database/sqlcommon/batch_sql.go index af25ee9174..d4f72d4caf 100644 --- a/internal/database/sqlcommon/batch_sql.go +++ b/internal/database/sqlcommon/batch_sql.go @@ -54,7 +54,7 @@ var ( } ) -func (s *SQLCommon) UpsertBatch(ctx context.Context, batch *fftypes.Batch, allowHashUpdate bool) (err error) { +func (s *SQLCommon) UpsertBatch(ctx context.Context, batch *fftypes.Batch) (err error) { ctx, tx, autoCommit, err := s.beginOrUseTx(ctx) if err != nil { return err @@ -72,7 +72,7 @@ func (s *SQLCommon) UpsertBatch(ctx context.Context, batch *fftypes.Batch, allow } existing := batchRows.Next() - if existing && !allowHashUpdate { + if existing { var hash *fftypes.Bytes32 _ = batchRows.Scan(&hash) if !fftypes.SafeHashCompare(hash, batch.Hash) { diff --git a/internal/database/sqlcommon/batch_sql_test.go b/internal/database/sqlcommon/batch_sql_test.go index c39ecd6003..27633b8156 100644 --- a/internal/database/sqlcommon/batch_sql_test.go +++ b/internal/database/sqlcommon/batch_sql_test.go @@ -62,7 +62,7 @@ func TestBatch2EWithDB(t *testing.T) { s.callbacks.On("UUIDCollectionNSEvent", database.CollectionBatches, fftypes.ChangeEventTypeCreated, "ns1", batchID, mock.Anything).Return() s.callbacks.On("UUIDCollectionNSEvent", database.CollectionBatches, fftypes.ChangeEventTypeUpdated, "ns1", batchID, mock.Anything).Return() - err := s.UpsertBatch(ctx, batch, true) + err := s.UpsertBatch(ctx, batch) assert.NoError(t, err) // Check we get the exact same batch back @@ -104,10 +104,11 @@ func TestBatch2EWithDB(t *testing.T) { } // Rejects hash change - err = s.UpsertBatch(context.Background(), batchUpdated, false) + err = s.UpsertBatch(context.Background(), batchUpdated) assert.Equal(t, database.HashMismatch, err) - err = s.UpsertBatch(context.Background(), batchUpdated, true) + batchUpdated.Hash = batch.Hash + err = s.UpsertBatch(context.Background(), batchUpdated) assert.NoError(t, err) // Check we get the exact same message back - note the removal of one of the batch elements @@ -163,7 +164,7 @@ func TestBatch2EWithDB(t *testing.T) { func TestUpsertBatchFailBegin(t *testing.T) { s, mock := newMockProvider().init() mock.ExpectBegin().WillReturnError(fmt.Errorf("pop")) - err := s.UpsertBatch(context.Background(), &fftypes.Batch{}, true) + err := s.UpsertBatch(context.Background(), &fftypes.Batch{}) assert.Regexp(t, "FF10114", err) assert.NoError(t, mock.ExpectationsWereMet()) } @@ -174,7 +175,7 @@ func TestUpsertBatchFailSelect(t *testing.T) { mock.ExpectQuery("SELECT .*").WillReturnError(fmt.Errorf("pop")) mock.ExpectRollback() batchID := fftypes.NewUUID() - err := s.UpsertBatch(context.Background(), &fftypes.Batch{ID: batchID}, true) + err := s.UpsertBatch(context.Background(), &fftypes.Batch{ID: batchID}) assert.Regexp(t, "FF10115", err) assert.NoError(t, mock.ExpectationsWereMet()) } @@ -186,7 +187,7 @@ func TestUpsertBatchFailInsert(t *testing.T) { mock.ExpectExec("INSERT .*").WillReturnError(fmt.Errorf("pop")) mock.ExpectRollback() batchID := fftypes.NewUUID() - err := s.UpsertBatch(context.Background(), &fftypes.Batch{ID: batchID}, true) + err := s.UpsertBatch(context.Background(), &fftypes.Batch{ID: batchID}) assert.Regexp(t, "FF10116", err) assert.NoError(t, mock.ExpectationsWereMet()) } @@ -194,11 +195,12 @@ func TestUpsertBatchFailInsert(t *testing.T) { func TestUpsertBatchFailUpdate(t *testing.T) { s, mock := newMockProvider().init() batchID := fftypes.NewUUID() + hash := fftypes.NewRandB32() mock.ExpectBegin() - mock.ExpectQuery("SELECT .*").WillReturnRows(sqlmock.NewRows([]string{"id"}).AddRow(batchID.String())) + mock.ExpectQuery("SELECT .*").WillReturnRows(sqlmock.NewRows([]string{"hash"}).AddRow(hash)) mock.ExpectExec("UPDATE .*").WillReturnError(fmt.Errorf("pop")) mock.ExpectRollback() - err := s.UpsertBatch(context.Background(), &fftypes.Batch{ID: batchID}, true) + err := s.UpsertBatch(context.Background(), &fftypes.Batch{ID: batchID, Hash: hash}) assert.Regexp(t, "FF10117", err) assert.NoError(t, mock.ExpectationsWereMet()) } @@ -210,7 +212,7 @@ func TestUpsertBatchFailCommit(t *testing.T) { mock.ExpectQuery("SELECT .*").WillReturnRows(sqlmock.NewRows([]string{"id"})) mock.ExpectExec("INSERT .*").WillReturnResult(sqlmock.NewResult(1, 1)) mock.ExpectCommit().WillReturnError(fmt.Errorf("pop")) - err := s.UpsertBatch(context.Background(), &fftypes.Batch{ID: batchID}, true) + err := s.UpsertBatch(context.Background(), &fftypes.Batch{ID: batchID}) assert.Regexp(t, "FF10119", err) assert.NoError(t, mock.ExpectationsWereMet()) } diff --git a/internal/events/persist_batch.go b/internal/events/persist_batch.go index 54bfd98686..09837d3b06 100644 --- a/internal/events/persist_batch.go +++ b/internal/events/persist_batch.go @@ -111,7 +111,7 @@ func (em *eventManager) persistBatch(ctx context.Context /* db TX context*/, bat batch.Confirmed = now // Upsert the batch itself, ensuring the hash does not change - err = em.database.UpsertBatch(ctx, batch, false) + err = em.database.UpsertBatch(ctx, batch) if err != nil { if err == database.HashMismatch { l.Errorf("Invalid batch '%s'. Batch hash mismatch with existing record", batch.ID) diff --git a/mocks/databasemocks/plugin.go b/mocks/databasemocks/plugin.go index 4b180b74b2..80856418a1 100644 --- a/mocks/databasemocks/plugin.go +++ b/mocks/databasemocks/plugin.go @@ -2465,13 +2465,13 @@ func (_m *Plugin) UpdateTransaction(ctx context.Context, id *fftypes.UUID, updat return r0 } -// UpsertBatch provides a mock function with given fields: ctx, data, allowHashUpdate -func (_m *Plugin) UpsertBatch(ctx context.Context, data *fftypes.Batch, allowHashUpdate bool) error { - ret := _m.Called(ctx, data, allowHashUpdate) +// UpsertBatch provides a mock function with given fields: ctx, data +func (_m *Plugin) UpsertBatch(ctx context.Context, data *fftypes.Batch) error { + ret := _m.Called(ctx, data) var r0 error - if rf, ok := ret.Get(0).(func(context.Context, *fftypes.Batch, bool) error); ok { - r0 = rf(ctx, data, allowHashUpdate) + if rf, ok := ret.Get(0).(func(context.Context, *fftypes.Batch) error); ok { + r0 = rf(ctx, data) } else { r0 = ret.Error(0) } diff --git a/pkg/database/plugin.go b/pkg/database/plugin.go index 307eaa0154..369534bec0 100644 --- a/pkg/database/plugin.go +++ b/pkg/database/plugin.go @@ -117,9 +117,8 @@ type iDataCollection interface { } type iBatchCollection interface { - // UpsertBatch - Upsert a batch - // allowHashUpdate=false throws HashMismatch error if the updated message has a different hash - UpsertBatch(ctx context.Context, data *fftypes.Batch, allowHashUpdate bool) (err error) + // UpsertBatch - Upsert a batch - the hash cannot change + UpsertBatch(ctx context.Context, data *fftypes.Batch) (err error) // UpdateBatch - Update data UpdateBatch(ctx context.Context, id *fftypes.UUID, update Update) (err error) diff --git a/pkg/fftypes/message.go b/pkg/fftypes/message.go index faf75d2934..02f27c90e6 100644 --- a/pkg/fftypes/message.go +++ b/pkg/fftypes/message.go @@ -55,6 +55,8 @@ var ( MessageStateStaged MessageState = ffEnum("messagestate", "staged") // MessageStateReady is a message created locally which is ready to send MessageStateReady MessageState = ffEnum("messagestate", "ready") + // MessageStateSent is a message created locally which has been sent in a batch + MessageStateSent MessageState = ffEnum("messagestate", "sent") // MessageStatePending is a message that has been received but is awaiting aggregation/confirmation MessageStatePending MessageState = ffEnum("messagestate", "pending") // MessageStateConfirmed is a message that has completed all required confirmations (blockchain if pinned, token transfer if transfer coupled, etc) From fb93472aa57793d8187717ac0a93892cc4c90a1e Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Wed, 9 Feb 2022 09:23:28 -0500 Subject: [PATCH 04/23] Further refinement and testing Signed-off-by: Peter Broadhurst --- internal/batch/batch_manager.go | 197 +++++++++--------- internal/batch/batch_manager_test.go | 155 +++++--------- internal/batch/batch_processor.go | 89 +++++--- internal/batch/batch_processor_test.go | 32 +-- internal/broadcast/manager.go | 6 +- internal/privatemessaging/privatemessaging.go | 6 +- internal/retry/retry.go | 6 +- internal/retry/retry_test.go | 5 + mocks/batchmocks/manager.go | 22 +- 9 files changed, 256 insertions(+), 262 deletions(-) diff --git a/internal/batch/batch_manager.go b/internal/batch/batch_manager.go index c7552c02df..1e8a845f19 100644 --- a/internal/batch/batch_manager.go +++ b/internal/batch/batch_manager.go @@ -32,17 +32,15 @@ import ( "github.com/hyperledger/firefly/pkg/fftypes" ) -const ( - msgBatchOffsetName = "ff_msgbatch" -) - func NewBatchManager(ctx context.Context, ni sysmessaging.LocalNodeInfo, di database.Plugin, dm data.Manager) (Manager, error) { if di == nil || dm == nil { return nil, i18n.NewError(ctx, i18n.MsgInitializationNilDepError) } + pCtx, cancelCtx := context.WithCancel(log.WithLogField(ctx, "role", "batchmgr")) readPageSize := config.GetUint(config.BatchManagerReadPageSize) bm := &batchManager{ - ctx: log.WithLogField(ctx, "role", "batchmgr"), + ctx: pCtx, + cancelCtx: cancelCtx, ni: ni, database: di, data: dm, @@ -51,8 +49,8 @@ func NewBatchManager(ctx context.Context, ni sysmessaging.LocalNodeInfo, di data messagePollTimeout: config.GetDuration(config.BatchManagerReadPollTimeout), startupOffsetRetryAttempts: config.GetInt(config.OrchestratorStartupAttempts), dispatchers: make(map[fftypes.MessageType]*dispatcher), - newMessages: make(chan int64), - sequencerClosed: make(chan struct{}), + newMessages: make(chan int64, 1), + done: make(chan struct{}), retry: &retry.Retry{ InitialDelay: config.GetDuration(config.BatchRetryInitDelay), MaximumDelay: config.GetDuration(config.BatchRetryMaxDelay), @@ -63,26 +61,32 @@ func NewBatchManager(ctx context.Context, ni sysmessaging.LocalNodeInfo, di data } type Manager interface { - RegisterDispatcher(msgTypes []fftypes.MessageType, handler DispatchHandler, batchOptions Options) + RegisterDispatcher(name string, msgTypes []fftypes.MessageType, handler DispatchHandler, batchOptions DispatcherOptions) NewMessages() chan<- int64 Start() error Close() WaitStop() + Status() []*ProcessorStatus +} + +type ProcessorStatus struct { + Dispatcher string `json:"dispatcher"` + Name string `json:"name"` + Status FlushStatus `json:"status"` } type batchManager struct { ctx context.Context + cancelCtx func() ni sysmessaging.LocalNodeInfo database database.Plugin data data.Manager + dispatcherMux sync.Mutex dispatchers map[fftypes.MessageType]*dispatcher newMessages chan int64 - sequencerClosed chan struct{} + done chan struct{} retry *retry.Retry - offsetID int64 - recoveryOffset int64 readOffset int64 - closed bool readPageSize uint64 messagePollTimeout time.Duration startupOffsetRetryAttempts int @@ -90,7 +94,7 @@ type batchManager struct { type DispatchHandler func(context.Context, *fftypes.Batch, []*fftypes.Bytes32) error -type Options struct { +type DispatcherOptions struct { BatchMaxSize uint BatchMaxBytes int64 BatchTimeout time.Duration @@ -98,17 +102,18 @@ type Options struct { } type dispatcher struct { - handler DispatchHandler - mux sync.Mutex - processors map[string]*batchProcessor - batchOptions Options + name string + handler DispatchHandler + processors map[string]*batchProcessor + options DispatcherOptions } -func (bm *batchManager) RegisterDispatcher(msgTypes []fftypes.MessageType, handler DispatchHandler, batchOptions Options) { +func (bm *batchManager) RegisterDispatcher(name string, msgTypes []fftypes.MessageType, handler DispatchHandler, options DispatcherOptions) { dispatcher := &dispatcher{ - handler: handler, - batchOptions: batchOptions, - processors: make(map[string]*batchProcessor), + name: name, + handler: handler, + options: options, + processors: make(map[string]*batchProcessor), } for _, msgType := range msgTypes { bm.dispatchers[msgType] = dispatcher @@ -116,9 +121,6 @@ func (bm *batchManager) RegisterDispatcher(msgTypes []fftypes.MessageType, handl } func (bm *batchManager) Start() error { - if err := bm.restoreOffset(); err != nil { - return err - } go bm.messageSequencer() return nil } @@ -127,83 +129,44 @@ func (bm *batchManager) NewMessages() chan<- int64 { return bm.newMessages } -func (bm *batchManager) restoreOffset() (err error) { - var offset *fftypes.Offset - for offset == nil { - offset, err = bm.database.GetOffset(bm.ctx, fftypes.OffsetTypeBatch, msgBatchOffsetName) - if err != nil { - return err - } - if offset == nil { - _ = bm.database.UpsertOffset(bm.ctx, &fftypes.Offset{ - Type: fftypes.OffsetTypeBatch, - Name: msgBatchOffsetName, - Current: 0, - }, false) - } - } - bm.offsetID = offset.RowID - bm.readOffset = offset.Current - bm.recoveryOffset = offset.Current - log.L(bm.ctx).Infof("Batch manager restored offset %d", offset.Current) - return nil -} - func (bm *batchManager) getProcessor(batchType fftypes.MessageType, group *fftypes.Bytes32, namespace string, identity *fftypes.Identity) (*batchProcessor, error) { + bm.dispatcherMux.Lock() + defer bm.dispatcherMux.Unlock() + dispatcher, ok := bm.dispatchers[batchType] if !ok { return nil, i18n.NewError(bm.ctx, i18n.MsgUnregisteredBatchType, batchType) } - dispatcher.mux.Lock() - key := fmt.Sprintf("%s:%s:%s[group=%v]", namespace, identity.Author, identity.Key, group) - processor, ok := dispatcher.processors[key] + name := fmt.Sprintf("%s|%s|%v", namespace, identity.Author, group) + processor, ok := dispatcher.processors[name] if !ok { processor = newBatchProcessor( bm.ctx, // Background context, not the call context bm.ni, bm.database, &batchProcessorConf{ - Options: dispatcher.batchOptions, - namespace: namespace, - identity: *identity, - group: group, - dispatch: dispatcher.handler, + DispatcherOptions: dispatcher.options, + name: name, + dispatcherName: dispatcher.name, + namespace: namespace, + identity: *identity, + group: group, + dispatch: dispatcher.handler, }, bm.retry, ) - dispatcher.processors[key] = processor + dispatcher.processors[name] = processor } - log.L(bm.ctx).Debugf("Created new processor: %s", key) - dispatcher.mux.Unlock() + log.L(bm.ctx).Debugf("Created new processor: %s", name) return processor, nil } -func (bm *batchManager) Close() { - var processors []*batchProcessor - if bm != nil && !bm.closed { - for _, d := range bm.dispatchers { - d.mux.Lock() - for _, p := range d.processors { - processors = append(processors, p) - p.close() - } - d.mux.Unlock() - } - bm.closed = true - close(bm.newMessages) - } - bm = nil - for _, p := range processors { - <-p.done - } -} - func (bm *batchManager) assembleMessageData(msg *fftypes.Message) (data []*fftypes.Data, err error) { var foundAll = false err = bm.retry.Do(bm.ctx, fmt.Sprintf("assemble message %s data", msg.Header.ID), func(attempt int) (retry bool, err error) { data, foundAll, err = bm.data.GetMessageData(bm.ctx, msg, true) // continual retry for persistence error (distinct from not-found) - return err != nil && !bm.closed, err + return true, err }) if err != nil { return nil, err @@ -211,7 +174,6 @@ func (bm *batchManager) assembleMessageData(msg *fftypes.Message) (data []*fftyp if !foundAll { return nil, i18n.NewError(bm.ctx, i18n.MsgDataNotFound, msg.Header.ID) } - log.L(bm.ctx).Infof("Detected new batch-pinned message %s sequence=%d", msg.Header.ID, msg.Sequence) return data, nil } @@ -224,10 +186,7 @@ func (bm *batchManager) readPage() ([]*fftypes.Message, error) { fb.Gt("sequence", bm.readOffset), fb.Eq("state", fftypes.MessageStateReady), ).Sort("sequence").Limit(bm.readPageSize)) - if err != nil { - return !bm.closed, err // Retry indefinitely, until closed (or context cancelled) - } - return false, nil + return true, err }) return msgs, err } @@ -235,15 +194,18 @@ func (bm *batchManager) readPage() ([]*fftypes.Message, error) { func (bm *batchManager) messageSequencer() { l := log.L(bm.ctx) l.Debugf("Started batch assembly message sequencer") - defer close(bm.sequencerClosed) + defer close(bm.done) dispatched := make(chan *batchDispatch, bm.readPageSize) - for !bm.closed { + for { + // Each time round the loop we check for quiescing processors + bm.reapQuiescing() + // Read messages from the DB - in an error condition we retry until success, or a closed context msgs, err := bm.readPage() if err != nil { - l.Debugf("Exiting: %s", err) // errors logged in readPage + l.Debugf("Exiting: %s", err) return } batchWasFull := false @@ -282,13 +244,16 @@ func (bm *batchManager) messageSequencer() { } // Wait to be woken again - if !bm.closed && !batchWasFull { - bm.waitForShoulderTapOrPollTimeout() + if !batchWasFull { + if done := bm.waitForShoulderTapOrPollTimeout(); done { + l.Debugf("Exiting: %s", err) + return + } } } } -func (bm *batchManager) waitForShoulderTapOrPollTimeout() { +func (bm *batchManager) waitForShoulderTapOrPollTimeout() (done bool) { l := log.L(bm.ctx) // Drain any new message notifications, moving back our @@ -308,7 +273,7 @@ func (bm *batchManager) waitForShoulderTapOrPollTimeout() { } } if newMessages { - return + return false } // Otherwise set a timeout @@ -316,10 +281,10 @@ func (bm *batchManager) waitForShoulderTapOrPollTimeout() { select { case <-timeout.C: l.Debugf("Woken after poll timeout") + return false case <-bm.ctx.Done(): l.Debugf("Exiting due to cancelled context") - bm.Close() - return + return true } } @@ -329,7 +294,7 @@ func (bm *batchManager) dispatchMessage(dispatched chan *batchDispatch, msg *fft if err != nil { return err } - l.Debugf("Dispatching message %s to %s batch", msg.Header.ID, msg.Header.Type) + l.Debugf("Dispatching message %s to %s batch processor %s", msg.Header.ID, msg.Header.Type, processor.conf.name) work := &batchWork{ msg: msg, data: data, @@ -339,17 +304,53 @@ func (bm *batchManager) dispatchMessage(dispatched chan *batchDispatch, msg *fft return nil } -func (bm *batchManager) WaitStop() { - <-bm.sequencerClosed +func (bm *batchManager) reapQuiescing() { + bm.dispatcherMux.Lock() + defer bm.dispatcherMux.Unlock() + + for _, d := range bm.dispatchers { + for k, p := range d.processors { + select { + case <-p.quescing: + // This is called on the goroutine where we dispatch the work, so it's safe to cleanup + delete(d.processors, k) + close(p.newWork) + default: + } + } + } +} + +func (bm *batchManager) getProcessors() []*batchProcessor { + bm.dispatcherMux.Lock() + defer bm.dispatcherMux.Unlock() + var processors []*batchProcessor for _, d := range bm.dispatchers { - d.mux.Lock() for _, p := range d.processors { processors = append(processors, p) } - d.mux.Unlock() } + return processors +} + +func (bm *batchManager) Status() []*ProcessorStatus { + processors := bm.getProcessors() + status := make([]*ProcessorStatus, len(processors)) + for i, p := range processors { + status[i] = p.status() + } + return status +} + +func (bm *batchManager) Close() { + bm.cancelCtx() // all processor contexts are child contexts +} + +func (bm *batchManager) WaitStop() { + <-bm.done + processors := bm.getProcessors() for _, p := range processors { - p.close() + <-p.done } } diff --git a/internal/batch/batch_manager_test.go b/internal/batch/batch_manager_test.go index fd2b800f88..e45bc4bf04 100644 --- a/internal/batch/batch_manager_test.go +++ b/internal/batch/batch_manager_test.go @@ -39,12 +39,6 @@ func TestE2EDispatchBroadcast(t *testing.T) { mdm := &datamocks.Manager{} mni := &sysmessagingmocks.LocalNodeInfo{} mni.On("GetNodeUUID", mock.Anything).Return(fftypes.NewUUID()) - mdi.On("GetOffset", mock.Anything, fftypes.OffsetTypeBatch, msgBatchOffsetName).Return(nil, nil).Once() - mdi.On("UpsertOffset", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) - mdi.On("UpdateOffset", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) - mdi.On("GetOffset", mock.Anything, fftypes.OffsetTypeBatch, msgBatchOffsetName).Return(&fftypes.Offset{ - RowID: 12345, - }, nil) readyForDispatch := make(chan bool) waitForDispatch := make(chan *fftypes.Batch) handler := func(ctx context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { @@ -76,7 +70,7 @@ func TestE2EDispatchBroadcast(t *testing.T) { bmi, _ := NewBatchManager(ctx, mni, mdi, mdm) bm := bmi.(*batchManager) - bm.RegisterDispatcher([]fftypes.MessageType{fftypes.MessageTypeBroadcast}, handler, Options{ + bm.RegisterDispatcher("utdispatcher", []fftypes.MessageType{fftypes.MessageTypeBroadcast}, handler, DispatcherOptions{ BatchMaxSize: 2, BatchTimeout: 0, DisposeTimeout: 120 * time.Second, @@ -114,7 +108,7 @@ func TestE2EDispatchBroadcast(t *testing.T) { mdi.On("UpdateMessages", mock.Anything, mock.MatchedBy(func(f database.Filter) bool { fi, err := f.Finalize() assert.NoError(t, err) - assert.Equal(t, fmt.Sprintf("id IN ['%s']", msg.Header.ID.String()), fi.String()) + assert.Equal(t, fmt.Sprintf("( id IN ['%s'] ) && ( state == 'ready' )", msg.Header.ID.String()), fi.String()) return true }), mock.Anything).Return(nil) mdi.On("InsertTransaction", mock.Anything, mock.Anything).Return(nil) @@ -130,8 +124,6 @@ func TestE2EDispatchBroadcast(t *testing.T) { assert.Equal(t, *msg.Header.ID, *b.Payload.Messages[0].Header.ID) assert.Equal(t, *data.ID, *b.Payload.Data[0].ID) - // Wait until everything closes - close(readyForDispatch) cancel() bm.WaitStop() @@ -144,12 +136,6 @@ func TestE2EDispatchPrivate(t *testing.T) { mdm := &datamocks.Manager{} mni := &sysmessagingmocks.LocalNodeInfo{} mni.On("GetNodeUUID", mock.Anything).Return(fftypes.NewUUID()) - mdi.On("GetOffset", mock.Anything, fftypes.OffsetTypeBatch, msgBatchOffsetName).Return(nil, nil).Once() - mdi.On("UpsertOffset", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) - mdi.On("UpdateOffset", mock.Anything, mock.Anything, mock.Anything, mock.Anything).Return(nil) - mdi.On("GetOffset", mock.Anything, fftypes.OffsetTypeBatch, msgBatchOffsetName).Return(&fftypes.Offset{ - RowID: 12345, - }, nil) readyForDispatch := make(chan bool) waitForDispatch := make(chan *fftypes.Batch) var groupID fftypes.Bytes32 @@ -184,7 +170,7 @@ func TestE2EDispatchPrivate(t *testing.T) { bmi, _ := NewBatchManager(ctx, mni, mdi, mdm) bm := bmi.(*batchManager) - bm.RegisterDispatcher([]fftypes.MessageType{fftypes.MessageTypePrivate}, handler, Options{ + bm.RegisterDispatcher("utdispatcher", []fftypes.MessageType{fftypes.MessageTypePrivate}, handler, DispatcherOptions{ BatchMaxSize: 2, BatchTimeout: 0, DisposeTimeout: 120 * time.Second, @@ -223,7 +209,7 @@ func TestE2EDispatchPrivate(t *testing.T) { mdi.On("UpdateMessages", mock.Anything, mock.MatchedBy(func(f database.Filter) bool { fi, err := f.Finalize() assert.NoError(t, err) - assert.Equal(t, fmt.Sprintf("id IN ['%s']", msg.Header.ID.String()), fi.String()) + assert.Equal(t, fmt.Sprintf("( id IN ['%s'] ) && ( state == 'ready' )", msg.Header.ID.String()), fi.String()) return true }), mock.Anything).Return(nil) ugcn := mdi.On("UpsertNonceNext", mock.Anything, mock.Anything).Return(nil) @@ -256,59 +242,11 @@ func TestInitFailNoPersistence(t *testing.T) { assert.Error(t, err) } -func TestInitRestoreExistingOffset(t *testing.T) { - mdi := &databasemocks.Plugin{} - mdm := &datamocks.Manager{} - mni := &sysmessagingmocks.LocalNodeInfo{} - mdi.On("GetOffset", mock.Anything, fftypes.OffsetTypeBatch, msgBatchOffsetName).Return(&fftypes.Offset{ - Type: fftypes.OffsetTypeBatch, - Name: msgBatchOffsetName, - Current: 12345, - }, nil) - bm, err := NewBatchManager(context.Background(), mni, mdi, mdm) - assert.NoError(t, err) - defer bm.Close() - err = bm.Start() - assert.NoError(t, err) - assert.Equal(t, int64(12345), bm.(*batchManager).readOffset) -} - -func TestInitFailCannotRestoreOffset(t *testing.T) { - mdi := &databasemocks.Plugin{} - mdm := &datamocks.Manager{} - mni := &sysmessagingmocks.LocalNodeInfo{} - mdi.On("GetOffset", mock.Anything, fftypes.OffsetTypeBatch, msgBatchOffsetName).Return(nil, fmt.Errorf("pop")) - bm, err := NewBatchManager(context.Background(), mni, mdi, mdm) - assert.NoError(t, err) - defer bm.Close() - bm.(*batchManager).retry.MaximumDelay = 1 * time.Microsecond - err = bm.Start() - assert.Regexp(t, "pop", err) -} - -func TestInitFailCannotCreateOffset(t *testing.T) { - mdi := &databasemocks.Plugin{} - mdm := &datamocks.Manager{} - mni := &sysmessagingmocks.LocalNodeInfo{} - mdi.On("GetOffset", mock.Anything, fftypes.OffsetTypeBatch, msgBatchOffsetName).Return(nil, nil).Once() - mdi.On("UpsertOffset", mock.Anything, mock.Anything, mock.Anything).Return(fmt.Errorf("pop")) - mdi.On("GetOffset", mock.Anything, fftypes.OffsetTypeBatch, msgBatchOffsetName).Return(nil, fmt.Errorf("pop")) - bm, err := NewBatchManager(context.Background(), mni, mdi, mdm) - assert.NoError(t, err) - defer bm.Close() - bm.(*batchManager).retry.MaximumDelay = 1 * time.Microsecond - err = bm.Start() - assert.Regexp(t, "pop", err) -} - func TestGetInvalidBatchTypeMsg(t *testing.T) { mdi := &databasemocks.Plugin{} mdm := &datamocks.Manager{} mni := &sysmessagingmocks.LocalNodeInfo{} - mdi.On("GetOffset", mock.Anything, fftypes.OffsetTypeBatch, msgBatchOffsetName).Return(&fftypes.Offset{ - Current: 12345, - }, nil) bm, _ := NewBatchManager(context.Background(), mni, mdi, mdm) defer bm.Close() msg := &fftypes.Message{Header: fftypes.MessageHeader{}} @@ -337,24 +275,26 @@ func TestMessageSequencerMissingMessageData(t *testing.T) { bm, _ := NewBatchManager(context.Background(), mni, mdi, mdm) dataID := fftypes.NewUUID() - gmMock := mdi.On("GetMessages", mock.Anything, mock.Anything, mock.Anything).Return([]*fftypes.Message{ - { - Header: fftypes.MessageHeader{ - ID: fftypes.NewUUID(), - Namespace: "ns1", - }, - Data: []*fftypes.DataRef{ - {ID: dataID}, - }}, - }, nil, nil) - gmMock.RunFn = func(a mock.Arguments) { - bm.Close() // so we only go round once - } + mdi.On("GetMessages", mock.Anything, mock.Anything, mock.Anything). + Return([]*fftypes.Message{ + { + Header: fftypes.MessageHeader{ + ID: fftypes.NewUUID(), + Namespace: "ns1", + }, + Data: []*fftypes.DataRef{ + {ID: dataID}, + }}, + }, nil, nil). + Run(func(args mock.Arguments) { + bm.Close() + }). + Once() + mdi.On("GetMessages", mock.Anything, mock.Anything, mock.Anything).Return([]*fftypes.Message{}, nil, nil) mdm.On("GetMessageData", mock.Anything, mock.Anything, true).Return(nil, false, nil) bm.(*batchManager).messageSequencer() - bm.Close() bm.WaitStop() mdi.AssertExpectations(t) @@ -368,25 +308,26 @@ func TestMessageSequencerDispatchFail(t *testing.T) { bm, _ := NewBatchManager(context.Background(), mni, mdi, mdm) dataID := fftypes.NewUUID() - gmMock := mdi.On("GetMessages", mock.Anything, mock.Anything, mock.Anything).Return([]*fftypes.Message{ - { - Header: fftypes.MessageHeader{ - ID: fftypes.NewUUID(), - Type: fftypes.MessageTypePrivate, - Namespace: "ns1", - }, - Data: []*fftypes.DataRef{ - {ID: dataID}, - }}, - }, nil, nil) - gmMock.RunFn = func(a mock.Arguments) { - bm.Close() // so we only go round once - } + mdi.On("GetMessages", mock.Anything, mock.Anything, mock.Anything). + Return([]*fftypes.Message{ + { + Header: fftypes.MessageHeader{ + ID: fftypes.NewUUID(), + Namespace: "ns1", + }, + Data: []*fftypes.DataRef{ + {ID: dataID}, + }}, + }, nil, nil). + Run(func(args mock.Arguments) { + bm.Close() + }). + Once() + mdi.On("GetMessages", mock.Anything, mock.Anything, mock.Anything).Return([]*fftypes.Message{}, nil, nil) mdm.On("GetMessageData", mock.Anything, mock.Anything, true).Return([]*fftypes.Data{{ID: dataID}}, true, nil) bm.(*batchManager).messageSequencer() - bm.Close() bm.WaitStop() mdi.AssertExpectations(t) @@ -400,9 +341,9 @@ func TestMessageSequencerUpdateMessagesFail(t *testing.T) { mni.On("GetNodeUUID", mock.Anything).Return(fftypes.NewUUID()) ctx, cancelCtx := context.WithCancel(context.Background()) bm, _ := NewBatchManager(ctx, mni, mdi, mdm) - bm.RegisterDispatcher([]fftypes.MessageType{fftypes.MessageTypeBroadcast}, func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { + bm.RegisterDispatcher("utdispatcher", []fftypes.MessageType{fftypes.MessageTypeBroadcast}, func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { return nil - }, Options{BatchMaxSize: 1, DisposeTimeout: 0}) + }, DispatcherOptions{BatchMaxSize: 1, DisposeTimeout: 0}) dataID := fftypes.NewUUID() mdi.On("GetMessages", mock.Anything, mock.Anything, mock.Anything).Return([]*fftypes.Message{ @@ -417,13 +358,16 @@ func TestMessageSequencerUpdateMessagesFail(t *testing.T) { }}, }, nil, nil) mdm.On("GetMessageData", mock.Anything, mock.Anything, true).Return([]*fftypes.Data{{ID: dataID}}, true, nil) + mdi.On("InsertTransaction", mock.Anything, mock.Anything).Return(nil) + mdi.On("InsertEvent", mock.Anything, mock.Anything).Return(nil) // transaction submit + mdi.On("UpsertBatch", mock.Anything, mock.Anything, mock.Anything).Return(nil) mdi.On("UpdateMessages", mock.Anything, mock.Anything, mock.Anything).Return(fmt.Errorf("fizzle")) rag := mdi.On("RunAsGroup", mock.Anything, mock.Anything, mock.Anything) rag.RunFn = func(a mock.Arguments) { ctx := a.Get(0).(context.Context) fn := a.Get(1).(func(context.Context) error) - err := fn(ctx).(error) - if err != nil && err.Error() == "fizzle" { + err, ok := fn(ctx).(error) + if ok && err.Error() == "fizzle" { cancelCtx() // so we only go round once bm.Close() } @@ -446,9 +390,9 @@ func TestMessageSequencerUpdateBatchFail(t *testing.T) { mni.On("GetNodeUUID", mock.Anything).Return(fftypes.NewUUID()) ctx, cancelCtx := context.WithCancel(context.Background()) bm, _ := NewBatchManager(ctx, mni, mdi, mdm) - bm.RegisterDispatcher([]fftypes.MessageType{fftypes.MessageTypeBroadcast}, func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { + bm.RegisterDispatcher("utdispatcher", []fftypes.MessageType{fftypes.MessageTypeBroadcast}, func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { return nil - }, Options{BatchMaxSize: 1, DisposeTimeout: 0}) + }, DispatcherOptions{BatchMaxSize: 1, DisposeTimeout: 0}) dataID := fftypes.NewUUID() mdi.On("GetMessages", mock.Anything, mock.Anything, mock.Anything).Return([]*fftypes.Message{ @@ -463,14 +407,13 @@ func TestMessageSequencerUpdateBatchFail(t *testing.T) { }}, }, nil, nil) mdm.On("GetMessageData", mock.Anything, mock.Anything, true).Return([]*fftypes.Data{{ID: dataID}}, true, nil) - mdi.On("UpdateMessages", mock.Anything, mock.Anything, mock.Anything).Return(nil) mdi.On("UpsertBatch", mock.Anything, mock.Anything, mock.Anything).Return(fmt.Errorf("fizzle")) rag := mdi.On("RunAsGroup", mock.Anything, mock.Anything, mock.Anything) rag.RunFn = func(a mock.Arguments) { ctx := a.Get(0).(context.Context) fn := a.Get(1).(func(context.Context) error) - err := fn(ctx).(error) - if err != nil && err.Error() == "fizzle" { + err, ok := fn(ctx).(error) + if ok && err.Error() == "fizzle" { cancelCtx() // so we only go round once bm.Close() } @@ -520,7 +463,7 @@ func TestGetMessageDataFail(t *testing.T) { bm, _ := NewBatchManager(context.Background(), mni, mdi, mdm) mdm.On("GetMessageData", mock.Anything, mock.Anything, true).Return(nil, false, fmt.Errorf("pop")) bm.Close() - _, err := bm.(*batchManager).assembleMessageData(&fftypes.Message{ + _, _ = bm.(*batchManager).assembleMessageData(&fftypes.Message{ Header: fftypes.MessageHeader{ ID: fftypes.NewUUID(), }, @@ -528,7 +471,7 @@ func TestGetMessageDataFail(t *testing.T) { {ID: fftypes.NewUUID(), Hash: fftypes.NewRandB32()}, }, }) - assert.EqualError(t, err, "pop") + mdm.AssertExpectations(t) } func TestGetMessageNotFound(t *testing.T) { diff --git a/internal/batch/batch_processor.go b/internal/batch/batch_processor.go index 671aa043ed..91f8911933 100644 --- a/internal/batch/batch_processor.go +++ b/internal/batch/batch_processor.go @@ -21,7 +21,6 @@ import ( "crypto/sha256" "database/sql/driver" "encoding/binary" - "fmt" "math" "sync" "time" @@ -46,12 +45,13 @@ type batchDispatch struct { } type batchProcessorConf struct { - Options + DispatcherOptions + name string + dispatcherName string namespace string identity fftypes.Identity group *fftypes.Bytes32 dispatch DispatchHandler - requestQuiesce func() bool } // FlushStatus is an object that can be returned on REST queries to understand the status @@ -59,14 +59,16 @@ type batchProcessorConf struct { type FlushStatus struct { LastFlushTime *fftypes.FFTime `json:"lastFlushStartTime"` Flushing *fftypes.UUID `json:"flushing,omitempty"` + Blocked bool `json:"blocked"` LastFlushError string `json:"lastFlushError,omitempty"` LastFlushErrorTime *fftypes.FFTime `json:"lastFlushErrorTime,omitempty"` AverageBatchBytes int64 `json:"averageBatchBytes"` AverageBatchMessages float64 `json:"averageBatchMessages"` AverageBatchData float64 `json:"averageBatchData"` AverageFlushTimeMS int64 `json:"averageFlushTimeMS"` + TotalBatches int64 `json:"totalBatches"` + TotalErrors int64 `json:"totalErrors"` - totalBatches int64 totalBytesFlushed int64 totalMessagesFlushed int64 totalDataFlushed int64 @@ -78,9 +80,9 @@ type batchProcessor struct { ni sysmessaging.LocalNodeInfo database database.Plugin txHelper txcommon.Helper - name string cancelCtx func() done chan struct{} + quescing chan bool newWork chan *batchWork assemblyID *fftypes.UUID assemblyQueue []*batchWork @@ -94,25 +96,31 @@ type batchProcessor struct { const batchSizeEstimateBase = int64(512) -func newBatchProcessor(ctx context.Context, ni sysmessaging.LocalNodeInfo, di database.Plugin, conf *batchProcessorConf, retry *retry.Retry) *batchProcessor { - name := fmt.Sprintf("%s:%s:%s", conf.namespace, conf.identity.Author, conf.identity.Key) - pCtx := log.WithLogField(ctx, "role", name) +func newBatchProcessor(ctx context.Context, ni sysmessaging.LocalNodeInfo, di database.Plugin, conf *batchProcessorConf, baseRetryConf *retry.Retry) *batchProcessor { + pCtx := log.WithLogField(log.WithLogField(ctx, "d", conf.dispatcherName), "p", conf.name) pCtx, cancelCtx := context.WithCancel(pCtx) bp := &batchProcessor{ - ctx: pCtx, - cancelCtx: cancelCtx, - ni: ni, - database: di, - txHelper: txcommon.NewTransactionHelper(di), - name: name, - newWork: make(chan *batchWork, conf.BatchMaxSize), - retry: retry, + ctx: pCtx, + cancelCtx: cancelCtx, + ni: ni, + database: di, + txHelper: txcommon.NewTransactionHelper(di), + newWork: make(chan *batchWork, conf.BatchMaxSize), + quescing: make(chan bool, 1), + done: make(chan struct{}), + retry: &retry.Retry{ + InitialDelay: baseRetryConf.InitialDelay, + MaximumDelay: baseRetryConf.MaximumDelay, + Factor: baseRetryConf.Factor, + }, conf: conf, flushedSequences: []int64{}, flushStatus: FlushStatus{ LastFlushTime: fftypes.Now(), }, } + // Capture flush errors for our status + bp.retry.ErrCallback = bp.captureFlushError bp.newAssembly() go bp.assemblyLoop() log.L(pCtx).Infof("Batch processor created") @@ -127,11 +135,14 @@ func (bw *batchWork) estimateSize() int64 { return sizeEstimate } -func (bp *batchProcessor) Status() *FlushStatus { +func (bp *batchProcessor) status() *ProcessorStatus { bp.statusMux.Lock() defer bp.statusMux.Unlock() - statusCopy := bp.flushStatus - return &statusCopy + return &ProcessorStatus{ + Dispatcher: bp.conf.dispatcherName, + Name: bp.conf.name, + Status: bp.flushStatus, // copy + } } func (bp *batchProcessor) newAssembly(initalWork ...*batchWork) { @@ -184,8 +195,9 @@ func (bp *batchProcessor) addWork(newWork *batchWork) (overflow bool) { func (bp *batchProcessor) startFlush(overflow bool) (id *fftypes.UUID, flushAssembly []*batchWork, byteSize int64) { bp.statusMux.Lock() - defer bp.statusMux.Lock() + defer bp.statusMux.Unlock() // Star the clock + bp.flushStatus.Blocked = false bp.flushStatus.LastFlushTime = fftypes.Now() // Split the current work if required for overflow overflowWork := make([]*batchWork, 0) @@ -205,7 +217,7 @@ func (bp *batchProcessor) startFlush(overflow bool) (id *fftypes.UUID, flushAsse if newFlushedSeqLen > maxFlushedSeqLen && maxFlushedSeqLen > len(bp.flushedSequences) { newFlushedSeqLen = maxFlushedSeqLen } - newFlushedSequnces := make([]int64, 0, newFlushedSeqLen) + newFlushedSequnces := make([]int64, newFlushedSeqLen) for i, fs := range flushAssembly { newFlushedSequnces[i] = fs.msg.Sequence } @@ -222,25 +234,36 @@ func (bp *batchProcessor) startFlush(overflow bool) (id *fftypes.UUID, flushAsse func (bp *batchProcessor) endFlush(batch *fftypes.Batch, byteSize int64) { bp.statusMux.Lock() - defer bp.statusMux.Lock() + defer bp.statusMux.Unlock() fs := &bp.flushStatus duration := time.Since(*fs.LastFlushTime.Time()) fs.Flushing = nil - fs.totalBatches++ + fs.TotalBatches++ fs.totalFlushDuration += duration - fs.AverageFlushTimeMS = (fs.totalFlushDuration / time.Duration(fs.totalBatches)).Milliseconds() + fs.AverageFlushTimeMS = (fs.totalFlushDuration / time.Duration(fs.TotalBatches)).Milliseconds() fs.totalBytesFlushed += byteSize - fs.AverageBatchBytes = (fs.totalBytesFlushed / fs.totalBatches) + fs.AverageBatchBytes = (fs.totalBytesFlushed / fs.TotalBatches) fs.totalMessagesFlushed += int64(len(batch.Payload.Messages)) - fs.AverageBatchMessages = math.Round((float64(fs.totalMessagesFlushed)/float64(fs.totalBatches))*100) / 100 + fs.AverageBatchMessages = math.Round((float64(fs.totalMessagesFlushed)/float64(fs.TotalBatches))*100) / 100 fs.totalDataFlushed += int64(len(batch.Payload.Data)) - fs.AverageBatchData = math.Round((float64(fs.totalDataFlushed)/float64(fs.totalBatches))*100) / 100 + fs.AverageBatchData = math.Round((float64(fs.totalDataFlushed)/float64(fs.TotalBatches))*100) / 100 +} + +func (bp *batchProcessor) captureFlushError(err error) { + bp.statusMux.Lock() + defer bp.statusMux.Unlock() + fs := &bp.flushStatus + + fs.TotalErrors++ + fs.Blocked = true + fs.LastFlushErrorTime = fftypes.Now() + fs.LastFlushError = err.Error() } // The assemblyLoop receives new work, sorts it, and waits for the size/timer to pop before @@ -268,7 +291,10 @@ func (bp *batchProcessor) assemblyLoop() { // It probably makes sense to exit, but we don't know if more work is coming our way, // so we just inform the manager we've hit our dispose timeout and see if they close // the input work channel (which they can safely do on the dispatcher routine) - bp.conf.requestQuiesce() + select { + case bp.quescing <- true: + default: + } } else { // We need to flush timedout = true @@ -450,7 +476,7 @@ func (bp *batchProcessor) markMessagesDispatched(batch *fftypes.Batch) error { return bp.retry.Do(bp.ctx, "batch persist", func(attempt int) (retry bool, err error) { return true, bp.database.RunAsGroup(bp.ctx, func(ctx context.Context) (err error) { // Update all the messages in the batch with the batch ID - msgIDs := make([]driver.Value, 0, len(batch.Payload.Messages)) + msgIDs := make([]driver.Value, len(batch.Payload.Messages)) for i, msg := range batch.Payload.Messages { msgIDs[i] = msg.Header.ID } @@ -466,8 +492,3 @@ func (bp *batchProcessor) markMessagesDispatched(batch *fftypes.Batch) error { }) }) } - -func (bp *batchProcessor) close() { - bp.cancelCtx() - <-bp.done -} diff --git a/internal/batch/batch_processor_test.go b/internal/batch/batch_processor_test.go index 4089b5140f..5456865c5d 100644 --- a/internal/batch/batch_processor_test.go +++ b/internal/batch/batch_processor_test.go @@ -39,7 +39,7 @@ func newTestBatchProcessor(dispatch DispatchHandler) (*databasemocks.Plugin, *ba namespace: "ns1", identity: fftypes.Identity{Author: "did:firefly:org/abcd", Key: "0x12345"}, dispatch: dispatch, - Options: Options{ + DispatcherOptions: DispatcherOptions{ BatchMaxSize: 10, BatchMaxBytes: 1024 * 1024, BatchTimeout: 10 * time.Millisecond, @@ -73,6 +73,7 @@ func TestUnfilledBatch(t *testing.T) { wg.Done() return nil }) + mockRunAsGroupPassthrough(mdi) mdi.On("UpdateMessages", mock.Anything, mock.Anything, mock.Anything).Return(nil) mdi.On("UpsertBatch", mock.Anything, mock.Anything, mock.Anything).Return(nil) @@ -86,7 +87,7 @@ func TestUnfilledBatch(t *testing.T) { for i := 0; i < len(work); i++ { msgid := fftypes.NewUUID() work[i] = &batchWork{ - msg: &fftypes.Message{Header: fftypes.MessageHeader{ID: msgid}}, + msg: &fftypes.Message{Header: fftypes.MessageHeader{ID: msgid}, Sequence: int64(1000 + i)}, dispatched: make(chan *batchDispatch), } } @@ -110,9 +111,8 @@ func TestUnfilledBatch(t *testing.T) { // Check we got all the messages in a single batch assert.Equal(t, len(dispatched[0].Payload.Messages), len(work)) - bp.close() - bp.close() - + bp.cancelCtx() + <-bp.done } func TestBatchSizeOverflow(t *testing.T) { @@ -166,9 +166,8 @@ func TestBatchSizeOverflow(t *testing.T) { assert.Equal(t, len(dispatched[0].Payload.Messages), 1) assert.Equal(t, len(dispatched[1].Payload.Messages), 1) - bp.close() - bp.close() - + bp.cancelCtx() + <-bp.done } func TestFilledBatchSlowPersistence(t *testing.T) { @@ -237,17 +236,17 @@ func TestFilledBatchSlowPersistence(t *testing.T) { assert.Equal(t, len(dispatched[0].Payload.Messages), 5) assert.Equal(t, len(dispatched[0].Payload.Data), 5) - bp.close() - bp.close() - + bp.cancelCtx() + <-bp.done } func TestCloseToUnblockDispatch(t *testing.T) { _, bp := newTestBatchProcessor(func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { return fmt.Errorf("pop") }) - bp.close() + bp.cancelCtx() bp.dispatchBatch(&fftypes.Batch{}, []*fftypes.Bytes32{}) + <-bp.done } func TestCloseToUnblockUpsertBatch(t *testing.T) { @@ -284,16 +283,14 @@ func TestCloseToUnblockUpsertBatch(t *testing.T) { close(waitForCall) // Close to unblock - bp.close() - bp.close() - + bp.cancelCtx() + <-bp.done } func TestCalcPinsFail(t *testing.T) { _, bp := newTestBatchProcessor(func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { return nil }) - defer bp.close() mdi := bp.database.(*databasemocks.Plugin) mdi.On("UpsertNonceNext", mock.Anything, mock.Anything).Return(fmt.Errorf("pop")) @@ -310,4 +307,7 @@ func TestCalcPinsFail(t *testing.T) { }, }) assert.Regexp(t, "pop", err) + + bp.cancelCtx() + <-bp.done } diff --git a/internal/broadcast/manager.go b/internal/broadcast/manager.go index 725b307e9d..aeec8d1b91 100644 --- a/internal/broadcast/manager.go +++ b/internal/broadcast/manager.go @@ -37,6 +37,8 @@ import ( "github.com/hyperledger/firefly/pkg/publicstorage" ) +const broadcastDispatcherName = "pinned_broadcast" + type Manager interface { NewBroadcast(ns string, in *fftypes.MessageInOut) sysmessaging.MessageSender BroadcastDatatype(ctx context.Context, ns string, datatype *fftypes.Datatype, waitConfirm bool) (msg *fftypes.Message, err error) @@ -81,13 +83,13 @@ func NewBroadcastManager(ctx context.Context, di database.Plugin, im identity.Ma batchpin: bp, maxBatchPayloadLength: config.GetByteSize(config.BroadcastBatchPayloadLimit), } - bo := batch.Options{ + bo := batch.DispatcherOptions{ BatchMaxSize: config.GetUint(config.BroadcastBatchSize), BatchMaxBytes: bm.maxBatchPayloadLength, BatchTimeout: config.GetDuration(config.BroadcastBatchTimeout), DisposeTimeout: config.GetDuration(config.BroadcastBatchAgentTimeout), } - ba.RegisterDispatcher([]fftypes.MessageType{ + ba.RegisterDispatcher(broadcastDispatcherName, []fftypes.MessageType{ fftypes.MessageTypeBroadcast, fftypes.MessageTypeDefinition, fftypes.MessageTypeTransferBroadcast, diff --git a/internal/privatemessaging/privatemessaging.go b/internal/privatemessaging/privatemessaging.go index b93b666749..f579f874da 100644 --- a/internal/privatemessaging/privatemessaging.go +++ b/internal/privatemessaging/privatemessaging.go @@ -37,6 +37,8 @@ import ( "github.com/karlseguin/ccache" ) +const pinnedPrivateDispatcherName = "pinned_private" + type Manager interface { GroupManager @@ -100,14 +102,14 @@ func NewPrivateMessaging(ctx context.Context, di database.Plugin, im identity.Ma MaxSize(config.GetByteSize(config.GroupCacheSize)), ) - bo := batch.Options{ + bo := batch.DispatcherOptions{ BatchMaxSize: config.GetUint(config.PrivateMessagingBatchSize), BatchMaxBytes: pm.maxBatchPayloadLength, BatchTimeout: config.GetDuration(config.PrivateMessagingBatchTimeout), DisposeTimeout: config.GetDuration(config.PrivateMessagingBatchAgentTimeout), } - ba.RegisterDispatcher([]fftypes.MessageType{ + ba.RegisterDispatcher(pinnedPrivateDispatcherName, []fftypes.MessageType{ fftypes.MessageTypeGroupInit, fftypes.MessageTypePrivate, fftypes.MessageTypeTransferPrivate, diff --git a/internal/retry/retry.go b/internal/retry/retry.go index 907136ea4a..5d5b1ceafe 100644 --- a/internal/retry/retry.go +++ b/internal/retry/retry.go @@ -1,4 +1,4 @@ -// Copyright © 2021 Kaleido, Inc. +// Copyright © 2022 Kaleido, Inc. // // SPDX-License-Identifier: Apache-2.0 // @@ -33,6 +33,7 @@ type Retry struct { InitialDelay time.Duration MaximumDelay time.Duration Factor float64 + ErrCallback func(err error) } // DoCustomLog disables the automatic attempt logging, so the caller should do logging for each attempt @@ -55,6 +56,9 @@ func (r *Retry) Do(ctx context.Context, logDescription string, f func(attempt in retry, err := f(attempt) if err != nil && logDescription != "" { log.L(ctx).Errorf("%s attempt %d: %s", logDescription, attempt, err) + if r.ErrCallback != nil { + r.ErrCallback(err) + } } if !retry || err == nil { return err diff --git a/internal/retry/retry_test.go b/internal/retry/retry_test.go index 7a0ece331c..e2c72c970c 100644 --- a/internal/retry/retry_test.go +++ b/internal/retry/retry_test.go @@ -26,13 +26,18 @@ import ( ) func TestRetryEventuallyOk(t *testing.T) { + var capturedErr error r := Retry{ MaximumDelay: 3 * time.Microsecond, InitialDelay: 1 * time.Microsecond, + ErrCallback: func(err error) { + capturedErr = err + }, } r.Do(context.Background(), "unit test", func(i int) (retry bool, err error) { return i < 10, fmt.Errorf("pop") }) + assert.EqualError(t, capturedErr, "pop") } func TestRetryDeadlineTimeout(t *testing.T) { diff --git a/mocks/batchmocks/manager.go b/mocks/batchmocks/manager.go index 79d7bf7b00..4ade0f2ab4 100644 --- a/mocks/batchmocks/manager.go +++ b/mocks/batchmocks/manager.go @@ -35,9 +35,9 @@ func (_m *Manager) NewMessages() chan<- int64 { return r0 } -// RegisterDispatcher provides a mock function with given fields: msgTypes, handler, batchOptions -func (_m *Manager) RegisterDispatcher(msgTypes []fftypes.FFEnum, handler batch.DispatchHandler, batchOptions batch.Options) { - _m.Called(msgTypes, handler, batchOptions) +// RegisterDispatcher provides a mock function with given fields: name, msgTypes, handler, batchOptions +func (_m *Manager) RegisterDispatcher(name string, msgTypes []fftypes.FFEnum, handler batch.DispatchHandler, batchOptions batch.DispatcherOptions) { + _m.Called(name, msgTypes, handler, batchOptions) } // Start provides a mock function with given fields: @@ -54,6 +54,22 @@ func (_m *Manager) Start() error { return r0 } +// Status provides a mock function with given fields: +func (_m *Manager) Status() []*batch.ProcessorStatus { + ret := _m.Called() + + var r0 []*batch.ProcessorStatus + if rf, ok := ret.Get(0).(func() []*batch.ProcessorStatus); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).([]*batch.ProcessorStatus) + } + } + + return r0 +} + // WaitStop provides a mock function with given fields: func (_m *Manager) WaitStop() { _m.Called() From 80c93cb8a5a71b9cc8218688102dd9492f8206a2 Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Wed, 9 Feb 2022 12:37:15 -0500 Subject: [PATCH 05/23] More enhancements on logic Signed-off-by: Peter Broadhurst --- internal/batch/batch_manager.go | 27 +--- internal/batch/batch_manager_test.go | 5 +- internal/batch/batch_processor.go | 24 ++-- internal/batch/batch_processor_test.go | 188 ++++++------------------- 4 files changed, 61 insertions(+), 183 deletions(-) diff --git a/internal/batch/batch_manager.go b/internal/batch/batch_manager.go index 1e8a845f19..edeb23de64 100644 --- a/internal/batch/batch_manager.go +++ b/internal/batch/batch_manager.go @@ -196,8 +196,6 @@ func (bm *batchManager) messageSequencer() { l.Debugf("Started batch assembly message sequencer") defer close(bm.done) - dispatched := make(chan *batchDispatch, bm.readPageSize) - for { // Each time round the loop we check for quiescing processors bm.reapQuiescing() @@ -208,11 +206,9 @@ func (bm *batchManager) messageSequencer() { l.Debugf("Exiting: %s", err) return } - batchWasFull := false + batchWasFull := (uint64(len(msgs)) == bm.readPageSize) if len(msgs) > 0 { - batchWasFull = (uint64(len(msgs)) == bm.readPageSize) - var dispatchCount int for _, msg := range msgs { data, err := bm.assembleMessageData(msg) if err != nil { @@ -220,23 +216,11 @@ func (bm *batchManager) messageSequencer() { continue } - err = bm.dispatchMessage(dispatched, msg, data...) + err = bm.dispatchMessage(msg, data...) if err != nil { l.Errorf("Failed to dispatch message %s: %s", msg.Header.ID, err) continue } - dispatchCount++ - } - - for i := 0; i < dispatchCount; i++ { - select { - case dispatched := <-dispatched: - l.Debugf("Dispatched message %s to batch %s", dispatched.msg.Header.ID, dispatched.batchID) - case <-bm.ctx.Done(): - l.Debugf("Message sequencer exiting (context closed)") - bm.Close() - return - } } // Next time round only read after the messages we just processed (unless we get a tap to rewind) @@ -288,7 +272,7 @@ func (bm *batchManager) waitForShoulderTapOrPollTimeout() (done bool) { } } -func (bm *batchManager) dispatchMessage(dispatched chan *batchDispatch, msg *fftypes.Message, data ...*fftypes.Data) error { +func (bm *batchManager) dispatchMessage(msg *fftypes.Message, data ...*fftypes.Data) error { l := log.L(bm.ctx) processor, err := bm.getProcessor(msg.Header.Type, msg.Header.Group, msg.Header.Namespace, &msg.Header.Identity) if err != nil { @@ -296,9 +280,8 @@ func (bm *batchManager) dispatchMessage(dispatched chan *batchDispatch, msg *fft } l.Debugf("Dispatching message %s to %s batch processor %s", msg.Header.ID, msg.Header.Type, processor.conf.name) work := &batchWork{ - msg: msg, - data: data, - dispatched: dispatched, + msg: msg, + data: data, } processor.newWork <- work return nil diff --git a/internal/batch/batch_manager_test.go b/internal/batch/batch_manager_test.go index e45bc4bf04..aa217b97dc 100644 --- a/internal/batch/batch_manager_test.go +++ b/internal/batch/batch_manager_test.go @@ -22,6 +22,7 @@ import ( "testing" "time" + "github.com/hyperledger/firefly/internal/config" "github.com/hyperledger/firefly/internal/log" "github.com/hyperledger/firefly/mocks/databasemocks" "github.com/hyperledger/firefly/mocks/datamocks" @@ -34,6 +35,7 @@ import ( func TestE2EDispatchBroadcast(t *testing.T) { log.SetLevel("debug") + config.Reset() mdi := &databasemocks.Plugin{} mdm := &datamocks.Manager{} @@ -131,6 +133,7 @@ func TestE2EDispatchBroadcast(t *testing.T) { func TestE2EDispatchPrivate(t *testing.T) { log.SetLevel("debug") + config.Reset() mdi := &databasemocks.Plugin{} mdm := &datamocks.Manager{} @@ -250,7 +253,7 @@ func TestGetInvalidBatchTypeMsg(t *testing.T) { bm, _ := NewBatchManager(context.Background(), mni, mdi, mdm) defer bm.Close() msg := &fftypes.Message{Header: fftypes.MessageHeader{}} - err := bm.(*batchManager).dispatchMessage(nil, msg) + err := bm.(*batchManager).dispatchMessage(msg) assert.Regexp(t, "FF10126", err) } diff --git a/internal/batch/batch_processor.go b/internal/batch/batch_processor.go index 91f8911933..bb7e8ad431 100644 --- a/internal/batch/batch_processor.go +++ b/internal/batch/batch_processor.go @@ -34,14 +34,8 @@ import ( ) type batchWork struct { - msg *fftypes.Message - data []*fftypes.Data - dispatched chan *batchDispatch -} - -type batchDispatch struct { - msg *fftypes.Message - batchID *fftypes.UUID + msg *fftypes.Message + data []*fftypes.Data } type batchProcessorConf struct { @@ -156,7 +150,7 @@ func (bp *batchProcessor) newAssembly(initalWork ...*batchWork) { // This helps in the case for parallel REST APIs all committing to the DB at a similar time. // With a sufficient batch size and batch timeout, the batch will still dispatch the messages // in DB sequence order (although this is not guaranteed). -func (bp *batchProcessor) addWork(newWork *batchWork) (overflow bool) { +func (bp *batchProcessor) addWork(newWork *batchWork) (full, overflow bool) { newQueue := make([]*batchWork, 0, len(bp.assemblyQueue)+1) added := false skip := false @@ -189,8 +183,9 @@ func (bp *batchProcessor) addWork(newWork *batchWork) (overflow bool) { bp.assemblyQueueBytes += newWork.estimateSize() } bp.assemblyQueue = newQueue - overflow = len(bp.assemblyQueue) > 1 && (bp.assemblyQueueBytes > bp.conf.BatchMaxBytes || len(bp.assemblyQueue) > int(bp.conf.BatchMaxSize)) - return overflow + full = len(bp.assemblyQueue) >= int(bp.conf.BatchMaxSize) || (bp.assemblyQueueBytes >= bp.conf.BatchMaxBytes) + overflow = len(bp.assemblyQueue) > 1 && (bp.assemblyQueueBytes > bp.conf.BatchMaxBytes) + return full, overflow } func (bp *batchProcessor) startFlush(overflow bool) (id *fftypes.UUID, flushAssembly []*batchWork, byteSize int64) { @@ -273,13 +268,12 @@ func (bp *batchProcessor) assemblyLoop() { defer close(bp.done) l := log.L(bp.ctx) - overflow := false var batchTimeout = time.NewTimer(bp.conf.DisposeTimeout) idle := true quescing := false for !quescing { - var timedout bool + var timedout, full, overflow bool select { case <-bp.ctx.Done(): l.Tracef("Batch processor shutting down") @@ -303,7 +297,7 @@ func (bp *batchProcessor) assemblyLoop() { if !ok { quescing = true } else { - overflow = bp.addWork(work) + full, overflow = bp.addWork(work) if idle { // We've hit a message while we were idle - we now need to wait for the batch to time out. _ = batchTimeout.Stop() @@ -312,7 +306,7 @@ func (bp *batchProcessor) assemblyLoop() { } } } - if overflow || timedout || quescing { + if full || timedout || quescing { // Let Go GC the old timer _ = batchTimeout.Stop() diff --git a/internal/batch/batch_processor_test.go b/internal/batch/batch_processor_test.go index 5456865c5d..d9685368fd 100644 --- a/internal/batch/batch_processor_test.go +++ b/internal/batch/batch_processor_test.go @@ -17,10 +17,10 @@ package batch import ( "context" "fmt" - "sync" "testing" "time" + "github.com/hyperledger/firefly/internal/config" "github.com/hyperledger/firefly/internal/log" "github.com/hyperledger/firefly/internal/retry" "github.com/hyperledger/firefly/mocks/databasemocks" @@ -42,8 +42,8 @@ func newTestBatchProcessor(dispatch DispatchHandler) (*databasemocks.Plugin, *ba DispatcherOptions: DispatcherOptions{ BatchMaxSize: 10, BatchMaxBytes: 1024 * 1024, - BatchTimeout: 10 * time.Millisecond, - DisposeTimeout: 20 * time.Millisecond, + BatchTimeout: 100 * time.Millisecond, + DisposeTimeout: 200 * time.Millisecond, }, }, &retry.Retry{ InitialDelay: 1 * time.Microsecond, @@ -63,14 +63,11 @@ func mockRunAsGroupPassthrough(mdi *databasemocks.Plugin) { func TestUnfilledBatch(t *testing.T) { log.SetLevel("debug") + config.Reset() - wg := sync.WaitGroup{} - wg.Add(2) - - dispatched := []*fftypes.Batch{} + dispatched := make(chan *fftypes.Batch) mdi, bp := newTestBatchProcessor(func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { - dispatched = append(dispatched, b) - wg.Done() + dispatched <- b return nil }) @@ -82,34 +79,21 @@ func TestUnfilledBatch(t *testing.T) { mth := bp.txHelper.(*txcommonmocks.Helper) mth.On("SubmitNewTransaction", mock.Anything, "ns1", fftypes.TransactionTypeBatchPin).Return(fftypes.NewUUID(), nil) - // Generate the work - work := make([]*batchWork, 5) - for i := 0; i < len(work); i++ { - msgid := fftypes.NewUUID() - work[i] = &batchWork{ - msg: &fftypes.Message{Header: fftypes.MessageHeader{ID: msgid}, Sequence: int64(1000 + i)}, - dispatched: make(chan *batchDispatch), - } - } - - // Kick off a go routine to consume the confirmations + // Dispatch the work go func() { - for i := 0; i < len(work); i++ { - <-work[i].dispatched + for i := 0; i < 5; i++ { + msgid := fftypes.NewUUID() + bp.newWork <- &batchWork{ + msg: &fftypes.Message{Header: fftypes.MessageHeader{ID: msgid}, Sequence: int64(1000 + i)}, + } } - wg.Done() }() - // Dispatch the work - for i := 0; i < len(work); i++ { - bp.newWork <- work[i] - } - // Wait for the confirmations, and the dispatch - wg.Wait() + batch := <-dispatched // Check we got all the messages in a single batch - assert.Equal(t, len(dispatched[0].Payload.Messages), len(work)) + assert.Equal(t, 5, len(batch.Payload.Messages)) bp.cancelCtx() <-bp.done @@ -117,17 +101,14 @@ func TestUnfilledBatch(t *testing.T) { func TestBatchSizeOverflow(t *testing.T) { log.SetLevel("debug") + config.Reset() - wg := sync.WaitGroup{} - wg.Add(3) - - dispatched := []*fftypes.Batch{} + dispatched := make(chan *fftypes.Batch) mdi, bp := newTestBatchProcessor(func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { - dispatched = append(dispatched, b) - wg.Done() + dispatched <- b return nil }) - bp.conf.BatchMaxBytes = 1 + bp.conf.BatchMaxBytes = batchSizeEstimateBase + (&fftypes.Message{}).EstimateSize(false) + 100 mockRunAsGroupPassthrough(mdi) mdi.On("UpdateMessages", mock.Anything, mock.Anything, mock.Anything).Return(nil) mdi.On("UpsertBatch", mock.Anything, mock.Anything, mock.Anything).Return(nil) @@ -136,105 +117,25 @@ func TestBatchSizeOverflow(t *testing.T) { mth := bp.txHelper.(*txcommonmocks.Helper) mth.On("SubmitNewTransaction", mock.Anything, "ns1", fftypes.TransactionTypeBatchPin).Return(fftypes.NewUUID(), nil) - // Generate the work - work := make([]*batchWork, 2) - for i := 0; i < 2; i++ { - msgid := fftypes.NewUUID() - work[i] = &batchWork{ - msg: &fftypes.Message{Header: fftypes.MessageHeader{ID: msgid}}, - dispatched: make(chan *batchDispatch), - } - } - - // Kick off a go routine to consume the confirmations + // Dispatch the work go func() { - for i := 0; i < len(work); i++ { - <-work[i].dispatched + for i := 0; i < 2; i++ { + msgid := fftypes.NewUUID() + bp.newWork <- &batchWork{ + msg: &fftypes.Message{Header: fftypes.MessageHeader{ID: msgid}, Sequence: int64(1000 + i)}, + } } - wg.Done() }() - // Dispatch the work - for i := 0; i < len(work); i++ { - bp.newWork <- work[i] - } - // Wait for the confirmations, and the dispatch - wg.Wait() + batch1 := <-dispatched + batch2 := <-dispatched // Check we got all messages across two batches - assert.Equal(t, len(dispatched[0].Payload.Messages), 1) - assert.Equal(t, len(dispatched[1].Payload.Messages), 1) - - bp.cancelCtx() - <-bp.done -} - -func TestFilledBatchSlowPersistence(t *testing.T) { - log.SetLevel("debug") - - wg := sync.WaitGroup{} - wg.Add(2) - - dispatched := []*fftypes.Batch{} - mdi, bp := newTestBatchProcessor(func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { - dispatched = append(dispatched, b) - wg.Done() - return nil - }) - bp.conf.BatchTimeout = 1 * time.Hour // Must fill the batch - mockUpsert := mdi.On("UpsertBatch", mock.Anything, mock.Anything, mock.Anything) - mockUpsert.ReturnArguments = mock.Arguments{nil} - unblockPersistence := make(chan time.Time) - mockUpsert.WaitFor = unblockPersistence - mockRunAsGroupPassthrough(mdi) - mdi.On("UpdateMessages", mock.Anything, mock.Anything, mock.Anything).Return(nil) - mdi.On("UpdateBatch", mock.Anything, mock.Anything, mock.Anything).Return(nil) - - mth := bp.txHelper.(*txcommonmocks.Helper) - mth.On("SubmitNewTransaction", mock.Anything, "ns1", fftypes.TransactionTypeBatchPin).Return(fftypes.NewUUID(), nil) - - // Generate the work - work := make([]*batchWork, 10) - for i := 0; i < 10; i++ { - msgid := fftypes.NewUUID() - if i%2 == 0 { - work[i] = &batchWork{ - msg: &fftypes.Message{Header: fftypes.MessageHeader{ID: msgid}}, - dispatched: make(chan *batchDispatch), - } - } else { - work[i] = &batchWork{ - data: []*fftypes.Data{{ID: msgid}}, - dispatched: make(chan *batchDispatch), - } - } - } - - // Kick off a go routine to consume the confirmations - go func() { - for i := 0; i < 10; i++ { - <-work[i].dispatched - } - wg.Done() - }() - - // Dispatch the work - for i := 0; i < 10; i++ { - bp.newWork <- work[i] - } - - // Unblock the dispatch - time.Sleep(10 * time.Millisecond) - mockUpsert.WaitFor = nil - unblockPersistence <- time.Now() // First call to write the first entry in the batch - - // Wait for comdiletion - wg.Wait() - - // Check we got all the messages in a single batch - assert.Equal(t, len(dispatched[0].Payload.Messages), 5) - assert.Equal(t, len(dispatched[0].Payload.Data), 5) + assert.Equal(t, 1, len(batch1.Payload.Messages)) + assert.Equal(t, int64(1000), batch1.Payload.Messages[0].Sequence) + assert.Equal(t, 1, len(batch2.Payload.Messages)) + assert.Equal(t, int64(1001), batch2.Payload.Messages[0].Sequence) bp.cancelCtx() <-bp.done @@ -251,32 +152,29 @@ func TestCloseToUnblockDispatch(t *testing.T) { func TestCloseToUnblockUpsertBatch(t *testing.T) { - wg := sync.WaitGroup{} - wg.Add(1) - mdi, bp := newTestBatchProcessor(func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { return nil }) bp.retry.MaximumDelay = 1 * time.Microsecond + bp.conf.BatchMaxSize = 1 bp.conf.BatchTimeout = 100 * time.Second mockRunAsGroupPassthrough(mdi) - mdi.On("UpdateMessages", mock.Anything, mock.Anything, mock.Anything).Return(nil) - mup := mdi.On("UpsertBatch", mock.Anything, mock.Anything, mock.Anything).Return(fmt.Errorf("pop")) waitForCall := make(chan bool) - mup.RunFn = func(a mock.Arguments) { - waitForCall <- true - <-waitForCall - } + mth := bp.txHelper.(*txcommonmocks.Helper) + mth.On("SubmitNewTransaction", mock.Anything, "ns1", fftypes.TransactionTypeBatchPin). + Run(func(a mock.Arguments) { + waitForCall <- true + <-waitForCall + }). + Return(nil, fmt.Errorf("pop")) // Generate the work msgid := fftypes.NewUUID() - work := &batchWork{ - msg: &fftypes.Message{Header: fftypes.MessageHeader{ID: msgid}}, - dispatched: make(chan *batchDispatch), - } - - // Dispatch the work - bp.newWork <- work + go func() { + bp.newWork <- &batchWork{ + msg: &fftypes.Message{Header: fftypes.MessageHeader{ID: msgid}, Sequence: int64(1000)}, + } + }() // Ensure the mock has been run <-waitForCall From dc1eae485aa7d631c029e8c21a5a0a5a2a9ddecc Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Thu, 10 Feb 2022 09:03:50 -0500 Subject: [PATCH 06/23] Private message send refactor to use batches always Signed-off-by: Peter Broadhurst --- docs/swagger/swagger.yaml | 13 + internal/batch/batch_manager.go | 24 +- internal/batch/batch_manager_test.go | 12 +- internal/batch/batch_processor.go | 48 +- internal/batch/batch_processor_test.go | 1 + internal/broadcast/manager.go | 12 +- internal/broadcast/manager_test.go | 12 +- internal/i18n/en_translations.go | 1 + internal/privatemessaging/message.go | 47 -- internal/privatemessaging/message_test.go | 535 +++++++++++------- internal/privatemessaging/privatemessaging.go | 111 ++-- .../privatemessaging/privatemessaging_test.go | 75 ++- internal/privatemessaging/recipients.go | 12 +- internal/privatemessaging/recipients_test.go | 6 +- mocks/batchmocks/manager.go | 6 +- pkg/fftypes/manifest_test.go | 17 +- pkg/fftypes/message.go | 6 + pkg/fftypes/message_test.go | 45 +- pkg/fftypes/transport_wrapper.go | 34 +- pkg/fftypes/transport_wrapper_test.go | 38 +- 20 files changed, 611 insertions(+), 444 deletions(-) diff --git a/docs/swagger/swagger.yaml b/docs/swagger/swagger.yaml index 35f99af619..f294e66f34 100644 --- a/docs/swagger/swagger.yaml +++ b/docs/swagger/swagger.yaml @@ -1087,6 +1087,7 @@ paths: enum: - staged - ready + - sent - pending - confirmed - rejected @@ -1239,6 +1240,7 @@ paths: enum: - staged - ready + - sent - pending - confirmed - rejected @@ -3479,6 +3481,7 @@ paths: enum: - staged - ready + - sent - pending - confirmed - rejected @@ -4294,6 +4297,7 @@ paths: enum: - staged - ready + - sent - pending - confirmed - rejected @@ -4402,6 +4406,7 @@ paths: enum: - staged - ready + - sent - pending - confirmed - rejected @@ -4841,6 +4846,7 @@ paths: enum: - staged - ready + - sent - pending - confirmed - rejected @@ -4901,6 +4907,7 @@ paths: enum: - staged - ready + - sent - pending - confirmed - rejected @@ -5048,6 +5055,7 @@ paths: enum: - staged - ready + - sent - pending - confirmed - rejected @@ -5108,6 +5116,7 @@ paths: enum: - staged - ready + - sent - pending - confirmed - rejected @@ -5265,6 +5274,7 @@ paths: enum: - staged - ready + - sent - pending - confirmed - rejected @@ -6498,6 +6508,7 @@ paths: enum: - staged - ready + - sent - pending - confirmed - rejected @@ -6752,6 +6763,7 @@ paths: enum: - staged - ready + - sent - pending - confirmed - rejected @@ -7535,6 +7547,7 @@ paths: enum: - staged - ready + - sent - pending - confirmed - rejected diff --git a/internal/batch/batch_manager.go b/internal/batch/batch_manager.go index edeb23de64..7db740c7fe 100644 --- a/internal/batch/batch_manager.go +++ b/internal/batch/batch_manager.go @@ -48,7 +48,7 @@ func NewBatchManager(ctx context.Context, ni sysmessaging.LocalNodeInfo, di data readPageSize: uint64(readPageSize), messagePollTimeout: config.GetDuration(config.BatchManagerReadPollTimeout), startupOffsetRetryAttempts: config.GetInt(config.OrchestratorStartupAttempts), - dispatchers: make(map[fftypes.MessageType]*dispatcher), + dispatchers: make(map[string]*dispatcher), newMessages: make(chan int64, 1), done: make(chan struct{}), retry: &retry.Retry{ @@ -61,7 +61,7 @@ func NewBatchManager(ctx context.Context, ni sysmessaging.LocalNodeInfo, di data } type Manager interface { - RegisterDispatcher(name string, msgTypes []fftypes.MessageType, handler DispatchHandler, batchOptions DispatcherOptions) + RegisterDispatcher(name string, txType fftypes.TransactionType, msgTypes []fftypes.MessageType, handler DispatchHandler, batchOptions DispatcherOptions) NewMessages() chan<- int64 Start() error Close() @@ -82,7 +82,7 @@ type batchManager struct { database database.Plugin data data.Manager dispatcherMux sync.Mutex - dispatchers map[fftypes.MessageType]*dispatcher + dispatchers map[string]*dispatcher newMessages chan int64 done chan struct{} retry *retry.Retry @@ -108,7 +108,11 @@ type dispatcher struct { options DispatcherOptions } -func (bm *batchManager) RegisterDispatcher(name string, msgTypes []fftypes.MessageType, handler DispatchHandler, options DispatcherOptions) { +func (bm *batchManager) getDispatcherKey(txType fftypes.TransactionType, msgType fftypes.MessageType) string { + return fmt.Sprintf("tx:%s/%s", txType, msgType) +} + +func (bm *batchManager) RegisterDispatcher(name string, txType fftypes.TransactionType, msgTypes []fftypes.MessageType, handler DispatchHandler, options DispatcherOptions) { dispatcher := &dispatcher{ name: name, handler: handler, @@ -116,7 +120,7 @@ func (bm *batchManager) RegisterDispatcher(name string, msgTypes []fftypes.Messa processors: make(map[string]*batchProcessor), } for _, msgType := range msgTypes { - bm.dispatchers[msgType] = dispatcher + bm.dispatchers[bm.getDispatcherKey(txType, msgType)] = dispatcher } } @@ -129,13 +133,14 @@ func (bm *batchManager) NewMessages() chan<- int64 { return bm.newMessages } -func (bm *batchManager) getProcessor(batchType fftypes.MessageType, group *fftypes.Bytes32, namespace string, identity *fftypes.Identity) (*batchProcessor, error) { +func (bm *batchManager) getProcessor(txType fftypes.TransactionType, msgType fftypes.MessageType, group *fftypes.Bytes32, namespace string, identity *fftypes.Identity) (*batchProcessor, error) { bm.dispatcherMux.Lock() defer bm.dispatcherMux.Unlock() - dispatcher, ok := bm.dispatchers[batchType] + dispatcherKey := bm.getDispatcherKey(txType, msgType) + dispatcher, ok := bm.dispatchers[dispatcherKey] if !ok { - return nil, i18n.NewError(bm.ctx, i18n.MsgUnregisteredBatchType, batchType) + return nil, i18n.NewError(bm.ctx, i18n.MsgUnregisteredBatchType, dispatcherKey) } name := fmt.Sprintf("%s|%s|%v", namespace, identity.Author, group) processor, ok := dispatcher.processors[name] @@ -147,6 +152,7 @@ func (bm *batchManager) getProcessor(batchType fftypes.MessageType, group *fftyp &batchProcessorConf{ DispatcherOptions: dispatcher.options, name: name, + txType: txType, dispatcherName: dispatcher.name, namespace: namespace, identity: *identity, @@ -274,7 +280,7 @@ func (bm *batchManager) waitForShoulderTapOrPollTimeout() (done bool) { func (bm *batchManager) dispatchMessage(msg *fftypes.Message, data ...*fftypes.Data) error { l := log.L(bm.ctx) - processor, err := bm.getProcessor(msg.Header.Type, msg.Header.Group, msg.Header.Namespace, &msg.Header.Identity) + processor, err := bm.getProcessor(msg.Header.TxType, msg.Header.Type, msg.Header.Group, msg.Header.Namespace, &msg.Header.Identity) if err != nil { return err } diff --git a/internal/batch/batch_manager_test.go b/internal/batch/batch_manager_test.go index aa217b97dc..345a404327 100644 --- a/internal/batch/batch_manager_test.go +++ b/internal/batch/batch_manager_test.go @@ -72,7 +72,7 @@ func TestE2EDispatchBroadcast(t *testing.T) { bmi, _ := NewBatchManager(ctx, mni, mdi, mdm) bm := bmi.(*batchManager) - bm.RegisterDispatcher("utdispatcher", []fftypes.MessageType{fftypes.MessageTypeBroadcast}, handler, DispatcherOptions{ + bm.RegisterDispatcher("utdispatcher", fftypes.TransactionTypeBatchPin, []fftypes.MessageType{fftypes.MessageTypeBroadcast}, handler, DispatcherOptions{ BatchMaxSize: 2, BatchTimeout: 0, DisposeTimeout: 120 * time.Second, @@ -82,6 +82,7 @@ func TestE2EDispatchBroadcast(t *testing.T) { dataHash := fftypes.NewRandB32() msg := &fftypes.Message{ Header: fftypes.MessageHeader{ + TxType: fftypes.TransactionTypeBatchPin, Type: fftypes.MessageTypeBroadcast, ID: fftypes.NewUUID(), Topics: []string{"topic1", "topic2"}, @@ -173,7 +174,7 @@ func TestE2EDispatchPrivate(t *testing.T) { bmi, _ := NewBatchManager(ctx, mni, mdi, mdm) bm := bmi.(*batchManager) - bm.RegisterDispatcher("utdispatcher", []fftypes.MessageType{fftypes.MessageTypePrivate}, handler, DispatcherOptions{ + bm.RegisterDispatcher("utdispatcher", fftypes.TransactionTypeBatchPin, []fftypes.MessageType{fftypes.MessageTypePrivate}, handler, DispatcherOptions{ BatchMaxSize: 2, BatchTimeout: 0, DisposeTimeout: 120 * time.Second, @@ -183,6 +184,7 @@ func TestE2EDispatchPrivate(t *testing.T) { dataHash := fftypes.NewRandB32() msg := &fftypes.Message{ Header: fftypes.MessageHeader{ + TxType: fftypes.TransactionTypeBatchPin, Type: fftypes.MessageTypePrivate, ID: fftypes.NewUUID(), Topics: []string{"topic1", "topic2"}, @@ -344,7 +346,7 @@ func TestMessageSequencerUpdateMessagesFail(t *testing.T) { mni.On("GetNodeUUID", mock.Anything).Return(fftypes.NewUUID()) ctx, cancelCtx := context.WithCancel(context.Background()) bm, _ := NewBatchManager(ctx, mni, mdi, mdm) - bm.RegisterDispatcher("utdispatcher", []fftypes.MessageType{fftypes.MessageTypeBroadcast}, func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { + bm.RegisterDispatcher("utdispatcher", fftypes.TransactionTypeBatchPin, []fftypes.MessageType{fftypes.MessageTypeBroadcast}, func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { return nil }, DispatcherOptions{BatchMaxSize: 1, DisposeTimeout: 0}) @@ -353,6 +355,7 @@ func TestMessageSequencerUpdateMessagesFail(t *testing.T) { { Header: fftypes.MessageHeader{ ID: fftypes.NewUUID(), + TxType: fftypes.TransactionTypeBatchPin, Type: fftypes.MessageTypeBroadcast, Namespace: "ns1", }, @@ -393,7 +396,7 @@ func TestMessageSequencerUpdateBatchFail(t *testing.T) { mni.On("GetNodeUUID", mock.Anything).Return(fftypes.NewUUID()) ctx, cancelCtx := context.WithCancel(context.Background()) bm, _ := NewBatchManager(ctx, mni, mdi, mdm) - bm.RegisterDispatcher("utdispatcher", []fftypes.MessageType{fftypes.MessageTypeBroadcast}, func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { + bm.RegisterDispatcher("utdispatcher", fftypes.TransactionTypeBatchPin, []fftypes.MessageType{fftypes.MessageTypeBroadcast}, func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { return nil }, DispatcherOptions{BatchMaxSize: 1, DisposeTimeout: 0}) @@ -402,6 +405,7 @@ func TestMessageSequencerUpdateBatchFail(t *testing.T) { { Header: fftypes.MessageHeader{ ID: fftypes.NewUUID(), + TxType: fftypes.TransactionTypeBatchPin, Type: fftypes.MessageTypeBroadcast, Namespace: "ns1", }, diff --git a/internal/batch/batch_processor.go b/internal/batch/batch_processor.go index bb7e8ad431..0d54fa344f 100644 --- a/internal/batch/batch_processor.go +++ b/internal/batch/batch_processor.go @@ -42,6 +42,7 @@ type batchProcessorConf struct { DispatcherOptions name string dispatcherName string + txType fftypes.TransactionType namespace string identity fftypes.Identity group *fftypes.Bytes32 @@ -441,13 +442,16 @@ func (bp *batchProcessor) maskContexts(ctx context.Context, batch *fftypes.Batch func (bp *batchProcessor) persistBatch(batch *fftypes.Batch) (contexts []*fftypes.Bytes32, err error) { err = bp.retry.Do(bp.ctx, "batch persist", func(attempt int) (retry bool, err error) { return true, bp.database.RunAsGroup(bp.ctx, func(ctx context.Context) (err error) { - // Generate a new Transaction, which will be used to record status of the associated transaction as it happens - if contexts, err = bp.maskContexts(ctx, batch); err != nil { - return err + + if bp.conf.txType == fftypes.TransactionTypeBatchPin { + // Generate a new Transaction, which will be used to record status of the associated transaction as it happens + if contexts, err = bp.maskContexts(ctx, batch); err != nil { + return err + } } - batch.Payload.TX.Type = fftypes.TransactionTypeBatchPin - if batch.Payload.TX.ID, err = bp.txHelper.SubmitNewTransaction(ctx, batch.Namespace, fftypes.TransactionTypeBatchPin); err != nil { + batch.Payload.TX.Type = bp.conf.txType + if batch.Payload.TX.ID, err = bp.txHelper.SubmitNewTransaction(ctx, batch.Namespace, bp.conf.txType); err != nil { return err } @@ -479,10 +483,36 @@ func (bp *batchProcessor) markMessagesDispatched(batch *fftypes.Batch) error { fb.In("id", msgIDs), fb.Eq("state", fftypes.MessageStateReady), // In the outside chance the next state transition happens first (which supersedes this) ) - update := database.MessageQueryFactory.NewUpdate(ctx). - Set("batch", batch.ID). // Mark the batch they are in - Set("state", fftypes.MessageStateSent) // Set them sent, so they won't be picked up and re-sent after restart/rewind - return bp.database.UpdateMessages(ctx, filter, update) + + var update database.Update + if bp.conf.txType == fftypes.TransactionTypeBatchPin { + // Sent state waiting for confirm + update = database.MessageQueryFactory.NewUpdate(ctx). + Set("batch", batch.ID). // Mark the batch they are in + Set("state", fftypes.MessageStateSent) // Set them sent, so they won't be picked up and re-sent after restart/rewind + } else { + // Immediate confirmation if no transaction + update = database.MessageQueryFactory.NewUpdate(ctx). + Set("batch", batch.ID). + Set("state", fftypes.MessageStateConfirmed). + Set("confirmed", fftypes.Now()) + } + + if err = bp.database.UpdateMessages(ctx, filter, update); err != nil { + return err + } + + if bp.conf.txType == fftypes.TransactionTypeNone { + for _, msg := range batch.Payload.Messages { + // Emit a confirmation event locally immediately + event := fftypes.NewEvent(fftypes.EventTypeMessageConfirmed, batch.Namespace, msg.Header.ID) + if err := bp.database.InsertEvent(ctx, event); err != nil { + return err + } + } + } + + return nil }) }) } diff --git a/internal/batch/batch_processor_test.go b/internal/batch/batch_processor_test.go index d9685368fd..cec858aa96 100644 --- a/internal/batch/batch_processor_test.go +++ b/internal/batch/batch_processor_test.go @@ -37,6 +37,7 @@ func newTestBatchProcessor(dispatch DispatchHandler) (*databasemocks.Plugin, *ba mni.On("GetNodeUUID", mock.Anything).Return(fftypes.NewUUID()).Maybe() bp := newBatchProcessor(context.Background(), mni, mdi, &batchProcessorConf{ namespace: "ns1", + txType: fftypes.TransactionTypeBatchPin, identity: fftypes.Identity{Author: "did:firefly:org/abcd", Key: "0x12345"}, dispatch: dispatch, DispatcherOptions: DispatcherOptions{ diff --git a/internal/broadcast/manager.go b/internal/broadcast/manager.go index aeec8d1b91..a50d9ec36a 100644 --- a/internal/broadcast/manager.go +++ b/internal/broadcast/manager.go @@ -89,11 +89,13 @@ func NewBroadcastManager(ctx context.Context, di database.Plugin, im identity.Ma BatchTimeout: config.GetDuration(config.BroadcastBatchTimeout), DisposeTimeout: config.GetDuration(config.BroadcastBatchAgentTimeout), } - ba.RegisterDispatcher(broadcastDispatcherName, []fftypes.MessageType{ - fftypes.MessageTypeBroadcast, - fftypes.MessageTypeDefinition, - fftypes.MessageTypeTransferBroadcast, - }, bm.dispatchBatch, bo) + ba.RegisterDispatcher(broadcastDispatcherName, + fftypes.TransactionTypeBatchPin, + []fftypes.MessageType{ + fftypes.MessageTypeBroadcast, + fftypes.MessageTypeDefinition, + fftypes.MessageTypeTransferBroadcast, + }, bm.dispatchBatch, bo) return bm, nil } diff --git a/internal/broadcast/manager_test.go b/internal/broadcast/manager_test.go index ad393f3026..40e02a28c0 100644 --- a/internal/broadcast/manager_test.go +++ b/internal/broadcast/manager_test.go @@ -53,11 +53,13 @@ func newTestBroadcast(t *testing.T) (*broadcastManager, func()) { mbp := &batchpinmocks.Submitter{} mbi.On("Name").Return("ut_blockchain").Maybe() mpi.On("Name").Return("ut_publicstorage").Maybe() - mba.On("RegisterDispatcher", []fftypes.MessageType{ - fftypes.MessageTypeBroadcast, - fftypes.MessageTypeDefinition, - fftypes.MessageTypeTransferBroadcast, - }, mock.Anything, mock.Anything).Return() + mba.On("RegisterDispatcher", + fftypes.TransactionTypeBatchPin, + []fftypes.MessageType{ + fftypes.MessageTypeBroadcast, + fftypes.MessageTypeDefinition, + fftypes.MessageTypeTransferBroadcast, + }, mock.Anything, mock.Anything).Return() rag := mdi.On("RunAsGroup", mock.Anything, mock.Anything).Maybe() rag.RunFn = func(a mock.Arguments) { diff --git a/internal/i18n/en_translations.go b/internal/i18n/en_translations.go index b36cb1e9d7..be8f5e85a6 100644 --- a/internal/i18n/en_translations.go +++ b/internal/i18n/en_translations.go @@ -259,4 +259,5 @@ var ( MsgAddressResolveFailed = ffm("FF10339", "Failed to resolve signing key string '%s': %s", 500) MsgAddressResolveBadStatus = ffm("FF10340", "Failed to resolve signing key string '%s' [%d]: %s", 500) MsgAddressResolveBadResData = ffm("FF10341", "Failed to resolve signing key string '%s' - invalid address returned '%s': %s", 500) + MsgInvalidTXTypeForMessage = ffm("FF10343", "Invalid transaction type for sending a message: %s", 400) ) diff --git a/internal/privatemessaging/message.go b/internal/privatemessaging/message.go index 2cb7bef6a4..0c56ff231e 100644 --- a/internal/privatemessaging/message.go +++ b/internal/privatemessaging/message.go @@ -75,8 +75,6 @@ const ( methodSend // methodSendAndWait requests that the message be sent and waits until it is pinned and confirmed by the blockchain methodSendAndWait - // methodSendImmediate requests that the message be sent immediately, with no blockchain pinning - methodSendImmediate ) func (s *messageSender) Prepare(ctx context.Context) error { @@ -84,16 +82,10 @@ func (s *messageSender) Prepare(ctx context.Context) error { } func (s *messageSender) Send(ctx context.Context) error { - if s.msg.Header.TxType == fftypes.TransactionTypeNone { - return s.resolveAndSend(ctx, methodSendImmediate) - } return s.resolveAndSend(ctx, methodSend) } func (s *messageSender) SendAndWait(ctx context.Context) error { - if s.msg.Header.TxType == fftypes.TransactionTypeNone { - return s.resolveAndSend(ctx, methodSendImmediate) - } return s.resolveAndSend(ctx, methodSendAndWait) } @@ -176,50 +168,11 @@ func (s *messageSender) sendInternal(ctx context.Context, method sendMethod) err return nil } - if method == methodSendImmediate { - s.msg.Confirmed = fftypes.Now() - // msg.Header.Key = "" // there is no on-chain signing assurance with this message - } - // Store the message - this asynchronously triggers the next step in process if err := s.mgr.database.UpsertMessage(ctx, &s.msg.Message, database.UpsertOptimizationNew); err != nil { return err } log.L(ctx).Infof("Sent private message %s:%s sequence=%d", s.msg.Header.Namespace, s.msg.Header.ID, s.msg.Sequence) - if method == methodSendImmediate { - if err := s.sendUnpinned(ctx); err != nil { - return err - } - - // Emit a confirmation event locally immediately - event := fftypes.NewEvent(fftypes.EventTypeMessageConfirmed, s.namespace, s.msg.Header.ID) - if err := s.mgr.database.InsertEvent(ctx, event); err != nil { - return err - } - } - return nil } - -func (s *messageSender) sendUnpinned(ctx context.Context) (err error) { - // Retrieve the group - group, nodes, err := s.mgr.groupManager.getGroupNodes(ctx, s.msg.Header.Group) - if err != nil { - return err - } - - data, _, err := s.mgr.data.GetMessageData(ctx, &s.msg.Message, true) - if err != nil { - return err - } - - tw := &fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeMessage, - Message: &s.msg.Message, - Data: data, - Group: group, - } - - return s.mgr.sendData(ctx, "message", s.msg.Header.ID, s.msg.Header.Group, s.namespace, nodes, tw, nil, data) -} diff --git a/internal/privatemessaging/message_test.go b/internal/privatemessaging/message_test.go index 0b2214861f..a0dfb8f8be 100644 --- a/internal/privatemessaging/message_test.go +++ b/internal/privatemessaging/message_test.go @@ -100,7 +100,6 @@ func TestSendUnpinnedMessageE2EOk(t *testing.T) { defer cancel() mim := pm.identity.(*identitymanagermocks.Manager) - mim.On("GetLocalOrgKey", pm.ctx).Return("localorgkey", nil) mim.On("ResolveInputIdentity", pm.ctx, mock.Anything).Run(func(args mock.Arguments) { identity := args[1].(*fftypes.Identity) identity.Author = "localorg" @@ -109,37 +108,14 @@ func TestSendUnpinnedMessageE2EOk(t *testing.T) { dataID := fftypes.NewUUID() groupID := fftypes.NewRandB32() - nodeID1 := fftypes.NewUUID() - nodeID2 := fftypes.NewUUID() mdm := pm.data.(*datamocks.Manager) mdm.On("ResolveInlineDataPrivate", pm.ctx, "ns1", mock.Anything).Return(fftypes.DataRefs{ {ID: dataID, Hash: fftypes.NewRandB32()}, }, nil) - mdm.On("GetMessageData", pm.ctx, mock.Anything, true).Return([]*fftypes.Data{ - {ID: dataID, Value: fftypes.JSONAnyPtr(`{"some": "data"}`)}, - }, true, nil).Once() mdi := pm.database.(*databasemocks.Plugin) - mdi.On("GetGroupByHash", pm.ctx, groupID).Return(&fftypes.Group{ - Hash: groupID, - GroupIdentity: fftypes.GroupIdentity{ - Members: fftypes.Members{ - {Node: nodeID1, Identity: "localorgkey"}, - {Node: nodeID2, Identity: "remoteorg"}, - }, - }, - }, nil).Once() - mdi.On("GetNodeByID", pm.ctx, nodeID1).Return(&fftypes.Node{ - ID: nodeID1, Name: "node1", Owner: "localorgkey", DX: fftypes.DXInfo{Peer: "peer1-local"}, - }, nil).Once() - mdi.On("GetNodeByID", pm.ctx, nodeID2).Return(&fftypes.Node{ - ID: nodeID2, Name: "node2", Owner: "org1", DX: fftypes.DXInfo{Peer: "peer2-remote"}, - }, nil).Once() mdi.On("UpsertMessage", pm.ctx, mock.Anything, database.UpsertOptimizationNew).Return(nil).Once() - mdi.On("InsertEvent", pm.ctx, mock.Anything).Return(nil).Once() - - mdx := pm.exchange.(*dataexchangemocks.Plugin) - mdx.On("SendMessage", pm.ctx, "peer2-remote", mock.Anything).Return("tracking1", nil).Once() + mdi.On("GetGroupByHash", pm.ctx, groupID).Return(&fftypes.Group{Hash: groupID}, nil) msg, err := pm.SendMessage(pm.ctx, "ns1", &fftypes.MessageInOut{ Message: fftypes.Message{ @@ -332,6 +308,9 @@ func TestSendUnpinnedMessageTooLarge(t *testing.T) { {ID: dataID, Hash: fftypes.NewRandB32(), ValueSize: 100001}, }, nil) + mdi := pm.database.(*databasemocks.Plugin) + mdi.On("GetGroupByHash", pm.ctx, groupID).Return(&fftypes.Group{Hash: groupID}, nil) + _, err := pm.SendMessage(pm.ctx, "ns1", &fftypes.MessageInOut{ Message: fftypes.Message{ Header: fftypes.MessageHeader{ @@ -428,116 +407,6 @@ func TestMessagePrepare(t *testing.T) { } -func TestSendUnpinnedMessageMarshalFail(t *testing.T) { - - pm, cancel := newTestPrivateMessaging(t) - defer cancel() - - mim := pm.identity.(*identitymanagermocks.Manager) - mim.On("ResolveInputIdentity", pm.ctx, mock.MatchedBy(func(identity *fftypes.Identity) bool { - assert.Equal(t, "localorg", identity.Author) - return true - })).Return(nil) - - groupID := fftypes.NewRandB32() - nodeID1 := fftypes.NewUUID() - nodeID2 := fftypes.NewUUID() - mdm := pm.data.(*datamocks.Manager) - mdm.On("GetMessageData", pm.ctx, mock.Anything, true).Return([]*fftypes.Data{ - {ID: fftypes.NewUUID(), Value: fftypes.JSONAnyPtr(`!Invalid JSON`)}, - }, true, nil).Once() - - mdi := pm.database.(*databasemocks.Plugin) - mdi.On("GetGroupByHash", pm.ctx, groupID).Return(&fftypes.Group{ - Hash: groupID, - GroupIdentity: fftypes.GroupIdentity{ - Members: fftypes.Members{ - {Node: nodeID1, Identity: "localorg"}, - {Node: nodeID2, Identity: "remoteorg"}, - }, - }, - }, nil).Once() - mdi.On("GetNodeByID", pm.ctx, nodeID1).Return(&fftypes.Node{ - ID: nodeID1, Name: "node1", Owner: "localorg", DX: fftypes.DXInfo{Peer: "peer1-local"}, - }, nil).Once() - mdi.On("GetNodeByID", pm.ctx, nodeID2).Return(&fftypes.Node{ - ID: nodeID2, Name: "node2", Owner: "org1", DX: fftypes.DXInfo{Peer: "peer2-remote"}, - }, nil).Once() - - message := &messageSender{ - mgr: pm, - msg: &fftypes.MessageInOut{ - Message: fftypes.Message{ - Header: fftypes.MessageHeader{ - Identity: fftypes.Identity{ - Author: "localorg", - }, - TxType: fftypes.TransactionTypeNone, - Group: groupID, - }, - }, - }, - } - - err := message.sendUnpinned(pm.ctx) - assert.Regexp(t, "FF10137", err) - - mdm.AssertExpectations(t) - mdi.AssertExpectations(t) - -} - -func TestSendUnpinnedMessageGetDataFail(t *testing.T) { - - pm, cancel := newTestPrivateMessaging(t) - defer cancel() - - groupID := fftypes.NewRandB32() - nodeID1 := fftypes.NewUUID() - nodeID2 := fftypes.NewUUID() - mdm := pm.data.(*datamocks.Manager) - mdm.On("GetMessageData", pm.ctx, mock.Anything, true).Return(nil, false, fmt.Errorf("pop")).Once() - - mdi := pm.database.(*databasemocks.Plugin) - mdi.On("GetGroupByHash", pm.ctx, groupID).Return(&fftypes.Group{ - Hash: groupID, - GroupIdentity: fftypes.GroupIdentity{ - Members: fftypes.Members{ - {Node: nodeID1, Identity: "localorg"}, - {Node: nodeID2, Identity: "remoteorg"}, - }, - }, - }, nil).Once() - mdi.On("GetNodeByID", pm.ctx, nodeID1).Return(&fftypes.Node{ - ID: nodeID1, Name: "node1", Owner: "localorg", DX: fftypes.DXInfo{Peer: "peer1-local"}, - }, nil).Once() - mdi.On("GetNodeByID", pm.ctx, nodeID2).Return(&fftypes.Node{ - ID: nodeID2, Name: "node2", Owner: "org1", DX: fftypes.DXInfo{Peer: "peer2-remote"}, - }, nil).Once() - - message := &messageSender{ - mgr: pm, - msg: &fftypes.MessageInOut{ - Message: fftypes.Message{ - Header: fftypes.MessageHeader{ - Identity: fftypes.Identity{ - Author: "localorg", - }, - TxType: fftypes.TransactionTypeNone, - Group: groupID, - }, - }, - }, - } - - err := message.sendUnpinned(pm.ctx) - assert.Regexp(t, "pop", err) - - mdm.AssertExpectations(t) - mdi.AssertExpectations(t) - -} - func TestSendUnpinnedMessageGroupLookupFail(t *testing.T) { pm, cancel := newTestPrivateMessaging(t) @@ -547,22 +416,23 @@ func TestSendUnpinnedMessageGroupLookupFail(t *testing.T) { mdi := pm.database.(*databasemocks.Plugin) mdi.On("GetGroupByHash", pm.ctx, groupID).Return(nil, fmt.Errorf("pop")).Once() - message := &messageSender{ - mgr: pm, - msg: &fftypes.MessageInOut{ - Message: fftypes.Message{ - Header: fftypes.MessageHeader{ - Identity: fftypes.Identity{ - Author: "org1", + err := pm.dispatchUnpinnedBatch(pm.ctx, &fftypes.Batch{ + ID: fftypes.NewUUID(), + Group: groupID, + Payload: fftypes.BatchPayload{ + Messages: []*fftypes.Message{ + { + Header: fftypes.MessageHeader{ + Identity: fftypes.Identity{ + Author: "org1", + }, + TxType: fftypes.TransactionTypeNone, + Group: groupID, }, - TxType: fftypes.TransactionTypeNone, - Group: groupID, }, }, }, - } - - err := message.sendUnpinned(pm.ctx) + }, []*fftypes.Bytes32{}) assert.Regexp(t, "pop", err) mdi.AssertExpectations(t) @@ -589,6 +459,7 @@ func TestSendUnpinnedMessageInsertFail(t *testing.T) { mdi := pm.database.(*databasemocks.Plugin) mdi.On("UpsertMessage", pm.ctx, mock.Anything, database.UpsertOptimizationNew).Return(fmt.Errorf("pop")).Once() + mdi.On("GetGroupByHash", pm.ctx, groupID).Return(&fftypes.Group{Hash: groupID}, nil) _, err := pm.SendMessage(pm.ctx, "ns1", &fftypes.MessageInOut{ Message: fftypes.Message{ @@ -649,19 +520,10 @@ func TestSendUnpinnedMessageResolveGroupFail(t *testing.T) { mim := pm.identity.(*identitymanagermocks.Manager) mim.On("ResolveInputIdentity", pm.ctx, mock.Anything).Return(nil) - dataID := fftypes.NewUUID() groupID := fftypes.NewRandB32() - mdm := pm.data.(*datamocks.Manager) - mdm.On("ResolveInlineDataPrivate", pm.ctx, "ns1", mock.Anything).Return(fftypes.DataRefs{ - {ID: dataID, Hash: fftypes.NewRandB32()}, - }, nil) mdi := pm.database.(*databasemocks.Plugin) - mdi.On("GetGroupByHash", pm.ctx, groupID).Return(nil, fmt.Errorf("pop")).Once() - mdi.On("UpsertMessage", pm.ctx, mock.Anything, database.UpsertOptimizationNew).Return(nil).Once() - - mdx := pm.exchange.(*dataexchangemocks.Plugin) - mdx.On("SendMessage", pm.ctx, "peer2-remote", mock.Anything).Return("tracking1", nil).Once() + mdi.On("GetGroupByHash", pm.ctx, groupID).Return(nil, fmt.Errorf("pop")) _, err := pm.SendMessage(pm.ctx, "ns1", &fftypes.MessageInOut{ Message: fftypes.Message{ @@ -681,54 +543,23 @@ func TestSendUnpinnedMessageResolveGroupFail(t *testing.T) { }, false) assert.EqualError(t, err, "pop") - mdm.AssertExpectations(t) mdi.AssertExpectations(t) mim.AssertExpectations(t) } -func TestSendUnpinnedMessageEventFail(t *testing.T) { +func TestSendUnpinnedMessageResolveGroupNotFound(t *testing.T) { pm, cancel := newTestPrivateMessaging(t) defer cancel() mim := pm.identity.(*identitymanagermocks.Manager) mim.On("ResolveInputIdentity", pm.ctx, mock.Anything).Return(nil) - mim.On("GetLocalOrgKey", pm.ctx).Return("localorgkey", nil) - dataID := fftypes.NewUUID() groupID := fftypes.NewRandB32() - nodeID1 := fftypes.NewUUID() - nodeID2 := fftypes.NewUUID() - mdm := pm.data.(*datamocks.Manager) - mdm.On("ResolveInlineDataPrivate", pm.ctx, "ns1", mock.Anything).Return(fftypes.DataRefs{ - {ID: dataID, Hash: fftypes.NewRandB32()}, - }, nil) - mdm.On("GetMessageData", pm.ctx, mock.Anything, true).Return([]*fftypes.Data{ - {ID: dataID, Value: fftypes.JSONAnyPtr(`{"some": "data"}`)}, - }, true, nil).Once() mdi := pm.database.(*databasemocks.Plugin) - mdi.On("GetGroupByHash", pm.ctx, groupID).Return(&fftypes.Group{ - Hash: groupID, - GroupIdentity: fftypes.GroupIdentity{ - Members: fftypes.Members{ - {Node: nodeID1, Identity: "localorgkey"}, - {Node: nodeID2, Identity: "remoteorg"}, - }, - }, - }, nil).Once() - mdi.On("GetNodeByID", pm.ctx, nodeID1).Return(&fftypes.Node{ - ID: nodeID1, Name: "node1", Owner: "localorgkey", DX: fftypes.DXInfo{Peer: "peer1-local"}, - }, nil).Once() - mdi.On("GetNodeByID", pm.ctx, nodeID2).Return(&fftypes.Node{ - ID: nodeID2, Name: "node2", Owner: "org1", DX: fftypes.DXInfo{Peer: "peer2-remote"}, - }, nil).Once() - mdi.On("UpsertMessage", pm.ctx, mock.Anything, database.UpsertOptimizationNew).Return(nil).Once() - mdi.On("InsertEvent", pm.ctx, mock.Anything).Return(fmt.Errorf("pop")).Once() - - mdx := pm.exchange.(*dataexchangemocks.Plugin) - mdx.On("SendMessage", pm.ctx, "peer2-remote", mock.Anything).Return("tracking1", nil).Once() + mdi.On("GetGroupByHash", pm.ctx, groupID).Return(nil, nil) _, err := pm.SendMessage(pm.ctx, "ns1", &fftypes.MessageInOut{ Message: fftypes.Message{ @@ -746,9 +577,8 @@ func TestSendUnpinnedMessageEventFail(t *testing.T) { }, }, }, false) - assert.EqualError(t, err, "pop") + assert.Regexp(t, "FF10226", err) - mdm.AssertExpectations(t) mdi.AssertExpectations(t) mim.AssertExpectations(t) @@ -804,14 +634,17 @@ func TestRequestReplySuccess(t *testing.T) { {ID: fftypes.NewUUID(), Hash: fftypes.NewRandB32()}, }, nil) + groupID := fftypes.NewRandB32() + mdi := pm.database.(*databasemocks.Plugin) mdi.On("UpsertMessage", pm.ctx, mock.Anything, database.UpsertOptimizationNew).Return(nil).Once() + mdi.On("GetGroupByHash", pm.ctx, groupID).Return(&fftypes.Group{Hash: groupID}, nil) _, err := pm.RequestReply(pm.ctx, "ns1", &fftypes.MessageInOut{ Message: fftypes.Message{ Header: fftypes.MessageHeader{ Tag: "mytag", - Group: fftypes.NewRandB32(), + Group: groupID, Identity: fftypes.Identity{ Author: "org1", }, @@ -820,3 +653,319 @@ func TestRequestReplySuccess(t *testing.T) { }) assert.NoError(t, err) } + +func TestDispatchedUnpinnedMessageMarshalFail(t *testing.T) { + + pm, cancel := newTestPrivateMessaging(t) + defer cancel() + + mim := pm.identity.(*identitymanagermocks.Manager) + mim.On("ResolveInputIdentity", pm.ctx, mock.MatchedBy(func(identity *fftypes.Identity) bool { + assert.Equal(t, "localorg", identity.Author) + return true + })).Return(nil) + + groupID := fftypes.NewRandB32() + nodeID1 := fftypes.NewUUID() + nodeID2 := fftypes.NewUUID() + + mdi := pm.database.(*databasemocks.Plugin) + mdi.On("GetGroupByHash", pm.ctx, groupID).Return(&fftypes.Group{ + Hash: groupID, + GroupIdentity: fftypes.GroupIdentity{ + Members: fftypes.Members{ + {Node: nodeID1, Identity: "localorg"}, + {Node: nodeID2, Identity: "remoteorg"}, + }, + }, + }, nil).Once() + mdi.On("GetNodeByID", pm.ctx, nodeID1).Return(&fftypes.Node{ + ID: nodeID1, Name: "node1", Owner: "localorg", DX: fftypes.DXInfo{Peer: "peer1-local"}, + }, nil).Once() + mdi.On("GetNodeByID", pm.ctx, nodeID2).Return(&fftypes.Node{ + ID: nodeID2, Name: "node2", Owner: "org1", DX: fftypes.DXInfo{Peer: "peer2-remote"}, + }, nil).Once() + + err := pm.dispatchUnpinnedBatch(pm.ctx, &fftypes.Batch{ + ID: fftypes.NewUUID(), + Group: groupID, + Payload: fftypes.BatchPayload{ + Data: []*fftypes.Data{ + {Value: fftypes.JSONAnyPtr("!Bad JSON")}, + }, + }, + }, []*fftypes.Bytes32{}) + assert.Regexp(t, "FF10137", err) + + mdi.AssertExpectations(t) + +} + +func TestDispatchedUnpinnedMessageOK(t *testing.T) { + + pm, cancel := newTestPrivateMessaging(t) + defer cancel() + + mim := pm.identity.(*identitymanagermocks.Manager) + mim.On("ResolveInputIdentity", pm.ctx, mock.MatchedBy(func(identity *fftypes.Identity) bool { + assert.Equal(t, "localorg", identity.Author) + return true + })).Return(nil) + mim.On("GetLocalOrgKey", pm.ctx).Return("localorg", nil) + + mdx := pm.exchange.(*dataexchangemocks.Plugin) + mdx.On("SendMessage", pm.ctx, "peer2-remote", mock.Anything).Return("", nil) + + groupID := fftypes.NewRandB32() + nodeID1 := fftypes.NewUUID() + nodeID2 := fftypes.NewUUID() + + mdi := pm.database.(*databasemocks.Plugin) + mdi.On("GetGroupByHash", pm.ctx, groupID).Return(&fftypes.Group{ + Hash: groupID, + GroupIdentity: fftypes.GroupIdentity{ + Members: fftypes.Members{ + {Node: nodeID1, Identity: "localorg"}, + {Node: nodeID2, Identity: "remoteorg"}, + }, + }, + }, nil).Once() + mdi.On("GetNodeByID", pm.ctx, nodeID1).Return(&fftypes.Node{ + ID: nodeID1, Name: "node1", Owner: "localorg", DX: fftypes.DXInfo{Peer: "peer1-local"}, + }, nil).Once() + mdi.On("GetNodeByID", pm.ctx, nodeID2).Return(&fftypes.Node{ + ID: nodeID2, Name: "node2", Owner: "org1", DX: fftypes.DXInfo{Peer: "peer2-remote"}, + }, nil).Once() + mdi.On("InsertOperation", pm.ctx, mock.Anything).Return(nil) + + err := pm.dispatchUnpinnedBatch(pm.ctx, &fftypes.Batch{ + ID: fftypes.NewUUID(), + Group: groupID, + Payload: fftypes.BatchPayload{ + TX: fftypes.TransactionRef{ + ID: fftypes.NewUUID(), + Type: fftypes.TransactionTypeNone, + }, + Messages: []*fftypes.Message{ + { + Header: fftypes.MessageHeader{ + Tag: "mytag", + Group: groupID, + Identity: fftypes.Identity{ + Author: "org1", + }, + }, + }, + }, + }, + }, []*fftypes.Bytes32{}) + assert.NoError(t, err) + + mdi.AssertExpectations(t) + +} + +func TestSendDataTransferBlobsFail(t *testing.T) { + + pm, cancel := newTestPrivateMessaging(t) + defer cancel() + + mim := pm.identity.(*identitymanagermocks.Manager) + mim.On("ResolveInputIdentity", pm.ctx, mock.MatchedBy(func(identity *fftypes.Identity) bool { + assert.Equal(t, "localorg", identity.Author) + return true + })).Return(nil) + mim.On("GetLocalOrgKey", pm.ctx).Return("localorg", nil) + + groupID := fftypes.NewRandB32() + nodeID2 := fftypes.NewUUID() + + mdi := pm.database.(*databasemocks.Plugin) + mdi.On("GetBlobMatchingHash", pm.ctx, mock.Anything).Return(nil, fmt.Errorf("pop")) + + nodes := []*fftypes.Node{{ + ID: nodeID2, Name: "node2", Owner: "org1", DX: fftypes.DXInfo{Peer: "peer2-remote"}, + }} + + err := pm.sendData(pm.ctx, &fftypes.TransportWrapper{ + Batch: &fftypes.Batch{ + ID: fftypes.NewUUID(), + Group: groupID, + Payload: fftypes.BatchPayload{ + Messages: []*fftypes.Message{ + { + Header: fftypes.MessageHeader{ + Tag: "mytag", + Group: groupID, + Identity: fftypes.Identity{ + Author: "org1", + }, + }, + }, + }, + Data: []*fftypes.Data{ + {ID: fftypes.NewUUID(), Value: fftypes.JSONAnyPtr("{}"), Blob: &fftypes.BlobRef{ + Hash: fftypes.NewRandB32(), + }}, + }, + }, + }, + }, nodes) + assert.Regexp(t, "pop", err) + + mdi.AssertExpectations(t) + +} + +func TestSendDataTransferFail(t *testing.T) { + + pm, cancel := newTestPrivateMessaging(t) + defer cancel() + + mim := pm.identity.(*identitymanagermocks.Manager) + mim.On("ResolveInputIdentity", pm.ctx, mock.MatchedBy(func(identity *fftypes.Identity) bool { + assert.Equal(t, "localorg", identity.Author) + return true + })).Return(nil) + mim.On("GetLocalOrgKey", pm.ctx).Return("localorg", nil) + + mdx := pm.exchange.(*dataexchangemocks.Plugin) + mdx.On("SendMessage", pm.ctx, "peer2-remote", mock.Anything).Return("", fmt.Errorf("pop")) + + groupID := fftypes.NewRandB32() + nodeID2 := fftypes.NewUUID() + + nodes := []*fftypes.Node{{ + ID: nodeID2, Name: "node2", Owner: "org1", DX: fftypes.DXInfo{Peer: "peer2-remote"}, + }} + + err := pm.sendData(pm.ctx, &fftypes.TransportWrapper{ + Batch: &fftypes.Batch{ + ID: fftypes.NewUUID(), + Group: groupID, + Payload: fftypes.BatchPayload{ + Messages: []*fftypes.Message{ + { + Header: fftypes.MessageHeader{ + Tag: "mytag", + Group: groupID, + Identity: fftypes.Identity{ + Author: "org1", + }, + }, + }, + }, + }, + }, + }, nodes) + assert.Regexp(t, "pop", err) + + mdx.AssertExpectations(t) + +} + +func TestSendDataTransferInsertOperationFail(t *testing.T) { + + pm, cancel := newTestPrivateMessaging(t) + defer cancel() + + mim := pm.identity.(*identitymanagermocks.Manager) + mim.On("ResolveInputIdentity", pm.ctx, mock.MatchedBy(func(identity *fftypes.Identity) bool { + assert.Equal(t, "localorg", identity.Author) + return true + })).Return(nil) + mim.On("GetLocalOrgKey", pm.ctx).Return("localorg", nil) + + mdx := pm.exchange.(*dataexchangemocks.Plugin) + mdx.On("SendMessage", pm.ctx, "peer2-remote", mock.Anything).Return("", nil) + + mdi := pm.database.(*databasemocks.Plugin) + mdi.On("InsertOperation", pm.ctx, mock.Anything).Return(fmt.Errorf("pop")) + + groupID := fftypes.NewRandB32() + nodeID2 := fftypes.NewUUID() + + nodes := []*fftypes.Node{{ + ID: nodeID2, Name: "node2", Owner: "org1", DX: fftypes.DXInfo{Peer: "peer2-remote"}, + }} + + err := pm.sendData(pm.ctx, &fftypes.TransportWrapper{ + Batch: &fftypes.Batch{ + ID: fftypes.NewUUID(), + Group: groupID, + Payload: fftypes.BatchPayload{ + Messages: []*fftypes.Message{ + { + Header: fftypes.MessageHeader{ + Tag: "mytag", + Group: groupID, + Identity: fftypes.Identity{ + Author: "org1", + }, + }, + }, + }, + }, + }, + }, nodes) + assert.Regexp(t, "pop", err) + + mdx.AssertExpectations(t) + +} + +func TestDispatchedUnpinnedMessageGetOrgFail(t *testing.T) { + + pm, cancel := newTestPrivateMessaging(t) + defer cancel() + + mim := pm.identity.(*identitymanagermocks.Manager) + mim.On("ResolveInputIdentity", pm.ctx, mock.MatchedBy(func(identity *fftypes.Identity) bool { + assert.Equal(t, "localorg", identity.Author) + return true + })).Return(nil) + mim.On("GetLocalOrgKey", pm.ctx).Return("", fmt.Errorf("pop")) + + groupID := fftypes.NewRandB32() + nodeID1 := fftypes.NewUUID() + nodeID2 := fftypes.NewUUID() + + mdi := pm.database.(*databasemocks.Plugin) + mdi.On("GetGroupByHash", pm.ctx, groupID).Return(&fftypes.Group{ + Hash: groupID, + GroupIdentity: fftypes.GroupIdentity{ + Members: fftypes.Members{ + {Node: nodeID1, Identity: "localorg"}, + {Node: nodeID2, Identity: "remoteorg"}, + }, + }, + }, nil).Once() + mdi.On("GetNodeByID", pm.ctx, nodeID1).Return(&fftypes.Node{ + ID: nodeID1, Name: "node1", Owner: "localorg", DX: fftypes.DXInfo{Peer: "peer1-local"}, + }, nil).Once() + mdi.On("GetNodeByID", pm.ctx, nodeID2).Return(&fftypes.Node{ + ID: nodeID2, Name: "node2", Owner: "org1", DX: fftypes.DXInfo{Peer: "peer2-remote"}, + }, nil).Once() + + err := pm.dispatchUnpinnedBatch(pm.ctx, &fftypes.Batch{ + ID: fftypes.NewUUID(), + Group: groupID, + Payload: fftypes.BatchPayload{ + Messages: []*fftypes.Message{ + { + Header: fftypes.MessageHeader{ + Tag: "mytag", + Group: groupID, + Identity: fftypes.Identity{ + Author: "org1", + }, + }, + }, + }, + }, + }, []*fftypes.Bytes32{}) + assert.Regexp(t, "pop", err) + + mdi.AssertExpectations(t) + +} diff --git a/internal/privatemessaging/privatemessaging.go b/internal/privatemessaging/privatemessaging.go index f579f874da..08bb198032 100644 --- a/internal/privatemessaging/privatemessaging.go +++ b/internal/privatemessaging/privatemessaging.go @@ -38,6 +38,7 @@ import ( ) const pinnedPrivateDispatcherName = "pinned_private" +const unpinnedPrivateDispatcherName = "unpinned_private" type Manager interface { GroupManager @@ -109,11 +110,21 @@ func NewPrivateMessaging(ctx context.Context, di database.Plugin, im identity.Ma DisposeTimeout: config.GetDuration(config.PrivateMessagingBatchAgentTimeout), } - ba.RegisterDispatcher(pinnedPrivateDispatcherName, []fftypes.MessageType{ - fftypes.MessageTypeGroupInit, - fftypes.MessageTypePrivate, - fftypes.MessageTypeTransferPrivate, - }, pm.dispatchBatch, bo) + ba.RegisterDispatcher(pinnedPrivateDispatcherName, + fftypes.TransactionTypeBatchPin, + []fftypes.MessageType{ + fftypes.MessageTypeGroupInit, + fftypes.MessageTypePrivate, + fftypes.MessageTypeTransferPrivate, + }, + pm.dispatchPinnedBatch, bo) + + ba.RegisterDispatcher(unpinnedPrivateDispatcherName, + fftypes.TransactionTypeNone, + []fftypes.MessageType{ + fftypes.MessageTypePrivate, + }, + pm.dispatchUnpinnedBatch, bo) return pm, nil } @@ -122,23 +133,37 @@ func (pm *privateMessaging) Start() error { return pm.exchange.Start() } -func (pm *privateMessaging) dispatchBatch(ctx context.Context, batch *fftypes.Batch, contexts []*fftypes.Bytes32) error { +func (pm *privateMessaging) dispatchPinnedBatch(ctx context.Context, batch *fftypes.Batch, contexts []*fftypes.Bytes32) error { + err := pm.dispatchBatchCommon(ctx, batch) + if err != nil { + return err + } + + return pm.batchpin.SubmitPinnedBatch(ctx, batch, contexts) +} + +func (pm *privateMessaging) dispatchUnpinnedBatch(ctx context.Context, batch *fftypes.Batch, contexts []*fftypes.Bytes32) error { + return pm.dispatchBatchCommon(ctx, batch) +} - // Serialize the full payload, which has already been sealed for us by the BatchManager +func (pm *privateMessaging) dispatchBatchCommon(ctx context.Context, batch *fftypes.Batch) error { tw := &fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeBatch, Batch: batch, } // Retrieve the group - _, nodes, err := pm.groupManager.getGroupNodes(ctx, batch.Group) + group, nodes, err := pm.groupManager.getGroupNodes(ctx, batch.Group) if err != nil { return err } - return pm.database.RunAsGroup(ctx, func(ctx context.Context) error { - return pm.sendAndSubmitBatch(ctx, batch, nodes, tw, contexts) - }) + if batch.Payload.TX.Type == fftypes.TransactionTypeNone { + // In the case of an un-pinned message we cannot be sure the group has been broadcast via the blockchain. + // So we have to take the hit of sending it along with every message. + tw.Group = group + } + + return pm.sendData(ctx, tw, nodes) } func (pm *privateMessaging) transferBlobs(ctx context.Context, data []*fftypes.Data, txid *fftypes.UUID, node *fftypes.Node) error { @@ -159,24 +184,23 @@ func (pm *privateMessaging) transferBlobs(ctx context.Context, data []*fftypes.D return err } - if txid != nil { - op := fftypes.NewOperation( - pm.exchange, - d.Namespace, - txid, - trackingID, - fftypes.OpTypeDataExchangeBlobSend) - if err = pm.database.InsertOperation(ctx, op); err != nil { - return err - } + op := fftypes.NewOperation( + pm.exchange, + d.Namespace, + txid, + trackingID, + fftypes.OpTypeDataExchangeBlobSend) + if err = pm.database.InsertOperation(ctx, op); err != nil { + return err } } } return nil } -func (pm *privateMessaging) sendData(ctx context.Context, mType string, mID *fftypes.UUID, group *fftypes.Bytes32, ns string, nodes []*fftypes.Node, tw *fftypes.TransportWrapper, txid *fftypes.UUID, data []*fftypes.Data) (err error) { +func (pm *privateMessaging) sendData(ctx context.Context, tw *fftypes.TransportWrapper, nodes []*fftypes.Node) (err error) { l := log.L(ctx) + batch := tw.Batch payload, err := json.Marshal(tw) if err != nil { @@ -193,14 +217,14 @@ func (pm *privateMessaging) sendData(ctx context.Context, mType string, mID *fft for i, node := range nodes { if node.Owner == localOrgSigingKey { - l.Debugf("Skipping send of %s for local node %s:%s for group=%s node=%s (%d/%d)", mType, ns, mID, group, node.ID, i+1, len(nodes)) + l.Debugf("Skipping send of batch for local node %s:%s for group=%s node=%s (%d/%d)", batch.Namespace, batch.ID, batch.Group, node.ID, i+1, len(nodes)) continue } - l.Debugf("Sending %s %s:%s to group=%s node=%s (%d/%d)", mType, ns, mID, group, node.ID, i+1, len(nodes)) + l.Debugf("Sending batch %s:%s to group=%s node=%s (%d/%d)", batch.Namespace, batch.ID, batch.Group, node.ID, i+1, len(nodes)) // Initiate transfer of any blobs first - if err = pm.transferBlobs(ctx, data, txid, node); err != nil { + if err = pm.transferBlobs(ctx, batch.Payload.Data, batch.Payload.TX.ID, node); err != nil { return err } @@ -210,33 +234,20 @@ func (pm *privateMessaging) sendData(ctx context.Context, mType string, mID *fft return err } - if txid != nil { - op := fftypes.NewOperation( - pm.exchange, - ns, - txid, - trackingID, - fftypes.OpTypeDataExchangeBatchSend) - op.Input = fftypes.JSONObject{ - "manifest": tw.Manifest().String(), - } - if err = pm.database.InsertOperation(ctx, op); err != nil { - return err - } + op := fftypes.NewOperation( + pm.exchange, + batch.Namespace, + batch.Payload.TX.ID, + trackingID, + fftypes.OpTypeDataExchangeBatchSend) + op.Input = fftypes.JSONObject{ + "manifest": tw.Batch.Manifest().String(), + } + if err = pm.database.InsertOperation(ctx, op); err != nil { + return err } } return nil } - -func (pm *privateMessaging) sendAndSubmitBatch(ctx context.Context, batch *fftypes.Batch, nodes []*fftypes.Node, tw *fftypes.TransportWrapper, contexts []*fftypes.Bytes32) (err error) { - if err = pm.sendData(ctx, "batch", batch.ID, batch.Group, batch.Namespace, nodes, tw, batch.Payload.TX.ID, batch.Payload.Data); err != nil { - return err - } - return pm.writeTransaction(ctx, batch, contexts) -} - -func (pm *privateMessaging) writeTransaction(ctx context.Context, batch *fftypes.Batch, contexts []*fftypes.Bytes32) error { - return pm.batchpin.SubmitPinnedBatch(ctx, batch, contexts) -} diff --git a/internal/privatemessaging/privatemessaging_test.go b/internal/privatemessaging/privatemessaging_test.go index 776794fc57..0fe654443b 100644 --- a/internal/privatemessaging/privatemessaging_test.go +++ b/internal/privatemessaging/privatemessaging_test.go @@ -50,11 +50,21 @@ func newTestPrivateMessaging(t *testing.T) (*privateMessaging, func()) { msa := &syncasyncmocks.Bridge{} mbp := &batchpinmocks.Submitter{} - mba.On("RegisterDispatcher", []fftypes.MessageType{ - fftypes.MessageTypeGroupInit, - fftypes.MessageTypePrivate, - fftypes.MessageTypeTransferPrivate, - }, mock.Anything, mock.Anything).Return() + mba.On("RegisterDispatcher", + pinnedPrivateDispatcherName, + fftypes.TransactionTypeBatchPin, + []fftypes.MessageType{ + fftypes.MessageTypeGroupInit, + fftypes.MessageTypePrivate, + fftypes.MessageTypeTransferPrivate, + }, mock.Anything, mock.Anything).Return() + + mba.On("RegisterDispatcher", + unpinnedPrivateDispatcherName, + fftypes.TransactionTypeNone, + []fftypes.MessageType{ + fftypes.MessageTypePrivate, + }, mock.Anything, mock.Anything).Return() rag := mdi.On("RunAsGroup", mock.Anything, mock.Anything).Maybe() rag.RunFn = func(a mock.Arguments) { @@ -149,7 +159,7 @@ func TestDispatchBatchWithBlobs(t *testing.T) { mbp.On("SubmitPinnedBatch", pm.ctx, mock.Anything, mock.Anything).Return(nil) - err := pm.dispatchBatch(pm.ctx, &fftypes.Batch{ + err := pm.dispatchPinnedBatch(pm.ctx, &fftypes.Batch{ ID: batchID, Identity: fftypes.Identity{ Author: "org1", @@ -185,7 +195,7 @@ func TestDispatchBatchBadData(t *testing.T) { mdi := pm.database.(*databasemocks.Plugin) mdi.On("GetGroupByHash", pm.ctx, groupID).Return(&fftypes.Group{}, nil) - err := pm.dispatchBatch(pm.ctx, &fftypes.Batch{ + err := pm.dispatchPinnedBatch(pm.ctx, &fftypes.Batch{ Group: groupID, Payload: fftypes.BatchPayload{ Data: []*fftypes.Data{ @@ -203,7 +213,7 @@ func TestDispatchErrorFindingGroup(t *testing.T) { mdi := pm.database.(*databasemocks.Plugin) mdi.On("GetGroupByHash", pm.ctx, mock.Anything).Return(nil, fmt.Errorf("pop")) - err := pm.dispatchBatch(pm.ctx, &fftypes.Batch{}, []*fftypes.Bytes32{}) + err := pm.dispatchPinnedBatch(pm.ctx, &fftypes.Batch{}, []*fftypes.Bytes32{}) assert.Regexp(t, "pop", err) } @@ -224,11 +234,11 @@ func TestSendAndSubmitBatchBadID(t *testing.T) { mbp := pm.batchpin.(*batchpinmocks.Submitter) mbp.On("SubmitPinnedBatch", pm.ctx, mock.Anything, mock.Anything).Return(fmt.Errorf("pop")) - err := pm.sendAndSubmitBatch(pm.ctx, &fftypes.Batch{ + err := pm.dispatchPinnedBatch(pm.ctx, &fftypes.Batch{ Identity: fftypes.Identity{ Author: "badauthor", }, - }, []*fftypes.Node{}, &fftypes.TransportWrapper{}, []*fftypes.Bytes32{}) + }, []*fftypes.Bytes32{}) assert.Regexp(t, "pop", err) } @@ -242,11 +252,11 @@ func TestSendAndSubmitBatchUnregisteredNode(t *testing.T) { mim := pm.identity.(*identitymanagermocks.Manager) mim.On("GetLocalOrgKey", pm.ctx).Return("", fmt.Errorf("pop")) - err := pm.sendAndSubmitBatch(pm.ctx, &fftypes.Batch{ + err := pm.dispatchPinnedBatch(pm.ctx, &fftypes.Batch{ Identity: fftypes.Identity{ Author: "badauthor", }, - }, []*fftypes.Node{}, &fftypes.TransportWrapper{}, []*fftypes.Bytes32{}) + }, []*fftypes.Bytes32{}) assert.Regexp(t, "pop", err) } @@ -254,24 +264,20 @@ func TestSendImmediateFail(t *testing.T) { pm, cancel := newTestPrivateMessaging(t) defer cancel() + mdi := pm.database.(*databasemocks.Plugin) + mdi.On("GetGroupByHash", pm.ctx, mock.Anything).Return(nil, fmt.Errorf("pop")) + mim := pm.identity.(*identitymanagermocks.Manager) mim.On("GetLocalOrgKey", pm.ctx).Return("localorg", nil) mdx := pm.exchange.(*dataexchangemocks.Plugin) mdx.On("SendMessage", pm.ctx, mock.Anything, mock.Anything).Return("", fmt.Errorf("pop")) - err := pm.sendAndSubmitBatch(pm.ctx, &fftypes.Batch{ + err := pm.dispatchPinnedBatch(pm.ctx, &fftypes.Batch{ Identity: fftypes.Identity{ Author: "org1", }, - }, []*fftypes.Node{ - { - DX: fftypes.DXInfo{ - Peer: "node1", - Endpoint: fftypes.JSONObject{"url": "https://node1.example.com"}, - }, - }, - }, &fftypes.TransportWrapper{}, []*fftypes.Bytes32{}) + }, []*fftypes.Bytes32{}) assert.Regexp(t, "pop", err) } @@ -286,9 +292,10 @@ func TestSendSubmitInsertOperationFail(t *testing.T) { mdx.On("SendMessage", pm.ctx, mock.Anything, mock.Anything).Return("tracking1", nil) mdi := pm.database.(*databasemocks.Plugin) + mdi.On("GetGroupByHash", pm.ctx, mock.Anything).Return(nil, fmt.Errorf("pop")) mdi.On("InsertOperation", pm.ctx, mock.Anything).Return(fmt.Errorf("pop")) - err := pm.sendAndSubmitBatch(pm.ctx, &fftypes.Batch{ + err := pm.dispatchPinnedBatch(pm.ctx, &fftypes.Batch{ Identity: fftypes.Identity{ Author: "org1", }, @@ -297,14 +304,7 @@ func TestSendSubmitInsertOperationFail(t *testing.T) { ID: fftypes.NewUUID(), }, }, - }, []*fftypes.Node{ - { - DX: fftypes.DXInfo{ - Peer: "node1", - Endpoint: fftypes.JSONObject{"url": "https://node1.example.com"}, - }, - }, - }, &fftypes.TransportWrapper{}, []*fftypes.Bytes32{}) + }, []*fftypes.Bytes32{}) assert.Regexp(t, "pop", err) } @@ -316,9 +316,10 @@ func TestSendSubmitBlobTransferFail(t *testing.T) { mim.On("GetLocalOrgKey", pm.ctx).Return("localorgkey", nil) mdi := pm.database.(*databasemocks.Plugin) + mdi.On("GetGroupByHash", pm.ctx, mock.Anything).Return(nil, fmt.Errorf("pop")) mdi.On("GetBlobMatchingHash", pm.ctx, mock.Anything).Return(nil, fmt.Errorf("pop")) - err := pm.sendAndSubmitBatch(pm.ctx, &fftypes.Batch{ + err := pm.dispatchPinnedBatch(pm.ctx, &fftypes.Batch{ Identity: fftypes.Identity{ Author: "org1", }, @@ -327,14 +328,7 @@ func TestSendSubmitBlobTransferFail(t *testing.T) { {ID: fftypes.NewUUID(), Blob: &fftypes.BlobRef{Hash: fftypes.NewRandB32()}}, }, }, - }, []*fftypes.Node{ - { - DX: fftypes.DXInfo{ - Peer: "node1", - Endpoint: fftypes.JSONObject{"url": "https://node1.example.com"}, - }, - }, - }, &fftypes.TransportWrapper{}, []*fftypes.Bytes32{}) + }, []*fftypes.Bytes32{}) assert.Regexp(t, "pop", err) } @@ -343,13 +337,14 @@ func TestWriteTransactionSubmitBatchPinFail(t *testing.T) { defer cancel() mdi := pm.database.(*databasemocks.Plugin) + mdi.On("GetGroupByHash", pm.ctx, mock.Anything).Return(nil, fmt.Errorf("pop")) mdi.On("UpsertTransaction", pm.ctx, mock.Anything, true, false).Return(nil) mdi.On("InsertOperation", pm.ctx, mock.Anything).Return(nil) mbp := pm.batchpin.(*batchpinmocks.Submitter) mbp.On("SubmitPinnedBatch", pm.ctx, mock.Anything, mock.Anything).Return(fmt.Errorf("pop")) - err := pm.writeTransaction(pm.ctx, &fftypes.Batch{ + err := pm.dispatchPinnedBatch(pm.ctx, &fftypes.Batch{ Identity: fftypes.Identity{ Author: "org1", }}, []*fftypes.Bytes32{}) diff --git a/internal/privatemessaging/recipients.go b/internal/privatemessaging/recipients.go index c3fed50bb4..bfb1edea10 100644 --- a/internal/privatemessaging/recipients.go +++ b/internal/privatemessaging/recipients.go @@ -1,4 +1,4 @@ -// Copyright © 2021 Kaleido, Inc. +// Copyright © 2022 Kaleido, Inc. // // SPDX-License-Identifier: Apache-2.0 // @@ -30,7 +30,15 @@ import ( func (pm *privateMessaging) resolveRecipientList(ctx context.Context, in *fftypes.MessageInOut) error { if in.Header.Group != nil { log.L(ctx).Debugf("Group '%s' specified for message", in.Header.Group) - return nil // validity of existing group checked later + group, err := pm.database.GetGroupByHash(ctx, in.Header.Group) + if err != nil { + return err + } + if group == nil { + return i18n.NewError(ctx, i18n.MsgGroupNotFound, in.Header.Group) + } + // We have a group already resolved + return nil } if in.Group == nil || len(in.Group.Members) == 0 { return i18n.NewError(ctx, i18n.MsgGroupMustHaveMembers) diff --git a/internal/privatemessaging/recipients_test.go b/internal/privatemessaging/recipients_test.go index c79fd2d78e..9ee867b5f9 100644 --- a/internal/privatemessaging/recipients_test.go +++ b/internal/privatemessaging/recipients_test.go @@ -418,10 +418,14 @@ func TestResolveReceipientListExisting(t *testing.T) { pm, cancel := newTestPrivateMessaging(t) defer cancel() + groupID := fftypes.NewRandB32() + mdi := pm.database.(*databasemocks.Plugin) + mdi.On("GetGroupByHash", pm.ctx, groupID).Return(&fftypes.Group{Hash: groupID}, nil) + err := pm.resolveRecipientList(pm.ctx, &fftypes.MessageInOut{ Message: fftypes.Message{ Header: fftypes.MessageHeader{ - Group: fftypes.NewRandB32(), + Group: groupID, }, }, }) diff --git a/mocks/batchmocks/manager.go b/mocks/batchmocks/manager.go index 4ade0f2ab4..9c98f6b727 100644 --- a/mocks/batchmocks/manager.go +++ b/mocks/batchmocks/manager.go @@ -35,9 +35,9 @@ func (_m *Manager) NewMessages() chan<- int64 { return r0 } -// RegisterDispatcher provides a mock function with given fields: name, msgTypes, handler, batchOptions -func (_m *Manager) RegisterDispatcher(name string, msgTypes []fftypes.FFEnum, handler batch.DispatchHandler, batchOptions batch.DispatcherOptions) { - _m.Called(name, msgTypes, handler, batchOptions) +// RegisterDispatcher provides a mock function with given fields: name, txType, msgTypes, handler, batchOptions +func (_m *Manager) RegisterDispatcher(name string, txType fftypes.FFEnum, msgTypes []fftypes.FFEnum, handler batch.DispatchHandler, batchOptions batch.DispatcherOptions) { + _m.Called(name, txType, msgTypes, handler, batchOptions) } // Start provides a mock function with given fields: diff --git a/pkg/fftypes/manifest_test.go b/pkg/fftypes/manifest_test.go index c22d77eb82..6f8df08127 100644 --- a/pkg/fftypes/manifest_test.go +++ b/pkg/fftypes/manifest_test.go @@ -24,13 +24,16 @@ import ( func TestManifestToString(t *testing.T) { - tw := &TransportWrapper{ - Type: TransportPayloadTypeMessage, - Message: &Message{Header: MessageHeader{ID: MustParseUUID("c38e76ec-92a6-4659-805d-8ae3b7437c40")}, Hash: MustParseBytes32("169ef5233cf44df3d71df59f25928743e9a76378bb1375e06539b732b1fc57e5")}, - Data: []*Data{ - {ID: MustParseUUID("7bc49647-cd1c-4633-98fa-ddbb208d61bd"), Hash: MustParseBytes32("2b849d47e44a291cd83bee4e7ace66178a5245a151d3bbd02011312ec2604ed6")}, - {ID: MustParseUUID("5b80eec3-04b5-4557-bced-6a458ecb9ef2"), Hash: MustParseBytes32("2bcddd992d17e89a5aafbe99c59d954018ddadf4e533a164808ae2389bbf33dc")}, + batch := &Batch{ + Payload: BatchPayload{ + Messages: []*Message{ + {Header: MessageHeader{ID: MustParseUUID("c38e76ec-92a6-4659-805d-8ae3b7437c40")}, Hash: MustParseBytes32("169ef5233cf44df3d71df59f25928743e9a76378bb1375e06539b732b1fc57e5")}, + }, + Data: []*Data{ + {ID: MustParseUUID("7bc49647-cd1c-4633-98fa-ddbb208d61bd"), Hash: MustParseBytes32("2b849d47e44a291cd83bee4e7ace66178a5245a151d3bbd02011312ec2604ed6")}, + {ID: MustParseUUID("5b80eec3-04b5-4557-bced-6a458ecb9ef2"), Hash: MustParseBytes32("2bcddd992d17e89a5aafbe99c59d954018ddadf4e533a164808ae2389bbf33dc")}, + }, }, } - assert.Equal(t, "{\"messages\":[{\"id\":\"c38e76ec-92a6-4659-805d-8ae3b7437c40\",\"hash\":\"169ef5233cf44df3d71df59f25928743e9a76378bb1375e06539b732b1fc57e5\"}],\"data\":[{\"id\":\"7bc49647-cd1c-4633-98fa-ddbb208d61bd\",\"hash\":\"2b849d47e44a291cd83bee4e7ace66178a5245a151d3bbd02011312ec2604ed6\"},{\"id\":\"5b80eec3-04b5-4557-bced-6a458ecb9ef2\",\"hash\":\"2bcddd992d17e89a5aafbe99c59d954018ddadf4e533a164808ae2389bbf33dc\"}]}", tw.Manifest().String()) + assert.Equal(t, "{\"messages\":[{\"id\":\"c38e76ec-92a6-4659-805d-8ae3b7437c40\",\"hash\":\"169ef5233cf44df3d71df59f25928743e9a76378bb1375e06539b732b1fc57e5\"}],\"data\":[{\"id\":\"7bc49647-cd1c-4633-98fa-ddbb208d61bd\",\"hash\":\"2b849d47e44a291cd83bee4e7ace66178a5245a151d3bbd02011312ec2604ed6\"},{\"id\":\"5b80eec3-04b5-4557-bced-6a458ecb9ef2\",\"hash\":\"2bcddd992d17e89a5aafbe99c59d954018ddadf4e533a164808ae2389bbf33dc\"}]}", batch.Manifest().String()) } diff --git a/pkg/fftypes/message.go b/pkg/fftypes/message.go index 02f27c90e6..135dd553ec 100644 --- a/pkg/fftypes/message.go +++ b/pkg/fftypes/message.go @@ -212,6 +212,12 @@ func (m *Message) DupDataCheck(ctx context.Context) (err error) { } func (m *Message) Verify(ctx context.Context) error { + switch m.Header.TxType { + case TransactionTypeBatchPin: + case TransactionTypeNone: + default: + return i18n.NewError(ctx, i18n.MsgInvalidTXTypeForMessage, m.Header.TxType) + } if err := m.Header.Topics.Validate(ctx, "header.topics", true); err != nil { return err } diff --git a/pkg/fftypes/message_test.go b/pkg/fftypes/message_test.go index b9ec461f67..1beb1f67e5 100644 --- a/pkg/fftypes/message_test.go +++ b/pkg/fftypes/message_test.go @@ -65,9 +65,35 @@ func TestSealBadTagString(t *testing.T) { assert.Regexp(t, `FF10131.*header.tag`, err) } +func TestVerifyTXType(t *testing.T) { + msg := Message{ + Header: MessageHeader{ + TxType: TransactionTypeBatchPin, + Topics: []string{"topic1"}, + }, + } + err := msg.Seal(context.Background()) + assert.NoError(t, err) + err = msg.Verify(context.Background()) + assert.NoError(t, err) + + msg.Header.TxType = TransactionTypeNone + err = msg.Seal(context.Background()) + assert.NoError(t, err) + err = msg.Verify(context.Background()) + assert.NoError(t, err) + + msg.Header.TxType = TransactionTypeTokenPool + err = msg.Seal(context.Background()) + assert.NoError(t, err) + err = msg.Verify(context.Background()) + assert.Regexp(t, "FF10343", err) +} + func TestVerifyEmptyTopicString(t *testing.T) { msg := Message{ Header: MessageHeader{ + TxType: TransactionTypeBatchPin, Topics: []string{""}, }, } @@ -78,7 +104,8 @@ func TestVerifyEmptyTopicString(t *testing.T) { func TestVerifyBadTagString(t *testing.T) { msg := Message{ Header: MessageHeader{ - Tag: "!wrong", + TxType: TransactionTypeBatchPin, + Tag: "!wrong", }, } err := msg.Verify(context.Background()) @@ -87,6 +114,9 @@ func TestVerifyBadTagString(t *testing.T) { func TestSealNilDataID(t *testing.T) { msg := Message{ + Header: MessageHeader{ + TxType: TransactionTypeBatchPin, + }, Data: DataRefs{ {}, }, @@ -97,6 +127,9 @@ func TestSealNilDataID(t *testing.T) { func TestVerifyNilDataHash(t *testing.T) { msg := Message{ + Header: MessageHeader{ + TxType: TransactionTypeBatchPin, + }, Data: DataRefs{ {ID: NewUUID()}, }, @@ -124,6 +157,9 @@ func TestVerifylDupDataHash(t *testing.T) { id2 := NewUUID() hash1 := NewRandB32() msg := Message{ + Header: MessageHeader{ + TxType: TransactionTypeBatchPin, + }, Data: DataRefs{ {ID: id1, Hash: hash1}, {ID: id2, Hash: hash1}, @@ -134,7 +170,11 @@ func TestVerifylDupDataHash(t *testing.T) { } func TestVerifyNilHashes(t *testing.T) { - msg := Message{} + msg := Message{ + Header: MessageHeader{ + TxType: TransactionTypeBatchPin, + }, + } err := msg.Verify(context.Background()) assert.Regexp(t, "FF10147", err) } @@ -142,6 +182,7 @@ func TestVerifyNilHashes(t *testing.T) { func TestVerifyNilMisMatchedHashes(t *testing.T) { msg := Message{ Header: MessageHeader{ + TxType: TransactionTypeBatchPin, DataHash: NewRandB32(), }, Hash: NewRandB32(), diff --git a/pkg/fftypes/transport_wrapper.go b/pkg/fftypes/transport_wrapper.go index 3d989d0c71..9ea4bcd7a7 100644 --- a/pkg/fftypes/transport_wrapper.go +++ b/pkg/fftypes/transport_wrapper.go @@ -25,38 +25,8 @@ var ( // TransportWrapper wraps paylaods over data exchange transfers, for easy deserialization at target type TransportWrapper struct { - Type TransportPayloadType `json:"type" ffenum:"transportpayload"` - Message *Message `json:"message,omitempty"` - Data []*Data `json:"data,omitempty"` - Batch *Batch `json:"batch,omitempty"` - Group *Group `json:"group,omitempty"` -} - -// Manifest lists the contents of the transmission in a Manifest, which can be compared with -// a signed receipt provided back by the DX plugin -func (tw *TransportWrapper) Manifest() *Manifest { - if tw.Type == TransportPayloadTypeBatch { - return tw.Batch.Manifest() - } else if tw.Type == TransportPayloadTypeMessage { - tm := &Manifest{ - Messages: []MessageRef{}, - Data: make([]DataRef, len(tw.Data)), - } - if tw.Message != nil { - tm.Messages = []MessageRef{ - { - ID: tw.Message.Header.ID, - Hash: tw.Message.Hash, - }, - } - } - for i, d := range tw.Data { - tm.Data[i].ID = d.ID - tm.Data[i].Hash = d.Hash - } - return tm - } - return nil + Group *Group `json:"group,omitempty"` + Batch *Batch `json:"batch,omitempty"` } type TransportStatusUpdate struct { diff --git a/pkg/fftypes/transport_wrapper_test.go b/pkg/fftypes/transport_wrapper_test.go index 1520c5a662..80207d2e28 100644 --- a/pkg/fftypes/transport_wrapper_test.go +++ b/pkg/fftypes/transport_wrapper_test.go @@ -25,7 +25,6 @@ import ( func TestBatchManifest(t *testing.T) { tw := TransportWrapper{ - Type: TransportPayloadTypeBatch, Batch: &Batch{ Payload: BatchPayload{ Messages: []*Message{ @@ -39,7 +38,7 @@ func TestBatchManifest(t *testing.T) { }, }, } - tm := tw.Manifest() + tm := tw.Batch.Manifest() assert.Equal(t, 2, len(tm.Messages)) assert.Equal(t, tw.Batch.Payload.Messages[0].Header.ID.String(), tm.Messages[0].ID.String()) assert.Equal(t, tw.Batch.Payload.Messages[1].Header.ID.String(), tm.Messages[1].ID.String()) @@ -53,40 +52,9 @@ func TestBatchManifest(t *testing.T) { } -func TestSingleMessageManifest(t *testing.T) { - - tw := TransportWrapper{ - Type: TransportPayloadTypeMessage, - Message: &Message{Header: MessageHeader{ID: NewUUID()}, Hash: NewRandB32()}, - Data: []*Data{ - {ID: NewUUID(), Hash: NewRandB32()}, - {ID: NewUUID(), Hash: NewRandB32()}, - }, - } - tm := tw.Manifest() - assert.Equal(t, 1, len(tm.Messages)) - assert.Equal(t, tw.Message.Header.ID.String(), tm.Messages[0].ID.String()) - assert.Equal(t, tw.Message.Hash.String(), tm.Messages[0].Hash.String()) - assert.Equal(t, 2, len(tm.Data)) - assert.Equal(t, tw.Data[0].ID.String(), tm.Data[0].ID.String()) - assert.Equal(t, tw.Data[1].ID.String(), tm.Data[1].ID.String()) - assert.Equal(t, tw.Data[0].Hash.String(), tm.Data[0].Hash.String()) - assert.Equal(t, tw.Data[1].Hash.String(), tm.Data[1].Hash.String()) - -} - -func TestUnknownManifest(t *testing.T) { - - tw := TransportWrapper{} - assert.Nil(t, tw.Manifest()) - -} - func TestNillBatchManifest(t *testing.T) { - tw := TransportWrapper{ - Type: TransportPayloadTypeBatch, - } - assert.Nil(t, tw.Manifest()) + tw := TransportWrapper{} + assert.Nil(t, tw.Batch.Manifest()) } From 1bd3735e29d95cb413549780c7f90db2acc89039 Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Thu, 10 Feb 2022 10:41:00 -0500 Subject: [PATCH 07/23] Prepare to be merge helled Signed-off-by: Peter Broadhurst --- internal/events/dx_callbacks.go | 98 +++++++-------------------------- 1 file changed, 20 insertions(+), 78 deletions(-) diff --git a/internal/events/dx_callbacks.go b/internal/events/dx_callbacks.go index aac7a7600b..0a0104b0e5 100644 --- a/internal/events/dx_callbacks.go +++ b/internal/events/dx_callbacks.go @@ -38,31 +38,24 @@ func (em *eventManager) MessageReceived(dx dataexchange.Plugin, peerID string, d l.Errorf("Invalid transmission from '%s': %s", peerID, err) return "", nil } + if wrapper.Batch == nil { + l.Errorf("Invalid transmission: nil batch") + return "", nil + } + l.Infof("Private batch received from '%s' (len=%d)", peerID, len(data)) - l.Infof("%s received from '%s' (len=%d)", wrapper.Type, peerID, len(data)) - - var mf *fftypes.Manifest - switch wrapper.Type { - case fftypes.TransportPayloadTypeBatch: - if wrapper.Batch == nil { - l.Errorf("Invalid transmission: nil batch") - return "", nil - } - mf, err = em.pinedBatchReceived(peerID, wrapper.Batch) - case fftypes.TransportPayloadTypeMessage: - if wrapper.Message == nil { - l.Errorf("Invalid transmission: nil message") - return "", nil + if wrapper.Group != nil { + valid, err := em.definitions.EnsureLocalGroup(em.ctx, wrapper.Group) + if err != nil { + return "", err } - if wrapper.Group == nil { - l.Errorf("Invalid transmission: nil group") + if !valid { + l.Errorf("Invalid transmission: invalid group") return "", nil } - mf, err = em.unpinnedMessageReceived(peerID, wrapper) - default: - l.Errorf("Invalid transmission: unknonwn type '%s'", wrapper.Type) - return "", nil } + + mf, err := em.privateBatchReceived(peerID, wrapper.Batch) manifestBytes := []byte{} if err == nil && mf != nil { manifestBytes, err = json.Marshal(&mf) @@ -121,7 +114,7 @@ func (em *eventManager) checkReceivedIdentity(ctx context.Context, peerID, autho return node, nil } -func (em *eventManager) pinedBatchReceived(peerID string, batch *fftypes.Batch) (manifest *fftypes.Manifest, err error) { +func (em *eventManager) privateBatchReceived(peerID string, batch *fftypes.Batch) (manifest *fftypes.Manifest, err error) { // Retry for persistence errors (not validation errors) err = em.retry.Do(em.ctx, "private batch received", func(attempt int) (bool, error) { @@ -138,15 +131,18 @@ func (em *eventManager) pinedBatchReceived(peerID string, batch *fftypes.Batch) } valid, err := em.persistBatch(ctx, batch) - if err != nil { + if err != nil || !valid { l.Errorf("Batch received from %s/%s processing failed valid=%t: %s", node.Owner, node.Name, valid, err) return err // retry - persistBatch only returns retryable errors } - if valid { + if batch.Payload.TX.Type == fftypes.TransactionTypeBatchPin { + // Poke the aggregator to do its stuff em.aggregator.offchainBatches <- batch.ID - manifest = batch.Manifest() + } else { + } + manifest = batch.Manifest() return nil }) }) @@ -280,57 +276,3 @@ func (em *eventManager) TransferResult(dx dataexchange.Plugin, trackingID string }) } - -func (em *eventManager) unpinnedMessageReceived(peerID string, tw *fftypes.TransportWrapper) (manifest *fftypes.Manifest, err error) { - message := tw.Message - - if message == nil || message.Header.TxType != fftypes.TransactionTypeNone { - log.L(em.ctx).Errorf("Unpinned message '%s' transaction type must be 'none'. TxType=%s", message.Header.ID, message.Header.TxType) - return nil, nil - } - - // Because we received this off chain, it's entirely possible the group init has not made it - // to us yet. So we need to go through the same processing as if we had initiated the group. - // This might result in both sides broadcasting a group-init message, but that's fine. - - err = em.retry.Do(em.ctx, "unpinned message received", func(attempt int) (bool, error) { - err := em.database.RunAsGroup(em.ctx, func(ctx context.Context) error { - - if valid, err := em.definitions.EnsureLocalGroup(ctx, tw.Group); err != nil || !valid { - return err - } - - node, err := em.checkReceivedIdentity(ctx, peerID, message.Header.Author, message.Header.Key) - if err != nil { - return err - } - if node == nil { - log.L(ctx).Errorf("Message received from invalid author '%s' for peer ID '%s'", message.Header.Author, peerID) - return nil - } - - // Persist the data - for i, d := range tw.Data { - if ok, err := em.persistReceivedData(ctx, i, d, "message", message.Header.ID, database.UpsertOptimizationSkip); err != nil || !ok { - return err - } - } - - // Persist the message - immediately considered confirmed as this is an unpinned receive - message.Confirmed = fftypes.Now() - if ok, err := em.persistReceivedMessage(ctx, 0, message, "message", message.Header.ID, database.UpsertOptimizationSkip); err != nil || !ok { - return err - } - - // Generate a manifest, as we received it ok - manifest = tw.Manifest() - - // Assuming all was good, we - event := fftypes.NewEvent(fftypes.EventTypeMessageConfirmed, message.Header.Namespace, message.Header.ID) - return em.database.InsertEvent(ctx, event) - }) - return err != nil, err - }) - return manifest, err - -} From c4ef11d375233391609dbf09f8fe22f75efb4c08 Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Thu, 10 Feb 2022 13:33:02 -0500 Subject: [PATCH 08/23] Work through the changes for batch-only private xfers Signed-off-by: Peter Broadhurst --- internal/broadcast/manager_test.go | 1 + internal/events/batch_pin_complete_test.go | 161 +++----- internal/events/dx_callbacks.go | 49 ++- internal/events/dx_callbacks_test.go | 346 ++++++------------ internal/events/persist_batch.go | 10 +- internal/events/persist_batch_test.go | 2 +- internal/i18n/en_translations.go | 1 + internal/privatemessaging/groupmanager.go | 4 + .../privatemessaging/groupmanager_test.go | 9 + 9 files changed, 228 insertions(+), 355 deletions(-) diff --git a/internal/broadcast/manager_test.go b/internal/broadcast/manager_test.go index ddf31f92e3..186762933e 100644 --- a/internal/broadcast/manager_test.go +++ b/internal/broadcast/manager_test.go @@ -54,6 +54,7 @@ func newTestBroadcast(t *testing.T) (*broadcastManager, func()) { mbi.On("Name").Return("ut_blockchain").Maybe() mpi.On("Name").Return("ut_publicstorage").Maybe() mba.On("RegisterDispatcher", + broadcastDispatcherName, fftypes.TransactionTypeBatchPin, []fftypes.MessageType{ fftypes.MessageTypeBroadcast, diff --git a/internal/events/batch_pin_complete_test.go b/internal/events/batch_pin_complete_test.go index 053d72c782..5744cea234 100644 --- a/internal/events/batch_pin_complete_test.go +++ b/internal/events/batch_pin_complete_test.go @@ -36,6 +36,37 @@ import ( "github.com/stretchr/testify/mock" ) +func sampleBatch(t *testing.T, txType fftypes.TransactionType, data ...*fftypes.Data) *fftypes.Batch { + identity := fftypes.Identity{Author: "signingOrg", Key: "0x12345"} + msg := &fftypes.Message{ + Header: fftypes.MessageHeader{ + Identity: identity, + ID: fftypes.NewUUID(), + TxType: txType, + }, + } + for _, d := range data { + err := d.Seal(context.Background(), nil) + assert.NoError(t, err) + } + batch := &fftypes.Batch{ + Identity: identity, + ID: fftypes.NewUUID(), + Node: fftypes.NewUUID(), + Payload: fftypes.BatchPayload{ + TX: fftypes.TransactionRef{ + ID: fftypes.NewUUID(), + Type: txType, + }, + Messages: []*fftypes.Message{msg}, + Data: data, + }, + } + err := msg.Seal(context.Background()) + assert.NoError(t, err) + batch.Hash = batch.Payload.Hash() + return batch +} func TestBatchPinCompleteOkBroadcast(t *testing.T) { em, cancel := newTestEventManager(t) defer cancel() @@ -106,7 +137,7 @@ func TestBatchPinCompleteOkBroadcast(t *testing.T) { return e.Type == fftypes.EventTypeBlockchainEvent })).Return(nil).Times(2) mdi.On("UpsertPin", mock.Anything, mock.Anything).Return(nil).Once() - mdi.On("UpsertBatch", mock.Anything, mock.Anything, false).Return(nil).Once() + mdi.On("UpsertBatch", mock.Anything, mock.Anything).Return(nil).Once() mbi := &blockchainmocks.Plugin{} mim := em.identity.(*identitymanagermocks.Manager) @@ -350,7 +381,7 @@ func TestPersistBatchUpsertBatchMismatchHash(t *testing.T) { batch.Hash = batch.Payload.Hash() mdi := em.database.(*databasemocks.Plugin) - mdi.On("UpsertBatch", mock.Anything, mock.Anything, false).Return(database.HashMismatch) + mdi.On("UpsertBatch", mock.Anything, mock.Anything).Return(database.HashMismatch) valid, err := em.persistBatch(context.Background(), batch) assert.False(t, valid) @@ -400,7 +431,7 @@ func TestPersistBatchUpsertBatchFail(t *testing.T) { batch.Hash = batch.Payload.Hash() mdi := em.database.(*databasemocks.Plugin) - mdi.On("UpsertBatch", mock.Anything, mock.Anything, false).Return(fmt.Errorf("pop")) + mdi.On("UpsertBatch", mock.Anything, mock.Anything).Return(fmt.Errorf("pop")) valid, err := em.persistBatch(context.Background(), batch) assert.False(t, valid) @@ -429,7 +460,7 @@ func TestPersistBatchSwallowBadData(t *testing.T) { batch.Hash = batch.Payload.Hash() mdi := em.database.(*databasemocks.Plugin) - mdi.On("UpsertBatch", mock.Anything, mock.Anything, false).Return(nil) + mdi.On("UpsertBatch", mock.Anything, mock.Anything).Return(nil) valid, err := em.persistBatch(context.Background(), batch) assert.False(t, valid) @@ -462,7 +493,7 @@ func TestPersistBatchGoodDataUpsertOptimizeExistingFail(t *testing.T) { batch.Hash = batch.Payload.Hash() mdi := em.database.(*databasemocks.Plugin) - mdi.On("UpsertBatch", mock.Anything, mock.Anything, false).Return(nil) + mdi.On("UpsertBatch", mock.Anything, mock.Anything).Return(nil) mdi.On("UpsertData", mock.Anything, mock.Anything, database.UpsertOptimizationExisting).Return(fmt.Errorf("pop")) valid, err := em.persistBatch(context.Background(), batch) @@ -495,7 +526,7 @@ func TestPersistBatchGoodDataUpsertOptimizeNewFail(t *testing.T) { batch.Hash = batch.Payload.Hash() mdi := em.database.(*databasemocks.Plugin) - mdi.On("UpsertBatch", mock.Anything, mock.Anything, false).Return(nil) + mdi.On("UpsertBatch", mock.Anything, mock.Anything).Return(nil) mdi.On("UpsertData", mock.Anything, mock.Anything, database.UpsertOptimizationNew).Return(fmt.Errorf("pop")) valid, err := em.persistBatch(context.Background(), batch) @@ -506,35 +537,14 @@ func TestPersistBatchGoodDataUpsertOptimizeNewFail(t *testing.T) { func TestPersistBatchGoodDataMessageFail(t *testing.T) { em, cancel := newTestEventManager(t) defer cancel() - batch := &fftypes.Batch{ - ID: fftypes.NewUUID(), - Identity: fftypes.Identity{ - Author: "author1", - Key: "0x12345", - }, - Namespace: "ns1", - Payload: fftypes.BatchPayload{ - TX: fftypes.TransactionRef{ - Type: fftypes.TransactionTypeBatchPin, - ID: fftypes.NewUUID(), - }, - Messages: []*fftypes.Message{ - {Header: fftypes.MessageHeader{ - ID: fftypes.NewUUID(), - Identity: fftypes.Identity{ - Author: "author1", - Key: "0x12345", - }, - }}, - }, - }, - } + batch := sampleBatch(t, fftypes.TransactionTypeBatchPin) + batch.Node = nil batch.Payload.Messages[0].Header.DataHash = batch.Payload.Messages[0].Data.Hash() batch.Payload.Messages[0].Hash = batch.Payload.Messages[0].Header.Hash() batch.Hash = batch.Payload.Hash() mdi := em.database.(*databasemocks.Plugin) - mdi.On("UpsertBatch", mock.Anything, mock.Anything, false).Return(nil) + mdi.On("UpsertBatch", mock.Anything, mock.Anything).Return(nil) mdi.On("UpsertMessage", mock.Anything, mock.Anything, database.UpsertOptimizationSkip).Return(fmt.Errorf("pop")) valid, err := em.persistBatch(context.Background(), batch) @@ -545,35 +555,14 @@ func TestPersistBatchGoodDataMessageFail(t *testing.T) { func TestPersistBatchGoodMessageAuthorMismatch(t *testing.T) { em, cancel := newTestEventManager(t) defer cancel() - batch := &fftypes.Batch{ - ID: fftypes.NewUUID(), - Identity: fftypes.Identity{ - Author: "author1", - Key: "0x12345", - }, - Namespace: "ns1", - Payload: fftypes.BatchPayload{ - TX: fftypes.TransactionRef{ - Type: fftypes.TransactionTypeBatchPin, - ID: fftypes.NewUUID(), - }, - Messages: []*fftypes.Message{ - {Header: fftypes.MessageHeader{ - ID: fftypes.NewUUID(), - Identity: fftypes.Identity{ - Author: "author1", - Key: "0x9999999", - }, - }}, - }, - }, - } + batch := sampleBatch(t, fftypes.TransactionTypeBatchPin) + batch.Payload.Messages[0].Header.Key = "0x9999999" batch.Payload.Messages[0].Header.DataHash = batch.Payload.Messages[0].Data.Hash() batch.Payload.Messages[0].Hash = batch.Payload.Messages[0].Header.Hash() batch.Hash = batch.Payload.Hash() mdi := em.database.(*databasemocks.Plugin) - mdi.On("UpsertBatch", mock.Anything, mock.Anything, false).Return(nil) + mdi.On("UpsertBatch", mock.Anything, mock.Anything).Return(nil) valid, err := em.persistBatch(context.Background(), batch) assert.False(t, valid) @@ -596,14 +585,12 @@ func TestPersistBatchDataNilData(t *testing.T) { func TestPersistBatchDataBadHash(t *testing.T) { em, cancel := newTestEventManager(t) defer cancel() - batch := &fftypes.Batch{ - ID: fftypes.NewUUID(), - } data := &fftypes.Data{ ID: fftypes.NewUUID(), Value: fftypes.JSONAnyPtr(`"test"`), - Hash: fftypes.NewRandB32(), } + batch := sampleBatch(t, fftypes.TransactionTypeBatchPin, data) + batch.Payload.Data[0].Hash = fftypes.NewRandB32() err := em.persistBatchData(context.Background(), batch, 0, data, database.UpsertOptimizationSkip) assert.NoError(t, err) } @@ -611,9 +598,8 @@ func TestPersistBatchDataBadHash(t *testing.T) { func TestPersistBatchDataUpsertHashMismatch(t *testing.T) { em, cancel := newTestEventManager(t) defer cancel() - batch := &fftypes.Batch{ - ID: fftypes.NewUUID(), - } + batch := sampleBatch(t, fftypes.TransactionTypeBatchPin) + batch.Hash = fftypes.NewRandB32() data := &fftypes.Data{ID: fftypes.NewUUID(), Value: fftypes.JSONAnyPtr(`"test"`)} data.Hash = data.Value.Hash() @@ -629,12 +615,9 @@ func TestPersistBatchDataUpsertHashMismatch(t *testing.T) { func TestPersistBatchDataUpsertDataError(t *testing.T) { em, cancel := newTestEventManager(t) defer cancel() - batch := &fftypes.Batch{ - ID: fftypes.NewUUID(), - } data := &fftypes.Data{ID: fftypes.NewUUID(), Value: fftypes.JSONAnyPtr(`"test"`)} - data.Hash = data.Value.Hash() + batch := sampleBatch(t, fftypes.TransactionTypeBatchPin, data) mdi := em.database.(*databasemocks.Plugin) mdi.On("UpsertData", mock.Anything, mock.Anything, database.UpsertOptimizationSkip).Return(fmt.Errorf("pop")) @@ -646,12 +629,9 @@ func TestPersistBatchDataUpsertDataError(t *testing.T) { func TestPersistBatchDataOk(t *testing.T) { em, cancel := newTestEventManager(t) defer cancel() - batch := &fftypes.Batch{ - ID: fftypes.NewUUID(), - } data := &fftypes.Data{ID: fftypes.NewUUID(), Value: fftypes.JSONAnyPtr(`"test"`)} - data.Hash = data.Value.Hash() + batch := sampleBatch(t, fftypes.TransactionTypeBatchPin, data) mdi := em.database.(*databasemocks.Plugin) mdi.On("UpsertData", mock.Anything, mock.Anything, database.UpsertOptimizationSkip).Return(nil) @@ -680,22 +660,13 @@ func TestPersistBatchMessageNilData(t *testing.T) { func TestPersistBatchMessageUpsertHashMismatch(t *testing.T) { em, cancel := newTestEventManager(t) defer cancel() - batch := &fftypes.Batch{ - ID: fftypes.NewUUID(), - } - msg := &fftypes.Message{ - Header: fftypes.MessageHeader{ - ID: fftypes.NewUUID(), - }, - } - msg.Header.DataHash = msg.Data.Hash() - msg.Hash = msg.Header.Hash() - assert.NoError(t, msg.Verify(context.Background())) + + batch := sampleBatch(t, fftypes.TransactionTypeBatchPin) mdi := em.database.(*databasemocks.Plugin) mdi.On("UpsertMessage", mock.Anything, mock.Anything, database.UpsertOptimizationSkip).Return(database.HashMismatch) - valid, err := em.persistBatchMessage(context.Background(), batch, 0, msg, database.UpsertOptimizationSkip) + valid, err := em.persistBatchMessage(context.Background(), batch, 0, batch.Payload.Messages[0], database.UpsertOptimizationSkip) assert.False(t, valid) assert.NoError(t, err) mdi.AssertExpectations(t) @@ -704,22 +675,12 @@ func TestPersistBatchMessageUpsertHashMismatch(t *testing.T) { func TestPersistBatchMessageUpsertMessageFail(t *testing.T) { em, cancel := newTestEventManager(t) defer cancel() - batch := &fftypes.Batch{ - ID: fftypes.NewUUID(), - } - msg := &fftypes.Message{ - Header: fftypes.MessageHeader{ - ID: fftypes.NewUUID(), - }, - } - msg.Header.DataHash = msg.Data.Hash() - msg.Hash = msg.Header.Hash() - assert.NoError(t, msg.Verify(context.Background())) + batch := sampleBatch(t, fftypes.TransactionTypeBatchPin) mdi := em.database.(*databasemocks.Plugin) mdi.On("UpsertMessage", mock.Anything, mock.Anything, database.UpsertOptimizationSkip).Return(fmt.Errorf("pop")) - valid, err := em.persistBatchMessage(context.Background(), batch, 0, msg, database.UpsertOptimizationSkip) + valid, err := em.persistBatchMessage(context.Background(), batch, 0, batch.Payload.Messages[0], database.UpsertOptimizationSkip) assert.False(t, valid) assert.EqualError(t, err, "pop") } @@ -727,22 +688,12 @@ func TestPersistBatchMessageUpsertMessageFail(t *testing.T) { func TestPersistBatchMessageOK(t *testing.T) { em, cancel := newTestEventManager(t) defer cancel() - batch := &fftypes.Batch{ - ID: fftypes.NewUUID(), - } - msg := &fftypes.Message{ - Header: fftypes.MessageHeader{ - ID: fftypes.NewUUID(), - }, - } - msg.Header.DataHash = msg.Data.Hash() - msg.Hash = msg.Header.Hash() - assert.NoError(t, msg.Verify(context.Background())) + batch := sampleBatch(t, fftypes.TransactionTypeBatchPin) mdi := em.database.(*databasemocks.Plugin) mdi.On("UpsertMessage", mock.Anything, mock.Anything, database.UpsertOptimizationSkip).Return(nil) - valid, err := em.persistBatchMessage(context.Background(), batch, 0, msg, database.UpsertOptimizationSkip) + valid, err := em.persistBatchMessage(context.Background(), batch, 0, batch.Payload.Messages[0], database.UpsertOptimizationSkip) assert.True(t, valid) assert.NoError(t, err) mdi.AssertExpectations(t) diff --git a/internal/events/dx_callbacks.go b/internal/events/dx_callbacks.go index a4ea351ae7..5332100d91 100644 --- a/internal/events/dx_callbacks.go +++ b/internal/events/dx_callbacks.go @@ -18,6 +18,7 @@ package events import ( "context" + "database/sql/driver" "encoding/json" "github.com/hyperledger/firefly/internal/i18n" @@ -44,7 +45,7 @@ func (em *eventManager) MessageReceived(dx dataexchange.Plugin, peerID string, d } l.Infof("Private batch received from '%s' (len=%d)", peerID, len(data)) - if wrapper.Group != nil { + if wrapper.Batch.Payload.TX.Type == fftypes.TransactionTypeNone { valid, err := em.definitions.EnsureLocalGroup(em.ctx, wrapper.Group) if err != nil { return "", err @@ -139,8 +140,11 @@ func (em *eventManager) privateBatchReceived(peerID string, batch *fftypes.Batch if batch.Payload.TX.Type == fftypes.TransactionTypeBatchPin { // Poke the aggregator to do its stuff em.aggregator.offchainBatches <- batch.ID - } else { - + } else if batch.Payload.TX.Type == fftypes.TransactionTypeNone { + // We need to confirm all these messages immediately. + if err := em.markUnpinnedMessagesConfirmed(ctx, batch); err != nil { + return err + } } manifest = batch.Manifest() return nil @@ -150,6 +154,39 @@ func (em *eventManager) privateBatchReceived(peerID string, batch *fftypes.Batch } +func (em *eventManager) markUnpinnedMessagesConfirmed(ctx context.Context, batch *fftypes.Batch) error { + + // Update all the messages in the batch with the batch ID + msgIDs := make([]driver.Value, len(batch.Payload.Messages)) + for i, msg := range batch.Payload.Messages { + msgIDs[i] = msg.Header.ID + } + fb := database.MessageQueryFactory.NewFilter(ctx) + filter := fb.And( + fb.In("id", msgIDs), + fb.Eq("state", fftypes.MessageStatePending), // In the outside chance another state transition happens first (which supersedes this) + ) + + // Immediate confirmation if no transaction + update := database.MessageQueryFactory.NewUpdate(ctx). + Set("batch", batch.ID). + Set("state", fftypes.MessageStateConfirmed). + Set("confirmed", fftypes.Now()) + + if err := em.database.UpdateMessages(ctx, filter, update); err != nil { + return err + } + + for _, msg := range batch.Payload.Messages { + event := fftypes.NewEvent(fftypes.EventTypeMessageConfirmed, batch.Namespace, msg.Header.ID) + if err := em.database.InsertEvent(ctx, event); err != nil { + return err + } + } + + return nil +} + func (em *eventManager) BLOBReceived(dx dataexchange.Plugin, peerID string, hash fftypes.Bytes32, size int64, payloadRef string) error { l := log.L(em.ctx) l.Debugf("Blob received event from data exhange: Peer='%s' Hash='%v' PayloadRef='%s'", peerID, &hash, payloadRef) @@ -234,7 +271,7 @@ func (em *eventManager) TransferResult(dx dataexchange.Plugin, trackingID string var operations []*fftypes.Operation fb := database.OperationQueryFactory.NewFilter(em.ctx) filter := fb.And( - fb.Eq("id", trackingID), + fb.Eq("backendid", trackingID), fb.Eq("plugin", dx.Name()), ) operations, _, err = em.database.GetOperations(em.ctx, filter) @@ -252,7 +289,7 @@ func (em *eventManager) TransferResult(dx dataexchange.Plugin, trackingID string return true, i18n.NewError(em.ctx, i18n.Msg404NotFound) } - // The manifest should exactly match that stored into the operation input, if supported + // The maniest should exactly match that stored into the operation input, if supported op := operations[0] if status == fftypes.OpStatusSucceeded && dx.Capabilities().Manifest { expectedManifest := op.Input.GetString("manifest") @@ -268,7 +305,7 @@ func (em *eventManager) TransferResult(dx dataexchange.Plugin, trackingID string update := database.OperationQueryFactory.NewUpdate(em.ctx). Set("status", status). Set("error", update.Error). - Set("output", update.Info) // Note that we don't need the manifest to be kept here, as it's already in the input + Set("output", update.Info) // We don't need the manifest to be kept here, as it's already in the input if err := em.database.UpdateOperation(em.ctx, op.ID, update); err != nil { return true, err // this is always retryable } diff --git a/internal/events/dx_callbacks_test.go b/internal/events/dx_callbacks_test.go index 3a298cf999..bac19b5230 100644 --- a/internal/events/dx_callbacks_test.go +++ b/internal/events/dx_callbacks_test.go @@ -31,27 +31,22 @@ import ( "github.com/stretchr/testify/mock" ) -func TestMessageReceiveOK(t *testing.T) { - em, cancel := newTestEventManager(t) - defer cancel() - - batch := &fftypes.Batch{ - ID: fftypes.NewUUID(), - Identity: fftypes.Identity{ - Author: "signingOrg", - Key: "0x12345", - }, - Payload: fftypes.BatchPayload{ - TX: fftypes.TransactionRef{ - ID: fftypes.NewUUID(), - }, - }, - } - batch.Hash = batch.Payload.Hash() +func sampleBatchTransfer(t *testing.T, txType fftypes.TransactionType, data ...*fftypes.Data) (*fftypes.Batch, []byte) { + batch := sampleBatch(t, txType, data...) b, _ := json.Marshal(&fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeBatch, Batch: batch, + Group: &fftypes.Group{ + Hash: fftypes.NewRandB32(), + }, }) + return batch, b +} + +func TestPinnedReceiveOK(t *testing.T) { + em, cancel := newTestEventManager(t) + defer cancel() + + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeBatchPin) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} @@ -64,7 +59,8 @@ func TestMessageReceiveOK(t *testing.T) { mdi.On("GetOrganizationByIdentity", em.ctx, "parentOrg").Return(&fftypes.Organization{ Identity: "parentOrg", }, nil) - mdi.On("UpsertBatch", em.ctx, mock.Anything, false).Return(nil, nil) + mdi.On("UpsertBatch", em.ctx, mock.Anything).Return(nil, nil) + mdi.On("UpsertMessage", em.ctx, mock.Anything, database.UpsertOptimizationNew).Return(nil, nil) m, err := em.MessageReceived(mdx, "peer1", b) assert.NoError(t, err) assert.NotNil(t, m) @@ -77,17 +73,7 @@ func TestMessageReceiveOkBadBatchIgnored(t *testing.T) { em, cancel := newTestEventManager(t) defer cancel() - batch := &fftypes.Batch{ - ID: nil, // so that we only test up to persistBatch which will return a non-retry error - Identity: fftypes.Identity{ - Author: "signingOrg", - Key: "0x12345", - }, - } - b, _ := json.Marshal(&fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeBatch, - Batch: batch, - }) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeTokenPool) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} @@ -112,23 +98,7 @@ func TestMessageReceivePersistBatchError(t *testing.T) { em, cancel := newTestEventManager(t) cancel() // retryable error - batch := &fftypes.Batch{ - ID: fftypes.NewUUID(), - Identity: fftypes.Identity{ - Author: "signingOrg", - Key: "0x12345", - }, - Payload: fftypes.BatchPayload{ - TX: fftypes.TransactionRef{ - ID: fftypes.NewUUID(), - }, - }, - } - batch.Hash = batch.Payload.Hash() - b, _ := json.Marshal(&fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeBatch, - Batch: batch, - }) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeBatchPin) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} @@ -141,7 +111,7 @@ func TestMessageReceivePersistBatchError(t *testing.T) { mdi.On("GetOrganizationByIdentity", em.ctx, "parentOrg").Return(&fftypes.Organization{ Identity: "parentOrg", }, nil) - mdi.On("UpsertBatch", em.ctx, mock.Anything, false).Return(fmt.Errorf("pop")) + mdi.On("UpsertBatch", em.ctx, mock.Anything).Return(fmt.Errorf("pop")) m, err := em.MessageReceived(mdx, "peer1", b) assert.Regexp(t, "FF10158", err) assert.Empty(t, m) @@ -219,7 +189,6 @@ func TestMessageReceiveNodeLookupError(t *testing.T) { batch := &fftypes.Batch{} b, _ := json.Marshal(&fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeBatch, Batch: batch, }) @@ -237,7 +206,6 @@ func TestMessageReceiveNodeNotFound(t *testing.T) { batch := &fftypes.Batch{} b, _ := json.Marshal(&fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeBatch, Batch: batch, }) @@ -255,7 +223,6 @@ func TestMessageReceiveAuthorLookupError(t *testing.T) { batch := &fftypes.Batch{} b, _ := json.Marshal(&fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeBatch, Batch: batch, }) @@ -276,7 +243,6 @@ func TestMessageReceiveAuthorNotFound(t *testing.T) { batch := &fftypes.Batch{} b, _ := json.Marshal(&fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeBatch, Batch: batch, }) @@ -295,17 +261,7 @@ func TestMessageReceiveGetCandidateOrgFail(t *testing.T) { em, cancel := newTestEventManager(t) cancel() // retryable error so we need to break the loop - batch := &fftypes.Batch{ - ID: nil, // so that we only test up to persistBatch which will return a non-retry error - Identity: fftypes.Identity{ - Author: "signingOrg", - Key: "0x12345", - }, - } - b, _ := json.Marshal(&fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeBatch, - Batch: batch, - }) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeBatchPin) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} @@ -328,17 +284,7 @@ func TestMessageReceiveGetCandidateOrgNotFound(t *testing.T) { em, cancel := newTestEventManager(t) defer cancel() - batch := &fftypes.Batch{ - ID: nil, // so that we only test up to persistBatch which will return a non-retry error - Identity: fftypes.Identity{ - Author: "signingOrg", - Key: "0x12345", - }, - } - b, _ := json.Marshal(&fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeBatch, - Batch: batch, - }) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeBatchPin) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} @@ -361,17 +307,7 @@ func TestMessageReceiveGetCandidateOrgNotMatch(t *testing.T) { em, cancel := newTestEventManager(t) defer cancel() - batch := &fftypes.Batch{ - ID: nil, // so that we only test up to persistBatch which will return a non-retry error - Identity: fftypes.Identity{ - Author: "signingOrg", - Key: "0x12345", - }, - } - b, _ := json.Marshal(&fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeBatch, - Batch: batch, - }) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeBatchPin) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} @@ -609,51 +545,33 @@ func TestTransferUpdateFail(t *testing.T) { } -func TestMessageReceiveMessageWrongType(t *testing.T) { +func TestMessageReceiveMessageIdentityFail(t *testing.T) { em, cancel := newTestEventManager(t) - defer cancel() + cancel() // to avoid infinite retry - msg := &fftypes.Message{ - Header: fftypes.MessageHeader{ - ID: fftypes.NewUUID(), - TxType: fftypes.TransactionTypeBatchPin, - }, - } - b, _ := json.Marshal(&fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeMessage, - Message: msg, - Group: &fftypes.Group{}, - }) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeBatchPin) + mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} + + msh := em.definitions.(*definitionsmocks.DefinitionHandlers) + msh.On("EnsureLocalGroup", em.ctx, mock.Anything).Return(true, nil) + + mdi.On("GetNodes", em.ctx, mock.Anything).Return(nil, nil, fmt.Errorf("pop")) + m, err := em.MessageReceived(mdx, "peer1", b) - assert.NoError(t, err) + assert.Regexp(t, "FF10158", err) assert.Empty(t, m) + mdi.AssertExpectations(t) mdx.AssertExpectations(t) } -func TestMessageReceiveMessageIdentityFail(t *testing.T) { +func TestMessageReceiveMessageIdentityIncorrect(t *testing.T) { em, cancel := newTestEventManager(t) cancel() // to avoid infinite retry - msg := &fftypes.Message{ - Header: fftypes.MessageHeader{ - Identity: fftypes.Identity{ - Author: "signingOrg", - Key: "0x12345", - }, - ID: fftypes.NewUUID(), - TxType: fftypes.TransactionTypeNone, - }, - } - err := msg.Seal(em.ctx) - assert.NoError(t, err) - b, _ := json.Marshal(&fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeMessage, - Message: msg, - Group: &fftypes.Group{}, - }) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeNone) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} @@ -661,37 +579,21 @@ func TestMessageReceiveMessageIdentityFail(t *testing.T) { msh := em.definitions.(*definitionsmocks.DefinitionHandlers) msh.On("EnsureLocalGroup", em.ctx, mock.Anything).Return(true, nil) - mdi.On("GetNodes", em.ctx, mock.Anything).Return(nil, nil, fmt.Errorf("pop")) + mdi.On("GetNodes", em.ctx, mock.Anything).Return([]*fftypes.Node{}, nil, nil) m, err := em.MessageReceived(mdx, "peer1", b) - assert.Regexp(t, "FF10158", err) + assert.NoError(t, err) assert.Empty(t, m) mdi.AssertExpectations(t) mdx.AssertExpectations(t) } -func TestMessageReceiveMessageIdentityIncorrect(t *testing.T) { +func TestMessageReceiveMessagePersistMessageFail(t *testing.T) { em, cancel := newTestEventManager(t) cancel() // to avoid infinite retry - msg := &fftypes.Message{ - Header: fftypes.MessageHeader{ - Identity: fftypes.Identity{ - Author: "signingOrg", - Key: "0x12345", - }, - ID: fftypes.NewUUID(), - TxType: fftypes.TransactionTypeNone, - }, - } - err := msg.Seal(em.ctx) - assert.NoError(t, err) - b, _ := json.Marshal(&fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeMessage, - Message: msg, - Group: &fftypes.Group{}, - }) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeNone) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} @@ -699,37 +601,32 @@ func TestMessageReceiveMessageIdentityIncorrect(t *testing.T) { msh := em.definitions.(*definitionsmocks.DefinitionHandlers) msh.On("EnsureLocalGroup", em.ctx, mock.Anything).Return(true, nil) - mdi.On("GetNodes", em.ctx, mock.Anything).Return([]*fftypes.Node{}, nil, nil) + mdi.On("GetNodes", em.ctx, mock.Anything).Return([]*fftypes.Node{ + {Name: "node1", Owner: "0x12345"}, + }, nil, nil) + mdi.On("GetOrganizationByIdentity", em.ctx, "0x12345").Return(&fftypes.Organization{ + Identity: "0x12345", + }, nil) + mdi.On("UpsertBatch", em.ctx, mock.Anything).Return(nil, nil) + mdi.On("UpsertMessage", em.ctx, mock.Anything, database.UpsertOptimizationNew).Return(fmt.Errorf("pop")) m, err := em.MessageReceived(mdx, "peer1", b) - assert.NoError(t, err) + assert.Regexp(t, "FF10158", err) assert.Empty(t, m) mdi.AssertExpectations(t) mdx.AssertExpectations(t) } -func TestMessageReceiveMessagePersistMessageFail(t *testing.T) { +func TestMessageReceiveMessagePersistDataFail(t *testing.T) { em, cancel := newTestEventManager(t) cancel() // to avoid infinite retry - msg := &fftypes.Message{ - Header: fftypes.MessageHeader{ - Identity: fftypes.Identity{ - Author: "signingOrg", - Key: "0x12345", - }, - ID: fftypes.NewUUID(), - TxType: fftypes.TransactionTypeNone, - }, + data := &fftypes.Data{ + ID: fftypes.NewUUID(), + Value: fftypes.JSONAnyPtr(`{}`), } - err := msg.Seal(em.ctx) - assert.NoError(t, err) - b, _ := json.Marshal(&fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeMessage, - Message: msg, - Group: &fftypes.Group{}, - }) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeNone, data) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} @@ -743,7 +640,8 @@ func TestMessageReceiveMessagePersistMessageFail(t *testing.T) { mdi.On("GetOrganizationByIdentity", em.ctx, "0x12345").Return(&fftypes.Organization{ Identity: "0x12345", }, nil) - mdi.On("UpsertMessage", em.ctx, mock.Anything, database.UpsertOptimizationSkip).Return(fmt.Errorf("pop")) + mdi.On("UpsertBatch", em.ctx, mock.Anything).Return(nil, nil) + mdi.On("UpsertData", em.ctx, mock.Anything, database.UpsertOptimizationNew).Return(fmt.Errorf("pop")) m, err := em.MessageReceived(mdx, "peer1", b) assert.Regexp(t, "FF10158", err) @@ -753,34 +651,50 @@ func TestMessageReceiveMessagePersistMessageFail(t *testing.T) { mdx.AssertExpectations(t) } -func TestMessageReceiveMessagePersistDataFail(t *testing.T) { +func TestMessageReceiveUnpinnedBatchOk(t *testing.T) { em, cancel := newTestEventManager(t) cancel() // to avoid infinite retry - msg := &fftypes.Message{ - Header: fftypes.MessageHeader{ - Identity: fftypes.Identity{ - Author: "signingOrg", - Key: "0x12345", - }, - ID: fftypes.NewUUID(), - TxType: fftypes.TransactionTypeNone, - }, - } data := &fftypes.Data{ ID: fftypes.NewUUID(), Value: fftypes.JSONAnyPtr(`{}`), } - err := msg.Seal(em.ctx) - assert.NoError(t, err) - err = data.Seal(em.ctx, nil) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeNone, data) + + mdi := em.database.(*databasemocks.Plugin) + mdx := &dataexchangemocks.Plugin{} + + msh := em.definitions.(*definitionsmocks.DefinitionHandlers) + msh.On("EnsureLocalGroup", em.ctx, mock.Anything).Return(true, nil) + + mdi.On("GetNodes", em.ctx, mock.Anything).Return([]*fftypes.Node{ + {Name: "node1", Owner: "0x12345"}, + }, nil, nil) + mdi.On("GetOrganizationByIdentity", em.ctx, "0x12345").Return(&fftypes.Organization{ + Identity: "0x12345", + }, nil) + mdi.On("UpsertBatch", em.ctx, mock.Anything).Return(nil, nil) + mdi.On("UpsertData", em.ctx, mock.Anything, database.UpsertOptimizationNew).Return(nil) + mdi.On("UpsertMessage", em.ctx, mock.Anything, database.UpsertOptimizationNew).Return(nil) + mdi.On("UpdateMessages", em.ctx, mock.Anything, mock.Anything).Return(nil) + mdi.On("InsertEvent", em.ctx, mock.Anything).Return(nil) + + m, err := em.MessageReceived(mdx, "peer1", b) assert.NoError(t, err) - b, _ := json.Marshal(&fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeMessage, - Message: msg, - Data: []*fftypes.Data{data}, - Group: &fftypes.Group{}, - }) + assert.NotEmpty(t, m) + + mdi.AssertExpectations(t) + mdx.AssertExpectations(t) +} +func TestMessageReceiveUnpinnedBatchConfirmMessagesFail(t *testing.T) { + em, cancel := newTestEventManager(t) + cancel() // to avoid infinite retry + + data := &fftypes.Data{ + ID: fftypes.NewUUID(), + Value: fftypes.JSONAnyPtr(`{}`), + } + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeNone, data) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} @@ -794,7 +708,10 @@ func TestMessageReceiveMessagePersistDataFail(t *testing.T) { mdi.On("GetOrganizationByIdentity", em.ctx, "0x12345").Return(&fftypes.Organization{ Identity: "0x12345", }, nil) - mdi.On("UpsertData", em.ctx, mock.Anything, database.UpsertOptimizationSkip).Return(fmt.Errorf("pop")) + mdi.On("UpsertBatch", em.ctx, mock.Anything).Return(nil, nil) + mdi.On("UpsertData", em.ctx, mock.Anything, database.UpsertOptimizationNew).Return(nil) + mdi.On("UpsertMessage", em.ctx, mock.Anything, database.UpsertOptimizationNew).Return(nil) + mdi.On("UpdateMessages", em.ctx, mock.Anything, mock.Anything).Return(fmt.Errorf("pop")) m, err := em.MessageReceived(mdx, "peer1", b) assert.Regexp(t, "FF10158", err) @@ -804,34 +721,15 @@ func TestMessageReceiveMessagePersistDataFail(t *testing.T) { mdx.AssertExpectations(t) } -func TestMessageReceiveMessagePersistEventFail(t *testing.T) { +func TestMessageReceiveUnpinnedBatchPersistEventFail(t *testing.T) { em, cancel := newTestEventManager(t) cancel() // to avoid infinite retry - msg := &fftypes.Message{ - Header: fftypes.MessageHeader{ - Identity: fftypes.Identity{ - Author: "signingOrg", - Key: "0x12345", - }, - ID: fftypes.NewUUID(), - TxType: fftypes.TransactionTypeNone, - }, - } data := &fftypes.Data{ ID: fftypes.NewUUID(), Value: fftypes.JSONAnyPtr(`{}`), } - err := msg.Seal(em.ctx) - assert.NoError(t, err) - err = data.Seal(em.ctx, nil) - assert.NoError(t, err) - b, _ := json.Marshal(&fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeMessage, - Message: msg, - Data: []*fftypes.Data{data}, - Group: &fftypes.Group{}, - }) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeNone, data) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} @@ -845,8 +743,10 @@ func TestMessageReceiveMessagePersistEventFail(t *testing.T) { mdi.On("GetOrganizationByIdentity", em.ctx, "0x12345").Return(&fftypes.Organization{ Identity: "0x12345", }, nil) - mdi.On("UpsertData", em.ctx, mock.Anything, database.UpsertOptimizationSkip).Return(nil) - mdi.On("UpsertMessage", em.ctx, mock.Anything, database.UpsertOptimizationSkip).Return(nil) + mdi.On("UpsertBatch", em.ctx, mock.Anything).Return(nil, nil) + mdi.On("UpsertData", em.ctx, mock.Anything, database.UpsertOptimizationNew).Return(nil) + mdi.On("UpsertMessage", em.ctx, mock.Anything, database.UpsertOptimizationNew).Return(nil) + mdi.On("UpdateMessages", em.ctx, mock.Anything, mock.Anything).Return(nil) mdi.On("InsertEvent", em.ctx, mock.Anything).Return(fmt.Errorf("pop")) m, err := em.MessageReceived(mdx, "peer1", b) @@ -861,30 +761,11 @@ func TestMessageReceiveMessageEnsureLocalGroupFail(t *testing.T) { em, cancel := newTestEventManager(t) cancel() // to avoid infinite retry - msg := &fftypes.Message{ - Header: fftypes.MessageHeader{ - Identity: fftypes.Identity{ - Author: "signingOrg", - Key: "0x12345", - }, - ID: fftypes.NewUUID(), - TxType: fftypes.TransactionTypeNone, - }, - } data := &fftypes.Data{ ID: fftypes.NewUUID(), Value: fftypes.JSONAnyPtr(`{}`), } - err := msg.Seal(em.ctx) - assert.NoError(t, err) - err = data.Seal(em.ctx, nil) - assert.NoError(t, err) - b, _ := json.Marshal(&fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeMessage, - Message: msg, - Data: []*fftypes.Data{data}, - Group: &fftypes.Group{}, - }) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeNone, data) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} @@ -893,7 +774,7 @@ func TestMessageReceiveMessageEnsureLocalGroupFail(t *testing.T) { msh.On("EnsureLocalGroup", em.ctx, mock.Anything).Return(false, fmt.Errorf("pop")) m, err := em.MessageReceived(mdx, "peer1", b) - assert.Regexp(t, "FF10158", err) + assert.Regexp(t, "pop", err) assert.Empty(t, m) mdi.AssertExpectations(t) @@ -904,30 +785,11 @@ func TestMessageReceiveMessageEnsureLocalGroupReject(t *testing.T) { em, cancel := newTestEventManager(t) cancel() // to avoid infinite retry - msg := &fftypes.Message{ - Header: fftypes.MessageHeader{ - Identity: fftypes.Identity{ - Author: "signingOrg", - Key: "0x12345", - }, - ID: fftypes.NewUUID(), - TxType: fftypes.TransactionTypeNone, - }, - } data := &fftypes.Data{ ID: fftypes.NewUUID(), Value: fftypes.JSONAnyPtr(`{}`), } - err := msg.Seal(em.ctx) - assert.NoError(t, err) - err = data.Seal(em.ctx, nil) - assert.NoError(t, err) - b, _ := json.Marshal(&fftypes.TransportWrapper{ - Type: fftypes.TransportPayloadTypeMessage, - Message: msg, - Data: []*fftypes.Data{data}, - Group: &fftypes.Group{}, - }) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeNone, data) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} diff --git a/internal/events/persist_batch.go b/internal/events/persist_batch.go index 09837d3b06..2ca191a020 100644 --- a/internal/events/persist_batch.go +++ b/internal/events/persist_batch.go @@ -96,7 +96,15 @@ func (em *eventManager) persistBatch(ctx context.Context /* db TX context*/, bat now := fftypes.Now() if batch.ID == nil || batch.Payload.TX.ID == nil { - l.Errorf("Invalid batch '%s'. Missing ID (%v) or transaction ID (%v)", batch.ID, batch.ID, batch.Payload.TX.ID) + l.Errorf("Invalid batch '%s'. Missing ID or transaction ID (%v)", batch.ID, batch.Payload.TX.ID) + return false, nil // This is not retryable. skip this batch + } + + switch batch.Payload.TX.Type { + case fftypes.TransactionTypeBatchPin: + case fftypes.TransactionTypeNone: + default: + l.Errorf("Invalid batch '%s'. Invalid transaction type: %s", batch.ID, batch.Payload.TX.Type) return false, nil // This is not retryable. skip this batch } diff --git a/internal/events/persist_batch_test.go b/internal/events/persist_batch_test.go index 65b439b433..e79ea2201a 100644 --- a/internal/events/persist_batch_test.go +++ b/internal/events/persist_batch_test.go @@ -37,7 +37,7 @@ func TestPersistBatchFromBroadcastRootOrg(t *testing.T) { mim.On("ResolveSigningKeyIdentity", em.ctx, mock.Anything).Return("", nil) mdi := em.database.(*databasemocks.Plugin) - mdi.On("UpsertBatch", em.ctx, mock.Anything, false).Return(fmt.Errorf(("pop"))) + mdi.On("UpsertBatch", em.ctx, mock.Anything).Return(fmt.Errorf(("pop"))) org := fftypes.Organization{ ID: fftypes.NewUUID(), diff --git a/internal/i18n/en_translations.go b/internal/i18n/en_translations.go index be8f5e85a6..6c3bbfcd22 100644 --- a/internal/i18n/en_translations.go +++ b/internal/i18n/en_translations.go @@ -260,4 +260,5 @@ var ( MsgAddressResolveBadStatus = ffm("FF10340", "Failed to resolve signing key string '%s' [%d]: %s", 500) MsgAddressResolveBadResData = ffm("FF10341", "Failed to resolve signing key string '%s' - invalid address returned '%s': %s", 500) MsgInvalidTXTypeForMessage = ffm("FF10343", "Invalid transaction type for sending a message: %s", 400) + MsgGroupRequired = ffm("FF10344", "Group must be set", 400) ) diff --git a/internal/privatemessaging/groupmanager.go b/internal/privatemessaging/groupmanager.go index 32ed6a41ab..f348d51986 100644 --- a/internal/privatemessaging/groupmanager.go +++ b/internal/privatemessaging/groupmanager.go @@ -49,6 +49,10 @@ type groupHashEntry struct { } func (gm *groupManager) EnsureLocalGroup(ctx context.Context, group *fftypes.Group) (ok bool, err error) { + if group == nil { + return false, i18n.NewError(ctx, i18n.MsgGroupRequired) + } + // In the case that we've received a private message for a group, it's possible (likely actually) // that the private message using the group will arrive before the group init message confirming // the group via the blockchain. diff --git a/internal/privatemessaging/groupmanager_test.go b/internal/privatemessaging/groupmanager_test.go index 5993dc01f9..69c5dc09b6 100644 --- a/internal/privatemessaging/groupmanager_test.go +++ b/internal/privatemessaging/groupmanager_test.go @@ -518,6 +518,15 @@ func TestEnsureLocalGroupNewOk(t *testing.T) { mdi.AssertExpectations(t) } +func TestEnsureLocalGroupNil(t *testing.T) { + pm, cancel := newTestPrivateMessaging(t) + defer cancel() + + ok, err := pm.EnsureLocalGroup(pm.ctx, nil) + assert.Regexp(t, "FF10344", err) + assert.False(t, ok) +} + func TestEnsureLocalGroupExistingOk(t *testing.T) { pm, cancel := newTestPrivateMessaging(t) defer cancel() From 9c553812aeee4d01ca42327fff9a465e2aa8a606 Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Thu, 10 Feb 2022 14:03:27 -0500 Subject: [PATCH 09/23] Tweaks Signed-off-by: Peter Broadhurst --- internal/batch/batch_manager.go | 30 ++++++++++++++++++------------ internal/events/dx_callbacks.go | 2 +- 2 files changed, 19 insertions(+), 13 deletions(-) diff --git a/internal/batch/batch_manager.go b/internal/batch/batch_manager.go index 7db740c7fe..79b1fb01ec 100644 --- a/internal/batch/batch_manager.go +++ b/internal/batch/batch_manager.go @@ -234,7 +234,7 @@ func (bm *batchManager) messageSequencer() { } // Wait to be woken again - if !batchWasFull { + if !batchWasFull && !bm.drainNewMessages() { if done := bm.waitForShoulderTapOrPollTimeout(); done { l.Debugf("Exiting: %s", err) return @@ -243,32 +243,38 @@ func (bm *batchManager) messageSequencer() { } } -func (bm *batchManager) waitForShoulderTapOrPollTimeout() (done bool) { - l := log.L(bm.ctx) +func (bm *batchManager) newMessageNotification(seq int64) { + log.L(bm.ctx).Debugf("Notification of message %d", seq) + if (seq - 1) < bm.readOffset { + bm.readOffset = seq - 1 + } +} - // Drain any new message notifications, moving back our - // readOffset as required +func (bm *batchManager) drainNewMessages() bool { + // Drain any new message notifications, moving back our readOffset as required newMessages := false checkingMessages := true for checkingMessages { select { case seq := <-bm.newMessages: - l.Debugf("Notification of message %d", seq) - if (seq - 1) < bm.readOffset { - bm.readOffset = seq - 1 - } + bm.newMessageNotification(seq) newMessages = true default: checkingMessages = false } } - if newMessages { - return false - } + return newMessages +} + +func (bm *batchManager) waitForShoulderTapOrPollTimeout() (done bool) { + l := log.L(bm.ctx) // Otherwise set a timeout timeout := time.NewTimer(bm.messagePollTimeout) select { + case seq := <-bm.newMessages: + bm.newMessageNotification(seq) + return false case <-timeout.C: l.Debugf("Woken after poll timeout") return false diff --git a/internal/events/dx_callbacks.go b/internal/events/dx_callbacks.go index 5332100d91..abb4ed6da6 100644 --- a/internal/events/dx_callbacks.go +++ b/internal/events/dx_callbacks.go @@ -271,7 +271,7 @@ func (em *eventManager) TransferResult(dx dataexchange.Plugin, trackingID string var operations []*fftypes.Operation fb := database.OperationQueryFactory.NewFilter(em.ctx) filter := fb.And( - fb.Eq("backendid", trackingID), + fb.Eq("id", trackingID), fb.Eq("plugin", dx.Name()), ) operations, _, err = em.database.GetOperations(em.ctx, filter) From 0c6912a2da20e61c184625b03d7a902efc92e783 Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Thu, 10 Feb 2022 14:19:08 -0500 Subject: [PATCH 10/23] Rename "none" to "unpinned" now we have a TX for them Signed-off-by: Peter Broadhurst --- docs/swagger/swagger.yaml | 3 +++ internal/batch/batch_processor.go | 2 +- internal/database/sqlcommon/batch_sql_test.go | 2 +- internal/database/sqlcommon/message_sql_test.go | 2 +- internal/events/dx_callbacks.go | 4 ++-- internal/events/dx_callbacks_test.go | 16 ++++++++-------- internal/events/persist_batch.go | 2 +- internal/privatemessaging/message.go | 7 ++++++- internal/privatemessaging/message_test.go | 16 ++++++++-------- internal/privatemessaging/privatemessaging.go | 4 ++-- .../privatemessaging/privatemessaging_test.go | 2 +- pkg/fftypes/message.go | 2 +- pkg/fftypes/message_test.go | 2 +- pkg/fftypes/transaction.go | 4 +++- test/e2e/onchain_offchain_test.go | 2 +- 15 files changed, 40 insertions(+), 30 deletions(-) diff --git a/docs/swagger/swagger.yaml b/docs/swagger/swagger.yaml index 50cc38371c..5bd4a0b924 100644 --- a/docs/swagger/swagger.yaml +++ b/docs/swagger/swagger.yaml @@ -4710,6 +4710,7 @@ paths: type: enum: - none + - unpinned - batch_pin - token_pool - token_transfer @@ -7820,6 +7821,7 @@ paths: type: enum: - none + - unpinned - batch_pin - token_pool - token_transfer @@ -7930,6 +7932,7 @@ paths: type: enum: - none + - unpinned - batch_pin - token_pool - token_transfer diff --git a/internal/batch/batch_processor.go b/internal/batch/batch_processor.go index 0d54fa344f..04da3a94aa 100644 --- a/internal/batch/batch_processor.go +++ b/internal/batch/batch_processor.go @@ -502,7 +502,7 @@ func (bp *batchProcessor) markMessagesDispatched(batch *fftypes.Batch) error { return err } - if bp.conf.txType == fftypes.TransactionTypeNone { + if bp.conf.txType == fftypes.TransactionTypeUnpinned { for _, msg := range batch.Payload.Messages { // Emit a confirmation event locally immediately event := fftypes.NewEvent(fftypes.EventTypeMessageConfirmed, batch.Namespace, msg.Header.ID) diff --git a/internal/database/sqlcommon/batch_sql_test.go b/internal/database/sqlcommon/batch_sql_test.go index 27633b8156..b82fd6f04b 100644 --- a/internal/database/sqlcommon/batch_sql_test.go +++ b/internal/database/sqlcommon/batch_sql_test.go @@ -54,7 +54,7 @@ func TestBatch2EWithDB(t *testing.T) { {Header: fftypes.MessageHeader{ID: msgID1}}, }, TX: fftypes.TransactionRef{ - Type: fftypes.TransactionTypeNone, + Type: fftypes.TransactionTypeUnpinned, }, }, } diff --git a/internal/database/sqlcommon/message_sql_test.go b/internal/database/sqlcommon/message_sql_test.go index ab00cf6565..72622e8a9c 100644 --- a/internal/database/sqlcommon/message_sql_test.go +++ b/internal/database/sqlcommon/message_sql_test.go @@ -58,7 +58,7 @@ func TestUpsertE2EWithDB(t *testing.T) { Topics: []string{"test1"}, Group: nil, DataHash: fftypes.NewRandB32(), - TxType: fftypes.TransactionTypeNone, + TxType: fftypes.TransactionTypeUnpinned, }, Hash: fftypes.NewRandB32(), State: fftypes.MessageStateStaged, diff --git a/internal/events/dx_callbacks.go b/internal/events/dx_callbacks.go index abb4ed6da6..4ac694dc33 100644 --- a/internal/events/dx_callbacks.go +++ b/internal/events/dx_callbacks.go @@ -45,7 +45,7 @@ func (em *eventManager) MessageReceived(dx dataexchange.Plugin, peerID string, d } l.Infof("Private batch received from '%s' (len=%d)", peerID, len(data)) - if wrapper.Batch.Payload.TX.Type == fftypes.TransactionTypeNone { + if wrapper.Batch.Payload.TX.Type == fftypes.TransactionTypeUnpinned { valid, err := em.definitions.EnsureLocalGroup(em.ctx, wrapper.Group) if err != nil { return "", err @@ -140,7 +140,7 @@ func (em *eventManager) privateBatchReceived(peerID string, batch *fftypes.Batch if batch.Payload.TX.Type == fftypes.TransactionTypeBatchPin { // Poke the aggregator to do its stuff em.aggregator.offchainBatches <- batch.ID - } else if batch.Payload.TX.Type == fftypes.TransactionTypeNone { + } else if batch.Payload.TX.Type == fftypes.TransactionTypeUnpinned { // We need to confirm all these messages immediately. if err := em.markUnpinnedMessagesConfirmed(ctx, batch); err != nil { return err diff --git a/internal/events/dx_callbacks_test.go b/internal/events/dx_callbacks_test.go index bac19b5230..1a5198155c 100644 --- a/internal/events/dx_callbacks_test.go +++ b/internal/events/dx_callbacks_test.go @@ -571,7 +571,7 @@ func TestMessageReceiveMessageIdentityIncorrect(t *testing.T) { em, cancel := newTestEventManager(t) cancel() // to avoid infinite retry - _, b := sampleBatchTransfer(t, fftypes.TransactionTypeNone) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeUnpinned) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} @@ -593,7 +593,7 @@ func TestMessageReceiveMessagePersistMessageFail(t *testing.T) { em, cancel := newTestEventManager(t) cancel() // to avoid infinite retry - _, b := sampleBatchTransfer(t, fftypes.TransactionTypeNone) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeUnpinned) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} @@ -626,7 +626,7 @@ func TestMessageReceiveMessagePersistDataFail(t *testing.T) { ID: fftypes.NewUUID(), Value: fftypes.JSONAnyPtr(`{}`), } - _, b := sampleBatchTransfer(t, fftypes.TransactionTypeNone, data) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeUnpinned, data) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} @@ -659,7 +659,7 @@ func TestMessageReceiveUnpinnedBatchOk(t *testing.T) { ID: fftypes.NewUUID(), Value: fftypes.JSONAnyPtr(`{}`), } - _, b := sampleBatchTransfer(t, fftypes.TransactionTypeNone, data) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeUnpinned, data) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} @@ -694,7 +694,7 @@ func TestMessageReceiveUnpinnedBatchConfirmMessagesFail(t *testing.T) { ID: fftypes.NewUUID(), Value: fftypes.JSONAnyPtr(`{}`), } - _, b := sampleBatchTransfer(t, fftypes.TransactionTypeNone, data) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeUnpinned, data) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} @@ -729,7 +729,7 @@ func TestMessageReceiveUnpinnedBatchPersistEventFail(t *testing.T) { ID: fftypes.NewUUID(), Value: fftypes.JSONAnyPtr(`{}`), } - _, b := sampleBatchTransfer(t, fftypes.TransactionTypeNone, data) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeUnpinned, data) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} @@ -765,7 +765,7 @@ func TestMessageReceiveMessageEnsureLocalGroupFail(t *testing.T) { ID: fftypes.NewUUID(), Value: fftypes.JSONAnyPtr(`{}`), } - _, b := sampleBatchTransfer(t, fftypes.TransactionTypeNone, data) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeUnpinned, data) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} @@ -789,7 +789,7 @@ func TestMessageReceiveMessageEnsureLocalGroupReject(t *testing.T) { ID: fftypes.NewUUID(), Value: fftypes.JSONAnyPtr(`{}`), } - _, b := sampleBatchTransfer(t, fftypes.TransactionTypeNone, data) + _, b := sampleBatchTransfer(t, fftypes.TransactionTypeUnpinned, data) mdi := em.database.(*databasemocks.Plugin) mdx := &dataexchangemocks.Plugin{} diff --git a/internal/events/persist_batch.go b/internal/events/persist_batch.go index 2ca191a020..174939a0a4 100644 --- a/internal/events/persist_batch.go +++ b/internal/events/persist_batch.go @@ -102,7 +102,7 @@ func (em *eventManager) persistBatch(ctx context.Context /* db TX context*/, bat switch batch.Payload.TX.Type { case fftypes.TransactionTypeBatchPin: - case fftypes.TransactionTypeNone: + case fftypes.TransactionTypeUnpinned: default: l.Errorf("Invalid batch '%s'. Invalid transaction type: %s", batch.ID, batch.Payload.TX.Type) return false, nil // This is not retryable. skip this batch diff --git a/internal/privatemessaging/message.go b/internal/privatemessaging/message.go index 0c56ff231e..1ffb657b4d 100644 --- a/internal/privatemessaging/message.go +++ b/internal/privatemessaging/message.go @@ -96,7 +96,12 @@ func (s *messageSender) setDefaults() { if s.msg.Header.Type == "" { s.msg.Header.Type = fftypes.MessageTypePrivate } - if s.msg.Header.TxType == "" { + switch s.msg.Header.TxType { + case fftypes.TransactionTypeUnpinned, fftypes.TransactionTypeNone: + // "unpinned" used to be called "none" (before we introduced batching + a TX on unppinned sends) + s.msg.Header.TxType = fftypes.TransactionTypeUnpinned + default: + // the only other valid option is "batch_pin" s.msg.Header.TxType = fftypes.TransactionTypeBatchPin } } diff --git a/internal/privatemessaging/message_test.go b/internal/privatemessaging/message_test.go index a9f3abbd7f..e48ad712a6 100644 --- a/internal/privatemessaging/message_test.go +++ b/internal/privatemessaging/message_test.go @@ -120,7 +120,7 @@ func TestSendUnpinnedMessageE2EOk(t *testing.T) { msg, err := pm.SendMessage(pm.ctx, "ns1", &fftypes.MessageInOut{ Message: fftypes.Message{ Header: fftypes.MessageHeader{ - TxType: fftypes.TransactionTypeNone, + TxType: fftypes.TransactionTypeUnpinned, Group: groupID, }, }, @@ -314,7 +314,7 @@ func TestSendUnpinnedMessageTooLarge(t *testing.T) { _, err := pm.SendMessage(pm.ctx, "ns1", &fftypes.MessageInOut{ Message: fftypes.Message{ Header: fftypes.MessageHeader{ - TxType: fftypes.TransactionTypeNone, + TxType: fftypes.TransactionTypeUnpinned, Group: groupID, }, }, @@ -426,7 +426,7 @@ func TestSendUnpinnedMessageGroupLookupFail(t *testing.T) { Identity: fftypes.Identity{ Author: "org1", }, - TxType: fftypes.TransactionTypeNone, + TxType: fftypes.TransactionTypeUnpinned, Group: groupID, }, }, @@ -464,7 +464,7 @@ func TestSendUnpinnedMessageInsertFail(t *testing.T) { _, err := pm.SendMessage(pm.ctx, "ns1", &fftypes.MessageInOut{ Message: fftypes.Message{ Header: fftypes.MessageHeader{ - TxType: fftypes.TransactionTypeNone, + TxType: fftypes.TransactionTypeUnpinned, Group: groupID, }, }, @@ -496,7 +496,7 @@ func TestSendUnpinnedMessageConfirmFail(t *testing.T) { _, err := pm.SendMessage(pm.ctx, "ns1", &fftypes.MessageInOut{ Message: fftypes.Message{ Header: fftypes.MessageHeader{ - TxType: fftypes.TransactionTypeNone, + TxType: fftypes.TransactionTypeUnpinned, Group: fftypes.NewRandB32(), }, }, @@ -528,7 +528,7 @@ func TestSendUnpinnedMessageResolveGroupFail(t *testing.T) { _, err := pm.SendMessage(pm.ctx, "ns1", &fftypes.MessageInOut{ Message: fftypes.Message{ Header: fftypes.MessageHeader{ - TxType: fftypes.TransactionTypeNone, + TxType: fftypes.TransactionTypeUnpinned, Group: groupID, }, }, @@ -567,7 +567,7 @@ func TestSendUnpinnedMessageResolveGroupNotFound(t *testing.T) { _, err := pm.SendMessage(pm.ctx, "ns1", &fftypes.MessageInOut{ Message: fftypes.Message{ Header: fftypes.MessageHeader{ - TxType: fftypes.TransactionTypeNone, + TxType: fftypes.TransactionTypeUnpinned, Group: groupID, }, }, @@ -747,7 +747,7 @@ func TestDispatchedUnpinnedMessageOK(t *testing.T) { Payload: fftypes.BatchPayload{ TX: fftypes.TransactionRef{ ID: fftypes.NewUUID(), - Type: fftypes.TransactionTypeNone, + Type: fftypes.TransactionTypeUnpinned, }, Messages: []*fftypes.Message{ { diff --git a/internal/privatemessaging/privatemessaging.go b/internal/privatemessaging/privatemessaging.go index 2d860d54a7..64e2a7f9a2 100644 --- a/internal/privatemessaging/privatemessaging.go +++ b/internal/privatemessaging/privatemessaging.go @@ -120,7 +120,7 @@ func NewPrivateMessaging(ctx context.Context, di database.Plugin, im identity.Ma pm.dispatchPinnedBatch, bo) ba.RegisterDispatcher(unpinnedPrivateDispatcherName, - fftypes.TransactionTypeNone, + fftypes.TransactionTypeUnpinned, []fftypes.MessageType{ fftypes.MessageTypePrivate, }, @@ -157,7 +157,7 @@ func (pm *privateMessaging) dispatchBatchCommon(ctx context.Context, batch *ffty return err } - if batch.Payload.TX.Type == fftypes.TransactionTypeNone { + if batch.Payload.TX.Type == fftypes.TransactionTypeUnpinned { // In the case of an un-pinned message we cannot be sure the group has been broadcast via the blockchain. // So we have to take the hit of sending it along with every message. tw.Group = group diff --git a/internal/privatemessaging/privatemessaging_test.go b/internal/privatemessaging/privatemessaging_test.go index e413303eb7..3c9c36d459 100644 --- a/internal/privatemessaging/privatemessaging_test.go +++ b/internal/privatemessaging/privatemessaging_test.go @@ -61,7 +61,7 @@ func newTestPrivateMessaging(t *testing.T) (*privateMessaging, func()) { mba.On("RegisterDispatcher", unpinnedPrivateDispatcherName, - fftypes.TransactionTypeNone, + fftypes.TransactionTypeUnpinned, []fftypes.MessageType{ fftypes.MessageTypePrivate, }, mock.Anything, mock.Anything).Return() diff --git a/pkg/fftypes/message.go b/pkg/fftypes/message.go index 135dd553ec..84f771d8cc 100644 --- a/pkg/fftypes/message.go +++ b/pkg/fftypes/message.go @@ -214,7 +214,7 @@ func (m *Message) DupDataCheck(ctx context.Context) (err error) { func (m *Message) Verify(ctx context.Context) error { switch m.Header.TxType { case TransactionTypeBatchPin: - case TransactionTypeNone: + case TransactionTypeUnpinned: default: return i18n.NewError(ctx, i18n.MsgInvalidTXTypeForMessage, m.Header.TxType) } diff --git a/pkg/fftypes/message_test.go b/pkg/fftypes/message_test.go index 1beb1f67e5..c08b7b2ae8 100644 --- a/pkg/fftypes/message_test.go +++ b/pkg/fftypes/message_test.go @@ -77,7 +77,7 @@ func TestVerifyTXType(t *testing.T) { err = msg.Verify(context.Background()) assert.NoError(t, err) - msg.Header.TxType = TransactionTypeNone + msg.Header.TxType = TransactionTypeUnpinned err = msg.Seal(context.Background()) assert.NoError(t, err) err = msg.Verify(context.Background()) diff --git a/pkg/fftypes/transaction.go b/pkg/fftypes/transaction.go index ccb140d079..d9d5bb2cf4 100644 --- a/pkg/fftypes/transaction.go +++ b/pkg/fftypes/transaction.go @@ -19,8 +19,10 @@ package fftypes type TransactionType = FFEnum var ( - // TransactionTypeNone indicates no transaction should be used for this message/batch + // TransactionTypeNone deprecreated - replaced by TransactionTypeUnpinned TransactionTypeNone TransactionType = ffEnum("txtype", "none") + // TransactionTypeUnpinned indicates no transaction should be used for this message/batch + TransactionTypeUnpinned TransactionType = ffEnum("txtype", "unpinned") // TransactionTypeBatchPin represents a pinning transaction, that verifies the originator of the data, and sequences the event deterministically between parties TransactionTypeBatchPin TransactionType = ffEnum("txtype", "batch_pin") // TransactionTypeTokenPool represents a token pool creation diff --git a/test/e2e/onchain_offchain_test.go b/test/e2e/onchain_offchain_test.go index 152c8fe996..91c80092c3 100644 --- a/test/e2e/onchain_offchain_test.go +++ b/test/e2e/onchain_offchain_test.go @@ -388,7 +388,7 @@ func (suite *OnChainOffChainTestSuite) TestE2EWebhookRequestReplyNoTx() { reply := RequestReply(suite.testState, suite.testState.client1, &data, []string{ suite.testState.org1.Name, suite.testState.org2.Name, - }, "myrequest", fftypes.TransactionTypeNone) + }, "myrequest", fftypes.TransactionTypeUnpinned) assert.NotNil(suite.T(), reply) bodyData := reply.InlineData[0].Value.JSONObject().GetString("body") From 4630c3d625672b38ce79e0ff2ae1f914be4de26e Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Thu, 10 Feb 2022 14:52:03 -0500 Subject: [PATCH 11/23] Correct the last pin index Signed-off-by: Peter Broadhurst --- internal/events/aggregator_batch_state.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/events/aggregator_batch_state.go b/internal/events/aggregator_batch_state.go index 917dd6c39e..1d1be34c50 100644 --- a/internal/events/aggregator_batch_state.go +++ b/internal/events/aggregator_batch_state.go @@ -214,7 +214,7 @@ func (bs *batchState) MarkMessageDispatched(ctx context.Context, batchID *fftype batchID: batchID, msgID: msg.Header.ID, firstPinIndex: msgBaseIndex, - lastPinIndex: msgBaseIndex + int64(len(msg.Header.Topics)), + lastPinIndex: msgBaseIndex + int64(len(msg.Header.Topics)) - 1, }) } From dfdb16759b0b3d42be95a521f7c4019b5f6c4f7a Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Thu, 10 Feb 2022 16:25:42 -0500 Subject: [PATCH 12/23] Close out on UTs Signed-off-by: Peter Broadhurst --- internal/batch/batch_manager.go | 13 ++- internal/batch/batch_manager_test.go | 122 +++++++++++++++++------- internal/batch/batch_processor.go | 22 +++-- internal/batch/batch_processor_test.go | 125 ++++++++++++++++++++++++- 4 files changed, 235 insertions(+), 47 deletions(-) diff --git a/internal/batch/batch_manager.go b/internal/batch/batch_manager.go index 79b1fb01ec..2bbc787bac 100644 --- a/internal/batch/batch_manager.go +++ b/internal/batch/batch_manager.go @@ -301,8 +301,7 @@ func (bm *batchManager) dispatchMessage(msg *fftypes.Message, data ...*fftypes.D func (bm *batchManager) reapQuiescing() { bm.dispatcherMux.Lock() - defer bm.dispatcherMux.Unlock() - + var reaped []*batchProcessor for _, d := range bm.dispatchers { for k, p := range d.processors { select { @@ -310,10 +309,20 @@ func (bm *batchManager) reapQuiescing() { // This is called on the goroutine where we dispatch the work, so it's safe to cleanup delete(d.processors, k) close(p.newWork) + reaped = append(reaped, p) default: } } } + bm.dispatcherMux.Unlock() + + for _, p := range reaped { + // We wait for the current process to close, which should be immediate, but there is a tiny + // chance that we dispatched one last message to it just as it was quiescing. + // If that's the case, we don't want to spin up a new one, until we've finished the dispatch + // of that piece of work that snuck in. + <-p.done + } } func (bm *batchManager) getProcessors() []*batchProcessor { diff --git a/internal/batch/batch_manager_test.go b/internal/batch/batch_manager_test.go index 345a404327..40a4bb4d89 100644 --- a/internal/batch/batch_manager_test.go +++ b/internal/batch/batch_manager_test.go @@ -75,7 +75,7 @@ func TestE2EDispatchBroadcast(t *testing.T) { bm.RegisterDispatcher("utdispatcher", fftypes.TransactionTypeBatchPin, []fftypes.MessageType{fftypes.MessageTypeBroadcast}, handler, DispatcherOptions{ BatchMaxSize: 2, BatchTimeout: 0, - DisposeTimeout: 120 * time.Second, + DisposeTimeout: 10 * time.Millisecond, }) dataID1 := fftypes.NewUUID() @@ -123,16 +123,29 @@ func TestE2EDispatchBroadcast(t *testing.T) { bm.NewMessages() <- msg.Sequence readyForDispatch <- true + + // Check the status while we know there's a flush going on + status := bm.Status() + assert.NotNil(t, status[0].Status.Flushing) + b := <-waitForDispatch assert.Equal(t, *msg.Header.ID, *b.Payload.Messages[0].Header.ID) assert.Equal(t, *data.ID, *b.Payload.Data[0].ID) + close(readyForDispatch) + + // Wait for the reaping + for len(bm.getProcessors()) > 0 { + time.Sleep(1 * time.Millisecond) + bm.NewMessages() <- msg.Sequence + } + cancel() bm.WaitStop() } -func TestE2EDispatchPrivate(t *testing.T) { +func TestE2EDispatchPrivateUnpinned(t *testing.T) { log.SetLevel("debug") config.Reset() @@ -242,6 +255,30 @@ func TestE2EDispatchPrivate(t *testing.T) { bm.WaitStop() } + +func TestDispatchUnknownType(t *testing.T) { + log.SetLevel("debug") + config.Reset() + + mdi := &databasemocks.Plugin{} + mdm := &datamocks.Manager{} + mni := &sysmessagingmocks.LocalNodeInfo{} + ctx, cancel := context.WithCancel(context.Background()) + bmi, _ := NewBatchManager(ctx, mni, mdi, mdm) + bm := bmi.(*batchManager) + + msg := &fftypes.Message{} + mdi.On("GetMessages", mock.Anything, mock.Anything).Return([]*fftypes.Message{msg}, nil, nil).Once() + mdm.On("GetMessageData", mock.Anything, mock.Anything, true).Return([]*fftypes.Data{}, true, nil) + + err := bm.Start() + assert.NoError(t, err) + + cancel() + bm.WaitStop() + +} + func TestInitFailNoPersistence(t *testing.T) { _, err := NewBatchManager(context.Background(), nil, nil, nil) assert.Error(t, err) @@ -306,39 +343,6 @@ func TestMessageSequencerMissingMessageData(t *testing.T) { mdm.AssertExpectations(t) } -func TestMessageSequencerDispatchFail(t *testing.T) { - mdi := &databasemocks.Plugin{} - mdm := &datamocks.Manager{} - mni := &sysmessagingmocks.LocalNodeInfo{} - bm, _ := NewBatchManager(context.Background(), mni, mdi, mdm) - - dataID := fftypes.NewUUID() - mdi.On("GetMessages", mock.Anything, mock.Anything, mock.Anything). - Return([]*fftypes.Message{ - { - Header: fftypes.MessageHeader{ - ID: fftypes.NewUUID(), - Namespace: "ns1", - }, - Data: []*fftypes.DataRef{ - {ID: dataID}, - }}, - }, nil, nil). - Run(func(args mock.Arguments) { - bm.Close() - }). - Once() - mdi.On("GetMessages", mock.Anything, mock.Anything, mock.Anything).Return([]*fftypes.Message{}, nil, nil) - mdm.On("GetMessageData", mock.Anything, mock.Anything, true).Return([]*fftypes.Data{{ID: dataID}}, true, nil) - - bm.(*batchManager).messageSequencer() - - bm.WaitStop() - - mdi.AssertExpectations(t) - mdm.AssertExpectations(t) -} - func TestMessageSequencerUpdateMessagesFail(t *testing.T) { mdi := &databasemocks.Plugin{} mdm := &datamocks.Manager{} @@ -389,6 +393,42 @@ func TestMessageSequencerUpdateMessagesFail(t *testing.T) { mdm.AssertExpectations(t) } +func TestMessageSequencerDispatchFail(t *testing.T) { + mdi := &databasemocks.Plugin{} + mdm := &datamocks.Manager{} + mni := &sysmessagingmocks.LocalNodeInfo{} + mni.On("GetNodeUUID", mock.Anything).Return(fftypes.NewUUID()) + ctx, cancelCtx := context.WithCancel(context.Background()) + bm, _ := NewBatchManager(ctx, mni, mdi, mdm) + bm.RegisterDispatcher("utdispatcher", fftypes.TransactionTypeBatchPin, []fftypes.MessageType{fftypes.MessageTypeBroadcast}, func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { + cancelCtx() + return fmt.Errorf("fizzle") + }, DispatcherOptions{BatchMaxSize: 1, DisposeTimeout: 0}) + + dataID := fftypes.NewUUID() + mdi.On("GetMessages", mock.Anything, mock.Anything, mock.Anything).Return([]*fftypes.Message{ + { + Header: fftypes.MessageHeader{ + ID: fftypes.NewUUID(), + TxType: fftypes.TransactionTypeBatchPin, + Type: fftypes.MessageTypeBroadcast, + Namespace: "ns1", + }, + Data: []*fftypes.DataRef{ + {ID: dataID}, + }}, + }, nil, nil) + mdm.On("GetMessageData", mock.Anything, mock.Anything, true).Return([]*fftypes.Data{{ID: dataID}}, true, nil) + mdi.On("RunAsGroup", mock.Anything, mock.Anything, mock.Anything).Return(nil) + + bm.(*batchManager).messageSequencer() + + bm.Close() + bm.WaitStop() + + mdi.AssertExpectations(t) + mdm.AssertExpectations(t) +} func TestMessageSequencerUpdateBatchFail(t *testing.T) { mdi := &databasemocks.Plugin{} mdm := &datamocks.Manager{} @@ -447,6 +487,18 @@ func TestWaitForPollTimeout(t *testing.T) { bm.(*batchManager).waitForShoulderTapOrPollTimeout() } +func TestWaitForNewMessage(t *testing.T) { + mdi := &databasemocks.Plugin{} + mdm := &datamocks.Manager{} + mni := &sysmessagingmocks.LocalNodeInfo{} + bmi, _ := NewBatchManager(context.Background(), mni, mdi, mdm) + bm := bmi.(*batchManager) + bm.readOffset = 22222 + bm.NewMessages() <- 12345 + bm.waitForShoulderTapOrPollTimeout() + assert.Equal(t, int64(12344), bm.readOffset) +} + func TestAssembleMessageDataNilData(t *testing.T) { mdi := &databasemocks.Plugin{} mdm := &datamocks.Manager{} diff --git a/internal/batch/batch_processor.go b/internal/batch/batch_processor.go index 04da3a94aa..f3ba850e8b 100644 --- a/internal/batch/batch_processor.go +++ b/internal/batch/batch_processor.go @@ -220,6 +220,7 @@ func (bp *batchProcessor) startFlush(overflow bool) (id *fftypes.UUID, flushAsse for i := 0; i < newFlushedSeqLen-len(flushAssembly); i++ { newFlushedSequnces[i+len(flushAssembly)] = bp.flushedSequences[i] } + bp.flushedSequences = newFlushedSequnces // Cycle to the next assembly id = bp.assemblyID byteSize = bp.assemblyQueueBytes @@ -262,6 +263,17 @@ func (bp *batchProcessor) captureFlushError(err error) { fs.LastFlushError = err.Error() } +func (bp *batchProcessor) startQuiesce() { + // We are ready to quiesce, but we can't safely close our input channel. + // We just do a non-blocking pass (queue length is 1) to the manager to + // ask them to close our channel before their next read. + // One more item of work might get through the pipe in an edge case here. + select { + case bp.quescing <- true: + default: + } +} + // The assemblyLoop receives new work, sorts it, and waits for the size/timer to pop before // flushing the batch. The newWork channel has up to one batch of slots queue length, // so that we can have one batch of work queuing for assembly, while we have one batch flushing. @@ -283,13 +295,7 @@ func (bp *batchProcessor) assemblyLoop() { case <-batchTimeout.C: l.Errorf("Batch timer popped") if len(bp.assemblyQueue) == 0 { - // It probably makes sense to exit, but we don't know if more work is coming our way, - // so we just inform the manager we've hit our dispose timeout and see if they close - // the input work channel (which they can safely do on the dispatcher routine) - select { - case bp.quescing <- true: - default: - } + bp.startQuiesce() } else { // We need to flush timedout = true @@ -307,7 +313,7 @@ func (bp *batchProcessor) assemblyLoop() { } } } - if full || timedout || quescing { + if (full || timedout || quescing) && len(bp.assemblyQueue) > 0 { // Let Go GC the old timer _ = batchTimeout.Stop() diff --git a/internal/batch/batch_processor_test.go b/internal/batch/batch_processor_test.go index cec858aa96..de5655d508 100644 --- a/internal/batch/batch_processor_test.go +++ b/internal/batch/batch_processor_test.go @@ -190,11 +190,13 @@ func TestCalcPinsFail(t *testing.T) { _, bp := newTestBatchProcessor(func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { return nil }) + bp.cancelCtx() mdi := bp.database.(*databasemocks.Plugin) mdi.On("UpsertNonceNext", mock.Anything, mock.Anything).Return(fmt.Errorf("pop")) + mockRunAsGroupPassthrough(mdi) gid := fftypes.NewRandB32() - _, err := bp.maskContexts(bp.ctx, &fftypes.Batch{ + _, err := bp.persistBatch(&fftypes.Batch{ Group: gid, Payload: fftypes.BatchPayload{ Messages: []*fftypes.Message{ @@ -205,8 +207,127 @@ func TestCalcPinsFail(t *testing.T) { }, }, }) - assert.Regexp(t, "pop", err) + assert.Regexp(t, "FF10158", err) + + <-bp.done + + mdi.AssertExpectations(t) +} + +func TestAddWorkInRecentlyFlushed(t *testing.T) { + _, bp := newTestBatchProcessor(func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { + return nil + }) + bp.flushedSequences = []int64{100, 500, 400, 900, 200, 700} + _, _ = bp.addWork(&batchWork{ + msg: &fftypes.Message{ + Sequence: 200, + }, + }) + assert.Empty(t, bp.assemblyQueue) + +} + +func TestAddWorkInSortDeDup(t *testing.T) { + _, bp := newTestBatchProcessor(func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { + return nil + }) + bp.assemblyQueue = []*batchWork{ + {msg: &fftypes.Message{Sequence: 200}}, + {msg: &fftypes.Message{Sequence: 201}}, + {msg: &fftypes.Message{Sequence: 202}}, + {msg: &fftypes.Message{Sequence: 204}}, + } + _, _ = bp.addWork(&batchWork{ + msg: &fftypes.Message{Sequence: 200}, + }) + _, _ = bp.addWork(&batchWork{ + msg: &fftypes.Message{Sequence: 203}, + }) + assert.Equal(t, []*batchWork{ + {msg: &fftypes.Message{Sequence: 200}}, + {msg: &fftypes.Message{Sequence: 201}}, + {msg: &fftypes.Message{Sequence: 202}}, + {msg: &fftypes.Message{Sequence: 203}}, + {msg: &fftypes.Message{Sequence: 204}}, + }, bp.assemblyQueue) +} + +func TestStartFlushOverflow(t *testing.T) { + _, bp := newTestBatchProcessor(func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { + return nil + }) + batchID := fftypes.NewUUID() + bp.assemblyID = batchID + bp.flushedSequences = []int64{100, 101, 102, 103, 104} + bp.assemblyQueue = []*batchWork{ + {msg: &fftypes.Message{Sequence: 200}}, + {msg: &fftypes.Message{Sequence: 201}}, + {msg: &fftypes.Message{Sequence: 202}}, + {msg: &fftypes.Message{Sequence: 203}}, + } + bp.conf.BatchMaxSize = 3 + + flushBatchID, flushAssembly, _ := bp.startFlush(true) + assert.Equal(t, batchID, flushBatchID) + assert.Equal(t, []int64{200, 201, 202, 100, 101, 102}, bp.flushedSequences) + assert.Equal(t, []*batchWork{ + {msg: &fftypes.Message{Sequence: 200}}, + {msg: &fftypes.Message{Sequence: 201}}, + {msg: &fftypes.Message{Sequence: 202}}, + }, flushAssembly) + assert.Equal(t, []*batchWork{ + {msg: &fftypes.Message{Sequence: 203}}, + }, bp.assemblyQueue) + assert.NotEqual(t, batchID, bp.assemblyID) +} + +func TestStartQuiesceNonBlocking(t *testing.T) { + _, bp := newTestBatchProcessor(func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { + return nil + }) + bp.startQuiesce() + bp.startQuiesce() // we're just checking this doesn't hang +} + +func TestMarkMessageDispatchedUnpinnedOK(t *testing.T) { + log.SetLevel("debug") + config.Reset() + + dispatched := make(chan *fftypes.Batch) + mdi, bp := newTestBatchProcessor(func(c context.Context, b *fftypes.Batch, s []*fftypes.Bytes32) error { + dispatched <- b + return nil + }) + bp.conf.txType = fftypes.TransactionTypeUnpinned + + mockRunAsGroupPassthrough(mdi) + mdi.On("UpdateMessages", mock.Anything, mock.Anything, mock.Anything).Return(nil) + mdi.On("UpsertBatch", mock.Anything, mock.Anything, mock.Anything).Return(nil) + mdi.On("InsertEvent", mock.Anything, mock.Anything).Return(fmt.Errorf("pop")).Once() + mdi.On("InsertEvent", mock.Anything, mock.Anything).Return(nil) + + mth := bp.txHelper.(*txcommonmocks.Helper) + mth.On("SubmitNewTransaction", mock.Anything, "ns1", fftypes.TransactionTypeUnpinned).Return(fftypes.NewUUID(), nil) + + // Dispatch the work + go func() { + for i := 0; i < 5; i++ { + msgid := fftypes.NewUUID() + bp.newWork <- &batchWork{ + msg: &fftypes.Message{Header: fftypes.MessageHeader{ID: msgid}, Sequence: int64(1000 + i)}, + } + } + }() + + // Wait for the confirmations, and the dispatch + batch := <-dispatched + + // Check we got all the messages in a single batch + assert.Equal(t, 5, len(batch.Payload.Messages)) bp.cancelCtx() <-bp.done + + mdi.AssertExpectations(t) } From 1d3cc7d1de4f9bb61b9701595c92d1f076363f17 Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Fri, 11 Feb 2022 08:32:02 -0500 Subject: [PATCH 13/23] Add exclusive table lock for event emission to ensure event detection with increasing sequence Signed-off-by: Peter Broadhurst --- internal/database/postgres/postgres.go | 4 +++ internal/database/postgres/postgres_test.go | 1 + internal/database/sqlcommon/event_sql.go | 10 ++++++- internal/database/sqlcommon/event_sql_test.go | 14 ++++++++++ internal/database/sqlcommon/provider.go | 5 ++-- .../database/sqlcommon/provider_mock_test.go | 4 +++ internal/database/sqlcommon/sqlcommon.go | 27 +++++++++++++++++++ internal/database/sqlcommon/sqlcommon_test.go | 13 +++++++++ internal/i18n/en_translations.go | 1 + 9 files changed, 76 insertions(+), 3 deletions(-) diff --git a/internal/database/postgres/postgres.go b/internal/database/postgres/postgres.go index 4e88256135..e4ae1a9463 100644 --- a/internal/database/postgres/postgres.go +++ b/internal/database/postgres/postgres.go @@ -18,6 +18,7 @@ package postgres import ( "context" + "fmt" "database/sql" @@ -53,6 +54,9 @@ func (psql *Postgres) Features() sqlcommon.SQLFeatures { features := sqlcommon.DefaultSQLProviderFeatures() features.PlaceholderFormat = sq.Dollar features.UseILIKE = false // slower than lower() + features.ExclusiveTableLockSQL = func(table string) string { + return fmt.Sprintf(`LOCK TABLE "%s" IN EXCLUSIVE MODE;`, table) + } return features } diff --git a/internal/database/postgres/postgres_test.go b/internal/database/postgres/postgres_test.go index 67be063596..2f8f50b4f5 100644 --- a/internal/database/postgres/postgres_test.go +++ b/internal/database/postgres/postgres_test.go @@ -40,6 +40,7 @@ func TestPostgresProvider(t *testing.T) { assert.Equal(t, "postgres", psql.Name()) assert.Equal(t, sq.Dollar, psql.Features().PlaceholderFormat) + assert.Equal(t, `LOCK TABLE "events" IN EXCLUSIVE MODE;`, psql.Features().ExclusiveTableLockSQL("events")) insert := sq.Insert("test").Columns("col1").Values("val1") insert, query := psql.UpdateInsertForSequenceReturn(insert) diff --git a/internal/database/sqlcommon/event_sql.go b/internal/database/sqlcommon/event_sql.go index a653e89b52..49940195fa 100644 --- a/internal/database/sqlcommon/event_sql.go +++ b/internal/database/sqlcommon/event_sql.go @@ -1,4 +1,4 @@ -// Copyright © 2021 Kaleido, Inc. +// Copyright © 2022 Kaleido, Inc. // // SPDX-License-Identifier: Apache-2.0 // @@ -49,6 +49,14 @@ func (s *SQLCommon) InsertEvent(ctx context.Context, event *fftypes.Event) (err } defer s.rollbackTx(ctx, tx, autoCommit) + // This is special to events - we take the cost of a full table lock on the events table, so + // that nobody can add rows that increment the sequence, until our transaction has committed. + // This allows us to rely on the sequence to always be increasing, even when writing events + // concurrently (it does not guarantee we won't get a gap in the sequences). + if err = s.lockTableExclusiveTx(ctx, tx, "events"); err != nil { + return err + } + event.Sequence, err = s.insertTx(ctx, tx, sq.Insert("events"). Columns(eventColumns...). diff --git a/internal/database/sqlcommon/event_sql_test.go b/internal/database/sqlcommon/event_sql_test.go index 178a3dcd29..9f428b6f67 100644 --- a/internal/database/sqlcommon/event_sql_test.go +++ b/internal/database/sqlcommon/event_sql_test.go @@ -18,6 +18,7 @@ package sqlcommon import ( "context" + "database/sql/driver" "encoding/json" "fmt" "testing" @@ -109,9 +110,21 @@ func TestInsertEventFailBegin(t *testing.T) { assert.NoError(t, mock.ExpectationsWereMet()) } +func TestInsertEventFailLock(t *testing.T) { + s, mock := newMockProvider().init() + mock.ExpectBegin() + mock.ExpectExec("LOCK .*").WillReturnError(fmt.Errorf("pop")) + mock.ExpectRollback() + eventID := fftypes.NewUUID() + err := s.InsertEvent(context.Background(), &fftypes.Event{ID: eventID}) + assert.Regexp(t, "FF10345", err) + assert.NoError(t, mock.ExpectationsWereMet()) +} + func TestInsertEventFailInsert(t *testing.T) { s, mock := newMockProvider().init() mock.ExpectBegin() + mock.ExpectExec("LOCK .*").WillReturnResult(driver.ResultNoRows) mock.ExpectExec("INSERT .*").WillReturnError(fmt.Errorf("pop")) mock.ExpectRollback() eventID := fftypes.NewUUID() @@ -124,6 +137,7 @@ func TestInsertEventFailCommit(t *testing.T) { s, mock := newMockProvider().init() eventID := fftypes.NewUUID() mock.ExpectBegin() + mock.ExpectExec("LOCK .*").WillReturnResult(driver.ResultNoRows) mock.ExpectExec("INSERT .*").WillReturnResult(sqlmock.NewResult(1, 1)) mock.ExpectCommit().WillReturnError(fmt.Errorf("pop")) err := s.InsertEvent(context.Background(), &fftypes.Event{ID: eventID}) diff --git a/internal/database/sqlcommon/provider.go b/internal/database/sqlcommon/provider.go index 092afc4eaa..905e6c2215 100644 --- a/internal/database/sqlcommon/provider.go +++ b/internal/database/sqlcommon/provider.go @@ -28,8 +28,9 @@ const ( ) type SQLFeatures struct { - UseILIKE bool - PlaceholderFormat sq.PlaceholderFormat + UseILIKE bool + PlaceholderFormat sq.PlaceholderFormat + ExclusiveTableLockSQL func(table string) string } func DefaultSQLProviderFeatures() SQLFeatures { diff --git a/internal/database/sqlcommon/provider_mock_test.go b/internal/database/sqlcommon/provider_mock_test.go index 8c3ac4a9e3..6c7c58523d 100644 --- a/internal/database/sqlcommon/provider_mock_test.go +++ b/internal/database/sqlcommon/provider_mock_test.go @@ -19,6 +19,7 @@ package sqlcommon import ( "context" "database/sql" + "fmt" "github.com/DATA-DOG/go-sqlmock" sq "github.com/Masterminds/squirrel" @@ -70,6 +71,9 @@ func (mp *mockProvider) MigrationsDir() string { func (psql *mockProvider) Features() SQLFeatures { features := DefaultSQLProviderFeatures() features.UseILIKE = true + features.ExclusiveTableLockSQL = func(table string) string { + return fmt.Sprintf(`LOCK TABLE "%s" IN EXCLUSIVE MODE;`, table) + } return features } diff --git a/internal/database/sqlcommon/sqlcommon.go b/internal/database/sqlcommon/sqlcommon.go index d0e85c3761..fb61395c17 100644 --- a/internal/database/sqlcommon/sqlcommon.go +++ b/internal/database/sqlcommon/sqlcommon.go @@ -46,6 +46,7 @@ type txContextKey struct{} type txWrapper struct { sqlTX *sql.Tx postCommit []func() + tableLocks []string } func (s *SQLCommon) Init(ctx context.Context, provider Provider, prefix config.Prefix, callbacks database.Callbacks, capabilities *database.Capabilities) (err error) { @@ -317,6 +318,32 @@ func (s *SQLCommon) postCommitEvent(tx *txWrapper, fn func()) { tx.postCommit = append(tx.postCommit, fn) } +func (tx *txWrapper) tableIsLocked(table string) bool { + for _, t := range tx.tableLocks { + if t == table { + return true + } + } + return false +} + +func (s *SQLCommon) lockTableExclusiveTx(ctx context.Context, tx *txWrapper, table string) error { + l := log.L(ctx) + if s.features.ExclusiveTableLockSQL != nil && !tx.tableIsLocked(table) { + sqlQuery := s.features.ExclusiveTableLockSQL(table) + + l.Debugf(`SQL-> lock: %s`, sqlQuery) + _, err := tx.sqlTX.ExecContext(ctx, sqlQuery) + if err != nil { + l.Errorf(`SQL lock failed: %s sql=[ %s ]`, err, sqlQuery) + return i18n.WrapError(ctx, err, i18n.MsgDBLockFailed) + } + tx.tableLocks = append(tx.tableLocks, table) + l.Debugf(`SQL<- lock %s`, table) + } + return nil +} + // rollbackTx be safely called as a defer, as it is a cheap no-op if the transaction is complete func (s *SQLCommon) rollbackTx(ctx context.Context, tx *txWrapper, autoCommit bool) { if autoCommit { diff --git a/internal/database/sqlcommon/sqlcommon_test.go b/internal/database/sqlcommon/sqlcommon_test.go index 246c735b0c..da73481c22 100644 --- a/internal/database/sqlcommon/sqlcommon_test.go +++ b/internal/database/sqlcommon/sqlcommon_test.go @@ -308,3 +308,16 @@ func TestQueryResSwallowError(t *testing.T) { }) assert.Equal(t, int64(-1), *res.TotalCount) } + +func TestDoubleLock(t *testing.T) { + s, mdb := newMockProvider().init() + mdb.ExpectBegin() + mdb.ExpectExec("LOCK .*").WillReturnResult(driver.ResultNoRows) + ctx, tx, _, err := s.beginOrUseTx(context.Background()) + assert.NoError(t, err) + err = s.lockTableExclusiveTx(ctx, tx, "table1") + assert.NoError(t, err) + err = s.lockTableExclusiveTx(ctx, tx, "table1") + assert.NoError(t, err) + assert.NoError(t, mdb.ExpectationsWereMet()) +} diff --git a/internal/i18n/en_translations.go b/internal/i18n/en_translations.go index 6c3bbfcd22..da753d466e 100644 --- a/internal/i18n/en_translations.go +++ b/internal/i18n/en_translations.go @@ -261,4 +261,5 @@ var ( MsgAddressResolveBadResData = ffm("FF10341", "Failed to resolve signing key string '%s' - invalid address returned '%s': %s", 500) MsgInvalidTXTypeForMessage = ffm("FF10343", "Invalid transaction type for sending a message: %s", 400) MsgGroupRequired = ffm("FF10344", "Group must be set", 400) + MsgDBLockFailed = ffm("FF10345", "Database lock failed") ) From 0108f36b9c7776b1717425949c9827f75d16520a Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Fri, 11 Feb 2022 11:31:23 -0500 Subject: [PATCH 14/23] Correct type of batch timer popped message Signed-off-by: Peter Broadhurst --- internal/batch/batch_processor.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/internal/batch/batch_processor.go b/internal/batch/batch_processor.go index f3ba850e8b..4a4f1408e8 100644 --- a/internal/batch/batch_processor.go +++ b/internal/batch/batch_processor.go @@ -293,7 +293,7 @@ func (bp *batchProcessor) assemblyLoop() { _ = batchTimeout.Stop() return case <-batchTimeout.C: - l.Errorf("Batch timer popped") + l.Debugf("Batch timer popped") if len(bp.assemblyQueue) == 0 { bp.startQuiesce() } else { From 5dd72b7959683c8e93dc01b17e9709545eb645a9 Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Fri, 11 Feb 2022 11:57:20 -0500 Subject: [PATCH 15/23] Review comments Signed-off-by: Peter Broadhurst --- internal/batch/batch_manager.go | 17 ++++++++++++----- internal/batch/batch_manager_test.go | 4 ++-- internal/batch/batch_processor.go | 2 +- pkg/fftypes/transaction.go | 2 +- 4 files changed, 16 insertions(+), 9 deletions(-) diff --git a/internal/batch/batch_manager.go b/internal/batch/batch_manager.go index 2bbc787bac..2bfb940132 100644 --- a/internal/batch/batch_manager.go +++ b/internal/batch/batch_manager.go @@ -108,6 +108,10 @@ type dispatcher struct { options DispatcherOptions } +func (bm *batchManager) getProcessorKey(namespace string, identity *fftypes.Identity, groupID *fftypes.Bytes32) string { + return fmt.Sprintf("%s|%s|%v", namespace, identity.Author, groupID) +} + func (bm *batchManager) getDispatcherKey(txType fftypes.TransactionType, msgType fftypes.MessageType) string { return fmt.Sprintf("tx:%s/%s", txType, msgType) } @@ -142,7 +146,7 @@ func (bm *batchManager) getProcessor(txType fftypes.TransactionType, msgType fft if !ok { return nil, i18n.NewError(bm.ctx, i18n.MsgUnregisteredBatchType, dispatcherKey) } - name := fmt.Sprintf("%s|%s|%v", namespace, identity.Author, group) + name := bm.getProcessorKey(namespace, identity, group) processor, ok := dispatcher.processors[name] if !ok { processor = newBatchProcessor( @@ -235,7 +239,7 @@ func (bm *batchManager) messageSequencer() { // Wait to be woken again if !batchWasFull && !bm.drainNewMessages() { - if done := bm.waitForShoulderTapOrPollTimeout(); done { + if done := bm.waitForNewMessages(); done { l.Debugf("Exiting: %s", err) return } @@ -245,8 +249,11 @@ func (bm *batchManager) messageSequencer() { func (bm *batchManager) newMessageNotification(seq int64) { log.L(bm.ctx).Debugf("Notification of message %d", seq) - if (seq - 1) < bm.readOffset { - bm.readOffset = seq - 1 + // The readOffset is the last sequence we have already read. + // So we need to ensure it is at least one earlier, than this message sequence + lastSequenceBeforeMsg := seq - 1 + if lastSequenceBeforeMsg < bm.readOffset { + bm.readOffset = lastSequenceBeforeMsg } } @@ -266,7 +273,7 @@ func (bm *batchManager) drainNewMessages() bool { return newMessages } -func (bm *batchManager) waitForShoulderTapOrPollTimeout() (done bool) { +func (bm *batchManager) waitForNewMessages() (done bool) { l := log.L(bm.ctx) // Otherwise set a timeout diff --git a/internal/batch/batch_manager_test.go b/internal/batch/batch_manager_test.go index 40a4bb4d89..fc6793aeeb 100644 --- a/internal/batch/batch_manager_test.go +++ b/internal/batch/batch_manager_test.go @@ -484,7 +484,7 @@ func TestWaitForPollTimeout(t *testing.T) { mni := &sysmessagingmocks.LocalNodeInfo{} bm, _ := NewBatchManager(context.Background(), mni, mdi, mdm) bm.(*batchManager).messagePollTimeout = 1 * time.Microsecond - bm.(*batchManager).waitForShoulderTapOrPollTimeout() + bm.(*batchManager).waitForNewMessages() } func TestWaitForNewMessage(t *testing.T) { @@ -495,7 +495,7 @@ func TestWaitForNewMessage(t *testing.T) { bm := bmi.(*batchManager) bm.readOffset = 22222 bm.NewMessages() <- 12345 - bm.waitForShoulderTapOrPollTimeout() + bm.waitForNewMessages() assert.Equal(t, int64(12344), bm.readOffset) } diff --git a/internal/batch/batch_processor.go b/internal/batch/batch_processor.go index 4a4f1408e8..491bf81dd1 100644 --- a/internal/batch/batch_processor.go +++ b/internal/batch/batch_processor.go @@ -155,7 +155,7 @@ func (bp *batchProcessor) addWork(newWork *batchWork) (full, overflow bool) { newQueue := make([]*batchWork, 0, len(bp.assemblyQueue)+1) added := false skip := false - // Check it's not in the recently flushed lish + // Check it's not in the recently flushed list for _, flushedSequence := range bp.flushedSequences { if newWork.msg.Sequence == flushedSequence { log.L(bp.ctx).Debugf("Ignoring add of recently flushed message %s sequence=%d to in-flight batch assembly %s", newWork.msg.Header.ID, newWork.msg.Sequence, bp.assemblyID) diff --git a/pkg/fftypes/transaction.go b/pkg/fftypes/transaction.go index d9d5bb2cf4..83d6d7edbf 100644 --- a/pkg/fftypes/transaction.go +++ b/pkg/fftypes/transaction.go @@ -21,7 +21,7 @@ type TransactionType = FFEnum var ( // TransactionTypeNone deprecreated - replaced by TransactionTypeUnpinned TransactionTypeNone TransactionType = ffEnum("txtype", "none") - // TransactionTypeUnpinned indicates no transaction should be used for this message/batch + // TransactionTypeUnpinned indicates the message will be sent without pinning any evidence to the blockchain. Not supported for broadcast. The FireFly transaction will be used to track the sends to all group members. TransactionTypeUnpinned TransactionType = ffEnum("txtype", "unpinned") // TransactionTypeBatchPin represents a pinning transaction, that verifies the originator of the data, and sequences the event deterministically between parties TransactionTypeBatchPin TransactionType = ffEnum("txtype", "batch_pin") From a6198ee29e76cb557ee0a8ba9d6e08fd0f57ca60 Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Fri, 11 Feb 2022 12:12:24 -0500 Subject: [PATCH 16/23] Add API for batch manager status Signed-off-by: Peter Broadhurst --- docs/swagger/swagger.yaml | 59 +++++++++++++++++++ .../route_get_status_batchmanager.go | 42 +++++++++++++ .../route_get_status_batchmanager_test.go | 42 +++++++++++++ internal/apiserver/routes.go | 1 + internal/batch/batch_manager.go | 16 +++-- internal/batch/batch_manager_test.go | 2 +- internal/orchestrator/orchestrator.go | 5 ++ internal/orchestrator/orchestrator_test.go | 1 + mocks/batchmocks/manager.go | 8 +-- mocks/orchestratormocks/orchestrator.go | 18 ++++++ pkg/fftypes/transaction.go | 2 +- 11 files changed, 185 insertions(+), 11 deletions(-) create mode 100644 internal/apiserver/route_get_status_batchmanager.go create mode 100644 internal/apiserver/route_get_status_batchmanager_test.go diff --git a/docs/swagger/swagger.yaml b/docs/swagger/swagger.yaml index 5bd4a0b924..e2163e8628 100644 --- a/docs/swagger/swagger.yaml +++ b/docs/swagger/swagger.yaml @@ -8697,5 +8697,64 @@ paths: description: Success default: description: "" + /status/batchmanager: + get: + description: 'TODO: Description' + operationId: getStatusBatchManager + parameters: + - description: Server-side request timeout (millseconds, or set a custom suffix + like 10s) + in: header + name: Request-Timeout + schema: + default: 120s + type: string + responses: + "200": + content: + application/json: + schema: + properties: + processors: + items: + properties: + dispatcher: + type: string + name: + type: string + status: + properties: + averageBatchBytes: + format: int64 + type: integer + averageBatchData: + format: double + type: number + averageBatchMessages: + format: double + type: number + averageFlushTimeMS: + format: int64 + type: integer + blocked: + type: boolean + flushing: {} + lastFlushError: + type: string + lastFlushErrorTime: {} + lastFlushStartTime: {} + totalBatches: + format: int64 + type: integer + totalErrors: + format: int64 + type: integer + type: object + type: object + type: array + type: object + description: Success + default: + description: "" servers: - url: http://localhost:12345 diff --git a/internal/apiserver/route_get_status_batchmanager.go b/internal/apiserver/route_get_status_batchmanager.go new file mode 100644 index 0000000000..4f30336b2e --- /dev/null +++ b/internal/apiserver/route_get_status_batchmanager.go @@ -0,0 +1,42 @@ +// Copyright © 2022 Kaleido, Inc. +// +// SPDX-License-Identifier: Apache-2.0 +// +// 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 apiserver + +import ( + "net/http" + + "github.com/hyperledger/firefly/internal/batch" + "github.com/hyperledger/firefly/internal/i18n" + "github.com/hyperledger/firefly/internal/oapispec" +) + +var getStatusBatchManager = &oapispec.Route{ + Name: "getStatusBatchManager", + Path: "status/batchmanager", + Method: http.MethodGet, + PathParams: nil, + QueryParams: nil, + FilterFactory: nil, + Description: i18n.MsgTBD, + JSONInputValue: nil, + JSONOutputValue: func() interface{} { return &batch.ManagerStatus{} }, + JSONOutputCodes: []int{http.StatusOK}, + JSONHandler: func(r *oapispec.APIRequest) (output interface{}, err error) { + output = getOr(r.Ctx).BatchManager().Status() + return output, nil + }, +} diff --git a/internal/apiserver/route_get_status_batchmanager_test.go b/internal/apiserver/route_get_status_batchmanager_test.go new file mode 100644 index 0000000000..07380af68c --- /dev/null +++ b/internal/apiserver/route_get_status_batchmanager_test.go @@ -0,0 +1,42 @@ +// Copyright © 2021 Kaleido, Inc. +// +// SPDX-License-Identifier: Apache-2.0 +// +// 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 apiserver + +import ( + "net/http/httptest" + "testing" + + "github.com/hyperledger/firefly/internal/batch" + "github.com/hyperledger/firefly/mocks/batchmocks" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/mock" +) + +func TestGetStatusBatching(t *testing.T) { + o, r := newTestAPIServer() + req := httptest.NewRequest("GET", "/api/v1/status/batchmanager", nil) + req.Header.Set("Content-Type", "application/json; charset=utf-8") + res := httptest.NewRecorder() + + mbm := &batchmocks.Manager{} + o.On("BatchManager").Return(mbm) + mbm.On("Status", mock.Anything, mock.Anything). + Return(&batch.ManagerStatus{}, nil) + r.ServeHTTP(res, req) + + assert.Equal(t, 200, res.Result().StatusCode) +} diff --git a/internal/apiserver/routes.go b/internal/apiserver/routes.go index 504a23b5ff..97ac7d6698 100644 --- a/internal/apiserver/routes.go +++ b/internal/apiserver/routes.go @@ -64,6 +64,7 @@ var routes = []*oapispec.Route{ getOpByID, getOps, getStatus, + getStatusBatchManager, getSubscriptionByID, getSubscriptions, getTxnByID, diff --git a/internal/batch/batch_manager.go b/internal/batch/batch_manager.go index 2bfb940132..d3342119aa 100644 --- a/internal/batch/batch_manager.go +++ b/internal/batch/batch_manager.go @@ -66,7 +66,11 @@ type Manager interface { Start() error Close() WaitStop() - Status() []*ProcessorStatus + Status() *ManagerStatus +} + +type ManagerStatus struct { + Processors []*ProcessorStatus `json:"processors"` } type ProcessorStatus struct { @@ -345,13 +349,15 @@ func (bm *batchManager) getProcessors() []*batchProcessor { return processors } -func (bm *batchManager) Status() []*ProcessorStatus { +func (bm *batchManager) Status() *ManagerStatus { processors := bm.getProcessors() - status := make([]*ProcessorStatus, len(processors)) + pStatus := make([]*ProcessorStatus, len(processors)) for i, p := range processors { - status[i] = p.status() + pStatus[i] = p.status() + } + return &ManagerStatus{ + Processors: pStatus, } - return status } func (bm *batchManager) Close() { diff --git a/internal/batch/batch_manager_test.go b/internal/batch/batch_manager_test.go index fc6793aeeb..3e8f1287ac 100644 --- a/internal/batch/batch_manager_test.go +++ b/internal/batch/batch_manager_test.go @@ -126,7 +126,7 @@ func TestE2EDispatchBroadcast(t *testing.T) { // Check the status while we know there's a flush going on status := bm.Status() - assert.NotNil(t, status[0].Status.Flushing) + assert.NotNil(t, status.Processors[0].Status.Flushing) b := <-waitForDispatch assert.Equal(t, *msg.Header.ID, *b.Payload.Messages[0].Header.ID) diff --git a/internal/orchestrator/orchestrator.go b/internal/orchestrator/orchestrator.go index 740ddfb348..fd3867d090 100644 --- a/internal/orchestrator/orchestrator.go +++ b/internal/orchestrator/orchestrator.go @@ -73,6 +73,7 @@ type Orchestrator interface { Assets() assets.Manager Contracts() contracts.Manager Metrics() metrics.Manager + BatchManager() batch.Manager IsPreInit() bool // Status @@ -255,6 +256,10 @@ func (or *orchestrator) Events() events.EventManager { return or.events } +func (or *orchestrator) BatchManager() batch.Manager { + return or.batch +} + func (or *orchestrator) NetworkMap() networkmap.Manager { return or.networkmap } diff --git a/internal/orchestrator/orchestrator_test.go b/internal/orchestrator/orchestrator_test.go index 5d1a36fa96..68f3a5d351 100644 --- a/internal/orchestrator/orchestrator_test.go +++ b/internal/orchestrator/orchestrator_test.go @@ -628,6 +628,7 @@ func TestInitOK(t *testing.T) { assert.Equal(t, or.mbm, or.Broadcast()) assert.Equal(t, or.mpm, or.PrivateMessaging()) assert.Equal(t, or.mem, or.Events()) + assert.Equal(t, or.mba, or.BatchManager()) assert.Equal(t, or.mnm, or.NetworkMap()) assert.Equal(t, or.mdm, or.Data()) assert.Equal(t, or.mam, or.Assets()) diff --git a/mocks/batchmocks/manager.go b/mocks/batchmocks/manager.go index 9c98f6b727..d1d712fd8d 100644 --- a/mocks/batchmocks/manager.go +++ b/mocks/batchmocks/manager.go @@ -55,15 +55,15 @@ func (_m *Manager) Start() error { } // Status provides a mock function with given fields: -func (_m *Manager) Status() []*batch.ProcessorStatus { +func (_m *Manager) Status() *batch.ManagerStatus { ret := _m.Called() - var r0 []*batch.ProcessorStatus - if rf, ok := ret.Get(0).(func() []*batch.ProcessorStatus); ok { + var r0 *batch.ManagerStatus + if rf, ok := ret.Get(0).(func() *batch.ManagerStatus); ok { r0 = rf() } else { if ret.Get(0) != nil { - r0 = ret.Get(0).([]*batch.ProcessorStatus) + r0 = ret.Get(0).(*batch.ManagerStatus) } } diff --git a/mocks/orchestratormocks/orchestrator.go b/mocks/orchestratormocks/orchestrator.go index aaa3035965..a5a1ca6883 100644 --- a/mocks/orchestratormocks/orchestrator.go +++ b/mocks/orchestratormocks/orchestrator.go @@ -4,6 +4,8 @@ package orchestratormocks import ( assets "github.com/hyperledger/firefly/internal/assets" + batch "github.com/hyperledger/firefly/internal/batch" + broadcast "github.com/hyperledger/firefly/internal/broadcast" context "context" @@ -48,6 +50,22 @@ func (_m *Orchestrator) Assets() assets.Manager { return r0 } +// BatchManager provides a mock function with given fields: +func (_m *Orchestrator) BatchManager() batch.Manager { + ret := _m.Called() + + var r0 batch.Manager + if rf, ok := ret.Get(0).(func() batch.Manager); ok { + r0 = rf() + } else { + if ret.Get(0) != nil { + r0 = ret.Get(0).(batch.Manager) + } + } + + return r0 +} + // Broadcast provides a mock function with given fields: func (_m *Orchestrator) Broadcast() broadcast.Manager { ret := _m.Called() diff --git a/pkg/fftypes/transaction.go b/pkg/fftypes/transaction.go index 83d6d7edbf..9549563f03 100644 --- a/pkg/fftypes/transaction.go +++ b/pkg/fftypes/transaction.go @@ -19,7 +19,7 @@ package fftypes type TransactionType = FFEnum var ( - // TransactionTypeNone deprecreated - replaced by TransactionTypeUnpinned + // TransactionTypeNone deprecated - replaced by TransactionTypeUnpinned TransactionTypeNone TransactionType = ffEnum("txtype", "none") // TransactionTypeUnpinned indicates the message will be sent without pinning any evidence to the blockchain. Not supported for broadcast. The FireFly transaction will be used to track the sends to all group members. TransactionTypeUnpinned TransactionType = ffEnum("txtype", "unpinned") From 7d8d2ebc616896dcce79cf449f2dd52d9f3b86be Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Fri, 11 Feb 2022 12:16:30 -0500 Subject: [PATCH 17/23] Add skip optimization Signed-off-by: Peter Broadhurst --- internal/batch/batch_processor.go | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/internal/batch/batch_processor.go b/internal/batch/batch_processor.go index 491bf81dd1..608cd4ed98 100644 --- a/internal/batch/batch_processor.go +++ b/internal/batch/batch_processor.go @@ -168,6 +168,7 @@ func (bp *batchProcessor) addWork(newWork *batchWork) (full, overflow bool) { if newWork.msg.Sequence == work.msg.Sequence { log.L(bp.ctx).Debugf("Ignoring duplicate add of message %s sequence=%d to in-flight batch assembly %s", newWork.msg.Header.ID, newWork.msg.Sequence, bp.assemblyID) skip = true + break } if !added && !skip && newWork.msg.Sequence < work.msg.Sequence { newQueue = append(newQueue, newWork) @@ -175,7 +176,9 @@ func (bp *batchProcessor) addWork(newWork *batchWork) (full, overflow bool) { } newQueue = append(newQueue, work) } - if !added && !skip { + if skip { + newQueue = bp.assemblyQueue + } else if !added { newQueue = append(newQueue, newWork) added = true } From d004b4a09927882723b20bc357bfa35a0b533eab Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Fri, 11 Feb 2022 12:38:44 -0500 Subject: [PATCH 18/23] Skip early Signed-off-by: Peter Broadhurst --- internal/batch/batch_processor.go | 24 +++++++++++++----------- 1 file changed, 13 insertions(+), 11 deletions(-) diff --git a/internal/batch/batch_processor.go b/internal/batch/batch_processor.go index 608cd4ed98..de433551e0 100644 --- a/internal/batch/batch_processor.go +++ b/internal/batch/batch_processor.go @@ -163,18 +163,20 @@ func (bp *batchProcessor) addWork(newWork *batchWork) (full, overflow bool) { break } } - // Build the new sorted work list, checking there for duplicates too - for _, work := range bp.assemblyQueue { - if newWork.msg.Sequence == work.msg.Sequence { - log.L(bp.ctx).Debugf("Ignoring duplicate add of message %s sequence=%d to in-flight batch assembly %s", newWork.msg.Header.ID, newWork.msg.Sequence, bp.assemblyID) - skip = true - break - } - if !added && !skip && newWork.msg.Sequence < work.msg.Sequence { - newQueue = append(newQueue, newWork) - added = true + if !skip { + // Build the new sorted work list, checking there for duplicates too + for _, work := range bp.assemblyQueue { + if newWork.msg.Sequence == work.msg.Sequence { + log.L(bp.ctx).Debugf("Ignoring duplicate add of message %s sequence=%d to in-flight batch assembly %s", newWork.msg.Header.ID, newWork.msg.Sequence, bp.assemblyID) + skip = true + break + } + if !added && !skip && newWork.msg.Sequence < work.msg.Sequence { + newQueue = append(newQueue, newWork) + added = true + } + newQueue = append(newQueue, work) } - newQueue = append(newQueue, work) } if skip { newQueue = bp.assemblyQueue From 5b471579348d939458fe8cdbf5e53734d69c7a07 Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Fri, 11 Feb 2022 12:42:10 -0500 Subject: [PATCH 19/23] Assume overflow and full are false if addWork is a no-op Signed-off-by: Peter Broadhurst --- internal/batch/batch_processor.go | 38 ++++++++++++------------------- 1 file changed, 14 insertions(+), 24 deletions(-) diff --git a/internal/batch/batch_processor.go b/internal/batch/batch_processor.go index de433551e0..fb635dd75d 100644 --- a/internal/batch/batch_processor.go +++ b/internal/batch/batch_processor.go @@ -154,40 +154,30 @@ func (bp *batchProcessor) newAssembly(initalWork ...*batchWork) { func (bp *batchProcessor) addWork(newWork *batchWork) (full, overflow bool) { newQueue := make([]*batchWork, 0, len(bp.assemblyQueue)+1) added := false - skip := false // Check it's not in the recently flushed list for _, flushedSequence := range bp.flushedSequences { if newWork.msg.Sequence == flushedSequence { log.L(bp.ctx).Debugf("Ignoring add of recently flushed message %s sequence=%d to in-flight batch assembly %s", newWork.msg.Header.ID, newWork.msg.Sequence, bp.assemblyID) - skip = true - break + return false, false } } - if !skip { - // Build the new sorted work list, checking there for duplicates too - for _, work := range bp.assemblyQueue { - if newWork.msg.Sequence == work.msg.Sequence { - log.L(bp.ctx).Debugf("Ignoring duplicate add of message %s sequence=%d to in-flight batch assembly %s", newWork.msg.Header.ID, newWork.msg.Sequence, bp.assemblyID) - skip = true - break - } - if !added && !skip && newWork.msg.Sequence < work.msg.Sequence { - newQueue = append(newQueue, newWork) - added = true - } - newQueue = append(newQueue, work) + // Build the new sorted work list, checking there for duplicates too + for _, work := range bp.assemblyQueue { + if newWork.msg.Sequence == work.msg.Sequence { + log.L(bp.ctx).Debugf("Ignoring duplicate add of message %s sequence=%d to in-flight batch assembly %s", newWork.msg.Header.ID, newWork.msg.Sequence, bp.assemblyID) + return false, false } + if !added && newWork.msg.Sequence < work.msg.Sequence { + newQueue = append(newQueue, newWork) + added = true + } + newQueue = append(newQueue, work) } - if skip { - newQueue = bp.assemblyQueue - } else if !added { + if !added { newQueue = append(newQueue, newWork) - added = true - } - if added { - log.L(bp.ctx).Debugf("Added message %s sequence=%d to in-flight batch assembly %s", newWork.msg.Header.ID, newWork.msg.Sequence, bp.assemblyID) - bp.assemblyQueueBytes += newWork.estimateSize() } + log.L(bp.ctx).Debugf("Added message %s sequence=%d to in-flight batch assembly %s", newWork.msg.Header.ID, newWork.msg.Sequence, bp.assemblyID) + bp.assemblyQueueBytes += newWork.estimateSize() bp.assemblyQueue = newQueue full = len(bp.assemblyQueue) >= int(bp.conf.BatchMaxSize) || (bp.assemblyQueueBytes >= bp.conf.BatchMaxBytes) overflow = len(bp.assemblyQueue) > 1 && (bp.assemblyQueueBytes > bp.conf.BatchMaxBytes) From 097f7942c4e73f834467bf40b5a32f8727a64527 Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Fri, 11 Feb 2022 12:52:05 -0500 Subject: [PATCH 20/23] Reverse the order of the flushedSequences as we add them, to allow trimming the oldest Signed-off-by: Peter Broadhurst --- internal/batch/batch_processor.go | 13 +++++++------ internal/batch/batch_processor_test.go | 4 ++-- 2 files changed, 9 insertions(+), 8 deletions(-) diff --git a/internal/batch/batch_processor.go b/internal/batch/batch_processor.go index fb635dd75d..577e6f3e8e 100644 --- a/internal/batch/batch_processor.go +++ b/internal/batch/batch_processor.go @@ -187,7 +187,7 @@ func (bp *batchProcessor) addWork(newWork *batchWork) (full, overflow bool) { func (bp *batchProcessor) startFlush(overflow bool) (id *fftypes.UUID, flushAssembly []*batchWork, byteSize int64) { bp.statusMux.Lock() defer bp.statusMux.Unlock() - // Star the clock + // Start the clock bp.flushStatus.Blocked = false bp.flushStatus.LastFlushTime = fftypes.Now() // Split the current work if required for overflow @@ -208,14 +208,15 @@ func (bp *batchProcessor) startFlush(overflow bool) (id *fftypes.UUID, flushAsse if newFlushedSeqLen > maxFlushedSeqLen && maxFlushedSeqLen > len(bp.flushedSequences) { newFlushedSeqLen = maxFlushedSeqLen } - newFlushedSequnces := make([]int64, newFlushedSeqLen) - for i, fs := range flushAssembly { - newFlushedSequnces[i] = fs.msg.Sequence + newFlushedSequences := make([]int64, newFlushedSeqLen) + for i := 0; i < len(flushAssembly); i++ { + // Add in reverse order - so we can trim the end of it off later (in the next block) and keep the newest + newFlushedSequences[len(flushAssembly)-i-1] = flushAssembly[i].msg.Sequence } for i := 0; i < newFlushedSeqLen-len(flushAssembly); i++ { - newFlushedSequnces[i+len(flushAssembly)] = bp.flushedSequences[i] + newFlushedSequences[i+len(flushAssembly)] = bp.flushedSequences[i] } - bp.flushedSequences = newFlushedSequnces + bp.flushedSequences = newFlushedSequences // Cycle to the next assembly id = bp.assemblyID byteSize = bp.assemblyQueueBytes diff --git a/internal/batch/batch_processor_test.go b/internal/batch/batch_processor_test.go index de5655d508..09e3afaf06 100644 --- a/internal/batch/batch_processor_test.go +++ b/internal/batch/batch_processor_test.go @@ -259,7 +259,7 @@ func TestStartFlushOverflow(t *testing.T) { }) batchID := fftypes.NewUUID() bp.assemblyID = batchID - bp.flushedSequences = []int64{100, 101, 102, 103, 104} + bp.flushedSequences = []int64{104, 103, 102, 101, 100} bp.assemblyQueue = []*batchWork{ {msg: &fftypes.Message{Sequence: 200}}, {msg: &fftypes.Message{Sequence: 201}}, @@ -270,7 +270,7 @@ func TestStartFlushOverflow(t *testing.T) { flushBatchID, flushAssembly, _ := bp.startFlush(true) assert.Equal(t, batchID, flushBatchID) - assert.Equal(t, []int64{200, 201, 202, 100, 101, 102}, bp.flushedSequences) + assert.Equal(t, []int64{202, 201, 200, 104, 103, 102}, bp.flushedSequences) assert.Equal(t, []*batchWork{ {msg: &fftypes.Message{Sequence: 200}}, {msg: &fftypes.Message{Sequence: 201}}, From d2bdcc4a519c6095d9611d0e7e3a780a10c1a0bd Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Fri, 11 Feb 2022 12:59:11 -0500 Subject: [PATCH 21/23] Addresss last set of review comments Signed-off-by: Peter Broadhurst --- internal/batch/batch_processor.go | 4 ++-- pkg/database/plugin.go | 5 ++++- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/internal/batch/batch_processor.go b/internal/batch/batch_processor.go index 577e6f3e8e..79d9d4532f 100644 --- a/internal/batch/batch_processor.go +++ b/internal/batch/batch_processor.go @@ -321,7 +321,7 @@ func (bp *batchProcessor) assemblyLoop() { err := bp.flush(overflow) if err != nil { - l.Tracef("Batch processor shutting down: %s", err) + l.Warnf("Batch processor shutting down: %s", err) _ = batchTimeout.Stop() return } @@ -493,7 +493,7 @@ func (bp *batchProcessor) markMessagesDispatched(batch *fftypes.Batch) error { Set("batch", batch.ID). // Mark the batch they are in Set("state", fftypes.MessageStateSent) // Set them sent, so they won't be picked up and re-sent after restart/rewind } else { - // Immediate confirmation if no transaction + // Immediate confirmation if no batch pinning update = database.MessageQueryFactory.NewUpdate(ctx). Set("batch", batch.ID). Set("state", fftypes.MessageStateConfirmed). diff --git a/pkg/database/plugin.go b/pkg/database/plugin.go index 3c989c2c03..755fb73bdf 100644 --- a/pkg/database/plugin.go +++ b/pkg/database/plugin.go @@ -228,7 +228,10 @@ type iSubscriptionCollection interface { } type iEventCollection interface { - // InsertEvent - Insert an event + // InsertEvent - Insert an event. The order of the sequences added to the database, must match the order that + // the rows/objects appear available to the event dispatcher. For a concurrency enabled database + // with multi-operation transactions (like PSQL or other enterprise SQL based DB) we need + // to hold an exclusive table lock. InsertEvent(ctx context.Context, data *fftypes.Event) (err error) // UpdateEvent - Update event From b19cc8f2863e28a9564cbe335f083a681676153c Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Fri, 11 Feb 2022 13:01:52 -0500 Subject: [PATCH 22/23] Only valid TX type for broadcast is batch_pin Signed-off-by: Peter Broadhurst --- internal/broadcast/message.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/internal/broadcast/message.go b/internal/broadcast/message.go index 23613cd5cc..6ade0c91b2 100644 --- a/internal/broadcast/message.go +++ b/internal/broadcast/message.go @@ -89,9 +89,8 @@ func (s *broadcastSender) setDefaults() { if s.msg.Header.Type == "" { s.msg.Header.Type = fftypes.MessageTypeBroadcast } - if s.msg.Header.TxType == "" { - s.msg.Header.TxType = fftypes.TransactionTypeBatchPin - } + // We only have one transaction type for broadcast currently + s.msg.Header.TxType = fftypes.TransactionTypeBatchPin } func (s *broadcastSender) resolveAndSend(ctx context.Context, method sendMethod) error { From 42335d84922050e9b772d54c6472ff625c5dd8c0 Mon Sep 17 00:00:00 2001 From: Peter Broadhurst Date: Fri, 11 Feb 2022 13:34:44 -0500 Subject: [PATCH 23/23] Make the logic clearer by shifting thing forwards and adding to the end Signed-off-by: Peter Broadhurst --- internal/batch/batch_processor.go | 45 +++++++++++++++----------- internal/batch/batch_processor_test.go | 4 +-- 2 files changed, 29 insertions(+), 20 deletions(-) diff --git a/internal/batch/batch_processor.go b/internal/batch/batch_processor.go index 79d9d4532f..7fefbc070e 100644 --- a/internal/batch/batch_processor.go +++ b/internal/batch/batch_processor.go @@ -184,6 +184,32 @@ func (bp *batchProcessor) addWork(newWork *batchWork) (full, overflow bool) { return full, overflow } +func (bp *batchProcessor) addFlushedSequences(flushAssembly []*batchWork) { + // We need to keep track of the sequences we're flushing, because until we finish our flush + // the batch processor might be re-queuing the same messages to use due to rewinds. + + // We keep twice the batch size, which might be made up of multiple batches + maxFlushedSeqLen := int(2 * bp.conf.BatchMaxSize) + + // We keep as much of the END of the existing set as we can, and shift it forwards. + // Then we add the whole of the current flush set after that + combinedLen := len(flushAssembly) + len(bp.flushedSequences) + newLength := combinedLen + if combinedLen > maxFlushedSeqLen { + newLength = maxFlushedSeqLen + } + dropLength := combinedLen - newLength + retainLength := len(bp.flushedSequences) - dropLength + newFlushedSequences := make([]int64, newLength) + for i := 0; i < retainLength; i++ { + newFlushedSequences[i] = bp.flushedSequences[dropLength+i] + } + for i := 0; i < len(flushAssembly); i++ { + newFlushedSequences[retainLength+i] = flushAssembly[i].msg.Sequence + } + bp.flushedSequences = newFlushedSequences +} + func (bp *batchProcessor) startFlush(overflow bool) (id *fftypes.UUID, flushAssembly []*batchWork, byteSize int64) { bp.statusMux.Lock() defer bp.statusMux.Unlock() @@ -199,24 +225,7 @@ func (bp *batchProcessor) startFlush(overflow bool) (id *fftypes.UUID, flushAsse } else { flushAssembly = bp.assemblyQueue } - // We need to keep track of the sequences we're flushing, because until we finish our flush - // the batch processor might be re-queuing the same messages to use due to rewinds. - // We keep all of the last batch, and up to twice the batch size over time (noting our channel - // size is our batch size - so the batch manager cannot get further than that ahead). - newFlushedSeqLen := len(flushAssembly) + len(bp.flushedSequences) - maxFlushedSeqLen := int(2 * bp.conf.BatchMaxSize) - if newFlushedSeqLen > maxFlushedSeqLen && maxFlushedSeqLen > len(bp.flushedSequences) { - newFlushedSeqLen = maxFlushedSeqLen - } - newFlushedSequences := make([]int64, newFlushedSeqLen) - for i := 0; i < len(flushAssembly); i++ { - // Add in reverse order - so we can trim the end of it off later (in the next block) and keep the newest - newFlushedSequences[len(flushAssembly)-i-1] = flushAssembly[i].msg.Sequence - } - for i := 0; i < newFlushedSeqLen-len(flushAssembly); i++ { - newFlushedSequences[i+len(flushAssembly)] = bp.flushedSequences[i] - } - bp.flushedSequences = newFlushedSequences + bp.addFlushedSequences(flushAssembly) // Cycle to the next assembly id = bp.assemblyID byteSize = bp.assemblyQueueBytes diff --git a/internal/batch/batch_processor_test.go b/internal/batch/batch_processor_test.go index 09e3afaf06..56abea0113 100644 --- a/internal/batch/batch_processor_test.go +++ b/internal/batch/batch_processor_test.go @@ -259,7 +259,7 @@ func TestStartFlushOverflow(t *testing.T) { }) batchID := fftypes.NewUUID() bp.assemblyID = batchID - bp.flushedSequences = []int64{104, 103, 102, 101, 100} + bp.flushedSequences = []int64{100, 101, 102, 103, 104} bp.assemblyQueue = []*batchWork{ {msg: &fftypes.Message{Sequence: 200}}, {msg: &fftypes.Message{Sequence: 201}}, @@ -270,7 +270,7 @@ func TestStartFlushOverflow(t *testing.T) { flushBatchID, flushAssembly, _ := bp.startFlush(true) assert.Equal(t, batchID, flushBatchID) - assert.Equal(t, []int64{202, 201, 200, 104, 103, 102}, bp.flushedSequences) + assert.Equal(t, []int64{102, 103, 104, 200, 201, 202}, bp.flushedSequences) assert.Equal(t, []*batchWork{ {msg: &fftypes.Message{Sequence: 200}}, {msg: &fftypes.Message{Sequence: 201}},