package pubsub

import (
	
	
	
	

	
)

type PeerScoreThresholds struct {
	// whether it is allowed to just set some params and not all of them.
	SkipAtomicValidation bool

	// GossipThreshold is the score threshold below which gossip propagation is suppressed;
	// should be negative.
	GossipThreshold float64

	// PublishThreshold is the score threshold below which we shouldn't publish when using flood
	// publishing (also applies to fanout and floodsub peers); should be negative and <= GossipThreshold.
	PublishThreshold float64

	// GraylistThreshold is the score threshold below which message processing is suppressed altogether,
	// implementing an effective gray list according to peer score; should be negative and <= PublishThreshold.
	GraylistThreshold float64

	// AcceptPXThreshold is the score threshold below which PX will be ignored; this should be positive
	// and limited to scores attainable by bootstrappers and other trusted nodes.
	AcceptPXThreshold float64

	// OpportunisticGraftThreshold is the median mesh score threshold before triggering opportunistic
	// grafting; this should have a small positive value.
	OpportunisticGraftThreshold float64
}

func ( *PeerScoreThresholds) () error {

	if !.SkipAtomicValidation || .PublishThreshold != 0 || .GossipThreshold != 0 || .GraylistThreshold != 0 {
		if .GossipThreshold > 0 || isInvalidNumber(.GossipThreshold) {
			return fmt.Errorf("invalid gossip threshold; it must be <= 0 and a valid number")
		}
		if .PublishThreshold > 0 || .PublishThreshold > .GossipThreshold || isInvalidNumber(.PublishThreshold) {
			return fmt.Errorf("invalid publish threshold; it must be <= 0 and <= gossip threshold and a valid number")
		}
		if .GraylistThreshold > 0 || .GraylistThreshold > .PublishThreshold || isInvalidNumber(.GraylistThreshold) {
			return fmt.Errorf("invalid graylist threshold; it must be <= 0 and <= publish threshold and a valid number")
		}
	}

	if !.SkipAtomicValidation || .AcceptPXThreshold != 0 {
		if .AcceptPXThreshold < 0 || isInvalidNumber(.AcceptPXThreshold) {
			return fmt.Errorf("invalid accept PX threshold; it must be >= 0 and a valid number")
		}
	}

	if !.SkipAtomicValidation || .OpportunisticGraftThreshold != 0 {
		if .OpportunisticGraftThreshold < 0 || isInvalidNumber(.OpportunisticGraftThreshold) {
			return fmt.Errorf("invalid opportunistic grafting threshold; it must be >= 0 and a valid number")
		}
	}

	return nil
}

type PeerScoreParams struct {
	// whether it is allowed to just set some params and not all of them.
	SkipAtomicValidation bool

	// Score parameters per topic.
	Topics map[string]*TopicScoreParams

	// Aggregate topic score cap; this limits the total contribution of topics towards a positive
	// score. It must be positive (or 0 for no cap).
	TopicScoreCap float64

	// P5: Application-specific peer scoring
	AppSpecificScore  func(p peer.ID) float64
	AppSpecificWeight float64

	// P6: IP-colocation factor.
	// The parameter has an associated counter which counts the number of peers with the same IP.
	// If the number of peers in the same IP exceeds IPColocationFactorThreshold, then the value
	// is the square of the difference, ie (PeersInSameIP - IPColocationThreshold)^2.
	// If the number of peers in the same IP is less than the threshold, then the value is 0.
	// The weight of the parameter MUST be negative, unless you want to disable for testing.
	// Note: In order to simulate many IPs in a managable manner when testing, you can set the weight to 0
	//       thus disabling the IP colocation penalty.
	IPColocationFactorWeight    float64
	IPColocationFactorThreshold int
	IPColocationFactorWhitelist []*net.IPNet

	// P7: behavioural pattern penalties.
	// This parameter has an associated counter which tracks misbehaviour as detected by the
	// router. The router currently applies penalties for the following behaviors:
	// - attempting to re-graft before the prune backoff time has elapsed.
	// - not following up in IWANT requests for messages advertised with IHAVE.
	//
	// The value of the parameter is the square of the counter over the threshold, which decays with
	// BehaviourPenaltyDecay.
	// The weight of the parameter MUST be negative (or zero to disable).
	BehaviourPenaltyWeight, BehaviourPenaltyThreshold, BehaviourPenaltyDecay float64

	// the decay interval for parameter counters.
	DecayInterval time.Duration

	// counter value below which it is considered 0.
	DecayToZero float64

	// time to remember counters for a disconnected peer.
	RetainScore time.Duration

	// time to remember a message delivery for. Default to global TimeCacheDuration if 0.
	SeenMsgTTL time.Duration
}

type TopicScoreParams struct {
	// whether it is allowed to just set some params and not all of them.
	SkipAtomicValidation bool

	// The weight of the topic.
	TopicWeight float64

	// P1: time in the mesh
	// This is the time the peer has been grafted in the mesh.
	// The value of the parameter is the time/TimeInMeshQuantum, capped by TimeInMeshCap.
	// The weight of the parameter MUST be positive (or zero to disable).
	TimeInMeshWeight  float64
	TimeInMeshQuantum time.Duration
	TimeInMeshCap     float64

	// P2: first message deliveries
	// This is the number of message deliveries in the topic.
	// The value of the parameter is a counter, decaying with FirstMessageDeliveriesDecay, and capped
	// by FirstMessageDeliveriesCap.
	// The weight of the parameter MUST be positive (or zero to disable).
	FirstMessageDeliveriesWeight, FirstMessageDeliveriesDecay float64
	FirstMessageDeliveriesCap                                 float64

	// P3: mesh message deliveries
	// This is the number of message deliveries in the mesh, within the MeshMessageDeliveriesWindow of
	// message validation; deliveries during validation also count and are retroactively applied
	// when validation succeeds.
	// This window accounts for the minimum time before a hostile mesh peer trying to game the score
	// could replay back a valid message we just sent them.
	// It effectively tracks first and near-first deliveries, i.e., a message seen from a mesh peer
	// before we have forwarded it to them.
	// The parameter has an associated counter, decaying with MeshMessageDeliveriesDecay.
	// If the counter exceeds the threshold, its value is 0.
	// If the counter is below the MeshMessageDeliveriesThreshold, the value is the square of
	// the deficit, ie (MessageDeliveriesThreshold - counter)^2
	// The penalty is only activated after MeshMessageDeliveriesActivation time in the mesh.
	// The weight of the parameter MUST be negative (or zero to disable).
	MeshMessageDeliveriesWeight, MeshMessageDeliveriesDecay      float64
	MeshMessageDeliveriesCap, MeshMessageDeliveriesThreshold     float64
	MeshMessageDeliveriesWindow, MeshMessageDeliveriesActivation time.Duration

	// P3b: sticky mesh propagation failures
	// This is a sticky penalty that applies when a peer gets pruned from the mesh with an active
	// mesh message delivery penalty.
	// The weight of the parameter MUST be negative (or zero to disable)
	MeshFailurePenaltyWeight, MeshFailurePenaltyDecay float64

	// P4: invalid messages
	// This is the number of invalid messages in the topic.
	// The value of the parameter is the square of the counter, decaying with
	// InvalidMessageDeliveriesDecay.
	// The weight of the parameter MUST be negative (or zero to disable).
	InvalidMessageDeliveriesWeight, InvalidMessageDeliveriesDecay float64
}

// peer score parameter validation
func ( *PeerScoreParams) () error {
	for ,  := range .Topics {
		 := .validate()
		if  != nil {
			return fmt.Errorf("invalid score parameters for topic %s: %w", , )
		}
	}

	if !.SkipAtomicValidation || .TopicScoreCap != 0 {
		// check that the topic score is 0 or something positive
		if .TopicScoreCap < 0 || isInvalidNumber(.TopicScoreCap) {
			return fmt.Errorf("invalid topic score cap; must be positive (or 0 for no cap) and a valid number")
		}
	}

	// check that we have an app specific score; the weight can be anything (but expected positive)
	if .AppSpecificScore == nil {
		if .SkipAtomicValidation {
			.AppSpecificScore = func( peer.ID) float64 {
				return 0
			}
		} else {
			return fmt.Errorf("missing application specific score function")
		}
	}

	if !.SkipAtomicValidation || .IPColocationFactorWeight != 0 {
		// check the IP collocation factor
		if .IPColocationFactorWeight > 0 || isInvalidNumber(.IPColocationFactorWeight) {
			return fmt.Errorf("invalid IPColocationFactorWeight; must be negative (or 0 to disable) and a valid number")
		}
		if .IPColocationFactorWeight != 0 && .IPColocationFactorThreshold < 1 {
			return fmt.Errorf("invalid IPColocationFactorThreshold; must be at least 1")
		}
	}

	// check the behaviour penalty
	if !.SkipAtomicValidation || .BehaviourPenaltyWeight != 0 || .BehaviourPenaltyThreshold != 0 {
		if .BehaviourPenaltyWeight > 0 || isInvalidNumber(.BehaviourPenaltyWeight) {
			return fmt.Errorf("invalid BehaviourPenaltyWeight; must be negative (or 0 to disable) and a valid number")
		}
		if .BehaviourPenaltyWeight != 0 && (.BehaviourPenaltyDecay <= 0 || .BehaviourPenaltyDecay >= 1 || isInvalidNumber(.BehaviourPenaltyDecay)) {
			return fmt.Errorf("invalid BehaviourPenaltyDecay; must be between 0 and 1")
		}
		if .BehaviourPenaltyThreshold < 0 || isInvalidNumber(.BehaviourPenaltyThreshold) {
			return fmt.Errorf("invalid BehaviourPenaltyThreshold; must be >= 0 and a valid number")
		}
	}

	// check the decay parameters
	if !.SkipAtomicValidation || .DecayInterval != 0 || .DecayToZero != 0 {
		if .DecayInterval < time.Second {
			return fmt.Errorf("invalid DecayInterval; must be at least 1s")
		}
		if .DecayToZero <= 0 || .DecayToZero >= 1 || isInvalidNumber(.DecayToZero) {
			return fmt.Errorf("invalid DecayToZero; must be between 0 and 1")
		}
	}

	// no need to check the score retention; a value of 0 means that we don't retain scores
	return nil
}

func ( *TopicScoreParams) () error {
	// make sure we have a sane topic weight
	if .TopicWeight < 0 || isInvalidNumber(.TopicWeight) {
		return fmt.Errorf("invalid topic weight; must be >= 0 and a valid number")
	}

	// check P1
	if  := .validateTimeInMeshParams();  != nil {
		return 
	}

	// check P2
	if  := .validateMessageDeliveryParams();  != nil {
		return 
	}
	// check P3
	if  := .validateMeshMessageDeliveryParams();  != nil {
		return 
	}

	// check P3b
	if  := .validateMessageFailurePenaltyParams();  != nil {
		return 
	}

	// check P4
	if  := .validateInvalidMessageDeliveryParams();  != nil {
		return 
	}

	return nil
}

func ( *TopicScoreParams) () error {
	if .SkipAtomicValidation {
		// in non-atomic mode, parameters at their zero values are dismissed from validation.
		if .TimeInMeshWeight == 0 && .TimeInMeshQuantum == 0 && .TimeInMeshCap == 0 {
			return nil
		}
	}

	// either atomic validation mode, or some parameters have been set a value,
	// hence, proceed with normal validation of all related parameters in this context.

	if .TimeInMeshQuantum == 0 {
		return fmt.Errorf("invalid TimeInMeshQuantum; must be non zero")
	}
	if .TimeInMeshWeight < 0 || isInvalidNumber(.TimeInMeshWeight) {
		return fmt.Errorf("invalid TimeInMeshWeight; must be positive (or 0 to disable) and a valid number")
	}
	if .TimeInMeshWeight != 0 && .TimeInMeshQuantum <= 0 {
		return fmt.Errorf("invalid TimeInMeshQuantum; must be positive")
	}
	if .TimeInMeshWeight != 0 && (.TimeInMeshCap <= 0 || isInvalidNumber(.TimeInMeshCap)) {
		return fmt.Errorf("invalid TimeInMeshCap; must be positive and a valid number")
	}

	return nil
}

func ( *TopicScoreParams) () error {
	if .SkipAtomicValidation {
		// in non-atomic mode, parameters at their zero values are dismissed from validation.
		if .FirstMessageDeliveriesWeight == 0 && .FirstMessageDeliveriesCap == 0 && .FirstMessageDeliveriesDecay == 0 {
			return nil
		}
	}

	// either atomic validation mode, or some parameters have been set a value,
	// hence, proceed with normal validation of all related parameters in this context.

	if .FirstMessageDeliveriesWeight < 0 || isInvalidNumber(.FirstMessageDeliveriesWeight) {
		return fmt.Errorf("invallid FirstMessageDeliveriesWeight; must be positive (or 0 to disable) and a valid number")
	}
	if .FirstMessageDeliveriesWeight != 0 && (.FirstMessageDeliveriesDecay <= 0 || .FirstMessageDeliveriesDecay >= 1 || isInvalidNumber(.FirstMessageDeliveriesDecay)) {
		return fmt.Errorf("invalid FirstMessageDeliveriesDecay; must be between 0 and 1")
	}
	if .FirstMessageDeliveriesWeight != 0 && (.FirstMessageDeliveriesCap <= 0 || isInvalidNumber(.FirstMessageDeliveriesCap)) {
		return fmt.Errorf("invalid FirstMessageDeliveriesCap; must be positive and a valid number")
	}

	return nil
}

func ( *TopicScoreParams) () error {
	if .SkipAtomicValidation {
		// in non-atomic mode, parameters at their zero values are dismissed from validation.
		if .MeshMessageDeliveriesWeight == 0 &&
			.MeshMessageDeliveriesCap == 0 &&
			.MeshMessageDeliveriesDecay == 0 &&
			.MeshMessageDeliveriesThreshold == 0 &&
			.MeshMessageDeliveriesWindow == 0 &&
			.MeshMessageDeliveriesActivation == 0 {
			return nil
		}
	}

	// either atomic validation mode, or some parameters have been set a value,
	// hence, proceed with normal validation of all related parameters in this context.

	if .MeshMessageDeliveriesWeight > 0 || isInvalidNumber(.MeshMessageDeliveriesWeight) {
		return fmt.Errorf("invalid MeshMessageDeliveriesWeight; must be negative (or 0 to disable) and a valid number")
	}
	if .MeshMessageDeliveriesWeight != 0 && (.MeshMessageDeliveriesDecay <= 0 || .MeshMessageDeliveriesDecay >= 1 || isInvalidNumber(.MeshMessageDeliveriesDecay)) {
		return fmt.Errorf("invalid MeshMessageDeliveriesDecay; must be between 0 and 1")
	}
	if .MeshMessageDeliveriesWeight != 0 && (.MeshMessageDeliveriesCap <= 0 || isInvalidNumber(.MeshMessageDeliveriesCap)) {
		return fmt.Errorf("invalid MeshMessageDeliveriesCap; must be positive and a valid number")
	}
	if .MeshMessageDeliveriesWeight != 0 && (.MeshMessageDeliveriesThreshold <= 0 || isInvalidNumber(.MeshMessageDeliveriesThreshold)) {
		return fmt.Errorf("invalid MeshMessageDeliveriesThreshold; must be positive and a valid number")
	}
	if .MeshMessageDeliveriesWindow < 0 {
		return fmt.Errorf("invalid MeshMessageDeliveriesWindow; must be non-negative")
	}
	if .MeshMessageDeliveriesWeight != 0 && .MeshMessageDeliveriesActivation < time.Second {
		return fmt.Errorf("invalid MeshMessageDeliveriesActivation; must be at least 1s")
	}

	return nil
}

func ( *TopicScoreParams) () error {
	if .SkipAtomicValidation {
		// in selective mode, parameters at their zero values are dismissed from validation.
		if .MeshFailurePenaltyDecay == 0 && .MeshFailurePenaltyWeight == 0 {
			return nil
		}
	}

	// either atomic validation mode, or some parameters have been set a value,
	// hence, proceed with normal validation of all related parameters in this context.

	if .MeshFailurePenaltyWeight > 0 || isInvalidNumber(.MeshFailurePenaltyWeight) {
		return fmt.Errorf("invalid MeshFailurePenaltyWeight; must be negative (or 0 to disable) and a valid number")
	}
	if .MeshFailurePenaltyWeight != 0 && (isInvalidNumber(.MeshFailurePenaltyDecay) || .MeshFailurePenaltyDecay <= 0 || .MeshFailurePenaltyDecay >= 1) {
		return fmt.Errorf("invalid MeshFailurePenaltyDecay; must be between 0 and 1")
	}

	return nil
}

func ( *TopicScoreParams) () error {
	if .SkipAtomicValidation {
		// in selective mode, parameters at their zero values are dismissed from validation.
		if .InvalidMessageDeliveriesDecay == 0 && .InvalidMessageDeliveriesWeight == 0 {
			return nil
		}
	}

	// either atomic validation mode, or some parameters have been set a value,
	// hence, proceed with normal validation of all related parameters in this context.

	if .InvalidMessageDeliveriesWeight > 0 || isInvalidNumber(.InvalidMessageDeliveriesWeight) {
		return fmt.Errorf("invalid InvalidMessageDeliveriesWeight; must be negative (or 0 to disable) and a valid number")
	}
	if .InvalidMessageDeliveriesDecay <= 0 || .InvalidMessageDeliveriesDecay >= 1 || isInvalidNumber(.InvalidMessageDeliveriesDecay) {
		return fmt.Errorf("invalid InvalidMessageDeliveriesDecay; must be between 0 and 1")
	}

	return nil
}

const (
	DefaultDecayInterval = time.Second
	DefaultDecayToZero   = 0.01
)

// ScoreParameterDecay computes the decay factor for a parameter, assuming the DecayInterval is 1s
// and that the value decays to zero if it drops below 0.01
func ( time.Duration) float64 {
	return ScoreParameterDecayWithBase(, DefaultDecayInterval, DefaultDecayToZero)
}

// ScoreParameterDecayWithBase computes the decay factor for a parameter using base as the DecayInterval
func ( time.Duration,  time.Duration,  float64) float64 {
	// the decay is linear, so after n ticks the value is factor^n
	// so factor^n = decayToZero => factor = decayToZero^(1/n)
	 := float64( / )
	return math.Pow(, 1/)
}

// checks whether the provided floating-point number is `Not a Number`
// or an infinite number.
func isInvalidNumber( float64) bool {
	return math.IsNaN() || math.IsInf(, 0)
}