From f23843ca95b3b48470787e2dcff592c42567222f Mon Sep 17 00:00:00 2001 From: Will Winder Date: Mon, 5 Aug 2024 10:01:25 -0400 Subject: [PATCH] exec: use checkMessage for isExecuted and preparing token data. (#50) --- execute/report/errors.go | 1 + execute/report/report.go | 162 +++++++----- execute/report/report_test.go | 475 +++++++++++++++++++++++++++++----- plugintypes/execute.go | 3 +- 4 files changed, 506 insertions(+), 135 deletions(-) diff --git a/execute/report/errors.go b/execute/report/errors.go index 3bd8198b5..a0c52e1af 100644 --- a/execute/report/errors.go +++ b/execute/report/errors.go @@ -3,3 +3,4 @@ package report import "errors" var ErrEmptyReport = errors.New("no messages can fit in the report") +var ErrNotReady = errors.New("token data not ready") diff --git a/execute/report/report.go b/execute/report/report.go index 2da972aa0..466a3284c 100644 --- a/execute/report/report.go +++ b/execute/report/report.go @@ -4,43 +4,46 @@ import ( "bytes" "context" "encoding/hex" + "errors" "fmt" "slices" "github.com/smartcontractkit/chainlink-common/pkg/logger" - cciptypes "github.com/smartcontractkit/chainlink-common/pkg/types/ccipocr3" - "github.com/smartcontractkit/chainlink-ccip/execute/types" "github.com/smartcontractkit/chainlink-ccip/internal/libs/slicelib" "github.com/smartcontractkit/chainlink-ccip/plugintypes" ) -// buildSingleChainReportHelper converts the on-chain event data stored in -// cciptypes.ExecutePluginCommitDataWithMessages into the final on-chain report format. +// buildSingleChainReportHelper converts the on-chain event data stored in cciptypes.ExecutePluginCommitData into the +// final on-chain report format. Messages in the report are selected based on the readyMessages argument. If +// readyMessages is empty all messages will be included. This allows the caller to create smaller reports if needed. // -// The hasher and encoding codec are provided as arguments to allow for chain-specific formats to be used. +// Before calling this function all messages should have been checked and processed by the checkMessage function. // -// The messages argument indicates which messages should be included in the report. If messages is empty -// all messages will be included. This allows the caller to create smaller reports if needed. Executed messages -// are skipped automatically. +// The hasher and encoding codec are provided as arguments to allow for chain-specific formats to be used. func buildSingleChainReportHelper( ctx context.Context, lggr logger.Logger, hasher cciptypes.MessageHasher, - tokenDataReader types.TokenDataReader, report plugintypes.ExecutePluginCommitData, - messages map[int]struct{}, + readyMessages map[int]struct{}, ) (cciptypes.ExecutePluginReportSingleChain, error) { - if len(messages) == 0 { - if messages == nil { - messages = make(map[int]struct{}) + if len(readyMessages) == 0 { + if readyMessages == nil { + readyMessages = make(map[int]struct{}) } for i := 0; i < len(report.Messages); i++ { - messages[i] = struct{}{} + readyMessages[i] = struct{}{} } } + numMsg := len(report.Messages) + if len(report.TokenData) != numMsg { + return cciptypes.ExecutePluginReportSingleChain{}, + fmt.Errorf("token data length mismatch: got %d, expected %d", len(report.TokenData), numMsg) + } + lggr.Infow( "constructing merkle tree", "sourceChain", report.SourceChain, @@ -71,36 +74,9 @@ func buildSingleChainReportHelper( var toExecute []int var offchainTokenData [][][]byte var msgInRoot []cciptypes.Message - executedIdx := 0 for i, msg := range report.Messages { - seqNum := report.SequenceNumberRange.Start() + cciptypes.SeqNum(i) - // Skip messages which are already executed - if executedIdx < len(report.ExecutedMessages) && report.ExecutedMessages[executedIdx] == seqNum { - executedIdx++ - } else if _, ok := messages[i]; ok { - var tokenData [][]byte - var err error - if tokenDataReader != nil { - tokenData, err = tokenDataReader.ReadTokenData(context.Background(), report.SourceChain, msg.Header.SequenceNumber) - if err != nil { - // TODO: skip message instead of failing the whole thing. - // that might mean moving the token data reading out of the loop. - lggr.Infow( - "unable to read token data", - "sourceChain", report.SourceChain, - "seqNum", msg.Header.SequenceNumber, - "error", err) - return cciptypes.ExecutePluginReportSingleChain{}, - fmt.Errorf("unable to read token data for message %d: %w", msg.Header.SequenceNumber, err) - } - - lggr.Infow( - "read token data", - "sourceChain", report.SourceChain, - "seqNum", msg.Header.SequenceNumber, - "data", tokenData) - } - offchainTokenData = append(offchainTokenData, tokenData) + if _, ok := readyMessages[i]; ok { + offchainTokenData = append(offchainTokenData, report.TokenData[i]) toExecute = append(toExecute, i) msgInRoot = append(msgInRoot, msg) } @@ -142,8 +118,10 @@ func buildSingleChainReportHelper( type messageStatus string const ( - ReadyToExecute messageStatus = "ready_to_execute" - AlreadyExecuted messageStatus = "already_executed" + ReadyToExecute messageStatus = "ready_to_execute" + AlreadyExecuted messageStatus = "already_executed" + TokenDataNotReady messageStatus = "token_data_not_ready" //nolint:gosec // this is not a password + TokenDataFetchError messageStatus = "token_data_fetch_error" /* SenderAlreadySkipped messageStatus = "sender_already_skipped" MessageMaxGasCalcError messageStatus = "message_max_gas_calc_error" @@ -153,8 +131,6 @@ const ( InvalidNonce messageStatus = "invalid_nonce" AggregateTokenValueComputeError messageStatus = "aggregate_token_value_compute_error" AggregateTokenLimitExceeded messageStatus = "aggregate_token_limit_exceeded" - TokenDataNotReady messageStatus = "token_data_not_ready" - TokenDataFetchError messageStatus = "token_data_fetch_error" TokenNotInDestTokenPrices messageStatus = "token_not_in_dest_token_prices" TokenNotInSrcTokenPrices messageStatus = "token_not_in_src_token_prices" InsufficientRemainingFee messageStatus = "insufficient_remaining_fee" @@ -163,14 +139,67 @@ const ( ) func (b *execReportBuilder) checkMessage( - _ context.Context, idx int, execReport plugintypes.ExecutePluginCommitData, + ctx context.Context, idx int, execReport plugintypes.ExecutePluginCommitData, // TODO: get rid of the nolint when the error is used -) (messageStatus, error) { // nolint this will use the error eventually - if slices.Contains(execReport.ExecutedMessages, execReport.Messages[idx].Header.SequenceNumber) { - return AlreadyExecuted, nil +) (plugintypes.ExecutePluginCommitData, messageStatus, error) { // nolint this will use the error eventually + if idx >= len(execReport.Messages) { + b.lggr.Errorw("message index out of range", "index", idx, "numMessages", len(execReport.Messages)) + return execReport, TokenDataFetchError, fmt.Errorf("message index out of range") + } + + msg := execReport.Messages[idx] + + // Check if the message has already been executed. + if slices.Contains(execReport.ExecutedMessages, msg.Header.SequenceNumber) { + b.lggr.Infow( + "message already executed", + "messageID", msg.Header.MessageID, + "sourceChain", execReport.SourceChain, + "seqNum", msg.Header.SequenceNumber) + return execReport, AlreadyExecuted, nil } - return ReadyToExecute, nil + // Check if token data is ready. + if b.tokenDataReader != nil { + tokenData, err := b.tokenDataReader.ReadTokenData(ctx, execReport.SourceChain, msg.Header.SequenceNumber) + if err != nil { + if errors.Is(err, ErrNotReady) { + b.lggr.Infow( + "unable to read token data - token data not ready", + "messageID", msg.Header.MessageID, + "sourceChain", execReport.SourceChain, + "seqNum", msg.Header.SequenceNumber, + "error", err) + return execReport, TokenDataNotReady, nil + } + b.lggr.Infow( + "unable to read token data - unknown error", + "messageID", msg.Header.MessageID, + "sourceChain", execReport.SourceChain, + "seqNum", msg.Header.SequenceNumber, + "error", err) + return execReport, TokenDataFetchError, nil + } + + // pad token data if needed + for len(execReport.TokenData) <= idx { + execReport.TokenData = append(execReport.TokenData, nil) + } + + execReport.TokenData[idx] = tokenData + b.lggr.Infow( + "read token data", + "messageID", msg.Header.MessageID, + "sourceChain", execReport.SourceChain, + "seqNum", msg.Header.SequenceNumber, + "data", tokenData) + } + + // TODO: Check for valid nonce + // TODO: Check for max gas limit + // TODO: Check for fee boost + + return execReport, ReadyToExecute, nil } func (b *execReportBuilder) verifyReport( @@ -216,9 +245,25 @@ func (b *execReportBuilder) buildSingleChainReport( commitReport = markNewMessagesExecuted(execReport, commitReport) return execReport, commitReport, nil } + + // Check which messages are ready to execute, and update report with any additional metadata needed for execution. + readyMessages := make(map[int]struct{}) + for i := 0; i < len(report.Messages); i++ { + updatedReport, status, err := b.checkMessage(ctx, i, report) + if err != nil { + return cciptypes.ExecutePluginReportSingleChain{}, + plugintypes.ExecutePluginCommitData{}, + fmt.Errorf("unable to check message: %w", err) + } + report = updatedReport + if status == ReadyToExecute { + readyMessages[i] = struct{}{} + } + } + // Attempt to include all messages in the report. finalReport, err := - buildSingleChainReportHelper(b.ctx, b.lggr, b.hasher, b.tokenDataReader, report, nil) + buildSingleChainReportHelper(b.ctx, b.lggr, b.hasher, report, readyMessages) if err != nil { return cciptypes.ExecutePluginReportSingleChain{}, plugintypes.ExecutePluginCommitData{}, @@ -237,20 +282,13 @@ func (b *execReportBuilder) buildSingleChainReport( finalReport = cciptypes.ExecutePluginReportSingleChain{} msgs := make(map[int]struct{}) for i := range report.Messages { - status, err := b.checkMessage(ctx, i, report) - if err != nil { - return cciptypes.ExecutePluginReportSingleChain{}, - plugintypes.ExecutePluginCommitData{}, - fmt.Errorf("unable to check message: %w", err) - } - if status != ReadyToExecute { + if _, ok := readyMessages[i]; !ok { continue } msgs[i] = struct{}{} - finalReport2, err := - buildSingleChainReportHelper(b.ctx, b.lggr, b.hasher, b.tokenDataReader, report, msgs) + finalReport2, err := buildSingleChainReportHelper(b.ctx, b.lggr, b.hasher, report, msgs) if err != nil { return cciptypes.ExecutePluginReportSingleChain{}, plugintypes.ExecutePluginCommitData{}, diff --git a/execute/report/report_test.go b/execute/report/report_test.go index 1eb080fc8..75afcf3db 100644 --- a/execute/report/report_test.go +++ b/execute/report/report_test.go @@ -4,12 +4,14 @@ import ( "context" "fmt" "math/rand" + "reflect" "strings" "testing" "time" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" + "go.uber.org/zap/zapcore" "github.com/smartcontractkit/chainlink-common/pkg/hashutil" "github.com/smartcontractkit/chainlink-common/pkg/logger" @@ -240,60 +242,39 @@ func setMessageData( return commitReport } -// TODO: better than this -type tdr struct{} - -func (t tdr) ReadTokenData( - ctx context.Context, srcChain cciptypes.ChainSelector, num cciptypes.SeqNum) ([][]byte, error, -) { - return nil, nil -} - type badHasher struct{} func (bh badHasher) Hash(context.Context, cciptypes.Message) (cciptypes.Bytes32, error) { return cciptypes.Bytes32{}, fmt.Errorf("bad hasher") } -type badTokenDataReader struct{} - -func (btdr badTokenDataReader) ReadTokenData( - _ context.Context, _ cciptypes.ChainSelector, _ cciptypes.SeqNum, -) ([][]byte, error) { - return nil, fmt.Errorf("bad token data reader") -} - -/* -// TODO: Use this to test the verifyReport function. -type badCodec struct{} - -func (bc badCodec) Encode(ctx context.Context, report cciptypes.ExecutePluginReport) ([]byte, error) { - return nil, fmt.Errorf("bad codec") -} - -func (bc badCodec) Decode(ctx context.Context, bytes []byte) (cciptypes.ExecutePluginReport, error) { - return cciptypes.ExecutePluginReport{}, fmt.Errorf("bad codec") -} -*/ - func Test_buildSingleChainReport_Errors(t *testing.T) { lggr := logger.Test(t) type args struct { - report plugintypes.ExecutePluginCommitData - hasher cciptypes.MessageHasher - tokenDataReader types.TokenDataReader + report plugintypes.ExecutePluginCommitData + hasher cciptypes.MessageHasher } tests := []struct { name string args args wantErr string }{ + { + name: "token data mismatch", + wantErr: "token data length mismatch: got 2, expected 0", + args: args{ + report: plugintypes.ExecutePluginCommitData{ + TokenData: make([][][]byte, 2), + }, + }, + }, { name: "wrong number of messages", wantErr: "unexpected number of messages: expected 1, got 2", args: args{ report: plugintypes.ExecutePluginCommitData{ + TokenData: make([][][]byte, 2), SequenceNumberRange: cciptypes.NewSeqNumRange(cciptypes.SeqNum(100), cciptypes.SeqNum(100)), Messages: []cciptypes.Message{ {Header: cciptypes.RampMessageHeader{}}, @@ -307,6 +288,7 @@ func Test_buildSingleChainReport_Errors(t *testing.T) { wantErr: "sequence number 102 outside of report range [100 -> 101]", args: args{ report: plugintypes.ExecutePluginCommitData{ + TokenData: make([][][]byte, 2), SequenceNumberRange: cciptypes.NewSeqNumRange(cciptypes.SeqNum(100), cciptypes.SeqNum(101)), Messages: []cciptypes.Message{ { @@ -328,13 +310,16 @@ func Test_buildSingleChainReport_Errors(t *testing.T) { wantErr: "unexpected source chain: expected 1111, got 2222", args: args{ report: plugintypes.ExecutePluginCommitData{ + TokenData: make([][][]byte, 1), SourceChain: 1111, SequenceNumberRange: cciptypes.NewSeqNumRange(cciptypes.SeqNum(100), cciptypes.SeqNum(100)), Messages: []cciptypes.Message{ - {Header: cciptypes.RampMessageHeader{ - SourceChainSelector: 2222, - SequenceNumber: cciptypes.SeqNum(100), - }}, + { + Header: cciptypes.RampMessageHeader{ + SourceChainSelector: 2222, + SequenceNumber: cciptypes.SeqNum(100), + }, + }, }, }, hasher: badHasher{}, @@ -345,35 +330,21 @@ func Test_buildSingleChainReport_Errors(t *testing.T) { wantErr: "unable to hash message (1234567, 100): bad hasher", args: args{ report: plugintypes.ExecutePluginCommitData{ + TokenData: make([][][]byte, 1), SourceChain: 1234567, SequenceNumberRange: cciptypes.NewSeqNumRange(cciptypes.SeqNum(100), cciptypes.SeqNum(100)), Messages: []cciptypes.Message{ - {Header: cciptypes.RampMessageHeader{ - SourceChainSelector: 1234567, - SequenceNumber: cciptypes.SeqNum(100), - }}, + { + Header: cciptypes.RampMessageHeader{ + SourceChainSelector: 1234567, + SequenceNumber: cciptypes.SeqNum(100), + }, + }, }, }, hasher: badHasher{}, }, }, - { - name: "bad token data reader", - wantErr: "unable to read token data for message 100: bad token data reader", - args: args{ - report: plugintypes.ExecutePluginCommitData{ - SourceChain: 1234567, - SequenceNumberRange: cciptypes.NewSeqNumRange(cciptypes.SeqNum(100), cciptypes.SeqNum(100)), - Messages: []cciptypes.Message{ - {Header: cciptypes.RampMessageHeader{ - SourceChainSelector: 1234567, - SequenceNumber: cciptypes.SeqNum(100), - }}, - }, - }, - tokenDataReader: badTokenDataReader{}, - }, - }, } for _, tt := range tests { tt := tt @@ -388,23 +359,21 @@ func Test_buildSingleChainReport_Errors(t *testing.T) { resolvedHasher = mocks.NewMessageHasher() } - // Select token data reader mock. - var resolvedTokenDataReader types.TokenDataReader - if tt.args.tokenDataReader != nil { - resolvedTokenDataReader = tt.args.tokenDataReader - } else { - resolvedTokenDataReader = tdr{} - } - ctx := context.Background() msgs := make(map[int]struct{}) for i := 0; i < len(tt.args.report.Messages); i++ { msgs[i] = struct{}{} } - _, err := buildSingleChainReportHelper( - ctx, lggr, resolvedHasher, resolvedTokenDataReader, tt.args.report, msgs) - require.Error(t, err) - assert.Contains(t, err.Error(), tt.wantErr) + + test := func(readyMessages map[int]struct{}) { + _, err := buildSingleChainReportHelper(ctx, lggr, resolvedHasher, tt.args.report, readyMessages) + require.Error(t, err) + assert.Contains(t, err.Error(), tt.wantErr) + } + + // Test with pre-built and all messages. + test(msgs) + test(nil) }) } } @@ -413,7 +382,7 @@ func Test_Builder_Build(t *testing.T) { hasher := mocks.NewMessageHasher() codec := mocks.NewExecutePluginJSONReportCodec() lggr := logger.Test(t) - var tokenDataReader tdr + tokenDataReader := tdr{mode: good} type args struct { reports []plugintypes.ExecutePluginCommitData @@ -681,3 +650,367 @@ func Test_Builder_Build(t *testing.T) { }) } } + +type badCodec struct{} + +func (bc badCodec) Encode(ctx context.Context, report cciptypes.ExecutePluginReport) ([]byte, error) { + return nil, fmt.Errorf("bad codec") +} + +func (bc badCodec) Decode(ctx context.Context, bytes []byte) (cciptypes.ExecutePluginReport, error) { + return cciptypes.ExecutePluginReport{}, fmt.Errorf("bad codec") +} + +func Test_execReportBuilder_verifyReport(t *testing.T) { + type fields struct { + encoder cciptypes.ExecutePluginCodec + maxReportSizeBytes uint64 + accumulated validationMetadata + } + type args struct { + execReport cciptypes.ExecutePluginReportSingleChain + } + tests := []struct { + name string + fields fields + args args + expectedLog string + expectedIsValid bool + expectedMetadata validationMetadata + expectedError string + }{ + { + name: "empty report", + args: args{ + execReport: cciptypes.ExecutePluginReportSingleChain{}, + }, + fields: fields{ + maxReportSizeBytes: 1000, + }, + expectedIsValid: true, + expectedMetadata: validationMetadata{ + encodedSizeBytes: 120, + }, + }, + { + name: "good report", + args: args{ + execReport: cciptypes.ExecutePluginReportSingleChain{ + Messages: []cciptypes.Message{ + makeMessage(1, 100, 0), + makeMessage(1, 101, 0), + makeMessage(1, 102, 0), + makeMessage(1, 103, 0), + }, + }, + }, + fields: fields{ + maxReportSizeBytes: 10000, + }, + expectedIsValid: true, + expectedMetadata: validationMetadata{ + encodedSizeBytes: 1633, + }, + }, + { + name: "oversized report", + args: args{ + execReport: cciptypes.ExecutePluginReportSingleChain{ + Messages: []cciptypes.Message{ + makeMessage(1, 100, 0), + makeMessage(1, 101, 0), + makeMessage(1, 102, 0), + makeMessage(1, 103, 0), + }, + }, + }, + fields: fields{ + maxReportSizeBytes: 1000, + }, + expectedLog: "invalid report, report size exceeds limit", + }, + { + name: "oversized report - accumulated size", + args: args{ + execReport: cciptypes.ExecutePluginReportSingleChain{ + Messages: []cciptypes.Message{ + makeMessage(1, 100, 0), + makeMessage(1, 101, 0), + makeMessage(1, 102, 0), + makeMessage(1, 103, 0), + }, + }, + }, + fields: fields{ + accumulated: validationMetadata{ + encodedSizeBytes: 1000, + }, + maxReportSizeBytes: 2000, + }, + expectedLog: "invalid report, report size exceeds limit", + }, + { + name: "bad token data reader", + args: args{ + execReport: cciptypes.ExecutePluginReportSingleChain{ + Messages: []cciptypes.Message{ + makeMessage(1, 100, 0), + makeMessage(1, 101, 0), + }, + }, + }, + fields: fields{ + encoder: badCodec{}, + }, + expectedError: "unable to encode report", + expectedLog: "unable to encode report", + }, + } + for _, tt := range tests { + tt := tt + lggr, logs := logger.TestObserved(t, zapcore.DebugLevel) + t.Run(tt.name, func(t *testing.T) { + t.Parallel() + + // Select token data reader mock. + var resolvedEncoder cciptypes.ExecutePluginCodec + if tt.fields.encoder != nil { + resolvedEncoder = tt.fields.encoder + } else { + resolvedEncoder = mocks.NewExecutePluginJSONReportCodec() + } + + b := &execReportBuilder{ + ctx: context.Background(), + lggr: lggr, + encoder: resolvedEncoder, + maxReportSizeBytes: tt.fields.maxReportSizeBytes, + accumulated: tt.fields.accumulated, + } + isValid, metadata, err := b.verifyReport(context.Background(), tt.args.execReport) + if tt.expectedError != "" { + assert.Contains(t, err.Error(), tt.expectedError) + return + } + if tt.expectedLog != "" { + found := false + for _, log := range logs.All() { + fmt.Println(log.Message) + found = found || strings.Contains(log.Message, tt.expectedLog) + } + assert.True(t, found, "expected log not found") + } + assert.Equalf(t, tt.expectedIsValid, isValid, "verifyReport(...)") + assert.Equalf(t, tt.expectedMetadata, metadata, "verifyReport(...)") + }) + } +} + +type tdr struct { + mode tokenDataMode +} + +type tokenDataMode int + +const ( + noop tokenDataMode = iota + 1 + good + bad + notReady +) + +func (t tdr) ReadTokenData( + ctx context.Context, srcChain cciptypes.ChainSelector, num cciptypes.SeqNum) ([][]byte, error, +) { + switch t.mode { + case noop: + return nil, nil + case good: + return [][]byte{{0x01, 0x02, 0x03}}, nil + case bad: + return nil, fmt.Errorf("bad token data reader") + case notReady: + return nil, ErrNotReady + default: + panic("mode should be one of the valid ones.") + } +} + +func Test_execReportBuilder_checkMessage(t *testing.T) { + type fields struct { + tokenDataReader types.TokenDataReader + } + type args struct { + idx int + execReport plugintypes.ExecutePluginCommitData + } + tests := []struct { + name string + fields fields + args args + expectedData plugintypes.ExecutePluginCommitData + expectedStatus messageStatus + expectedMetadata validationMetadata + expectedError string + expectedLog string + }{ + { + name: "empty", + expectedError: "message index out of range", + expectedLog: "message index out of range", + }, + { + name: "already executed", + args: args{ + idx: 0, + execReport: plugintypes.ExecutePluginCommitData{ + Messages: []cciptypes.Message{ + makeMessage(1, 100, 0), + }, + ExecutedMessages: []cciptypes.SeqNum{100}, + }, + }, + expectedStatus: AlreadyExecuted, + expectedLog: "message already executed", + }, + { + name: "bad token data", + args: args{ + idx: 0, + execReport: plugintypes.ExecutePluginCommitData{ + Messages: []cciptypes.Message{ + makeMessage(1, 100, 0), + }, + }, + }, + fields: fields{ + tokenDataReader: tdr{mode: bad}, + }, + expectedStatus: TokenDataFetchError, + expectedLog: "unable to read token data - unknown error", + }, + { + name: "token data not ready", + args: args{ + idx: 0, + execReport: plugintypes.ExecutePluginCommitData{ + Messages: []cciptypes.Message{ + makeMessage(1, 100, 0), + }, + }, + }, + fields: fields{ + tokenDataReader: tdr{mode: notReady}, + }, + expectedStatus: TokenDataNotReady, + expectedLog: "unable to read token data - token data not ready", + }, + { + name: "good token data is cached", + args: args{ + idx: 0, + execReport: plugintypes.ExecutePluginCommitData{ + Messages: []cciptypes.Message{ + makeMessage(1, 100, 0), + }, + }, + }, + fields: fields{ + tokenDataReader: tdr{mode: good}, + }, + expectedStatus: ReadyToExecute, + expectedData: plugintypes.ExecutePluginCommitData{ + Messages: []cciptypes.Message{ + makeMessage(1, 100, 0), + }, + TokenData: [][][]byte{ + {{0x01, 0x02, 0x03}}, + }, + }, + }, + { + name: "good - no token data - 1 msg", + args: args{ + idx: 0, + execReport: plugintypes.ExecutePluginCommitData{ + Messages: []cciptypes.Message{ + makeMessage(1, 100, 0), + }, + }, + }, + fields: fields{ + tokenDataReader: tdr{mode: noop}, + }, + expectedStatus: ReadyToExecute, + expectedData: plugintypes.ExecutePluginCommitData{ + Messages: []cciptypes.Message{ + makeMessage(1, 100, 0), + }, + TokenData: [][][]byte{nil}, + }, + }, + { + name: "good - no token data - 2nd msg pads slice", + args: args{ + idx: 1, + execReport: plugintypes.ExecutePluginCommitData{ + Messages: []cciptypes.Message{ + makeMessage(1, 100, 0), + makeMessage(1, 101, 0), + }, + }, + }, + fields: fields{ + tokenDataReader: tdr{mode: noop}, + }, + expectedStatus: ReadyToExecute, + expectedData: plugintypes.ExecutePluginCommitData{ + Messages: []cciptypes.Message{ + makeMessage(1, 100, 0), + makeMessage(1, 101, 0), + }, + TokenData: [][][]byte{nil, nil}, + }, + }, + } + for _, tt := range tests { + tt := tt + t.Run(tt.name, func(t *testing.T) { + t.Parallel() + lggr, logs := logger.TestObserved(t, zapcore.DebugLevel) + + // Select token data reader mock. + var resolvedTokenDataReader types.TokenDataReader + if tt.fields.tokenDataReader != nil { + resolvedTokenDataReader = tt.fields.tokenDataReader + } else { + resolvedTokenDataReader = tdr{mode: good} + } + + b := &execReportBuilder{ + lggr: lggr, + tokenDataReader: resolvedTokenDataReader, + } + data, status, err := b.checkMessage(context.Background(), tt.args.idx, tt.args.execReport) + if tt.expectedError != "" { + assert.Contains(t, err.Error(), tt.expectedError) + return + } + if tt.expectedLog != "" { + found := false + for _, log := range logs.All() { + fmt.Println(log.Message) + found = found || strings.Contains(log.Message, tt.expectedLog) + } + assert.True(t, found, "expected log not found") + } + assert.Equalf(t, tt.expectedStatus, status, "checkMessage(...)") + // If expected data not provided, we expect the result to be the same as the input. + if reflect.DeepEqual(tt.expectedData, plugintypes.ExecutePluginCommitData{}) { + assert.Equalf(t, tt.args.execReport, data, "checkMessage(...)") + } else { + assert.Equalf(t, tt.expectedData, data, "checkMessage(...)") + } + }) + } +} diff --git a/plugintypes/execute.go b/plugintypes/execute.go index 8409b0b19..1e52b4dd7 100644 --- a/plugintypes/execute.go +++ b/plugintypes/execute.go @@ -31,9 +31,8 @@ type ExecutePluginCommitData struct { // ExecutedMessages are the messages in this report that have already been executed. ExecutedMessages []cciptypes.SeqNum `json:"executedMessages"` - // TODO: cache for token data. // TokenData for each message. - // TokenData [][][]byte `json:"-"` + TokenData [][][]byte `json:"-"` } type ExecutePluginCommitObservations map[cciptypes.ChainSelector][]ExecutePluginCommitData