[#280] ir: Add frostfs processor unit tests

Signed-off-by: Dmitrii Stepanov <d.stepanov@yadro.com>
bugfix/304-morph-client
Dmitrii Stepanov 2023-04-26 16:25:50 +03:00 committed by Evgenii Stratonikov
parent 5010b35466
commit 31b4da225a
16 changed files with 491 additions and 104 deletions

View File

@ -18,7 +18,7 @@ func (np *Processor) handleDeposit(ev event.Event) {
// send event to the worker pool // send event to the worker pool
err := np.pool.Submit(func() { np.processDeposit(&deposit) }) err := np.pool.Submit(func() { np.processDeposit(deposit) })
if err != nil { if err != nil {
// there system can be moved into controlled degradation stage // there system can be moved into controlled degradation stage
np.log.Warn(logs.FrostFSFrostfsProcessorWorkerPoolDrained, np.log.Warn(logs.FrostFSFrostfsProcessorWorkerPoolDrained,
@ -34,7 +34,7 @@ func (np *Processor) handleWithdraw(ev event.Event) {
// send event to the worker pool // send event to the worker pool
err := np.pool.Submit(func() { np.processWithdraw(&withdraw) }) err := np.pool.Submit(func() { np.processWithdraw(withdraw) })
if err != nil { if err != nil {
// there system can be moved into controlled degradation stage // there system can be moved into controlled degradation stage
np.log.Warn(logs.FrostFSFrostfsProcessorWorkerPoolDrained, np.log.Warn(logs.FrostFSFrostfsProcessorWorkerPoolDrained,
@ -50,7 +50,7 @@ func (np *Processor) handleCheque(ev event.Event) {
// send event to the worker pool // send event to the worker pool
err := np.pool.Submit(func() { np.processCheque(&cheque) }) err := np.pool.Submit(func() { np.processCheque(cheque) })
if err != nil { if err != nil {
// there system can be moved into controlled degradation stage // there system can be moved into controlled degradation stage
np.log.Warn(logs.FrostFSFrostfsProcessorWorkerPoolDrained, np.log.Warn(logs.FrostFSFrostfsProcessorWorkerPoolDrained,
@ -67,7 +67,7 @@ func (np *Processor) handleConfig(ev event.Event) {
// send event to the worker pool // send event to the worker pool
err := np.pool.Submit(func() { np.processConfig(&cfg) }) err := np.pool.Submit(func() { np.processConfig(cfg) })
if err != nil { if err != nil {
// there system can be moved into controlled degradation stage // there system can be moved into controlled degradation stage
np.log.Warn(logs.FrostFSFrostfsProcessorWorkerPoolDrained, np.log.Warn(logs.FrostFSFrostfsProcessorWorkerPoolDrained,
@ -83,7 +83,7 @@ func (np *Processor) handleBind(ev event.Event) {
// send event to the worker pool // send event to the worker pool
err := np.pool.Submit(func() { np.processBind(e) }) err := np.pool.Submit(func() { np.processBind(e, true) })
if err != nil { if err != nil {
// there system can be moved into controlled degradation stage // there system can be moved into controlled degradation stage
np.log.Warn(logs.FrostFSFrostfsProcessorWorkerPoolDrained, np.log.Warn(logs.FrostFSFrostfsProcessorWorkerPoolDrained,
@ -99,7 +99,7 @@ func (np *Processor) handleUnbind(ev event.Event) {
// send event to the worker pool // send event to the worker pool
err := np.pool.Submit(func() { np.processBind(e) }) err := np.pool.Submit(func() { np.processBind(e, false) })
if err != nil { if err != nil {
// there system can be moved into controlled degradation stage // there system can be moved into controlled degradation stage
np.log.Warn(logs.FrostFSFrostfsProcessorWorkerPoolDrained, np.log.Warn(logs.FrostFSFrostfsProcessorWorkerPoolDrained,

View File

@ -0,0 +1,371 @@
package frostfs
import (
"testing"
"time"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/client/balance"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/client/frostfsid"
nmClient "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/client/netmap"
frostfsEvent "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/event/frostfs"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/util/logger/test"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/user"
"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/util"
"github.com/stretchr/testify/require"
)
func TestHandleDeposit(t *testing.T) {
t.Parallel()
es := &testEpochState{
epochCounter: 100,
}
b := &testBalaceClient{}
m := &testMorphClient{
balance: 150,
}
proc, err := newTestProc(t, func(p *Params) {
p.EpochState = es
p.BalanceClient = b
p.MorphClient = m
})
require.NoError(t, err, "failed to create processor")
ev := frostfsEvent.Deposit{
IDValue: []byte{1, 2, 3, 4, 5},
FromValue: util.Uint160{100},
ToValue: util.Uint160{200},
AmountValue: 1000,
}
proc.handleDeposit(ev)
for proc.pool.Running() > 0 {
time.Sleep(10 * time.Millisecond)
}
var expMint balance.MintPrm
expMint.SetAmount(ev.AmountValue)
expMint.SetID(ev.IDValue)
expMint.SetTo(ev.ToValue)
require.EqualValues(t, []balance.MintPrm{expMint}, b.mint, "invalid mint value")
require.EqualValues(t, []transferGas{
{
receiver: ev.ToValue,
amount: fixedn.Fixed8(50),
},
}, m.transferGas, "invalid transfer gas")
es.epochCounter = 109
proc.handleDeposit(ev)
for proc.pool.Running() > 0 {
time.Sleep(10 * time.Millisecond)
}
expMint.SetAmount(ev.AmountValue)
expMint.SetID(ev.IDValue)
expMint.SetTo(ev.ToValue)
require.EqualValues(t, []balance.MintPrm{expMint, expMint}, b.mint, "invalid mint value")
require.EqualValues(t, []transferGas{
{
receiver: ev.ToValue,
amount: fixedn.Fixed8(50),
},
}, m.transferGas, "invalid transfer gas")
}
func TestHandleWithdraw(t *testing.T) {
t.Parallel()
es := &testEpochState{
epochCounter: 100,
}
b := &testBalaceClient{}
m := &testMorphClient{
balance: 150,
}
proc, err := newTestProc(t, func(p *Params) {
p.EpochState = es
p.BalanceClient = b
p.MorphClient = m
})
require.NoError(t, err, "failed to create processor")
ev := frostfsEvent.Withdraw{
IDValue: []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20},
UserValue: util.Uint160{100},
AmountValue: 1000,
}
proc.handleWithdraw(ev)
for proc.pool.Running() > 0 {
time.Sleep(10 * time.Millisecond)
}
lock, err := util.Uint160DecodeBytesBE(ev.ID()[:util.Uint160Size])
require.NoError(t, err, "failed to decode ID")
var expLock balance.LockPrm
expLock.SetAmount(ev.AmountValue)
expLock.SetID(ev.IDValue)
expLock.SetDueEpoch(int64(es.epochCounter) + int64(lockAccountLifetime))
expLock.SetLock(lock)
expLock.SetUser(ev.UserValue)
require.EqualValues(t, []balance.LockPrm{expLock}, b.lock, "invalid lock value")
}
func TestHandleCheque(t *testing.T) {
t.Parallel()
es := &testEpochState{
epochCounter: 100,
}
b := &testBalaceClient{}
m := &testMorphClient{
balance: 150,
}
proc, err := newTestProc(t, func(p *Params) {
p.BalanceClient = b
p.MorphClient = m
p.EpochState = es
})
require.NoError(t, err, "failed to create processor")
ev := frostfsEvent.Cheque{
IDValue: []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20},
UserValue: util.Uint160{100},
AmountValue: 1000,
LockValue: util.Uint160{200},
}
proc.handleCheque(ev)
for proc.pool.Running() > 0 {
time.Sleep(10 * time.Millisecond)
}
var expBurn balance.BurnPrm
expBurn.SetAmount(ev.AmountValue)
expBurn.SetID(ev.IDValue)
expBurn.SetTo(util.Uint160{200})
require.EqualValues(t, []balance.BurnPrm{expBurn}, b.burn, "invalid burn value")
}
func TestHandleConfig(t *testing.T) {
t.Parallel()
es := &testEpochState{
epochCounter: 100,
}
nm := &testNetmapClient{}
m := &testMorphClient{
balance: 150,
}
proc, err := newTestProc(t, func(p *Params) {
p.NetmapClient = nm
p.MorphClient = m
p.EpochState = es
})
require.NoError(t, err, "failed to create processor")
ev := frostfsEvent.Config{
IDValue: []byte{1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17, 18, 19, 20},
KeyValue: []byte{1, 2, 3, 4, 5},
ValueValue: []byte{6, 7, 8, 9, 0},
TxHashValue: util.Uint256{100},
}
proc.handleConfig(ev)
for proc.pool.Running() > 0 {
time.Sleep(10 * time.Millisecond)
}
var expConfig nmClient.SetConfigPrm
expConfig.SetHash(ev.TxHashValue)
expConfig.SetID(ev.IDValue)
expConfig.SetKey(ev.KeyValue)
expConfig.SetValue(ev.ValueValue)
require.EqualValues(t, []nmClient.SetConfigPrm{expConfig}, nm.config, "invalid config value")
}
func TestHandleUnbind(t *testing.T) {
t.Parallel()
es := &testEpochState{
epochCounter: 100,
}
m := &testMorphClient{
balance: 150,
}
id := &testIDClient{}
proc, err := newTestProc(t, func(p *Params) {
p.EpochState = es
p.MorphClient = m
p.FrostFSIDClient = id
})
require.NoError(t, err, "failed to create processor")
p, err := keys.NewPrivateKey()
require.NoError(t, err)
evUnbind := frostfsEvent.Unbind{
BindCommon: frostfsEvent.BindCommon{
UserValue: util.Uint160{49}.BytesBE(),
KeysValue: [][]byte{
p.PublicKey().Bytes(),
},
TxHashValue: util.Uint256{100},
},
}
proc.handleUnbind(evUnbind)
for proc.pool.Running() > 0 {
time.Sleep(10 * time.Millisecond)
}
var userID user.ID
userID.SetScriptHash(util.Uint160{49})
var expBind frostfsid.CommonBindPrm
expBind.SetOwnerID(userID.WalletBytes())
expBind.SetKeys(evUnbind.BindCommon.KeysValue)
expBind.SetHash(evUnbind.BindCommon.TxHashValue)
var expNilSlice []frostfsid.CommonBindPrm
require.EqualValues(t, []frostfsid.CommonBindPrm{expBind}, id.remove, "invalid remove keys value")
require.EqualValues(t, expNilSlice, id.add, "invalid add keys value")
evBind := frostfsEvent.Bind{
BindCommon: frostfsEvent.BindCommon{
UserValue: util.Uint160{49}.BytesBE(),
KeysValue: [][]byte{
p.PublicKey().Bytes(),
},
TxHashValue: util.Uint256{100},
},
}
proc.handleBind(evBind)
time.Sleep(time.Second)
require.EqualValues(t, []frostfsid.CommonBindPrm{expBind}, id.remove, "invalid remove keys value")
require.EqualValues(t, []frostfsid.CommonBindPrm{expBind}, id.add, "invalid add keys value")
}
func newTestProc(t *testing.T, nonDefault func(p *Params)) (*Processor, error) {
p := &Params{
Log: test.NewLogger(t, true),
PoolSize: 1,
FrostFSContract: util.Uint160{0},
FrostFSIDClient: &testIDClient{},
BalanceClient: &testBalaceClient{},
NetmapClient: &testNetmapClient{},
MorphClient: &testMorphClient{},
EpochState: &testEpochState{},
AlphabetState: &testAlphabetState{isAlphabet: true},
Converter: &testPrecisionConverter{},
MintEmitCacheSize: 100,
MintEmitThreshold: 10,
MintEmitValue: fixedn.Fixed8(50),
GasBalanceThreshold: 50,
}
nonDefault(p)
return New(p)
}
type testEpochState struct {
epochCounter uint64
}
func (s *testEpochState) EpochCounter() uint64 {
return s.epochCounter
}
type testAlphabetState struct {
isAlphabet bool
}
func (s *testAlphabetState) IsAlphabet() bool {
return s.isAlphabet
}
type testPrecisionConverter struct {
}
func (c *testPrecisionConverter) ToBalancePrecision(v int64) int64 {
return v
}
type testBalaceClient struct {
mint []balance.MintPrm
lock []balance.LockPrm
burn []balance.BurnPrm
}
func (c *testBalaceClient) Mint(p balance.MintPrm) error {
c.mint = append(c.mint, p)
return nil
}
func (c *testBalaceClient) Lock(p balance.LockPrm) error {
c.lock = append(c.lock, p)
return nil
}
func (c *testBalaceClient) Burn(p balance.BurnPrm) error {
c.burn = append(c.burn, p)
return nil
}
type testNetmapClient struct {
config []nmClient.SetConfigPrm
}
func (c *testNetmapClient) SetConfig(p nmClient.SetConfigPrm) error {
c.config = append(c.config, p)
return nil
}
type transferGas struct {
receiver util.Uint160
amount fixedn.Fixed8
}
type testMorphClient struct {
balance int64
transferGas []transferGas
}
func (c *testMorphClient) GasBalance() (res int64, err error) {
return c.balance, nil
}
func (c *testMorphClient) TransferGas(receiver util.Uint160, amount fixedn.Fixed8) error {
c.transferGas = append(c.transferGas, transferGas{
receiver: receiver,
amount: amount,
})
return nil
}
type testIDClient struct {
add []frostfsid.CommonBindPrm
remove []frostfsid.CommonBindPrm
}
func (c *testIDClient) AddKeys(p frostfsid.CommonBindPrm) error {
c.add = append(c.add, p)
return nil
}
func (c *testIDClient) RemoveKeys(args frostfsid.CommonBindPrm) error {
c.remove = append(c.remove, args)
return nil
}

View File

@ -15,7 +15,7 @@ const (
// Process deposit event by invoking a balance contract and sending native // Process deposit event by invoking a balance contract and sending native
// gas in the sidechain. // gas in the sidechain.
func (np *Processor) processDeposit(deposit *frostfsEvent.Deposit) { func (np *Processor) processDeposit(deposit frostfsEvent.Deposit) {
if !np.alphabetState.IsAlphabet() { if !np.alphabetState.IsAlphabet() {
np.log.Info(logs.FrostFSNonAlphabetModeIgnoreDeposit) np.log.Info(logs.FrostFSNonAlphabetModeIgnoreDeposit)
return return
@ -80,7 +80,7 @@ func (np *Processor) processDeposit(deposit *frostfsEvent.Deposit) {
} }
// Process withdraw event by locking assets in the balance account. // Process withdraw event by locking assets in the balance account.
func (np *Processor) processWithdraw(withdraw *frostfsEvent.Withdraw) { func (np *Processor) processWithdraw(withdraw frostfsEvent.Withdraw) {
if !np.alphabetState.IsAlphabet() { if !np.alphabetState.IsAlphabet() {
np.log.Info(logs.FrostFSNonAlphabetModeIgnoreWithdraw) np.log.Info(logs.FrostFSNonAlphabetModeIgnoreWithdraw)
return return
@ -111,7 +111,7 @@ func (np *Processor) processWithdraw(withdraw *frostfsEvent.Withdraw) {
// Process cheque event by transferring assets from the lock account back to // Process cheque event by transferring assets from the lock account back to
// the reserve account. // the reserve account.
func (np *Processor) processCheque(cheque *frostfsEvent.Cheque) { func (np *Processor) processCheque(cheque frostfsEvent.Cheque) {
if !np.alphabetState.IsAlphabet() { if !np.alphabetState.IsAlphabet() {
np.log.Info(logs.FrostFSNonAlphabetModeIgnoreCheque) np.log.Info(logs.FrostFSNonAlphabetModeIgnoreCheque)
return return

View File

@ -6,7 +6,6 @@ import (
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs" "git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/client/frostfsid" "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/client/frostfsid"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/event/frostfs"
"git.frostfs.info/TrueCloudLab/frostfs-sdk-go/user" "git.frostfs.info/TrueCloudLab/frostfs-sdk-go/user"
"github.com/nspcc-dev/neo-go/pkg/crypto/keys" "github.com/nspcc-dev/neo-go/pkg/crypto/keys"
"github.com/nspcc-dev/neo-go/pkg/util" "github.com/nspcc-dev/neo-go/pkg/util"
@ -19,7 +18,7 @@ type bindCommon interface {
TxHash() util.Uint256 TxHash() util.Uint256
} }
func (np *Processor) processBind(e bindCommon) { func (np *Processor) processBind(e bindCommon, bind bool) {
if !np.alphabetState.IsAlphabet() { if !np.alphabetState.IsAlphabet() {
np.log.Info(logs.FrostFSNonAlphabetModeIgnoreBind) np.log.Info(logs.FrostFSNonAlphabetModeIgnoreBind)
return return
@ -27,10 +26,9 @@ func (np *Processor) processBind(e bindCommon) {
c := &bindCommonContext{ c := &bindCommonContext{
bindCommon: e, bindCommon: e,
bind: bind,
} }
_, c.bind = e.(frostfs.Bind)
err := np.checkBindCommon(c) err := np.checkBindCommon(c)
if err != nil { if err != nil {
np.log.Error(logs.FrostFSInvalidManageKeyEvent, np.log.Error(logs.FrostFSInvalidManageKeyEvent,

View File

@ -9,7 +9,7 @@ import (
// Process config event by setting configuration value from the mainchain in // Process config event by setting configuration value from the mainchain in
// the sidechain. // the sidechain.
func (np *Processor) processConfig(config *frostfsEvent.Config) { func (np *Processor) processConfig(config frostfsEvent.Config) {
if !np.alphabetState.IsAlphabet() { if !np.alphabetState.IsAlphabet() {
np.log.Info(logs.FrostFSNonAlphabetModeIgnoreConfig) np.log.Info(logs.FrostFSNonAlphabetModeIgnoreConfig)
return return

View File

@ -6,7 +6,6 @@ import (
"sync" "sync"
"git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs" "git.frostfs.info/TrueCloudLab/frostfs-node/internal/logs"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/client"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/client/balance" "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/client/balance"
"git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/client/frostfsid" "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/client/frostfsid"
nmClient "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/client/netmap" nmClient "git.frostfs.info/TrueCloudLab/frostfs-node/pkg/morph/client/netmap"
@ -36,14 +35,34 @@ type (
ToBalancePrecision(int64) int64 ToBalancePrecision(int64) int64
} }
BalanceClient interface {
Mint(p balance.MintPrm) error
Lock(p balance.LockPrm) error
Burn(p balance.BurnPrm) error
}
NetmapClient interface {
SetConfig(p nmClient.SetConfigPrm) error
}
MorphClient interface {
GasBalance() (res int64, err error)
TransferGas(receiver util.Uint160, amount fixedn.Fixed8) error
}
IDClient interface {
AddKeys(p frostfsid.CommonBindPrm) error
RemoveKeys(args frostfsid.CommonBindPrm) error
}
// Processor of events produced by frostfs contract in main net. // Processor of events produced by frostfs contract in main net.
Processor struct { Processor struct {
log *logger.Logger log *logger.Logger
pool *ants.Pool pool *ants.Pool
frostfsContract util.Uint160 frostfsContract util.Uint160
balanceClient *balance.Client balanceClient BalanceClient
netmapClient *nmClient.Client netmapClient NetmapClient
morphClient *client.Client morphClient MorphClient
epochState EpochState epochState EpochState
alphabetState AlphabetState alphabetState AlphabetState
converter PrecisionConverter converter PrecisionConverter
@ -52,8 +71,7 @@ type (
mintEmitThreshold uint64 mintEmitThreshold uint64
mintEmitValue fixedn.Fixed8 mintEmitValue fixedn.Fixed8
gasBalanceThreshold int64 gasBalanceThreshold int64
frostfsIDClient IDClient
frostfsIDClient *frostfsid.Client
} }
// Params of the processor constructor. // Params of the processor constructor.
@ -61,10 +79,10 @@ type (
Log *logger.Logger Log *logger.Logger
PoolSize int PoolSize int
FrostFSContract util.Uint160 FrostFSContract util.Uint160
FrostFSIDClient *frostfsid.Client FrostFSIDClient IDClient
BalanceClient *balance.Client BalanceClient BalanceClient
NetmapClient *nmClient.Client NetmapClient NetmapClient
MorphClient *client.Client MorphClient MorphClient
EpochState EpochState EpochState EpochState
AlphabetState AlphabetState AlphabetState AlphabetState
Converter PrecisionConverter Converter PrecisionConverter

View File

@ -11,31 +11,31 @@ import (
) )
type Bind struct { type Bind struct {
bindCommon BindCommon
} }
type bindCommon struct { type BindCommon struct {
user []byte UserValue []byte
keys [][]byte KeysValue [][]byte
// txHash is used in notary environmental // TxHashValue is used in notary environmental
// for calculating unique but same for // for calculating unique but same for
// all notification receivers values. // all notification receivers values.
txHash util.Uint256 TxHashValue util.Uint256
} }
// TxHash returns hash of the TX with new epoch // TxHash returns hash of the TX with new epoch
// notification. // notification.
func (b bindCommon) TxHash() util.Uint256 { func (b BindCommon) TxHash() util.Uint256 {
return b.txHash return b.TxHashValue
} }
// MorphEvent implements Neo:Morph Event interface. // MorphEvent implements Neo:Morph Event interface.
func (bindCommon) MorphEvent() {} func (BindCommon) MorphEvent() {}
func (b bindCommon) Keys() [][]byte { return b.keys } func (b BindCommon) Keys() [][]byte { return b.KeysValue }
func (b bindCommon) User() []byte { return b.user } func (b BindCommon) User() []byte { return b.UserValue }
func ParseBind(e *state.ContainedNotificationEvent) (event.Event, error) { func ParseBind(e *state.ContainedNotificationEvent) (event.Event, error) {
var ( var (
@ -48,17 +48,17 @@ func ParseBind(e *state.ContainedNotificationEvent) (event.Event, error) {
return nil, fmt.Errorf("could not parse stack items from notify event: %w", err) return nil, fmt.Errorf("could not parse stack items from notify event: %w", err)
} }
err = parseBind(&ev.bindCommon, params) err = parseBind(&ev.BindCommon, params)
if err != nil { if err != nil {
return nil, err return nil, err
} }
ev.txHash = e.Container ev.TxHashValue = e.Container
return ev, nil return ev, nil
} }
func parseBind(dst *bindCommon, params []stackitem.Item) error { func parseBind(dst *BindCommon, params []stackitem.Item) error {
if ln := len(params); ln != 2 { if ln := len(params); ln != 2 {
return event.WrongNumberOfParameters(2, ln) return event.WrongNumberOfParameters(2, ln)
} }
@ -66,7 +66,7 @@ func parseBind(dst *bindCommon, params []stackitem.Item) error {
var err error var err error
// parse user // parse user
dst.user, err = client.BytesFromStackItem(params[0]) dst.UserValue, err = client.BytesFromStackItem(params[0])
if err != nil { if err != nil {
return fmt.Errorf("could not get bind user: %w", err) return fmt.Errorf("could not get bind user: %w", err)
} }
@ -77,7 +77,7 @@ func parseBind(dst *bindCommon, params []stackitem.Item) error {
return fmt.Errorf("could not get bind keys: %w", err) return fmt.Errorf("could not get bind keys: %w", err)
} }
dst.keys = make([][]byte, 0, len(bindKeys)) dst.KeysValue = make([][]byte, 0, len(bindKeys))
for i := range bindKeys { for i := range bindKeys {
rawKey, err := client.BytesFromStackItem(bindKeys[i]) rawKey, err := client.BytesFromStackItem(bindKeys[i])
@ -85,7 +85,7 @@ func parseBind(dst *bindCommon, params []stackitem.Item) error {
return fmt.Errorf("could not get bind public key: %w", err) return fmt.Errorf("could not get bind public key: %w", err)
} }
dst.keys = append(dst.keys, rawKey) dst.KeysValue = append(dst.KeysValue, rawKey)
} }
return nil return nil

View File

@ -11,26 +11,26 @@ import (
// Cheque structure of frostfs.Cheque notification from mainnet chain. // Cheque structure of frostfs.Cheque notification from mainnet chain.
type Cheque struct { type Cheque struct {
id []byte IDValue []byte
amount int64 // Fixed8 AmountValue int64 // Fixed8
user util.Uint160 UserValue util.Uint160
lock util.Uint160 LockValue util.Uint160
} }
// MorphEvent implements Neo:Morph Event interface. // MorphEvent implements Neo:Morph Event interface.
func (Cheque) MorphEvent() {} func (Cheque) MorphEvent() {}
// ID is a withdraw transaction hash. // ID is a withdraw transaction hash.
func (c Cheque) ID() []byte { return c.id } func (c Cheque) ID() []byte { return c.IDValue }
// User returns withdraw receiver script hash from main net. // User returns withdraw receiver script hash from main net.
func (c Cheque) User() util.Uint160 { return c.user } func (c Cheque) User() util.Uint160 { return c.UserValue }
// Amount of the sent assets. // Amount of the sent assets.
func (c Cheque) Amount() int64 { return c.amount } func (c Cheque) Amount() int64 { return c.AmountValue }
// LockAccount return script hash for balance contract wallet. // LockAccount return script hash for balance contract wallet.
func (c Cheque) LockAccount() util.Uint160 { return c.lock } func (c Cheque) LockAccount() util.Uint160 { return c.LockValue }
// ParseCheque from notification into cheque structure. // ParseCheque from notification into cheque structure.
func ParseCheque(e *state.ContainedNotificationEvent) (event.Event, error) { func ParseCheque(e *state.ContainedNotificationEvent) (event.Event, error) {
@ -49,7 +49,7 @@ func ParseCheque(e *state.ContainedNotificationEvent) (event.Event, error) {
} }
// parse id // parse id
ev.id, err = client.BytesFromStackItem(params[0]) ev.IDValue, err = client.BytesFromStackItem(params[0])
if err != nil { if err != nil {
return nil, fmt.Errorf("could not get cheque id: %w", err) return nil, fmt.Errorf("could not get cheque id: %w", err)
} }
@ -60,13 +60,13 @@ func ParseCheque(e *state.ContainedNotificationEvent) (event.Event, error) {
return nil, fmt.Errorf("could not get cheque user: %w", err) return nil, fmt.Errorf("could not get cheque user: %w", err)
} }
ev.user, err = util.Uint160DecodeBytesBE(user) ev.UserValue, err = util.Uint160DecodeBytesBE(user)
if err != nil { if err != nil {
return nil, fmt.Errorf("could not convert cheque user to uint160: %w", err) return nil, fmt.Errorf("could not convert cheque user to uint160: %w", err)
} }
// parse amount // parse amount
ev.amount, err = client.IntFromStackItem(params[2]) ev.AmountValue, err = client.IntFromStackItem(params[2])
if err != nil { if err != nil {
return nil, fmt.Errorf("could not get cheque amount: %w", err) return nil, fmt.Errorf("could not get cheque amount: %w", err)
} }
@ -77,7 +77,7 @@ func ParseCheque(e *state.ContainedNotificationEvent) (event.Event, error) {
return nil, fmt.Errorf("could not get cheque lock account: %w", err) return nil, fmt.Errorf("could not get cheque lock account: %w", err)
} }
ev.lock, err = util.Uint160DecodeBytesBE(lock) ev.LockValue, err = util.Uint160DecodeBytesBE(lock)
if err != nil { if err != nil {
return nil, fmt.Errorf("could not convert cheque lock account to uint160: %w", err) return nil, fmt.Errorf("could not convert cheque lock account to uint160: %w", err)
} }

View File

@ -77,10 +77,10 @@ func TestParseCheque(t *testing.T) {
require.NoError(t, err) require.NoError(t, err)
require.Equal(t, Cheque{ require.Equal(t, Cheque{
id: id, IDValue: id,
amount: amount, AmountValue: amount,
user: user, UserValue: user,
lock: lock, LockValue: lock,
}, ev) }, ev)
}) })
} }

View File

@ -10,30 +10,30 @@ import (
) )
type Config struct { type Config struct {
key []byte KeyValue []byte
value []byte ValueValue []byte
id []byte IDValue []byte
// txHash is used in notary environmental // TxHashValue is used in notary environmental
// for calculating unique but same for // for calculating unique but same for
// all notification receivers values. // all notification receivers values.
txHash util.Uint256 TxHashValue util.Uint256
} }
// TxHash returns hash of the TX with new epoch // TxHash returns hash of the TX with new epoch
// notification. // notification.
func (u Config) TxHash() util.Uint256 { func (u Config) TxHash() util.Uint256 {
return u.txHash return u.TxHashValue
} }
// MorphEvent implements Neo:Morph Event interface. // MorphEvent implements Neo:Morph Event interface.
func (Config) MorphEvent() {} func (Config) MorphEvent() {}
func (u Config) ID() []byte { return u.id } func (u Config) ID() []byte { return u.IDValue }
func (u Config) Key() []byte { return u.key } func (u Config) Key() []byte { return u.KeyValue }
func (u Config) Value() []byte { return u.value } func (u Config) Value() []byte { return u.ValueValue }
func ParseConfig(e *state.ContainedNotificationEvent) (event.Event, error) { func ParseConfig(e *state.ContainedNotificationEvent) (event.Event, error) {
var ( var (
@ -51,24 +51,24 @@ func ParseConfig(e *state.ContainedNotificationEvent) (event.Event, error) {
} }
// parse id // parse id
ev.id, err = client.BytesFromStackItem(params[0]) ev.IDValue, err = client.BytesFromStackItem(params[0])
if err != nil { if err != nil {
return nil, fmt.Errorf("could not get config update id: %w", err) return nil, fmt.Errorf("could not get config update id: %w", err)
} }
// parse key // parse key
ev.key, err = client.BytesFromStackItem(params[1]) ev.KeyValue, err = client.BytesFromStackItem(params[1])
if err != nil { if err != nil {
return nil, fmt.Errorf("could not get config key: %w", err) return nil, fmt.Errorf("could not get config key: %w", err)
} }
// parse value // parse value
ev.value, err = client.BytesFromStackItem(params[2]) ev.ValueValue, err = client.BytesFromStackItem(params[2])
if err != nil { if err != nil {
return nil, fmt.Errorf("could not get config value: %w", err) return nil, fmt.Errorf("could not get config value: %w", err)
} }
ev.txHash = e.Container ev.TxHashValue = e.Container
return ev, nil return ev, nil
} }

View File

@ -60,9 +60,9 @@ func TestParseConfig(t *testing.T) {
require.NoError(t, err) require.NoError(t, err)
require.Equal(t, Config{ require.Equal(t, Config{
id: id, IDValue: id,
key: key, KeyValue: key,
value: value, ValueValue: value,
}, ev) }, ev)
}) })
} }

View File

@ -11,26 +11,26 @@ import (
// Deposit structure of frostfs.Deposit notification from mainnet chain. // Deposit structure of frostfs.Deposit notification from mainnet chain.
type Deposit struct { type Deposit struct {
id []byte IDValue []byte
amount int64 // Fixed8 AmountValue int64 // Fixed8
from util.Uint160 FromValue util.Uint160
to util.Uint160 ToValue util.Uint160
} }
// MorphEvent implements Neo:Morph Event interface. // MorphEvent implements Neo:Morph Event interface.
func (Deposit) MorphEvent() {} func (Deposit) MorphEvent() {}
// ID is a deposit transaction hash. // ID is a deposit transaction hash.
func (d Deposit) ID() []byte { return d.id } func (d Deposit) ID() []byte { return d.IDValue }
// From is a script hash of asset sender in main net. // From is a script hash of asset sender in main net.
func (d Deposit) From() util.Uint160 { return d.from } func (d Deposit) From() util.Uint160 { return d.FromValue }
// To is a script hash of asset receiver in balance contract. // To is a script hash of asset receiver in balance contract.
func (d Deposit) To() util.Uint160 { return d.to } func (d Deposit) To() util.Uint160 { return d.ToValue }
// Amount of transferred assets. // Amount of transferred assets.
func (d Deposit) Amount() int64 { return d.amount } func (d Deposit) Amount() int64 { return d.AmountValue }
// ParseDeposit notification into deposit structure. // ParseDeposit notification into deposit structure.
func ParseDeposit(e *state.ContainedNotificationEvent) (event.Event, error) { func ParseDeposit(e *state.ContainedNotificationEvent) (event.Event, error) {
@ -51,13 +51,13 @@ func ParseDeposit(e *state.ContainedNotificationEvent) (event.Event, error) {
return nil, fmt.Errorf("could not get deposit sender: %w", err) return nil, fmt.Errorf("could not get deposit sender: %w", err)
} }
ev.from, err = util.Uint160DecodeBytesBE(from) ev.FromValue, err = util.Uint160DecodeBytesBE(from)
if err != nil { if err != nil {
return nil, fmt.Errorf("could not convert deposit sender to uint160: %w", err) return nil, fmt.Errorf("could not convert deposit sender to uint160: %w", err)
} }
// parse amount // parse amount
ev.amount, err = client.IntFromStackItem(params[1]) ev.AmountValue, err = client.IntFromStackItem(params[1])
if err != nil { if err != nil {
return nil, fmt.Errorf("could not get deposit amount: %w", err) return nil, fmt.Errorf("could not get deposit amount: %w", err)
} }
@ -68,13 +68,13 @@ func ParseDeposit(e *state.ContainedNotificationEvent) (event.Event, error) {
return nil, fmt.Errorf("could not get deposit receiver: %w", err) return nil, fmt.Errorf("could not get deposit receiver: %w", err)
} }
ev.to, err = util.Uint160DecodeBytesBE(to) ev.ToValue, err = util.Uint160DecodeBytesBE(to)
if err != nil { if err != nil {
return nil, fmt.Errorf("could not convert deposit receiver to uint160: %w", err) return nil, fmt.Errorf("could not convert deposit receiver to uint160: %w", err)
} }
// parse id // parse id
ev.id, err = client.BytesFromStackItem(params[3]) ev.IDValue, err = client.BytesFromStackItem(params[3])
if err != nil { if err != nil {
return nil, fmt.Errorf("could not get deposit id: %w", err) return nil, fmt.Errorf("could not get deposit id: %w", err)
} }

View File

@ -77,10 +77,10 @@ func TestParseDeposit(t *testing.T) {
require.NoError(t, err) require.NoError(t, err)
require.Equal(t, Deposit{ require.Equal(t, Deposit{
id: id, IDValue: id,
amount: amount, AmountValue: amount,
from: from, FromValue: from,
to: to, ToValue: to,
}, ev) }, ev)
}) })
} }

View File

@ -8,7 +8,7 @@ import (
) )
type Unbind struct { type Unbind struct {
bindCommon BindCommon
} }
func ParseUnbind(e *state.ContainedNotificationEvent) (event.Event, error) { func ParseUnbind(e *state.ContainedNotificationEvent) (event.Event, error) {
@ -22,12 +22,12 @@ func ParseUnbind(e *state.ContainedNotificationEvent) (event.Event, error) {
return nil, fmt.Errorf("could not parse stack items from notify event: %w", err) return nil, fmt.Errorf("could not parse stack items from notify event: %w", err)
} }
err = parseBind(&ev.bindCommon, params) err = parseBind(&ev.BindCommon, params)
if err != nil { if err != nil {
return nil, err return nil, err
} }
ev.txHash = e.Container ev.TxHashValue = e.Container
return ev, nil return ev, nil
} }

View File

@ -11,22 +11,22 @@ import (
// Withdraw structure of frostfs.Withdraw notification from mainnet chain. // Withdraw structure of frostfs.Withdraw notification from mainnet chain.
type Withdraw struct { type Withdraw struct {
id []byte IDValue []byte
amount int64 // Fixed8 AmountValue int64 // Fixed8
user util.Uint160 UserValue util.Uint160
} }
// MorphEvent implements Neo:Morph Event interface. // MorphEvent implements Neo:Morph Event interface.
func (Withdraw) MorphEvent() {} func (Withdraw) MorphEvent() {}
// ID is a withdraw transaction hash. // ID is a withdraw transaction hash.
func (w Withdraw) ID() []byte { return w.id } func (w Withdraw) ID() []byte { return w.IDValue }
// User returns withdraw receiver script hash from main net. // User returns withdraw receiver script hash from main net.
func (w Withdraw) User() util.Uint160 { return w.user } func (w Withdraw) User() util.Uint160 { return w.UserValue }
// Amount of the withdraw assets. // Amount of the withdraw assets.
func (w Withdraw) Amount() int64 { return w.amount } func (w Withdraw) Amount() int64 { return w.AmountValue }
// ParseWithdraw notification into withdraw structure. // ParseWithdraw notification into withdraw structure.
func ParseWithdraw(e *state.ContainedNotificationEvent) (event.Event, error) { func ParseWithdraw(e *state.ContainedNotificationEvent) (event.Event, error) {
@ -47,19 +47,19 @@ func ParseWithdraw(e *state.ContainedNotificationEvent) (event.Event, error) {
return nil, fmt.Errorf("could not get withdraw user: %w", err) return nil, fmt.Errorf("could not get withdraw user: %w", err)
} }
ev.user, err = util.Uint160DecodeBytesBE(user) ev.UserValue, err = util.Uint160DecodeBytesBE(user)
if err != nil { if err != nil {
return nil, fmt.Errorf("could not convert withdraw user to uint160: %w", err) return nil, fmt.Errorf("could not convert withdraw user to uint160: %w", err)
} }
// parse amount // parse amount
ev.amount, err = client.IntFromStackItem(params[1]) ev.AmountValue, err = client.IntFromStackItem(params[1])
if err != nil { if err != nil {
return nil, fmt.Errorf("could not get withdraw amount: %w", err) return nil, fmt.Errorf("could not get withdraw amount: %w", err)
} }
// parse id // parse id
ev.id, err = client.BytesFromStackItem(params[2]) ev.IDValue, err = client.BytesFromStackItem(params[2])
if err != nil { if err != nil {
return nil, fmt.Errorf("could not get withdraw id: %w", err) return nil, fmt.Errorf("could not get withdraw id: %w", err)
} }

View File

@ -64,9 +64,9 @@ func TestParseWithdraw(t *testing.T) {
require.NoError(t, err) require.NoError(t, err)
require.Equal(t, Withdraw{ require.Equal(t, Withdraw{
id: id, IDValue: id,
amount: amount, AmountValue: amount,
user: user, UserValue: user,
}, ev) }, ev)
}) })
} }