From b78393e40438728b069b0bd338024ee4220590ab Mon Sep 17 00:00:00 2001 From: Bruce Riley Date: Wed, 4 Oct 2023 13:16:13 -0500 Subject: [PATCH] CCQ/Node: Guardian Changes --- node/cmd/guardiand/node.go | 16 +- node/cmd/spy/spy.go | 6 + node/pkg/adminrpc/adminserver_test.go | 5 + node/pkg/common/mode.go | 26 + node/pkg/common/mode_test.go | 51 ++ node/pkg/node/node.go | 21 + node/pkg/node/node_test.go | 2 +- node/pkg/node/options.go | 62 +- node/pkg/p2p/ccq_p2p.go | 297 ++++++++ node/pkg/p2p/p2p.go | 31 + node/pkg/p2p/watermark_test.go | 6 + node/pkg/proto/gossip/v1/gossip.pb.go | 606 ++++++++++------ node/pkg/query/dev.guardian.key | 8 + node/pkg/query/helpers_test.go | 6 + node/pkg/query/msg_test.go | 544 ++++++++++++++ node/pkg/query/query.go | 412 +++++++++++ node/pkg/query/query_test.go | 668 ++++++++++++++++++ node/pkg/query/request.go | 470 ++++++++++++ node/pkg/query/response.go | 508 +++++++++++++ node/pkg/watchers/algorand/config.go | 3 + node/pkg/watchers/aptos/config.go | 3 + node/pkg/watchers/cosmwasm/config.go | 3 + node/pkg/watchers/evm/config.go | 5 +- node/pkg/watchers/evm/connectors/celo.go | 14 + node/pkg/watchers/evm/connectors/common.go | 13 + node/pkg/watchers/evm/connectors/ethereum.go | 3 + node/pkg/watchers/evm/connectors/poller.go | 12 +- .../watchers/evm/connectors/poller_test.go | 5 + .../watchers/evm/finalizers/moonbeam_test.go | 5 + node/pkg/watchers/evm/watcher.go | 247 +++++++ node/pkg/watchers/mock/config.go | 3 + node/pkg/watchers/near/config.go | 3 + node/pkg/watchers/solana/config.go | 3 + node/pkg/watchers/sui/config.go | 3 + node/pkg/watchers/watchers.go | 3 + proto/gossip/v1/gossip.proto | 18 + 36 files changed, 3869 insertions(+), 222 deletions(-) create mode 100644 node/pkg/common/mode_test.go create mode 100644 node/pkg/p2p/ccq_p2p.go create mode 100644 node/pkg/query/dev.guardian.key create mode 100644 node/pkg/query/helpers_test.go create mode 100644 node/pkg/query/msg_test.go create mode 100644 node/pkg/query/query.go create mode 100644 node/pkg/query/query_test.go create mode 100644 node/pkg/query/request.go create mode 100644 node/pkg/query/response.go diff --git a/node/cmd/guardiand/node.go b/node/cmd/guardiand/node.go index e82b42ae6b..80c232bba6 100644 --- a/node/cmd/guardiand/node.go +++ b/node/cmd/guardiand/node.go @@ -205,6 +205,12 @@ var ( chainGovernorEnabled *bool + ccqEnabled *bool + ccqAllowedRequesters *string + ccqP2pPort *uint + ccqP2pBootstrap *string + ccqAllowedPeers *string + gatewayRelayerContract *string gatewayRelayerKeyPath *string gatewayRelayerKeyPassPhrase *string @@ -370,6 +376,12 @@ func init() { chainGovernorEnabled = NodeCmd.Flags().Bool("chainGovernorEnabled", false, "Run the chain governor") + ccqEnabled = NodeCmd.Flags().Bool("ccqEnabled", false, "Enable cross chain query support") + ccqAllowedRequesters = NodeCmd.Flags().String("ccqAllowedRequesters", "", "Comma separated list of signers allowed to submit cross chain queries") + ccqP2pPort = NodeCmd.Flags().Uint("ccqP2pPort", 8996, "CCQ P2P UDP listener port") + ccqP2pBootstrap = NodeCmd.Flags().String("ccqP2pBootstrap", "", "CCQ P2P bootstrap peers (comma-separated)") + ccqAllowedPeers = NodeCmd.Flags().String("ccqAllowedPeers", "", "CCQ allowed P2P peers (comma-separated)") + gatewayRelayerContract = NodeCmd.Flags().String("gatewayRelayerContract", "", "Address of the smart contract on wormchain to receive relayed VAAs") gatewayRelayerKeyPath = NodeCmd.Flags().String("gatewayRelayerKeyPath", "", "Path to gateway relayer private key for signing transactions") gatewayRelayerKeyPassPhrase = NodeCmd.Flags().String("gatewayRelayerKeyPassPhrase", "", "Pass phrase used to unarmor the gateway relayer key file") @@ -469,6 +481,7 @@ func runNode(cmd *cobra.Command, args []string) { // Use the first guardian node as bootstrap *p2pBootstrap = fmt.Sprintf("/dns4/guardian-0.guardian/udp/%d/quic/p2p/%s", *p2pPort, g0key.String()) + *ccqP2pBootstrap = fmt.Sprintf("/dns4/guardian-0.guardian/udp/%d/quic/p2p/%s", *ccqP2pPort, g0key.String()) // Deterministic ganache ETH devnet address. *ethContract = unsafeDevModeEvmContractAddress(*ethContract) @@ -1422,8 +1435,9 @@ func runNode(cmd *cobra.Command, args []string) { node.GuardianOptionAccountant(*accountantContract, *accountantWS, *accountantCheckEnabled, accountantWormchainConn), node.GuardianOptionGovernor(*chainGovernorEnabled), node.GuardianOptionGatewayRelayer(*gatewayRelayerContract, gatewayRelayerWormchainConn), + node.GuardianOptionQueryHandler(*ccqEnabled, *ccqAllowedRequesters), node.GuardianOptionAdminService(*adminSocketPath, ethRPC, ethContract, rpcMap), - node.GuardianOptionP2P(p2pKey, *p2pNetworkID, *p2pBootstrap, *nodeName, *disableHeartbeatVerify, *p2pPort, ibc.GetFeatures), + node.GuardianOptionP2P(p2pKey, *p2pNetworkID, *p2pBootstrap, *nodeName, *disableHeartbeatVerify, *p2pPort, *ccqP2pBootstrap, *ccqP2pPort, *ccqAllowedPeers, ibc.GetFeatures), node.GuardianOptionStatusServer(*statusAddr), node.GuardianOptionProcessor(), } diff --git a/node/cmd/spy/spy.go b/node/cmd/spy/spy.go index a38141f233..49f4c09cbe 100644 --- a/node/cmd/spy/spy.go +++ b/node/cmd/spy/spy.go @@ -356,6 +356,12 @@ func runSpy(cmd *cobra.Command, args []string) { components, nil, // ibc feature string false, // gateway relayer enabled + false, // ccqEnabled + nil, // query requests + nil, // query responses + "", // query bootstrap peers + 0, // query port + "", // query allow list )); err != nil { return err } diff --git a/node/pkg/adminrpc/adminserver_test.go b/node/pkg/adminrpc/adminserver_test.go index fa6dcd5973..7426f842d3 100644 --- a/node/pkg/adminrpc/adminserver_test.go +++ b/node/pkg/adminrpc/adminserver_test.go @@ -15,6 +15,7 @@ import ( "github.com/ethereum/go-ethereum/core/types" ethcrypto "github.com/ethereum/go-ethereum/crypto" "github.com/ethereum/go-ethereum/event" + ethRpc "github.com/ethereum/go-ethereum/rpc" "github.com/stretchr/testify/require" "github.com/wormhole-foundation/wormhole/sdk/vaa" "go.uber.org/zap" @@ -68,6 +69,10 @@ func (m mockEVMConnector) RawCallContext(ctx context.Context, result interface{} panic("unimplemented") } +func (m mockEVMConnector) RawBatchCallContext(ctx context.Context, b []ethRpc.BatchElem) error { + panic("unimplemented") +} + func generateGS(num int) (keys []*ecdsa.PrivateKey, addrs []common.Address) { for i := 0; i < num; i++ { key, err := ethcrypto.GenerateKey() diff --git a/node/pkg/common/mode.go b/node/pkg/common/mode.go index b1d3f9eef0..823f213517 100644 --- a/node/pkg/common/mode.go +++ b/node/pkg/common/mode.go @@ -1,5 +1,10 @@ package common +import ( + "fmt" + "strings" +) + type Environment string const ( @@ -9,3 +14,24 @@ const ( GoTest Environment = "unit-test" AccountantMock Environment = "accountant-mock" // Used for mocking accountant with a Wormchain connection ) + +// ParseEnvironment parses a string into the corresponding Environment value, allowing various reasonable variations. +func ParseEnvironment(str string) (Environment, error) { + str = strings.ToLower(str) + if str == "prod" || str == "mainnet" { + return MainNet, nil + } + if str == "test" || str == "testnet" { + return TestNet, nil + } + if str == "dev" || str == "devnet" || str == "unsafedevnet" { + return UnsafeDevNet, nil + } + if str == "unit-test" || str == "gotest" { + return GoTest, nil + } + if str == "accountant-mock" || str == "accountantmock" { + return AccountantMock, nil + } + return UnsafeDevNet, fmt.Errorf("invalid environment string: %s", str) +} diff --git a/node/pkg/common/mode_test.go b/node/pkg/common/mode_test.go new file mode 100644 index 0000000000..9f5d2cd4fd --- /dev/null +++ b/node/pkg/common/mode_test.go @@ -0,0 +1,51 @@ +package common + +import ( + "testing" + + "github.com/stretchr/testify/assert" +) + +func TestParseEnvironment(t *testing.T) { + type test struct { + input string + output Environment + err bool + } + + tests := []test{ + {input: "MainNet", output: MainNet, err: false}, + {input: "Prod", output: MainNet, err: false}, + + {input: "TestNet", output: TestNet, err: false}, + {input: "test", output: TestNet, err: false}, + + {input: "UnsafeDevNet", output: UnsafeDevNet, err: false}, + {input: "devnet", output: UnsafeDevNet, err: false}, + {input: "dev", output: UnsafeDevNet, err: false}, + + {input: "GoTest", output: GoTest, err: false}, + {input: "unit-test", output: GoTest, err: false}, + + {input: "AccountantMock", output: AccountantMock, err: false}, + {input: "accountant-mock", output: AccountantMock, err: false}, + + {input: "junk", output: UnsafeDevNet, err: true}, + {input: "", output: UnsafeDevNet, err: true}, + } + + for _, tc := range tests { + t.Run(tc.input, func(t *testing.T) { + output, err := ParseEnvironment(tc.input) + if err != nil { + if tc.err == false { + assert.NoError(t, err) + } + } else if tc.err { + assert.Error(t, err) + } else { + assert.Equal(t, tc.output, output) + } + }) + } +} diff --git a/node/pkg/node/node.go b/node/pkg/node/node.go index 206fa143df..dd088beda9 100644 --- a/node/pkg/node/node.go +++ b/node/pkg/node/node.go @@ -11,7 +11,9 @@ import ( "github.com/certusone/wormhole/node/pkg/governor" "github.com/certusone/wormhole/node/pkg/gwrelayer" gossipv1 "github.com/certusone/wormhole/node/pkg/proto/gossip/v1" + "github.com/certusone/wormhole/node/pkg/query" "github.com/certusone/wormhole/node/pkg/supervisor" + "github.com/wormhole-foundation/wormhole/sdk/vaa" "go.uber.org/zap" "google.golang.org/grpc" @@ -59,6 +61,7 @@ type G struct { acct *accountant.Accountant gov *governor.ChainGovernor gatewayRelayer *gwrelayer.GatewayRelayer + queryHandler *query.QueryHandler publicrpcServer *grpc.Server // runnables @@ -82,6 +85,12 @@ type G struct { obsvReqSendC channelPair[*gossipv1.ObservationRequest] // acctC is the channel where messages will be put after they reached quorum in the accountant. acctC channelPair[*common.MessagePublication] + + // Cross Chain Query Handler channels + chainQueryReqC map[vaa.ChainID]chan *query.PerChainQueryInternal + signedQueryReqC channelPair[*gossipv1.SignedQueryRequest] + queryResponseC channelPair[*query.PerChainQueryResponseInternal] + queryResponsePublicationC channelPair[*query.QueryResponsePublication] } func NewGuardianNode( @@ -108,6 +117,11 @@ func (g *G) initializeBasic(rootCtxCancel context.CancelFunc) { g.obsvReqC = makeChannelPair[*gossipv1.ObservationRequest](observationRequestInboundBufferSize) g.obsvReqSendC = makeChannelPair[*gossipv1.ObservationRequest](observationRequestOutboundBufferSize) g.acctC = makeChannelPair[*common.MessagePublication](accountant.MsgChannelCapacity) + // Cross Chain Query Handler channels + g.chainQueryReqC = make(map[vaa.ChainID]chan *query.PerChainQueryInternal) + g.signedQueryReqC = makeChannelPair[*gossipv1.SignedQueryRequest](query.SignedQueryRequestChannelSize) + g.queryResponseC = makeChannelPair[*query.PerChainQueryResponseInternal](0) + g.queryResponsePublicationC = makeChannelPair[*query.QueryResponsePublication](0) // Guardian set state managed by processor g.gst = common.NewGuardianSetState(nil) @@ -191,6 +205,13 @@ func (g *G) Run(rootCtxCancel context.CancelFunc, options ...*GuardianOption) su } } + if g.queryHandler != nil { + logger.Info("Starting query handler", zap.String("component", "ccq")) + if err := g.queryHandler.Start(ctx); err != nil { + logger.Fatal("failed to create chain governor", zap.Error(err), zap.String("component", "ccq")) + } + } + // Start any other runnables for name, runnable := range g.runnables { if err := supervisor.Run(ctx, name, runnable); err != nil { diff --git a/node/pkg/node/node_test.go b/node/pkg/node/node_test.go index 3c89e54a55..2d12c398ca 100644 --- a/node/pkg/node/node_test.go +++ b/node/pkg/node/node_test.go @@ -190,7 +190,7 @@ func mockGuardianRunnable(t testing.TB, gs []*mockGuardian, mockGuardianIndex ui GuardianOptionNoAccountant(), // disable accountant GuardianOptionGovernor(true), GuardianOptionGatewayRelayer("", nil), // disable gateway relayer - GuardianOptionP2P(gs[mockGuardianIndex].p2pKey, networkID, bootstrapPeers, nodeName, false, cfg.p2pPort, func() string { return "" }), + GuardianOptionP2P(gs[mockGuardianIndex].p2pKey, networkID, bootstrapPeers, nodeName, false, cfg.p2pPort, "", 0, "", func() string { return "" }), GuardianOptionPublicRpcSocket(cfg.publicSocket, publicRpcLogDetail), GuardianOptionPublicrpcTcpService(cfg.publicRpc, publicRpcLogDetail), GuardianOptionPublicWeb(cfg.publicWeb, cfg.publicSocket, "", false, ""), diff --git a/node/pkg/node/options.go b/node/pkg/node/options.go index e2b530cba7..af701bc39b 100644 --- a/node/pkg/node/options.go +++ b/node/pkg/node/options.go @@ -16,6 +16,7 @@ import ( "github.com/certusone/wormhole/node/pkg/p2p" "github.com/certusone/wormhole/node/pkg/processor" gossipv1 "github.com/certusone/wormhole/node/pkg/proto/gossip/v1" + "github.com/certusone/wormhole/node/pkg/query" "github.com/certusone/wormhole/node/pkg/readiness" "github.com/certusone/wormhole/node/pkg/supervisor" "github.com/certusone/wormhole/node/pkg/watchers" @@ -38,7 +39,7 @@ type GuardianOption struct { // GuardianOptionP2P configures p2p networking. // Dependencies: Accountant, Governor -func GuardianOptionP2P(p2pKey libp2p_crypto.PrivKey, networkId string, bootstrapPeers string, nodeName string, disableHeartbeatVerify bool, port uint, ibcFeaturesFunc func() string) *GuardianOption { +func GuardianOptionP2P(p2pKey libp2p_crypto.PrivKey, networkId string, bootstrapPeers string, nodeName string, disableHeartbeatVerify bool, port uint, ccqBootstrapPeers string, ccqPort uint, ccqAllowedPeers string, ibcFeaturesFunc func() string) *GuardianOption { return &GuardianOption{ name: "p2p", dependencies: []string{"accountant", "governor", "gateway-relayer"}, @@ -72,6 +73,36 @@ func GuardianOptionP2P(p2pKey libp2p_crypto.PrivKey, networkId string, bootstrap components, ibcFeaturesFunc, (g.gatewayRelayer != nil), + (g.queryHandler != nil), + g.signedQueryReqC.writeC, + g.queryResponsePublicationC.readC, + ccqBootstrapPeers, + ccqPort, + ccqAllowedPeers, + ) + + return nil + }} +} + +// GuardianOptionQueryHandler configures the Cross Chain Query module. +func GuardianOptionQueryHandler(ccqEnabled bool, allowedRequesters string) *GuardianOption { + return &GuardianOption{ + name: "query", + f: func(ctx context.Context, logger *zap.Logger, g *G) error { + if !ccqEnabled { + logger.Info("ccq: cross chain query is disabled", zap.String("component", "ccq")) + return nil + } + + g.queryHandler = query.NewQueryHandler( + logger, + g.env, + allowedRequesters, + g.signedQueryReqC.readC, + g.chainQueryReqC, + g.queryResponseC.readC, + g.queryResponsePublicationC.writeC, ) return nil @@ -301,6 +332,32 @@ func GuardianOptionWatchers(watcherConfigs []watchers.WatcherConfig, ibcWatcherC }(chainMsgC[chainId], chainId) } + // Per-chain query response channel + chainQueryResponseC := make(map[vaa.ChainID]chan *query.PerChainQueryResponseInternal) + // aggregate per-chain msgC into msgC. + // SECURITY defense-in-depth: This way we enforce that a watcher must set the msg.EmitterChain to its chainId, which makes the code easier to audit + for _, chainId := range vaa.GetAllNetworkIDs() { + chainQueryResponseC[chainId] = make(chan *query.PerChainQueryResponseInternal) + go func(c <-chan *query.PerChainQueryResponseInternal, chainId vaa.ChainID) { + for { + select { + case <-ctx.Done(): + return + case response := <-c: + if response.ChainId != chainId { + // SECURITY: This should never happen. If it does, a watcher has been compromised. + logger.Fatal("SECURITY CRITICAL: Received query response from a chain that was not marked as originating from that chain", + zap.Uint16("responseChainId", uint16(response.ChainId)), + zap.Stringer("watcherChainId", chainId), + ) + } else { + g.queryResponseC.writeC <- response + } + } + } + }(chainQueryResponseC[chainId], chainId) + } + watchers := make(map[watchers.NetworkID]interfaces.L1Finalizer) for _, wc := range watcherConfigs { @@ -316,6 +373,7 @@ func GuardianOptionWatchers(watcherConfigs []watchers.WatcherConfig, ibcWatcherC } chainObsvReqC[wc.GetChainID()] = make(chan *gossipv1.ObservationRequest, observationRequestPerChainBufferSize) + g.chainQueryReqC[wc.GetChainID()] = make(chan *query.PerChainQueryInternal, query.QueryRequestBufferSize) if wc.RequiredL1Finalizer() != "" { l1watcher, ok := watchers[wc.RequiredL1Finalizer()] @@ -327,7 +385,7 @@ func GuardianOptionWatchers(watcherConfigs []watchers.WatcherConfig, ibcWatcherC wc.SetL1Finalizer(l1watcher) } - l1finalizer, runnable, err := wc.Create(chainMsgC[wc.GetChainID()], chainObsvReqC[wc.GetChainID()], g.setC.writeC, g.env) + l1finalizer, runnable, err := wc.Create(chainMsgC[wc.GetChainID()], chainObsvReqC[wc.GetChainID()], g.chainQueryReqC[wc.GetChainID()], chainQueryResponseC[wc.GetChainID()], g.setC.writeC, g.env) if err != nil { return fmt.Errorf("error creating watcher: %w", err) diff --git a/node/pkg/p2p/ccq_p2p.go b/node/pkg/p2p/ccq_p2p.go new file mode 100644 index 0000000000..6926312e4c --- /dev/null +++ b/node/pkg/p2p/ccq_p2p.go @@ -0,0 +1,297 @@ +package p2p + +import ( + "context" + "crypto/ecdsa" + "errors" + "fmt" + "strings" + + "github.com/certusone/wormhole/node/pkg/common" + "github.com/certusone/wormhole/node/pkg/query" + ethcrypto "github.com/ethereum/go-ethereum/crypto" + "github.com/prometheus/client_golang/prometheus" + "github.com/prometheus/client_golang/prometheus/promauto" + "google.golang.org/protobuf/proto" + + gossipv1 "github.com/certusone/wormhole/node/pkg/proto/gossip/v1" + + "github.com/libp2p/go-libp2p" + dht "github.com/libp2p/go-libp2p-kad-dht" + pubsub "github.com/libp2p/go-libp2p-pubsub" + "github.com/libp2p/go-libp2p/core/crypto" + "github.com/libp2p/go-libp2p/core/host" + "github.com/libp2p/go-libp2p/core/peer" + "github.com/libp2p/go-libp2p/core/protocol" + "github.com/libp2p/go-libp2p/core/routing" + libp2ptls "github.com/libp2p/go-libp2p/p2p/security/tls" + libp2pquic "github.com/libp2p/go-libp2p/p2p/transport/quic" + "go.uber.org/zap" +) + +var ( + ccqP2pMessagesSent = promauto.NewCounter( + prometheus.CounterOpts{ + Name: "wormhole_ccqp2p_broadcast_messages_sent_total", + Help: "Total number of ccq p2p pubsub broadcast messages sent", + }) + ccqP2pMessagesReceived = promauto.NewCounterVec( + prometheus.CounterOpts{ + Name: "wormhole_ccqp2p_broadcast_messages_received_total", + Help: "Total number of ccq p2p pubsub broadcast messages received", + }, []string{"type"}) +) + +type ccqP2p struct { + logger *zap.Logger + + h host.Host + th_req *pubsub.Topic + th_resp *pubsub.Topic + sub *pubsub.Subscription + allowedPeers map[string]struct{} +} + +func newCcqRunP2p( + logger *zap.Logger, + allowedPeersStr string, +) *ccqP2p { + l := logger.With(zap.String("component", "ccqp2p")) + allowedPeers := make(map[string]struct{}) + for _, peerID := range strings.Split(allowedPeersStr, ",") { + if peerID != "" { + l.Info("will allow requests from peer", zap.String("peerID", peerID)) + allowedPeers[peerID] = struct{}{} + } + } + + return &ccqP2p{ + logger: l, + allowedPeers: allowedPeers, + } +} + +func (ccq *ccqP2p) run( + ctx context.Context, + priv crypto.PrivKey, + gk *ecdsa.PrivateKey, + networkID string, + bootstrapPeers string, + port uint, + signedQueryReqC chan<- *gossipv1.SignedQueryRequest, + queryResponseReadC <-chan *query.QueryResponsePublication, + errC chan error, +) error { + var err error + + components := DefaultComponents() + if components == nil { + return fmt.Errorf("components is not initialized") + } + components.Port = port + + ccq.h, err = libp2p.New( + // Use the keypair we generated + libp2p.Identity(priv), + + // Multiple listen addresses + libp2p.ListenAddrStrings( + components.ListeningAddresses()..., + ), + + // Enable TLS security as the only security protocol. + libp2p.Security(libp2ptls.ID, libp2ptls.New), + + // Enable QUIC transport as the only transport. + libp2p.Transport(libp2pquic.NewTransport), + + // Let's prevent our peer from having too many + // connections by attaching a connection manager. + libp2p.ConnectionManager(components.ConnMgr), // TODO: Can we use the same connection manager? + + // Let this host use the DHT to find other hosts + libp2p.Routing(func(h host.Host) (routing.PeerRouting, error) { + ccq.logger.Info("Connecting to bootstrap peers", zap.String("bootstrap_peers", bootstrapPeers)) + + bootstrappers, _ := bootstrapAddrs(ccq.logger, bootstrapPeers, h.ID()) + + // TODO(leo): Persistent data store (i.e. address book) + idht, err := dht.New(ctx, h, dht.Mode(dht.ModeServer), + // This intentionally makes us incompatible with the global IPFS DHT + dht.ProtocolPrefix(protocol.ID("/"+networkID)), + dht.BootstrapPeers(bootstrappers...), + ) + return idht, err + }), + ) + + if err != nil { + return fmt.Errorf("failed to create p2p: %w", err) + } + + topic_req := fmt.Sprintf("%s/%s", networkID, "ccq_req") + topic_resp := fmt.Sprintf("%s/%s", networkID, "ccq_resp") + + ccq.logger.Info("Creating pubsub topics", zap.String("request_topic", topic_req), zap.String("response_topic", topic_resp)) + ps, err := pubsub.NewGossipSub(ctx, ccq.h, + // We only want to accept subscribes from peers in the allow list. + pubsub.WithPeerFilter(func(peerID peer.ID, topic string) bool { + if len(ccq.allowedPeers) == 0 { + return true + } + if _, found := ccq.allowedPeers[peerID.String()]; found { + return true + } + ccq.logger.Debug("Dropping subscribe attempt from unknown peer", zap.String("peerID", peerID.String())) + return false + })) + if err != nil { + return fmt.Errorf("failed to create new gossip sub for req: %w", err) + } + + // We want to join and subscribe to the request topic. We will receive messages from there, but never write to it. + ccq.th_req, err = ps.Join(topic_req) + if err != nil { + return fmt.Errorf("failed to join topic_req: %w", err) + } + + // We only want to join the response topic. We will only write to it. + ccq.th_resp, err = ps.Join(topic_resp) + if err != nil { + return fmt.Errorf("failed to join topic_resp: %w", err) + } + + // We only want to accept messages from peers in the allow list. + err = ps.RegisterTopicValidator(topic_req, func(ctx context.Context, from peer.ID, msg *pubsub.Message) bool { + if len(ccq.allowedPeers) == 0 { + return true + } + if _, found := ccq.allowedPeers[from.String()]; found { + return true + } + ccq.logger.Debug("Dropping message from unknown peer", zap.String("fromPeerID", from.String())) + return false + }) + if err != nil { + return fmt.Errorf("failed to register message filter: %w", err) + } + + // Increase the buffer size to prevent failed delivery to slower subscribers + ccq.sub, err = ccq.th_req.Subscribe(pubsub.WithBufferSize(1024)) + if err != nil { + return fmt.Errorf("failed to subscribe topic_req: %w", err) + } + + common.StartRunnable(ctx, errC, false, "ccqp2p_listener", func(ctx context.Context) error { + return ccq.listener(ctx, signedQueryReqC) + }) + + common.StartRunnable(ctx, errC, false, "ccqp2p_publisher", func(ctx context.Context) error { + return ccq.publisher(ctx, gk, queryResponseReadC) + }) + + ccq.logger.Info("Node has been started", zap.String("peer_id", ccq.h.ID().String()), zap.String("addrs", fmt.Sprintf("%v", ccq.h.Addrs()))) + return nil +} + +func (ccq *ccqP2p) close() { + ccq.logger.Info("entering close") + + if err := ccq.th_req.Close(); err != nil && !errors.Is(err, context.Canceled) { + ccq.logger.Error("Error closing the topic_req", zap.Error(err)) + } + if err := ccq.th_resp.Close(); err != nil && !errors.Is(err, context.Canceled) { + ccq.logger.Error("Error closing the topic_req", zap.Error(err)) + } + + ccq.sub.Cancel() + + if err := ccq.h.Close(); err != nil { + ccq.logger.Error("error closing the host", zap.Error(err)) + } +} + +func (ccq *ccqP2p) listener(ctx context.Context, signedQueryReqC chan<- *gossipv1.SignedQueryRequest) error { + for { + envelope, err := ccq.sub.Next(ctx) // Note: sub.Next(ctx) will return an error once ctx is canceled + if err != nil { + return fmt.Errorf("failed to receive pubsub message: %w", err) + } + + var msg gossipv1.GossipMessage + err = proto.Unmarshal(envelope.Data, &msg) + if err != nil { + ccq.logger.Info("received invalid message", + zap.Binary("data", envelope.Data), + zap.String("from", envelope.GetFrom().String())) + ccqP2pMessagesReceived.WithLabelValues("invalid").Inc() + continue + } + + ccq.logger.Info("received message", //TODO: Change to Debug + zap.Any("payload", msg.Message), + zap.Binary("raw", envelope.Data), + zap.String("from", envelope.GetFrom().String())) + + switch m := msg.Message.(type) { + case *gossipv1.GossipMessage_SignedQueryRequest: + if err := query.PostSignedQueryRequest(signedQueryReqC, m.SignedQueryRequest); err != nil { + ccq.logger.Warn("failed to handle query request", zap.Error(err)) + } + default: + ccqP2pMessagesReceived.WithLabelValues("unknown").Inc() + ccq.logger.Warn("received unknown message type (running outdated software?)", + zap.Any("payload", msg.Message), + zap.Binary("raw", envelope.Data), + zap.String("from", envelope.GetFrom().String())) + } + } +} + +func (ccq *ccqP2p) publisher(ctx context.Context, gk *ecdsa.PrivateKey, queryResponseReadC <-chan *query.QueryResponsePublication) error { + for { + select { + case <-ctx.Done(): + return nil + case msg := <-queryResponseReadC: + msgBytes, err := msg.Marshal() + if err != nil { + ccq.logger.Error("failed to marshal query response", zap.Error(err)) + continue + } + digest := query.GetQueryResponseDigestFromBytes(msgBytes) + sig, err := ethcrypto.Sign(digest.Bytes(), gk) + if err != nil { + panic(err) + } + envelope := &gossipv1.GossipMessage{ + Message: &gossipv1.GossipMessage_SignedQueryResponse{ + SignedQueryResponse: &gossipv1.SignedQueryResponse{ + QueryResponse: msgBytes, + Signature: sig, + }, + }, + } + b, err := proto.Marshal(envelope) + if err != nil { + panic(err) + } + err = ccq.th_resp.Publish(ctx, b) + ccqP2pMessagesSent.Inc() + if err != nil { + ccq.logger.Error("failed to publish query response", + zap.String("requestID", msg.RequestID()), + zap.Any("query_response", msg), + zap.Any("signature", sig), + zap.Error(err), + ) + } else { + ccq.logger.Info("published signed query response", //TODO: Change to Debug + zap.String("requestID", msg.RequestID()), + zap.Any("query_response", msg), + zap.Any("signature", sig), + ) + } + } + } +} diff --git a/node/pkg/p2p/p2p.go b/node/pkg/p2p/p2p.go index 98a63302e2..ef3d91fa1b 100644 --- a/node/pkg/p2p/p2p.go +++ b/node/pkg/p2p/p2p.go @@ -13,6 +13,7 @@ import ( "github.com/certusone/wormhole/node/pkg/accountant" "github.com/certusone/wormhole/node/pkg/common" "github.com/certusone/wormhole/node/pkg/governor" + "github.com/certusone/wormhole/node/pkg/query" "github.com/certusone/wormhole/node/pkg/version" eth_common "github.com/ethereum/go-ethereum/common" ethcrypto "github.com/ethereum/go-ethereum/crypto" @@ -249,6 +250,12 @@ func Run( components *Components, ibcFeaturesFunc func() string, gatewayRelayerEnabled bool, + ccqEnabled bool, + signedQueryReqC chan<- *gossipv1.SignedQueryRequest, + queryResponseReadC <-chan *query.QueryResponsePublication, + ccqBootstrapPeers string, + ccqPort uint, + ccqAllowedPeers string, ) func(ctx context.Context) error { if components == nil { components = DefaultComponents() @@ -342,6 +349,27 @@ func Run( bootTime := time.Now() + if ccqEnabled { + ccqErrC := make(chan error) + ccq := newCcqRunP2p(logger, ccqAllowedPeers) + if err := ccq.run(ctx, priv, gk, networkID, ccqBootstrapPeers, ccqPort, signedQueryReqC, queryResponseReadC, ccqErrC); err != nil { + return fmt.Errorf("failed to start p2p for CCQ: %w", err) + } + defer ccq.close() + go func() { + for { + select { + case <-ctx.Done(): + return + case ccqErr := <-ccqErrC: + logger.Error("ccqp2p returned an error", zap.Error(ccqErr), zap.String("component", "ccqp2p")) + rootCtxCancel() + return + } + } + }() + } + // Periodically run guardian state set cleanup. go func() { ticker := time.NewTicker(15 * time.Second) @@ -403,6 +431,9 @@ func Run( if gatewayRelayerEnabled { features = append(features, "gwrelayer") } + if ccqEnabled { + features = append(features, "ccq") + } heartbeat := &gossipv1.Heartbeat{ NodeName: nodeName, diff --git a/node/pkg/p2p/watermark_test.go b/node/pkg/p2p/watermark_test.go index a1f7b925e3..3e1373b282 100644 --- a/node/pkg/p2p/watermark_test.go +++ b/node/pkg/p2p/watermark_test.go @@ -185,5 +185,11 @@ func startGuardian(t *testing.T, ctx context.Context, g *G) { g.components, nil, // ibc feature string false, // gateway relayer enabled + false, // ccqEnabled + nil, // signed query request channel + nil, // query response channel + "", // query bootstrap peers + 0, // query port + "", // query allowed peers )) } diff --git a/node/pkg/proto/gossip/v1/gossip.pb.go b/node/pkg/proto/gossip/v1/gossip.pb.go index 291a25c3ac..2165fe345d 100644 --- a/node/pkg/proto/gossip/v1/gossip.pb.go +++ b/node/pkg/proto/gossip/v1/gossip.pb.go @@ -35,6 +35,8 @@ type GossipMessage struct { // *GossipMessage_SignedBatchVaaWithQuorum // *GossipMessage_SignedChainGovernorConfig // *GossipMessage_SignedChainGovernorStatus + // *GossipMessage_SignedQueryRequest + // *GossipMessage_SignedQueryResponse Message isGossipMessage_Message `protobuf_oneof:"message"` } @@ -133,6 +135,20 @@ func (x *GossipMessage) GetSignedChainGovernorStatus() *SignedChainGovernorStatu return nil } +func (x *GossipMessage) GetSignedQueryRequest() *SignedQueryRequest { + if x, ok := x.GetMessage().(*GossipMessage_SignedQueryRequest); ok { + return x.SignedQueryRequest + } + return nil +} + +func (x *GossipMessage) GetSignedQueryResponse() *SignedQueryResponse { + if x, ok := x.GetMessage().(*GossipMessage_SignedQueryResponse); ok { + return x.SignedQueryResponse + } + return nil +} + type isGossipMessage_Message interface { isGossipMessage_Message() } @@ -169,6 +185,14 @@ type GossipMessage_SignedChainGovernorStatus struct { SignedChainGovernorStatus *SignedChainGovernorStatus `protobuf:"bytes,9,opt,name=signed_chain_governor_status,json=signedChainGovernorStatus,proto3,oneof"` } +type GossipMessage_SignedQueryRequest struct { + SignedQueryRequest *SignedQueryRequest `protobuf:"bytes,10,opt,name=signed_query_request,json=signedQueryRequest,proto3,oneof"` +} + +type GossipMessage_SignedQueryResponse struct { + SignedQueryResponse *SignedQueryResponse `protobuf:"bytes,11,opt,name=signed_query_response,json=signedQueryResponse,proto3,oneof"` +} + func (*GossipMessage_SignedObservation) isGossipMessage_Message() {} func (*GossipMessage_SignedHeartbeat) isGossipMessage_Message() {} @@ -185,6 +209,10 @@ func (*GossipMessage_SignedChainGovernorConfig) isGossipMessage_Message() {} func (*GossipMessage_SignedChainGovernorStatus) isGossipMessage_Message() {} +func (*GossipMessage_SignedQueryRequest) isGossipMessage_Message() {} + +func (*GossipMessage_SignedQueryResponse) isGossipMessage_Message() {} + type SignedHeartbeat struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1127,6 +1155,120 @@ func (x *ChainGovernorStatus) GetChains() []*ChainGovernorStatus_Chain { return nil } +type SignedQueryRequest struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Serialized QueryRequest message. + QueryRequest []byte `protobuf:"bytes,1,opt,name=query_request,json=queryRequest,proto3" json:"query_request,omitempty"` + // ECDSA signature using the requestor's public key. + Signature []byte `protobuf:"bytes,2,opt,name=signature,proto3" json:"signature,omitempty"` +} + +func (x *SignedQueryRequest) Reset() { + *x = SignedQueryRequest{} + if protoimpl.UnsafeEnabled { + mi := &file_gossip_v1_gossip_proto_msgTypes[13] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SignedQueryRequest) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SignedQueryRequest) ProtoMessage() {} + +func (x *SignedQueryRequest) ProtoReflect() protoreflect.Message { + mi := &file_gossip_v1_gossip_proto_msgTypes[13] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SignedQueryRequest.ProtoReflect.Descriptor instead. +func (*SignedQueryRequest) Descriptor() ([]byte, []int) { + return file_gossip_v1_gossip_proto_rawDescGZIP(), []int{13} +} + +func (x *SignedQueryRequest) GetQueryRequest() []byte { + if x != nil { + return x.QueryRequest + } + return nil +} + +func (x *SignedQueryRequest) GetSignature() []byte { + if x != nil { + return x.Signature + } + return nil +} + +type SignedQueryResponse struct { + state protoimpl.MessageState + sizeCache protoimpl.SizeCache + unknownFields protoimpl.UnknownFields + + // Serialized QueryResponse message. + QueryResponse []byte `protobuf:"bytes,1,opt,name=query_response,json=queryResponse,proto3" json:"query_response,omitempty"` + // ECDSA signature using the node's guardian public key. + Signature []byte `protobuf:"bytes,2,opt,name=signature,proto3" json:"signature,omitempty"` +} + +func (x *SignedQueryResponse) Reset() { + *x = SignedQueryResponse{} + if protoimpl.UnsafeEnabled { + mi := &file_gossip_v1_gossip_proto_msgTypes[14] + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + ms.StoreMessageInfo(mi) + } +} + +func (x *SignedQueryResponse) String() string { + return protoimpl.X.MessageStringOf(x) +} + +func (*SignedQueryResponse) ProtoMessage() {} + +func (x *SignedQueryResponse) ProtoReflect() protoreflect.Message { + mi := &file_gossip_v1_gossip_proto_msgTypes[14] + if protoimpl.UnsafeEnabled && x != nil { + ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) + if ms.LoadMessageInfo() == nil { + ms.StoreMessageInfo(mi) + } + return ms + } + return mi.MessageOf(x) +} + +// Deprecated: Use SignedQueryResponse.ProtoReflect.Descriptor instead. +func (*SignedQueryResponse) Descriptor() ([]byte, []int) { + return file_gossip_v1_gossip_proto_rawDescGZIP(), []int{14} +} + +func (x *SignedQueryResponse) GetQueryResponse() []byte { + if x != nil { + return x.QueryResponse + } + return nil +} + +func (x *SignedQueryResponse) GetSignature() []byte { + if x != nil { + return x.Signature + } + return nil +} + type Heartbeat_Network struct { state protoimpl.MessageState sizeCache protoimpl.SizeCache @@ -1145,7 +1287,7 @@ type Heartbeat_Network struct { func (x *Heartbeat_Network) Reset() { *x = Heartbeat_Network{} if protoimpl.UnsafeEnabled { - mi := &file_gossip_v1_gossip_proto_msgTypes[13] + mi := &file_gossip_v1_gossip_proto_msgTypes[15] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1158,7 +1300,7 @@ func (x *Heartbeat_Network) String() string { func (*Heartbeat_Network) ProtoMessage() {} func (x *Heartbeat_Network) ProtoReflect() protoreflect.Message { - mi := &file_gossip_v1_gossip_proto_msgTypes[13] + mi := &file_gossip_v1_gossip_proto_msgTypes[15] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1215,7 +1357,7 @@ type ChainGovernorConfig_Chain struct { func (x *ChainGovernorConfig_Chain) Reset() { *x = ChainGovernorConfig_Chain{} if protoimpl.UnsafeEnabled { - mi := &file_gossip_v1_gossip_proto_msgTypes[14] + mi := &file_gossip_v1_gossip_proto_msgTypes[16] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1228,7 +1370,7 @@ func (x *ChainGovernorConfig_Chain) String() string { func (*ChainGovernorConfig_Chain) ProtoMessage() {} func (x *ChainGovernorConfig_Chain) ProtoReflect() protoreflect.Message { - mi := &file_gossip_v1_gossip_proto_msgTypes[14] + mi := &file_gossip_v1_gossip_proto_msgTypes[16] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1278,7 +1420,7 @@ type ChainGovernorConfig_Token struct { func (x *ChainGovernorConfig_Token) Reset() { *x = ChainGovernorConfig_Token{} if protoimpl.UnsafeEnabled { - mi := &file_gossip_v1_gossip_proto_msgTypes[15] + mi := &file_gossip_v1_gossip_proto_msgTypes[17] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1291,7 +1433,7 @@ func (x *ChainGovernorConfig_Token) String() string { func (*ChainGovernorConfig_Token) ProtoMessage() {} func (x *ChainGovernorConfig_Token) ProtoReflect() protoreflect.Message { - mi := &file_gossip_v1_gossip_proto_msgTypes[15] + mi := &file_gossip_v1_gossip_proto_msgTypes[17] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1342,7 +1484,7 @@ type ChainGovernorStatus_EnqueuedVAA struct { func (x *ChainGovernorStatus_EnqueuedVAA) Reset() { *x = ChainGovernorStatus_EnqueuedVAA{} if protoimpl.UnsafeEnabled { - mi := &file_gossip_v1_gossip_proto_msgTypes[16] + mi := &file_gossip_v1_gossip_proto_msgTypes[18] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1355,7 +1497,7 @@ func (x *ChainGovernorStatus_EnqueuedVAA) String() string { func (*ChainGovernorStatus_EnqueuedVAA) ProtoMessage() {} func (x *ChainGovernorStatus_EnqueuedVAA) ProtoReflect() protoreflect.Message { - mi := &file_gossip_v1_gossip_proto_msgTypes[16] + mi := &file_gossip_v1_gossip_proto_msgTypes[18] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1412,7 +1554,7 @@ type ChainGovernorStatus_Emitter struct { func (x *ChainGovernorStatus_Emitter) Reset() { *x = ChainGovernorStatus_Emitter{} if protoimpl.UnsafeEnabled { - mi := &file_gossip_v1_gossip_proto_msgTypes[17] + mi := &file_gossip_v1_gossip_proto_msgTypes[19] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1425,7 +1567,7 @@ func (x *ChainGovernorStatus_Emitter) String() string { func (*ChainGovernorStatus_Emitter) ProtoMessage() {} func (x *ChainGovernorStatus_Emitter) ProtoReflect() protoreflect.Message { - mi := &file_gossip_v1_gossip_proto_msgTypes[17] + mi := &file_gossip_v1_gossip_proto_msgTypes[19] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1475,7 +1617,7 @@ type ChainGovernorStatus_Chain struct { func (x *ChainGovernorStatus_Chain) Reset() { *x = ChainGovernorStatus_Chain{} if protoimpl.UnsafeEnabled { - mi := &file_gossip_v1_gossip_proto_msgTypes[18] + mi := &file_gossip_v1_gossip_proto_msgTypes[20] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -1488,7 +1630,7 @@ func (x *ChainGovernorStatus_Chain) String() string { func (*ChainGovernorStatus_Chain) ProtoMessage() {} func (x *ChainGovernorStatus_Chain) ProtoReflect() protoreflect.Message { - mi := &file_gossip_v1_gossip_proto_msgTypes[18] + mi := &file_gossip_v1_gossip_proto_msgTypes[20] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -1530,7 +1672,7 @@ var File_gossip_v1_gossip_proto protoreflect.FileDescriptor var file_gossip_v1_gossip_proto_rawDesc = []byte{ 0x0a, 0x16, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x2f, 0x76, 0x31, 0x2f, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x12, 0x09, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, - 0x2e, 0x76, 0x31, 0x22, 0x86, 0x06, 0x0a, 0x0d, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x4d, 0x65, + 0x2e, 0x76, 0x31, 0x22, 0xaf, 0x07, 0x0a, 0x0d, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x12, 0x4d, 0x0a, 0x12, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x5f, 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, @@ -1578,180 +1720,202 @@ var file_gossip_v1_gossip_proto_rawDesc = []byte{ 0x64, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x47, 0x6f, 0x76, 0x65, 0x72, 0x6e, 0x6f, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x48, 0x00, 0x52, 0x19, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x47, 0x6f, 0x76, 0x65, 0x72, 0x6e, 0x6f, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, - 0x73, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x72, 0x0a, 0x0f, - 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x12, - 0x1c, 0x0a, 0x09, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0c, 0x52, 0x09, 0x68, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x12, 0x1c, 0x0a, + 0x73, 0x12, 0x51, 0x0a, 0x14, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x5f, 0x71, 0x75, 0x65, 0x72, + 0x79, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x0a, 0x20, 0x01, 0x28, 0x0b, 0x32, + 0x1d, 0x2e, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x2e, 0x76, 0x31, 0x2e, 0x53, 0x69, 0x67, 0x6e, + 0x65, 0x64, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x48, 0x00, + 0x52, 0x12, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x12, 0x54, 0x0a, 0x15, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x5f, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x0b, 0x20, + 0x01, 0x28, 0x0b, 0x32, 0x1e, 0x2e, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x2e, 0x76, 0x31, 0x2e, + 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x48, 0x00, 0x52, 0x13, 0x73, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x51, 0x75, 0x65, + 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x42, 0x09, 0x0a, 0x07, 0x6d, 0x65, + 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x72, 0x0a, 0x0f, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x48, + 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x68, 0x65, 0x61, 0x72, + 0x74, 0x62, 0x65, 0x61, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x68, 0x65, 0x61, + 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, + 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, + 0x74, 0x75, 0x72, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x67, 0x75, 0x61, 0x72, 0x64, 0x69, 0x61, 0x6e, + 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x67, 0x75, 0x61, + 0x72, 0x64, 0x69, 0x61, 0x6e, 0x41, 0x64, 0x64, 0x72, 0x22, 0xbb, 0x03, 0x0a, 0x09, 0x48, 0x65, + 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x12, 0x1b, 0x0a, 0x09, 0x6e, 0x6f, 0x64, 0x65, 0x5f, + 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6e, 0x6f, 0x64, 0x65, + 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x1c, + 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x03, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x38, 0x0a, 0x08, + 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, + 0x2e, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x2e, 0x76, 0x31, 0x2e, 0x48, 0x65, 0x61, 0x72, 0x74, + 0x62, 0x65, 0x61, 0x74, 0x2e, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x52, 0x08, 0x6e, 0x65, + 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x12, 0x18, 0x0a, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, + 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, + 0x12, 0x23, 0x0a, 0x0d, 0x67, 0x75, 0x61, 0x72, 0x64, 0x69, 0x61, 0x6e, 0x5f, 0x61, 0x64, 0x64, + 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x67, 0x75, 0x61, 0x72, 0x64, 0x69, 0x61, + 0x6e, 0x41, 0x64, 0x64, 0x72, 0x12, 0x25, 0x0a, 0x0e, 0x62, 0x6f, 0x6f, 0x74, 0x5f, 0x74, 0x69, + 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x07, 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x62, + 0x6f, 0x6f, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x1a, 0x0a, 0x08, + 0x66, 0x65, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x18, 0x08, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, + 0x66, 0x65, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x12, 0x1e, 0x0a, 0x0b, 0x70, 0x32, 0x70, 0x5f, + 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x09, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x70, + 0x32, 0x70, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x64, 0x1a, 0x7d, 0x0a, 0x07, 0x4e, 0x65, 0x74, 0x77, + 0x6f, 0x72, 0x6b, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x02, 0x69, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x68, 0x65, 0x69, 0x67, 0x68, 0x74, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x03, 0x52, 0x06, 0x68, 0x65, 0x69, 0x67, 0x68, 0x74, 0x12, 0x29, 0x0a, 0x10, 0x63, + 0x6f, 0x6e, 0x74, 0x72, 0x61, 0x63, 0x74, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x61, 0x63, 0x74, 0x41, + 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x1f, 0x0a, 0x0b, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, + 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x65, 0x72, 0x72, + 0x6f, 0x72, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x91, 0x01, 0x0a, 0x11, 0x53, 0x69, 0x67, 0x6e, + 0x65, 0x64, 0x4f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, + 0x04, 0x61, 0x64, 0x64, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x61, 0x64, 0x64, + 0x72, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x61, 0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x04, 0x68, 0x61, 0x73, 0x68, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, + 0x72, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, + 0x75, 0x72, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x78, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x04, + 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x74, 0x78, 0x48, 0x61, 0x73, 0x68, 0x12, 0x1d, 0x0a, 0x0a, + 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, + 0x52, 0x09, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x49, 0x64, 0x22, 0x27, 0x0a, 0x13, 0x53, + 0x69, 0x67, 0x6e, 0x65, 0x64, 0x56, 0x41, 0x41, 0x57, 0x69, 0x74, 0x68, 0x51, 0x75, 0x6f, 0x72, + 0x75, 0x6d, 0x12, 0x10, 0x0a, 0x03, 0x76, 0x61, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x03, 0x76, 0x61, 0x61, 0x22, 0x8e, 0x01, 0x0a, 0x18, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x4f, + 0x62, 0x73, 0x65, 0x72, 0x76, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x2f, 0x0a, 0x13, 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x61, 0x74, 0x69, 0x6f, 0x6e, + 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x12, + 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, + 0x12, 0x23, 0x0a, 0x0d, 0x67, 0x75, 0x61, 0x72, 0x64, 0x69, 0x61, 0x6e, 0x5f, 0x61, 0x64, 0x64, + 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x67, 0x75, 0x61, 0x72, 0x64, 0x69, 0x61, + 0x6e, 0x41, 0x64, 0x64, 0x72, 0x22, 0x48, 0x0a, 0x12, 0x4f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x61, + 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x63, + 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x63, + 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x78, 0x5f, 0x68, 0x61, 0x73, + 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x74, 0x78, 0x48, 0x61, 0x73, 0x68, 0x22, + 0xbf, 0x01, 0x0a, 0x16, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, + 0x62, 0x73, 0x65, 0x72, 0x76, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x61, 0x64, + 0x64, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x61, 0x64, 0x64, 0x72, 0x12, 0x12, + 0x0a, 0x04, 0x68, 0x61, 0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x68, 0x61, + 0x73, 0x68, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, + 0x12, 0x13, 0x0a, 0x05, 0x74, 0x78, 0x5f, 0x69, 0x64, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x04, 0x74, 0x78, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, + 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, + 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x6f, 0x6e, 0x63, 0x65, 0x18, 0x06, 0x20, 0x01, 0x28, 0x0d, 0x52, + 0x05, 0x6e, 0x6f, 0x6e, 0x63, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, + 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x62, 0x61, 0x74, 0x63, 0x68, 0x49, + 0x64, 0x22, 0x98, 0x01, 0x0a, 0x18, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x42, 0x61, 0x74, 0x63, + 0x68, 0x56, 0x41, 0x41, 0x57, 0x69, 0x74, 0x68, 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x12, 0x1b, + 0x0a, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x76, 0x61, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, + 0x0c, 0x52, 0x08, 0x62, 0x61, 0x74, 0x63, 0x68, 0x56, 0x61, 0x61, 0x12, 0x19, 0x0a, 0x08, 0x63, + 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x63, + 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x78, 0x5f, 0x69, 0x64, 0x18, + 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x74, 0x78, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x6e, + 0x6f, 0x6e, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x6e, 0x6f, 0x6e, 0x63, + 0x65, 0x12, 0x19, 0x0a, 0x08, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x07, 0x62, 0x61, 0x74, 0x63, 0x68, 0x49, 0x64, 0x22, 0x76, 0x0a, 0x19, + 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x47, 0x6f, 0x76, 0x65, 0x72, + 0x6e, 0x6f, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x16, 0x0a, 0x06, 0x63, 0x6f, 0x6e, + 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, + 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x12, + 0x23, 0x0a, 0x0d, 0x67, 0x75, 0x61, 0x72, 0x64, 0x69, 0x61, 0x6e, 0x5f, 0x61, 0x64, 0x64, 0x72, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x67, 0x75, 0x61, 0x72, 0x64, 0x69, 0x61, 0x6e, + 0x41, 0x64, 0x64, 0x72, 0x22, 0xd1, 0x03, 0x0a, 0x13, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x47, 0x6f, + 0x76, 0x65, 0x72, 0x6e, 0x6f, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x1b, 0x0a, 0x09, + 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x6f, 0x75, + 0x6e, 0x74, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x63, 0x6f, 0x75, 0x6e, + 0x74, 0x65, 0x72, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, + 0x70, 0x12, 0x3c, 0x0a, 0x06, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, + 0x0b, 0x32, 0x24, 0x2e, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x68, + 0x61, 0x69, 0x6e, 0x47, 0x6f, 0x76, 0x65, 0x72, 0x6e, 0x6f, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, + 0x67, 0x2e, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x52, 0x06, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x73, 0x12, + 0x3c, 0x0a, 0x06, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x73, 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, + 0x24, 0x2e, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x68, 0x61, 0x69, + 0x6e, 0x47, 0x6f, 0x76, 0x65, 0x72, 0x6e, 0x6f, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x2e, + 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x06, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x73, 0x1a, 0x7b, 0x0a, + 0x05, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, + 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, + 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x6e, 0x6f, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x6c, 0x69, + 0x6d, 0x69, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x6e, 0x6f, 0x74, 0x69, 0x6f, + 0x6e, 0x61, 0x6c, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x12, 0x30, 0x0a, 0x14, 0x62, 0x69, 0x67, 0x5f, + 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x73, 0x69, 0x7a, 0x65, + 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x12, 0x62, 0x69, 0x67, 0x54, 0x72, 0x61, 0x6e, 0x73, + 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x69, 0x7a, 0x65, 0x1a, 0x6c, 0x0a, 0x05, 0x54, 0x6f, + 0x6b, 0x65, 0x6e, 0x12, 0x26, 0x0a, 0x0f, 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x5f, 0x63, 0x68, + 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0d, 0x6f, 0x72, + 0x69, 0x67, 0x69, 0x6e, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x6f, + 0x72, 0x69, 0x67, 0x69, 0x6e, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x09, 0x52, 0x0d, 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x41, 0x64, 0x64, 0x72, 0x65, + 0x73, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x70, 0x72, 0x69, 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, + 0x02, 0x52, 0x05, 0x70, 0x72, 0x69, 0x63, 0x65, 0x22, 0x76, 0x0a, 0x19, 0x53, 0x69, 0x67, 0x6e, + 0x65, 0x64, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x47, 0x6f, 0x76, 0x65, 0x72, 0x6e, 0x6f, 0x72, 0x53, + 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x16, 0x0a, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, + 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x67, 0x75, 0x61, 0x72, 0x64, 0x69, 0x61, 0x6e, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x67, 0x75, 0x61, 0x72, 0x64, 0x69, 0x61, 0x6e, 0x41, 0x64, 0x64, 0x72, - 0x22, 0xbb, 0x03, 0x0a, 0x09, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x12, 0x1b, - 0x0a, 0x09, 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x09, 0x52, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x63, - 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x63, 0x6f, - 0x75, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, - 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, - 0x61, 0x6d, 0x70, 0x12, 0x38, 0x0a, 0x08, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x18, - 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x1c, 0x2e, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x2e, 0x76, - 0x31, 0x2e, 0x48, 0x65, 0x61, 0x72, 0x74, 0x62, 0x65, 0x61, 0x74, 0x2e, 0x4e, 0x65, 0x74, 0x77, - 0x6f, 0x72, 0x6b, 0x52, 0x08, 0x6e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x73, 0x12, 0x18, 0x0a, - 0x07, 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, - 0x76, 0x65, 0x72, 0x73, 0x69, 0x6f, 0x6e, 0x12, 0x23, 0x0a, 0x0d, 0x67, 0x75, 0x61, 0x72, 0x64, - 0x69, 0x61, 0x6e, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18, 0x06, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, - 0x67, 0x75, 0x61, 0x72, 0x64, 0x69, 0x61, 0x6e, 0x41, 0x64, 0x64, 0x72, 0x12, 0x25, 0x0a, 0x0e, - 0x62, 0x6f, 0x6f, 0x74, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x07, - 0x20, 0x01, 0x28, 0x03, 0x52, 0x0d, 0x62, 0x6f, 0x6f, 0x74, 0x54, 0x69, 0x6d, 0x65, 0x73, 0x74, - 0x61, 0x6d, 0x70, 0x12, 0x1a, 0x0a, 0x08, 0x66, 0x65, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x18, - 0x08, 0x20, 0x03, 0x28, 0x09, 0x52, 0x08, 0x66, 0x65, 0x61, 0x74, 0x75, 0x72, 0x65, 0x73, 0x12, - 0x1e, 0x0a, 0x0b, 0x70, 0x32, 0x70, 0x5f, 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x69, 0x64, 0x18, 0x09, - 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x70, 0x32, 0x70, 0x4e, 0x6f, 0x64, 0x65, 0x49, 0x64, 0x1a, - 0x7d, 0x0a, 0x07, 0x4e, 0x65, 0x74, 0x77, 0x6f, 0x72, 0x6b, 0x12, 0x0e, 0x0a, 0x02, 0x69, 0x64, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x02, 0x69, 0x64, 0x12, 0x16, 0x0a, 0x06, 0x68, 0x65, - 0x69, 0x67, 0x68, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x06, 0x68, 0x65, 0x69, 0x67, - 0x68, 0x74, 0x12, 0x29, 0x0a, 0x10, 0x63, 0x6f, 0x6e, 0x74, 0x72, 0x61, 0x63, 0x74, 0x5f, 0x61, - 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0f, 0x63, 0x6f, - 0x6e, 0x74, 0x72, 0x61, 0x63, 0x74, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x1f, 0x0a, - 0x0b, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x18, 0x04, 0x20, 0x01, - 0x28, 0x04, 0x52, 0x0a, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x43, 0x6f, 0x75, 0x6e, 0x74, 0x22, 0x91, - 0x01, 0x0a, 0x11, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x4f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x61, - 0x74, 0x69, 0x6f, 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x61, 0x64, 0x64, 0x72, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x0c, 0x52, 0x04, 0x61, 0x64, 0x64, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x61, 0x73, 0x68, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x68, 0x61, 0x73, 0x68, 0x12, 0x1c, 0x0a, 0x09, - 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, - 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x78, - 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x74, 0x78, 0x48, - 0x61, 0x73, 0x68, 0x12, 0x1d, 0x0a, 0x0a, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x5f, 0x69, - 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x09, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, - 0x49, 0x64, 0x22, 0x27, 0x0a, 0x13, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x56, 0x41, 0x41, 0x57, - 0x69, 0x74, 0x68, 0x51, 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x12, 0x10, 0x0a, 0x03, 0x76, 0x61, 0x61, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x76, 0x61, 0x61, 0x22, 0x8e, 0x01, 0x0a, 0x18, - 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x4f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x2f, 0x0a, 0x13, 0x6f, 0x62, 0x73, 0x65, - 0x72, 0x76, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, - 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x12, 0x6f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x61, 0x74, 0x69, - 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, - 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, - 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x67, 0x75, 0x61, 0x72, 0x64, - 0x69, 0x61, 0x6e, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, - 0x67, 0x75, 0x61, 0x72, 0x64, 0x69, 0x61, 0x6e, 0x41, 0x64, 0x64, 0x72, 0x22, 0x48, 0x0a, 0x12, - 0x4f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x61, 0x74, 0x69, 0x6f, 0x6e, 0x52, 0x65, 0x71, 0x75, 0x65, - 0x73, 0x74, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x17, 0x0a, - 0x07, 0x74, 0x78, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, - 0x74, 0x78, 0x48, 0x61, 0x73, 0x68, 0x22, 0xbf, 0x01, 0x0a, 0x16, 0x53, 0x69, 0x67, 0x6e, 0x65, - 0x64, 0x42, 0x61, 0x74, 0x63, 0x68, 0x4f, 0x62, 0x73, 0x65, 0x72, 0x76, 0x61, 0x74, 0x69, 0x6f, - 0x6e, 0x12, 0x12, 0x0a, 0x04, 0x61, 0x64, 0x64, 0x72, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, - 0x04, 0x61, 0x64, 0x64, 0x72, 0x12, 0x12, 0x0a, 0x04, 0x68, 0x61, 0x73, 0x68, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0c, 0x52, 0x04, 0x68, 0x61, 0x73, 0x68, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, - 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, - 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x12, 0x13, 0x0a, 0x05, 0x74, 0x78, 0x5f, 0x69, 0x64, - 0x18, 0x04, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x74, 0x78, 0x49, 0x64, 0x12, 0x19, 0x0a, 0x08, - 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, - 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x6f, 0x6e, 0x63, 0x65, - 0x18, 0x06, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x05, 0x6e, 0x6f, 0x6e, 0x63, 0x65, 0x12, 0x19, 0x0a, - 0x08, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x69, 0x64, 0x18, 0x07, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x07, 0x62, 0x61, 0x74, 0x63, 0x68, 0x49, 0x64, 0x22, 0x98, 0x01, 0x0a, 0x18, 0x53, 0x69, 0x67, - 0x6e, 0x65, 0x64, 0x42, 0x61, 0x74, 0x63, 0x68, 0x56, 0x41, 0x41, 0x57, 0x69, 0x74, 0x68, 0x51, - 0x75, 0x6f, 0x72, 0x75, 0x6d, 0x12, 0x1b, 0x0a, 0x09, 0x62, 0x61, 0x74, 0x63, 0x68, 0x5f, 0x76, - 0x61, 0x61, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x08, 0x62, 0x61, 0x74, 0x63, 0x68, 0x56, - 0x61, 0x61, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x13, 0x0a, - 0x05, 0x74, 0x78, 0x5f, 0x69, 0x64, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x04, 0x74, 0x78, - 0x49, 0x64, 0x12, 0x14, 0x0a, 0x05, 0x6e, 0x6f, 0x6e, 0x63, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, - 0x0d, 0x52, 0x05, 0x6e, 0x6f, 0x6e, 0x63, 0x65, 0x12, 0x19, 0x0a, 0x08, 0x62, 0x61, 0x74, 0x63, - 0x68, 0x5f, 0x69, 0x64, 0x18, 0x05, 0x20, 0x01, 0x28, 0x09, 0x52, 0x07, 0x62, 0x61, 0x74, 0x63, - 0x68, 0x49, 0x64, 0x22, 0x76, 0x0a, 0x19, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x43, 0x68, 0x61, - 0x69, 0x6e, 0x47, 0x6f, 0x76, 0x65, 0x72, 0x6e, 0x6f, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, - 0x12, 0x16, 0x0a, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, - 0x52, 0x06, 0x63, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, - 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, - 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x67, 0x75, 0x61, 0x72, 0x64, 0x69, - 0x61, 0x6e, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x67, - 0x75, 0x61, 0x72, 0x64, 0x69, 0x61, 0x6e, 0x41, 0x64, 0x64, 0x72, 0x22, 0xd1, 0x03, 0x0a, 0x13, - 0x43, 0x68, 0x61, 0x69, 0x6e, 0x47, 0x6f, 0x76, 0x65, 0x72, 0x6e, 0x6f, 0x72, 0x43, 0x6f, 0x6e, - 0x66, 0x69, 0x67, 0x12, 0x1b, 0x0a, 0x09, 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x4e, 0x61, 0x6d, 0x65, - 0x12, 0x18, 0x0a, 0x07, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, - 0x03, 0x52, 0x07, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, - 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, - 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x3c, 0x0a, 0x06, 0x63, 0x68, 0x61, 0x69, - 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x67, 0x6f, 0x73, 0x73, 0x69, - 0x70, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x47, 0x6f, 0x76, 0x65, 0x72, 0x6e, - 0x6f, 0x72, 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x2e, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x52, 0x06, - 0x63, 0x68, 0x61, 0x69, 0x6e, 0x73, 0x12, 0x3c, 0x0a, 0x06, 0x74, 0x6f, 0x6b, 0x65, 0x6e, 0x73, - 0x18, 0x05, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x2e, - 0x76, 0x31, 0x2e, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x47, 0x6f, 0x76, 0x65, 0x72, 0x6e, 0x6f, 0x72, - 0x43, 0x6f, 0x6e, 0x66, 0x69, 0x67, 0x2e, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x52, 0x06, 0x74, 0x6f, - 0x6b, 0x65, 0x6e, 0x73, 0x1a, 0x7b, 0x0a, 0x05, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x12, 0x19, 0x0a, - 0x08, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, - 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x6e, 0x6f, 0x74, 0x69, - 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x6c, 0x69, 0x6d, 0x69, 0x74, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, - 0x52, 0x0d, 0x6e, 0x6f, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x4c, 0x69, 0x6d, 0x69, 0x74, 0x12, - 0x30, 0x0a, 0x14, 0x62, 0x69, 0x67, 0x5f, 0x74, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, - 0x6f, 0x6e, 0x5f, 0x73, 0x69, 0x7a, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x12, 0x62, - 0x69, 0x67, 0x54, 0x72, 0x61, 0x6e, 0x73, 0x61, 0x63, 0x74, 0x69, 0x6f, 0x6e, 0x53, 0x69, 0x7a, - 0x65, 0x1a, 0x6c, 0x0a, 0x05, 0x54, 0x6f, 0x6b, 0x65, 0x6e, 0x12, 0x26, 0x0a, 0x0f, 0x6f, 0x72, - 0x69, 0x67, 0x69, 0x6e, 0x5f, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0d, 0x52, 0x0d, 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x43, 0x68, 0x61, 0x69, 0x6e, - 0x49, 0x64, 0x12, 0x25, 0x0a, 0x0e, 0x6f, 0x72, 0x69, 0x67, 0x69, 0x6e, 0x5f, 0x61, 0x64, 0x64, - 0x72, 0x65, 0x73, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0d, 0x6f, 0x72, 0x69, 0x67, - 0x69, 0x6e, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x14, 0x0a, 0x05, 0x70, 0x72, 0x69, - 0x63, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x02, 0x52, 0x05, 0x70, 0x72, 0x69, 0x63, 0x65, 0x22, - 0x76, 0x0a, 0x19, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x47, 0x6f, - 0x76, 0x65, 0x72, 0x6e, 0x6f, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x16, 0x0a, 0x06, - 0x73, 0x74, 0x61, 0x74, 0x75, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x06, 0x73, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, - 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, - 0x72, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x67, 0x75, 0x61, 0x72, 0x64, 0x69, 0x61, 0x6e, 0x5f, 0x61, - 0x64, 0x64, 0x72, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x67, 0x75, 0x61, 0x72, 0x64, - 0x69, 0x61, 0x6e, 0x41, 0x64, 0x64, 0x72, 0x22, 0x98, 0x05, 0x0a, 0x13, 0x43, 0x68, 0x61, 0x69, - 0x6e, 0x47, 0x6f, 0x76, 0x65, 0x72, 0x6e, 0x6f, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, - 0x1b, 0x0a, 0x09, 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, - 0x28, 0x09, 0x52, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, - 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x63, - 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x12, 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, - 0x61, 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, 0x28, 0x03, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, - 0x74, 0x61, 0x6d, 0x70, 0x12, 0x3c, 0x0a, 0x06, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x73, 0x18, 0x04, - 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x2e, 0x76, 0x31, - 0x2e, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x47, 0x6f, 0x76, 0x65, 0x72, 0x6e, 0x6f, 0x72, 0x53, 0x74, - 0x61, 0x74, 0x75, 0x73, 0x2e, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x52, 0x06, 0x63, 0x68, 0x61, 0x69, - 0x6e, 0x73, 0x1a, 0x8c, 0x01, 0x0a, 0x0b, 0x45, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x64, 0x56, - 0x41, 0x41, 0x12, 0x1a, 0x0a, 0x08, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, - 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x73, 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x21, - 0x0a, 0x0c, 0x72, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, - 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x72, 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x54, 0x69, 0x6d, - 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x6e, 0x6f, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x76, 0x61, - 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x6e, 0x6f, 0x74, 0x69, 0x6f, - 0x6e, 0x61, 0x6c, 0x56, 0x61, 0x6c, 0x75, 0x65, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x78, 0x5f, 0x68, - 0x61, 0x73, 0x68, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x06, 0x74, 0x78, 0x48, 0x61, 0x73, - 0x68, 0x1a, 0xb3, 0x01, 0x0a, 0x07, 0x45, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x72, 0x12, 0x27, 0x0a, - 0x0f, 0x65, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x72, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, - 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0e, 0x65, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x72, 0x41, - 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, 0x2e, 0x0a, 0x13, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, - 0x65, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x64, 0x5f, 0x76, 0x61, 0x61, 0x73, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x04, 0x52, 0x11, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x45, 0x6e, 0x71, 0x75, 0x65, 0x75, - 0x65, 0x64, 0x56, 0x61, 0x61, 0x73, 0x12, 0x4f, 0x0a, 0x0d, 0x65, 0x6e, 0x71, 0x75, 0x65, 0x75, - 0x65, 0x64, 0x5f, 0x76, 0x61, 0x61, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2a, 0x2e, + 0x22, 0x98, 0x05, 0x0a, 0x13, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x47, 0x6f, 0x76, 0x65, 0x72, 0x6e, + 0x6f, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x6e, 0x6f, 0x64, 0x65, + 0x5f, 0x6e, 0x61, 0x6d, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, 0x08, 0x6e, 0x6f, 0x64, + 0x65, 0x4e, 0x61, 0x6d, 0x65, 0x12, 0x18, 0x0a, 0x07, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, + 0x18, 0x02, 0x20, 0x01, 0x28, 0x03, 0x52, 0x07, 0x63, 0x6f, 0x75, 0x6e, 0x74, 0x65, 0x72, 0x12, + 0x1c, 0x0a, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x03, 0x52, 0x09, 0x74, 0x69, 0x6d, 0x65, 0x73, 0x74, 0x61, 0x6d, 0x70, 0x12, 0x3c, 0x0a, + 0x06, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x73, 0x18, 0x04, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x24, 0x2e, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x47, - 0x6f, 0x76, 0x65, 0x72, 0x6e, 0x6f, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2e, 0x45, 0x6e, - 0x71, 0x75, 0x65, 0x75, 0x65, 0x64, 0x56, 0x41, 0x41, 0x52, 0x0c, 0x65, 0x6e, 0x71, 0x75, 0x65, - 0x75, 0x65, 0x64, 0x56, 0x61, 0x61, 0x73, 0x1a, 0xa8, 0x01, 0x0a, 0x05, 0x43, 0x68, 0x61, 0x69, - 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, - 0x01, 0x28, 0x0d, 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x40, 0x0a, 0x1c, - 0x72, 0x65, 0x6d, 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x5f, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, - 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x6f, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x18, 0x02, 0x20, 0x01, - 0x28, 0x04, 0x52, 0x1a, 0x72, 0x65, 0x6d, 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x41, 0x76, 0x61, - 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x4e, 0x6f, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x12, 0x42, - 0x0a, 0x08, 0x65, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, - 0x32, 0x26, 0x2e, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x68, 0x61, - 0x69, 0x6e, 0x47, 0x6f, 0x76, 0x65, 0x72, 0x6e, 0x6f, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, - 0x2e, 0x45, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x72, 0x52, 0x08, 0x65, 0x6d, 0x69, 0x74, 0x74, 0x65, - 0x72, 0x73, 0x42, 0x41, 0x5a, 0x3f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, - 0x2f, 0x63, 0x65, 0x72, 0x74, 0x75, 0x73, 0x6f, 0x6e, 0x65, 0x2f, 0x77, 0x6f, 0x72, 0x6d, 0x68, - 0x6f, 0x6c, 0x65, 0x2f, 0x6e, 0x6f, 0x64, 0x65, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, - 0x74, 0x6f, 0x2f, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x2f, 0x76, 0x31, 0x3b, 0x67, 0x6f, 0x73, - 0x73, 0x69, 0x70, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x6f, 0x76, 0x65, 0x72, 0x6e, 0x6f, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2e, 0x43, 0x68, + 0x61, 0x69, 0x6e, 0x52, 0x06, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x73, 0x1a, 0x8c, 0x01, 0x0a, 0x0b, + 0x45, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x64, 0x56, 0x41, 0x41, 0x12, 0x1a, 0x0a, 0x08, 0x73, + 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x18, 0x01, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x73, + 0x65, 0x71, 0x75, 0x65, 0x6e, 0x63, 0x65, 0x12, 0x21, 0x0a, 0x0c, 0x72, 0x65, 0x6c, 0x65, 0x61, + 0x73, 0x65, 0x5f, 0x74, 0x69, 0x6d, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x0b, 0x72, + 0x65, 0x6c, 0x65, 0x61, 0x73, 0x65, 0x54, 0x69, 0x6d, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x6e, 0x6f, + 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x5f, 0x76, 0x61, 0x6c, 0x75, 0x65, 0x18, 0x03, 0x20, 0x01, + 0x28, 0x04, 0x52, 0x0d, 0x6e, 0x6f, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x56, 0x61, 0x6c, 0x75, + 0x65, 0x12, 0x17, 0x0a, 0x07, 0x74, 0x78, 0x5f, 0x68, 0x61, 0x73, 0x68, 0x18, 0x04, 0x20, 0x01, + 0x28, 0x09, 0x52, 0x06, 0x74, 0x78, 0x48, 0x61, 0x73, 0x68, 0x1a, 0xb3, 0x01, 0x0a, 0x07, 0x45, + 0x6d, 0x69, 0x74, 0x74, 0x65, 0x72, 0x12, 0x27, 0x0a, 0x0f, 0x65, 0x6d, 0x69, 0x74, 0x74, 0x65, + 0x72, 0x5f, 0x61, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x18, 0x01, 0x20, 0x01, 0x28, 0x09, 0x52, + 0x0e, 0x65, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x72, 0x41, 0x64, 0x64, 0x72, 0x65, 0x73, 0x73, 0x12, + 0x2e, 0x0a, 0x13, 0x74, 0x6f, 0x74, 0x61, 0x6c, 0x5f, 0x65, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, + 0x64, 0x5f, 0x76, 0x61, 0x61, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x11, 0x74, 0x6f, + 0x74, 0x61, 0x6c, 0x45, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x64, 0x56, 0x61, 0x61, 0x73, 0x12, + 0x4f, 0x0a, 0x0d, 0x65, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x64, 0x5f, 0x76, 0x61, 0x61, 0x73, + 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x2a, 0x2e, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x2e, + 0x76, 0x31, 0x2e, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x47, 0x6f, 0x76, 0x65, 0x72, 0x6e, 0x6f, 0x72, + 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2e, 0x45, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x64, 0x56, + 0x41, 0x41, 0x52, 0x0c, 0x65, 0x6e, 0x71, 0x75, 0x65, 0x75, 0x65, 0x64, 0x56, 0x61, 0x61, 0x73, + 0x1a, 0xa8, 0x01, 0x0a, 0x05, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x68, + 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x07, 0x63, 0x68, + 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x40, 0x0a, 0x1c, 0x72, 0x65, 0x6d, 0x61, 0x69, 0x6e, 0x69, + 0x6e, 0x67, 0x5f, 0x61, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x5f, 0x6e, 0x6f, 0x74, + 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x1a, 0x72, 0x65, 0x6d, + 0x61, 0x69, 0x6e, 0x69, 0x6e, 0x67, 0x41, 0x76, 0x61, 0x69, 0x6c, 0x61, 0x62, 0x6c, 0x65, 0x4e, + 0x6f, 0x74, 0x69, 0x6f, 0x6e, 0x61, 0x6c, 0x12, 0x42, 0x0a, 0x08, 0x65, 0x6d, 0x69, 0x74, 0x74, + 0x65, 0x72, 0x73, 0x18, 0x03, 0x20, 0x03, 0x28, 0x0b, 0x32, 0x26, 0x2e, 0x67, 0x6f, 0x73, 0x73, + 0x69, 0x70, 0x2e, 0x76, 0x31, 0x2e, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x47, 0x6f, 0x76, 0x65, 0x72, + 0x6e, 0x6f, 0x72, 0x53, 0x74, 0x61, 0x74, 0x75, 0x73, 0x2e, 0x45, 0x6d, 0x69, 0x74, 0x74, 0x65, + 0x72, 0x52, 0x08, 0x65, 0x6d, 0x69, 0x74, 0x74, 0x65, 0x72, 0x73, 0x22, 0x57, 0x0a, 0x12, 0x53, + 0x69, 0x67, 0x6e, 0x65, 0x64, 0x51, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x12, 0x23, 0x0a, 0x0d, 0x71, 0x75, 0x65, 0x72, 0x79, 0x5f, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x0c, 0x71, 0x75, 0x65, 0x72, 0x79, 0x52, + 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, + 0x75, 0x72, 0x65, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, + 0x74, 0x75, 0x72, 0x65, 0x22, 0x5a, 0x0a, 0x13, 0x53, 0x69, 0x67, 0x6e, 0x65, 0x64, 0x51, 0x75, + 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x25, 0x0a, 0x0e, 0x71, + 0x75, 0x65, 0x72, 0x79, 0x5f, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x01, 0x20, + 0x01, 0x28, 0x0c, 0x52, 0x0d, 0x71, 0x75, 0x65, 0x72, 0x79, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, + 0x73, 0x65, 0x12, 0x1c, 0x0a, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, 0x18, + 0x02, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x09, 0x73, 0x69, 0x67, 0x6e, 0x61, 0x74, 0x75, 0x72, 0x65, + 0x42, 0x41, 0x5a, 0x3f, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x63, + 0x65, 0x72, 0x74, 0x75, 0x73, 0x6f, 0x6e, 0x65, 0x2f, 0x77, 0x6f, 0x72, 0x6d, 0x68, 0x6f, 0x6c, + 0x65, 0x2f, 0x6e, 0x6f, 0x64, 0x65, 0x2f, 0x70, 0x6b, 0x67, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, + 0x2f, 0x67, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x2f, 0x76, 0x31, 0x3b, 0x67, 0x6f, 0x73, 0x73, 0x69, + 0x70, 0x76, 0x31, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -1766,7 +1930,7 @@ func file_gossip_v1_gossip_proto_rawDescGZIP() []byte { return file_gossip_v1_gossip_proto_rawDescData } -var file_gossip_v1_gossip_proto_msgTypes = make([]protoimpl.MessageInfo, 19) +var file_gossip_v1_gossip_proto_msgTypes = make([]protoimpl.MessageInfo, 21) var file_gossip_v1_gossip_proto_goTypes = []interface{}{ (*GossipMessage)(nil), // 0: gossip.v1.GossipMessage (*SignedHeartbeat)(nil), // 1: gossip.v1.SignedHeartbeat @@ -1781,12 +1945,14 @@ var file_gossip_v1_gossip_proto_goTypes = []interface{}{ (*ChainGovernorConfig)(nil), // 10: gossip.v1.ChainGovernorConfig (*SignedChainGovernorStatus)(nil), // 11: gossip.v1.SignedChainGovernorStatus (*ChainGovernorStatus)(nil), // 12: gossip.v1.ChainGovernorStatus - (*Heartbeat_Network)(nil), // 13: gossip.v1.Heartbeat.Network - (*ChainGovernorConfig_Chain)(nil), // 14: gossip.v1.ChainGovernorConfig.Chain - (*ChainGovernorConfig_Token)(nil), // 15: gossip.v1.ChainGovernorConfig.Token - (*ChainGovernorStatus_EnqueuedVAA)(nil), // 16: gossip.v1.ChainGovernorStatus.EnqueuedVAA - (*ChainGovernorStatus_Emitter)(nil), // 17: gossip.v1.ChainGovernorStatus.Emitter - (*ChainGovernorStatus_Chain)(nil), // 18: gossip.v1.ChainGovernorStatus.Chain + (*SignedQueryRequest)(nil), // 13: gossip.v1.SignedQueryRequest + (*SignedQueryResponse)(nil), // 14: gossip.v1.SignedQueryResponse + (*Heartbeat_Network)(nil), // 15: gossip.v1.Heartbeat.Network + (*ChainGovernorConfig_Chain)(nil), // 16: gossip.v1.ChainGovernorConfig.Chain + (*ChainGovernorConfig_Token)(nil), // 17: gossip.v1.ChainGovernorConfig.Token + (*ChainGovernorStatus_EnqueuedVAA)(nil), // 18: gossip.v1.ChainGovernorStatus.EnqueuedVAA + (*ChainGovernorStatus_Emitter)(nil), // 19: gossip.v1.ChainGovernorStatus.Emitter + (*ChainGovernorStatus_Chain)(nil), // 20: gossip.v1.ChainGovernorStatus.Chain } var file_gossip_v1_gossip_proto_depIdxs = []int32{ 3, // 0: gossip.v1.GossipMessage.signed_observation:type_name -> gossip.v1.SignedObservation @@ -1797,17 +1963,19 @@ var file_gossip_v1_gossip_proto_depIdxs = []int32{ 8, // 5: gossip.v1.GossipMessage.signed_batch_vaa_with_quorum:type_name -> gossip.v1.SignedBatchVAAWithQuorum 9, // 6: gossip.v1.GossipMessage.signed_chain_governor_config:type_name -> gossip.v1.SignedChainGovernorConfig 11, // 7: gossip.v1.GossipMessage.signed_chain_governor_status:type_name -> gossip.v1.SignedChainGovernorStatus - 13, // 8: gossip.v1.Heartbeat.networks:type_name -> gossip.v1.Heartbeat.Network - 14, // 9: gossip.v1.ChainGovernorConfig.chains:type_name -> gossip.v1.ChainGovernorConfig.Chain - 15, // 10: gossip.v1.ChainGovernorConfig.tokens:type_name -> gossip.v1.ChainGovernorConfig.Token - 18, // 11: gossip.v1.ChainGovernorStatus.chains:type_name -> gossip.v1.ChainGovernorStatus.Chain - 16, // 12: gossip.v1.ChainGovernorStatus.Emitter.enqueued_vaas:type_name -> gossip.v1.ChainGovernorStatus.EnqueuedVAA - 17, // 13: gossip.v1.ChainGovernorStatus.Chain.emitters:type_name -> gossip.v1.ChainGovernorStatus.Emitter - 14, // [14:14] is the sub-list for method output_type - 14, // [14:14] is the sub-list for method input_type - 14, // [14:14] is the sub-list for extension type_name - 14, // [14:14] is the sub-list for extension extendee - 0, // [0:14] is the sub-list for field type_name + 13, // 8: gossip.v1.GossipMessage.signed_query_request:type_name -> gossip.v1.SignedQueryRequest + 14, // 9: gossip.v1.GossipMessage.signed_query_response:type_name -> gossip.v1.SignedQueryResponse + 15, // 10: gossip.v1.Heartbeat.networks:type_name -> gossip.v1.Heartbeat.Network + 16, // 11: gossip.v1.ChainGovernorConfig.chains:type_name -> gossip.v1.ChainGovernorConfig.Chain + 17, // 12: gossip.v1.ChainGovernorConfig.tokens:type_name -> gossip.v1.ChainGovernorConfig.Token + 20, // 13: gossip.v1.ChainGovernorStatus.chains:type_name -> gossip.v1.ChainGovernorStatus.Chain + 18, // 14: gossip.v1.ChainGovernorStatus.Emitter.enqueued_vaas:type_name -> gossip.v1.ChainGovernorStatus.EnqueuedVAA + 19, // 15: gossip.v1.ChainGovernorStatus.Chain.emitters:type_name -> gossip.v1.ChainGovernorStatus.Emitter + 16, // [16:16] is the sub-list for method output_type + 16, // [16:16] is the sub-list for method input_type + 16, // [16:16] is the sub-list for extension type_name + 16, // [16:16] is the sub-list for extension extendee + 0, // [0:16] is the sub-list for field type_name } func init() { file_gossip_v1_gossip_proto_init() } @@ -1973,7 +2141,7 @@ func file_gossip_v1_gossip_proto_init() { } } file_gossip_v1_gossip_proto_msgTypes[13].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*Heartbeat_Network); i { + switch v := v.(*SignedQueryRequest); i { case 0: return &v.state case 1: @@ -1985,7 +2153,7 @@ func file_gossip_v1_gossip_proto_init() { } } file_gossip_v1_gossip_proto_msgTypes[14].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ChainGovernorConfig_Chain); i { + switch v := v.(*SignedQueryResponse); i { case 0: return &v.state case 1: @@ -1997,7 +2165,7 @@ func file_gossip_v1_gossip_proto_init() { } } file_gossip_v1_gossip_proto_msgTypes[15].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ChainGovernorConfig_Token); i { + switch v := v.(*Heartbeat_Network); i { case 0: return &v.state case 1: @@ -2009,7 +2177,7 @@ func file_gossip_v1_gossip_proto_init() { } } file_gossip_v1_gossip_proto_msgTypes[16].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ChainGovernorStatus_EnqueuedVAA); i { + switch v := v.(*ChainGovernorConfig_Chain); i { case 0: return &v.state case 1: @@ -2021,7 +2189,7 @@ func file_gossip_v1_gossip_proto_init() { } } file_gossip_v1_gossip_proto_msgTypes[17].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*ChainGovernorStatus_Emitter); i { + switch v := v.(*ChainGovernorConfig_Token); i { case 0: return &v.state case 1: @@ -2033,6 +2201,30 @@ func file_gossip_v1_gossip_proto_init() { } } file_gossip_v1_gossip_proto_msgTypes[18].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ChainGovernorStatus_EnqueuedVAA); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_gossip_v1_gossip_proto_msgTypes[19].Exporter = func(v interface{}, i int) interface{} { + switch v := v.(*ChainGovernorStatus_Emitter); i { + case 0: + return &v.state + case 1: + return &v.sizeCache + case 2: + return &v.unknownFields + default: + return nil + } + } + file_gossip_v1_gossip_proto_msgTypes[20].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*ChainGovernorStatus_Chain); i { case 0: return &v.state @@ -2054,6 +2246,8 @@ func file_gossip_v1_gossip_proto_init() { (*GossipMessage_SignedBatchVaaWithQuorum)(nil), (*GossipMessage_SignedChainGovernorConfig)(nil), (*GossipMessage_SignedChainGovernorStatus)(nil), + (*GossipMessage_SignedQueryRequest)(nil), + (*GossipMessage_SignedQueryResponse)(nil), } type x struct{} out := protoimpl.TypeBuilder{ @@ -2061,7 +2255,7 @@ func file_gossip_v1_gossip_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_gossip_v1_gossip_proto_rawDesc, NumEnums: 0, - NumMessages: 19, + NumMessages: 21, NumExtensions: 0, NumServices: 0, }, diff --git a/node/pkg/query/dev.guardian.key b/node/pkg/query/dev.guardian.key new file mode 100644 index 0000000000..0be9e035bd --- /dev/null +++ b/node/pkg/query/dev.guardian.key @@ -0,0 +1,8 @@ +-----BEGIN WORMHOLE GUARDIAN PRIVATE KEY----- +PublicKey: 0xbeFA429d57cD18b7F8A4d91A2da9AB4AF05d0FBe +Description: auto-generated deterministic devnet key + +CiDPsSMDoZzeWAu03XcWObDSa8aDU2RVcajP9RarLuEToBAB +=VN/A +-----END WORMHOLE GUARDIAN PRIVATE KEY----- + diff --git a/node/pkg/query/helpers_test.go b/node/pkg/query/helpers_test.go new file mode 100644 index 0000000000..25d7eb47a4 --- /dev/null +++ b/node/pkg/query/helpers_test.go @@ -0,0 +1,6 @@ +package query + +func makeChannelPair[T any](cap int) (<-chan T, chan<- T) { + out := make(chan T, cap) + return out, out +} diff --git a/node/pkg/query/msg_test.go b/node/pkg/query/msg_test.go new file mode 100644 index 0000000000..ce461f40ae --- /dev/null +++ b/node/pkg/query/msg_test.go @@ -0,0 +1,544 @@ +package query + +import ( + "encoding/hex" + "fmt" + "strings" + "testing" + "time" + + gossipv1 "github.com/certusone/wormhole/node/pkg/proto/gossip/v1" + "github.com/wormhole-foundation/wormhole/sdk/vaa" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "github.com/ethereum/go-ethereum/accounts/abi" + ethCommon "github.com/ethereum/go-ethereum/common" +) + +func createQueryRequestForTesting(chainId vaa.ChainID) *QueryRequest { + // Create a query request. + wethAbi, err := abi.JSON(strings.NewReader("[{\"constant\":true,\"inputs\":[],\"name\":\"name\",\"outputs\":[{\"name\":\"\",\"type\":\"string\"}],\"payable\":false,\"stateMutability\":\"view\",\"type\":\"function\"},{\"constant\":true,\"inputs\":[],\"name\":\"totalSupply\",\"outputs\":[{\"name\":\"\",\"type\":\"uint256\"}],\"payable\":false,\"stateMutability\":\"view\",\"type\":\"function\"}]")) + if err != nil { + panic(err) + } + + data1, err := wethAbi.Pack("name") + if err != nil { + panic(err) + } + data2, err := wethAbi.Pack("totalSupply") + if err != nil { + panic(err) + } + + to, _ := hex.DecodeString("0d500b1d8e8ef31e21c99d1db9a6444d3adf1270") + block := "0x28d9630" + callData := []*EthCallData{ + { + To: to, + Data: data1, + }, + { + To: to, + Data: data2, + }, + } + callRequest := &EthCallQueryRequest{ + BlockId: block, + CallData: callData, + } + + perChainQuery := &PerChainQueryRequest{ + ChainId: chainId, + Query: callRequest, + } + + queryRequest := &QueryRequest{ + Nonce: 1, + PerChainQueries: []*PerChainQueryRequest{perChainQuery}, + } + + return queryRequest +} + +// A timestamp has nanos, but we only marshal down to micros, so trim our time to micros for testing purposes. +func timeForTest(t time.Time) time.Time { + return time.UnixMicro(t.UnixMicro()) +} + +func TestQueryRequestMarshalUnmarshal(t *testing.T) { + queryRequest := createQueryRequestForTesting(vaa.ChainIDPolygon) + queryRequestBytes, err := queryRequest.Marshal() + require.NoError(t, err) + + var queryRequest2 QueryRequest + err = queryRequest2.Unmarshal(queryRequestBytes) + require.NoError(t, err) + + assert.True(t, queryRequest.Equal(&queryRequest2)) +} + +func TestMarshalOfQueryRequestWithNoPerChainQueriesShouldFail(t *testing.T) { + queryRequest := &QueryRequest{ + Nonce: 1, + PerChainQueries: []*PerChainQueryRequest{ + { + ChainId: vaa.ChainIDPolygon, + // Leave Query nil. + }, + }, + } + _, err := queryRequest.Marshal() + require.Error(t, err) +} + +func TestMarshalOfQueryRequestWithTooManyPerChainQueriesShouldFail(t *testing.T) { + perChainQueries := []*PerChainQueryRequest{} + for count := 0; count < 300; count++ { + callData := []*EthCallData{{ + + To: []byte(fmt.Sprintf("%-20s", fmt.Sprintf("To for %d", count))), + Data: []byte(fmt.Sprintf("CallData for %d", count)), + }, + } + + perChainQueries = append(perChainQueries, &PerChainQueryRequest{ + ChainId: vaa.ChainIDPolygon, + Query: &EthCallQueryRequest{ + BlockId: "0x28d9630", + CallData: callData, + }, + }) + } + + queryRequest := &QueryRequest{ + Nonce: 1, + PerChainQueries: perChainQueries, + } + _, err := queryRequest.Marshal() + require.Error(t, err) +} + +func TestMarshalOfQueryRequestForInvalidChainIdShouldFail(t *testing.T) { + queryRequest := createQueryRequestForTesting(vaa.ChainIDUnset) + _, err := queryRequest.Marshal() + require.Error(t, err) +} + +func TestMarshalOfQueryRequestWithInvalidBlockIdShouldFail(t *testing.T) { + callData := []*EthCallData{{ + To: []byte(fmt.Sprintf("%-20s", fmt.Sprintf("To for %d", 0))), + Data: []byte(fmt.Sprintf("CallData for %d", 0)), + }} + + perChainQuery := &PerChainQueryRequest{ + ChainId: vaa.ChainIDPolygon, + Query: &EthCallQueryRequest{ + BlockId: "latest", + CallData: callData, + }, + } + + queryRequest := &QueryRequest{ + Nonce: 1, + PerChainQueries: []*PerChainQueryRequest{perChainQuery}, + } + _, err := queryRequest.Marshal() + require.Error(t, err) +} + +func TestMarshalOfQueryRequestWithNoCallDataEntriesShouldFail(t *testing.T) { + callData := []*EthCallData{} + perChainQuery := &PerChainQueryRequest{ + ChainId: vaa.ChainIDPolygon, + Query: &EthCallQueryRequest{ + BlockId: "0x28d9630", + CallData: callData, + }, + } + + queryRequest := &QueryRequest{ + Nonce: 1, + PerChainQueries: []*PerChainQueryRequest{perChainQuery}, + } + _, err := queryRequest.Marshal() + require.Error(t, err) +} + +func TestMarshalOfQueryRequestWithNilCallDataEntriesShouldFail(t *testing.T) { + perChainQuery := &PerChainQueryRequest{ + ChainId: vaa.ChainIDPolygon, + Query: &EthCallQueryRequest{ + BlockId: "0x28d9630", + CallData: nil, + }, + } + + queryRequest := &QueryRequest{ + Nonce: 1, + PerChainQueries: []*PerChainQueryRequest{perChainQuery}, + } + _, err := queryRequest.Marshal() + require.Error(t, err) +} + +func TestMarshalOfQueryRequestWithTooManyCallDataEntriesShouldFail(t *testing.T) { + callData := []*EthCallData{} + for count := 0; count < 300; count++ { + callData = append(callData, &EthCallData{ + To: []byte(fmt.Sprintf("%-20s", fmt.Sprintf("To for %d", count))), + Data: []byte(fmt.Sprintf("CallData for %d", count)), + }) + } + + perChainQuery := &PerChainQueryRequest{ + ChainId: vaa.ChainIDPolygon, + Query: &EthCallQueryRequest{ + BlockId: "0x28d9630", + CallData: callData, + }, + } + + queryRequest := &QueryRequest{ + Nonce: 1, + PerChainQueries: []*PerChainQueryRequest{perChainQuery}, + } + _, err := queryRequest.Marshal() + require.Error(t, err) +} + +func TestMarshalOfEthCallQueryWithNilToShouldFail(t *testing.T) { + perChainQuery := &PerChainQueryRequest{ + ChainId: vaa.ChainIDPolygon, + Query: &EthCallQueryRequest{ + BlockId: "0x28d9630", + CallData: []*EthCallData{ + { + To: nil, + Data: []byte("This can't be zero length"), + }, + }, + }, + } + + queryRequest := &QueryRequest{ + Nonce: 1, + PerChainQueries: []*PerChainQueryRequest{perChainQuery}, + } + _, err := queryRequest.Marshal() + require.Error(t, err) +} + +func TestMarshalOfEthCallQueryWithEmptyToShouldFail(t *testing.T) { + perChainQuery := &PerChainQueryRequest{ + ChainId: vaa.ChainIDPolygon, + Query: &EthCallQueryRequest{ + BlockId: "0x28d9630", + CallData: []*EthCallData{ + { + To: []byte{}, + Data: []byte("This can't be zero length"), + }, + }, + }, + } + + queryRequest := &QueryRequest{ + Nonce: 1, + PerChainQueries: []*PerChainQueryRequest{perChainQuery}, + } + _, err := queryRequest.Marshal() + require.Error(t, err) +} + +func TestMarshalOfEthCallQueryWithWrongLengthToShouldFail(t *testing.T) { + perChainQuery := &PerChainQueryRequest{ + ChainId: vaa.ChainIDPolygon, + Query: &EthCallQueryRequest{ + BlockId: "0x28d9630", + CallData: []*EthCallData{ + { + To: []byte("TooShort"), + Data: []byte("This can't be zero length"), + }, + }, + }, + } + + queryRequest := &QueryRequest{ + Nonce: 1, + PerChainQueries: []*PerChainQueryRequest{perChainQuery}, + } + _, err := queryRequest.Marshal() + require.Error(t, err) +} + +func TestMarshalOfEthCallQueryWithNilDataShouldFail(t *testing.T) { + perChainQuery := &PerChainQueryRequest{ + ChainId: vaa.ChainIDPolygon, + Query: &EthCallQueryRequest{ + BlockId: "0x28d9630", + CallData: []*EthCallData{ + { + To: []byte(fmt.Sprintf("%-20s", fmt.Sprintf("To for %d", 0))), + Data: nil, + }, + }, + }, + } + + queryRequest := &QueryRequest{ + Nonce: 1, + PerChainQueries: []*PerChainQueryRequest{perChainQuery}, + } + _, err := queryRequest.Marshal() + require.Error(t, err) +} + +func TestMarshalOfEthCallQueryWithEmptyDataShouldFail(t *testing.T) { + perChainQuery := &PerChainQueryRequest{ + ChainId: vaa.ChainIDPolygon, + Query: &EthCallQueryRequest{ + BlockId: "0x28d9630", + CallData: []*EthCallData{ + { + To: []byte(fmt.Sprintf("%-20s", fmt.Sprintf("To for %d", 0))), + Data: []byte{}, + }, + }, + }, + } + + queryRequest := &QueryRequest{ + Nonce: 1, + PerChainQueries: []*PerChainQueryRequest{perChainQuery}, + } + _, err := queryRequest.Marshal() + require.Error(t, err) +} + +func TestMarshalOfEthCallQueryWithWrongToLengthShouldFail(t *testing.T) { + perChainQuery := &PerChainQueryRequest{ + ChainId: vaa.ChainIDPolygon, + Query: &EthCallQueryRequest{ + BlockId: "0x28d9630", + CallData: []*EthCallData{ + { + To: []byte("This is too short!"), + Data: []byte("This can't be zero length"), + }, + }, + }, + } + + queryRequest := &QueryRequest{ + Nonce: 1, + PerChainQueries: []*PerChainQueryRequest{perChainQuery}, + } + _, err := queryRequest.Marshal() + require.Error(t, err) +} + +func TestPostSignedQueryRequestShouldFailIfNoOneIsListening(t *testing.T) { + queryRequest := createQueryRequestForTesting(vaa.ChainIDPolygon) + queryRequestBytes, err := queryRequest.Marshal() + require.NoError(t, err) + + sig := [65]byte{} + signedQueryRequest := &gossipv1.SignedQueryRequest{ + QueryRequest: queryRequestBytes, + Signature: sig[:], + } + + var signedQueryReqSendC chan<- *gossipv1.SignedQueryRequest + assert.Error(t, PostSignedQueryRequest(signedQueryReqSendC, signedQueryRequest)) +} + +func createQueryResponseFromRequest(t *testing.T, queryRequest *QueryRequest) *QueryResponsePublication { + queryRequestBytes, err := queryRequest.Marshal() + require.NoError(t, err) + + sig := [65]byte{} + signedQueryRequest := &gossipv1.SignedQueryRequest{ + QueryRequest: queryRequestBytes, + Signature: sig[:], + } + + perChainResponses := []*PerChainQueryResponse{} + for idx, pcr := range queryRequest.PerChainQueries { + switch req := pcr.Query.(type) { + case *EthCallQueryRequest: + results := [][]byte{} + for idx := range req.CallData { + result := []byte([]byte(fmt.Sprintf("Result %d", idx))) + results = append(results, result[:]) + } + perChainResponses = append(perChainResponses, &PerChainQueryResponse{ + ChainId: pcr.ChainId, + Response: &EthCallQueryResponse{ + BlockNumber: uint64(1000 + idx), + Hash: ethCommon.HexToHash("0x9999bac44d09a7f69ee7941819b0a19c59ccb1969640cc513be09ef95ed2d8e2"), + Time: timeForTest(time.Now()), + Results: results, + }, + }) + default: + panic("invalid query type!") + } + + } + + return &QueryResponsePublication{ + Request: signedQueryRequest, + PerChainResponses: perChainResponses, + } +} + +func TestQueryResponseMarshalUnmarshal(t *testing.T) { + queryRequest := createQueryRequestForTesting(vaa.ChainIDPolygon) + respPub := createQueryResponseFromRequest(t, queryRequest) + + respPubBytes, err := respPub.Marshal() + require.NoError(t, err) + + var respPub2 QueryResponsePublication + err = respPub2.Unmarshal(respPubBytes) + require.NoError(t, err) + require.NotNil(t, respPub2) + + assert.True(t, respPub.Equal(&respPub2)) +} + +func TestMarshalUnmarshalQueryResponseWithNoPerChainResponsesShouldFail(t *testing.T) { + queryRequest := createQueryRequestForTesting(vaa.ChainIDPolygon) + queryRequestBytes, err := queryRequest.Marshal() + require.NoError(t, err) + + sig := [65]byte{} + signedQueryRequest := &gossipv1.SignedQueryRequest{ + QueryRequest: queryRequestBytes, + Signature: sig[:], + } + + respPub := &QueryResponsePublication{ + Request: signedQueryRequest, + PerChainResponses: []*PerChainQueryResponse{}, + } + + _, err = respPub.Marshal() + require.Error(t, err) +} + +func TestMarshalUnmarshalQueryResponseWithNilPerChainResponsesShouldFail(t *testing.T) { + queryRequest := createQueryRequestForTesting(vaa.ChainIDPolygon) + queryRequestBytes, err := queryRequest.Marshal() + require.NoError(t, err) + + sig := [65]byte{} + signedQueryRequest := &gossipv1.SignedQueryRequest{ + QueryRequest: queryRequestBytes, + Signature: sig[:], + } + + respPub := &QueryResponsePublication{ + Request: signedQueryRequest, + PerChainResponses: nil, + } + + _, err = respPub.Marshal() + require.Error(t, err) +} + +func TestMarshalUnmarshalQueryResponseWithTooManyPerChainResponsesShouldFail(t *testing.T) { + queryRequest := createQueryRequestForTesting(vaa.ChainIDPolygon) + respPub := createQueryResponseFromRequest(t, queryRequest) + + for count := 0; count < 300; count++ { + respPub.PerChainResponses = append(respPub.PerChainResponses, respPub.PerChainResponses[0]) + } + + _, err := respPub.Marshal() + require.Error(t, err) +} + +func TestMarshalUnmarshalQueryResponseWithWrongNumberOfPerChainResponsesShouldFail(t *testing.T) { + queryRequest := createQueryRequestForTesting(vaa.ChainIDPolygon) + respPub := createQueryResponseFromRequest(t, queryRequest) + + respPub.PerChainResponses = append(respPub.PerChainResponses, respPub.PerChainResponses[0]) + + _, err := respPub.Marshal() + require.Error(t, err) +} + +func TestMarshalUnmarshalQueryResponseWithInvalidChainIDShouldFail(t *testing.T) { + queryRequest := createQueryRequestForTesting(vaa.ChainIDPolygon) + respPub := createQueryResponseFromRequest(t, queryRequest) + + respPub.PerChainResponses[0].ChainId = vaa.ChainIDUnset + + _, err := respPub.Marshal() + require.Error(t, err) +} + +func TestMarshalUnmarshalQueryResponseWithNilResponseShouldFail(t *testing.T) { + queryRequest := createQueryRequestForTesting(vaa.ChainIDPolygon) + respPub := createQueryResponseFromRequest(t, queryRequest) + + respPub.PerChainResponses[0].Response = nil + + _, err := respPub.Marshal() + require.Error(t, err) +} + +func TestMarshalUnmarshalQueryResponseWithNoResultsShouldFail(t *testing.T) { + queryRequest := createQueryRequestForTesting(vaa.ChainIDPolygon) + respPub := createQueryResponseFromRequest(t, queryRequest) + + switch resp := respPub.PerChainResponses[0].Response.(type) { + case *EthCallQueryResponse: + resp.Results = [][]byte{} + default: + panic("invalid query type!") + } + + _, err := respPub.Marshal() + require.Error(t, err) +} + +func TestMarshalUnmarshalQueryResponseWithNilResultsShouldFail(t *testing.T) { + queryRequest := createQueryRequestForTesting(vaa.ChainIDPolygon) + respPub := createQueryResponseFromRequest(t, queryRequest) + + switch resp := respPub.PerChainResponses[0].Response.(type) { + case *EthCallQueryResponse: + resp.Results = nil + default: + panic("invalid query type!") + } + + _, err := respPub.Marshal() + require.Error(t, err) +} + +func TestMarshalUnmarshalQueryResponseWithTooManyResultsShouldFail(t *testing.T) { + queryRequest := createQueryRequestForTesting(vaa.ChainIDPolygon) + respPub := createQueryResponseFromRequest(t, queryRequest) + + results := [][]byte{} + for count := 0; count < 300; count++ { + results = append(results, []byte{}) + } + + switch resp := respPub.PerChainResponses[0].Response.(type) { + case *EthCallQueryResponse: + resp.Results = results + default: + panic("invalid query type!") + } + + _, err := respPub.Marshal() + require.Error(t, err) +} diff --git a/node/pkg/query/query.go b/node/pkg/query/query.go new file mode 100644 index 0000000000..94d894510f --- /dev/null +++ b/node/pkg/query/query.go @@ -0,0 +1,412 @@ +package query + +import ( + "context" + "encoding/hex" + "fmt" + "strings" + "time" + + "github.com/certusone/wormhole/node/pkg/common" + gossipv1 "github.com/certusone/wormhole/node/pkg/proto/gossip/v1" + "github.com/certusone/wormhole/node/pkg/supervisor" + "github.com/wormhole-foundation/wormhole/sdk/vaa" + + ethCommon "github.com/ethereum/go-ethereum/common" + ethCrypto "github.com/ethereum/go-ethereum/crypto" + + "go.uber.org/zap" +) + +const ( + // RequestTimeout indicates how long before a request is considered to have timed out. + RequestTimeout = 1 * time.Minute + + // RetryInterval specifies how long we will wait between retry intervals. This is the interval of our ticker. + RetryInterval = 10 * time.Second + + // SignedQueryRequestChannelSize is the buffer size of the incoming query request channel. + SignedQueryRequestChannelSize = 50 + + // QueryRequestBufferSize is the buffer size of the per-network query request channel. + QueryRequestBufferSize = 25 +) + +func NewQueryHandler( + logger *zap.Logger, + env common.Environment, + allowedRequestorsStr string, + signedQueryReqC <-chan *gossipv1.SignedQueryRequest, + chainQueryReqC map[vaa.ChainID]chan *PerChainQueryInternal, + queryResponseReadC <-chan *PerChainQueryResponseInternal, + queryResponseWriteC chan<- *QueryResponsePublication, +) *QueryHandler { + return &QueryHandler{ + logger: logger.With(zap.String("component", "ccq")), + env: env, + allowedRequestorsStr: allowedRequestorsStr, + signedQueryReqC: signedQueryReqC, + chainQueryReqC: chainQueryReqC, + queryResponseReadC: queryResponseReadC, + queryResponseWriteC: queryResponseWriteC, + } +} + +type ( + // QueryHandler defines the cross chain query handler. + QueryHandler struct { + logger *zap.Logger + env common.Environment + allowedRequestorsStr string + signedQueryReqC <-chan *gossipv1.SignedQueryRequest + chainQueryReqC map[vaa.ChainID]chan *PerChainQueryInternal + queryResponseReadC <-chan *PerChainQueryResponseInternal + queryResponseWriteC chan<- *QueryResponsePublication + allowedRequestors map[ethCommon.Address]struct{} + } + + // pendingQuery is the cache entry for a given query. + pendingQuery struct { + signedRequest *gossipv1.SignedQueryRequest + request *QueryRequest + requestID string + receiveTime time.Time + queries []*perChainQuery + responses []*PerChainQueryResponseInternal + + // respPub is only populated when we need to retry sending the response to p2p. + respPub *QueryResponsePublication + } + + // perChainQuery is the data associated with a single per chain query in a query request. + perChainQuery struct { + req *PerChainQueryInternal + channel chan *PerChainQueryInternal + lastUpdateTime time.Time + } +) + +// Start initializes the query handler and starts the runnable. +func (qh *QueryHandler) Start(ctx context.Context) error { + qh.logger.Debug("entering Start", zap.String("enforceFlag", qh.allowedRequestorsStr)) + + var err error + qh.allowedRequestors, err = parseAllowedRequesters(qh.allowedRequestorsStr) + if err != nil { + return fmt.Errorf("failed to parse allowed requesters: %w", err) + } + + if err := supervisor.Run(ctx, "query_handler", common.WrapWithScissors(qh.handleQueryRequests, "query_handler")); err != nil { + return fmt.Errorf("failed to start query handler routine: %w", err) + } + + return nil +} + +// handleQueryRequests multiplexes observation requests to the appropriate chain +func (qh *QueryHandler) handleQueryRequests(ctx context.Context) error { + return handleQueryRequestsImpl(ctx, qh.logger, qh.signedQueryReqC, qh.chainQueryReqC, qh.allowedRequestors, qh.queryResponseReadC, qh.queryResponseWriteC, qh.env, RequestTimeout, RetryInterval) +} + +// handleQueryRequestsImpl allows instantiating the handler in the test environment with shorter timeout and retry parameters. +func handleQueryRequestsImpl( + ctx context.Context, + logger *zap.Logger, + signedQueryReqC <-chan *gossipv1.SignedQueryRequest, + chainQueryReqC map[vaa.ChainID]chan *PerChainQueryInternal, + allowedRequestors map[ethCommon.Address]struct{}, + queryResponseReadC <-chan *PerChainQueryResponseInternal, + queryResponseWriteC chan<- *QueryResponsePublication, + env common.Environment, + requestTimeoutImpl time.Duration, + retryIntervalImpl time.Duration, +) error { + qLogger := logger.With(zap.String("component", "ccqhandler")) + qLogger.Info("cross chain queries are enabled", zap.Any("allowedRequestors", allowedRequestors), zap.String("env", string(env))) + + pendingQueries := make(map[string]*pendingQuery) // Key is requestID. + + // TODO: This should only include watchers that are actually running. Also need to test all these chains. + supportedChains := map[vaa.ChainID]struct{}{ + vaa.ChainIDEthereum: {}, + vaa.ChainIDBSC: {}, + vaa.ChainIDPolygon: {}, + vaa.ChainIDAvalanche: {}, + vaa.ChainIDOasis: {}, + vaa.ChainIDAurora: {}, + vaa.ChainIDFantom: {}, + vaa.ChainIDKarura: {}, + vaa.ChainIDAcala: {}, + vaa.ChainIDKlaytn: {}, + vaa.ChainIDCelo: {}, + vaa.ChainIDMoonbeam: {}, + vaa.ChainIDNeon: {}, + vaa.ChainIDArbitrum: {}, + vaa.ChainIDOptimism: {}, + vaa.ChainIDBase: {}, + vaa.ChainIDSepolia: {}, + } + + ticker := time.NewTicker(retryIntervalImpl) + defer ticker.Stop() + + for { + select { + case <-ctx.Done(): + return nil + + case signedRequest := <-signedQueryReqC: // Inbound query request. + // requestor validation happens here + // request type validation is currently handled by the watcher + // in the future, it may be worthwhile to catch certain types of + // invalid requests here for tracking purposes + // e.g. + // - length check on "signature" 65 bytes + // - length check on "to" address 20 bytes + // - valid "block" strings + + requestID := hex.EncodeToString(signedRequest.Signature) + digest := QueryRequestDigest(env, signedRequest.QueryRequest) + + qLogger.Info("received a query request", zap.String("requestID", requestID)) + + signerBytes, err := ethCrypto.Ecrecover(digest.Bytes(), signedRequest.Signature) + if err != nil { + qLogger.Error("failed to recover public key", zap.String("requestID", requestID)) + continue + } + + signerAddress := ethCommon.BytesToAddress(ethCrypto.Keccak256(signerBytes[1:])[12:]) + + if _, exists := allowedRequestors[signerAddress]; !exists { + qLogger.Error("invalid requestor", zap.String("requestor", signerAddress.Hex()), zap.String("requestID", requestID)) + continue + } + + // Make sure this is not a duplicate request. TODO: Should we do something smarter here than just dropping the duplicate? + if oldReq, exists := pendingQueries[requestID]; exists { + qLogger.Warn("dropping duplicate query request", zap.String("requestID", requestID), zap.Stringer("origRecvTime", oldReq.receiveTime)) + continue + } + + var queryRequest QueryRequest + err = queryRequest.Unmarshal(signedRequest.QueryRequest) + if err != nil { + qLogger.Error("failed to unmarshal query request", zap.String("requestor", signerAddress.Hex()), zap.String("requestID", requestID), zap.Error(err)) + continue + } + + if err := queryRequest.Validate(); err != nil { + qLogger.Error("received invalid message", zap.String("requestor", signerAddress.Hex()), zap.String("requestID", requestID), zap.Error(err)) + continue + } + + // Build the set of per chain queries and placeholders for the per chain responses. + errorFound := false + queries := []*perChainQuery{} + responses := make([]*PerChainQueryResponseInternal, len(queryRequest.PerChainQueries)) + receiveTime := time.Now() + + for requestIdx, pcq := range queryRequest.PerChainQueries { + chainID := vaa.ChainID(pcq.ChainId) + if _, exists := supportedChains[chainID]; !exists { + qLogger.Error("chain does not support cross chain queries", zap.String("requestID", requestID), zap.Stringer("chainID", chainID)) + errorFound = true + break + } + + channel, channelExists := chainQueryReqC[chainID] + if !channelExists { + qLogger.Error("unknown chain ID for query request, dropping it", zap.String("requestID", requestID), zap.Stringer("chain_id", chainID)) + errorFound = true + break + } + + queries = append(queries, &perChainQuery{ + req: &PerChainQueryInternal{ + RequestID: requestID, + RequestIdx: requestIdx, + Request: pcq, + }, + channel: channel, + }) + } + + if errorFound { + continue + } + + // Create the pending query and add it to the cache. + pq := &pendingQuery{ + signedRequest: signedRequest, + request: &queryRequest, + requestID: requestID, + receiveTime: receiveTime, + queries: queries, + responses: responses, + } + pendingQueries[requestID] = pq + + // Forward the requests to the watchers. + for _, pcq := range pq.queries { + pcq.ccqForwardToWatcher(qLogger, pq.receiveTime) + } + + case resp := <-queryResponseReadC: // Response from a watcher. + if resp.Status == QuerySuccess { + if resp.Response == nil { + qLogger.Error("received a successful query response with no results, dropping it!", zap.String("requestID", resp.RequestID)) + continue + } + + pq, exists := pendingQueries[resp.RequestID] + if !exists { + qLogger.Warn("received a success response with no outstanding query, dropping it", zap.String("requestID", resp.RequestID), zap.Int("requestIdx", resp.RequestIdx)) + continue + } + + if resp.RequestIdx >= len(pq.responses) { + qLogger.Error("received a response with an invalid index", zap.String("requestID", resp.RequestID), zap.Int("requestIdx", resp.RequestIdx)) + continue + } + + // Store the result, which will mark this per-chain query as completed. + pq.responses[resp.RequestIdx] = resp + + // If we still have other outstanding per chain queries for this request, keep waiting. + numStillPending := pq.numPendingRequests() + if numStillPending > 0 { + qLogger.Info("received a per chain query response, still waiting for more", zap.String("requestID", resp.RequestID), zap.Int("requestIdx", resp.RequestIdx), zap.Int("numStillPending", numStillPending)) + continue + } else { + qLogger.Info("received final per chain query response, ready to publish", zap.String("requestID", resp.RequestID), zap.Int("requestIdx", resp.RequestIdx)) + } + + // Build the list of per chain response publications and the overall query response publication. + responses := []*PerChainQueryResponse{} + for _, resp := range pq.responses { + if resp == nil { + qLogger.Error("unexpected null response in pending query!", zap.String("requestID", resp.RequestID), zap.Int("requestIdx", resp.RequestIdx)) + continue + } + + responses = append(responses, &PerChainQueryResponse{ + ChainId: resp.ChainId, + Response: resp.Response, + }) + } + + respPub := &QueryResponsePublication{ + Request: pq.signedRequest, + PerChainResponses: responses, + } + + // Send the response to be published. + select { + case queryResponseWriteC <- respPub: + qLogger.Info("forwarded query response to p2p", zap.String("requestID", resp.RequestID)) + delete(pendingQueries, resp.RequestID) + default: + qLogger.Warn("failed to publish query response to p2p, will retry publishing next interval", zap.String("requestID", resp.RequestID)) + pq.respPub = respPub + } + } else if resp.Status == QueryRetryNeeded { + if _, exists := pendingQueries[resp.RequestID]; exists { + qLogger.Warn("query failed, will retry next interval", zap.String("requestID", resp.RequestID), zap.Int("requestIdx", resp.RequestIdx)) + } else { + qLogger.Warn("received a retry needed response with no outstanding query, dropping it", zap.String("requestID", resp.RequestID), zap.Int("requestIdx", resp.RequestIdx)) + } + } else if resp.Status == QueryFatalError { + qLogger.Warn("received a fatal error response, dropping the whole request", zap.String("requestID", resp.RequestID), zap.Int("requestIdx", resp.RequestIdx)) + delete(pendingQueries, resp.RequestID) + } else { + qLogger.Warn("received an unexpected query status, dropping the whole request", zap.String("requestID", resp.RequestID), zap.Int("requestIdx", resp.RequestIdx), zap.Int("status", int(resp.Status))) + delete(pendingQueries, resp.RequestID) + } + + case <-ticker.C: // Retry audit timer. + now := time.Now() + for reqId, pq := range pendingQueries { + timeout := pq.receiveTime.Add(requestTimeoutImpl) + qLogger.Debug("audit", zap.String("requestId", reqId), zap.Stringer("receiveTime", pq.receiveTime), zap.Stringer("timeout", timeout)) + if timeout.Before(now) { + qLogger.Warn("query request timed out, dropping it", zap.String("requestId", reqId), zap.Stringer("receiveTime", pq.receiveTime)) + delete(pendingQueries, reqId) + } else { + if pq.respPub != nil { + // Resend the response to be published. + select { + case queryResponseWriteC <- pq.respPub: + qLogger.Debug("resend of query response to p2p succeeded", zap.String("requestID", reqId)) + delete(pendingQueries, reqId) + default: + qLogger.Warn("resend of query response to p2p failed again, will keep retrying", zap.String("requestID", reqId)) + } + } else { + for requestIdx, pcq := range pq.queries { + if pq.responses[requestIdx] == nil && pcq.lastUpdateTime.Add(retryIntervalImpl).Before(now) { + qLogger.Info("retrying query request", + zap.String("requestId", reqId), + zap.Int("requestIdx", requestIdx), + zap.Stringer("receiveTime", pq.receiveTime), + zap.Stringer("lastUpdateTime", pcq.lastUpdateTime), + zap.String("chainID", pq.queries[requestIdx].req.Request.ChainId.String()), + ) + pcq.ccqForwardToWatcher(qLogger, pq.receiveTime) + } + } + } + } + } + } + } +} + +// parseAllowedRequesters parses a comma separated list of allowed requesters into a map to be used for look ups. +func parseAllowedRequesters(ccqAllowedRequesters string) (map[ethCommon.Address]struct{}, error) { + if ccqAllowedRequesters == "" { + return nil, fmt.Errorf("if cross chain query is enabled `--ccqAllowedRequesters` must be specified") + } + + var nullAddr ethCommon.Address + result := make(map[ethCommon.Address]struct{}) + for _, str := range strings.Split(ccqAllowedRequesters, ",") { + addr := ethCommon.BytesToAddress(ethCommon.Hex2Bytes(str)) + if addr == nullAddr { + return nil, fmt.Errorf("invalid value in `--ccqAllowedRequesters`: `%s`", str) + } + result[addr] = struct{}{} + } + + if len(result) <= 0 { + return nil, fmt.Errorf("no allowed requestors specified, ccqAllowedRequesters: `%s`", ccqAllowedRequesters) + } + + return result, nil +} + +// ccqForwardToWatcher submits a query request to the appropriate watcher. It updates the request object if the write succeeds. +// If the write fails, it does not update the last update time, which will cause a retry next interval (until it times out) +func (pcq *perChainQuery) ccqForwardToWatcher(qLogger *zap.Logger, receiveTime time.Time) { + select { + // TODO: only send the query request itself and reassemble in this module + case pcq.channel <- pcq.req: + qLogger.Debug("forwarded query request to watcher", zap.String("requestID", pcq.req.RequestID), zap.Stringer("chainID", pcq.req.Request.ChainId)) + pcq.lastUpdateTime = receiveTime + default: + // By leaving lastUpdateTime unset, we will retry next interval. + qLogger.Warn("failed to send query request to watcher, will retry next interval", zap.String("requestID", pcq.req.RequestID), zap.Stringer("chain_id", pcq.req.Request.ChainId)) + } +} + +// numPendingRequests returns the number of per chain queries in a request that are still awaiting responses. Zero means the request can now be published. +func (pq *pendingQuery) numPendingRequests() int { + numPending := 0 + for _, resp := range pq.responses { + if resp == nil { + numPending += 1 + } + } + + return numPending +} diff --git a/node/pkg/query/query_test.go b/node/pkg/query/query_test.go new file mode 100644 index 0000000000..944c643a1e --- /dev/null +++ b/node/pkg/query/query_test.go @@ -0,0 +1,668 @@ +package query + +import ( + "bytes" + "context" + "crypto/ecdsa" + "encoding/hex" + "fmt" + "math" + "strconv" + "strings" + "sync" + "testing" + "time" + + "github.com/certusone/wormhole/node/pkg/common" + gossipv1 "github.com/certusone/wormhole/node/pkg/proto/gossip/v1" + "github.com/wormhole-foundation/wormhole/sdk/vaa" + + ethCommon "github.com/ethereum/go-ethereum/common" + ethCrypto "github.com/ethereum/go-ethereum/crypto" + + "github.com/stretchr/testify/assert" + "github.com/stretchr/testify/require" + + "go.uber.org/zap" +) + +const ( + testSigner = "beFA429d57cD18b7F8A4d91A2da9AB4AF05d0FBe" + + // Magic retry values used to cause special behavior in the watchers. + fatalError = math.MaxInt + ignoreQuery = math.MaxInt - 1 + + // Speed things up for testing purposes. + requestTimeoutForTest = 100 * time.Millisecond + retryIntervalForTest = 10 * time.Millisecond + pollIntervalForTest = 5 * time.Millisecond +) + +var ( + nonce = uint32(0) + + watcherChainsForTest = []vaa.ChainID{vaa.ChainIDPolygon, vaa.ChainIDBSC} +) + +// createPerChainQueryForTesting creates a per chain query for use in tests. The To and Data fields are meaningless gibberish, not ABI. +func createPerChainQueryForTesting( + chainId vaa.ChainID, + block string, + numCalls int, +) *PerChainQueryRequest { + callData := []*EthCallData{} + for count := 0; count < numCalls; count++ { + callData = append(callData, &EthCallData{ + To: []byte(fmt.Sprintf("%-20s", fmt.Sprintf("To for %d:%d", chainId, count))), + Data: []byte(fmt.Sprintf("CallData for %d:%d", chainId, count)), + }) + } + + callRequest := &EthCallQueryRequest{ + BlockId: block, + CallData: callData, + } + + return &PerChainQueryRequest{ + ChainId: chainId, + Query: callRequest, + } +} + +// createSignedQueryRequestForTesting creates a query request object and signs it using the specified key. +func createSignedQueryRequestForTesting( + sk *ecdsa.PrivateKey, + perChainQueries []*PerChainQueryRequest, +) (*gossipv1.SignedQueryRequest, *QueryRequest) { + nonce += 1 + queryRequest := &QueryRequest{ + Nonce: nonce, + PerChainQueries: perChainQueries, + } + + queryRequestBytes, err := queryRequest.Marshal() + if err != nil { + panic(err) + } + + digest := QueryRequestDigest(common.UnsafeDevNet, queryRequestBytes) + sig, err := ethCrypto.Sign(digest.Bytes(), sk) + if err != nil { + panic(err) + } + + signedQueryRequest := &gossipv1.SignedQueryRequest{ + QueryRequest: queryRequestBytes, + Signature: sig, + } + + return signedQueryRequest, queryRequest +} + +// createExpectedResultsForTest generates an array of the results expected for a request. These results are returned by the watcher, and used to validate the response. +func createExpectedResultsForTest(perChainQueries []*PerChainQueryRequest) []PerChainQueryResponse { + expectedResults := []PerChainQueryResponse{} + for _, pcq := range perChainQueries { + switch req := pcq.Query.(type) { + case *EthCallQueryRequest: + now := time.Now() + blockNum, err := strconv.ParseUint(strings.TrimPrefix(req.BlockId, "0x"), 16, 64) + if err != nil { + panic("invalid blockNum!") + } + resp := &EthCallQueryResponse{ + BlockNumber: blockNum, + Hash: ethCommon.HexToHash("0x9999bac44d09a7f69ee7941819b0a19c59ccb1969640cc513be09ef95ed2d8e2"), + Time: timeForTest(timeForTest(now)), + Results: [][]byte{}, + } + for _, cd := range req.CallData { + resp.Results = append(resp.Results, []byte(hex.EncodeToString(cd.To)+":"+hex.EncodeToString(cd.Data))) + } + expectedResults = append(expectedResults, PerChainQueryResponse{ + ChainId: pcq.ChainId, + Response: resp, + }) + + default: + panic("Invalid call data type!") + } + } + + return expectedResults +} + +// validateResponseForTest performs validation on the responses generated by these tests. Note that it is not a generalized validate function. +func validateResponseForTest( + t *testing.T, + response *QueryResponsePublication, + signedRequest *gossipv1.SignedQueryRequest, + queryRequest *QueryRequest, + expectedResults []PerChainQueryResponse, +) bool { + require.NotNil(t, response) + require.True(t, SignedQueryRequestEqual(signedRequest, response.Request)) + require.Equal(t, len(queryRequest.PerChainQueries), len(response.PerChainResponses)) + require.True(t, bytes.Equal(response.Request.Signature, signedRequest.Signature)) + require.Equal(t, len(response.PerChainResponses), len(expectedResults)) + for idx := range response.PerChainResponses { + require.True(t, response.PerChainResponses[idx].Equal(&expectedResults[idx])) + } + + return true +} + +func TestParseAllowedRequestersSuccess(t *testing.T) { + ccqAllowedRequestersList, err := parseAllowedRequesters(testSigner) + require.NoError(t, err) + require.NotNil(t, ccqAllowedRequestersList) + require.Equal(t, 1, len(ccqAllowedRequestersList)) + + _, exists := ccqAllowedRequestersList[ethCommon.BytesToAddress(ethCommon.Hex2Bytes(testSigner))] + require.True(t, exists) + _, exists = ccqAllowedRequestersList[ethCommon.BytesToAddress(ethCommon.Hex2Bytes("beFA429d57cD18b7F8A4d91A2da9AB4AF05d0FBf"))] + require.False(t, exists) + + ccqAllowedRequestersList, err = parseAllowedRequesters("beFA429d57cD18b7F8A4d91A2da9AB4AF05d0FBe,beFA429d57cD18b7F8A4d91A2da9AB4AF05d0FBf") + require.NoError(t, err) + require.NotNil(t, ccqAllowedRequestersList) + require.Equal(t, 2, len(ccqAllowedRequestersList)) + + _, exists = ccqAllowedRequestersList[ethCommon.BytesToAddress(ethCommon.Hex2Bytes(testSigner))] + require.True(t, exists) + _, exists = ccqAllowedRequestersList[ethCommon.BytesToAddress(ethCommon.Hex2Bytes("beFA429d57cD18b7F8A4d91A2da9AB4AF05d0FBf"))] + require.True(t, exists) +} + +func TestParseAllowedRequestersFailsIfParameterEmpty(t *testing.T) { + ccqAllowedRequestersList, err := parseAllowedRequesters("") + require.Error(t, err) + require.Nil(t, ccqAllowedRequestersList) + + ccqAllowedRequestersList, err = parseAllowedRequesters(",") + require.Error(t, err) + require.Nil(t, ccqAllowedRequestersList) +} + +func TestParseAllowedRequestersFailsIfInvalidParameter(t *testing.T) { + ccqAllowedRequestersList, err := parseAllowedRequesters("Hello") + require.Error(t, err) + require.Nil(t, ccqAllowedRequestersList) +} + +// mockData is the data structure used to mock up the query handler environment. +type mockData struct { + sk *ecdsa.PrivateKey + + signedQueryReqReadC <-chan *gossipv1.SignedQueryRequest + signedQueryReqWriteC chan<- *gossipv1.SignedQueryRequest + + chainQueryReqC map[vaa.ChainID]chan *PerChainQueryInternal + + queryResponseReadC <-chan *PerChainQueryResponseInternal + queryResponseWriteC chan<- *PerChainQueryResponseInternal + + queryResponsePublicationReadC <-chan *QueryResponsePublication + queryResponsePublicationWriteC chan<- *QueryResponsePublication + + mutex sync.Mutex + queryResponsePublication *QueryResponsePublication + expectedResults []PerChainQueryResponse + requestsPerChain map[vaa.ChainID]int + retriesPerChain map[vaa.ChainID]int +} + +// resetState() is used to reset mock data between queries in the same test. +func (md *mockData) resetState() { + md.mutex.Lock() + defer md.mutex.Unlock() + md.queryResponsePublication = nil + md.expectedResults = nil + md.requestsPerChain = make(map[vaa.ChainID]int) + md.retriesPerChain = make(map[vaa.ChainID]int) +} + +// setExpectedResults sets the results to be returned by the watchers. +func (md *mockData) setExpectedResults(expectedResults []PerChainQueryResponse) { + md.mutex.Lock() + defer md.mutex.Unlock() + md.expectedResults = expectedResults +} + +// setRetries allows a test to specify how many times a given watcher should retry before returning success. +// If the count is the special value `fatalError`, the watcher will return QueryFatalError. +func (md *mockData) setRetries(chainId vaa.ChainID, count int) { + md.mutex.Lock() + defer md.mutex.Unlock() + md.retriesPerChain[chainId] = count +} + +// incrementRequestsPerChainAlreadyLocked is used by the watchers to keep track of how many times they were invoked in a given test. +func (md *mockData) incrementRequestsPerChainAlreadyLocked(chainId vaa.ChainID) { + if val, exists := md.requestsPerChain[chainId]; exists { + md.requestsPerChain[chainId] = val + 1 + } else { + md.requestsPerChain[chainId] = 1 + } +} + +// getQueryResponsePublication returns the latest query response publication received by the mock. +func (md *mockData) getQueryResponsePublication() *QueryResponsePublication { + md.mutex.Lock() + defer md.mutex.Unlock() + return md.queryResponsePublication +} + +// getRequestsPerChain returns the count of the number of times the given watcher was invoked in a given test. +func (md *mockData) getRequestsPerChain(chainId vaa.ChainID) int { + md.mutex.Lock() + defer md.mutex.Unlock() + if ret, exists := md.requestsPerChain[chainId]; exists { + return ret + } + return 0 +} + +// shouldIgnoreAlreadyLocked is used by the watchers to see if they should ignore a query (causing a retry). +func (md *mockData) shouldIgnoreAlreadyLocked(chainId vaa.ChainID) bool { + if val, exists := md.retriesPerChain[chainId]; exists { + if val == ignoreQuery { + delete(md.retriesPerChain, chainId) + return true + } + } + return false +} + +// getStatusAlreadyLocked is used by the watchers to determine what query status they should return, based on the `retriesPerChain`. +func (md *mockData) getStatusAlreadyLocked(chainId vaa.ChainID) QueryStatus { + if val, exists := md.retriesPerChain[chainId]; exists { + if val == fatalError { + return QueryFatalError + } + val -= 1 + if val > 0 { + md.retriesPerChain[chainId] = val + } else { + delete(md.retriesPerChain, chainId) + } + return QueryRetryNeeded + } + return QuerySuccess +} + +// createQueryHandlerForTest creates the query handler mock environment, including the set of watchers and the response listener. +// Most tests will use this function to set up the mock. +func createQueryHandlerForTest(t *testing.T, ctx context.Context, logger *zap.Logger, chains []vaa.ChainID) *mockData { + md := createQueryHandlerForTestWithoutPublisher(t, ctx, logger, chains) + md.startResponseListener(ctx) + return md +} + +// createQueryHandlerForTestWithoutPublisher creates the query handler mock environment, including the set of watchers but not the response listener. +// This function can be invoked directly to test retries of response publication (by delaying the start of the response listener). +func createQueryHandlerForTestWithoutPublisher(t *testing.T, ctx context.Context, logger *zap.Logger, chains []vaa.ChainID) *mockData { + md := mockData{} + var err error + + md.sk, err = common.LoadGuardianKey("dev.guardian.key", true) + require.NoError(t, err) + require.NotNil(t, md.sk) + + ccqAllowedRequestersList, err := parseAllowedRequesters(testSigner) + require.NoError(t, err) + + // Inbound observation requests from the p2p service (for all chains) + md.signedQueryReqReadC, md.signedQueryReqWriteC = makeChannelPair[*gossipv1.SignedQueryRequest](SignedQueryRequestChannelSize) + + // Per-chain query requests + md.chainQueryReqC = make(map[vaa.ChainID]chan *PerChainQueryInternal) + for _, chainId := range chains { + md.chainQueryReqC[chainId] = make(chan *PerChainQueryInternal) + } + + // Query responses from watchers to query handler aggregated across all chains + md.queryResponseReadC, md.queryResponseWriteC = makeChannelPair[*PerChainQueryResponseInternal](0) + + // Query responses from query handler to p2p + md.queryResponsePublicationReadC, md.queryResponsePublicationWriteC = makeChannelPair[*QueryResponsePublication](0) + + md.resetState() + + go func() { + err := handleQueryRequestsImpl(ctx, logger, md.signedQueryReqReadC, md.chainQueryReqC, ccqAllowedRequestersList, + md.queryResponseReadC, md.queryResponsePublicationWriteC, common.GoTest, requestTimeoutForTest, retryIntervalForTest) + assert.NoError(t, err) + }() + + // Create a routine for each configured watcher. It will take a per chain query and return the corresponding expected result. + // It also pegs a counter of the number of requests the watcher received, for verification purposes. + for chainId := range md.chainQueryReqC { + go func(chainId vaa.ChainID, chainQueryReqC <-chan *PerChainQueryInternal) { + for { + select { + case <-ctx.Done(): + return + case pcqr := <-chainQueryReqC: + require.Equal(t, chainId, pcqr.Request.ChainId) + md.mutex.Lock() + md.incrementRequestsPerChainAlreadyLocked(chainId) + if md.shouldIgnoreAlreadyLocked(chainId) { + logger.Info("watcher ignoring query", zap.String("chainId", chainId.String()), zap.Int("requestIdx", pcqr.RequestIdx)) + } else { + results := md.expectedResults[pcqr.RequestIdx].Response + status := md.getStatusAlreadyLocked(chainId) + logger.Info("watcher returning", zap.String("chainId", chainId.String()), zap.Int("requestIdx", pcqr.RequestIdx), zap.Int("status", int(status))) + queryResponse := CreatePerChainQueryResponseInternal(pcqr.RequestID, pcqr.RequestIdx, pcqr.Request.ChainId, status, results) + md.queryResponseWriteC <- queryResponse + } + md.mutex.Unlock() + } + } + }(chainId, md.chainQueryReqC[chainId]) + } + + return &md +} + +// startResponseListener starts the response listener routine. It is called as part of the standard mock environment set up. Or, it can be used +// along with `createQueryHandlerForTestWithoutPublisher“ to test retries of response publication (by delaying the start of the response listener). +func (md *mockData) startResponseListener(ctx context.Context) { + go func() { + for { + select { + case <-ctx.Done(): + return + case qrp := <-md.queryResponsePublicationReadC: + md.mutex.Lock() + md.queryResponsePublication = qrp + md.mutex.Unlock() + } + } + }() +} + +// waitForResponse is used by the tests to wait for a response publication. It will eventually timeout if the query fails. +func (md *mockData) waitForResponse() *QueryResponsePublication { + for count := 0; count < 50; count++ { + time.Sleep(pollIntervalForTest) + ret := md.getQueryResponsePublication() + if ret != nil { + return ret + } + } + return nil +} + +// TestInvalidQueries tests all the obvious reasons why a query may fail (aside from watcher failures). +func TestInvalidQueries(t *testing.T) { + ctx := context.Background() + logger, err := zap.NewDevelopment() + require.NoError(t, err) + + md := createQueryHandlerForTest(t, ctx, logger, watcherChainsForTest) + + var perChainQueries []*PerChainQueryRequest + var signedQueryRequest *gossipv1.SignedQueryRequest + + // Query with a bad signature should fail. + md.resetState() + perChainQueries = []*PerChainQueryRequest{createPerChainQueryForTesting(vaa.ChainIDPolygon, "0x28d9630", 2)} + signedQueryRequest, _ = createSignedQueryRequestForTesting(md.sk, perChainQueries) + signedQueryRequest.Signature[0] += 1 // Corrupt the signature. + md.signedQueryReqWriteC <- signedQueryRequest + require.Nil(t, md.waitForResponse()) + + // Query for an unsupported chain should fail. The supported chains are defined in supportedChains in query.go + md.resetState() + perChainQueries = []*PerChainQueryRequest{createPerChainQueryForTesting(vaa.ChainIDAlgorand, "0x28d9630", 2)} + signedQueryRequest, _ = createSignedQueryRequestForTesting(md.sk, perChainQueries) + md.signedQueryReqWriteC <- signedQueryRequest + require.Nil(t, md.waitForResponse()) + + // Query for a chain that supports queries but that is not in the watcher channel map should fail. + md.resetState() + perChainQueries = []*PerChainQueryRequest{createPerChainQueryForTesting(vaa.ChainIDSepolia, "0x28d9630", 2)} + signedQueryRequest, _ = createSignedQueryRequestForTesting(md.sk, perChainQueries) + md.signedQueryReqWriteC <- signedQueryRequest + require.Nil(t, md.waitForResponse()) +} + +func TestSingleQueryShouldSucceed(t *testing.T) { + ctx := context.Background() + logger, err := zap.NewDevelopment() + require.NoError(t, err) + + md := createQueryHandlerForTest(t, ctx, logger, watcherChainsForTest) + + // Create the request and the expected results. Give the expected results to the mock. + perChainQueries := []*PerChainQueryRequest{createPerChainQueryForTesting(vaa.ChainIDPolygon, "0x28d9630", 2)} + signedQueryRequest, queryRequest := createSignedQueryRequestForTesting(md.sk, perChainQueries) + expectedResults := createExpectedResultsForTest(queryRequest.PerChainQueries) + md.setExpectedResults(expectedResults) + + // Submit the query request to the handler. + md.signedQueryReqWriteC <- signedQueryRequest + + // Wait until we receive a response or timeout. + queryResponsePublication := md.waitForResponse() + require.NotNil(t, queryResponsePublication) + + assert.Equal(t, 1, md.getRequestsPerChain(vaa.ChainIDPolygon)) + assert.True(t, validateResponseForTest(t, queryResponsePublication, signedQueryRequest, queryRequest, expectedResults)) +} + +func TestBatchOfTwoQueriesShouldSucceed(t *testing.T) { + ctx := context.Background() + logger, err := zap.NewDevelopment() + require.NoError(t, err) + + md := createQueryHandlerForTest(t, ctx, logger, watcherChainsForTest) + + // Create the request and the expected results. Give the expected results to the mock. + perChainQueries := []*PerChainQueryRequest{ + createPerChainQueryForTesting(vaa.ChainIDPolygon, "0x28d9630", 2), + createPerChainQueryForTesting(vaa.ChainIDBSC, "0x28d9123", 3), + } + signedQueryRequest, queryRequest := createSignedQueryRequestForTesting(md.sk, perChainQueries) + expectedResults := createExpectedResultsForTest(queryRequest.PerChainQueries) + md.setExpectedResults(expectedResults) + + // Submit the query request to the handler. + md.signedQueryReqWriteC <- signedQueryRequest + + // Wait until we receive a response or timeout. + queryResponsePublication := md.waitForResponse() + require.NotNil(t, queryResponsePublication) + + assert.Equal(t, 1, md.getRequestsPerChain(vaa.ChainIDPolygon)) + assert.Equal(t, 1, md.getRequestsPerChain(vaa.ChainIDBSC)) + assert.True(t, validateResponseForTest(t, queryResponsePublication, signedQueryRequest, queryRequest, expectedResults)) +} + +func TestQueryWithLimitedRetriesShouldSucceed(t *testing.T) { + ctx := context.Background() + logger, err := zap.NewDevelopment() + require.NoError(t, err) + + md := createQueryHandlerForTest(t, ctx, logger, watcherChainsForTest) + + // Create the request and the expected results. Give the expected results to the mock. + perChainQueries := []*PerChainQueryRequest{createPerChainQueryForTesting(vaa.ChainIDPolygon, "0x28d9630", 2)} + signedQueryRequest, queryRequest := createSignedQueryRequestForTesting(md.sk, perChainQueries) + expectedResults := createExpectedResultsForTest(queryRequest.PerChainQueries) + md.setExpectedResults(expectedResults) + + // Make it retry a couple of times, but not enough to make it fail. + retries := 2 + md.setRetries(vaa.ChainIDPolygon, retries) + + // Submit the query request to the handler. + md.signedQueryReqWriteC <- signedQueryRequest + + // The request should eventually succeed. + queryResponsePublication := md.waitForResponse() + require.NotNil(t, queryResponsePublication) + + assert.Equal(t, retries+1, md.getRequestsPerChain(vaa.ChainIDPolygon)) + assert.True(t, validateResponseForTest(t, queryResponsePublication, signedQueryRequest, queryRequest, expectedResults)) +} + +func TestQueryWithRetryDueToTimeoutShouldSucceed(t *testing.T) { + ctx := context.Background() + logger, err := zap.NewDevelopment() + require.NoError(t, err) + + md := createQueryHandlerForTest(t, ctx, logger, watcherChainsForTest) + + // Create the request and the expected results. Give the expected results to the mock. + perChainQueries := []*PerChainQueryRequest{createPerChainQueryForTesting(vaa.ChainIDPolygon, "0x28d9630", 2)} + signedQueryRequest, queryRequest := createSignedQueryRequestForTesting(md.sk, perChainQueries) + expectedResults := createExpectedResultsForTest(queryRequest.PerChainQueries) + md.setExpectedResults(expectedResults) + + // Make the first per chain query timeout, but the retry should succeed. + md.setRetries(vaa.ChainIDPolygon, ignoreQuery) + + // Submit the query request to the handler. + md.signedQueryReqWriteC <- signedQueryRequest + + // The request should eventually succeed. + queryResponsePublication := md.waitForResponse() + require.NotNil(t, queryResponsePublication) + + assert.Equal(t, 2, md.getRequestsPerChain(vaa.ChainIDPolygon)) + assert.True(t, validateResponseForTest(t, queryResponsePublication, signedQueryRequest, queryRequest, expectedResults)) +} + +func TestQueryWithTooManyRetriesShouldFail(t *testing.T) { + ctx := context.Background() + logger, err := zap.NewDevelopment() + require.NoError(t, err) + + md := createQueryHandlerForTest(t, ctx, logger, watcherChainsForTest) + + // Create the request and the expected results. Give the expected results to the mock. + perChainQueries := []*PerChainQueryRequest{ + createPerChainQueryForTesting(vaa.ChainIDPolygon, "0x28d9630", 2), + createPerChainQueryForTesting(vaa.ChainIDBSC, "0x28d9123", 3), + } + signedQueryRequest, queryRequest := createSignedQueryRequestForTesting(md.sk, perChainQueries) + expectedResults := createExpectedResultsForTest(queryRequest.PerChainQueries) + md.setExpectedResults(expectedResults) + + // Make polygon retry a couple of times, but not enough to make it fail. + retriesForPolygon := 2 + md.setRetries(vaa.ChainIDPolygon, retriesForPolygon) + + // Make BSC retry so many times that the request times out. + md.setRetries(vaa.ChainIDBSC, 1000) + + // Submit the query request to the handler. + md.signedQueryReqWriteC <- signedQueryRequest + + // The request should timeout. + queryResponsePublication := md.waitForResponse() + require.Nil(t, queryResponsePublication) + + assert.Equal(t, retriesForPolygon+1, md.getRequestsPerChain(vaa.ChainIDPolygon)) +} + +func TestQueryWithLimitedRetriesOnMultipleChainsShouldSucceed(t *testing.T) { + ctx := context.Background() + logger, err := zap.NewDevelopment() + require.NoError(t, err) + + md := createQueryHandlerForTest(t, ctx, logger, watcherChainsForTest) + + // Create the request and the expected results. Give the expected results to the mock. + perChainQueries := []*PerChainQueryRequest{ + createPerChainQueryForTesting(vaa.ChainIDPolygon, "0x28d9630", 2), + createPerChainQueryForTesting(vaa.ChainIDBSC, "0x28d9123", 3), + } + signedQueryRequest, queryRequest := createSignedQueryRequestForTesting(md.sk, perChainQueries) + expectedResults := createExpectedResultsForTest(queryRequest.PerChainQueries) + md.setExpectedResults(expectedResults) + + // Make both chains retry a couple of times, but not enough to make it fail. + retriesForPolygon := 2 + md.setRetries(vaa.ChainIDPolygon, retriesForPolygon) + + retriesForBSC := 3 + md.setRetries(vaa.ChainIDBSC, retriesForBSC) + + // Submit the query request to the handler. + md.signedQueryReqWriteC <- signedQueryRequest + + // The request should eventually succeed. + queryResponsePublication := md.waitForResponse() + require.NotNil(t, queryResponsePublication) + + assert.Equal(t, retriesForPolygon+1, md.getRequestsPerChain(vaa.ChainIDPolygon)) + assert.Equal(t, retriesForBSC+1, md.getRequestsPerChain(vaa.ChainIDBSC)) + assert.True(t, validateResponseForTest(t, queryResponsePublication, signedQueryRequest, queryRequest, expectedResults)) +} + +func TestFatalErrorOnPerChainQueryShouldCauseRequestToFail(t *testing.T) { + ctx := context.Background() + logger, err := zap.NewDevelopment() + require.NoError(t, err) + + md := createQueryHandlerForTest(t, ctx, logger, watcherChainsForTest) + + // Create the request and the expected results. Give the expected results to the mock. + perChainQueries := []*PerChainQueryRequest{ + createPerChainQueryForTesting(vaa.ChainIDPolygon, "0x28d9630", 2), + createPerChainQueryForTesting(vaa.ChainIDBSC, "0x28d9123", 3), + } + signedQueryRequest, queryRequest := createSignedQueryRequestForTesting(md.sk, perChainQueries) + expectedResults := createExpectedResultsForTest(queryRequest.PerChainQueries) + md.setExpectedResults(expectedResults) + + // Make BSC return a fatal error. + md.setRetries(vaa.ChainIDBSC, fatalError) + + // Submit the query request to the handler. + md.signedQueryReqWriteC <- signedQueryRequest + + // The request should timeout. + queryResponsePublication := md.waitForResponse() + require.Nil(t, queryResponsePublication) + + assert.Equal(t, 1, md.getRequestsPerChain(vaa.ChainIDPolygon)) + assert.Equal(t, 1, md.getRequestsPerChain(vaa.ChainIDBSC)) +} + +func TestPublishRetrySucceeds(t *testing.T) { + ctx := context.Background() + logger, err := zap.NewDevelopment() + require.NoError(t, err) + + md := createQueryHandlerForTestWithoutPublisher(t, ctx, logger, watcherChainsForTest) + + // Create the request and the expected results. Give the expected results to the mock. + perChainQueries := []*PerChainQueryRequest{createPerChainQueryForTesting(vaa.ChainIDPolygon, "0x28d9630", 2)} + signedQueryRequest, queryRequest := createSignedQueryRequestForTesting(md.sk, perChainQueries) + expectedResults := createExpectedResultsForTest(queryRequest.PerChainQueries) + md.setExpectedResults(expectedResults) + + // Submit the query request to the handler. + md.signedQueryReqWriteC <- signedQueryRequest + + // Sleep for a bit before we start listening for published results. + // If you look in the log, you should see one of these: "failed to publish query response to p2p, will retry publishing next interval" + // and at least one of these: "resend of query response to p2p failed again, will keep retrying". + time.Sleep(retryIntervalForTest * 3) + + // Now start the publisher routine. + // If you look in the log, you should see one of these: "resend of query response to p2p succeeded". + md.startResponseListener(ctx) + + // The response should still get published. + queryResponsePublication := md.waitForResponse() + require.NotNil(t, queryResponsePublication) + + assert.Equal(t, 1, md.getRequestsPerChain(vaa.ChainIDPolygon)) + assert.True(t, validateResponseForTest(t, queryResponsePublication, signedQueryRequest, queryRequest, expectedResults)) +} diff --git a/node/pkg/query/request.go b/node/pkg/query/request.go new file mode 100644 index 0000000000..38ac24cbe9 --- /dev/null +++ b/node/pkg/query/request.go @@ -0,0 +1,470 @@ +package query + +import ( + "bytes" + "encoding/binary" + "fmt" + "math" + "strings" + + "github.com/certusone/wormhole/node/pkg/common" + gossipv1 "github.com/certusone/wormhole/node/pkg/proto/gossip/v1" + "github.com/wormhole-foundation/wormhole/sdk/vaa" + + ethCommon "github.com/ethereum/go-ethereum/common" + ethCrypto "github.com/ethereum/go-ethereum/crypto" +) + +// QueryRequest defines a cross chain query request to be submitted to the guardians. +// It is the payload of the SignedQueryRequest gossip message. +type QueryRequest struct { + Nonce uint32 + PerChainQueries []*PerChainQueryRequest +} + +// PerChainQueryRequest represents a query request for a single chain. +type PerChainQueryRequest struct { + // ChainId indicates which chain this query is destine for. + ChainId vaa.ChainID + + // Query is the chain specific query data. + Query ChainSpecificQuery +} + +// ChainSpecificQuery is the interface that must be implemented by a chain specific query. +type ChainSpecificQuery interface { + Type() ChainSpecificQueryType + Marshal() ([]byte, error) + Unmarshal(data []byte) error + UnmarshalFromReader(reader *bytes.Reader) error + Validate() error +} + +// ChainSpecificQueryType is used to interpret the data in a per chain query request. +type ChainSpecificQueryType uint8 + +// EthCallQueryRequestType is the type of an EVM eth_call query request. +const EthCallQueryRequestType ChainSpecificQueryType = 1 + +// EthCallQueryRequest implements ChainSpecificQuery for an EVM eth_call query request. +type EthCallQueryRequest struct { + // BlockId identifies the block to be queried. It mus be a hex string starting with 0x. It may be a block number or a block hash. + BlockId string + + // CallData is an array of specific queries to be performed on the specified block, in a single RPC call. + CallData []*EthCallData +} + +// EthCallData specifies the parameters to a single EVM eth_call request. +type EthCallData struct { + // To specifies the contract address to be queried. + To []byte + + // Data is the ABI encoded parameters to the query. + Data []byte +} + +const EvmContractAddressLength = 20 + +// PerChainQueryInternal is an internal representation of a query request that is passed to the watcher. +type PerChainQueryInternal struct { + RequestID string + RequestIdx int + Request *PerChainQueryRequest +} + +// QueryRequestDigest returns the query signing prefix based on the environment. +func QueryRequestDigest(env common.Environment, b []byte) ethCommon.Hash { + // TODO: should this use a different standard of signing messages, like https://eips.ethereum.org/EIPS/eip-712 + var queryRequestPrefix []byte + if env == common.MainNet { + queryRequestPrefix = []byte("mainnet_query_request_000000000000|") + } else if env == common.TestNet { + queryRequestPrefix = []byte("testnet_query_request_000000000000|") + } else { + queryRequestPrefix = []byte("devnet_query_request_0000000000000|") + } + + return ethCrypto.Keccak256Hash(append(queryRequestPrefix, b...)) +} + +// PostSignedQueryRequest posts a signed query request to the specified channel. +func PostSignedQueryRequest(signedQueryReqSendC chan<- *gossipv1.SignedQueryRequest, req *gossipv1.SignedQueryRequest) error { + select { + case signedQueryReqSendC <- req: + return nil + default: + return common.ErrChanFull + } +} + +// +// Implementation of QueryRequest. +// + +// Marshal serializes the binary representation of a query request. +// This method calls Validate() and relies on it to range checks lengths, etc. +func (queryRequest *QueryRequest) Marshal() ([]byte, error) { + if err := queryRequest.Validate(); err != nil { + return nil, err + } + + buf := new(bytes.Buffer) + + vaa.MustWrite(buf, binary.BigEndian, uint8(1)) // version + vaa.MustWrite(buf, binary.BigEndian, queryRequest.Nonce) // uint32 + + vaa.MustWrite(buf, binary.BigEndian, uint8(len(queryRequest.PerChainQueries))) + for _, perChainQuery := range queryRequest.PerChainQueries { + pcqBuf, err := perChainQuery.Marshal() + if err != nil { + return nil, fmt.Errorf("failed to marshal per chain query: %w", err) + } + buf.Write(pcqBuf) + } + + return buf.Bytes(), nil +} + +// Unmarshal deserializes the binary representation of a query request from a byte array +func (queryRequest *QueryRequest) Unmarshal(data []byte) error { + reader := bytes.NewReader(data[:]) + return queryRequest.UnmarshalFromReader(reader) +} + +// UnmarshalFromReader deserializes the binary representation of a query request from an existing reader +func (queryRequest *QueryRequest) UnmarshalFromReader(reader *bytes.Reader) error { + var version uint8 + if err := binary.Read(reader, binary.BigEndian, &version); err != nil { + return fmt.Errorf("failed to read message version: %w", err) + } + + if version != 1 { + return fmt.Errorf("unsupported message version: %d", version) + } + + if err := binary.Read(reader, binary.BigEndian, &queryRequest.Nonce); err != nil { + return fmt.Errorf("failed to read request nonce: %w", err) + } + + numPerChainQueries := uint8(0) + if err := binary.Read(reader, binary.BigEndian, &numPerChainQueries); err != nil { + return fmt.Errorf("failed to read number of per chain queries: %w", err) + } + + for count := 0; count < int(numPerChainQueries); count++ { + perChainQuery := PerChainQueryRequest{} + err := perChainQuery.UnmarshalFromReader(reader) + if err != nil { + return fmt.Errorf("failed to Unmarshal per chain query: %w", err) + } + queryRequest.PerChainQueries = append(queryRequest.PerChainQueries, &perChainQuery) + } + + return nil +} + +// Validate does basic validation on a received query request. +func (queryRequest *QueryRequest) Validate() error { + // Nothing to validate on the Nonce. + if len(queryRequest.PerChainQueries) <= 0 { + return fmt.Errorf("request does not contain any per chain queries") + } + if len(queryRequest.PerChainQueries) > math.MaxUint8 { + return fmt.Errorf("too many per chain queries") + } + for idx, perChainQuery := range queryRequest.PerChainQueries { + if err := perChainQuery.Validate(); err != nil { + return fmt.Errorf("failed to validate per chain query %d: %w", idx, err) + } + } + return nil +} + +// Equal verifies that two query requests are equal. +func (left *QueryRequest) Equal(right *QueryRequest) bool { + if left.Nonce != right.Nonce { + return false + } + if len(left.PerChainQueries) != len(right.PerChainQueries) { + return false + } + + for idx := range left.PerChainQueries { + if !left.PerChainQueries[idx].Equal(right.PerChainQueries[idx]) { + return false + } + } + return true +} + +// +// Implementation of PerChainQueryRequest. +// + +// Marshal serializes the binary representation of a per chain query request. +// This method calls Validate() and relies on it to range checks lengths, etc. +func (perChainQuery *PerChainQueryRequest) Marshal() ([]byte, error) { + if err := perChainQuery.Validate(); err != nil { + return nil, err + } + + buf := new(bytes.Buffer) + vaa.MustWrite(buf, binary.BigEndian, perChainQuery.ChainId) + vaa.MustWrite(buf, binary.BigEndian, perChainQuery.Query.Type()) + queryBuf, err := perChainQuery.Query.Marshal() + if err != nil { + return nil, err + } + + // Write the length of the query to facilitate on-chain parsing. + if len(queryBuf) > math.MaxUint32 { + return nil, fmt.Errorf("query too long") + } + vaa.MustWrite(buf, binary.BigEndian, uint32(len(queryBuf))) + + buf.Write(queryBuf) + return buf.Bytes(), nil +} + +// Unmarshal deserializes the binary representation of a per chain query request from a byte array +func (perChainQuery *PerChainQueryRequest) Unmarshal(data []byte) error { + reader := bytes.NewReader(data[:]) + return perChainQuery.UnmarshalFromReader(reader) +} + +// UnmarshalFromReader deserializes the binary representation of a per chain query request from an existing reader +func (perChainQuery *PerChainQueryRequest) UnmarshalFromReader(reader *bytes.Reader) error { + if err := binary.Read(reader, binary.BigEndian, &perChainQuery.ChainId); err != nil { + return fmt.Errorf("failed to read request chain: %w", err) + } + + qt := uint8(0) + if err := binary.Read(reader, binary.BigEndian, &qt); err != nil { + return fmt.Errorf("failed to read request type: %w", err) + } + queryType := ChainSpecificQueryType(qt) + + if err := ValidatePerChainQueryRequestType(queryType); err != nil { + return err + } + + // Skip the query length. + var queryLength uint32 + if err := binary.Read(reader, binary.BigEndian, &queryLength); err != nil { + return fmt.Errorf("failed to read query length: %w", err) + } + + switch queryType { + case EthCallQueryRequestType: + q := EthCallQueryRequest{} + if err := q.UnmarshalFromReader(reader); err != nil { + return fmt.Errorf("failed to unmarshal eth call request: %w", err) + } + perChainQuery.Query = &q + default: + return fmt.Errorf("unsupported query type: %d", queryType) + } + + return nil +} + +// Validate does basic validation on a per chain query request. +func (perChainQuery *PerChainQueryRequest) Validate() error { + str := perChainQuery.ChainId.String() + if _, err := vaa.ChainIDFromString(str); err != nil { + return fmt.Errorf("invalid chainID: %d", uint16(perChainQuery.ChainId)) + } + + if perChainQuery.Query == nil { + return fmt.Errorf("query is nil") + } + + if err := ValidatePerChainQueryRequestType(perChainQuery.Query.Type()); err != nil { + return err + } + + if err := perChainQuery.Query.Validate(); err != nil { + return fmt.Errorf("chain specific query is invalid: %w", err) + } + + return nil +} + +// Equal verifies that two query requests are equal. +func (left *PerChainQueryRequest) Equal(right *PerChainQueryRequest) bool { + if left.ChainId != right.ChainId { + return false + } + + if left.Query == nil && right.Query == nil { + return true + } + + if left.Query == nil || right.Query == nil { + return false + } + + if left.Query.Type() != right.Query.Type() { + return false + } + + switch leftEcq := left.Query.(type) { + case *EthCallQueryRequest: + switch rightEcd := right.Query.(type) { + case *EthCallQueryRequest: + return leftEcq.Equal(rightEcd) + default: + panic("unsupported query type on right") // We checked this above! + } + default: + panic("unsupported query type on left") // We checked this above! + } +} + +// +// Implementation of EthCallQueryRequest, which implements the ChainSpecificQuery interface. +// + +func (e *EthCallQueryRequest) Type() ChainSpecificQueryType { + return EthCallQueryRequestType +} + +// Marshal serializes the binary representation of an EVM eth_call request. +// This method calls Validate() and relies on it to range checks lengths, etc. +func (ecd *EthCallQueryRequest) Marshal() ([]byte, error) { + if err := ecd.Validate(); err != nil { + return nil, err + } + + buf := new(bytes.Buffer) + vaa.MustWrite(buf, binary.BigEndian, uint32(len(ecd.BlockId))) + buf.Write([]byte(ecd.BlockId)) + + vaa.MustWrite(buf, binary.BigEndian, uint8(len(ecd.CallData))) + for _, callData := range ecd.CallData { + buf.Write(callData.To) + vaa.MustWrite(buf, binary.BigEndian, uint32(len(callData.Data))) + buf.Write(callData.Data) + } + return buf.Bytes(), nil +} + +// Unmarshal deserializes an EVM eth_call query from a byte array +func (ecd *EthCallQueryRequest) Unmarshal(data []byte) error { + reader := bytes.NewReader(data[:]) + return ecd.UnmarshalFromReader(reader) +} + +// UnmarshalFromReader deserializes an EVM eth_call query from a byte array +func (ecd *EthCallQueryRequest) UnmarshalFromReader(reader *bytes.Reader) error { + blockIdLen := uint32(0) + if err := binary.Read(reader, binary.BigEndian, &blockIdLen); err != nil { + return fmt.Errorf("failed to read call Data len: %w", err) + } + + blockId := make([]byte, blockIdLen) + if n, err := reader.Read(blockId[:]); err != nil || n != int(blockIdLen) { + return fmt.Errorf("failed to read call To [%d]: %w", n, err) + } + ecd.BlockId = string(blockId[:]) + + numCallData := uint8(0) + if err := binary.Read(reader, binary.BigEndian, &numCallData); err != nil { + return fmt.Errorf("failed to read number of call data entries: %w", err) + } + + for count := 0; count < int(numCallData); count++ { + to := [EvmContractAddressLength]byte{} + if n, err := reader.Read(to[:]); err != nil || n != EvmContractAddressLength { + return fmt.Errorf("failed to read call To [%d]: %w", n, err) + } + + dataLen := uint32(0) + if err := binary.Read(reader, binary.BigEndian, &dataLen); err != nil { + return fmt.Errorf("failed to read call Data len: %w", err) + } + data := make([]byte, dataLen) + if n, err := reader.Read(data[:]); err != nil || n != int(dataLen) { + return fmt.Errorf("failed to read call To [%d]: %w", n, err) + } + + callData := &EthCallData{ + To: to[:], + Data: data[:], + } + + ecd.CallData = append(ecd.CallData, callData) + } + + return nil +} + +// Validate does basic validation on an EVM eth_call query. +func (ecd *EthCallQueryRequest) Validate() error { + if len(ecd.BlockId) > math.MaxUint32 { + return fmt.Errorf("block id too long") + } + if !strings.HasPrefix(ecd.BlockId, "0x") { + return fmt.Errorf("block id must be a hex number or hash starting with 0x") + } + if len(ecd.CallData) <= 0 { + return fmt.Errorf("does not contain any call data") + } + if len(ecd.CallData) > math.MaxUint8 { + return fmt.Errorf("too many call data entries") + } + for _, callData := range ecd.CallData { + if callData.To == nil || len(callData.To) <= 0 { + return fmt.Errorf("no call data to") + } + if len(callData.To) != EvmContractAddressLength { + return fmt.Errorf("invalid length for To contract") + } + if callData.Data == nil || len(callData.Data) <= 0 { + return fmt.Errorf("no call data data") + } + if len(callData.Data) > math.MaxUint32 { + return fmt.Errorf("call data data too long") + } + } + + return nil +} + +// Equal verifies that two EVM eth_call queries are equal. +func (left *EthCallQueryRequest) Equal(right *EthCallQueryRequest) bool { + if left.BlockId != right.BlockId { + return false + } + if len(left.CallData) != len(right.CallData) { + return false + } + for idx := range left.CallData { + if !bytes.Equal(left.CallData[idx].To, right.CallData[idx].To) { + return false + } + if !bytes.Equal(left.CallData[idx].Data, right.CallData[idx].Data) { + return false + } + } + + return true +} + +func ValidatePerChainQueryRequestType(qt ChainSpecificQueryType) error { + if qt != EthCallQueryRequestType { + return fmt.Errorf("invalid query request type: %d", qt) + } + return nil +} + +func SignedQueryRequestEqual(left *gossipv1.SignedQueryRequest, right *gossipv1.SignedQueryRequest) bool { + if !bytes.Equal(left.QueryRequest, right.QueryRequest) { + return false + } + if !bytes.Equal(left.Signature, right.Signature) { + return false + } + return true +} diff --git a/node/pkg/query/response.go b/node/pkg/query/response.go new file mode 100644 index 0000000000..326195b981 --- /dev/null +++ b/node/pkg/query/response.go @@ -0,0 +1,508 @@ +package query + +import ( + "bytes" + "encoding/binary" + "encoding/hex" + "fmt" + "math" + "time" + + gossipv1 "github.com/certusone/wormhole/node/pkg/proto/gossip/v1" + "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/crypto" + "github.com/wormhole-foundation/wormhole/sdk/vaa" +) + +// QueryStatus is the status returned from the watcher to the query handler. +type QueryStatus int + +const ( + // QuerySuccess means the query was successful and the response should be returned to the requester. + QuerySuccess QueryStatus = 1 + + // QueryRetryNeeded means the query failed, but a retry may be helpful. + QueryRetryNeeded QueryStatus = 0 + + // QueryFatalError means the query failed, and there is no point in retrying it. + QueryFatalError QueryStatus = -1 +) + +// This is the query response returned from the watcher to the query handler. +type PerChainQueryResponseInternal struct { + RequestID string + RequestIdx int + ChainId vaa.ChainID + Status QueryStatus + Response ChainSpecificResponse +} + +// CreatePerChainQueryResponseInternal creates a PerChainQueryResponseInternal and returns a pointer to it. +func CreatePerChainQueryResponseInternal(reqId string, reqIdx int, chainId vaa.ChainID, status QueryStatus, response ChainSpecificResponse) *PerChainQueryResponseInternal { + return &PerChainQueryResponseInternal{ + RequestID: reqId, + RequestIdx: reqIdx, + ChainId: chainId, + Status: status, + Response: response, + } +} + +var queryResponsePrefix = []byte("query_response_0000000000000000000|") + +// QueryResponsePublication is the response to a QueryRequest. +type QueryResponsePublication struct { + Request *gossipv1.SignedQueryRequest + PerChainResponses []*PerChainQueryResponse +} + +// PerChainQueryResponse represents a query response for a single chain. +type PerChainQueryResponse struct { + // ChainId indicates which chain this query was destine for. + ChainId vaa.ChainID + + // Response is the chain specific query data. + Response ChainSpecificResponse +} + +// ChainSpecificResponse is the interface that must be implemented by a chain specific response. +type ChainSpecificResponse interface { + Type() ChainSpecificQueryType + Marshal() ([]byte, error) + Unmarshal(data []byte) error + UnmarshalFromReader(reader *bytes.Reader) error + Validate() error +} + +// EthCallQueryResponse implements ChainSpecificResponse for an EVM eth_call query response. +type EthCallQueryResponse struct { + BlockNumber uint64 + Hash common.Hash + Time time.Time + + // Results is the array of responses matching CallData in EthCallQueryRequest + Results [][]byte +} + +// +// Implementation of QueryResponsePublication. +// + +// Marshal serializes the binary representation of a query response. +// This method calls Validate() and relies on it to range checks lengths, etc. +func (msg *QueryResponsePublication) Marshal() ([]byte, error) { + if err := msg.Validate(); err != nil { + return nil, err + } + + buf := new(bytes.Buffer) + + vaa.MustWrite(buf, binary.BigEndian, uint8(1)) // version + + // Source + // TODO: support writing off-chain and on-chain requests + // Here, unset represents an off-chain request + vaa.MustWrite(buf, binary.BigEndian, vaa.ChainIDUnset) + + buf.Write(msg.Request.Signature[:]) + + // Write the length of the request to facilitate on-chain parsing. + if len(msg.Request.QueryRequest) > math.MaxUint32 { + return nil, fmt.Errorf("request too long") + } + vaa.MustWrite(buf, binary.BigEndian, uint32(len(msg.Request.QueryRequest))) + + buf.Write(msg.Request.QueryRequest) + + // Per chain responses + vaa.MustWrite(buf, binary.BigEndian, uint8(len(msg.PerChainResponses))) + for idx := range msg.PerChainResponses { + pcrBuf, err := msg.PerChainResponses[idx].Marshal() + if err != nil { + return nil, fmt.Errorf("failed to marshal per chain response: %w", err) + } + buf.Write(pcrBuf) + } + + return buf.Bytes(), nil +} + +// Unmarshal deserializes the binary representation of a query response +func (msg *QueryResponsePublication) Unmarshal(data []byte) error { + reader := bytes.NewReader(data[:]) + + var version uint8 + if err := binary.Read(reader, binary.BigEndian, &version); err != nil { + return fmt.Errorf("failed to read message version: %w", err) + } + + if version != 1 { + return fmt.Errorf("unsupported message version: %d", version) + } + + // Request + requestChain := vaa.ChainID(0) + if err := binary.Read(reader, binary.BigEndian, &requestChain); err != nil { + return fmt.Errorf("failed to read request chain: %w", err) + } + if requestChain != vaa.ChainIDUnset { + // TODO: support reading off-chain and on-chain requests + return fmt.Errorf("unsupported request chain: %d", requestChain) + } + + signedQueryRequest := &gossipv1.SignedQueryRequest{} + signature := [65]byte{} + if n, err := reader.Read(signature[:]); err != nil || n != 65 { + return fmt.Errorf("failed to read signature [%d]: %w", n, err) + } + signedQueryRequest.Signature = signature[:] + + // Skip the query length. + queryRequestLen := uint32(0) + if err := binary.Read(reader, binary.BigEndian, &queryRequestLen); err != nil { + return fmt.Errorf("failed to read length of query request: %w", err) + } + + queryRequest := QueryRequest{} + err := queryRequest.UnmarshalFromReader(reader) + if err != nil { + return fmt.Errorf("failed to unmarshal query request: %w", err) + } + + queryRequestBytes, err := queryRequest.Marshal() + if err != nil { + return err + } + signedQueryRequest.QueryRequest = queryRequestBytes + + msg.Request = signedQueryRequest + + // Responses + numPerChainResponses := uint8(0) + if err := binary.Read(reader, binary.BigEndian, &numPerChainResponses); err != nil { + return fmt.Errorf("failed to read number of per chain responses: %w", err) + } + + for count := 0; count < int(numPerChainResponses); count++ { + var pcr PerChainQueryResponse + err := pcr.UnmarshalFromReader(reader) + if err != nil { + return fmt.Errorf("failed to unmarshal per chain response: %w", err) + } + msg.PerChainResponses = append(msg.PerChainResponses, &pcr) + } + + return nil +} + +// Validate does basic validation on a received query request. +func (msg *QueryResponsePublication) Validate() error { + // Unmarshal and validate the contained query request. + var queryRequest QueryRequest + err := queryRequest.Unmarshal(msg.Request.QueryRequest) + if err != nil { + return fmt.Errorf("failed to unmarshal query request") + } + if err := queryRequest.Validate(); err != nil { + return fmt.Errorf("query request is invalid: %w", err) + } + + if len(msg.PerChainResponses) <= 0 { + return fmt.Errorf("response does not contain any per chain responses") + } + if len(msg.PerChainResponses) > math.MaxUint8 { + return fmt.Errorf("too many per chain responses") + } + if len(msg.PerChainResponses) != len(queryRequest.PerChainQueries) { + return fmt.Errorf("number of responses does not match number of queries") + } + for idx, pcr := range msg.PerChainResponses { + if err := pcr.Validate(); err != nil { + return fmt.Errorf("failed to validate per chain query %d: %w", idx, err) + } + if pcr.Response.Type() != queryRequest.PerChainQueries[idx].Query.Type() { + return fmt.Errorf("type of response %d does not match the query", idx) + } + } + return nil +} + +// Equal checks for equality on two query response publications. +func (left *QueryResponsePublication) Equal(right *QueryResponsePublication) bool { + if !bytes.Equal(left.Request.QueryRequest, right.Request.QueryRequest) || !bytes.Equal(left.Request.Signature, right.Request.Signature) { + return false + } + if len(left.PerChainResponses) != len(right.PerChainResponses) { + return false + } + for idx := range left.PerChainResponses { + if !left.PerChainResponses[idx].Equal(right.PerChainResponses[idx]) { + return false + } + } + return true +} + +func (resp *QueryResponsePublication) RequestID() string { + if resp == nil || resp.Request == nil { + return "nil" + } + return hex.EncodeToString(resp.Request.Signature) +} + +// Similar to sdk/vaa/structs.go, +// In order to save space in the solana signature verification instruction, we hash twice so we only need to pass in +// the first hash (32 bytes) vs the full body data. +// TODO: confirm if this works / is worthwhile. +func (msg *QueryResponsePublication) SigningDigest() (common.Hash, error) { + msgBytes, err := msg.Marshal() + if err != nil { + return common.Hash{}, err + } + return GetQueryResponseDigestFromBytes(msgBytes), nil +} + +// GetQueryResponseDigestFromBytes computes the digest bytes for a query response byte array. +func GetQueryResponseDigestFromBytes(b []byte) common.Hash { + return crypto.Keccak256Hash(append(queryResponsePrefix, crypto.Keccak256Hash(b).Bytes()...)) +} + +// +// Implementation of PerChainQueryResponse. +// + +// Marshal marshalls a per chain query response. +func (perChainResponse *PerChainQueryResponse) Marshal() ([]byte, error) { + if err := perChainResponse.Validate(); err != nil { + return nil, err + } + + buf := new(bytes.Buffer) + vaa.MustWrite(buf, binary.BigEndian, perChainResponse.ChainId) + vaa.MustWrite(buf, binary.BigEndian, perChainResponse.Response.Type()) + respBuf, err := perChainResponse.Response.Marshal() + if err != nil { + return nil, err + } + + // Write the length of the response to facilitate on-chain parsing. + if len(respBuf) > math.MaxUint32 { + return nil, fmt.Errorf("response is too long") + } + vaa.MustWrite(buf, binary.BigEndian, uint32(len(respBuf))) + buf.Write(respBuf) + return buf.Bytes(), nil +} + +// Unmarshal deserializes the binary representation of a per chain query response from a byte array +func (perChainResponse *PerChainQueryResponse) Unmarshal(data []byte) error { + reader := bytes.NewReader(data[:]) + return perChainResponse.UnmarshalFromReader(reader) +} + +// UnmarshalFromReader deserializes the binary representation of a per chain query response from an existing reader +func (perChainResponse *PerChainQueryResponse) UnmarshalFromReader(reader *bytes.Reader) error { + if err := binary.Read(reader, binary.BigEndian, &perChainResponse.ChainId); err != nil { + return fmt.Errorf("failed to read response chain: %w", err) + } + + qt := uint8(0) + if err := binary.Read(reader, binary.BigEndian, &qt); err != nil { + return fmt.Errorf("failed to read response type: %w", err) + } + queryType := ChainSpecificQueryType(qt) + + if err := ValidatePerChainQueryRequestType(queryType); err != nil { + return err + } + + // Skip the response length. + var respLength uint32 + if err := binary.Read(reader, binary.BigEndian, &respLength); err != nil { + return fmt.Errorf("failed to read response length: %w", err) + } + + switch queryType { + case EthCallQueryRequestType: + r := EthCallQueryResponse{} + if err := r.UnmarshalFromReader(reader); err != nil { + return fmt.Errorf("failed to unmarshal eth call response: %w", err) + } + perChainResponse.Response = &r + default: + return fmt.Errorf("unsupported query type: %d", queryType) + } + + return nil +} + +// ValidatePerChainResponse performs basic validation on a per chain query response. +func (perChainResponse *PerChainQueryResponse) Validate() error { + str := perChainResponse.ChainId.String() + if _, err := vaa.ChainIDFromString(str); err != nil { + return fmt.Errorf("invalid chainID: %d", uint16(perChainResponse.ChainId)) + } + + if perChainResponse.Response == nil { + return fmt.Errorf("response is nil") + } + + if err := ValidatePerChainQueryRequestType(perChainResponse.Response.Type()); err != nil { + return err + } + + if err := perChainResponse.Response.Validate(); err != nil { + return fmt.Errorf("chain specific response is invalid: %w", err) + } + + return nil +} + +// Equal checks for equality on two per chain query responses. +func (left *PerChainQueryResponse) Equal(right *PerChainQueryResponse) bool { + if left.ChainId != right.ChainId { + return false + } + + if left.Response == nil && right.Response == nil { + return true + } + + if left.Response == nil || right.Response == nil { + return false + } + + if left.Response.Type() != right.Response.Type() { + return false + } + + switch leftEcq := left.Response.(type) { + case *EthCallQueryResponse: + switch rightEcd := right.Response.(type) { + case *EthCallQueryResponse: + return leftEcq.Equal(rightEcd) + default: + panic("unsupported query type on right") // We checked this above! + } + default: + panic("unsupported query type on left") // We checked this above! + } +} + +// +// Implementation of EthCallQueryResponse, which implements the ChainSpecificResponse for an EVM eth_call query response. +// + +func (e *EthCallQueryResponse) Type() ChainSpecificQueryType { + return EthCallQueryRequestType +} + +// Marshal serializes the binary representation of an EVM eth_call response. +// This method calls Validate() and relies on it to range checks lengths, etc. +func (ecr *EthCallQueryResponse) Marshal() ([]byte, error) { + if err := ecr.Validate(); err != nil { + return nil, err + } + + buf := new(bytes.Buffer) + vaa.MustWrite(buf, binary.BigEndian, ecr.BlockNumber) + buf.Write(ecr.Hash[:]) + vaa.MustWrite(buf, binary.BigEndian, ecr.Time.UnixMicro()) + + vaa.MustWrite(buf, binary.BigEndian, uint8(len(ecr.Results))) + for idx := range ecr.Results { + vaa.MustWrite(buf, binary.BigEndian, uint32(len(ecr.Results[idx]))) + buf.Write(ecr.Results[idx]) + } + + return buf.Bytes(), nil +} + +// Unmarshal deserializes an EVM eth_call response from a byte array +func (ecr *EthCallQueryResponse) Unmarshal(data []byte) error { + reader := bytes.NewReader(data[:]) + return ecr.UnmarshalFromReader(reader) +} + +// UnmarshalFromReader deserializes an EVM eth_call response from a byte array +func (ecr *EthCallQueryResponse) UnmarshalFromReader(reader *bytes.Reader) error { + if err := binary.Read(reader, binary.BigEndian, &ecr.BlockNumber); err != nil { + return fmt.Errorf("failed to read response number: %w", err) + } + + responseHash := common.Hash{} + if n, err := reader.Read(responseHash[:]); err != nil || n != 32 { + return fmt.Errorf("failed to read response hash [%d]: %w", n, err) + } + ecr.Hash = responseHash + + unixMicros := int64(0) + if err := binary.Read(reader, binary.BigEndian, &unixMicros); err != nil { + return fmt.Errorf("failed to read response timestamp: %w", err) + } + ecr.Time = time.UnixMicro(unixMicros) + + numResults := uint8(0) + if err := binary.Read(reader, binary.BigEndian, &numResults); err != nil { + return fmt.Errorf("failed to read number of results: %w", err) + } + + for count := 0; count < int(numResults); count++ { + resultLen := uint32(0) + if err := binary.Read(reader, binary.BigEndian, &resultLen); err != nil { + return fmt.Errorf("failed to read result len: %w", err) + } + result := make([]byte, resultLen) + if n, err := reader.Read(result[:]); err != nil || n != int(resultLen) { + return fmt.Errorf("failed to read result [%d]: %w", n, err) + } + + ecr.Results = append(ecr.Results, result) + } + + return nil +} + +// Validate does basic validation on an EVM eth_call response. +func (ecr *EthCallQueryResponse) Validate() error { + // Not checking for BlockNumber == 0, because maybe that could happen?? + + if len(ecr.Hash) != 32 { + return fmt.Errorf("invalid length for block hash") + } + + if len(ecr.Results) <= 0 { + return fmt.Errorf("does not contain any results") + } + if len(ecr.Results) > math.MaxUint8 { + return fmt.Errorf("too many results") + } + for _, result := range ecr.Results { + if len(result) > math.MaxUint32 { + return fmt.Errorf("result too long") + } + } + return nil +} + +// Equal verifies that two EVM eth_call responses are equal. +func (left *EthCallQueryResponse) Equal(right *EthCallQueryResponse) bool { + if left.BlockNumber != right.BlockNumber { + return false + } + + if !bytes.Equal(left.Hash.Bytes(), right.Hash.Bytes()) { + return false + } + + if len(left.Results) != len(right.Results) { + return false + } + for idx := range left.Results { + if !bytes.Equal(left.Results[idx], right.Results[idx]) { + return false + } + } + + return true +} diff --git a/node/pkg/watchers/algorand/config.go b/node/pkg/watchers/algorand/config.go index fff2a818b9..aa1f3355a9 100644 --- a/node/pkg/watchers/algorand/config.go +++ b/node/pkg/watchers/algorand/config.go @@ -3,6 +3,7 @@ package algorand import ( "github.com/certusone/wormhole/node/pkg/common" gossipv1 "github.com/certusone/wormhole/node/pkg/proto/gossip/v1" + "github.com/certusone/wormhole/node/pkg/query" "github.com/certusone/wormhole/node/pkg/supervisor" "github.com/certusone/wormhole/node/pkg/watchers" "github.com/certusone/wormhole/node/pkg/watchers/interfaces" @@ -38,6 +39,8 @@ func (wc *WatcherConfig) SetL1Finalizer(l1finalizer interfaces.L1Finalizer) { func (wc *WatcherConfig) Create( msgC chan<- *common.MessagePublication, obsvReqC <-chan *gossipv1.ObservationRequest, + _ <-chan *query.PerChainQueryInternal, + _ chan<- *query.PerChainQueryResponseInternal, _ chan<- *common.GuardianSet, env common.Environment, ) (interfaces.L1Finalizer, supervisor.Runnable, error) { diff --git a/node/pkg/watchers/aptos/config.go b/node/pkg/watchers/aptos/config.go index b01a51a664..632b8a46e3 100644 --- a/node/pkg/watchers/aptos/config.go +++ b/node/pkg/watchers/aptos/config.go @@ -3,6 +3,7 @@ package aptos import ( "github.com/certusone/wormhole/node/pkg/common" gossipv1 "github.com/certusone/wormhole/node/pkg/proto/gossip/v1" + "github.com/certusone/wormhole/node/pkg/query" "github.com/certusone/wormhole/node/pkg/supervisor" "github.com/certusone/wormhole/node/pkg/watchers" "github.com/certusone/wormhole/node/pkg/watchers/interfaces" @@ -36,6 +37,8 @@ func (wc *WatcherConfig) SetL1Finalizer(l1finalizer interfaces.L1Finalizer) { func (wc *WatcherConfig) Create( msgC chan<- *common.MessagePublication, obsvReqC <-chan *gossipv1.ObservationRequest, + _ <-chan *query.PerChainQueryInternal, + _ chan<- *query.PerChainQueryResponseInternal, _ chan<- *common.GuardianSet, env common.Environment, ) (interfaces.L1Finalizer, supervisor.Runnable, error) { diff --git a/node/pkg/watchers/cosmwasm/config.go b/node/pkg/watchers/cosmwasm/config.go index 4a5cb40929..b9837730dc 100644 --- a/node/pkg/watchers/cosmwasm/config.go +++ b/node/pkg/watchers/cosmwasm/config.go @@ -3,6 +3,7 @@ package cosmwasm import ( "github.com/certusone/wormhole/node/pkg/common" gossipv1 "github.com/certusone/wormhole/node/pkg/proto/gossip/v1" + "github.com/certusone/wormhole/node/pkg/query" "github.com/certusone/wormhole/node/pkg/supervisor" "github.com/certusone/wormhole/node/pkg/watchers" "github.com/certusone/wormhole/node/pkg/watchers/interfaces" @@ -36,6 +37,8 @@ func (wc *WatcherConfig) SetL1Finalizer(l1finalizer interfaces.L1Finalizer) { func (wc *WatcherConfig) Create( msgC chan<- *common.MessagePublication, obsvReqC <-chan *gossipv1.ObservationRequest, + _ <-chan *query.PerChainQueryInternal, + _ chan<- *query.PerChainQueryResponseInternal, _ chan<- *common.GuardianSet, env common.Environment, ) (interfaces.L1Finalizer, supervisor.Runnable, error) { diff --git a/node/pkg/watchers/evm/config.go b/node/pkg/watchers/evm/config.go index 3e777e87a1..3891f1d6f3 100644 --- a/node/pkg/watchers/evm/config.go +++ b/node/pkg/watchers/evm/config.go @@ -3,6 +3,7 @@ package evm import ( "github.com/certusone/wormhole/node/pkg/common" gossipv1 "github.com/certusone/wormhole/node/pkg/proto/gossip/v1" + "github.com/certusone/wormhole/node/pkg/query" "github.com/certusone/wormhole/node/pkg/supervisor" "github.com/certusone/wormhole/node/pkg/watchers" "github.com/certusone/wormhole/node/pkg/watchers/interfaces" @@ -42,6 +43,8 @@ func (wc *WatcherConfig) SetL1Finalizer(l1finalizer interfaces.L1Finalizer) { func (wc *WatcherConfig) Create( msgC chan<- *common.MessagePublication, obsvReqC <-chan *gossipv1.ObservationRequest, + queryReqC <-chan *query.PerChainQueryInternal, + queryResponseC chan<- *query.PerChainQueryResponseInternal, setC chan<- *common.GuardianSet, env common.Environment, ) (interfaces.L1Finalizer, supervisor.Runnable, error) { @@ -54,7 +57,7 @@ func (wc *WatcherConfig) Create( var devMode bool = (env == common.UnsafeDevNet) - watcher := NewEthWatcher(wc.Rpc, eth_common.HexToAddress(wc.Contract), string(wc.NetworkID), wc.ChainID, msgC, setWriteC, obsvReqC, devMode) + watcher := NewEthWatcher(wc.Rpc, eth_common.HexToAddress(wc.Contract), string(wc.NetworkID), wc.ChainID, msgC, setWriteC, obsvReqC, queryReqC, queryResponseC, devMode) watcher.SetWaitForConfirmations(wc.WaitForConfirmations) if err := watcher.SetRootChainParams(wc.RootChainRpc, wc.RootChainContract); err != nil { return nil, nil, err diff --git a/node/pkg/watchers/evm/connectors/celo.go b/node/pkg/watchers/evm/connectors/celo.go index 9e46881c15..d1b5d355df 100644 --- a/node/pkg/watchers/evm/connectors/celo.go +++ b/node/pkg/watchers/evm/connectors/celo.go @@ -17,6 +17,7 @@ import ( ethCommon "github.com/ethereum/go-ethereum/common" ethTypes "github.com/ethereum/go-ethereum/core/types" ethEvent "github.com/ethereum/go-ethereum/event" + ethRpc "github.com/ethereum/go-ethereum/rpc" "github.com/certusone/wormhole/node/pkg/common" "go.uber.org/zap" @@ -181,6 +182,19 @@ func (c *CeloConnector) RawCallContext(ctx context.Context, result interface{}, return c.rawClient.CallContext(ctx, result, method, args...) } +func (c *CeloConnector) RawBatchCallContext(ctx context.Context, b []ethRpc.BatchElem) error { + celoB := make([]celoRpc.BatchElem, len(b)) + for i, v := range b { + celoB[i] = celoRpc.BatchElem{ + Method: v.Method, + Args: v.Args, + Result: v.Result, + Error: v.Error, + } + } + return c.rawClient.BatchCallContext(ctx, celoB) +} + func convertCeloEventToEth(ev *celoAbi.AbiLogMessagePublished) *ethAbi.AbiLogMessagePublished { return ðAbi.AbiLogMessagePublished{ Sender: ethCommon.BytesToAddress(ev.Sender.Bytes()), diff --git a/node/pkg/watchers/evm/connectors/common.go b/node/pkg/watchers/evm/connectors/common.go index 187026ae0c..f177c8a539 100644 --- a/node/pkg/watchers/evm/connectors/common.go +++ b/node/pkg/watchers/evm/connectors/common.go @@ -10,10 +10,22 @@ import ( "github.com/ethereum/go-ethereum" "github.com/ethereum/go-ethereum/common" + "github.com/ethereum/go-ethereum/common/hexutil" "github.com/ethereum/go-ethereum/core/types" "github.com/ethereum/go-ethereum/event" + "github.com/ethereum/go-ethereum/rpc" ) +type BlockMarshaller struct { + Number *hexutil.Big + Hash common.Hash `json:"hash"` + Time hexutil.Uint64 `json:"timestamp"` + + // L1BlockNumber is the L1 block number in which an Arbitrum batch containing this block was submitted. + // This field is only populated when connecting to Arbitrum. + L1BlockNumber *hexutil.Big +} + type NewBlock struct { Number *big.Int Hash common.Hash @@ -33,6 +45,7 @@ type Connector interface { ParseLogMessagePublished(log types.Log) (*ethabi.AbiLogMessagePublished, error) SubscribeForBlocks(ctx context.Context, errC chan error, sink chan<- *NewBlock) (ethereum.Subscription, error) RawCallContext(ctx context.Context, result interface{}, method string, args ...interface{}) error + RawBatchCallContext(ctx context.Context, b []rpc.BatchElem) error } type PollSubscription struct { diff --git a/node/pkg/watchers/evm/connectors/ethereum.go b/node/pkg/watchers/evm/connectors/ethereum.go index 1c5c3bcf9d..e28d53ef10 100644 --- a/node/pkg/watchers/evm/connectors/ethereum.go +++ b/node/pkg/watchers/evm/connectors/ethereum.go @@ -133,7 +133,10 @@ func (e *EthereumConnector) SubscribeForBlocks(ctx context.Context, errC chan er func (e *EthereumConnector) RawCallContext(ctx context.Context, result interface{}, method string, args ...interface{}) error { return e.rawClient.CallContext(ctx, result, method, args...) +} +func (e *EthereumConnector) RawBatchCallContext(ctx context.Context, b []ethRpc.BatchElem) error { + return e.rawClient.BatchCallContext(ctx, b) } func (e *EthereumConnector) Client() *ethClient.Client { diff --git a/node/pkg/watchers/evm/connectors/poller.go b/node/pkg/watchers/evm/connectors/poller.go index b26c81441d..4e5506b7e7 100644 --- a/node/pkg/watchers/evm/connectors/poller.go +++ b/node/pkg/watchers/evm/connectors/poller.go @@ -11,7 +11,6 @@ import ( ethEvent "github.com/ethereum/go-ethereum/event" ethereum "github.com/ethereum/go-ethereum" - ethCommon "github.com/ethereum/go-ethereum/common" ethHexUtils "github.com/ethereum/go-ethereum/common/hexutil" "go.uber.org/zap" @@ -224,16 +223,7 @@ func getBlock(ctx context.Context, logger *zap.Logger, conn Connector, number *b numStr = "latest" } - type Marshaller struct { - Number *ethHexUtils.Big - Hash ethCommon.Hash `json:"hash"` - - // L1BlockNumber is the L1 block number in which an Arbitrum batch containing this block was submitted. - // This field is only populated when connecting to Arbitrum. - L1BlockNumber *ethHexUtils.Big - } - - var m Marshaller + var m BlockMarshaller err := conn.RawCallContext(ctx, &m, "eth_getBlockByNumber", numStr, false) if err != nil { logger.Error("failed to get block", diff --git a/node/pkg/watchers/evm/connectors/poller_test.go b/node/pkg/watchers/evm/connectors/poller_test.go index d3b520da68..ef2e1cc103 100644 --- a/node/pkg/watchers/evm/connectors/poller_test.go +++ b/node/pkg/watchers/evm/connectors/poller_test.go @@ -20,6 +20,7 @@ import ( ethTypes "github.com/ethereum/go-ethereum/core/types" ethClient "github.com/ethereum/go-ethereum/ethclient" ethEvent "github.com/ethereum/go-ethereum/event" + ethRpc "github.com/ethereum/go-ethereum/rpc" ) // mockConnectorForPoller implements the connector interface for testing purposes. @@ -107,6 +108,10 @@ func (e *mockConnectorForPoller) RawCallContext(ctx context.Context, result inte return } +func (e *mockConnectorForPoller) RawBatchCallContext(ctx context.Context, b []ethRpc.BatchElem) error { + panic("method not implemented by mockConnectorForPoller") +} + func (e *mockConnectorForPoller) setBlockNumber(blockNumber uint64) { e.mutex.Lock() e.blockNumber = blockNumber diff --git a/node/pkg/watchers/evm/finalizers/moonbeam_test.go b/node/pkg/watchers/evm/finalizers/moonbeam_test.go index 2f68b3f50b..554a71f28d 100644 --- a/node/pkg/watchers/evm/finalizers/moonbeam_test.go +++ b/node/pkg/watchers/evm/finalizers/moonbeam_test.go @@ -15,6 +15,7 @@ import ( ethCommon "github.com/ethereum/go-ethereum/common" ethTypes "github.com/ethereum/go-ethereum/core/types" ethEvent "github.com/ethereum/go-ethereum/event" + ethRpc "github.com/ethereum/go-ethereum/rpc" "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" @@ -36,6 +37,10 @@ func (e *moonbeamMockConnector) RawCallContext(ctx context.Context, result inter return } +func (e *moonbeamMockConnector) RawBatchCallContext(ctx context.Context, b []ethRpc.BatchElem) error { + panic("method not implemented by moonbeamMockConnector") +} + func (e *moonbeamMockConnector) NetworkName() string { return "moonbeamMockConnector" } diff --git a/node/pkg/watchers/evm/watcher.go b/node/pkg/watchers/evm/watcher.go index 0cbc15996f..f40ad62f66 100644 --- a/node/pkg/watchers/evm/watcher.go +++ b/node/pkg/watchers/evm/watcher.go @@ -3,6 +3,8 @@ package evm import ( "context" "fmt" + "math" + "math/big" "sync" "sync/atomic" "time" @@ -24,6 +26,7 @@ import ( "go.uber.org/zap" "github.com/certusone/wormhole/node/pkg/common" + "github.com/certusone/wormhole/node/pkg/query" "github.com/certusone/wormhole/node/pkg/readiness" "github.com/certusone/wormhole/node/pkg/supervisor" "github.com/wormhole-foundation/wormhole/sdk/vaa" @@ -94,6 +97,13 @@ type ( // include requests for our chainID. obsvReqC <-chan *gossipv1.ObservationRequest + // Incoming query requests from the network. Pre-filtered to only + // include requests for our chainID. + queryReqC <-chan *query.PerChainQueryInternal + + // Outbound query responses to query requests + queryResponseC chan<- *query.PerChainQueryResponseInternal + pending map[pendingKey]*pendingMessage pendingMu sync.Mutex @@ -143,6 +153,8 @@ func NewEthWatcher( msgC chan<- *common.MessagePublication, setC chan<- *common.GuardianSet, obsvReqC <-chan *gossipv1.ObservationRequest, + queryReqC <-chan *query.PerChainQueryInternal, + queryResponseC chan<- *query.PerChainQueryResponseInternal, unsafeDevMode bool, ) *Watcher { @@ -157,6 +169,8 @@ func NewEthWatcher( msgC: msgC, setC: setC, obsvReqC: obsvReqC, + queryReqC: queryReqC, + queryResponseC: queryResponseC, pending: map[pendingKey]*pendingMessage{}, unsafeDevMode: unsafeDevMode, } @@ -515,6 +529,228 @@ func (w *Watcher) Run(parentCtx context.Context) error { } }) + common.RunWithScissors(ctx, errC, "evm_fetch_query_req", func(ctx context.Context) error { + ccqMaxBlockNumber := big.NewInt(0).SetUint64(math.MaxUint64) + for { + select { + case <-ctx.Done(): + return nil + case queryRequest := <-w.queryReqC: + // This can't happen unless there is a programming error - the caller + // is expected to send us only requests for our chainID. + if queryRequest.Request.ChainId != w.chainID { + panic("ccqevm: invalid chain ID") + } + + switch req := queryRequest.Request.Query.(type) { + case *query.EthCallQueryRequest: + block := req.BlockId + logger.Info("received query request", + zap.String("eth_network", w.networkName), + zap.String("block", block), + zap.Int("numRequests", len(req.CallData)), + zap.String("component", "ccqevm"), + ) + + timeout, cancel := context.WithTimeout(ctx, 5*time.Second) + // like https://github.com/ethereum/go-ethereum/blob/master/ethclient/ethclient.go#L610 + + var blockMethod string + var callBlockArg interface{} + // TODO: try making these error and see what happens + // 1. 66 chars but not 0x hex + // 2. 64 chars but not hex + // 3. bad blocks + // 4. bad 0x lengths + // 5. strings that aren't "latest", "safe", "finalized" + // 6. "safe" on a chain that doesn't support safe + // etc? + // I would expect this to trip within this scissor (if at all) but maybe this should get more defensive + if len(block) == 66 || len(block) == 64 { + blockMethod = "eth_getBlockByHash" + // looks like a hash which requires the object parameter + // https://eips.ethereum.org/EIPS/eip-1898 + // https://docs.alchemy.com/reference/eth-call + hash := eth_common.HexToHash(block) + callBlockArg = rpc.BlockNumberOrHash{ + BlockHash: &hash, + RequireCanonical: true, + } + } else { + blockMethod = "eth_getBlockByNumber" + callBlockArg = block + } + + // EvmCallData contains the details of a single query in the batch. + type EvmCallData struct { + to eth_common.Address + data string + callTransactionArg map[string]interface{} + callResult *eth_hexutil.Bytes + callErr error + } + + // We build two slices. The first is the batch submitted to the RPC call. It contains one entry for each query plus one to query the block. + // The second is the data associated with each request (but not the block request). The index into both is the index into the request call data. + batch := []rpc.BatchElem{} + evmCallData := []EvmCallData{} + + // Add each requested query to the batch. + for _, callData := range req.CallData { + // like https://github.com/ethereum/go-ethereum/blob/master/ethclient/ethclient.go#L610 + to := eth_common.BytesToAddress(callData.To) + data := eth_hexutil.Encode(callData.Data) + ecd := EvmCallData{ + to: to, + data: data, + callTransactionArg: map[string]interface{}{ + "to": to, + "data": data, + }, + callResult: ð_hexutil.Bytes{}, + } + evmCallData = append(evmCallData, ecd) + + batch = append(batch, rpc.BatchElem{ + Method: "eth_call", + Args: []interface{}{ + ecd.callTransactionArg, + callBlockArg, + }, + Result: ecd.callResult, + Error: ecd.callErr, + }) + } + + // Add the block query to the batch. + var blockResult connectors.BlockMarshaller + var blockError error + batch = append(batch, rpc.BatchElem{ + Method: blockMethod, + Args: []interface{}{ + block, + false, // no full transaction details + }, + Result: &blockResult, + Error: blockError, + }) + + // Query the RPC. + err := w.ethConn.RawBatchCallContext(timeout, batch) + cancel() + + if err != nil { + logger.Error("failed to process query request", + zap.Error(err), zap.String("eth_network", w.networkName), + zap.String("block", block), + zap.Any("batch", batch), + zap.String("component", "ccqevm"), + ) + w.ccqSendQueryResponse(logger, queryRequest, query.QueryRetryNeeded, nil) + continue + } + + if blockError != nil { + logger.Error("failed to process query block request", + zap.Error(blockError), zap.String("eth_network", w.networkName), + zap.String("block", block), + zap.Any("batch", batch), + zap.String("component", "ccqevm"), + ) + w.ccqSendQueryResponse(logger, queryRequest, query.QueryRetryNeeded, nil) + continue + } + + if blockResult.Number == nil { + logger.Error("invalid query block result", + zap.String("eth_network", w.networkName), + zap.String("block", block), + zap.Any("batch", batch), + zap.String("component", "ccqevm"), + ) + w.ccqSendQueryResponse(logger, queryRequest, query.QueryRetryNeeded, nil) + continue + } + + if blockResult.Number.ToInt().Cmp(ccqMaxBlockNumber) > 0 { + logger.Error("block number too large", + zap.String("eth_network", w.networkName), + zap.String("block", block), + zap.Any("batch", batch), + zap.String("component", "ccqevm"), + ) + w.ccqSendQueryResponse(logger, queryRequest, query.QueryRetryNeeded, nil) + continue + } + + resp := query.EthCallQueryResponse{ + BlockNumber: blockResult.Number.ToInt().Uint64(), + Hash: blockResult.Hash, + Time: time.Unix(int64(blockResult.Time), 0), + Results: [][]byte{}, + } + + errFound := false + for idx := range req.CallData { + if evmCallData[idx].callErr != nil { + logger.Error("failed to process query call request", + zap.Error(evmCallData[idx].callErr), zap.String("eth_network", w.networkName), + zap.String("block", block), + zap.Int("errorIdx", idx), + zap.Any("batch", batch), + zap.String("component", "ccqevm"), + ) + w.ccqSendQueryResponse(logger, queryRequest, query.QueryRetryNeeded, nil) + errFound = true + break + } + + // Nil or Empty results are not valid + // eth_call will return empty when the state doesn't exist for a block + if len(*evmCallData[idx].callResult) == 0 { + logger.Error("invalid call result", + zap.String("eth_network", w.networkName), + zap.String("block", block), + zap.Int("errorIdx", idx), + zap.Any("batch", batch), + zap.String("component", "ccqevm"), + ) + w.ccqSendQueryResponse(logger, queryRequest, query.QueryRetryNeeded, nil) + errFound = true + break + } + + logger.Info("query result", + zap.String("eth_network", w.networkName), + zap.String("block", block), + zap.String("blockNumber", blockResult.Number.String()), + zap.String("blockHash", blockResult.Hash.Hex()), + zap.String("blockTime", blockResult.Time.String()), + zap.Int("idx", idx), + zap.String("to", evmCallData[idx].to.Hex()), + zap.Any("data", evmCallData[idx].data), + zap.String("result", evmCallData[idx].callResult.String()), + zap.String("component", "ccqevm"), + ) + + resp.Results = append(resp.Results, *evmCallData[idx].callResult) + } + + if !errFound { + w.ccqSendQueryResponse(logger, queryRequest, query.QuerySuccess, &resp) + } + + default: + logger.Warn("received unsupported request type", + zap.Uint8("payload", uint8(queryRequest.Request.Query.Type())), + zap.String("component", "ccqevm"), + ) + w.ccqSendQueryResponse(logger, queryRequest, query.QueryFatalError, nil) + } + } + } + }) + common.RunWithScissors(ctx, errC, "evm_fetch_messages", func(ctx context.Context) error { for { select { @@ -930,3 +1166,14 @@ func (w *Watcher) SetWaitForConfirmations(waitForConfirmations bool) { func (w *Watcher) SetMaxWaitConfirmations(maxWaitConfirmations uint64) { w.maxWaitConfirmations = maxWaitConfirmations } + +// ccqSendQueryResponse sends an error response back to the query handler. +func (w *Watcher) ccqSendQueryResponse(logger *zap.Logger, req *query.PerChainQueryInternal, status query.QueryStatus, resp *query.EthCallQueryResponse) { + queryResponse := query.CreatePerChainQueryResponseInternal(req.RequestID, req.RequestIdx, req.Request.ChainId, status, resp) + select { + case w.queryResponseC <- queryResponse: + logger.Debug("published query response error to handler", zap.String("component", "ccqevm")) + default: + logger.Error("failed to published query response error to handler", zap.String("component", "ccqevm")) + } +} diff --git a/node/pkg/watchers/mock/config.go b/node/pkg/watchers/mock/config.go index 8d4a34897b..cc352f5686 100644 --- a/node/pkg/watchers/mock/config.go +++ b/node/pkg/watchers/mock/config.go @@ -3,6 +3,7 @@ package mock import ( "github.com/certusone/wormhole/node/pkg/common" gossipv1 "github.com/certusone/wormhole/node/pkg/proto/gossip/v1" + "github.com/certusone/wormhole/node/pkg/query" "github.com/certusone/wormhole/node/pkg/supervisor" "github.com/certusone/wormhole/node/pkg/watchers" "github.com/certusone/wormhole/node/pkg/watchers/interfaces" @@ -42,6 +43,8 @@ func (wc *WatcherConfig) SetL1Finalizer(l1finalizer interfaces.L1Finalizer) { func (wc *WatcherConfig) Create( msgC chan<- *common.MessagePublication, obsvReqC <-chan *gossipv1.ObservationRequest, + _ <-chan *query.PerChainQueryInternal, + _ chan<- *query.PerChainQueryResponseInternal, setC chan<- *common.GuardianSet, env common.Environment, ) (interfaces.L1Finalizer, supervisor.Runnable, error) { diff --git a/node/pkg/watchers/near/config.go b/node/pkg/watchers/near/config.go index 879016294c..ce5d36a138 100644 --- a/node/pkg/watchers/near/config.go +++ b/node/pkg/watchers/near/config.go @@ -3,6 +3,7 @@ package near import ( "github.com/certusone/wormhole/node/pkg/common" gossipv1 "github.com/certusone/wormhole/node/pkg/proto/gossip/v1" + "github.com/certusone/wormhole/node/pkg/query" "github.com/certusone/wormhole/node/pkg/supervisor" "github.com/certusone/wormhole/node/pkg/watchers" "github.com/certusone/wormhole/node/pkg/watchers/interfaces" @@ -35,6 +36,8 @@ func (wc *WatcherConfig) SetL1Finalizer(l1finalizer interfaces.L1Finalizer) { func (wc *WatcherConfig) Create( msgC chan<- *common.MessagePublication, obsvReqC <-chan *gossipv1.ObservationRequest, + _ <-chan *query.PerChainQueryInternal, + _ chan<- *query.PerChainQueryResponseInternal, _ chan<- *common.GuardianSet, env common.Environment, ) (interfaces.L1Finalizer, supervisor.Runnable, error) { diff --git a/node/pkg/watchers/solana/config.go b/node/pkg/watchers/solana/config.go index c506a6a329..a2263874a4 100644 --- a/node/pkg/watchers/solana/config.go +++ b/node/pkg/watchers/solana/config.go @@ -3,6 +3,7 @@ package solana import ( "github.com/certusone/wormhole/node/pkg/common" gossipv1 "github.com/certusone/wormhole/node/pkg/proto/gossip/v1" + "github.com/certusone/wormhole/node/pkg/query" "github.com/certusone/wormhole/node/pkg/supervisor" "github.com/certusone/wormhole/node/pkg/watchers" "github.com/certusone/wormhole/node/pkg/watchers/interfaces" @@ -40,6 +41,8 @@ func (wc *WatcherConfig) SetL1Finalizer(l1finalizer interfaces.L1Finalizer) { func (wc *WatcherConfig) Create( msgC chan<- *common.MessagePublication, obsvReqC <-chan *gossipv1.ObservationRequest, + _ <-chan *query.PerChainQueryInternal, + _ chan<- *query.PerChainQueryResponseInternal, _ chan<- *common.GuardianSet, env common.Environment, ) (interfaces.L1Finalizer, supervisor.Runnable, error) { diff --git a/node/pkg/watchers/sui/config.go b/node/pkg/watchers/sui/config.go index fea2bf4108..eb99c6d84f 100644 --- a/node/pkg/watchers/sui/config.go +++ b/node/pkg/watchers/sui/config.go @@ -3,6 +3,7 @@ package sui import ( "github.com/certusone/wormhole/node/pkg/common" gossipv1 "github.com/certusone/wormhole/node/pkg/proto/gossip/v1" + "github.com/certusone/wormhole/node/pkg/query" "github.com/certusone/wormhole/node/pkg/supervisor" "github.com/certusone/wormhole/node/pkg/watchers" "github.com/certusone/wormhole/node/pkg/watchers/interfaces" @@ -36,6 +37,8 @@ func (wc *WatcherConfig) SetL1Finalizer(l1finalizer interfaces.L1Finalizer) { func (wc *WatcherConfig) Create( msgC chan<- *common.MessagePublication, obsvReqC <-chan *gossipv1.ObservationRequest, + _ <-chan *query.PerChainQueryInternal, + _ chan<- *query.PerChainQueryResponseInternal, _ chan<- *common.GuardianSet, env common.Environment, ) (interfaces.L1Finalizer, supervisor.Runnable, error) { diff --git a/node/pkg/watchers/watchers.go b/node/pkg/watchers/watchers.go index ac60038c59..047e228a9c 100644 --- a/node/pkg/watchers/watchers.go +++ b/node/pkg/watchers/watchers.go @@ -3,6 +3,7 @@ package watchers import ( "github.com/certusone/wormhole/node/pkg/common" gossipv1 "github.com/certusone/wormhole/node/pkg/proto/gossip/v1" + "github.com/certusone/wormhole/node/pkg/query" "github.com/certusone/wormhole/node/pkg/supervisor" "github.com/certusone/wormhole/node/pkg/watchers/interfaces" "github.com/wormhole-foundation/wormhole/sdk/vaa" @@ -20,6 +21,8 @@ type WatcherConfig interface { Create( msgC chan<- *common.MessagePublication, obsvReqC <-chan *gossipv1.ObservationRequest, + queryReqC <-chan *query.PerChainQueryInternal, + queryResponseC chan<- *query.PerChainQueryResponseInternal, setC chan<- *common.GuardianSet, env common.Environment, ) (interfaces.L1Finalizer, supervisor.Runnable, error) diff --git a/proto/gossip/v1/gossip.proto b/proto/gossip/v1/gossip.proto index 19d8a4ac83..04685a831d 100644 --- a/proto/gossip/v1/gossip.proto +++ b/proto/gossip/v1/gossip.proto @@ -14,6 +14,8 @@ message GossipMessage { SignedBatchVAAWithQuorum signed_batch_vaa_with_quorum = 7; SignedChainGovernorConfig signed_chain_governor_config = 8; SignedChainGovernorStatus signed_chain_governor_status = 9; + SignedQueryRequest signed_query_request = 10; + SignedQueryResponse signed_query_response = 11; } } @@ -231,3 +233,19 @@ message ChainGovernorStatus { int64 timestamp = 3; repeated Chain chains = 4; } + +message SignedQueryRequest { + // Serialized QueryRequest message. + bytes query_request = 1; + + // ECDSA signature using the requestor's public key. + bytes signature = 2; +} + +message SignedQueryResponse { + // Serialized QueryResponse message. + bytes query_response = 1; + + // ECDSA signature using the node's guardian public key. + bytes signature = 2; +}