[NOD-614] Add ban score (#760)

* [NOD-614] Copy bitcoin-core ban score policy

* [NOD-614] Add ban score to disconnects

* [NOD-614] Fix wrong branch of AddBanScore

* [NOD-614] Add ban score on sending too many addresses

* [NOD-614] Add comments

* [NOD-614] Remove redundant reject messages

* [NOD-614] Fix log message

* [NOD-614] Ban every node that sends invalid invs

* [NOD-614] Make constants for ban scores
This commit is contained in:
Ori Newman 2020-06-15 12:12:38 +03:00 committed by GitHub
parent 829979b6c7
commit d4c9fdf6ac
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
14 changed files with 148 additions and 76 deletions

View File

@ -162,10 +162,10 @@ const (
// to a getAddr. If we have less than this amount, we send everything. // to a getAddr. If we have less than this amount, we send everything.
getAddrMin = 50 getAddrMin = 50
// getAddrMax is the most addresses that we will send in response // GetAddrMax is the most addresses that we will send in response
// to a getAddr (in practise the most addresses we will return from a // to a getAddr (in practise the most addresses we will return from a
// call to AddressCache()). // call to AddressCache()).
getAddrMax = 2500 GetAddrMax = 2500
// getAddrPercent is the percentage of total addresses known that we // getAddrPercent is the percentage of total addresses known that we
// will share with a call to AddressCache. // will share with a call to AddressCache.
@ -839,8 +839,8 @@ func (a *AddrManager) AddressCache(includeAllSubnetworks bool, subnetworkID *sub
} }
numAddresses := len(allAddr) * getAddrPercent / 100 numAddresses := len(allAddr) * getAddrPercent / 100
if numAddresses > getAddrMax { if numAddresses > GetAddrMax {
numAddresses = getAddrMax numAddresses = GetAddrMax
} }
if len(allAddr) < getAddrMin { if len(allAddr) < getAddrMin {
numAddresses = len(allAddr) numAddresses = len(allAddr)

View File

@ -165,7 +165,7 @@ type BlockDAG struct {
// //
// This function is safe for concurrent access. // This function is safe for concurrent access.
func (dag *BlockDAG) IsKnownBlock(hash *daghash.Hash) bool { func (dag *BlockDAG) IsKnownBlock(hash *daghash.Hash) bool {
return dag.IsInDAG(hash) || dag.IsKnownOrphan(hash) || dag.isKnownDelayedBlock(hash) return dag.IsInDAG(hash) || dag.IsKnownOrphan(hash) || dag.isKnownDelayedBlock(hash) || dag.IsKnownInvalid(hash)
} }
// AreKnownBlocks returns whether or not the DAG instances has all blocks represented // AreKnownBlocks returns whether or not the DAG instances has all blocks represented

View File

@ -376,9 +376,8 @@ func (sm *SyncManager) handleTxMsg(tmsg *txMsg) {
// If we didn't ask for this transaction then the peer is misbehaving. // If we didn't ask for this transaction then the peer is misbehaving.
txID := tmsg.tx.ID() txID := tmsg.tx.ID()
if _, exists = state.requestedTxns[*txID]; !exists { if _, exists = state.requestedTxns[*txID]; !exists {
log.Warnf("Got unrequested transaction %s from %s -- "+ peer.AddBanScoreAndPushRejectMsg(wire.CmdTx, wire.RejectNotRequested, (*daghash.Hash)(txID),
"disconnecting", txID, peer.Addr()) peerpkg.BanScoreUnrequestedTx, 0, fmt.Sprintf("got unrequested transaction %s", txID))
peer.Disconnect()
return return
} }
@ -412,19 +411,25 @@ func (sm *SyncManager) handleTxMsg(tmsg *txMsg) {
// When the error is a rule error, it means the transaction was // When the error is a rule error, it means the transaction was
// simply rejected as opposed to something actually going wrong, // simply rejected as opposed to something actually going wrong,
// so log it as such. Otherwise, something really did go wrong, // so log it as such. Otherwise, something really did go wrong,
// so log it as an actual error. // so panic.
if errors.As(err, &mempool.RuleError{}) { ruleErr := &mempool.RuleError{}
log.Debugf("Rejected transaction %s from %s: %s", if !errors.As(err, ruleErr) {
txID, peer, err) panic(errors.Wrapf(err, "failed to process transaction %s", txID))
} else {
log.Errorf("Failed to process transaction %s: %s",
txID, err)
} }
// Convert the error into an appropriate reject message and shouldIncreaseBanScore := false
// send it. if txRuleErr := (&mempool.TxRuleError{}); errors.As(ruleErr.Err, txRuleErr) {
code, reason := mempool.ErrToRejectErr(err) if txRuleErr.RejectCode == wire.RejectInvalid {
peer.PushRejectMsg(wire.CmdTx, code, reason, (*daghash.Hash)(txID), false) shouldIncreaseBanScore = true
}
} else if dagRuleErr := (&blockdag.RuleError{}); errors.As(ruleErr.Err, dagRuleErr) {
shouldIncreaseBanScore = true
}
if shouldIncreaseBanScore {
peer.AddBanScoreAndPushRejectMsg(wire.CmdTx, wire.RejectInvalid, (*daghash.Hash)(txID),
peerpkg.BanScoreInvalidTx, 0, fmt.Sprintf("rejected transaction %s: %s", txID, err))
}
return return
} }
@ -480,9 +485,8 @@ func (sm *SyncManager) handleBlockMsg(bmsg *blockMsg) {
// mode in this case so the DAG code is actually fed the // mode in this case so the DAG code is actually fed the
// duplicate blocks. // duplicate blocks.
if sm.dagParams != &dagconfig.RegressionNetParams { if sm.dagParams != &dagconfig.RegressionNetParams {
log.Warnf("Got unrequested block %s from %s -- "+ peer.AddBanScoreAndPushRejectMsg(wire.CmdBlock, wire.RejectNotRequested, blockHash,
"disconnecting", blockHash, peer.Addr()) peerpkg.BanScoreUnrequestedBlock, 0, fmt.Sprintf("got unrequested block %s", blockHash))
peer.Disconnect()
return return
} }
} }
@ -518,13 +522,8 @@ func (sm *SyncManager) handleBlockMsg(bmsg *blockMsg) {
log.Infof("Rejected block %s from %s: %s", blockHash, log.Infof("Rejected block %s from %s: %s", blockHash,
peer, err) peer, err)
// Convert the error into an appropriate reject message and peer.AddBanScoreAndPushRejectMsg(wire.CmdBlock, wire.RejectInvalid, blockHash,
// send it. peerpkg.BanScoreInvalidBlock, 0, fmt.Sprintf("got invalid block: %s", err))
code, reason := mempool.ErrToRejectErr(err)
peer.PushRejectMsg(wire.CmdBlock, code, reason, blockHash, false)
// Disconnect from the misbehaving peer.
peer.Disconnect()
return return
} }
@ -718,6 +717,10 @@ func (sm *SyncManager) handleInvMsg(imsg *invMsg) {
} }
if iv.IsBlockOrSyncBlock() { if iv.IsBlockOrSyncBlock() {
if sm.dag.IsKnownInvalid(iv.Hash) {
peer.AddBanScoreAndPushRejectMsg(imsg.inv.Command(), wire.RejectInvalid, iv.Hash, peerpkg.BanScoreInvalidInvBlock, 0, fmt.Sprintf("sent inv of invalid block %s", iv.Hash))
return
}
// The block is an orphan block that we already have. // The block is an orphan block that we already have.
// When the existing orphan was processed, it requested // When the existing orphan was processed, it requested
// the missing parent blocks. When this scenario // the missing parent blocks. When this scenario
@ -913,9 +916,9 @@ func (sm *SyncManager) handleSelectedTipMsg(msg *selectedTipMsg) {
selectedTipHash := msg.selectedTipHash selectedTipHash := msg.selectedTipHash
state := sm.peerStates[peer] state := sm.peerStates[peer]
if !state.peerShouldSendSelectedTip { if !state.peerShouldSendSelectedTip {
log.Warnf("Got unrequested selected tip message from %s -- "+ peer.AddBanScoreAndPushRejectMsg(wire.CmdSelectedTip, wire.RejectNotRequested, nil,
"disconnecting", peer.Addr()) peerpkg.BanScoreUnrequestedSelectedTip, 0, "got unrequested selected tip message")
peer.Disconnect() return
} }
state.peerShouldSendSelectedTip = false state.peerShouldSendSelectedTip = false
if selectedTipHash.IsEqual(peer.SelectedTipHash()) { if selectedTipHash.IsEqual(peer.SelectedTipHash()) {

34
peer/banscores.go Normal file
View File

@ -0,0 +1,34 @@
package peer
// Ban scores for misbehaving nodes
const (
BanScoreUnrequestedBlock = 100
BanScoreInvalidBlock = 100
BanScoreInvalidInvBlock = 100
BanScoreUnrequestedSelectedTip = 20
BanScoreUnrequestedTx = 20
BanScoreInvalidTx = 100
BanScoreMalformedMessage = 10
BanScoreNonVersionFirstMessage = 1
BanScoreDuplicateVersion = 1
BanScoreDuplicateVerack = 1
BanScoreSentTooManyAddresses = 20
BanScoreMsgAddrWithInvalidSubnetwork = 10
BanScoreInvalidFeeFilter = 100
BanScoreNoFilterLoaded = 5
BanScoreInvalidMsgGetBlockInvs = 10
BanScoreInvalidMsgBlockLocator = 100
BanScoreSentTxToBlocksOnly = 20
BanScoreNodeBloomFlagViolation = 100
BanScoreStallTimeout = 1
)

View File

@ -199,6 +199,13 @@ type Config struct {
// the DAG. // the DAG.
IsInDAG func(*daghash.Hash) bool IsInDAG func(*daghash.Hash) bool
// AddBanScore increases the persistent and decaying ban score fields by the
// values passed as parameters. If the resulting score exceeds half of the ban
// threshold, a warning is logged including the reason provided. Further, if
// the score is above the ban threshold, the peer will be banned and
// disconnected.
AddBanScore func(persistent, transient uint32, reason string)
// HostToNetAddress returns the netaddress for the given host. This can be // HostToNetAddress returns the netaddress for the given host. This can be
// nil in which case the host will be parsed as an IP address. // nil in which case the host will be parsed as an IP address.
HostToNetAddress HostToNetAddrFunc HostToNetAddress HostToNetAddrFunc
@ -646,6 +653,22 @@ func (p *Peer) IsSelectedTipKnown() bool {
return !p.cfg.IsInDAG(p.selectedTipHash) return !p.cfg.IsInDAG(p.selectedTipHash)
} }
// AddBanScore increases the persistent and decaying ban score fields by the
// values passed as parameters. If the resulting score exceeds half of the ban
// threshold, a warning is logged including the reason provided. Further, if
// the score is above the ban threshold, the peer will be banned and
// disconnected.
func (p *Peer) AddBanScore(persistent, transient uint32, reason string) {
p.cfg.AddBanScore(persistent, transient, reason)
}
// AddBanScoreAndPushRejectMsg increases ban score and sends a
// reject message to the misbehaving peer.
func (p *Peer) AddBanScoreAndPushRejectMsg(command string, code wire.RejectCode, hash *daghash.Hash, persistent, transient uint32, reason string) {
p.PushRejectMsg(command, code, reason, hash, true)
p.cfg.AddBanScore(persistent, transient, reason)
}
// LastSend returns the last send time of the peer. // LastSend returns the last send time of the peer.
// //
// This function is safe for concurrent access. // This function is safe for concurrent access.
@ -1239,9 +1262,7 @@ out:
continue continue
} }
log.Debugf("Peer %s appears to be stalled or "+ p.AddBanScore(BanScoreStallTimeout, 0, fmt.Sprintf("got timeout for command %s", command))
"misbehaving, %s timeout -- "+
"disconnecting", p, command)
p.Disconnect() p.Disconnect()
break break
} }
@ -1316,15 +1337,15 @@ out:
log.Errorf(errMsg) log.Errorf(errMsg)
} }
// Push a reject message for the malformed message and wait for // Add ban score, push a reject message for the malformed message
// the message to be sent before disconnecting. // and wait for the message to be sent before disconnecting.
// //
// NOTE: Ideally this would include the command in the header if // NOTE: Ideally this would include the command in the header if
// at least that much of the message was valid, but that is not // at least that much of the message was valid, but that is not
// currently exposed by wire, so just used malformed for the // currently exposed by wire, so just used malformed for the
// command. // command.
p.PushRejectMsg("malformed", wire.RejectMalformed, errMsg, nil, p.AddBanScoreAndPushRejectMsg("malformed", wire.RejectMalformed, nil,
true) BanScoreMalformedMessage, 0, errMsg)
} }
break out break out
} }
@ -1336,18 +1357,18 @@ out:
switch msg := rmsg.(type) { switch msg := rmsg.(type) {
case *wire.MsgVersion: case *wire.MsgVersion:
p.PushRejectMsg(msg.Command(), wire.RejectDuplicate, reason := "duplicate version message"
"duplicate version message", nil, true) p.AddBanScoreAndPushRejectMsg(msg.Command(), wire.RejectDuplicate, nil,
break out BanScoreDuplicateVersion, 0, reason)
case *wire.MsgVerAck: case *wire.MsgVerAck:
// No read lock is necessary because verAckReceived is not written // No read lock is necessary because verAckReceived is not written
// to in any other goroutine. // to in any other goroutine.
if p.verAckReceived { if p.verAckReceived {
log.Infof("Already received 'verack' from peer %s -- "+ p.AddBanScoreAndPushRejectMsg(msg.Command(), wire.RejectDuplicate, nil,
"disconnecting", p) BanScoreDuplicateVerack, 0, "verack sent twice")
break out log.Warnf("Already received 'verack' from peer %s", p)
} }
p.markVerAckReceived() p.markVerAckReceived()
if p.cfg.Listeners.OnVerAck != nil { if p.cfg.Listeners.OnVerAck != nil {
@ -1867,6 +1888,8 @@ func (p *Peer) readRemoteVersionMsg() error {
errStr := "A version message must precede all others" errStr := "A version message must precede all others"
log.Errorf(errStr) log.Errorf(errStr)
p.AddBanScore(BanScoreNonVersionFirstMessage, 0, errStr)
rejectMsg := wire.NewMsgReject(msg.Command(), wire.RejectMalformed, rejectMsg := wire.NewMsgReject(msg.Command(), wire.RejectMalformed,
errStr) errStr)
return p.writeMessage(rejectMsg) return p.writeMessage(rejectMsg)

View File

@ -1,6 +1,8 @@
package p2p package p2p
import ( import (
"fmt"
"github.com/kaspanet/kaspad/addrmgr"
"github.com/kaspanet/kaspad/config" "github.com/kaspanet/kaspad/config"
"github.com/kaspanet/kaspad/peer" "github.com/kaspanet/kaspad/peer"
"github.com/kaspanet/kaspad/wire" "github.com/kaspanet/kaspad/wire"
@ -18,10 +20,16 @@ func (sp *Peer) OnAddr(_ *peer.Peer, msg *wire.MsgAddr) {
return return
} }
if len(msg.AddrList) > addrmgr.GetAddrMax {
sp.AddBanScoreAndPushRejectMsg(msg.Command(), wire.RejectInvalid, nil,
peer.BanScoreSentTooManyAddresses, 0, fmt.Sprintf("address count excceeded %d", addrmgr.GetAddrMax))
return
}
if msg.IncludeAllSubnetworks { if msg.IncludeAllSubnetworks {
peerLog.Errorf("Got unexpected IncludeAllSubnetworks=true in [%s] command from %s", sp.AddBanScoreAndPushRejectMsg(msg.Command(), wire.RejectInvalid, nil,
msg.Command(), sp.Peer) peer.BanScoreMsgAddrWithInvalidSubnetwork, 0,
sp.Disconnect() fmt.Sprintf("got unexpected IncludeAllSubnetworks=true in [%s] command", msg.Command()))
return return
} else if !msg.SubnetworkID.IsEqual(config.ActiveConfig().SubnetworkID) && msg.SubnetworkID != nil { } else if !msg.SubnetworkID.IsEqual(config.ActiveConfig().SubnetworkID) && msg.SubnetworkID != nil {
peerLog.Errorf("Only full nodes and %s subnetwork IDs are allowed in [%s] command, but got subnetwork ID %s from %s", peerLog.Errorf("Only full nodes and %s subnetwork IDs are allowed in [%s] command, but got subnetwork ID %s from %s",

View File

@ -1,6 +1,7 @@
package p2p package p2p
import ( import (
"fmt"
"github.com/kaspanet/kaspad/peer" "github.com/kaspanet/kaspad/peer"
"github.com/kaspanet/kaspad/util" "github.com/kaspanet/kaspad/util"
"github.com/kaspanet/kaspad/wire" "github.com/kaspanet/kaspad/wire"
@ -14,9 +15,8 @@ import (
func (sp *Peer) OnFeeFilter(_ *peer.Peer, msg *wire.MsgFeeFilter) { func (sp *Peer) OnFeeFilter(_ *peer.Peer, msg *wire.MsgFeeFilter) {
// Check that the passed minimum fee is a valid amount. // Check that the passed minimum fee is a valid amount.
if msg.MinFee < 0 || msg.MinFee > util.MaxSompi { if msg.MinFee < 0 || msg.MinFee > util.MaxSompi {
peerLog.Debugf("Peer %s sent an invalid feefilter '%s' -- "+ sp.AddBanScoreAndPushRejectMsg(msg.Command(), wire.RejectInvalid, nil,
"disconnecting", sp, util.Amount(msg.MinFee)) peer.BanScoreInvalidFeeFilter, 0, fmt.Sprintf("sent an invalid feefilter '%s'", util.Amount(msg.MinFee)))
sp.Disconnect()
return return
} }

View File

@ -17,9 +17,8 @@ func (sp *Peer) OnFilterAdd(_ *peer.Peer, msg *wire.MsgFilterAdd) {
} }
if sp.filter.IsLoaded() { if sp.filter.IsLoaded() {
peerLog.Debugf("%s sent a filteradd request with no filter "+ sp.AddBanScoreAndPushRejectMsg(wire.CmdFilterAdd, wire.RejectInvalid, nil,
"loaded -- disconnecting", sp) peer.BanScoreNoFilterLoaded, 0, "sent a filteradd request with no filter loaded")
sp.Disconnect()
return return
} }

View File

@ -17,9 +17,8 @@ func (sp *Peer) OnFilterClear(_ *peer.Peer, msg *wire.MsgFilterClear) {
} }
if !sp.filter.IsLoaded() { if !sp.filter.IsLoaded() {
peerLog.Debugf("%s sent a filterclear request with no "+ sp.AddBanScoreAndPushRejectMsg(wire.CmdFilterClear, wire.RejectInvalid, nil,
"filter loaded -- disconnecting", sp) peer.BanScoreNoFilterLoaded, 0, "sent a filterclear request with no filter loaded")
sp.Disconnect()
return return
} }

View File

@ -1,6 +1,7 @@
package p2p package p2p
import ( import (
"fmt"
"github.com/kaspanet/kaspad/peer" "github.com/kaspanet/kaspad/peer"
"github.com/kaspanet/kaspad/wire" "github.com/kaspanet/kaspad/wire"
) )
@ -23,8 +24,9 @@ func (sp *Peer) OnGetBlockInvs(_ *peer.Peer, msg *wire.MsgGetBlockInvs) {
hashList, err := dag.AntiPastHashesBetween(msg.LowHash, msg.HighHash, hashList, err := dag.AntiPastHashesBetween(msg.LowHash, msg.HighHash,
wire.MaxInvPerMsg) wire.MaxInvPerMsg)
if err != nil { if err != nil {
peerLog.Warnf("Error getting antiPast hashes between %s and %s: %s", msg.LowHash, msg.HighHash, err) sp.AddBanScoreAndPushRejectMsg(wire.CmdGetBlockInvs, wire.RejectInvalid, nil,
sp.Disconnect() peer.BanScoreInvalidMsgGetBlockInvs, 0,
fmt.Sprintf("error getting antiPast hashes between %s and %s: %s", msg.LowHash, msg.HighHash, err))
return return
} }

View File

@ -11,13 +11,13 @@ import (
func (sp *Peer) OnGetBlockLocator(_ *peer.Peer, msg *wire.MsgGetBlockLocator) { func (sp *Peer) OnGetBlockLocator(_ *peer.Peer, msg *wire.MsgGetBlockLocator) {
locator, err := sp.server.DAG.BlockLocatorFromHashes(msg.HighHash, msg.LowHash) locator, err := sp.server.DAG.BlockLocatorFromHashes(msg.HighHash, msg.LowHash)
if err != nil || len(locator) == 0 { if err != nil || len(locator) == 0 {
warning := fmt.Sprintf("Couldn't build a block locator between blocks "+
"%s and %s that was requested from peer %s", msg.HighHash, msg.LowHash, sp)
if err != nil { if err != nil {
warning = fmt.Sprintf("%s: %s", warning, err) peerLog.Warnf("Couldn't build a block locator between blocks "+
"%s and %s that was requested from peer %s: %s", msg.HighHash, msg.LowHash, sp, err)
} }
peerLog.Warnf(warning) sp.AddBanScoreAndPushRejectMsg(msg.Command(), wire.RejectInvalid, nil,
sp.Disconnect() peer.BanScoreInvalidMsgBlockLocator, 0,
fmt.Sprintf("couldn't build a block locator between blocks %s and %s", msg.HighHash, msg.LowHash))
return return
} }

View File

@ -23,9 +23,8 @@ func (sp *Peer) OnInv(_ *peer.Peer, msg *wire.MsgInv) {
if invVect.Type == wire.InvTypeTx { if invVect.Type == wire.InvTypeTx {
peerLog.Tracef("Ignoring tx %s in inv from %s -- "+ peerLog.Tracef("Ignoring tx %s in inv from %s -- "+
"blocksonly enabled", invVect.Hash, sp) "blocksonly enabled", invVect.Hash, sp)
peerLog.Infof("Peer %s is announcing "+ sp.AddBanScoreAndPushRejectMsg(msg.Command(), wire.RejectNotRequested, invVect.Hash,
"transactions -- disconnecting", sp) peer.BanScoreSentTxToBlocksOnly, 0, "announced transactions when blocksonly is enabled")
sp.Disconnect()
return return
} }
err := newInv.AddInvVect(invVect) err := newInv.AddInvVect(invVect)

View File

@ -8,6 +8,7 @@ package p2p
import ( import (
"crypto/rand" "crypto/rand"
"encoding/binary" "encoding/binary"
"fmt"
"math" "math"
"net" "net"
"runtime" "runtime"
@ -328,13 +329,8 @@ func (sp *Peer) pushAddrMsg(addresses []*wire.NetAddress, subnetworkID *subnetwo
// the score is above the ban threshold, the peer will be banned and // the score is above the ban threshold, the peer will be banned and
// disconnected. // disconnected.
func (sp *Peer) addBanScore(persistent, transient uint32, reason string) { func (sp *Peer) addBanScore(persistent, transient uint32, reason string) {
// No warning is logged and no score is calculated if banning is disabled.
if config.ActiveConfig().DisableBanning {
return
}
if sp.isWhitelisted { if sp.isWhitelisted {
peerLog.Debugf("Misbehaving whitelisted peer %s: %s", sp, reason) peerLog.Debugf("Misbehaving whitelisted peer %s: %s", sp, reason)
return
} }
warnThreshold := config.ActiveConfig().BanThreshold >> 1 warnThreshold := config.ActiveConfig().BanThreshold >> 1
@ -348,16 +344,22 @@ func (sp *Peer) addBanScore(persistent, transient uint32, reason string) {
} }
return return
} }
score := sp.DynamicBanScore.Increase(persistent, transient) score := sp.DynamicBanScore.Increase(persistent, transient)
if score > warnThreshold { logMsg := fmt.Sprintf("Misbehaving peer %s: %s -- ban score increased to %d",
peerLog.Warnf("Misbehaving peer %s: %s -- ban score increased to %d",
sp, reason, score) sp, reason, score)
if score > config.ActiveConfig().BanThreshold { if score > warnThreshold {
peerLog.Warn(logMsg)
if !config.ActiveConfig().DisableBanning && !sp.isWhitelisted && score > config.ActiveConfig().BanThreshold {
peerLog.Warnf("Misbehaving peer %s -- banning and disconnecting", peerLog.Warnf("Misbehaving peer %s -- banning and disconnecting",
sp) sp)
sp.server.BanPeer(sp) sp.server.BanPeer(sp)
sp.Disconnect() sp.Disconnect()
} }
} else if persistent != 0 {
peerLog.Warn(logMsg)
} else {
peerLog.Trace(logMsg)
} }
} }
@ -375,7 +377,7 @@ func (sp *Peer) enforceNodeBloomFlag(cmd string) bool {
// Disconnect the peer regardless of whether it was // Disconnect the peer regardless of whether it was
// banned. // banned.
sp.addBanScore(100, 0, cmd) sp.addBanScore(peer.BanScoreNodeBloomFlagViolation, 0, cmd)
sp.Disconnect() sp.Disconnect()
return false return false
} }
@ -937,6 +939,7 @@ func newPeerConfig(sp *Peer) *peer.Config {
}, },
SelectedTipHash: sp.selectedTipHash, SelectedTipHash: sp.selectedTipHash,
IsInDAG: sp.blockExists, IsInDAG: sp.blockExists,
AddBanScore: sp.addBanScore,
HostToNetAddress: sp.server.addrManager.HostToNetAddress, HostToNetAddress: sp.server.addrManager.HostToNetAddress,
Proxy: config.ActiveConfig().Proxy, Proxy: config.ActiveConfig().Proxy,
UserAgentName: userAgentName, UserAgentName: userAgentName,

View File

@ -21,6 +21,7 @@ const (
RejectInvalid RejectCode = 0x10 RejectInvalid RejectCode = 0x10
RejectObsolete RejectCode = 0x11 RejectObsolete RejectCode = 0x11
RejectDuplicate RejectCode = 0x12 RejectDuplicate RejectCode = 0x12
RejectNotRequested RejectCode = 0x13
RejectNonstandard RejectCode = 0x40 RejectNonstandard RejectCode = 0x40
RejectDust RejectCode = 0x41 RejectDust RejectCode = 0x41
RejectInsufficientFee RejectCode = 0x42 RejectInsufficientFee RejectCode = 0x42
@ -39,6 +40,7 @@ var rejectCodeStrings = map[RejectCode]string{
RejectInsufficientFee: "REJECT_INSUFFICIENTFEE", RejectInsufficientFee: "REJECT_INSUFFICIENTFEE",
RejectFinality: "REJECT_FINALITY", RejectFinality: "REJECT_FINALITY",
RejectDifficulty: "REJECT_DIFFICULTY", RejectDifficulty: "REJECT_DIFFICULTY",
RejectNotRequested: "REJECT_NOTREQUESTED",
} }
// String returns the RejectCode in human-readable form. // String returns the RejectCode in human-readable form.