package netmap

import (
	"fmt"
	"testing"
	"time"

	v2netmap "git.frostfs.info/TrueCloudLab/frostfs-api-go/v2/netmap"
	netmapContract "git.frostfs.info/TrueCloudLab/frostfs-contract/netmap"
	"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/innerring/processors/governance"
	timerEvent "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/innerring/timers"
	cntClient "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/client/container"
	netmapclient "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/client/netmap"
	"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/event"
	netmapEvent "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/event/netmap"
	"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/logger/test"
	"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/netmap"
	"github.com/nspcc-dev/neo-go/pkg/core/transaction"
	"github.com/nspcc-dev/neo-go/pkg/crypto/keys"
	"github.com/nspcc-dev/neo-go/pkg/encoding/fixedn"
	"github.com/nspcc-dev/neo-go/pkg/network/payload"
	"github.com/nspcc-dev/neo-go/pkg/util"
	"github.com/stretchr/testify/require"
)

func TestNewEpochTick(t *testing.T) {
	t.Parallel()
	es := &testEpochState{
		counter: 100,
	}
	nc := &testNetmapClient{}

	proc, err := newTestProc(t, func(p *Params) {
		p.NotaryDisabled = true
		p.CleanupEnabled = true
		p.EpochState = es
		p.NetmapClient = nc
	})

	require.NoError(t, err, "failed to create processor")

	ev := timerEvent.NewEpochTick{}
	proc.HandleNewEpochTick(ev)

	for proc.pool.Running() > 0 {
		time.Sleep(10 * time.Millisecond)
	}

	require.EqualValues(t, []uint64{101}, nc.newEpochs, "invalid epochs")
}

func TestNewEpoch(t *testing.T) {
	t.Parallel()
	var node1 netmap.NodeInfo
	key1, err := keys.NewPublicKeyFromString("038c862959e56b43e20f79187c4fe9e0bc7c8c66c1603e6cf0ec7f87ab6b08dc35")
	require.NoError(t, err, "failed to parse key1")
	node1.SetPublicKey(key1.Bytes())

	var node2 netmap.NodeInfo
	key2, err := keys.NewPublicKeyFromString("02ac920cd7df0b61b289072e6b946e2da4e1a31b9ab1c621bb475e30fa4ab102c3")
	require.NoError(t, err, "failed to parse key2")
	node2.SetPublicKey(key2.Bytes())

	network := &netmap.NetMap{}
	network.SetNodes([]netmap.NodeInfo{node1, node2})

	es := &testEpochState{
		counter:  100,
		duration: 10,
	}
	r := &testEpochResetter{}
	cc := &testContainerClient{}
	nc := &testNetmapClient{
		epochDuration: 20,
		txHeights: map[util.Uint256]uint32{
			{101}: 10_000,
		},
		netmap: network,
	}
	eh := &testEventHandler{}

	proc, err := newTestProc(t, func(p *Params) {
		p.NotaryDisabled = true
		p.NotaryDepositHandler = eh.Handle
		p.AlphabetSyncHandler = eh.Handle
		p.NetmapClient = nc
		p.ContainerWrapper = cc
		p.EpochTimer = r
		p.EpochState = es
	})

	require.NoError(t, err, "failed to create processor")

	ev := netmapEvent.NewEpoch{
		Num:  101,
		Hash: util.Uint256{101},
	}
	proc.handleNewEpoch(ev)

	for proc.pool.Running() > 0 {
		time.Sleep(10 * time.Millisecond)
	}

	require.Equal(t, nc.epochDuration, es.duration, "invalid epoch duration")
	require.Equal(t, ev.Num, es.counter, "invalid epoch counter")
	require.EqualValues(t, []uint32{nc.txHeights[ev.Hash]}, r.timers, "invalid epoch timer resets")

	var expEstimation cntClient.StartEstimationPrm
	expEstimation.SetEpoch(ev.Num - 1)
	expEstimation.SetHash(ev.Hash)
	require.EqualValues(t, []cntClient.StartEstimationPrm{expEstimation}, cc.estimations, "invalid estimations")

	require.EqualValues(t, []event.Event{
		governance.NewSyncEvent(ev.TxHash()),
		ev,
	}, eh.handledEvents, "invalid handled events")
}

func TestAddPeer(t *testing.T) {
	t.Parallel()

	t.Run("with notary", func(t *testing.T) {
		t.Parallel()
		nc := &testNetmapClient{
			contractAddress: util.Uint160{47},
		}

		proc, err := newTestProc(t, func(p *Params) {
			p.NotaryDisabled = true
			p.NetmapClient = nc
		})

		require.NoError(t, err, "failed to create processor")

		var node netmap.NodeInfo
		key, err := keys.NewPublicKeyFromString("038c862959e56b43e20f79187c4fe9e0bc7c8c66c1603e6cf0ec7f87ab6b08dc35")
		require.NoError(t, err, "failed to parse key1")
		node.SetPublicKey(key.Bytes())

		ev := netmapEvent.AddPeer{
			NodeBytes: node.Marshal(),
			Request: &payload.P2PNotaryRequest{
				MainTransaction: &transaction.Transaction{
					Nonce: 100,
				},
			},
		}
		proc.handleAddPeer(ev)

		for proc.pool.Running() > 0 {
			time.Sleep(10 * time.Millisecond)
		}

		require.EqualValues(t, []notaryInvoke{
			{
				contract: nc.contractAddress,
				fee:      0,
				nonce:    ev.Request.MainTransaction.Nonce,
				vub:      nil,
				method:   "addPeerIR",
				args:     []any{ev.Node()},
			},
		}, nc.notaryInvokes, "invalid notary invokes")
	})

	t.Run("without notary", func(t *testing.T) {
		t.Parallel()

		nc := &testNetmapClient{
			contractAddress: util.Uint160{47},
		}

		proc, err := newTestProc(t, func(p *Params) {
			p.NotaryDisabled = true
			p.NetmapClient = nc
		})

		require.NoError(t, err, "failed to create processor")

		var node netmap.NodeInfo
		key, err := keys.NewPublicKeyFromString("038c862959e56b43e20f79187c4fe9e0bc7c8c66c1603e6cf0ec7f87ab6b08dc35")
		require.NoError(t, err, "failed to parse key")
		node.SetPublicKey(key.Bytes())

		ev := netmapEvent.AddPeer{
			NodeBytes: node.Marshal(),
		}
		proc.handleAddPeer(ev)

		for proc.pool.Running() > 0 {
			time.Sleep(10 * time.Millisecond)
		}

		var addPeerExp netmapclient.AddPeerPrm
		addPeerExp.SetNodeInfo(node)
		require.EqualValues(t, []netmapclient.AddPeerPrm{addPeerExp}, nc.addPeers, "invalid peers")
	})
}

func TestUpdateState(t *testing.T) {
	t.Parallel()

	t.Run("with notary", func(t *testing.T) {
		t.Parallel()
		ns := &testNodeStateSettings{
			maintAllowed: true,
		}
		nc := &testNetmapClient{}

		proc, err := newTestProc(t, func(p *Params) {
			p.NotaryDisabled = true
			p.NodeStateSettings = ns
			p.NetmapClient = nc
		})

		require.NoError(t, err, "failed to create processor")

		key, err := keys.NewPublicKeyFromString("038c862959e56b43e20f79187c4fe9e0bc7c8c66c1603e6cf0ec7f87ab6b08dc35")
		require.NoError(t, err, "failed to parse key")

		ev := netmapEvent.UpdatePeer{
			State:  netmapContract.NodeStateOnline,
			PubKey: key,
			Request: &payload.P2PNotaryRequest{
				MainTransaction: &transaction.Transaction{
					Nonce: 100,
				},
			},
		}
		proc.handleUpdateState(ev)

		for proc.pool.Running() > 0 {
			time.Sleep(10 * time.Millisecond)
		}

		require.EqualValues(t, []*transaction.Transaction{
			ev.Request.MainTransaction,
		}, nc.invokedTxs, "invalid invoked transactions")
	})

	t.Run("without notary", func(t *testing.T) {
		t.Parallel()
		ns := &testNodeStateSettings{
			maintAllowed: true,
		}
		nc := &testNetmapClient{}

		proc, err := newTestProc(t, func(p *Params) {
			p.NetmapClient = nc
			p.NodeStateSettings = ns
		})

		require.NoError(t, err, "failed to create processor")

		key, err := keys.NewPublicKeyFromString("038c862959e56b43e20f79187c4fe9e0bc7c8c66c1603e6cf0ec7f87ab6b08dc35")
		require.NoError(t, err, "failed to parse key")

		ev := netmapEvent.UpdatePeer{
			State:  netmapContract.NodeStateOnline,
			PubKey: key,
		}
		proc.handleUpdateState(ev)

		for proc.pool.Running() > 0 {
			time.Sleep(10 * time.Millisecond)
		}

		var expUpdPeer netmapclient.UpdatePeerPrm
		expUpdPeer.SetMaintenance()
		expUpdPeer.SetOnline()
		expUpdPeer.SetKey(ev.PubKey.Bytes())

		require.EqualValues(t, []netmapclient.UpdatePeerPrm{expUpdPeer}, nc.peerStateUpdates, "invalid peer state updates")
	})
}

func TestCleanupTick(t *testing.T) {
	t.Parallel()

	t.Run("notary disabled", func(t *testing.T) {
		t.Parallel()

		nc := &testNetmapClient{}

		proc, err := newTestProc(t, func(p *Params) {
			p.NetmapClient = nc
			p.NotaryDisabled = true
			p.CleanupEnabled = true
		})

		require.NoError(t, err, "failed to create processor")

		key1Str := "038c862959e56b43e20f79187c4fe9e0bc7c8c66c1603e6cf0ec7f87ab6b08dc35"
		proc.netmapSnapshot.lastAccess[key1Str] = epochStampWithNodeInfo{
			epochStamp: epochStamp{
				epoch:      95,
				removeFlag: false,
			},
		}
		key2Str := "02ac920cd7df0b61b289072e6b946e2da4e1a31b9ab1c621bb475e30fa4ab102c3"
		proc.netmapSnapshot.lastAccess[key2Str] = epochStampWithNodeInfo{
			epochStamp: epochStamp{
				epoch:      98,
				removeFlag: false,
			},
		}

		ev := netmapCleanupTick{
			epoch:  100,
			txHash: util.Uint256{123},
		}

		proc.handleCleanupTick(ev)

		for proc.pool.Running() > 0 {
			time.Sleep(10 * time.Millisecond)
		}

		keyExp, err := keys.NewPublicKeyFromString(key1Str)
		require.NoError(t, err, "failed to parse expired key")

		updExp := netmapclient.UpdatePeerPrm{}
		updExp.SetKey(keyExp.Bytes())
		updExp.SetHash(ev.TxHash())

		require.EqualValues(t, []netmapclient.UpdatePeerPrm{updExp}, nc.peerStateUpdates, "invalid peer updates")
		require.True(t, proc.netmapSnapshot.lastAccess[key1Str].removeFlag, "invalid expired removed flag")
		require.False(t, proc.netmapSnapshot.lastAccess[key2Str].removeFlag, "invalid non expired removed flag")
	})

	t.Run("notary enabled", func(t *testing.T) {
		t.Parallel()

		nc := &testNetmapClient{
			contractAddress: util.Uint160{111},
		}
		proc, err := newTestProc(t,
			func(p *Params) {
				p.NetmapClient = nc
				p.CleanupEnabled = true
			},
		)

		require.NoError(t, err, "failed to create processor")

		key1Str := "038c862959e56b43e20f79187c4fe9e0bc7c8c66c1603e6cf0ec7f87ab6b08dc35"
		proc.netmapSnapshot.lastAccess[key1Str] = epochStampWithNodeInfo{
			epochStamp: epochStamp{
				epoch:      95,
				removeFlag: false,
			},
		}
		key2Str := "02ac920cd7df0b61b289072e6b946e2da4e1a31b9ab1c621bb475e30fa4ab102c3"
		proc.netmapSnapshot.lastAccess[key2Str] = epochStampWithNodeInfo{
			epochStamp: epochStamp{
				epoch:      98,
				removeFlag: false,
			},
		}

		ev := netmapCleanupTick{
			epoch:  100,
			txHash: util.Uint256{123},
		}

		proc.handleCleanupTick(ev)

		for proc.pool.Running() > 0 {
			time.Sleep(10 * time.Millisecond)
		}

		keyExp, err := keys.NewPublicKeyFromString(key1Str)
		require.NoError(t, err, "failed to parse expired key")

		updExp := netmapclient.UpdatePeerPrm{}
		updExp.SetKey(keyExp.Bytes())
		updExp.SetHash(ev.TxHash())

		require.EqualValues(t, []notaryInvoke{
			{
				contract: nc.contractAddress,
				fee:      0,
				nonce:    uint32(ev.epoch),
				vub:      nil,
				method:   "updateStateIR",
				args:     []any{int64(v2netmap.Offline), keyExp.Bytes()},
			},
		}, nc.notaryInvokes, "invalid notary invokes")
		require.True(t, proc.netmapSnapshot.lastAccess[key1Str].removeFlag, "invalid expired removed flag")
		require.False(t, proc.netmapSnapshot.lastAccess[key2Str].removeFlag, "invalid non expired removed flag")
	})
}

func newTestProc(t *testing.T, nonDefault func(p *Params)) (*Processor, error) {
	ns := &testNodeStateSettings{}
	es := &testEpochState{}
	r := &testEpochResetter{}
	as := &testAlphabetState{
		isAlphabet: true,
	}
	cc := &testContainerClient{}
	nc := &testNetmapClient{}
	eh := &testEventHandler{}

	p := &Params{
		Log:                  test.NewLogger(t, true),
		PoolSize:             1,
		CleanupEnabled:       false,
		CleanupThreshold:     3,
		NotaryDisabled:       false,
		NodeStateSettings:    ns,
		NodeValidator:        &testValidator{},
		EpochState:           es,
		EpochTimer:           r,
		AlphabetState:        as,
		ContainerWrapper:     cc,
		NetmapClient:         nc,
		NotaryDepositHandler: eh.Handle,
		AlphabetSyncHandler:  eh.Handle,
	}

	nonDefault(p)

	return New(p)
}

type testNodeStateSettings struct {
	maintAllowed bool
}

func (s *testNodeStateSettings) MaintenanceModeAllowed() error {
	if s.maintAllowed {
		return nil
	}
	return fmt.Errorf("maintenance mode not allowed")
}

type testValidator struct{}

func (v *testValidator) VerifyAndUpdate(*netmap.NodeInfo) error {
	return nil
}

type testEpochState struct {
	counter  uint64
	duration uint64
}

func (s *testEpochState) SetEpochCounter(c uint64) {
	s.counter = c
}
func (s *testEpochState) EpochCounter() uint64 {
	return s.counter
}
func (s *testEpochState) SetEpochDuration(d uint64) {
	s.duration = d
}
func (s *testEpochState) EpochDuration() uint64 {
	return s.duration
}

type testEpochResetter struct {
	timers []uint32
}

func (r *testEpochResetter) ResetEpochTimer(t uint32) error {
	r.timers = append(r.timers, t)
	return nil
}

type testAlphabetState struct {
	isAlphabet bool
}

func (s *testAlphabetState) IsAlphabet() bool {
	return s.isAlphabet
}

type testContainerClient struct {
	estimations []cntClient.StartEstimationPrm
}

func (c *testContainerClient) StartEstimation(p cntClient.StartEstimationPrm) error {
	c.estimations = append(c.estimations, p)
	return nil
}

type notaryInvoke struct {
	contract util.Uint160
	fee      fixedn.Fixed8
	nonce    uint32
	vub      *uint32
	method   string
	args     []any
}

type testNetmapClient struct {
	contractAddress util.Uint160
	epochDuration   uint64
	netmap          *netmap.NetMap
	txHeights       map[util.Uint256]uint32

	peerStateUpdates []netmapclient.UpdatePeerPrm
	notaryInvokes    []notaryInvoke
	newEpochs        []uint64
	addPeers         []netmapclient.AddPeerPrm
	invokedTxs       []*transaction.Transaction
}

func (c *testNetmapClient) UpdatePeerState(p netmapclient.UpdatePeerPrm) error {
	c.peerStateUpdates = append(c.peerStateUpdates, p)
	return nil
}
func (c *testNetmapClient) MorphNotaryInvoke(contract util.Uint160, fee fixedn.Fixed8, nonce uint32, vub *uint32, method string, args ...any) error {
	c.notaryInvokes = append(c.notaryInvokes, notaryInvoke{
		contract: contract,
		fee:      fee,
		nonce:    nonce,
		vub:      vub,
		method:   method,
		args:     args,
	})
	return nil
}
func (c *testNetmapClient) ContractAddress() util.Uint160 {
	return c.contractAddress
}
func (c *testNetmapClient) EpochDuration() (uint64, error) {
	return c.epochDuration, nil
}
func (c *testNetmapClient) MorphTxHeight(h util.Uint256) (uint32, error) {
	if res, found := c.txHeights[h]; found {
		return res, nil
	}
	return 0, fmt.Errorf("not found")
}
func (c *testNetmapClient) NetMap() (*netmap.NetMap, error) {
	return c.netmap, nil
}
func (c *testNetmapClient) NewEpoch(epoch uint64, force bool) error {
	c.newEpochs = append(c.newEpochs, epoch)
	return nil
}
func (c *testNetmapClient) MorphIsValidScript(script []byte, signers []transaction.Signer) (valid bool, err error) {
	return true, nil
}
func (c *testNetmapClient) AddPeer(p netmapclient.AddPeerPrm) error {
	c.addPeers = append(c.addPeers, p)
	return nil
}
func (c *testNetmapClient) MorphNotarySignAndInvokeTX(mainTx *transaction.Transaction) error {
	c.invokedTxs = append(c.invokedTxs, mainTx)
	return nil
}

type testEventHandler struct {
	handledEvents []event.Event
}

func (h *testEventHandler) Handle(e event.Event) {
	h.handledEvents = append(h.handledEvents, e)
}