From 4be015b9696ed6cbc4a0597c9f2ed705cc3f83d3 Mon Sep 17 00:00:00 2001 From: Stephen Buttolph Date: Fri, 8 Mar 2024 10:55:34 -0500 Subject: [PATCH] Combine AppGossip and AppGossipSpecific (#2836) --- network/network.go | 115 ++++--- network/network_test.go | 29 +- network/p2p/client.go | 21 +- network/p2p/gossip/gossip.go | 9 +- network/p2p/network_test.go | 22 +- proto/appsender/appsender.proto | 16 +- proto/pb/appsender/appsender.pb.go | 306 +++++++----------- proto/pb/appsender/appsender_grpc.pb.go | 37 --- .../common/appsender/appsender_client.go | 34 +- .../common/appsender/appsender_server.go | 22 +- snow/engine/common/mock_sender.go | 22 +- snow/engine/common/sender.go | 13 +- snow/engine/common/test_sender.go | 47 +-- snow/networking/sender/external_sender.go | 15 +- .../networking/sender/mock_external_sender.go | 23 +- snow/networking/sender/sender.go | 125 ++++--- snow/networking/sender/sender_test.go | 74 +++-- .../networking/sender/test_external_sender.go | 39 +-- snow/networking/sender/traced_sender.go | 24 +- vms/avm/network/network_test.go | 4 +- vms/platformvm/block/builder/helpers_test.go | 2 +- vms/platformvm/network/network_test.go | 2 +- vms/platformvm/validator_set_property_test.go | 2 +- vms/platformvm/vm_test.go | 22 +- 24 files changed, 403 insertions(+), 622 deletions(-) diff --git a/network/network.go b/network/network.go index c487d8244b6..a6ac6fdd28b 100644 --- a/network/network.go +++ b/network/network.go @@ -25,6 +25,7 @@ import ( "github.com/ava-labs/avalanchego/network/dialer" "github.com/ava-labs/avalanchego/network/peer" "github.com/ava-labs/avalanchego/network/throttling" + "github.com/ava-labs/avalanchego/snow/engine/common" "github.com/ava-labs/avalanchego/snow/networking/router" "github.com/ava-labs/avalanchego/snow/networking/sender" "github.com/ava-labs/avalanchego/subnets" @@ -47,8 +48,7 @@ const ( ) var ( - _ sender.ExternalSender = (*network)(nil) - _ Network = (*network)(nil) + _ Network = (*network)(nil) errNotValidator = errors.New("node is not a validator") errNotTracked = errors.New("subnet is not tracked") @@ -310,25 +310,42 @@ func NewNetwork( return n, nil } -func (n *network) Send(msg message.OutboundMessage, nodeIDs set.Set[ids.NodeID], subnetID ids.ID, allower subnets.Allower) set.Set[ids.NodeID] { - peers := n.getPeers(nodeIDs, subnetID, allower) - n.peerConfig.Metrics.MultipleSendsFailed( - msg.Op(), - nodeIDs.Len()-len(peers), - ) - return n.send(msg, peers) -} - -func (n *network) Gossip( +func (n *network) Send( msg message.OutboundMessage, + config common.SendConfig, subnetID ids.ID, - numValidatorsToSend int, - numNonValidatorsToSend int, - numPeersToSend int, allower subnets.Allower, ) set.Set[ids.NodeID] { - peers := n.samplePeers(subnetID, numValidatorsToSend, numNonValidatorsToSend, numPeersToSend, allower) - return n.send(msg, peers) + namedPeers := n.getPeers(config.NodeIDs, subnetID, allower) + n.peerConfig.Metrics.MultipleSendsFailed( + msg.Op(), + config.NodeIDs.Len()-len(namedPeers), + ) + + var ( + sampledPeers = n.samplePeers(config, subnetID, allower) + sentTo = set.NewSet[ids.NodeID](len(namedPeers) + len(sampledPeers)) + now = n.peerConfig.Clock.Time() + ) + + // send to peers and update metrics + // + // Note: It is guaranteed that namedPeers and sampledPeers are disjoint. + for _, peers := range [][]peer.Peer{namedPeers, sampledPeers} { + for _, peer := range peers { + if peer.Send(n.onCloseCtx, msg) { + sentTo.Add(peer.ID()) + + // TODO: move send fail rate calculations into the peer metrics + // record metrics for success + n.sendFailRateCalculator.Observe(0, now) + } else { + // record metrics for failure + n.sendFailRateCalculator.Observe(1, now) + } + } + } + return sentTo } // HealthCheck returns information about several network layer health checks. @@ -695,25 +712,24 @@ func (n *network) getPeers( return peers } +// samplePeers samples connected peers attempting to align with the number of +// requested validators, non-validators, and peers. This function will +// explicitly ignore nodeIDs already included in the send config. func (n *network) samplePeers( + config common.SendConfig, subnetID ids.ID, - numValidatorsToSample, - numNonValidatorsToSample int, - numPeersToSample int, allower subnets.Allower, ) []peer.Peer { - // If there are fewer validators than [numValidatorsToSample], then only - // sample [numValidatorsToSample] validators. - subnetValidatorsLen := n.config.Validators.Count(subnetID) - if subnetValidatorsLen < numValidatorsToSample { - numValidatorsToSample = subnetValidatorsLen - } + // As an optimization, if there are fewer validators than + // [numValidatorsToSample], only attempt to sample [numValidatorsToSample] + // validators to potentially avoid iterating over the entire peer set. + numValidatorsToSample := min(config.Validators, n.config.Validators.Count(subnetID)) n.peersLock.RLock() defer n.peersLock.RUnlock() return n.connectedPeers.Sample( - numValidatorsToSample+numNonValidatorsToSample+numPeersToSample, + numValidatorsToSample+config.NonValidators+config.Peers, func(p peer.Peer) bool { // Only return peers that are tracking [subnetID] trackedSubnets := p.TrackedSubnets() @@ -722,14 +738,20 @@ func (n *network) samplePeers( } peerID := p.ID() + // if the peer was already explicitly included, don't include in the + // sample + if config.NodeIDs.Contains(peerID) { + return false + } + _, isValidator := n.config.Validators.GetValidator(subnetID, peerID) // check if the peer is allowed to connect to the subnet if !allower.IsAllowed(peerID, isValidator) { return false } - if numPeersToSample > 0 { - numPeersToSample-- + if config.Peers > 0 { + config.Peers-- return true } @@ -738,37 +760,12 @@ func (n *network) samplePeers( return numValidatorsToSample >= 0 } - numNonValidatorsToSample-- - return numNonValidatorsToSample >= 0 + config.NonValidators-- + return config.NonValidators >= 0 }, ) } -// send the message to the provided peers. -// -// send takes ownership of the provided message reference. So, the provided -// message should only be inspected if the reference has been externally -// increased. -func (n *network) send(msg message.OutboundMessage, peers []peer.Peer) set.Set[ids.NodeID] { - sentTo := set.NewSet[ids.NodeID](len(peers)) - now := n.peerConfig.Clock.Time() - - // send to peer and update metrics - for _, peer := range peers { - if peer.Send(n.onCloseCtx, msg) { - sentTo.Add(peer.ID()) - - // TODO: move send fail rate calculations into the peer metrics - // record metrics for success - n.sendFailRateCalculator.Observe(0, now) - } else { - // record metrics for failure - n.sendFailRateCalculator.Observe(1, now) - } - } - return sentTo -} - func (n *network) disconnectedFromConnecting(nodeID ids.NodeID) { n.peersLock.Lock() defer n.peersLock.Unlock() @@ -1208,10 +1205,10 @@ func (n *network) runTimers() { // pullGossipPeerLists requests validators from peers in the network func (n *network) pullGossipPeerLists() { peers := n.samplePeers( + common.SendConfig{ + Validators: 1, + }, constants.PrimaryNetworkID, - 1, // numValidatorsToSample - 0, // numNonValidatorsToSample - 0, // numPeersToSample subnets.NoOpAllower, ) diff --git a/network/network_test.go b/network/network_test.go index f48b904fa1b..fb70f2966b2 100644 --- a/network/network_test.go +++ b/network/network_test.go @@ -21,6 +21,7 @@ import ( "github.com/ava-labs/avalanchego/network/peer" "github.com/ava-labs/avalanchego/network/throttling" "github.com/ava-labs/avalanchego/proto/pb/p2p" + "github.com/ava-labs/avalanchego/snow/engine/common" "github.com/ava-labs/avalanchego/snow/networking/router" "github.com/ava-labs/avalanchego/snow/networking/tracker" "github.com/ava-labs/avalanchego/snow/uptime" @@ -327,7 +328,14 @@ func TestSend(t *testing.T) { require.NoError(err) toSend := set.Of(nodeIDs[1]) - sentTo := net0.Send(outboundGetMsg, toSend, constants.PrimaryNetworkID, subnets.NoOpAllower) + sentTo := net0.Send( + outboundGetMsg, + common.SendConfig{ + NodeIDs: toSend, + }, + constants.PrimaryNetworkID, + subnets.NoOpAllower, + ) require.Equal(toSend, sentTo) inboundGetMsg := <-received @@ -339,7 +347,7 @@ func TestSend(t *testing.T) { wg.Wait() } -func TestSendAndGossipWithFilter(t *testing.T) { +func TestSendWithFilter(t *testing.T) { require := require.New(t) received := make(chan message.InboundMessage) @@ -366,21 +374,20 @@ func TestSendAndGossipWithFilter(t *testing.T) { toSend := set.Of(nodeIDs...) validNodeID := nodeIDs[1] - sentTo := net0.Send(outboundGetMsg, toSend, constants.PrimaryNetworkID, newNodeIDConnector(validNodeID)) + sentTo := net0.Send( + outboundGetMsg, + common.SendConfig{ + NodeIDs: toSend, + }, + constants.PrimaryNetworkID, + newNodeIDConnector(validNodeID), + ) require.Len(sentTo, 1) require.Contains(sentTo, validNodeID) inboundGetMsg := <-received require.Equal(message.GetOp, inboundGetMsg.Op()) - // Test Gossip now - sentTo = net0.Gossip(outboundGetMsg, constants.PrimaryNetworkID, 0, 0, len(nodeIDs), newNodeIDConnector(validNodeID)) - require.Len(sentTo, 1) - require.Contains(sentTo, validNodeID) - - inboundGetMsg = <-received - require.Equal(message.GetOp, inboundGetMsg.Op()) - for _, net := range networks { net.StartClose() } diff --git a/network/p2p/client.go b/network/p2p/client.go index 3d14e5684ed..80d0118513e 100644 --- a/network/p2p/client.go +++ b/network/p2p/client.go @@ -108,29 +108,12 @@ func (c *Client) AppRequest( // AppGossip sends a gossip message to a random set of peers. func (c *Client) AppGossip( ctx context.Context, + config common.SendConfig, appGossipBytes []byte, - numValidators int, - numNonValidators int, - numPeers int, ) error { return c.sender.SendAppGossip( ctx, - PrefixMessage(c.handlerPrefix, appGossipBytes), - numValidators, - numNonValidators, - numPeers, - ) -} - -// AppGossipSpecific sends a gossip message to a predetermined set of peers. -func (c *Client) AppGossipSpecific( - ctx context.Context, - nodeIDs set.Set[ids.NodeID], - appGossipBytes []byte, -) error { - return c.sender.SendAppGossipSpecific( - ctx, - nodeIDs, + config, PrefixMessage(c.handlerPrefix, appGossipBytes), ) } diff --git a/network/p2p/gossip/gossip.go b/network/p2p/gossip/gossip.go index 32d77a32bab..c6950f9871f 100644 --- a/network/p2p/gossip/gossip.go +++ b/network/p2p/gossip/gossip.go @@ -16,6 +16,7 @@ import ( "github.com/ava-labs/avalanchego/cache" "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/network/p2p" + "github.com/ava-labs/avalanchego/snow/engine/common" "github.com/ava-labs/avalanchego/utils" "github.com/ava-labs/avalanchego/utils/bloom" "github.com/ava-labs/avalanchego/utils/buffer" @@ -462,10 +463,12 @@ func (p *PushGossiper[T]) gossip( return p.client.AppGossip( ctx, + common.SendConfig{ + Validators: gossipParams.Validators, + NonValidators: gossipParams.NonValidators, + Peers: gossipParams.Peers, + }, msgBytes, - gossipParams.Validators, - gossipParams.NonValidators, - gossipParams.Peers, ) } diff --git a/network/p2p/network_test.go b/network/p2p/network_test.go index 9d5a4cacd0d..ff826d65577 100644 --- a/network/p2p/network_test.go +++ b/network/p2p/network_test.go @@ -68,7 +68,13 @@ func TestMessageRouting(t *testing.T) { require.NoError(network.AddHandler(1, testHandler)) client := network.NewClient(1) - require.NoError(client.AppGossip(ctx, wantMsg, 0, 0, 1)) + require.NoError(client.AppGossip( + ctx, + common.SendConfig{ + Peers: 1, + }, + wantMsg, + )) require.NoError(network.AppGossip(ctx, wantNodeID, <-sender.SentAppGossip)) require.True(appGossipCalled) @@ -89,7 +95,6 @@ func TestClientPrefixesMessages(t *testing.T) { sender := common.FakeSender{ SentAppRequest: make(chan []byte, 1), SentAppGossip: make(chan []byte, 1), - SentAppGossipSpecific: make(chan []byte, 1), SentCrossChainAppRequest: make(chan []byte, 1), } @@ -129,15 +134,16 @@ func TestClientPrefixesMessages(t *testing.T) { require.Equal(handlerPrefix, gotCrossChainAppRequest[0]) require.Equal(want, gotCrossChainAppRequest[1:]) - require.NoError(client.AppGossip(ctx, want, 0, 0, 1)) + require.NoError(client.AppGossip( + ctx, + common.SendConfig{ + Peers: 1, + }, + want, + )) gotAppGossip := <-sender.SentAppGossip require.Equal(handlerPrefix, gotAppGossip[0]) require.Equal(want, gotAppGossip[1:]) - - require.NoError(client.AppGossipSpecific(ctx, set.Of(ids.EmptyNodeID), want)) - gotAppGossip = <-sender.SentAppGossipSpecific - require.Equal(handlerPrefix, gotAppGossip[0]) - require.Equal(want, gotAppGossip[1:]) } // Tests that the Client callback is called on a successful response diff --git a/proto/appsender/appsender.proto b/proto/appsender/appsender.proto index 93fdee7b073..5a03d449359 100644 --- a/proto/appsender/appsender.proto +++ b/proto/appsender/appsender.proto @@ -11,7 +11,6 @@ service AppSender { rpc SendAppResponse(SendAppResponseMsg) returns (google.protobuf.Empty); rpc SendAppError(SendAppErrorMsg) returns (google.protobuf.Empty); rpc SendAppGossip(SendAppGossipMsg) returns (google.protobuf.Empty); - rpc SendAppGossipSpecific(SendAppGossipSpecificMsg) returns (google.protobuf.Empty); rpc SendCrossChainAppRequest(SendCrossChainAppRequestMsg) returns (google.protobuf.Empty); rpc SendCrossChainAppResponse(SendCrossChainAppResponseMsg) returns (google.protobuf.Empty); @@ -48,18 +47,13 @@ message SendAppErrorMsg { } message SendAppGossipMsg { - // The message body - bytes msg = 1; - uint64 num_validators = 2; - uint64 num_non_validators = 3; - uint64 num_peers = 4; -} - -message SendAppGossipSpecificMsg { - // The nodes to send this request to + // Who to send this message to repeated bytes node_ids = 1; + uint64 validators = 2; + uint64 non_validators = 3; + uint64 peers = 4; // The message body - bytes msg = 2; + bytes msg = 5; } message SendCrossChainAppRequestMsg { diff --git a/proto/pb/appsender/appsender.pb.go b/proto/pb/appsender/appsender.pb.go index a3437f287cd..16ccb35821e 100644 --- a/proto/pb/appsender/appsender.pb.go +++ b/proto/pb/appsender/appsender.pb.go @@ -233,11 +233,13 @@ type SendAppGossipMsg struct { sizeCache protoimpl.SizeCache unknownFields protoimpl.UnknownFields + // Who to send this message to + NodeIds [][]byte `protobuf:"bytes,1,rep,name=node_ids,json=nodeIds,proto3" json:"node_ids,omitempty"` + Validators uint64 `protobuf:"varint,2,opt,name=validators,proto3" json:"validators,omitempty"` + NonValidators uint64 `protobuf:"varint,3,opt,name=non_validators,json=nonValidators,proto3" json:"non_validators,omitempty"` + Peers uint64 `protobuf:"varint,4,opt,name=peers,proto3" json:"peers,omitempty"` // The message body - Msg []byte `protobuf:"bytes,1,opt,name=msg,proto3" json:"msg,omitempty"` - NumValidators uint64 `protobuf:"varint,2,opt,name=num_validators,json=numValidators,proto3" json:"num_validators,omitempty"` - NumNonValidators uint64 `protobuf:"varint,3,opt,name=num_non_validators,json=numNonValidators,proto3" json:"num_non_validators,omitempty"` - NumPeers uint64 `protobuf:"varint,4,opt,name=num_peers,json=numPeers,proto3" json:"num_peers,omitempty"` + Msg []byte `protobuf:"bytes,5,opt,name=msg,proto3" json:"msg,omitempty"` } func (x *SendAppGossipMsg) Reset() { @@ -272,85 +274,35 @@ func (*SendAppGossipMsg) Descriptor() ([]byte, []int) { return file_appsender_appsender_proto_rawDescGZIP(), []int{3} } -func (x *SendAppGossipMsg) GetMsg() []byte { +func (x *SendAppGossipMsg) GetNodeIds() [][]byte { if x != nil { - return x.Msg + return x.NodeIds } return nil } -func (x *SendAppGossipMsg) GetNumValidators() uint64 { +func (x *SendAppGossipMsg) GetValidators() uint64 { if x != nil { - return x.NumValidators + return x.Validators } return 0 } -func (x *SendAppGossipMsg) GetNumNonValidators() uint64 { +func (x *SendAppGossipMsg) GetNonValidators() uint64 { if x != nil { - return x.NumNonValidators + return x.NonValidators } return 0 } -func (x *SendAppGossipMsg) GetNumPeers() uint64 { +func (x *SendAppGossipMsg) GetPeers() uint64 { if x != nil { - return x.NumPeers + return x.Peers } return 0 } -type SendAppGossipSpecificMsg struct { - state protoimpl.MessageState - sizeCache protoimpl.SizeCache - unknownFields protoimpl.UnknownFields - - // The nodes to send this request to - NodeIds [][]byte `protobuf:"bytes,1,rep,name=node_ids,json=nodeIds,proto3" json:"node_ids,omitempty"` - // The message body - Msg []byte `protobuf:"bytes,2,opt,name=msg,proto3" json:"msg,omitempty"` -} - -func (x *SendAppGossipSpecificMsg) Reset() { - *x = SendAppGossipSpecificMsg{} - if protoimpl.UnsafeEnabled { - mi := &file_appsender_appsender_proto_msgTypes[4] - ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) - ms.StoreMessageInfo(mi) - } -} - -func (x *SendAppGossipSpecificMsg) String() string { - return protoimpl.X.MessageStringOf(x) -} - -func (*SendAppGossipSpecificMsg) ProtoMessage() {} - -func (x *SendAppGossipSpecificMsg) ProtoReflect() protoreflect.Message { - mi := &file_appsender_appsender_proto_msgTypes[4] - 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 SendAppGossipSpecificMsg.ProtoReflect.Descriptor instead. -func (*SendAppGossipSpecificMsg) Descriptor() ([]byte, []int) { - return file_appsender_appsender_proto_rawDescGZIP(), []int{4} -} - -func (x *SendAppGossipSpecificMsg) GetNodeIds() [][]byte { - if x != nil { - return x.NodeIds - } - return nil -} - -func (x *SendAppGossipSpecificMsg) GetMsg() []byte { +func (x *SendAppGossipMsg) GetMsg() []byte { if x != nil { return x.Msg } @@ -373,7 +325,7 @@ type SendCrossChainAppRequestMsg struct { func (x *SendCrossChainAppRequestMsg) Reset() { *x = SendCrossChainAppRequestMsg{} if protoimpl.UnsafeEnabled { - mi := &file_appsender_appsender_proto_msgTypes[5] + mi := &file_appsender_appsender_proto_msgTypes[4] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -386,7 +338,7 @@ func (x *SendCrossChainAppRequestMsg) String() string { func (*SendCrossChainAppRequestMsg) ProtoMessage() {} func (x *SendCrossChainAppRequestMsg) ProtoReflect() protoreflect.Message { - mi := &file_appsender_appsender_proto_msgTypes[5] + mi := &file_appsender_appsender_proto_msgTypes[4] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -399,7 +351,7 @@ func (x *SendCrossChainAppRequestMsg) ProtoReflect() protoreflect.Message { // Deprecated: Use SendCrossChainAppRequestMsg.ProtoReflect.Descriptor instead. func (*SendCrossChainAppRequestMsg) Descriptor() ([]byte, []int) { - return file_appsender_appsender_proto_rawDescGZIP(), []int{5} + return file_appsender_appsender_proto_rawDescGZIP(), []int{4} } func (x *SendCrossChainAppRequestMsg) GetChainId() []byte { @@ -439,7 +391,7 @@ type SendCrossChainAppResponseMsg struct { func (x *SendCrossChainAppResponseMsg) Reset() { *x = SendCrossChainAppResponseMsg{} if protoimpl.UnsafeEnabled { - mi := &file_appsender_appsender_proto_msgTypes[6] + mi := &file_appsender_appsender_proto_msgTypes[5] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -452,7 +404,7 @@ func (x *SendCrossChainAppResponseMsg) String() string { func (*SendCrossChainAppResponseMsg) ProtoMessage() {} func (x *SendCrossChainAppResponseMsg) ProtoReflect() protoreflect.Message { - mi := &file_appsender_appsender_proto_msgTypes[6] + mi := &file_appsender_appsender_proto_msgTypes[5] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -465,7 +417,7 @@ func (x *SendCrossChainAppResponseMsg) ProtoReflect() protoreflect.Message { // Deprecated: Use SendCrossChainAppResponseMsg.ProtoReflect.Descriptor instead. func (*SendCrossChainAppResponseMsg) Descriptor() ([]byte, []int) { - return file_appsender_appsender_proto_rawDescGZIP(), []int{6} + return file_appsender_appsender_proto_rawDescGZIP(), []int{5} } func (x *SendCrossChainAppResponseMsg) GetChainId() []byte { @@ -507,7 +459,7 @@ type SendCrossChainAppErrorMsg struct { func (x *SendCrossChainAppErrorMsg) Reset() { *x = SendCrossChainAppErrorMsg{} if protoimpl.UnsafeEnabled { - mi := &file_appsender_appsender_proto_msgTypes[7] + mi := &file_appsender_appsender_proto_msgTypes[6] ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) ms.StoreMessageInfo(mi) } @@ -520,7 +472,7 @@ func (x *SendCrossChainAppErrorMsg) String() string { func (*SendCrossChainAppErrorMsg) ProtoMessage() {} func (x *SendCrossChainAppErrorMsg) ProtoReflect() protoreflect.Message { - mi := &file_appsender_appsender_proto_msgTypes[7] + mi := &file_appsender_appsender_proto_msgTypes[6] if protoimpl.UnsafeEnabled && x != nil { ms := protoimpl.X.MessageStateOf(protoimpl.Pointer(x)) if ms.LoadMessageInfo() == nil { @@ -533,7 +485,7 @@ func (x *SendCrossChainAppErrorMsg) ProtoReflect() protoreflect.Message { // Deprecated: Use SendCrossChainAppErrorMsg.ProtoReflect.Descriptor instead. func (*SendCrossChainAppErrorMsg) Descriptor() ([]byte, []int) { - return file_appsender_appsender_proto_rawDescGZIP(), []int{7} + return file_appsender_appsender_proto_rawDescGZIP(), []int{6} } func (x *SendCrossChainAppErrorMsg) GetChainId() []byte { @@ -593,90 +545,81 @@ var file_appsender_appsender_proto_rawDesc = []byte{ 0x18, 0x03, 0x20, 0x01, 0x28, 0x11, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x4d, - 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x96, 0x01, 0x0a, 0x10, 0x53, 0x65, 0x6e, 0x64, 0x41, - 0x70, 0x70, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x4d, 0x73, 0x67, 0x12, 0x10, 0x0a, 0x03, 0x6d, - 0x73, 0x67, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6d, 0x73, 0x67, 0x12, 0x25, 0x0a, - 0x0e, 0x6e, 0x75, 0x6d, 0x5f, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x18, - 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, 0x6e, 0x75, 0x6d, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, - 0x74, 0x6f, 0x72, 0x73, 0x12, 0x2c, 0x0a, 0x12, 0x6e, 0x75, 0x6d, 0x5f, 0x6e, 0x6f, 0x6e, 0x5f, - 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, - 0x52, 0x10, 0x6e, 0x75, 0x6d, 0x4e, 0x6f, 0x6e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, - 0x72, 0x73, 0x12, 0x1b, 0x0a, 0x09, 0x6e, 0x75, 0x6d, 0x5f, 0x70, 0x65, 0x65, 0x72, 0x73, 0x18, - 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x08, 0x6e, 0x75, 0x6d, 0x50, 0x65, 0x65, 0x72, 0x73, 0x22, - 0x47, 0x0a, 0x18, 0x53, 0x65, 0x6e, 0x64, 0x41, 0x70, 0x70, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, - 0x53, 0x70, 0x65, 0x63, 0x69, 0x66, 0x69, 0x63, 0x4d, 0x73, 0x67, 0x12, 0x19, 0x0a, 0x08, 0x6e, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x22, 0x9c, 0x01, 0x0a, 0x10, 0x53, 0x65, 0x6e, 0x64, 0x41, + 0x70, 0x70, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x4d, 0x73, 0x67, 0x12, 0x19, 0x0a, 0x08, 0x6e, 0x6f, 0x64, 0x65, 0x5f, 0x69, 0x64, 0x73, 0x18, 0x01, 0x20, 0x03, 0x28, 0x0c, 0x52, 0x07, 0x6e, - 0x6f, 0x64, 0x65, 0x49, 0x64, 0x73, 0x12, 0x10, 0x0a, 0x03, 0x6d, 0x73, 0x67, 0x18, 0x02, 0x20, - 0x01, 0x28, 0x0c, 0x52, 0x03, 0x6d, 0x73, 0x67, 0x22, 0x71, 0x0a, 0x1b, 0x53, 0x65, 0x6e, 0x64, - 0x43, 0x72, 0x6f, 0x73, 0x73, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x41, 0x70, 0x70, 0x52, 0x65, 0x71, - 0x75, 0x65, 0x73, 0x74, 0x4d, 0x73, 0x67, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x68, 0x61, 0x69, 0x6e, - 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, - 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, - 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, - 0x64, 0x12, 0x18, 0x0a, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x03, 0x20, 0x01, - 0x28, 0x0c, 0x52, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x74, 0x0a, 0x1c, 0x53, - 0x65, 0x6e, 0x64, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x41, 0x70, 0x70, - 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x4d, 0x73, 0x67, 0x12, 0x19, 0x0a, 0x08, 0x63, - 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x63, - 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, - 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, - 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, - 0x65, 0x22, 0x99, 0x01, 0x0a, 0x19, 0x53, 0x65, 0x6e, 0x64, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x43, - 0x68, 0x61, 0x69, 0x6e, 0x41, 0x70, 0x70, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x73, 0x67, 0x12, - 0x19, 0x0a, 0x08, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, - 0x0c, 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, - 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x09, - 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x65, 0x72, 0x72, - 0x6f, 0x72, 0x5f, 0x63, 0x6f, 0x64, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x11, 0x52, 0x09, 0x65, - 0x72, 0x72, 0x6f, 0x72, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, - 0x72, 0x5f, 0x6d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, - 0x0c, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x32, 0x8f, 0x05, - 0x0a, 0x09, 0x41, 0x70, 0x70, 0x53, 0x65, 0x6e, 0x64, 0x65, 0x72, 0x12, 0x46, 0x0a, 0x0e, 0x53, - 0x65, 0x6e, 0x64, 0x41, 0x70, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x2e, - 0x61, 0x70, 0x70, 0x73, 0x65, 0x6e, 0x64, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x6e, 0x64, 0x41, 0x70, - 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x4d, 0x73, 0x67, 0x1a, 0x16, 0x2e, 0x67, 0x6f, - 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, - 0x70, 0x74, 0x79, 0x12, 0x48, 0x0a, 0x0f, 0x53, 0x65, 0x6e, 0x64, 0x41, 0x70, 0x70, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x2e, 0x61, 0x70, 0x70, 0x73, 0x65, 0x6e, 0x64, - 0x65, 0x72, 0x2e, 0x53, 0x65, 0x6e, 0x64, 0x41, 0x70, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, - 0x73, 0x65, 0x4d, 0x73, 0x67, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x42, 0x0a, - 0x0c, 0x53, 0x65, 0x6e, 0x64, 0x41, 0x70, 0x70, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x1a, 0x2e, - 0x61, 0x70, 0x70, 0x73, 0x65, 0x6e, 0x64, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x6e, 0x64, 0x41, 0x70, - 0x70, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x73, 0x67, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, - 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, - 0x79, 0x12, 0x44, 0x0a, 0x0d, 0x53, 0x65, 0x6e, 0x64, 0x41, 0x70, 0x70, 0x47, 0x6f, 0x73, 0x73, - 0x69, 0x70, 0x12, 0x1b, 0x2e, 0x61, 0x70, 0x70, 0x73, 0x65, 0x6e, 0x64, 0x65, 0x72, 0x2e, 0x53, - 0x65, 0x6e, 0x64, 0x41, 0x70, 0x70, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x4d, 0x73, 0x67, 0x1a, - 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, - 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x54, 0x0a, 0x15, 0x53, 0x65, 0x6e, 0x64, 0x41, - 0x70, 0x70, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x53, 0x70, 0x65, 0x63, 0x69, 0x66, 0x69, 0x63, - 0x12, 0x23, 0x2e, 0x61, 0x70, 0x70, 0x73, 0x65, 0x6e, 0x64, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x6e, - 0x64, 0x41, 0x70, 0x70, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x53, 0x70, 0x65, 0x63, 0x69, 0x66, - 0x69, 0x63, 0x4d, 0x73, 0x67, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, - 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x5a, 0x0a, - 0x18, 0x53, 0x65, 0x6e, 0x64, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x41, - 0x70, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x26, 0x2e, 0x61, 0x70, 0x70, 0x73, - 0x65, 0x6e, 0x64, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x6e, 0x64, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x43, - 0x68, 0x61, 0x69, 0x6e, 0x41, 0x70, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x4d, 0x73, - 0x67, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, - 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x5c, 0x0a, 0x19, 0x53, 0x65, 0x6e, + 0x6f, 0x64, 0x65, 0x49, 0x64, 0x73, 0x12, 0x1e, 0x0a, 0x0a, 0x76, 0x61, 0x6c, 0x69, 0x64, 0x61, + 0x74, 0x6f, 0x72, 0x73, 0x18, 0x02, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0a, 0x76, 0x61, 0x6c, 0x69, + 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x12, 0x25, 0x0a, 0x0e, 0x6e, 0x6f, 0x6e, 0x5f, 0x76, 0x61, + 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x18, 0x03, 0x20, 0x01, 0x28, 0x04, 0x52, 0x0d, + 0x6e, 0x6f, 0x6e, 0x56, 0x61, 0x6c, 0x69, 0x64, 0x61, 0x74, 0x6f, 0x72, 0x73, 0x12, 0x14, 0x0a, + 0x05, 0x70, 0x65, 0x65, 0x72, 0x73, 0x18, 0x04, 0x20, 0x01, 0x28, 0x04, 0x52, 0x05, 0x70, 0x65, + 0x65, 0x72, 0x73, 0x12, 0x10, 0x0a, 0x03, 0x6d, 0x73, 0x67, 0x18, 0x05, 0x20, 0x01, 0x28, 0x0c, + 0x52, 0x03, 0x6d, 0x73, 0x67, 0x22, 0x71, 0x0a, 0x1b, 0x53, 0x65, 0x6e, 0x64, 0x43, 0x72, 0x6f, + 0x73, 0x73, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x41, 0x70, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, + 0x74, 0x4d, 0x73, 0x67, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, + 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, + 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, + 0x01, 0x28, 0x0d, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x18, + 0x0a, 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x18, 0x03, 0x20, 0x01, 0x28, 0x0c, 0x52, + 0x07, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x22, 0x74, 0x0a, 0x1c, 0x53, 0x65, 0x6e, 0x64, + 0x43, 0x72, 0x6f, 0x73, 0x73, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x41, 0x70, 0x70, 0x52, 0x65, 0x73, + 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x4d, 0x73, 0x67, 0x12, 0x19, 0x0a, 0x08, 0x63, 0x68, 0x61, 0x69, + 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, 0x63, 0x68, 0x61, 0x69, + 0x6e, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x5f, 0x69, + 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x09, 0x72, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x49, 0x64, 0x12, 0x1a, 0x0a, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x18, 0x03, + 0x20, 0x01, 0x28, 0x0c, 0x52, 0x08, 0x72, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x22, 0x99, + 0x01, 0x0a, 0x19, 0x53, 0x65, 0x6e, 0x64, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x43, 0x68, 0x61, 0x69, + 0x6e, 0x41, 0x70, 0x70, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x73, 0x67, 0x12, 0x19, 0x0a, 0x08, + 0x63, 0x68, 0x61, 0x69, 0x6e, 0x5f, 0x69, 0x64, 0x18, 0x01, 0x20, 0x01, 0x28, 0x0c, 0x52, 0x07, + 0x63, 0x68, 0x61, 0x69, 0x6e, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x72, 0x65, 0x71, 0x75, 0x65, + 0x73, 0x74, 0x5f, 0x69, 0x64, 0x18, 0x02, 0x20, 0x01, 0x28, 0x0d, 0x52, 0x09, 0x72, 0x65, 0x71, + 0x75, 0x65, 0x73, 0x74, 0x49, 0x64, 0x12, 0x1d, 0x0a, 0x0a, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, + 0x63, 0x6f, 0x64, 0x65, 0x18, 0x03, 0x20, 0x01, 0x28, 0x11, 0x52, 0x09, 0x65, 0x72, 0x72, 0x6f, + 0x72, 0x43, 0x6f, 0x64, 0x65, 0x12, 0x23, 0x0a, 0x0d, 0x65, 0x72, 0x72, 0x6f, 0x72, 0x5f, 0x6d, + 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x18, 0x04, 0x20, 0x01, 0x28, 0x09, 0x52, 0x0c, 0x65, 0x72, + 0x72, 0x6f, 0x72, 0x4d, 0x65, 0x73, 0x73, 0x61, 0x67, 0x65, 0x32, 0xb9, 0x04, 0x0a, 0x09, 0x41, + 0x70, 0x70, 0x53, 0x65, 0x6e, 0x64, 0x65, 0x72, 0x12, 0x46, 0x0a, 0x0e, 0x53, 0x65, 0x6e, 0x64, + 0x41, 0x70, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, 0x12, 0x1c, 0x2e, 0x61, 0x70, 0x70, + 0x73, 0x65, 0x6e, 0x64, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x6e, 0x64, 0x41, 0x70, 0x70, 0x52, 0x65, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x4d, 0x73, 0x67, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, + 0x12, 0x48, 0x0a, 0x0f, 0x53, 0x65, 0x6e, 0x64, 0x41, 0x70, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x12, 0x1d, 0x2e, 0x61, 0x70, 0x70, 0x73, 0x65, 0x6e, 0x64, 0x65, 0x72, 0x2e, + 0x53, 0x65, 0x6e, 0x64, 0x41, 0x70, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x4d, + 0x73, 0x67, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, + 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x42, 0x0a, 0x0c, 0x53, 0x65, + 0x6e, 0x64, 0x41, 0x70, 0x70, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x1a, 0x2e, 0x61, 0x70, 0x70, + 0x73, 0x65, 0x6e, 0x64, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x6e, 0x64, 0x41, 0x70, 0x70, 0x45, 0x72, + 0x72, 0x6f, 0x72, 0x4d, 0x73, 0x67, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x44, + 0x0a, 0x0d, 0x53, 0x65, 0x6e, 0x64, 0x41, 0x70, 0x70, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x12, + 0x1b, 0x2e, 0x61, 0x70, 0x70, 0x73, 0x65, 0x6e, 0x64, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x6e, 0x64, + 0x41, 0x70, 0x70, 0x47, 0x6f, 0x73, 0x73, 0x69, 0x70, 0x4d, 0x73, 0x67, 0x1a, 0x16, 0x2e, 0x67, + 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, + 0x6d, 0x70, 0x74, 0x79, 0x12, 0x5a, 0x0a, 0x18, 0x53, 0x65, 0x6e, 0x64, 0x43, 0x72, 0x6f, 0x73, + 0x73, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x41, 0x70, 0x70, 0x52, 0x65, 0x71, 0x75, 0x65, 0x73, 0x74, + 0x12, 0x26, 0x2e, 0x61, 0x70, 0x70, 0x73, 0x65, 0x6e, 0x64, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x6e, 0x64, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x41, 0x70, 0x70, 0x52, 0x65, - 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x27, 0x2e, 0x61, 0x70, 0x70, 0x73, 0x65, 0x6e, 0x64, - 0x65, 0x72, 0x2e, 0x53, 0x65, 0x6e, 0x64, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x43, 0x68, 0x61, 0x69, - 0x6e, 0x41, 0x70, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x4d, 0x73, 0x67, 0x1a, - 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, - 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x56, 0x0a, 0x16, 0x53, 0x65, 0x6e, 0x64, 0x43, - 0x72, 0x6f, 0x73, 0x73, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x41, 0x70, 0x70, 0x45, 0x72, 0x72, 0x6f, - 0x72, 0x12, 0x24, 0x2e, 0x61, 0x70, 0x70, 0x73, 0x65, 0x6e, 0x64, 0x65, 0x72, 0x2e, 0x53, 0x65, - 0x6e, 0x64, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x41, 0x70, 0x70, 0x45, - 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x73, 0x67, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, - 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x42, - 0x34, 0x5a, 0x32, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x76, - 0x61, 0x2d, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x61, 0x76, 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x68, 0x65, - 0x67, 0x6f, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, 0x70, 0x62, 0x2f, 0x61, 0x70, 0x70, 0x73, - 0x65, 0x6e, 0x64, 0x65, 0x72, 0x62, 0x06, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x33, + 0x71, 0x75, 0x65, 0x73, 0x74, 0x4d, 0x73, 0x67, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, + 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, + 0x12, 0x5c, 0x0a, 0x19, 0x53, 0x65, 0x6e, 0x64, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x43, 0x68, 0x61, + 0x69, 0x6e, 0x41, 0x70, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, 0x6e, 0x73, 0x65, 0x12, 0x27, 0x2e, + 0x61, 0x70, 0x70, 0x73, 0x65, 0x6e, 0x64, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x6e, 0x64, 0x43, 0x72, + 0x6f, 0x73, 0x73, 0x43, 0x68, 0x61, 0x69, 0x6e, 0x41, 0x70, 0x70, 0x52, 0x65, 0x73, 0x70, 0x6f, + 0x6e, 0x73, 0x65, 0x4d, 0x73, 0x67, 0x1a, 0x16, 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, + 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x12, 0x56, + 0x0a, 0x16, 0x53, 0x65, 0x6e, 0x64, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x43, 0x68, 0x61, 0x69, 0x6e, + 0x41, 0x70, 0x70, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x12, 0x24, 0x2e, 0x61, 0x70, 0x70, 0x73, 0x65, + 0x6e, 0x64, 0x65, 0x72, 0x2e, 0x53, 0x65, 0x6e, 0x64, 0x43, 0x72, 0x6f, 0x73, 0x73, 0x43, 0x68, + 0x61, 0x69, 0x6e, 0x41, 0x70, 0x70, 0x45, 0x72, 0x72, 0x6f, 0x72, 0x4d, 0x73, 0x67, 0x1a, 0x16, + 0x2e, 0x67, 0x6f, 0x6f, 0x67, 0x6c, 0x65, 0x2e, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x62, 0x75, 0x66, + 0x2e, 0x45, 0x6d, 0x70, 0x74, 0x79, 0x42, 0x34, 0x5a, 0x32, 0x67, 0x69, 0x74, 0x68, 0x75, 0x62, + 0x2e, 0x63, 0x6f, 0x6d, 0x2f, 0x61, 0x76, 0x61, 0x2d, 0x6c, 0x61, 0x62, 0x73, 0x2f, 0x61, 0x76, + 0x61, 0x6c, 0x61, 0x6e, 0x63, 0x68, 0x65, 0x67, 0x6f, 0x2f, 0x70, 0x72, 0x6f, 0x74, 0x6f, 0x2f, + 0x70, 0x62, 0x2f, 0x61, 0x70, 0x70, 0x73, 0x65, 0x6e, 0x64, 0x65, 0x72, 0x62, 0x06, 0x70, 0x72, + 0x6f, 0x74, 0x6f, 0x33, } var ( @@ -691,37 +634,34 @@ func file_appsender_appsender_proto_rawDescGZIP() []byte { return file_appsender_appsender_proto_rawDescData } -var file_appsender_appsender_proto_msgTypes = make([]protoimpl.MessageInfo, 8) +var file_appsender_appsender_proto_msgTypes = make([]protoimpl.MessageInfo, 7) var file_appsender_appsender_proto_goTypes = []interface{}{ (*SendAppRequestMsg)(nil), // 0: appsender.SendAppRequestMsg (*SendAppResponseMsg)(nil), // 1: appsender.SendAppResponseMsg (*SendAppErrorMsg)(nil), // 2: appsender.SendAppErrorMsg (*SendAppGossipMsg)(nil), // 3: appsender.SendAppGossipMsg - (*SendAppGossipSpecificMsg)(nil), // 4: appsender.SendAppGossipSpecificMsg - (*SendCrossChainAppRequestMsg)(nil), // 5: appsender.SendCrossChainAppRequestMsg - (*SendCrossChainAppResponseMsg)(nil), // 6: appsender.SendCrossChainAppResponseMsg - (*SendCrossChainAppErrorMsg)(nil), // 7: appsender.SendCrossChainAppErrorMsg - (*emptypb.Empty)(nil), // 8: google.protobuf.Empty + (*SendCrossChainAppRequestMsg)(nil), // 4: appsender.SendCrossChainAppRequestMsg + (*SendCrossChainAppResponseMsg)(nil), // 5: appsender.SendCrossChainAppResponseMsg + (*SendCrossChainAppErrorMsg)(nil), // 6: appsender.SendCrossChainAppErrorMsg + (*emptypb.Empty)(nil), // 7: google.protobuf.Empty } var file_appsender_appsender_proto_depIdxs = []int32{ 0, // 0: appsender.AppSender.SendAppRequest:input_type -> appsender.SendAppRequestMsg 1, // 1: appsender.AppSender.SendAppResponse:input_type -> appsender.SendAppResponseMsg 2, // 2: appsender.AppSender.SendAppError:input_type -> appsender.SendAppErrorMsg 3, // 3: appsender.AppSender.SendAppGossip:input_type -> appsender.SendAppGossipMsg - 4, // 4: appsender.AppSender.SendAppGossipSpecific:input_type -> appsender.SendAppGossipSpecificMsg - 5, // 5: appsender.AppSender.SendCrossChainAppRequest:input_type -> appsender.SendCrossChainAppRequestMsg - 6, // 6: appsender.AppSender.SendCrossChainAppResponse:input_type -> appsender.SendCrossChainAppResponseMsg - 7, // 7: appsender.AppSender.SendCrossChainAppError:input_type -> appsender.SendCrossChainAppErrorMsg - 8, // 8: appsender.AppSender.SendAppRequest:output_type -> google.protobuf.Empty - 8, // 9: appsender.AppSender.SendAppResponse:output_type -> google.protobuf.Empty - 8, // 10: appsender.AppSender.SendAppError:output_type -> google.protobuf.Empty - 8, // 11: appsender.AppSender.SendAppGossip:output_type -> google.protobuf.Empty - 8, // 12: appsender.AppSender.SendAppGossipSpecific:output_type -> google.protobuf.Empty - 8, // 13: appsender.AppSender.SendCrossChainAppRequest:output_type -> google.protobuf.Empty - 8, // 14: appsender.AppSender.SendCrossChainAppResponse:output_type -> google.protobuf.Empty - 8, // 15: appsender.AppSender.SendCrossChainAppError:output_type -> google.protobuf.Empty - 8, // [8:16] is the sub-list for method output_type - 0, // [0:8] is the sub-list for method input_type + 4, // 4: appsender.AppSender.SendCrossChainAppRequest:input_type -> appsender.SendCrossChainAppRequestMsg + 5, // 5: appsender.AppSender.SendCrossChainAppResponse:input_type -> appsender.SendCrossChainAppResponseMsg + 6, // 6: appsender.AppSender.SendCrossChainAppError:input_type -> appsender.SendCrossChainAppErrorMsg + 7, // 7: appsender.AppSender.SendAppRequest:output_type -> google.protobuf.Empty + 7, // 8: appsender.AppSender.SendAppResponse:output_type -> google.protobuf.Empty + 7, // 9: appsender.AppSender.SendAppError:output_type -> google.protobuf.Empty + 7, // 10: appsender.AppSender.SendAppGossip:output_type -> google.protobuf.Empty + 7, // 11: appsender.AppSender.SendCrossChainAppRequest:output_type -> google.protobuf.Empty + 7, // 12: appsender.AppSender.SendCrossChainAppResponse:output_type -> google.protobuf.Empty + 7, // 13: appsender.AppSender.SendCrossChainAppError:output_type -> google.protobuf.Empty + 7, // [7:14] is the sub-list for method output_type + 0, // [0:7] is the sub-list for method input_type 0, // [0:0] is the sub-list for extension type_name 0, // [0:0] is the sub-list for extension extendee 0, // [0:0] is the sub-list for field type_name @@ -782,18 +722,6 @@ func file_appsender_appsender_proto_init() { } } file_appsender_appsender_proto_msgTypes[4].Exporter = func(v interface{}, i int) interface{} { - switch v := v.(*SendAppGossipSpecificMsg); i { - case 0: - return &v.state - case 1: - return &v.sizeCache - case 2: - return &v.unknownFields - default: - return nil - } - } - file_appsender_appsender_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SendCrossChainAppRequestMsg); i { case 0: return &v.state @@ -805,7 +733,7 @@ func file_appsender_appsender_proto_init() { return nil } } - file_appsender_appsender_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { + file_appsender_appsender_proto_msgTypes[5].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SendCrossChainAppResponseMsg); i { case 0: return &v.state @@ -817,7 +745,7 @@ func file_appsender_appsender_proto_init() { return nil } } - file_appsender_appsender_proto_msgTypes[7].Exporter = func(v interface{}, i int) interface{} { + file_appsender_appsender_proto_msgTypes[6].Exporter = func(v interface{}, i int) interface{} { switch v := v.(*SendCrossChainAppErrorMsg); i { case 0: return &v.state @@ -836,7 +764,7 @@ func file_appsender_appsender_proto_init() { GoPackagePath: reflect.TypeOf(x{}).PkgPath(), RawDescriptor: file_appsender_appsender_proto_rawDesc, NumEnums: 0, - NumMessages: 8, + NumMessages: 7, NumExtensions: 0, NumServices: 1, }, diff --git a/proto/pb/appsender/appsender_grpc.pb.go b/proto/pb/appsender/appsender_grpc.pb.go index 6873c7768fb..2bd4c9cf664 100644 --- a/proto/pb/appsender/appsender_grpc.pb.go +++ b/proto/pb/appsender/appsender_grpc.pb.go @@ -24,7 +24,6 @@ const ( AppSender_SendAppResponse_FullMethodName = "/appsender.AppSender/SendAppResponse" AppSender_SendAppError_FullMethodName = "/appsender.AppSender/SendAppError" AppSender_SendAppGossip_FullMethodName = "/appsender.AppSender/SendAppGossip" - AppSender_SendAppGossipSpecific_FullMethodName = "/appsender.AppSender/SendAppGossipSpecific" AppSender_SendCrossChainAppRequest_FullMethodName = "/appsender.AppSender/SendCrossChainAppRequest" AppSender_SendCrossChainAppResponse_FullMethodName = "/appsender.AppSender/SendCrossChainAppResponse" AppSender_SendCrossChainAppError_FullMethodName = "/appsender.AppSender/SendCrossChainAppError" @@ -38,7 +37,6 @@ type AppSenderClient interface { SendAppResponse(ctx context.Context, in *SendAppResponseMsg, opts ...grpc.CallOption) (*emptypb.Empty, error) SendAppError(ctx context.Context, in *SendAppErrorMsg, opts ...grpc.CallOption) (*emptypb.Empty, error) SendAppGossip(ctx context.Context, in *SendAppGossipMsg, opts ...grpc.CallOption) (*emptypb.Empty, error) - SendAppGossipSpecific(ctx context.Context, in *SendAppGossipSpecificMsg, opts ...grpc.CallOption) (*emptypb.Empty, error) SendCrossChainAppRequest(ctx context.Context, in *SendCrossChainAppRequestMsg, opts ...grpc.CallOption) (*emptypb.Empty, error) SendCrossChainAppResponse(ctx context.Context, in *SendCrossChainAppResponseMsg, opts ...grpc.CallOption) (*emptypb.Empty, error) SendCrossChainAppError(ctx context.Context, in *SendCrossChainAppErrorMsg, opts ...grpc.CallOption) (*emptypb.Empty, error) @@ -88,15 +86,6 @@ func (c *appSenderClient) SendAppGossip(ctx context.Context, in *SendAppGossipMs return out, nil } -func (c *appSenderClient) SendAppGossipSpecific(ctx context.Context, in *SendAppGossipSpecificMsg, opts ...grpc.CallOption) (*emptypb.Empty, error) { - out := new(emptypb.Empty) - err := c.cc.Invoke(ctx, AppSender_SendAppGossipSpecific_FullMethodName, in, out, opts...) - if err != nil { - return nil, err - } - return out, nil -} - func (c *appSenderClient) SendCrossChainAppRequest(ctx context.Context, in *SendCrossChainAppRequestMsg, opts ...grpc.CallOption) (*emptypb.Empty, error) { out := new(emptypb.Empty) err := c.cc.Invoke(ctx, AppSender_SendCrossChainAppRequest_FullMethodName, in, out, opts...) @@ -132,7 +121,6 @@ type AppSenderServer interface { SendAppResponse(context.Context, *SendAppResponseMsg) (*emptypb.Empty, error) SendAppError(context.Context, *SendAppErrorMsg) (*emptypb.Empty, error) SendAppGossip(context.Context, *SendAppGossipMsg) (*emptypb.Empty, error) - SendAppGossipSpecific(context.Context, *SendAppGossipSpecificMsg) (*emptypb.Empty, error) SendCrossChainAppRequest(context.Context, *SendCrossChainAppRequestMsg) (*emptypb.Empty, error) SendCrossChainAppResponse(context.Context, *SendCrossChainAppResponseMsg) (*emptypb.Empty, error) SendCrossChainAppError(context.Context, *SendCrossChainAppErrorMsg) (*emptypb.Empty, error) @@ -155,9 +143,6 @@ func (UnimplementedAppSenderServer) SendAppError(context.Context, *SendAppErrorM func (UnimplementedAppSenderServer) SendAppGossip(context.Context, *SendAppGossipMsg) (*emptypb.Empty, error) { return nil, status.Errorf(codes.Unimplemented, "method SendAppGossip not implemented") } -func (UnimplementedAppSenderServer) SendAppGossipSpecific(context.Context, *SendAppGossipSpecificMsg) (*emptypb.Empty, error) { - return nil, status.Errorf(codes.Unimplemented, "method SendAppGossipSpecific not implemented") -} func (UnimplementedAppSenderServer) SendCrossChainAppRequest(context.Context, *SendCrossChainAppRequestMsg) (*emptypb.Empty, error) { return nil, status.Errorf(codes.Unimplemented, "method SendCrossChainAppRequest not implemented") } @@ -252,24 +237,6 @@ func _AppSender_SendAppGossip_Handler(srv interface{}, ctx context.Context, dec return interceptor(ctx, in, info, handler) } -func _AppSender_SendAppGossipSpecific_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { - in := new(SendAppGossipSpecificMsg) - if err := dec(in); err != nil { - return nil, err - } - if interceptor == nil { - return srv.(AppSenderServer).SendAppGossipSpecific(ctx, in) - } - info := &grpc.UnaryServerInfo{ - Server: srv, - FullMethod: AppSender_SendAppGossipSpecific_FullMethodName, - } - handler := func(ctx context.Context, req interface{}) (interface{}, error) { - return srv.(AppSenderServer).SendAppGossipSpecific(ctx, req.(*SendAppGossipSpecificMsg)) - } - return interceptor(ctx, in, info, handler) -} - func _AppSender_SendCrossChainAppRequest_Handler(srv interface{}, ctx context.Context, dec func(interface{}) error, interceptor grpc.UnaryServerInterceptor) (interface{}, error) { in := new(SendCrossChainAppRequestMsg) if err := dec(in); err != nil { @@ -347,10 +314,6 @@ var AppSender_ServiceDesc = grpc.ServiceDesc{ MethodName: "SendAppGossip", Handler: _AppSender_SendAppGossip_Handler, }, - { - MethodName: "SendAppGossipSpecific", - Handler: _AppSender_SendAppGossipSpecific_Handler, - }, { MethodName: "SendCrossChainAppRequest", Handler: _AppSender_SendCrossChainAppRequest_Handler, diff --git a/snow/engine/common/appsender/appsender_client.go b/snow/engine/common/appsender/appsender_client.go index 0513fa9d940..dcf2ef3dc55 100644 --- a/snow/engine/common/appsender/appsender_client.go +++ b/snow/engine/common/appsender/appsender_client.go @@ -107,35 +107,23 @@ func (c *Client) SendAppError(ctx context.Context, nodeID ids.NodeID, requestID func (c *Client) SendAppGossip( ctx context.Context, + config common.SendConfig, msg []byte, - numValidators int, - numNonValidators int, - numPeers int, ) error { - _, err := c.client.SendAppGossip( - ctx, - &appsenderpb.SendAppGossipMsg{ - Msg: msg, - NumValidators: uint64(numValidators), - NumNonValidators: uint64(numNonValidators), - NumPeers: uint64(numPeers), - }, - ) - return err -} - -func (c *Client) SendAppGossipSpecific(ctx context.Context, nodeIDs set.Set[ids.NodeID], msg []byte) error { - nodeIDsBytes := make([][]byte, nodeIDs.Len()) + nodeIDs := make([][]byte, config.NodeIDs.Len()) i := 0 - for nodeID := range nodeIDs { - nodeIDsBytes[i] = nodeID.Bytes() + for nodeID := range config.NodeIDs { + nodeIDs[i] = nodeID.Bytes() i++ } - _, err := c.client.SendAppGossipSpecific( + _, err := c.client.SendAppGossip( ctx, - &appsenderpb.SendAppGossipSpecificMsg{ - NodeIds: nodeIDsBytes, - Msg: msg, + &appsenderpb.SendAppGossipMsg{ + NodeIds: nodeIDs, + Validators: uint64(config.Validators), + NonValidators: uint64(config.NonValidators), + Peers: uint64(config.Peers), + Msg: msg, }, ) return err diff --git a/snow/engine/common/appsender/appsender_server.go b/snow/engine/common/appsender/appsender_server.go index 455fc7498dc..2a3734d8934 100644 --- a/snow/engine/common/appsender/appsender_server.go +++ b/snow/engine/common/appsender/appsender_server.go @@ -87,17 +87,6 @@ func (s *Server) SendAppError(ctx context.Context, req *appsenderpb.SendAppError } func (s *Server) SendAppGossip(ctx context.Context, req *appsenderpb.SendAppGossipMsg) (*emptypb.Empty, error) { - err := s.appSender.SendAppGossip( - ctx, - req.Msg, - int(req.NumValidators), - int(req.NumNonValidators), - int(req.NumPeers), - ) - return &emptypb.Empty{}, err -} - -func (s *Server) SendAppGossipSpecific(ctx context.Context, req *appsenderpb.SendAppGossipSpecificMsg) (*emptypb.Empty, error) { nodeIDs := set.NewSet[ids.NodeID](len(req.NodeIds)) for _, nodeIDBytes := range req.NodeIds { nodeID, err := ids.ToNodeID(nodeIDBytes) @@ -106,6 +95,15 @@ func (s *Server) SendAppGossipSpecific(ctx context.Context, req *appsenderpb.Sen } nodeIDs.Add(nodeID) } - err := s.appSender.SendAppGossipSpecific(ctx, nodeIDs, req.Msg) + err := s.appSender.SendAppGossip( + ctx, + common.SendConfig{ + NodeIDs: nodeIDs, + Validators: int(req.Validators), + NonValidators: int(req.NonValidators), + Peers: int(req.Peers), + }, + req.Msg, + ) return &emptypb.Empty{}, err } diff --git a/snow/engine/common/mock_sender.go b/snow/engine/common/mock_sender.go index ecc4a4f851b..d1d3d1a68e5 100644 --- a/snow/engine/common/mock_sender.go +++ b/snow/engine/common/mock_sender.go @@ -104,31 +104,17 @@ func (mr *MockSenderMockRecorder) SendAppError(ctx, nodeID, requestID, errorCode } // SendAppGossip mocks base method. -func (m *MockSender) SendAppGossip(ctx context.Context, appGossipBytes []byte, numValidators, numNonValidators, numPeers int) error { +func (m *MockSender) SendAppGossip(ctx context.Context, config SendConfig, appGossipBytes []byte) error { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "SendAppGossip", ctx, appGossipBytes, numValidators, numNonValidators, numPeers) + ret := m.ctrl.Call(m, "SendAppGossip", ctx, config, appGossipBytes) ret0, _ := ret[0].(error) return ret0 } // SendAppGossip indicates an expected call of SendAppGossip. -func (mr *MockSenderMockRecorder) SendAppGossip(ctx, appGossipBytes, numValidators, numNonValidators, numPeers any) *gomock.Call { +func (mr *MockSenderMockRecorder) SendAppGossip(ctx, config, appGossipBytes any) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendAppGossip", reflect.TypeOf((*MockSender)(nil).SendAppGossip), ctx, appGossipBytes, numValidators, numNonValidators, numPeers) -} - -// SendAppGossipSpecific mocks base method. -func (m *MockSender) SendAppGossipSpecific(ctx context.Context, nodeIDs set.Set[ids.NodeID], appGossipBytes []byte) error { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "SendAppGossipSpecific", ctx, nodeIDs, appGossipBytes) - ret0, _ := ret[0].(error) - return ret0 -} - -// SendAppGossipSpecific indicates an expected call of SendAppGossipSpecific. -func (mr *MockSenderMockRecorder) SendAppGossipSpecific(ctx, nodeIDs, appGossipBytes any) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendAppGossipSpecific", reflect.TypeOf((*MockSender)(nil).SendAppGossipSpecific), ctx, nodeIDs, appGossipBytes) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "SendAppGossip", reflect.TypeOf((*MockSender)(nil).SendAppGossip), ctx, config, appGossipBytes) } // SendAppRequest mocks base method. diff --git a/snow/engine/common/sender.go b/snow/engine/common/sender.go index adb4be513c6..09389061985 100644 --- a/snow/engine/common/sender.go +++ b/snow/engine/common/sender.go @@ -10,6 +10,14 @@ import ( "github.com/ava-labs/avalanchego/utils/set" ) +// SendConfig is used to specify who to send messages to over the p2p network. +type SendConfig struct { + NodeIDs set.Set[ids.NodeID] + Validators int + NonValidators int + Peers int +} + // Sender defines how a consensus engine sends messages and requests to other // validators. // @@ -174,12 +182,9 @@ type NetworkAppSender interface { // Gossip an application-level message. SendAppGossip( ctx context.Context, + config SendConfig, appGossipBytes []byte, - numValidators int, - numNonValidators int, - numPeers int, ) error - SendAppGossipSpecific(ctx context.Context, nodeIDs set.Set[ids.NodeID], appGossipBytes []byte) error } // CrossChainAppSender sends local VM-level messages to another VM. diff --git a/snow/engine/common/test_sender.go b/snow/engine/common/test_sender.go index cbec25f33f6..e3cb44165c5 100644 --- a/snow/engine/common/test_sender.go +++ b/snow/engine/common/test_sender.go @@ -17,11 +17,10 @@ var ( _ Sender = (*SenderTest)(nil) _ AppSender = (*FakeSender)(nil) - errSendAppRequest = errors.New("unexpectedly called SendAppRequest") - errSendAppResponse = errors.New("unexpectedly called SendAppResponse") - errSendAppError = errors.New("unexpectedly called SendAppError") - errSendAppGossip = errors.New("unexpectedly called SendAppGossip") - errSendAppGossipSpecific = errors.New("unexpectedly called SendAppGossipSpecific") + errSendAppRequest = errors.New("unexpectedly called SendAppRequest") + errSendAppResponse = errors.New("unexpectedly called SendAppResponse") + errSendAppError = errors.New("unexpectedly called SendAppError") + errSendAppGossip = errors.New("unexpectedly called SendAppGossip") ) // SenderTest is a test sender @@ -35,7 +34,7 @@ type SenderTest struct { CantSendGet, CantSendGetAncestors, CantSendPut, CantSendAncestors, CantSendPullQuery, CantSendPushQuery, CantSendChits, CantSendAppRequest, CantSendAppResponse, CantSendAppError, - CantSendAppGossip, CantSendAppGossipSpecific, + CantSendAppGossip, CantSendCrossChainAppRequest, CantSendCrossChainAppResponse, CantSendCrossChainAppError bool SendGetStateSummaryFrontierF func(context.Context, set.Set[ids.NodeID], uint32) @@ -56,8 +55,7 @@ type SenderTest struct { SendAppRequestF func(context.Context, set.Set[ids.NodeID], uint32, []byte) error SendAppResponseF func(context.Context, ids.NodeID, uint32, []byte) error SendAppErrorF func(context.Context, ids.NodeID, uint32, int32, string) error - SendAppGossipF func(context.Context, []byte, int, int, int) error - SendAppGossipSpecificF func(context.Context, set.Set[ids.NodeID], []byte) error + SendAppGossipF func(context.Context, SendConfig, []byte) error SendCrossChainAppRequestF func(context.Context, ids.ID, uint32, []byte) SendCrossChainAppResponseF func(context.Context, ids.ID, uint32, []byte) SendCrossChainAppErrorF func(context.Context, ids.ID, uint32, int32, string) @@ -83,7 +81,6 @@ func (s *SenderTest) Default(cant bool) { s.CantSendAppRequest = cant s.CantSendAppResponse = cant s.CantSendAppGossip = cant - s.CantSendAppGossipSpecific = cant s.CantSendCrossChainAppRequest = cant s.CantSendCrossChainAppResponse = cant } @@ -333,37 +330,22 @@ func (s *SenderTest) SendAppError(ctx context.Context, nodeID ids.NodeID, reques // initialized, then testing will fail. func (s *SenderTest) SendAppGossip( ctx context.Context, + config SendConfig, appGossipBytes []byte, - numValidators int, - numNonValidators int, - numPeers int, ) error { switch { case s.SendAppGossipF != nil: - return s.SendAppGossipF(ctx, appGossipBytes, numValidators, numNonValidators, numPeers) + return s.SendAppGossipF(ctx, config, appGossipBytes) case s.CantSendAppGossip && s.T != nil: require.FailNow(s.T, errSendAppGossip.Error()) } return errSendAppGossip } -// SendAppGossipSpecific calls SendAppGossipSpecificF if it was initialized. If it wasn't -// initialized and this function shouldn't be called and testing was -// initialized, then testing will fail. -func (s *SenderTest) SendAppGossipSpecific(ctx context.Context, nodeIDs set.Set[ids.NodeID], appGossipBytes []byte) error { - switch { - case s.SendAppGossipSpecificF != nil: - return s.SendAppGossipSpecificF(ctx, nodeIDs, appGossipBytes) - case s.CantSendAppGossipSpecific && s.T != nil: - require.FailNow(s.T, errSendAppGossipSpecific.Error()) - } - return errSendAppGossipSpecific -} - // FakeSender is used for testing type FakeSender struct { SentAppRequest, SentAppResponse, - SentAppGossip, SentAppGossipSpecific, + SentAppGossip, SentCrossChainAppRequest, SentCrossChainAppResponse chan []byte SentAppError, SentCrossChainAppError chan *AppError @@ -399,7 +381,7 @@ func (f FakeSender) SendAppError(_ context.Context, _ ids.NodeID, _ uint32, erro return nil } -func (f FakeSender) SendAppGossip(_ context.Context, bytes []byte, _ int, _ int, _ int) error { +func (f FakeSender) SendAppGossip(_ context.Context, _ SendConfig, bytes []byte) error { if f.SentAppGossip == nil { return nil } @@ -408,15 +390,6 @@ func (f FakeSender) SendAppGossip(_ context.Context, bytes []byte, _ int, _ int, return nil } -func (f FakeSender) SendAppGossipSpecific(_ context.Context, _ set.Set[ids.NodeID], bytes []byte) error { - if f.SentAppGossipSpecific == nil { - return nil - } - - f.SentAppGossipSpecific <- bytes - return nil -} - func (f FakeSender) SendCrossChainAppRequest(_ context.Context, _ ids.ID, _ uint32, bytes []byte) error { if f.SentCrossChainAppRequest == nil { return nil diff --git a/snow/networking/sender/external_sender.go b/snow/networking/sender/external_sender.go index 7d279889e3a..f8f90a2cce4 100644 --- a/snow/networking/sender/external_sender.go +++ b/snow/networking/sender/external_sender.go @@ -6,6 +6,7 @@ package sender import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/message" + "github.com/ava-labs/avalanchego/snow/engine/common" "github.com/ava-labs/avalanchego/subnets" "github.com/ava-labs/avalanchego/utils/set" ) @@ -13,22 +14,10 @@ import ( // ExternalSender sends consensus messages to other validators // Right now this is implemented in the networking package type ExternalSender interface { - // Send a message to a specific set of nodes Send( msg message.OutboundMessage, - nodeIDs set.Set[ids.NodeID], + config common.SendConfig, subnetID ids.ID, allower subnets.Allower, ) set.Set[ids.NodeID] - - // Send a message to a random group of nodes in a subnet. - // Nodes are sampled based on their validator status. - Gossip( - msg message.OutboundMessage, - subnetID ids.ID, - numValidatorsToSend int, - numNonValidatorsToSend int, - numPeersToSend int, - allower subnets.Allower, - ) set.Set[ids.NodeID] } diff --git a/snow/networking/sender/mock_external_sender.go b/snow/networking/sender/mock_external_sender.go index 9dc0a50d1af..420f3b79fc4 100644 --- a/snow/networking/sender/mock_external_sender.go +++ b/snow/networking/sender/mock_external_sender.go @@ -14,6 +14,7 @@ import ( ids "github.com/ava-labs/avalanchego/ids" message "github.com/ava-labs/avalanchego/message" + common "github.com/ava-labs/avalanchego/snow/engine/common" subnets "github.com/ava-labs/avalanchego/subnets" set "github.com/ava-labs/avalanchego/utils/set" gomock "go.uber.org/mock/gomock" @@ -42,30 +43,16 @@ func (m *MockExternalSender) EXPECT() *MockExternalSenderMockRecorder { return m.recorder } -// Gossip mocks base method. -func (m *MockExternalSender) Gossip(msg message.OutboundMessage, subnetID ids.ID, numValidatorsToSend, numNonValidatorsToSend, numPeersToSend int, allower subnets.Allower) set.Set[ids.NodeID] { - m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Gossip", msg, subnetID, numValidatorsToSend, numNonValidatorsToSend, numPeersToSend, allower) - ret0, _ := ret[0].(set.Set[ids.NodeID]) - return ret0 -} - -// Gossip indicates an expected call of Gossip. -func (mr *MockExternalSenderMockRecorder) Gossip(msg, subnetID, numValidatorsToSend, numNonValidatorsToSend, numPeersToSend, allower any) *gomock.Call { - mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Gossip", reflect.TypeOf((*MockExternalSender)(nil).Gossip), msg, subnetID, numValidatorsToSend, numNonValidatorsToSend, numPeersToSend, allower) -} - // Send mocks base method. -func (m *MockExternalSender) Send(msg message.OutboundMessage, nodeIDs set.Set[ids.NodeID], subnetID ids.ID, allower subnets.Allower) set.Set[ids.NodeID] { +func (m *MockExternalSender) Send(msg message.OutboundMessage, config common.SendConfig, subnetID ids.ID, allower subnets.Allower) set.Set[ids.NodeID] { m.ctrl.T.Helper() - ret := m.ctrl.Call(m, "Send", msg, nodeIDs, subnetID, allower) + ret := m.ctrl.Call(m, "Send", msg, config, subnetID, allower) ret0, _ := ret[0].(set.Set[ids.NodeID]) return ret0 } // Send indicates an expected call of Send. -func (mr *MockExternalSenderMockRecorder) Send(msg, nodeIDs, subnetID, allower any) *gomock.Call { +func (mr *MockExternalSenderMockRecorder) Send(msg, config, subnetID, allower any) *gomock.Call { mr.mock.ctrl.T.Helper() - return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Send", reflect.TypeOf((*MockExternalSender)(nil).Send), msg, nodeIDs, subnetID, allower) + return mr.mock.ctrl.RecordCallWithMethodType(mr.mock, "Send", reflect.TypeOf((*MockExternalSender)(nil).Send), msg, config, subnetID, allower) } diff --git a/snow/networking/sender/sender.go b/snow/networking/sender/sender.go index eeef6f72248..b7fa9bd395d 100644 --- a/snow/networking/sender/sender.go +++ b/snow/networking/sender/sender.go @@ -145,7 +145,9 @@ func (s *sender) SendGetStateSummaryFrontier(ctx context.Context, nodeIDs set.Se if err == nil { sentTo = s.sender.Send( outMsg, - nodeIDs, + common.SendConfig{ + NodeIDs: nodeIDs, + }, s.ctx.SubnetID, s.subnet, ) @@ -207,7 +209,9 @@ func (s *sender) SendStateSummaryFrontier(ctx context.Context, nodeID ids.NodeID nodeIDs := set.Of(nodeID) sentTo := s.sender.Send( outMsg, - nodeIDs, + common.SendConfig{ + NodeIDs: nodeIDs, + }, s.ctx.SubnetID, s.subnet, ) @@ -288,7 +292,9 @@ func (s *sender) SendGetAcceptedStateSummary(ctx context.Context, nodeIDs set.Se if err == nil { sentTo = s.sender.Send( outMsg, - nodeIDs, + common.SendConfig{ + NodeIDs: nodeIDs, + }, s.ctx.SubnetID, s.subnet, ) @@ -350,7 +356,9 @@ func (s *sender) SendAcceptedStateSummary(ctx context.Context, nodeID ids.NodeID nodeIDs := set.Of(nodeID) sentTo := s.sender.Send( outMsg, - nodeIDs, + common.SendConfig{ + NodeIDs: nodeIDs, + }, s.ctx.SubnetID, s.subnet, ) @@ -423,7 +431,9 @@ func (s *sender) SendGetAcceptedFrontier(ctx context.Context, nodeIDs set.Set[id if err == nil { sentTo = s.sender.Send( outMsg, - nodeIDs, + common.SendConfig{ + NodeIDs: nodeIDs, + }, s.ctx.SubnetID, s.subnet, ) @@ -485,7 +495,9 @@ func (s *sender) SendAcceptedFrontier(ctx context.Context, nodeID ids.NodeID, re nodeIDs := set.Of(nodeID) sentTo := s.sender.Send( outMsg, - nodeIDs, + common.SendConfig{ + NodeIDs: nodeIDs, + }, s.ctx.SubnetID, s.subnet, ) @@ -560,7 +572,9 @@ func (s *sender) SendGetAccepted(ctx context.Context, nodeIDs set.Set[ids.NodeID if err == nil { sentTo = s.sender.Send( outMsg, - nodeIDs, + common.SendConfig{ + NodeIDs: nodeIDs, + }, s.ctx.SubnetID, s.subnet, ) @@ -618,7 +632,9 @@ func (s *sender) SendAccepted(ctx context.Context, nodeID ids.NodeID, requestID nodeIDs := set.Of(nodeID) sentTo := s.sender.Send( outMsg, - nodeIDs, + common.SendConfig{ + NodeIDs: nodeIDs, + }, s.ctx.SubnetID, s.subnet, ) @@ -698,7 +714,9 @@ func (s *sender) SendGetAncestors(ctx context.Context, nodeID ids.NodeID, reques nodeIDs := set.Of(nodeID) sentTo := s.sender.Send( outMsg, - nodeIDs, + common.SendConfig{ + NodeIDs: nodeIDs, + }, s.ctx.SubnetID, s.subnet, ) @@ -734,7 +752,9 @@ func (s *sender) SendAncestors(_ context.Context, nodeID ids.NodeID, requestID u nodeIDs := set.Of(nodeID) sentTo := s.sender.Send( outMsg, - nodeIDs, + common.SendConfig{ + NodeIDs: nodeIDs, + }, s.ctx.SubnetID, s.subnet, ) @@ -804,7 +824,9 @@ func (s *sender) SendGet(ctx context.Context, nodeID ids.NodeID, requestID uint3 nodeIDs := set.Of(nodeID) sentTo = s.sender.Send( outMsg, - nodeIDs, + common.SendConfig{ + NodeIDs: nodeIDs, + }, s.ctx.SubnetID, s.subnet, ) @@ -851,7 +873,9 @@ func (s *sender) SendPut(_ context.Context, nodeID ids.NodeID, requestID uint32, nodeIDs := set.Of(nodeID) sentTo := s.sender.Send( outMsg, - nodeIDs, + common.SendConfig{ + NodeIDs: nodeIDs, + }, s.ctx.SubnetID, s.subnet, ) @@ -965,7 +989,9 @@ func (s *sender) SendPushQuery( if err == nil { sentTo = s.sender.Send( outMsg, - nodeIDs, + common.SendConfig{ + NodeIDs: nodeIDs, + }, s.ctx.SubnetID, s.subnet, ) @@ -1102,7 +1128,9 @@ func (s *sender) SendPullQuery( if err == nil { sentTo = s.sender.Send( outMsg, - nodeIDs, + common.SendConfig{ + NodeIDs: nodeIDs, + }, s.ctx.SubnetID, s.subnet, ) @@ -1186,7 +1214,9 @@ func (s *sender) SendChits( nodeIDs := set.Of(nodeID) sentTo := s.sender.Send( outMsg, - nodeIDs, + common.SendConfig{ + NodeIDs: nodeIDs, + }, s.ctx.SubnetID, s.subnet, ) @@ -1349,7 +1379,9 @@ func (s *sender) SendAppRequest(ctx context.Context, nodeIDs set.Set[ids.NodeID] if err == nil { sentTo = s.sender.Send( outMsg, - nodeIDs, + common.SendConfig{ + NodeIDs: nodeIDs, + }, s.ctx.SubnetID, s.subnet, ) @@ -1432,7 +1464,9 @@ func (s *sender) SendAppResponse(ctx context.Context, nodeID ids.NodeID, request nodeIDs := set.Of(nodeID) sentTo := s.sender.Send( outMsg, - nodeIDs, + common.SendConfig{ + NodeIDs: nodeIDs, + }, s.ctx.SubnetID, s.subnet, ) @@ -1495,7 +1529,9 @@ func (s *sender) SendAppError(ctx context.Context, nodeID ids.NodeID, requestID // Send the message over the network. sentTo := s.sender.Send( outMsg, - set.Of(nodeID), + common.SendConfig{ + NodeIDs: set.Of(nodeID), + }, s.ctx.SubnetID, s.subnet, ) @@ -1523,55 +1559,10 @@ func (s *sender) SendAppError(ctx context.Context, nodeID ids.NodeID, requestID return nil } -func (s *sender) SendAppGossipSpecific(_ context.Context, nodeIDs set.Set[ids.NodeID], appGossipBytes []byte) error { - // Create the outbound message. - outMsg, err := s.msgCreator.AppGossip(s.ctx.ChainID, appGossipBytes) - if err != nil { - s.ctx.Log.Error("failed to build message", - zap.Stringer("messageOp", message.AppGossipOp), - zap.Stringer("chainID", s.ctx.ChainID), - zap.Binary("payload", appGossipBytes), - zap.Error(err), - ) - return nil - } - - // Send the message over the network. - sentTo := s.sender.Send( - outMsg, - nodeIDs, - s.ctx.SubnetID, - s.subnet, - ) - if sentTo.Len() == 0 { - for nodeID := range nodeIDs { - if !sentTo.Contains(nodeID) { - if s.ctx.Log.Enabled(logging.Verbo) { - s.ctx.Log.Verbo("failed to send message", - zap.Stringer("messageOp", message.AppGossipOp), - zap.Stringer("nodeID", nodeID), - zap.Stringer("chainID", s.ctx.ChainID), - zap.Binary("payload", appGossipBytes), - ) - } else { - s.ctx.Log.Debug("failed to send message", - zap.Stringer("messageOp", message.AppGossipOp), - zap.Stringer("nodeID", nodeID), - zap.Stringer("chainID", s.ctx.ChainID), - ) - } - } - } - } - return nil -} - func (s *sender) SendAppGossip( _ context.Context, + config common.SendConfig, appGossipBytes []byte, - numValidators int, - numNonValidators int, - numPeers int, ) error { // Create the outbound message. outMsg, err := s.msgCreator.AppGossip(s.ctx.ChainID, appGossipBytes) @@ -1585,12 +1576,10 @@ func (s *sender) SendAppGossip( return nil } - sentTo := s.sender.Gossip( + sentTo := s.sender.Send( outMsg, + config, s.ctx.SubnetID, - numValidators, - numNonValidators, - numPeers, s.subnet, ) if sentTo.Len() == 0 { diff --git a/snow/networking/sender/sender_test.go b/snow/networking/sender/sender_test.go index 3f369c26ef4..d48d347176d 100644 --- a/snow/networking/sender/sender_test.go +++ b/snow/networking/sender/sender_test.go @@ -288,14 +288,14 @@ func TestTimeout(t *testing.T) { } // Send messages to disconnected peers - externalSender.SendF = func(message.OutboundMessage, set.Set[ids.NodeID], ids.ID, subnets.Allower) set.Set[ids.NodeID] { + externalSender.SendF = func(message.OutboundMessage, common.SendConfig, ids.ID, subnets.Allower) set.Set[ids.NodeID] { return nil } sendAll() // Send messages to connected peers - externalSender.SendF = func(_ message.OutboundMessage, nodeIDs set.Set[ids.NodeID], _ ids.ID, _ subnets.Allower) set.Set[ids.NodeID] { - return nodeIDs + externalSender.SendF = func(_ message.OutboundMessage, config common.SendConfig, _ ids.ID, _ subnets.Allower) set.Set[ids.NodeID] { + return config.NodeIDs } sendAll() @@ -649,8 +649,10 @@ func TestSender_Bootstrap_Requests(t *testing.T) { setExternalSenderExpect: func(externalSender *MockExternalSender) { externalSender.EXPECT().Send( gomock.Any(), // Outbound message - // Note [myNodeID] is not in this set - set.Of(successNodeID, failedNodeID), + common.SendConfig{ + // Note [myNodeID] is not in this set + NodeIDs: set.Of(successNodeID, failedNodeID), + }, ctx.SubnetID, // Subnet ID gomock.Any(), ).Return(set.Of(successNodeID)) @@ -692,8 +694,10 @@ func TestSender_Bootstrap_Requests(t *testing.T) { setExternalSenderExpect: func(externalSender *MockExternalSender) { externalSender.EXPECT().Send( gomock.Any(), // Outbound message - // Note [myNodeID] is not in this set - set.Of(successNodeID, failedNodeID), + common.SendConfig{ + // Note [myNodeID] is not in this set + NodeIDs: set.Of(successNodeID, failedNodeID), + }, ctx.SubnetID, // Subnet ID gomock.Any(), ).Return(set.Of(successNodeID)) @@ -732,8 +736,10 @@ func TestSender_Bootstrap_Requests(t *testing.T) { setExternalSenderExpect: func(externalSender *MockExternalSender) { externalSender.EXPECT().Send( gomock.Any(), // Outbound message - // Note [myNodeID] is not in this set - set.Of(successNodeID, failedNodeID), + common.SendConfig{ + // Note [myNodeID] is not in this set + NodeIDs: set.Of(successNodeID, failedNodeID), + }, ctx.SubnetID, // Subnet ID gomock.Any(), ).Return(set.Of(successNodeID)) @@ -774,8 +780,10 @@ func TestSender_Bootstrap_Requests(t *testing.T) { setExternalSenderExpect: func(externalSender *MockExternalSender) { externalSender.EXPECT().Send( gomock.Any(), // Outbound message - // Note [myNodeID] is not in this set - set.Of(successNodeID, failedNodeID), + common.SendConfig{ + // Note [myNodeID] is not in this set + NodeIDs: set.Of(successNodeID, failedNodeID), + }, ctx.SubnetID, // Subnet ID gomock.Any(), ).Return(set.Of(successNodeID)) @@ -901,9 +909,11 @@ func TestSender_Bootstrap_Responses(t *testing.T) { }, setExternalSenderExpect: func(externalSender *MockExternalSender) { externalSender.EXPECT().Send( - gomock.Any(), // Outbound message - set.Of(destinationNodeID), // Node IDs - ctx.SubnetID, // Subnet ID + gomock.Any(), // Outbound message + common.SendConfig{ + NodeIDs: set.Of(destinationNodeID), + }, + ctx.SubnetID, // Subnet ID gomock.Any(), ).Return(nil) }, @@ -931,9 +941,11 @@ func TestSender_Bootstrap_Responses(t *testing.T) { }, setExternalSenderExpect: func(externalSender *MockExternalSender) { externalSender.EXPECT().Send( - gomock.Any(), // Outbound message - set.Of(destinationNodeID), // Node IDs - ctx.SubnetID, // Subnet ID + gomock.Any(), // Outbound message + common.SendConfig{ + NodeIDs: set.Of(destinationNodeID), + }, + ctx.SubnetID, // Subnet ID gomock.Any(), ).Return(nil) }, @@ -959,9 +971,11 @@ func TestSender_Bootstrap_Responses(t *testing.T) { }, setExternalSenderExpect: func(externalSender *MockExternalSender) { externalSender.EXPECT().Send( - gomock.Any(), // Outbound message - set.Of(destinationNodeID), // Node IDs - ctx.SubnetID, // Subnet ID + gomock.Any(), // Outbound message + common.SendConfig{ + NodeIDs: set.Of(destinationNodeID), + }, + ctx.SubnetID, // Subnet ID gomock.Any(), ).Return(nil) }, @@ -989,9 +1003,11 @@ func TestSender_Bootstrap_Responses(t *testing.T) { }, setExternalSenderExpect: func(externalSender *MockExternalSender) { externalSender.EXPECT().Send( - gomock.Any(), // Outbound message - set.Of(destinationNodeID), // Node IDs - ctx.SubnetID, // Subnet ID + gomock.Any(), // Outbound message + common.SendConfig{ + NodeIDs: set.Of(destinationNodeID), + }, + ctx.SubnetID, // Subnet ID gomock.Any(), ).Return(nil) }, @@ -1111,8 +1127,10 @@ func TestSender_Single_Request(t *testing.T) { }, setExternalSenderExpect: func(externalSender *MockExternalSender, sentTo set.Set[ids.NodeID]) { externalSender.EXPECT().Send( - gomock.Any(), // Outbound message - set.Of(destinationNodeID), // Node IDs + gomock.Any(), // Outbound message + common.SendConfig{ + NodeIDs: set.Of(destinationNodeID), + }, ctx.SubnetID, gomock.Any(), ).Return(sentTo) @@ -1150,8 +1168,10 @@ func TestSender_Single_Request(t *testing.T) { }, setExternalSenderExpect: func(externalSender *MockExternalSender, sentTo set.Set[ids.NodeID]) { externalSender.EXPECT().Send( - gomock.Any(), // Outbound message - set.Of(destinationNodeID), // Node IDs + gomock.Any(), // Outbound message + common.SendConfig{ + NodeIDs: set.Of(destinationNodeID), + }, ctx.SubnetID, gomock.Any(), ).Return(sentTo) diff --git a/snow/networking/sender/test_external_sender.go b/snow/networking/sender/test_external_sender.go index ae06187216b..3d5e688492b 100644 --- a/snow/networking/sender/test_external_sender.go +++ b/snow/networking/sender/test_external_sender.go @@ -9,39 +9,39 @@ import ( "github.com/ava-labs/avalanchego/ids" "github.com/ava-labs/avalanchego/message" + "github.com/ava-labs/avalanchego/snow/engine/common" "github.com/ava-labs/avalanchego/subnets" "github.com/ava-labs/avalanchego/utils/set" ) var ( - errSend = errors.New("unexpectedly called Send") - errGossip = errors.New("unexpectedly called Gossip") + _ ExternalSender = (*ExternalSenderTest)(nil) + + errSend = errors.New("unexpectedly called Send") ) // ExternalSenderTest is a test sender type ExternalSenderTest struct { TB testing.TB - CantSend, CantGossip bool + CantSend bool - SendF func(msg message.OutboundMessage, nodeIDs set.Set[ids.NodeID], subnetID ids.ID, allower subnets.Allower) set.Set[ids.NodeID] - GossipF func(msg message.OutboundMessage, subnetID ids.ID, numValidatorsToSend, numNonValidatorsToSend, numPeersToSend int, allower subnets.Allower) set.Set[ids.NodeID] + SendF func(msg message.OutboundMessage, config common.SendConfig, subnetID ids.ID, allower subnets.Allower) set.Set[ids.NodeID] } // Default set the default callable value to [cant] func (s *ExternalSenderTest) Default(cant bool) { s.CantSend = cant - s.CantGossip = cant } func (s *ExternalSenderTest) Send( msg message.OutboundMessage, - nodeIDs set.Set[ids.NodeID], + config common.SendConfig, subnetID ids.ID, allower subnets.Allower, ) set.Set[ids.NodeID] { if s.SendF != nil { - return s.SendF(msg, nodeIDs, subnetID, allower) + return s.SendF(msg, config, subnetID, allower) } if s.CantSend { if s.TB != nil { @@ -51,26 +51,3 @@ func (s *ExternalSenderTest) Send( } return nil } - -// Given a msg type, the corresponding mock function is called if it was initialized. -// If it wasn't initialized and this function shouldn't be called and testing was -// initialized, then testing will fail. -func (s *ExternalSenderTest) Gossip( - msg message.OutboundMessage, - subnetID ids.ID, - numValidatorsToSend int, - numNonValidatorsToSend int, - numPeersToSend int, - allower subnets.Allower, -) set.Set[ids.NodeID] { - if s.GossipF != nil { - return s.GossipF(msg, subnetID, numValidatorsToSend, numNonValidatorsToSend, numPeersToSend, allower) - } - if s.CantGossip { - if s.TB != nil { - s.TB.Helper() - s.TB.Fatal(errGossip) - } - } - return nil -} diff --git a/snow/networking/sender/traced_sender.go b/snow/networking/sender/traced_sender.go index 5157b59c8b3..0e25602c84c 100644 --- a/snow/networking/sender/traced_sender.go +++ b/snow/networking/sender/traced_sender.go @@ -258,35 +258,23 @@ func (s *tracedSender) SendAppError(ctx context.Context, nodeID ids.NodeID, requ return s.sender.SendAppError(ctx, nodeID, requestID, errorCode, errorMessage) } -func (s *tracedSender) SendAppGossipSpecific(ctx context.Context, nodeIDs set.Set[ids.NodeID], appGossipBytes []byte) error { - _, span := s.tracer.Start(ctx, "tracedSender.SendAppGossipSpecific", oteltrace.WithAttributes( - attribute.Int("gossipLen", len(appGossipBytes)), - )) - defer span.End() - - return s.sender.SendAppGossipSpecific(ctx, nodeIDs, appGossipBytes) -} - func (s *tracedSender) SendAppGossip( ctx context.Context, + config common.SendConfig, appGossipBytes []byte, - numValidators int, - numNonValidators int, - numPeers int, ) error { _, span := s.tracer.Start(ctx, "tracedSender.SendAppGossip", oteltrace.WithAttributes( + attribute.Int("numNodeIDs", config.NodeIDs.Len()), + attribute.Int("numValidators", config.Validators), + attribute.Int("numNonValidators", config.NonValidators), + attribute.Int("numPeers", config.Peers), attribute.Int("gossipLen", len(appGossipBytes)), - attribute.Int("numValidators", numValidators), - attribute.Int("numNonValidators", numNonValidators), - attribute.Int("numPeers", numPeers), )) defer span.End() return s.sender.SendAppGossip( ctx, + config, appGossipBytes, - numValidators, - numNonValidators, - numPeers, ) } diff --git a/vms/avm/network/network_test.go b/vms/avm/network/network_test.go index 4fdd09b9230..f107b8b389c 100644 --- a/vms/avm/network/network_test.go +++ b/vms/avm/network/network_test.go @@ -307,7 +307,7 @@ func TestNetworkIssueTxFromRPC(t *testing.T) { }, appSenderFunc: func(ctrl *gomock.Controller) common.AppSender { appSender := common.NewMockSender(ctrl) - appSender.EXPECT().SendAppGossip(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) + appSender.EXPECT().SendAppGossip(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) return appSender }, expectedErr: nil, @@ -401,7 +401,7 @@ func TestNetworkIssueTxFromRPCWithoutVerification(t *testing.T) { }, appSenderFunc: func(ctrl *gomock.Controller) common.AppSender { appSender := common.NewMockSender(ctrl) - appSender.EXPECT().SendAppGossip(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) + appSender.EXPECT().SendAppGossip(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) return appSender }, expectedErr: nil, diff --git a/vms/platformvm/block/builder/helpers_test.go b/vms/platformvm/block/builder/helpers_test.go index 05164aff296..4fdf634a190 100644 --- a/vms/platformvm/block/builder/helpers_test.go +++ b/vms/platformvm/block/builder/helpers_test.go @@ -177,7 +177,7 @@ func newEnvironment(t *testing.T, f fork) *environment { //nolint:unparam registerer := prometheus.NewRegistry() res.sender = &common.SenderTest{T: t} - res.sender.SendAppGossipF = func(context.Context, []byte, int, int, int) error { + res.sender.SendAppGossipF = func(context.Context, common.SendConfig, []byte) error { return nil } diff --git a/vms/platformvm/network/network_test.go b/vms/platformvm/network/network_test.go index 98b92801c84..822a4b47878 100644 --- a/vms/platformvm/network/network_test.go +++ b/vms/platformvm/network/network_test.go @@ -279,7 +279,7 @@ func TestNetworkIssueTxFromRPC(t *testing.T) { }, appSenderFunc: func(ctrl *gomock.Controller) common.AppSender { appSender := common.NewMockSender(ctrl) - appSender.EXPECT().SendAppGossip(gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) + appSender.EXPECT().SendAppGossip(gomock.Any(), gomock.Any(), gomock.Any()).Return(nil) return appSender }, expectedErr: nil, diff --git a/vms/platformvm/validator_set_property_test.go b/vms/platformvm/validator_set_property_test.go index 4242fa5f8bd..4ade828c33e 100644 --- a/vms/platformvm/validator_set_property_test.go +++ b/vms/platformvm/validator_set_property_test.go @@ -646,7 +646,7 @@ func buildVM(t *testing.T) (*VM, ids.ID, error) { defer ctx.Lock.Unlock() appSender := &common.SenderTest{} appSender.CantSendAppGossip = true - appSender.SendAppGossipF = func(context.Context, []byte, int, int, int) error { + appSender.SendAppGossipF = func(context.Context, common.SendConfig, []byte) error { return nil } diff --git a/vms/platformvm/vm_test.go b/vms/platformvm/vm_test.go index 23c1c45ba7f..b753f42626c 100644 --- a/vms/platformvm/vm_test.go +++ b/vms/platformvm/vm_test.go @@ -274,7 +274,7 @@ func defaultVM(t *testing.T, f fork) (*VM, database.Database, *mutableSharedMemo _, genesisBytes := defaultGenesis(t, ctx.AVAXAssetID) appSender := &common.SenderTest{} appSender.CantSendAppGossip = true - appSender.SendAppGossipF = func(context.Context, []byte, int, int, int) error { + appSender.SendAppGossipF = func(context.Context, common.SendConfig, []byte) error { return nil } @@ -1503,7 +1503,7 @@ func TestBootstrapPartiallyAccepted(t *testing.T) { ctx.Lock.Lock() var reqID uint32 - externalSender.SendF = func(msg message.OutboundMessage, nodeIDs set.Set[ids.NodeID], _ ids.ID, _ subnets.Allower) set.Set[ids.NodeID] { + externalSender.SendF = func(msg message.OutboundMessage, config common.SendConfig, _ ids.ID, _ subnets.Allower) set.Set[ids.NodeID] { inMsg, err := mc.Parse(msg.Bytes(), ctx.NodeID, func() {}) require.NoError(err) require.Equal(message.GetAcceptedFrontierOp, inMsg.Op()) @@ -1512,24 +1512,24 @@ func TestBootstrapPartiallyAccepted(t *testing.T) { require.True(ok) reqID = requestID - return nodeIDs + return config.NodeIDs } require.NoError(bootstrapper.Connected(context.Background(), peerID, version.CurrentApp)) - externalSender.SendF = func(msg message.OutboundMessage, nodeIDs set.Set[ids.NodeID], _ ids.ID, _ subnets.Allower) set.Set[ids.NodeID] { + externalSender.SendF = func(msg message.OutboundMessage, config common.SendConfig, _ ids.ID, _ subnets.Allower) set.Set[ids.NodeID] { inMsgIntf, err := mc.Parse(msg.Bytes(), ctx.NodeID, func() {}) require.NoError(err) require.Equal(message.GetAcceptedOp, inMsgIntf.Op()) inMsg := inMsgIntf.Message().(*p2p.GetAccepted) reqID = inMsg.RequestId - return nodeIDs + return config.NodeIDs } require.NoError(bootstrapper.AcceptedFrontier(context.Background(), peerID, reqID, advanceTimeBlkID)) - externalSender.SendF = func(msg message.OutboundMessage, nodeIDs set.Set[ids.NodeID], _ ids.ID, _ subnets.Allower) set.Set[ids.NodeID] { + externalSender.SendF = func(msg message.OutboundMessage, config common.SendConfig, _ ids.ID, _ subnets.Allower) set.Set[ids.NodeID] { inMsgIntf, err := mc.Parse(msg.Bytes(), ctx.NodeID, func() {}) require.NoError(err) require.Equal(message.GetAncestorsOp, inMsgIntf.Op()) @@ -1540,13 +1540,13 @@ func TestBootstrapPartiallyAccepted(t *testing.T) { containerID, err := ids.ToID(inMsg.ContainerId) require.NoError(err) require.Equal(advanceTimeBlkID, containerID) - return nodeIDs + return config.NodeIDs } frontier := set.Of(advanceTimeBlkID) require.NoError(bootstrapper.Accepted(context.Background(), peerID, reqID, frontier)) - externalSender.SendF = func(msg message.OutboundMessage, nodeIDs set.Set[ids.NodeID], _ ids.ID, _ subnets.Allower) set.Set[ids.NodeID] { + externalSender.SendF = func(msg message.OutboundMessage, config common.SendConfig, _ ids.ID, _ subnets.Allower) set.Set[ids.NodeID] { inMsg, err := mc.Parse(msg.Bytes(), ctx.NodeID, func() {}) require.NoError(err) require.Equal(message.GetAcceptedFrontierOp, inMsg.Op()) @@ -1555,19 +1555,19 @@ func TestBootstrapPartiallyAccepted(t *testing.T) { require.True(ok) reqID = requestID - return nodeIDs + return config.NodeIDs } require.NoError(bootstrapper.Ancestors(context.Background(), peerID, reqID, [][]byte{advanceTimeBlkBytes})) - externalSender.SendF = func(msg message.OutboundMessage, nodeIDs set.Set[ids.NodeID], _ ids.ID, _ subnets.Allower) set.Set[ids.NodeID] { + externalSender.SendF = func(msg message.OutboundMessage, config common.SendConfig, _ ids.ID, _ subnets.Allower) set.Set[ids.NodeID] { inMsgIntf, err := mc.Parse(msg.Bytes(), ctx.NodeID, func() {}) require.NoError(err) require.Equal(message.GetAcceptedOp, inMsgIntf.Op()) inMsg := inMsgIntf.Message().(*p2p.GetAccepted) reqID = inMsg.RequestId - return nodeIDs + return config.NodeIDs } require.NoError(bootstrapper.AcceptedFrontier(context.Background(), peerID, reqID, advanceTimeBlkID))