diff --git a/node/cmd/guardiand/node.go b/node/cmd/guardiand/node.go index 208d355317..d7af5d7c22 100644 --- a/node/cmd/guardiand/node.go +++ b/node/cmd/guardiand/node.go @@ -65,7 +65,6 @@ var ( guardianKeyPath *string guardianSignerUri *string - solanaContract *string ethRPC *string ethContract *string @@ -153,7 +152,9 @@ var ( suiRPC *string suiMoveEventType *string - solanaRPC *string + solanaRPC *string + solanaContract *string + solanaShimContract *string pythnetContract *string pythnetRPC *string @@ -292,7 +293,8 @@ func init() { guardianKeyPath = NodeCmd.Flags().String("guardianKey", "", "Path to guardian key") guardianSignerUri = NodeCmd.Flags().String("guardianSignerUri", "", "Guardian signer URI") - solanaContract = NodeCmd.Flags().String("solanaContract", "", "Address of the Solana program (required)") + solanaContract = NodeCmd.Flags().String("solanaContract", "", "Address of the Solana program (required if solanaRpc is specified)") + solanaShimContract = NodeCmd.Flags().String("solanaShimContract", "", "Address of the Solana shim program") ethRPC = node.RegisterFlagWithValidationOrFail(NodeCmd, "ethRPC", "Ethereum RPC URL", "ws://eth-devnet:8545", []string{"ws", "wss"}) ethContract = NodeCmd.Flags().String("ethContract", "", "Ethereum contract address") @@ -847,6 +849,10 @@ func runNode(cmd *cobra.Command, args []string) { logger.Fatal("Both --solanaContract and --solanaRPC must be set or both unset") } + if *solanaShimContract != "" && *solanaContract == "" { + logger.Fatal("--solanaShimContract may only be specified if --solanaContract is specified") + } + if !argsConsistent([]string{*pythnetContract, *pythnetRPC, *pythnetWS}) { logger.Fatal("Either --pythnetContract, --pythnetRPC and --pythnetWS must all be set or all unset") } @@ -1589,6 +1595,7 @@ func runNode(cmd *cobra.Command, args []string) { Rpc: *solanaRPC, Websocket: "", Contract: *solanaContract, + ShimContract: *solanaShimContract, ReceiveObsReq: false, Commitment: rpc.CommitmentConfirmed, } @@ -1602,6 +1609,7 @@ func runNode(cmd *cobra.Command, args []string) { Rpc: *solanaRPC, Websocket: "", Contract: *solanaContract, + ShimContract: *solanaShimContract, ReceiveObsReq: true, Commitment: rpc.CommitmentFinalized, } diff --git a/node/pkg/watchers/solana/client.go b/node/pkg/watchers/solana/client.go index 6dae0a2506..fe7ea02bcf 100644 --- a/node/pkg/watchers/solana/client.go +++ b/node/pkg/watchers/solana/client.go @@ -76,6 +76,12 @@ type ( ccqConfig query.PerChainConfig ccqLogger *zap.Logger + + shimContractStr string + shimContractAddr solana.PublicKey + shimEnabled bool + shimPostMessageDiscriminator []byte + shimMessageEventDiscriminator []byte } EventSubscriptionError struct { @@ -221,6 +227,8 @@ func NewSolanaWatcher( chainID vaa.ChainID, queryReqC <-chan *query.PerChainQueryInternal, queryResponseC chan<- *query.PerChainQueryResponseInternal, + shimContractStr string, + shimContractAddr solana.PublicKey, ) *SolanaWatcher { msgObservedLogLevel := zapcore.InfoLevel if chainID == vaa.ChainIDPythNet { @@ -243,6 +251,8 @@ func NewSolanaWatcher( queryReqC: queryReqC, queryResponseC: queryResponseC, ccqConfig: query.GetPerChainConfig(chainID), + shimContractStr: shimContractStr, + shimContractAddr: shimContractAddr, } } @@ -334,10 +344,13 @@ func (s *SolanaWatcher) Run(ctx context.Context) error { zap.String("wsUrl", wsUrl), zap.String("contract", contractAddr), zap.String("rawContract", s.rawContract), + zap.String("shimContractAddr", s.shimContractStr), ) logger.Info("Solana watcher connecting to RPC node ", zap.String("url", s.rpcUrl)) + s.shimSetup() + s.errC = make(chan error) s.pumpData = make(chan []byte) @@ -561,6 +574,7 @@ func (s *SolanaWatcher) fetchBlock(ctx context.Context, logger *zap.Logger, slot // If the logs don't contain the contract address, skip the transaction. // ex: "Program 3u8hJUVTA4jH1wYAyUur7FFZVQ8H635K3tSHHF4ssjQ5 invoke [2]", + // Assumption: Transactions for the shim contract also contain the core contract address so this check is still valid. var possiblyWormhole bool for i := 0; i < len(txRpc.Meta.LogMessages) && !possiblyWormhole; i++ { possiblyWormhole = strings.HasPrefix(txRpc.Meta.LogMessages[i], s.whLogPrefix) @@ -592,10 +606,16 @@ func (s *SolanaWatcher) fetchBlock(ctx context.Context, logger *zap.Logger, slot signature := tx.Signatures[0] var programIndex uint16 + var shimProgramIndex uint16 + var shimFound bool for n, key := range tx.Message.AccountKeys { if key.Equals(s.contract) { programIndex = uint16(n) } + if s.shimEnabled && key.Equals(s.shimContractAddr) { + shimProgramIndex = uint16(n) + shimFound = true + } } if programIndex == 0 { continue @@ -608,30 +628,30 @@ func (s *SolanaWatcher) fetchBlock(ctx context.Context, logger *zap.Logger, slot zap.String("commitment", string(s.commitment))) } + alreadyProcessed := ShimAlreadyProcessed{} + // Find top-level instructions for i, inst := range tx.Message.Instructions { - found, err := s.processInstruction(ctx, logger, slot, inst, programIndex, tx, signature, i, isReobservation) - if err != nil { - logger.Error("malformed Wormhole instruction", - zap.Error(err), - zap.Int("idx", i), - zap.Stringer("signature", signature), - zap.Uint64("slot", slot), - zap.String("commitment", string(s.commitment)), - zap.Binary("data", inst.Data)) - } else if found { - if logger.Level().Enabled(zapcore.DebugLevel) { - logger.Debug("found a top-level Wormhole instruction", + if shimFound && inst.ProgramIDIndex == shimProgramIndex { + found, err := s.shimProcessTopLevelInstruction(logger, programIndex, shimProgramIndex, tx, txRpc.Meta.InnerInstructions, i, alreadyProcessed, isReobservation) + if err != nil { + logger.Error("malformed wormhole shim instruction", + zap.Error(err), zap.Int("idx", i), zap.Stringer("signature", signature), zap.Uint64("slot", slot), - zap.String("commitment", string(s.commitment))) + zap.String("commitment", string(s.commitment)), + zap.Binary("data", inst.Data)) + } else if found { + if logger.Level().Enabled(zapcore.DebugLevel) { + logger.Debug("found a top-level wormhole shim instruction", + zap.Int("idx", i), + zap.Stringer("signature", signature), + zap.Uint64("slot", slot), + zap.String("commitment", string(s.commitment))) + } } - } - } - - for _, inner := range txRpc.Meta.InnerInstructions { - for i, inst := range inner.Instructions { + } else { found, err := s.processInstruction(ctx, logger, slot, inst, programIndex, tx, signature, i, isReobservation) if err != nil { logger.Error("malformed Wormhole instruction", @@ -639,10 +659,11 @@ func (s *SolanaWatcher) fetchBlock(ctx context.Context, logger *zap.Logger, slot zap.Int("idx", i), zap.Stringer("signature", signature), zap.Uint64("slot", slot), - zap.String("commitment", string(s.commitment))) + zap.String("commitment", string(s.commitment)), + zap.Binary("data", inst.Data)) } else if found { if logger.Level().Enabled(zapcore.DebugLevel) { - logger.Debug("found an inner Wormhole instruction", + logger.Debug("found a top-level Wormhole instruction", zap.Int("idx", i), zap.Stringer("signature", signature), zap.Uint64("slot", slot), @@ -651,6 +672,54 @@ func (s *SolanaWatcher) fetchBlock(ctx context.Context, logger *zap.Logger, slot } } } + + for outerIdx, inner := range txRpc.Meta.InnerInstructions { + for innerIdx, inst := range inner.Instructions { + if !alreadyProcessed.exists(outerIdx, innerIdx) { + if shimFound && inst.ProgramIDIndex == shimProgramIndex { + found, err := s.shimProcessInnerInstruction(logger, programIndex, shimProgramIndex, tx, inner.Instructions, outerIdx, innerIdx, alreadyProcessed, isReobservation) + if err != nil { + logger.Error("malformed inner wormhole shim instruction", + zap.Error(err), + zap.Int("outerIdx", outerIdx), + zap.Int("innerIdx", innerIdx), + zap.Stringer("signature", signature), + zap.Uint64("slot", slot), + zap.String("commitment", string(s.commitment))) + } else if found { + if logger.Level().Enabled(zapcore.DebugLevel) { + logger.Debug("found an inner wormhole shim instruction", + zap.Int("outerIdx", outerIdx), + zap.Int("innerIdx", innerIdx), + zap.Stringer("signature", signature), + zap.Uint64("slot", slot), + zap.String("commitment", string(s.commitment))) + } + } + } else { + found, err := s.processInstruction(ctx, logger, slot, inst, programIndex, tx, signature, innerIdx, isReobservation) + if err != nil { + logger.Error("malformed Wormhole instruction", + zap.Error(err), + zap.Int("outerIdx", outerIdx), + zap.Int("innerIdx", innerIdx), + zap.Stringer("signature", signature), + zap.Uint64("slot", slot), + zap.String("commitment", string(s.commitment))) + } else if found { + if logger.Level().Enabled(zapcore.DebugLevel) { + logger.Debug("found an inner Wormhole instruction", + zap.Int("outerIdx", outerIdx), + zap.Int("innerIdx", innerIdx), + zap.Stringer("signature", signature), + zap.Uint64("slot", slot), + zap.String("commitment", string(s.commitment))) + } + } + } + } + } + } } if emptyRetry > 0 && logger.Level().Enabled(zapcore.DebugLevel) { @@ -933,6 +1002,21 @@ func (s *SolanaWatcher) processMessageAccount(logger *zap.Logger, data []byte, a Unreliable: !reliable, } + if !reliable && len(observation.Payload) == 0 { + logger.Debug("ignoring an observation because it is marked unreliable and has a zero length payload, probably from the shim", + zap.Stringer("account", acc), + zap.Time("timestamp", observation.Timestamp), + zap.Uint32("nonce", observation.Nonce), + zap.Uint64("sequence", observation.Sequence), + zap.Stringer("emitter_chain", observation.EmitterChain), + zap.Stringer("emitter_address", observation.EmitterAddress), + zap.Bool("isReobservation", isReobservation), + zap.Binary("payload", observation.Payload), + zap.Uint8("consistency_level", observation.ConsistencyLevel), + ) + return + } + solanaMessagesConfirmed.WithLabelValues(s.networkName).Inc() if logger.Level().Enabled(s.msgObservedLogLevel) { diff --git a/node/pkg/watchers/solana/config.go b/node/pkg/watchers/solana/config.go index 0abca5af0f..00123faa8a 100644 --- a/node/pkg/watchers/solana/config.go +++ b/node/pkg/watchers/solana/config.go @@ -7,6 +7,7 @@ import ( "github.com/certusone/wormhole/node/pkg/supervisor" "github.com/certusone/wormhole/node/pkg/watchers" "github.com/certusone/wormhole/node/pkg/watchers/interfaces" + "github.com/gagliardetto/solana-go" solana_types "github.com/gagliardetto/solana-go" solana_rpc "github.com/gagliardetto/solana-go/rpc" "github.com/wormhole-foundation/wormhole/sdk/vaa" @@ -19,6 +20,7 @@ type WatcherConfig struct { Rpc string // RPC URL Websocket string // Websocket URL Contract string // hex representation of the contract address + ShimContract string // Address of the shim contract (empty string if disabled) Commitment solana_rpc.CommitmentType } @@ -51,11 +53,19 @@ func (wc *WatcherConfig) Create( return nil, nil, err } + var shimContractAddr solana.PublicKey + if wc.ShimContract != "" { + shimContractAddr, err = solana_types.PublicKeyFromBase58(wc.Contract) + if err != nil { + return nil, nil, err + } + } + if !wc.ReceiveObsReq { obsvReqC = nil } - watcher := NewSolanaWatcher(wc.Rpc, &wc.Websocket, solAddress, wc.Contract, msgC, obsvReqC, wc.Commitment, wc.ChainID, queryReqC, queryResponseC) + watcher := NewSolanaWatcher(wc.Rpc, &wc.Websocket, solAddress, wc.Contract, msgC, obsvReqC, wc.Commitment, wc.ChainID, queryReqC, queryResponseC, wc.ShimContract, shimContractAddr) return watcher, watcher.Run, nil } diff --git a/node/pkg/watchers/solana/shim.go b/node/pkg/watchers/solana/shim.go new file mode 100644 index 0000000000..9908b3a5a5 --- /dev/null +++ b/node/pkg/watchers/solana/shim.go @@ -0,0 +1,301 @@ +package solana + +// TODO: Need to change MessagePublication.TxHash to be a byte array rather than a ethCommon.hash. +// TODO: Need to make sure reobservations of shim events works. + +import ( + "bytes" + "encoding/hex" + "errors" + "fmt" + "time" + + "github.com/certusone/wormhole/node/pkg/common" + eth_common "github.com/ethereum/go-ethereum/common" + "github.com/gagliardetto/solana-go" + "github.com/gagliardetto/solana-go/rpc" + "github.com/near/borsh-go" + "go.uber.org/zap" +) + +const ( + shimPostMessageDiscriminatorStr = "d63264d12622074c" + shimMessageEventDiscriminatorStr = "e445a52e51cb9a1d441b8f004d4c8970" +) + +// ShimPostMessageData defines the shim PostMessage payload following the eight byte discriminator (shimPostMessageDiscriminatorStr) +type ShimPostMessageData struct { + Nonce uint32 + ConsistencyLevel ConsistencyLevel + Payload []byte +} + +// ShimMessageEventData defines the shim MessageEvent payload following the sixteen byte discriminator (shimMessageEventDiscriminatorStr) +type ShimMessageEventData struct { + EmitterAddress [32]byte + Sequence uint64 + Timestamp uint32 +} + +// shimSetup performs any initialization that is specific to monitoring the shim contract. +func (s *SolanaWatcher) shimSetup() { + s.shimEnabled = s.shimContractStr != "" + if s.shimEnabled { + var err error + s.shimPostMessageDiscriminator, err = hex.DecodeString(shimPostMessageDiscriminatorStr) + if err != nil { + panic("failed to decode shim post message discriminator") + } + s.shimMessageEventDiscriminator, err = hex.DecodeString(shimMessageEventDiscriminatorStr) + if err != nil { + panic("failed to decode shim post message discriminator") + } + } +} + +// shimMatchPrefix verifies that the instruction data starts with the specified prefix bytes. +func shimMatchPrefix(discriminator []byte, buf []byte) bool { + if len(buf) < len(discriminator) { + return false + } + return bytes.Equal(discriminator, buf[:len(discriminator)]) +} + +// shimParsePostMessage parses a shim PostMessage and returns the results. +func shimParsePostMessage(shimPostMessageDiscriminator []byte, buf []byte) (*ShimPostMessageData, error) { + if len(buf) <= 8 { + return nil, errors.New("payload too short") + } + + if !shimMatchPrefix(shimPostMessageDiscriminator, buf) { + return nil, nil + } + + data := new(ShimPostMessageData) + if err := borsh.Deserialize(data, buf[8:]); err != nil { + return nil, fmt.Errorf("failed to deserialize shim post message: %w", err) + } + + return data, nil +} + +// shimVerifyCoreMessage verifies that an instruction from the core contract is what we expect to accompany a shim instruction. +// This includes being marked unreliable and having a zero length payload. +func shimVerifyCoreMessage(buf []byte) (bool, error) { + if len(buf) == 0 { + return false, nil + } + + if buf[0] != postMessageUnreliableInstructionID { + return false, nil + } + + var data PostMessageData + if err := borsh.Deserialize(&data, buf[1:]); err != nil { + return false, fmt.Errorf("failed to deserialize core instruction data: %w", err) + } + + if len(data.Payload) != 0 { + return false, nil + } + + return true, nil +} + +// shimParseMessageEvent parses a shim MessageEvent and returns the results. +func shimParseMessageEvent(shimMessageEvent []byte, buf []byte) (*ShimMessageEventData, error) { + if len(buf) <= 16 { + return nil, errors.New("payload too short") + } + + if !shimMatchPrefix(shimMessageEvent, buf) { + return nil, nil + } + + data := new(ShimMessageEventData) + if err := borsh.Deserialize(data, buf[16:]); err != nil { + return nil, fmt.Errorf("failed to deserialize shim message event: %w", err) + } + + return data, nil +} + +// shimProcessTopLevelInstruction handles a top-level instruction where the program ID matches the shim contract. It does the following: +// - Verifies that the instruction is a shim PostMessage. If not, it just returns. If it is, it parses it. +// - Searches the sets of inner instructions to find the ones generated by this top-level instruction (by matching the index). +// - Searches through those inner instructions to find the entry for the core. Makes sure it is unreliable with no payload. +// - Searches for the inner shim MessageEvent to get the remaining fields needed to generate the observation. +// - Publishes the observation. +func (s *SolanaWatcher) shimProcessTopLevelInstruction( + logger *zap.Logger, + whProgramIndex uint16, + shimProgramIndex uint16, + tx *solana.Transaction, + innerInstructions []rpc.InnerInstruction, + topLevelIndex int, +) (bool, error) { + topLevelIdx := uint16(topLevelIndex) + inst := tx.Message.Instructions[topLevelIdx] + + // The only top-level instruction generated by the shim contract is the PostMessage event. Parse that to get + // the fields we need to generate an observation. + postMessage, err := shimParsePostMessage(s.shimPostMessageDiscriminator, inst.Data) + if err != nil { + return false, fmt.Errorf("failed to parse top-level shim instruction %d: %w", topLevelIdx, err) + } + + if postMessage == nil { + return false, nil + } + + level, err := postMessage.ConsistencyLevel.Commitment() + if err != nil { + return false, fmt.Errorf("failed to determine commitment: %w", err) + } + + if level != s.commitment { + return true, nil + } + + // Find the set of inner instructions that go with this top level instruction by matching the index. + innerInstructionsIdx := -1 + for idx, inner := range innerInstructions { + if inner.Index == topLevelIdx { + innerInstructionsIdx = int(idx) + break + } + } + + if innerInstructionsIdx == -1 { + return false, fmt.Errorf("failed to find inner instructions for top-level shim instruction %d", topLevelIdx) + } + + // Process the inner instructions associated with this shim top-level instruction and produce an observation event. + err = s.shimProcessRest(logger, whProgramIndex, shimProgramIndex, tx, innerInstructions[innerInstructionsIdx].Instructions, topLevelIndex, 0, postMessage) + if err != nil { + return false, fmt.Errorf("failed to process inner instructions for top-level shim instruction %d: %w", topLevelIdx, err) + } + + return true, nil +} + +// shimProcessInnerInstruction handles an inner instruction where the program ID matches the shim contract. It does the following: +// - Verifies that the instruction is a shim PostMessage. If not, it just returns. If it is, it parses it. +// - Searches through the subsequent inner instructions in the set to find the entry for the core. Makes sure it is unreliable with no payload. +// - Searches for the inner shim MessageEvent to get the remaining fields needed to generate the observation. +// - Publishes the observation. +func (s *SolanaWatcher) shimProcessInnerInstruction( + logger *zap.Logger, + whProgramIndex uint16, + shimProgramIndex uint16, + tx *solana.Transaction, + innerInstructions []solana.CompiledInstruction, + outerIdx int, + startIdx int, +) (bool, error) { + // See if this is a PostMessage event from the shim contract. If so, parse it. If not, bail out now. + postMessage, err := shimParsePostMessage(s.shimPostMessageDiscriminator, innerInstructions[startIdx].Data) + if err != nil { + return false, fmt.Errorf("failed to parse inner shim post message instruction %d, %d: %w", outerIdx, startIdx, err) + } + + if postMessage == nil { + return false, nil + } + + level, err := postMessage.ConsistencyLevel.Commitment() + if err != nil { + return false, fmt.Errorf("failed to determine commitment: %w", err) + } + + if level != s.commitment { + return true, nil + } + + err = s.shimProcessRest(logger, whProgramIndex, shimProgramIndex, tx, innerInstructions, outerIdx, startIdx+1, postMessage) + if err != nil { + return false, fmt.Errorf("failed to process inner instructions for top-level shim instruction %d: %w", outerIdx, err) + } + + return true, nil +} + +func (s *SolanaWatcher) shimProcessRest( + logger *zap.Logger, + whProgramIndex uint16, + shimProgramIndex uint16, + tx *solana.Transaction, + innerInstructions []solana.CompiledInstruction, + outerIdx int, + startIdx int, + postMessage *ShimPostMessageData, +) error { + // Loop through the inner instructions after the shim PostMessage and do two things: + // 1) Find the core event and verify it is unreliable with an empty payload. + // 2) Find the shim MessageEvent to get the rest of the fields we need for the observation. + + var verifiedCoreEvent bool + var messageEvent *ShimMessageEventData + var err error + for idx := startIdx; idx < len(innerInstructions); idx++ { + inst := innerInstructions[idx] + if inst.ProgramIDIndex == whProgramIndex { + if verifiedCoreEvent, err = shimVerifyCoreMessage(inst.Data); err != nil { + return fmt.Errorf("failed to verify inner core instruction for top-level shim instruction %d: %w", outerIdx, err) + } + } else if inst.ProgramIDIndex == shimProgramIndex { + messageEvent, err = shimParseMessageEvent(s.shimMessageEventDiscriminator, inst.Data) + if err != nil { + return fmt.Errorf("failed to parse inner shim message event instruction for top-level shim instruction %d: %w", outerIdx, err) + } + } + + if verifiedCoreEvent && messageEvent != nil { + break + } + } + + if !verifiedCoreEvent { + return fmt.Errorf("failed to find inner core instruction for top-level shim instruction %d", outerIdx) + } + + if messageEvent == nil { + return fmt.Errorf("failed to find inner shim message event instruction for top-level shim instruction %d", outerIdx) + } + + var txHash eth_common.Hash + copy(txHash[:], tx.Signatures[0][:]) + + observation := &common.MessagePublication{ + TxHash: txHash, + Timestamp: time.Unix(int64(messageEvent.Timestamp), 0), + Nonce: postMessage.Nonce, + Sequence: messageEvent.Sequence, + EmitterChain: s.chainID, + EmitterAddress: messageEvent.EmitterAddress, + Payload: postMessage.Payload, + ConsistencyLevel: uint8(postMessage.ConsistencyLevel), + IsReobservation: false, + Unreliable: false, + } + + solanaMessagesConfirmed.WithLabelValues(s.networkName).Inc() + + if logger.Level().Enabled(s.msgObservedLogLevel) { + logger.Log(s.msgObservedLogLevel, "message observed from shim", + zap.Stringer("signature", tx.Signatures[0]), + zap.Time("timestamp", observation.Timestamp), + zap.Uint32("nonce", observation.Nonce), + zap.Uint64("sequence", observation.Sequence), + zap.Stringer("emitter_chain", observation.EmitterChain), + zap.Stringer("emitter_address", observation.EmitterAddress), + zap.Bool("isReobservation", false), + zap.Binary("payload", observation.Payload), + zap.Uint8("consistency_level", observation.ConsistencyLevel), + ) + } + + s.msgC <- observation + + return nil +} diff --git a/node/pkg/watchers/solana/shim_test.go b/node/pkg/watchers/solana/shim_test.go new file mode 100644 index 0000000000..5017fe949c --- /dev/null +++ b/node/pkg/watchers/solana/shim_test.go @@ -0,0 +1,1009 @@ +package solana + +import ( + "bytes" + "encoding/hex" + "encoding/json" + "testing" + "time" + + "github.com/certusone/wormhole/node/pkg/common" + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + "github.com/wormhole-foundation/wormhole/sdk/vaa" + "go.uber.org/zap" + + "github.com/gagliardetto/solana-go" + "github.com/gagliardetto/solana-go/rpc" +) + +func TestShimMatchPrefix(t *testing.T) { + type test struct { + input string + result bool + label string + } + tests := []test{ + {input: "d63264d12622074cdeadbeef", result: true, label: "Success"}, + {input: "d6", result: false, label: "Too_short"}, + {input: "", result: false, label: "Empty"}, + {input: shimPostMessageDiscriminatorStr, result: true, label: "Exact_match"}, + {input: "d73264d12622074cdeadbeef", result: false, label: "No_match"}, + } + + shimPostMessage, err := hex.DecodeString(shimPostMessageDiscriminatorStr) + require.NoError(t, err) + + for _, tc := range tests { + t.Run(tc.label, func(t *testing.T) { + buf, err := hex.DecodeString(tc.input) + require.NoError(t, err) + + assert.Equal(t, tc.result, shimMatchPrefix(shimPostMessage, buf)) + }) + } +} + +func Test_shimParsePostMessage(t *testing.T) { + shimPostMessage, err := hex.DecodeString(shimPostMessageDiscriminatorStr) + require.NoError(t, err) + + data, err := hex.DecodeString("d63264d12622074c2a000000010b00000068656c6c6f20776f726c64") + require.NoError(t, err) + + postMsgData, err := shimParsePostMessage(shimPostMessage, data) + require.NoError(t, err) + require.NotNil(t, postMsgData) + + assert.Equal(t, uint32(42), postMsgData.Nonce) + assert.Equal(t, consistencyLevelFinalized, postMsgData.ConsistencyLevel) + assert.Equal(t, 11, len(postMsgData.Payload)) + assert.True(t, bytes.Equal([]byte("hello world"), postMsgData.Payload)) +} + +func Test_shimVerifyCoreMessage(t *testing.T) { + data, err := hex.DecodeString("082a0000000000000001") + require.NoError(t, err) + + coreMsgDataAsExpected, err := shimVerifyCoreMessage(data) + require.NoError(t, err) + assert.True(t, coreMsgDataAsExpected) +} + +func Test_shimParseMessageEvent(t *testing.T) { + shimMessageEvent, err := hex.DecodeString("e445a52e51cb9a1d441b8f004d4c8970") + require.NoError(t, err) + + expectedEmitter, err := hex.DecodeString("041c657e845d65d009d59ceeb1dda172bd6bc9e7ee5a19e56573197cf7fdffde") + require.NoError(t, err) + + data, err := hex.DecodeString("e445a52e51cb9a1d441b8f004d4c8970041c657e845d65d009d59ceeb1dda172bd6bc9e7ee5a19e56573197cf7fdffde00000000000000007c5b8167") + require.NoError(t, err) + + msgEventData, err := shimParseMessageEvent(shimMessageEvent, data) + require.NoError(t, err) + require.NotNil(t, msgEventData) + + assert.True(t, bytes.Equal(expectedEmitter, msgEventData.EmitterAddress[:])) + assert.Equal(t, uint64(0), msgEventData.Sequence) + assert.Equal(t, uint32(1736530812), msgEventData.Timestamp) +} + +func TestShimAlreadyProcessed(t *testing.T) { + alreadyProcessed := ShimAlreadyProcessed{} + assert.False(t, alreadyProcessed.exists(5, 7)) + alreadyProcessed.add(5, 7) + assert.True(t, alreadyProcessed.exists(5, 7)) + assert.False(t, alreadyProcessed.exists(5, 8)) +} + +// WARNING: This only populates a few fields needed by the shim code! +func shimNewWatcherForTest(t *testing.T, msgC chan<- *common.MessagePublication) *SolanaWatcher { + t.Helper() + + rawContract := "worm2ZoG2kUd4vFXhvjh93UUH596ayRfgQ2MgjNMTth" + contractAddress, err := solana.PublicKeyFromBase58(rawContract) + require.NoError(t, err) + + shimContractStr := "EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX" + shimContractAddr, err := solana.PublicKeyFromBase58(shimContractStr) + require.NoError(t, err) + + s := &SolanaWatcher{ + contract: contractAddress, + rawContract: rawContract, + shimContractStr: shimContractStr, + shimContractAddr: shimContractAddr, + chainID: vaa.ChainIDSolana, + commitment: rpc.CommitmentFinalized, + msgC: msgC, + } + + s.shimSetup() + return s +} + +func TestVerifyShimSetup(t *testing.T) { + msgC := make(chan *common.MessagePublication, 10) + s := shimNewWatcherForTest(t, msgC) + assert.True(t, s.shimEnabled) + assert.Equal(t, shimPostMessageDiscriminatorStr, hex.EncodeToString(s.shimPostMessageDiscriminator)) + assert.Equal(t, shimMessageEventDiscriminatorStr, hex.EncodeToString(s.shimMessageEventDiscriminator)) +} + +func TestShimDirect(t *testing.T) { + eventJson := ` + { + "blockTime": 1736530812, + "meta": { + "computeUnitsConsumed": 84252, + "err": null, + "fee": 5000, + "innerInstructions": [ + { + "index": 1, + "instructions": [ + { + "accounts": [1, 3, 0, 4, 0, 2, 8, 5, 9], + "data": "TbyPDfUoyRxsr", + "programIdIndex": 10, + "stackHeight": 2 + }, + { + "accounts": [0, 4], + "data": "3Bxs4NLhqXb3ofom", + "programIdIndex": 5, + "stackHeight": 3 + }, + { + "accounts": [4], + "data": "9krTD1mFP1husSVM", + "programIdIndex": 5, + "stackHeight": 3 + }, + { + "accounts": [4], + "data": "SYXsBvR59WTsF4KEVN8LCQ1X9MekXCGPPNo3Af36taxCQBED", + "programIdIndex": 5, + "stackHeight": 3 + }, + { + "accounts": [0, 3], + "data": "3Bxs4bm7oSCPMeKR", + "programIdIndex": 5, + "stackHeight": 3 + }, + { + "accounts": [3], + "data": "9krTDGKFuDw9nLmM", + "programIdIndex": 5, + "stackHeight": 3 + }, + { + "accounts": [3], + "data": "SYXsBvR59WTsF4KEVN8LCQ1X9MekXCGPPNo3Af36taxCQBED", + "programIdIndex": 5, + "stackHeight": 3 + }, + { + "accounts": [7], + "data": "hTEY7jEqBPdDRkTWweeDPgyCUykRXEQVCUwrYmn4HZo84DdQrTJT2nBMiJFB3jXUVxHVd9mGq7BX9htuAN", + "programIdIndex": 6, + "stackHeight": 2 + } + ] + } + ], + "loadedAddresses": { + "readonly": [], + "writable": [] + }, + "logMessages": [ + "Program 11111111111111111111111111111111 invoke [1]", + "Program 11111111111111111111111111111111 success", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX invoke [1]", + "Program log: Instruction: PostMessage", + "Program worm2ZoG2kUd4vFXhvjh93UUH596ayRfgQ2MgjNMTth invoke [2]", + "Program 11111111111111111111111111111111 invoke [3]", + "Program 11111111111111111111111111111111 success", + "Program 11111111111111111111111111111111 invoke [3]", + "Program 11111111111111111111111111111111 success", + "Program 11111111111111111111111111111111 invoke [3]", + "Program 11111111111111111111111111111111 success", + "Program log: Sequence: 0", + "Program 11111111111111111111111111111111 invoke [3]", + "Program 11111111111111111111111111111111 success", + "Program 11111111111111111111111111111111 invoke [3]", + "Program 11111111111111111111111111111111 success", + "Program 11111111111111111111111111111111 invoke [3]", + "Program 11111111111111111111111111111111 success", + "Program worm2ZoG2kUd4vFXhvjh93UUH596ayRfgQ2MgjNMTth consumed 60384 of 380989 compute units", + "Program worm2ZoG2kUd4vFXhvjh93UUH596ayRfgQ2MgjNMTth success", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX invoke [2]", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX consumed 2000 of 318068 compute units", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX success", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX consumed 84102 of 399850 compute units", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX success" + ], + "postBalances": [ + 499999999997496260, 1057920, 2350640170, 1552080, 946560, 1, 1141440, 0, + 1169280, 1009200, 1141440 + ], + "postTokenBalances": [], + "preBalances": [ + 500000000000000000, 1057920, 2350640070, 0, 0, 1, 1141440, 0, 1169280, + 1009200, 1141440 + ], + "preTokenBalances": [], + "rewards": [], + "status": { + "Ok": null + } + }, + "slot": 3, + "transaction": { + "message": { + "header": { + "numReadonlySignedAccounts": 0, + "numReadonlyUnsignedAccounts": 6, + "numRequiredSignatures": 1 + }, + "accountKeys": [ + "H3kCPjpQDT4hgwWHr9E9pC99rZT2yHAwiwSwku6Bne9", + "2yVjuQwpsvdsrywzsJJVs9Ueh4zayyo5DYJbBNc3DDpn", + "9bFNrXNb2WTx8fMHXCheaZqkLZ3YCCaiqTftHxeintHy", + "9vohBn118ZEctRmuTRvoUZg1B1HGfSH8C5QX6twtUFrJ", + "HeccUHmoyMi5S6nuTcyUBh4w4me3FP541a52ErYJRT8a", + "11111111111111111111111111111111", + "EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX", + "HQS31aApX3DDkuXgSpV9XyDUNtFgQ31pUn5BNWHG2PSp", + "SysvarC1ock11111111111111111111111111111111", + "SysvarRent111111111111111111111111111111111", + "worm2ZoG2kUd4vFXhvjh93UUH596ayRfgQ2MgjNMTth" + ], + "recentBlockhash": "CMqPGm4icRdNuHsWJUK4Kgu4Cbe2nDQkYNqugQkKPa4Y", + "instructions": [ + { + "accounts": [0, 2], + "data": "3Bxs4HanWsHUZCbH", + "programIdIndex": 5, + "stackHeight": null + }, + { + "accounts": [1, 3, 0, 4, 0, 2, 8, 5, 9, 10, 7, 6], + "data": "3Cn8VBJReY7Bku3RduhBfYpk7tiw1R6pKcTWv9R", + "programIdIndex": 6, + "stackHeight": null + } + ], + "indexToProgramIds": {} + }, + "signatures": [ + "3NACxoZLehbdKGjTWZKTTXJPuovyqAih1AD1BrkYj8nzDAtjiQUEaNmhkoU1jcFfoPTAjrvnaLFgTafNWr3fBrdB" + ] + }, + "version": "legacy" + } + ` + + ///////// A bunch of checks to verify we parsed the JSON correctly. + var txRpc rpc.TransactionWithMeta + err := json.Unmarshal([]byte(eventJson), &txRpc) + require.NoError(t, err) + + tx, err := txRpc.GetParsedTransaction() + require.NoError(t, err) + + require.Equal(t, 2, len(tx.Message.Instructions)) + require.Equal(t, 1, len(txRpc.Meta.InnerInstructions)) + + ///////// Now we start the real test. + + logger := zap.NewNop() + msgC := make(chan *common.MessagePublication, 10) + s := shimNewWatcherForTest(t, msgC) + require.True(t, s.shimEnabled) + + var whProgramIndex uint16 + var shimProgramIndex uint16 + var shimFound bool + for n, key := range tx.Message.AccountKeys { + if key.Equals(s.contract) { + whProgramIndex = uint16(n) + } + if key.Equals(s.shimContractAddr) { + shimProgramIndex = uint16(n) + shimFound = true + } + } + + require.Equal(t, uint16(10), whProgramIndex) + require.True(t, shimFound) + require.Equal(t, uint16(6), shimProgramIndex) + + alreadyProcessed := ShimAlreadyProcessed{} + found, err := s.shimProcessTopLevelInstruction(logger, whProgramIndex, shimProgramIndex, tx, txRpc.Meta.InnerInstructions, 1, alreadyProcessed, false) + require.NoError(t, err) + require.True(t, found) + require.Equal(t, 1, len(s.msgC)) + + msg := <-msgC + require.NotNil(t, msg) + + // TODO: Can't check this until we switch MessagePublication.TxHash to be a byte array rather than a hash. + // expectedTxHash, err := vaa.StringToHash("7647cd98fd14c6e3cdfe35bc64bbc476abcdb5ab12e8d31e3151d132ed1e0eeb4595fda4779f69dbe00ff14aadad3fdcf537b88a22f48f3acb7b31f340670506") + // require.NoError(t, err) + + expectedEmitterAddress, err := vaa.StringToAddress("041c657e845d65d009d59ceeb1dda172bd6bc9e7ee5a19e56573197cf7fdffde") + require.NoError(t, err) + + // assert.Equal(t, expectedTxHash, msg.TxHash) + assert.Equal(t, time.Unix(int64(1736530812), 0), msg.Timestamp) + assert.Equal(t, uint32(42), msg.Nonce) + assert.Equal(t, uint64(0), msg.Sequence) + assert.Equal(t, vaa.ChainIDSolana, msg.EmitterChain) + assert.Equal(t, expectedEmitterAddress, msg.EmitterAddress) + assert.Equal(t, uint8(1), msg.ConsistencyLevel) + assert.Equal(t, "hello world", string(msg.Payload)) + assert.False(t, msg.IsReobservation) + assert.False(t, msg.Unreliable) +} + +func TestShimFromIntegrator(t *testing.T) { + eventJson := ` + { + "blockTime": 1736542615, + "meta": { + "computeUnitsConsumed": 48958, + "err": null, + "fee": 5000, + "innerInstructions": [ + { + "index": 1, + "instructions": [ + { + "accounts": [1, 4, 11, 3, 0, 2, 9, 5, 10, 12, 8, 7], + "data": "BeHixXyfSZ8dzFJzxTYRV18L6KSgTuqcTjaqeXgDVbXHC7mCjAgSyhz", + "programIdIndex": 7, + "stackHeight": 2 + }, + { + "accounts": [1, 4, 11, 3, 0, 2, 9, 5, 10], + "data": "T4xyMHqZi66JU", + "programIdIndex": 12, + "stackHeight": 3 + }, + { + "accounts": [8], + "data": "hTEY7jEqBPdDRkTWweeDPgzBpsiybJCHnVTVt8aCDem8p58yeQcQLJWk7hgGHrX79qZyKmCM89vCgPY7SE", + "programIdIndex": 7, + "stackHeight": 3 + } + ] + } + ], + "loadedAddresses": { "readonly": [], "writable": [] }, + "logMessages": [ + "Program 11111111111111111111111111111111 invoke [1]", + "Program 11111111111111111111111111111111 success", + "Program AEwubmehHNvkMXoH2C5MgDSemZgQ3HUSYpeaF3UrNZdQ invoke [1]", + "Program log: Instruction: PostMessage", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX invoke [2]", + "Program log: Instruction: PostMessage", + "Program worm2ZoG2kUd4vFXhvjh93UUH596ayRfgQ2MgjNMTth invoke [3]", + "Program log: Sequence: 1", + "Program worm2ZoG2kUd4vFXhvjh93UUH596ayRfgQ2MgjNMTth consumed 18679 of 375180 compute units", + "Program worm2ZoG2kUd4vFXhvjh93UUH596ayRfgQ2MgjNMTth success", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX invoke [3]", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX consumed 2000 of 353964 compute units", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX success", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX consumed 33649 of 385286 compute units", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX success", + "Program AEwubmehHNvkMXoH2C5MgDSemZgQ3HUSYpeaF3UrNZdQ consumed 48808 of 399850 compute units", + "Program AEwubmehHNvkMXoH2C5MgDSemZgQ3HUSYpeaF3UrNZdQ success" + ], + "postBalances": [ + 499999999997491140, 1057920, 2350640270, 946560, 1552080, 1, 1141440, + 1141440, 0, 1169280, 1009200, 0, 1141440 + ], + "postTokenBalances": [], + "preBalances": [ + 499999999997496260, 1057920, 2350640170, 946560, 1552080, 1, 1141440, + 1141440, 0, 1169280, 1009200, 0, 1141440 + ], + "preTokenBalances": [], + "rewards": [], + "status": { "Ok": null } + }, + "slot": 5, + "transaction": { + "message": { + "header": { + "numReadonlySignedAccounts": 0, + "numReadonlyUnsignedAccounts": 8, + "numRequiredSignatures": 1 + }, + "accountKeys": [ + "H3kCPjpQDT4hgwWHr9E9pC99rZT2yHAwiwSwku6Bne9", + "2yVjuQwpsvdsrywzsJJVs9Ueh4zayyo5DYJbBNc3DDpn", + "9bFNrXNb2WTx8fMHXCheaZqkLZ3YCCaiqTftHxeintHy", + "G4zDzQLktwvU4rn6A4dSAy9eU76cJxppCaumZhjjhXjv", + "GXUAWs1h6Nh1KLByvfeEyig9yn92LmKMjXDNxHGddyXR", + "11111111111111111111111111111111", + "AEwubmehHNvkMXoH2C5MgDSemZgQ3HUSYpeaF3UrNZdQ", + "EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX", + "HQS31aApX3DDkuXgSpV9XyDUNtFgQ31pUn5BNWHG2PSp", + "SysvarC1ock11111111111111111111111111111111", + "SysvarRent111111111111111111111111111111111", + "UvCifi1D8qj5FSJQdWL3KENnmaZjm62XUMa7NReceer", + "worm2ZoG2kUd4vFXhvjh93UUH596ayRfgQ2MgjNMTth" + ], + "recentBlockhash": "EqNQXbHebHwD1Vs4BSStmUVh2y6GjMxF3NBsDXsYuvRh", + "instructions": [ + { + "accounts": [0, 2], + "data": "3Bxs4HanWsHUZCbH", + "programIdIndex": 5, + "stackHeight": null + }, + { + "accounts": [0, 7, 1, 4, 11, 3, 2, 9, 5, 10, 12, 8], + "data": "cpyiD6CEaBD", + "programIdIndex": 6, + "stackHeight": null + } + ], + "indexToProgramIds": {} + }, + "signatures": [ + "G4jVHcH6F4Np1NRvYC6ridv5jGfPSVGgiEVZrjprpMdBFhJH7eVxUuxsvkDF2rkx4JseUftz3HnWoSomGt3czSY" + ] + }, + "version": "legacy" + } + ` + + ///////// A bunch of checks to verify we parsed the JSON correctly. + var txRpc rpc.TransactionWithMeta + err := json.Unmarshal([]byte(eventJson), &txRpc) + require.NoError(t, err) + + tx, err := txRpc.GetParsedTransaction() + require.NoError(t, err) + + require.Equal(t, 2, len(tx.Message.Instructions)) + require.Equal(t, 1, len(txRpc.Meta.InnerInstructions)) + + ///////// Now we start the real test. + + logger := zap.NewNop() + msgC := make(chan *common.MessagePublication, 10) + s := shimNewWatcherForTest(t, msgC) + require.True(t, s.shimEnabled) + + var whProgramIndex uint16 + var shimProgramIndex uint16 + var shimFound bool + for n, key := range tx.Message.AccountKeys { + if key.Equals(s.contract) { + whProgramIndex = uint16(n) + } + if key.Equals(s.shimContractAddr) { + shimProgramIndex = uint16(n) + shimFound = true + } + } + + require.Equal(t, uint16(12), whProgramIndex) + require.True(t, shimFound) + require.Equal(t, uint16(7), shimProgramIndex) + + alreadyProcessed := ShimAlreadyProcessed{} + found, err := s.shimProcessInnerInstruction(logger, whProgramIndex, shimProgramIndex, tx, txRpc.Meta.InnerInstructions[0].Instructions, 0, 0, alreadyProcessed, false) + require.NoError(t, err) + require.True(t, found) + require.Equal(t, 1, len(s.msgC)) + + msg := <-msgC + require.NotNil(t, msg) + + // TODO: Can't check this until we switch MessagePublication.TxHash to be a byte array rather than a hash. + // expectedTxHash, err := vaa.StringToHash("0cfdad68fdee85b49aea65e48c0d8def74f0968e7e1cf2c33305cfc33fec02a4742895c1d32f7c4093f75133104e70bd126fbbf8b71e5d8cb723a390cd976305") + // require.NoError(t, err) + + expectedEmitterAddress, err := vaa.StringToAddress("0726d66bf942e942332ddf34a2edb7b83c4cdfd25b15d4247e2e15057cdfc3cf") + require.NoError(t, err) + + // assert.Equal(t, expectedTxHash, msg.TxHash) + assert.Equal(t, time.Unix(int64(1736542615), 0), msg.Timestamp) + assert.Equal(t, uint32(0), msg.Nonce) + assert.Equal(t, uint64(1), msg.Sequence) + assert.Equal(t, vaa.ChainIDSolana, msg.EmitterChain) + assert.Equal(t, expectedEmitterAddress, msg.EmitterAddress) + assert.Equal(t, uint8(1), msg.ConsistencyLevel) + assert.Equal(t, "your message goes here!", string(msg.Payload)) + assert.False(t, msg.IsReobservation) + assert.False(t, msg.Unreliable) +} + +func TestShimDirectWithMultipleShimTransactions(t *testing.T) { + eventJson := ` + { + "blockTime": 1736530812, + "meta": { + "computeUnitsConsumed": 84252, + "err": null, + "fee": 5000, + "innerInstructions": [ + { + "index": 1, + "instructions": [ + { + "accounts": [1, 3, 0, 4, 0, 2, 8, 5, 9], + "data": "TbyPDfUoyRxsr", + "programIdIndex": 10, + "stackHeight": 2 + }, + { + "accounts": [0, 4], + "data": "3Bxs4NLhqXb3ofom", + "programIdIndex": 5, + "stackHeight": 3 + }, + { + "accounts": [4], + "data": "9krTD1mFP1husSVM", + "programIdIndex": 5, + "stackHeight": 3 + }, + { + "accounts": [4], + "data": "SYXsBvR59WTsF4KEVN8LCQ1X9MekXCGPPNo3Af36taxCQBED", + "programIdIndex": 5, + "stackHeight": 3 + }, + { + "accounts": [0, 3], + "data": "3Bxs4bm7oSCPMeKR", + "programIdIndex": 5, + "stackHeight": 3 + }, + { + "accounts": [3], + "data": "9krTDGKFuDw9nLmM", + "programIdIndex": 5, + "stackHeight": 3 + }, + { + "accounts": [3], + "data": "SYXsBvR59WTsF4KEVN8LCQ1X9MekXCGPPNo3Af36taxCQBED", + "programIdIndex": 5, + "stackHeight": 3 + }, + { + "accounts": [7], + "data": "hTEY7jEqBPdDRkTWweeDPgyCUykRXEQVCUwrYmn4HZo84DdQrTJT2nBMiJFB3jXUVxHVd9mGq7BX9htuAN", + "programIdIndex": 6, + "stackHeight": 2 + } + ] + }, + { + "index": 2, + "instructions": [ + { + "accounts": [1, 3, 0, 4, 0, 2, 8, 5, 9], + "data": "TbyPDfUoyRxsr", + "programIdIndex": 10, + "stackHeight": 2 + }, + { + "accounts": [7], + "data": "hTEY7jEqBPdDRkTWweeDPgyCUykRXEQVCUwrYmn4HZo84DdQrTJT2nBMiJFB3jXUVxJb75nmkwJkw2Varz", + "programIdIndex": 6, + "stackHeight": 2 + } + ] + } + ], + "loadedAddresses": { + "readonly": [], + "writable": [] + }, + "logMessages": [ + "Program 11111111111111111111111111111111 invoke [1]", + "Program 11111111111111111111111111111111 success", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX invoke [1]", + "Program log: Instruction: PostMessage", + "Program worm2ZoG2kUd4vFXhvjh93UUH596ayRfgQ2MgjNMTth invoke [2]", + "Program 11111111111111111111111111111111 invoke [3]", + "Program 11111111111111111111111111111111 success", + "Program 11111111111111111111111111111111 invoke [3]", + "Program 11111111111111111111111111111111 success", + "Program 11111111111111111111111111111111 invoke [3]", + "Program 11111111111111111111111111111111 success", + "Program log: Sequence: 0", + "Program 11111111111111111111111111111111 invoke [3]", + "Program 11111111111111111111111111111111 success", + "Program 11111111111111111111111111111111 invoke [3]", + "Program 11111111111111111111111111111111 success", + "Program 11111111111111111111111111111111 invoke [3]", + "Program 11111111111111111111111111111111 success", + "Program worm2ZoG2kUd4vFXhvjh93UUH596ayRfgQ2MgjNMTth consumed 60384 of 380989 compute units", + "Program worm2ZoG2kUd4vFXhvjh93UUH596ayRfgQ2MgjNMTth success", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX invoke [2]", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX consumed 2000 of 318068 compute units", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX success", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX consumed 84102 of 399850 compute units", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX success" + ], + "postBalances": [ + 499999999997496260, 1057920, 2350640170, 1552080, 946560, 1, 1141440, 0, + 1169280, 1009200, 1141440 + ], + "postTokenBalances": [], + "preBalances": [ + 500000000000000000, 1057920, 2350640070, 0, 0, 1, 1141440, 0, 1169280, + 1009200, 1141440 + ], + "preTokenBalances": [], + "rewards": [], + "status": { + "Ok": null + } + }, + "slot": 3, + "transaction": { + "message": { + "header": { + "numReadonlySignedAccounts": 0, + "numReadonlyUnsignedAccounts": 6, + "numRequiredSignatures": 1 + }, + "accountKeys": [ + "H3kCPjpQDT4hgwWHr9E9pC99rZT2yHAwiwSwku6Bne9", + "2yVjuQwpsvdsrywzsJJVs9Ueh4zayyo5DYJbBNc3DDpn", + "9bFNrXNb2WTx8fMHXCheaZqkLZ3YCCaiqTftHxeintHy", + "9vohBn118ZEctRmuTRvoUZg1B1HGfSH8C5QX6twtUFrJ", + "HeccUHmoyMi5S6nuTcyUBh4w4me3FP541a52ErYJRT8a", + "11111111111111111111111111111111", + "EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX", + "HQS31aApX3DDkuXgSpV9XyDUNtFgQ31pUn5BNWHG2PSp", + "SysvarC1ock11111111111111111111111111111111", + "SysvarRent111111111111111111111111111111111", + "worm2ZoG2kUd4vFXhvjh93UUH596ayRfgQ2MgjNMTth" + ], + "recentBlockhash": "CMqPGm4icRdNuHsWJUK4Kgu4Cbe2nDQkYNqugQkKPa4Y", + "instructions": [ + { + "accounts": [0, 2], + "data": "3Bxs4HanWsHUZCbH", + "programIdIndex": 5, + "stackHeight": null + }, + { + "accounts": [1, 3, 0, 4, 0, 2, 8, 5, 9, 10, 7, 6], + "data": "3Cn8VBJReY7Bku3RduhBfYpk7tiw1R6pKcTWv9R", + "programIdIndex": 6, + "stackHeight": null + }, + { + "accounts": [1, 3, 0, 4, 0, 2, 8, 5, 9, 10, 7, 6], + "data": "3Cn8VBJReY7BmhufSsqDhta36ruYh9KwEjTqqv3", + "programIdIndex": 6, + "stackHeight": null + } + ], + "indexToProgramIds": {} + }, + "signatures": [ + "3NACxoZLehbdKGjTWZKTTXJPuovyqAih1AD1BrkYj8nzDAtjiQUEaNmhkoU1jcFfoPTAjrvnaLFgTafNWr3fBrdB" + ] + }, + "version": "legacy" + } + ` + + ///////// A bunch of checks to verify we parsed the JSON correctly. + var txRpc rpc.TransactionWithMeta + err := json.Unmarshal([]byte(eventJson), &txRpc) + require.NoError(t, err) + + tx, err := txRpc.GetParsedTransaction() + require.NoError(t, err) + + require.Equal(t, 3, len(tx.Message.Instructions)) + require.Equal(t, 2, len(txRpc.Meta.InnerInstructions)) + + ///////// Set up the watcher and do the one-time transaction processing. + + logger := zap.NewNop() + msgC := make(chan *common.MessagePublication, 10) + s := shimNewWatcherForTest(t, msgC) + require.True(t, s.shimEnabled) + + var whProgramIndex uint16 + var shimProgramIndex uint16 + var shimFound bool + for n, key := range tx.Message.AccountKeys { + if key.Equals(s.contract) { + whProgramIndex = uint16(n) + } + if key.Equals(s.shimContractAddr) { + shimProgramIndex = uint16(n) + shimFound = true + } + } + + require.Equal(t, uint16(10), whProgramIndex) + require.True(t, shimFound) + require.Equal(t, uint16(6), shimProgramIndex) + + // TODO: Can't check this until we switch MessagePublication.TxHash to be a byte array rather than a hash. + // expectedTxHash, err := vaa.StringToHash("7647cd98fd14c6e3cdfe35bc64bbc476abcdb5ab12e8d31e3151d132ed1e0eeb4595fda4779f69dbe00ff14aadad3fdcf537b88a22f48f3acb7b31f340670506") + // require.NoError(t, err) + + expectedEmitterAddress, err := vaa.StringToAddress("041c657e845d65d009d59ceeb1dda172bd6bc9e7ee5a19e56573197cf7fdffde") + require.NoError(t, err) + + //////////// Process the first shim top level instruction. + + alreadyProcessed := ShimAlreadyProcessed{} + found, err := s.shimProcessTopLevelInstruction(logger, whProgramIndex, shimProgramIndex, tx, txRpc.Meta.InnerInstructions, 1, alreadyProcessed, false) + require.NoError(t, err) + require.True(t, found) + require.Equal(t, 1, len(s.msgC)) + + msg := <-msgC + require.NotNil(t, msg) + + // assert.Equal(t, expectedTxHash, msg.TxHash) + assert.Equal(t, time.Unix(int64(1736530812), 0), msg.Timestamp) + assert.Equal(t, uint32(42), msg.Nonce) + assert.Equal(t, uint64(0), msg.Sequence) + assert.Equal(t, vaa.ChainIDSolana, msg.EmitterChain) + assert.Equal(t, expectedEmitterAddress, msg.EmitterAddress) + assert.Equal(t, uint8(1), msg.ConsistencyLevel) + assert.Equal(t, "hello world", string(msg.Payload)) + assert.False(t, msg.IsReobservation) + assert.False(t, msg.Unreliable) + + //////////// Process the second shim top level instruction. + + found, err = s.shimProcessTopLevelInstruction(logger, whProgramIndex, shimProgramIndex, tx, txRpc.Meta.InnerInstructions, 2, alreadyProcessed, false) + require.NoError(t, err) + require.True(t, found) + require.Equal(t, 1, len(s.msgC)) + + msg = <-msgC + require.NotNil(t, msg) + + // assert.Equal(t, expectedTxHash, msg.TxHash) + assert.Equal(t, time.Unix(int64(1736530813), 0), msg.Timestamp) + assert.Equal(t, uint32(43), msg.Nonce) + assert.Equal(t, uint64(1), msg.Sequence) + assert.Equal(t, vaa.ChainIDSolana, msg.EmitterChain) + assert.Equal(t, expectedEmitterAddress, msg.EmitterAddress) + assert.Equal(t, uint8(1), msg.ConsistencyLevel) + assert.Equal(t, "hello world", string(msg.Payload)) + assert.False(t, msg.IsReobservation) + assert.False(t, msg.Unreliable) +} + +func TestShimFromIntegratorWithMultipleShimTransactions(t *testing.T) { + eventJson := ` + { + "blockTime": 1736542615, + "meta": { + "computeUnitsConsumed": 48958, + "err": null, + "fee": 5000, + "innerInstructions": [ + { + "index": 1, + "instructions": [ + { + "accounts": [1, 4, 11, 3, 0, 2, 9, 5, 10, 12, 8, 7], + "data": "BeHixXyfSZ8dzFJzxTYRV18L6KSgTuqcTjaqeXgDVbXHC7mCjAgSyhz", + "programIdIndex": 7, + "stackHeight": 2 + }, + { + "accounts": [1, 4, 11, 3, 0, 2, 9, 5, 10], + "data": "T4xyMHqZi66JU", + "programIdIndex": 12, + "stackHeight": 3 + }, + { + "accounts": [8], + "data": "hTEY7jEqBPdDRkTWweeDPgzBpsiybJCHnVTVt8aCDem8p58yeQcQLJWk7hgGHrX79qZyKmCM89vCgPY7SE", + "programIdIndex": 7, + "stackHeight": 3 + }, + { + "accounts": [1, 4, 11, 3, 0, 2, 9, 5, 10, 12, 8, 7], + "data": "BeHixXyfSZ8gpCS9kw5xbo7V9NN3f6bDP3Bi4G3sPsbod54LvCUimUU", + "programIdIndex": 7, + "stackHeight": 2 + }, + { + "accounts": [1, 4, 11, 3, 0, 2, 9, 5, 10], + "data": "T4xyMHqZi66JU", + "programIdIndex": 12, + "stackHeight": 3 + }, + { + "accounts": [8], + "data": "hTEY7jEqBPdDRkTWweeDPgzBpsiybJCHnVTVt8aCDem8p58yeQcQLJWk7hgGHrX79qb4ohDr3z3STi8o8r", + "programIdIndex": 7, + "stackHeight": 3 + } + ] + } + ], + "loadedAddresses": { "readonly": [], "writable": [] }, + "logMessages": [ + "Program 11111111111111111111111111111111 invoke [1]", + "Program 11111111111111111111111111111111 success", + "Program AEwubmehHNvkMXoH2C5MgDSemZgQ3HUSYpeaF3UrNZdQ invoke [1]", + "Program log: Instruction: PostMessage", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX invoke [2]", + "Program log: Instruction: PostMessage", + "Program worm2ZoG2kUd4vFXhvjh93UUH596ayRfgQ2MgjNMTth invoke [3]", + "Program log: Sequence: 1", + "Program worm2ZoG2kUd4vFXhvjh93UUH596ayRfgQ2MgjNMTth consumed 18679 of 375180 compute units", + "Program worm2ZoG2kUd4vFXhvjh93UUH596ayRfgQ2MgjNMTth success", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX invoke [3]", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX consumed 2000 of 353964 compute units", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX success", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX consumed 33649 of 385286 compute units", + "Program EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX success", + "Program AEwubmehHNvkMXoH2C5MgDSemZgQ3HUSYpeaF3UrNZdQ consumed 48808 of 399850 compute units", + "Program AEwubmehHNvkMXoH2C5MgDSemZgQ3HUSYpeaF3UrNZdQ success" + ], + "postBalances": [ + 499999999997491140, 1057920, 2350640270, 946560, 1552080, 1, 1141440, + 1141440, 0, 1169280, 1009200, 0, 1141440 + ], + "postTokenBalances": [], + "preBalances": [ + 499999999997496260, 1057920, 2350640170, 946560, 1552080, 1, 1141440, + 1141440, 0, 1169280, 1009200, 0, 1141440 + ], + "preTokenBalances": [], + "rewards": [], + "status": { "Ok": null } + }, + "slot": 5, + "transaction": { + "message": { + "header": { + "numReadonlySignedAccounts": 0, + "numReadonlyUnsignedAccounts": 8, + "numRequiredSignatures": 1 + }, + "accountKeys": [ + "H3kCPjpQDT4hgwWHr9E9pC99rZT2yHAwiwSwku6Bne9", + "2yVjuQwpsvdsrywzsJJVs9Ueh4zayyo5DYJbBNc3DDpn", + "9bFNrXNb2WTx8fMHXCheaZqkLZ3YCCaiqTftHxeintHy", + "G4zDzQLktwvU4rn6A4dSAy9eU76cJxppCaumZhjjhXjv", + "GXUAWs1h6Nh1KLByvfeEyig9yn92LmKMjXDNxHGddyXR", + "11111111111111111111111111111111", + "AEwubmehHNvkMXoH2C5MgDSemZgQ3HUSYpeaF3UrNZdQ", + "EtZMZM22ViKMo4r5y4Anovs3wKQ2owUmDpjygnMMcdEX", + "HQS31aApX3DDkuXgSpV9XyDUNtFgQ31pUn5BNWHG2PSp", + "SysvarC1ock11111111111111111111111111111111", + "SysvarRent111111111111111111111111111111111", + "UvCifi1D8qj5FSJQdWL3KENnmaZjm62XUMa7NReceer", + "worm2ZoG2kUd4vFXhvjh93UUH596ayRfgQ2MgjNMTth" + ], + "recentBlockhash": "EqNQXbHebHwD1Vs4BSStmUVh2y6GjMxF3NBsDXsYuvRh", + "instructions": [ + { + "accounts": [0, 2], + "data": "3Bxs4HanWsHUZCbH", + "programIdIndex": 5, + "stackHeight": null + }, + { + "accounts": [0, 7, 1, 4, 11, 3, 2, 9, 5, 10, 12, 8], + "data": "cpyiD6CEaBD", + "programIdIndex": 6, + "stackHeight": null + } + ], + "indexToProgramIds": {} + }, + "signatures": [ + "G4jVHcH6F4Np1NRvYC6ridv5jGfPSVGgiEVZrjprpMdBFhJH7eVxUuxsvkDF2rkx4JseUftz3HnWoSomGt3czSY" + ] + }, + "version": "legacy" + } + ` + + ///////// A bunch of checks to verify we parsed the JSON correctly. + var txRpc rpc.TransactionWithMeta + err := json.Unmarshal([]byte(eventJson), &txRpc) + require.NoError(t, err) + + tx, err := txRpc.GetParsedTransaction() + require.NoError(t, err) + + require.Equal(t, 2, len(tx.Message.Instructions)) + require.Equal(t, 1, len(txRpc.Meta.InnerInstructions)) + + ///////// Set up the watcher and do the one-time transaction processing. + + logger := zap.NewNop() + msgC := make(chan *common.MessagePublication, 10) + s := shimNewWatcherForTest(t, msgC) + require.True(t, s.shimEnabled) + + var whProgramIndex uint16 + var shimProgramIndex uint16 + var shimFound bool + for n, key := range tx.Message.AccountKeys { + if key.Equals(s.contract) { + whProgramIndex = uint16(n) + } + if key.Equals(s.shimContractAddr) { + shimProgramIndex = uint16(n) + shimFound = true + } + } + + require.Equal(t, uint16(12), whProgramIndex) + require.True(t, shimFound) + require.Equal(t, uint16(7), shimProgramIndex) + + // TODO: Can't check this until we switch MessagePublication.TxHash to be a byte array rather than a hash. + // expectedTxHash, err := vaa.StringToHash("0cfdad68fdee85b49aea65e48c0d8def74f0968e7e1cf2c33305cfc33fec02a4742895c1d32f7c4093f75133104e70bd126fbbf8b71e5d8cb723a390cd976305") + // require.NoError(t, err) + + expectedEmitterAddress, err := vaa.StringToAddress("0726d66bf942e942332ddf34a2edb7b83c4cdfd25b15d4247e2e15057cdfc3cf") + require.NoError(t, err) + + //////////// Process the first shim inner instruction. + + alreadyProcessed := ShimAlreadyProcessed{} + found, err := s.shimProcessInnerInstruction(logger, whProgramIndex, shimProgramIndex, tx, txRpc.Meta.InnerInstructions[0].Instructions, 0, 0, alreadyProcessed, false) + require.NoError(t, err) + require.True(t, found) + require.Equal(t, 1, len(s.msgC)) + + msg := <-msgC + require.NotNil(t, msg) + + // assert.Equal(t, expectedTxHash, msg.TxHash) + assert.Equal(t, time.Unix(int64(1736542615), 0), msg.Timestamp) + assert.Equal(t, uint32(0), msg.Nonce) + assert.Equal(t, uint64(1), msg.Sequence) + assert.Equal(t, vaa.ChainIDSolana, msg.EmitterChain) + assert.Equal(t, expectedEmitterAddress, msg.EmitterAddress) + assert.Equal(t, uint8(1), msg.ConsistencyLevel) + assert.Equal(t, "your message goes here!", string(msg.Payload)) + assert.False(t, msg.IsReobservation) + assert.False(t, msg.Unreliable) + + //////////// Process the second shim inner instruction. + + found, err = s.shimProcessInnerInstruction(logger, whProgramIndex, shimProgramIndex, tx, txRpc.Meta.InnerInstructions[0].Instructions, 0, 3, alreadyProcessed, false) + require.NoError(t, err) + require.True(t, found) + require.Equal(t, 1, len(s.msgC)) + + msg = <-msgC + require.NotNil(t, msg) + + // assert.Equal(t, expectedTxHash, msg.TxHash) + assert.Equal(t, time.Unix(int64(1736542616), 0), msg.Timestamp) + assert.Equal(t, uint32(42), msg.Nonce) + assert.Equal(t, uint64(2), msg.Sequence) + assert.Equal(t, vaa.ChainIDSolana, msg.EmitterChain) + assert.Equal(t, expectedEmitterAddress, msg.EmitterAddress) + assert.Equal(t, uint8(1), msg.ConsistencyLevel) + assert.Equal(t, "your message goes here!", string(msg.Payload)) + assert.False(t, msg.IsReobservation) + assert.False(t, msg.Unreliable) +}