stateroot: allow to sign new roots
This commit is contained in:
parent
ac227a80fe
commit
3c65ed1507
16 changed files with 514 additions and 18 deletions
|
@ -28,4 +28,5 @@ type ApplicationConfiguration struct {
|
|||
UnlockWallet Wallet `yaml:"UnlockWallet"`
|
||||
Oracle OracleConfiguration `yaml:"Oracle"`
|
||||
P2PNotary P2PNotary `yaml:"P2PNotary"`
|
||||
StateRoot StateRoot `yaml:"StateRoot"`
|
||||
}
|
||||
|
|
7
pkg/config/state_root.go
Normal file
7
pkg/config/state_root.go
Normal file
|
@ -0,0 +1,7 @@
|
|||
package config
|
||||
|
||||
// StateRoot contains state root service configuration.
|
||||
type StateRoot struct {
|
||||
Enabled bool `yaml:"Enabled"`
|
||||
UnlockWallet Wallet `yaml:"UnlockWallet"`
|
||||
}
|
|
@ -13,5 +13,8 @@ type StateRoot interface {
|
|||
CurrentValidatedHeight() uint32
|
||||
GetStateProof(root util.Uint256, key []byte) ([][]byte, error)
|
||||
GetStateRoot(height uint32) (*state.MPTRoot, error)
|
||||
GetStateValidators(height uint32) keys.PublicKeys
|
||||
SetSignAndSendCallback(func(*state.MPTRoot) error)
|
||||
SetUpdateValidatorsCallback(func(keys.PublicKeys))
|
||||
UpdateStateValidators(height uint32, pubs keys.PublicKeys)
|
||||
}
|
||||
|
|
20
pkg/core/stateroot/callbacks.go
Normal file
20
pkg/core/stateroot/callbacks.go
Normal file
|
@ -0,0 +1,20 @@
|
|||
package stateroot
|
||||
|
||||
import (
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/state"
|
||||
"github.com/nspcc-dev/neo-go/pkg/crypto/keys"
|
||||
)
|
||||
|
||||
// SetSignAndSendCb sets callback for sending signed root.
|
||||
func (s *Module) SetSignAndSendCallback(f func(*state.MPTRoot) error) {
|
||||
s.mtx.Lock()
|
||||
defer s.mtx.Unlock()
|
||||
s.signAndSendCb = f
|
||||
}
|
||||
|
||||
// SetUpdateValidatorsCallback sets callback for sending signed root.
|
||||
func (s *Module) SetUpdateValidatorsCallback(f func(keys.PublicKeys)) {
|
||||
s.mtx.Lock()
|
||||
defer s.mtx.Unlock()
|
||||
s.updateValidatorsCb = f
|
||||
}
|
|
@ -30,6 +30,9 @@ type (
|
|||
|
||||
mtx sync.RWMutex
|
||||
keys []keyCache
|
||||
|
||||
updateValidatorsCb func(publicKeys keys.PublicKeys)
|
||||
signAndSendCb func(*state.MPTRoot) error
|
||||
}
|
||||
|
||||
keyCache struct {
|
||||
|
|
|
@ -31,6 +31,9 @@ func (s *Module) addLocalStateRoot(sr *state.MPTRoot) error {
|
|||
s.validatedHeight.Store(sr.Index)
|
||||
updateStateHeightMetric(sr.Index)
|
||||
}
|
||||
if s.signAndSendCb != nil {
|
||||
return s.signAndSendCb(sr)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
|
|
|
@ -1,8 +1,6 @@
|
|||
package stateroot
|
||||
|
||||
import (
|
||||
"sort"
|
||||
|
||||
"github.com/nspcc-dev/neo-go/pkg/crypto/hash"
|
||||
"github.com/nspcc-dev/neo-go/pkg/crypto/keys"
|
||||
"github.com/nspcc-dev/neo-go/pkg/smartcontract"
|
||||
|
@ -14,6 +12,9 @@ func (s *Module) UpdateStateValidators(height uint32, pubs keys.PublicKeys) {
|
|||
h := hash.Hash160(script)
|
||||
|
||||
s.mtx.Lock()
|
||||
if s.updateValidatorsCb != nil {
|
||||
s.updateValidatorsCb(pubs)
|
||||
}
|
||||
kc := s.getKeyCacheForHeight(height)
|
||||
if kc.validatorsHash != h {
|
||||
s.keys = append(s.keys, keyCache{
|
||||
|
@ -27,11 +28,17 @@ func (s *Module) UpdateStateValidators(height uint32, pubs keys.PublicKeys) {
|
|||
}
|
||||
|
||||
func (s *Module) getKeyCacheForHeight(h uint32) keyCache {
|
||||
index := sort.Search(len(s.keys), func(i int) bool {
|
||||
return s.keys[i].height >= h
|
||||
})
|
||||
if index == len(s.keys) {
|
||||
return keyCache{}
|
||||
for i := len(s.keys) - 1; i >= 0; i-- {
|
||||
if s.keys[i].height <= h && (i+1 == len(s.keys) || s.keys[i+1].height < h) {
|
||||
return s.keys[i]
|
||||
}
|
||||
}
|
||||
return s.keys[index]
|
||||
return keyCache{}
|
||||
}
|
||||
|
||||
// GetStateValidators returns current state validators.
|
||||
func (s *Module) GetStateValidators(height uint32) keys.PublicKeys {
|
||||
s.mtx.RLock()
|
||||
defer s.mtx.RUnlock()
|
||||
return s.getKeyCacheForHeight(height).validatorsKeys.Copy()
|
||||
}
|
||||
|
|
|
@ -2,10 +2,13 @@ package core
|
|||
|
||||
import (
|
||||
"errors"
|
||||
"os"
|
||||
"path"
|
||||
"sort"
|
||||
"testing"
|
||||
|
||||
"github.com/nspcc-dev/neo-go/internal/testserdes"
|
||||
"github.com/nspcc-dev/neo-go/pkg/config"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/native"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/state"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/storage"
|
||||
|
@ -20,6 +23,7 @@ import (
|
|||
"github.com/nspcc-dev/neo-go/pkg/vm/emit"
|
||||
"github.com/nspcc-dev/neo-go/pkg/wallet"
|
||||
"github.com/stretchr/testify/require"
|
||||
"go.uber.org/zap/zaptest"
|
||||
)
|
||||
|
||||
func testSignStateRoot(t *testing.T, r *state.MPTRoot, pubs keys.PublicKeys, accs ...*wallet.Account) []byte {
|
||||
|
@ -71,7 +75,12 @@ func TestStateRoot(t *testing.T) {
|
|||
updateIndex := bc.BlockHeight()
|
||||
transferTokenFromMultisigAccount(t, bc, h, bc.contracts.GAS.Hash, 1_0000_0000)
|
||||
|
||||
srv, err := stateroot.New(bc.GetStateModule())
|
||||
tmpDir := path.Join(os.TempDir(), "neogo.initsnz")
|
||||
require.NoError(t, os.Mkdir(tmpDir, os.ModePerm))
|
||||
defer os.RemoveAll(tmpDir)
|
||||
w := createAndWriteWallet(t, accs[0], path.Join(tmpDir, "w"), "pass")
|
||||
cfg := createStateRootConfig(w.Path(), "pass")
|
||||
srv, err := stateroot.New(cfg, zaptest.NewLogger(t), bc.GetStateModule())
|
||||
require.NoError(t, err)
|
||||
require.EqualValues(t, 0, srv.CurrentValidatedHeight())
|
||||
r, err := srv.GetStateRoot(bc.BlockHeight())
|
||||
|
@ -136,7 +145,12 @@ func TestStateRootInitNonZeroHeight(t *testing.T) {
|
|||
|
||||
_, err := persistBlock(bc)
|
||||
require.NoError(t, err)
|
||||
srv, err := stateroot.New(bc.GetStateModule())
|
||||
tmpDir := path.Join(os.TempDir(), "neogo.initsnz")
|
||||
require.NoError(t, os.Mkdir(tmpDir, os.ModePerm))
|
||||
defer os.RemoveAll(tmpDir)
|
||||
w := createAndWriteWallet(t, accs[0], path.Join(tmpDir, "w"), "pass")
|
||||
cfg := createStateRootConfig(w.Path(), "pass")
|
||||
srv, err := stateroot.New(cfg, zaptest.NewLogger(t), bc.GetStateModule())
|
||||
require.NoError(t, err)
|
||||
r, err := srv.GetStateRoot(2)
|
||||
require.NoError(t, err)
|
||||
|
@ -151,3 +165,81 @@ func TestStateRootInitNonZeroHeight(t *testing.T) {
|
|||
require.EqualValues(t, 2, srv.CurrentValidatedHeight())
|
||||
require.Equal(t, root, srv.CurrentLocalStateRoot())
|
||||
}
|
||||
|
||||
func createAndWriteWallet(t *testing.T, acc *wallet.Account, path, password string) *wallet.Wallet {
|
||||
w, err := wallet.NewWallet(path)
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, acc.Encrypt(password))
|
||||
w.AddAccount(acc)
|
||||
require.NoError(t, w.Save())
|
||||
w.Close()
|
||||
return w
|
||||
}
|
||||
|
||||
func createStateRootConfig(walletPath, password string) config.StateRoot {
|
||||
return config.StateRoot{
|
||||
Enabled: true,
|
||||
UnlockWallet: config.Wallet{
|
||||
Path: walletPath,
|
||||
Password: password,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
func TestStateRootFull(t *testing.T) {
|
||||
tmpDir := path.Join(os.TempDir(), "neogo.stateroot4")
|
||||
require.NoError(t, os.Mkdir(tmpDir, os.ModePerm))
|
||||
defer os.RemoveAll(tmpDir)
|
||||
|
||||
bc := newTestChain(t)
|
||||
|
||||
h, pubs, accs := newMajorityMultisigWithGAS(t, 2)
|
||||
w := createAndWriteWallet(t, accs[1], path.Join(tmpDir, "wallet2"), "two")
|
||||
cfg := createStateRootConfig(w.Path(), "two")
|
||||
srv, err := stateroot.New(cfg, zaptest.NewLogger(t), bc.GetStateModule())
|
||||
require.NoError(t, err)
|
||||
|
||||
var lastValidated *payload.Extensible
|
||||
srv.SetRelayCallback(func(ep *payload.Extensible) {
|
||||
lastValidated = ep
|
||||
})
|
||||
|
||||
bc.setNodesByRole(t, true, native.RoleStateValidator, pubs)
|
||||
transferTokenFromMultisigAccount(t, bc, h, bc.contracts.GAS.Hash, 1_0000_0000)
|
||||
checkVoteBroadcasted(t, bc, lastValidated, 2, 1)
|
||||
_, err = persistBlock(bc)
|
||||
checkVoteBroadcasted(t, bc, lastValidated, 3, 1)
|
||||
|
||||
r, err := srv.GetStateRoot(2)
|
||||
require.NoError(t, err)
|
||||
require.NoError(t, srv.AddSignature(2, 0, accs[0].PrivateKey().SignHash(r.GetSignedHash())))
|
||||
require.NotNil(t, lastValidated)
|
||||
|
||||
msg := new(stateroot.Message)
|
||||
require.NoError(t, testserdes.DecodeBinary(lastValidated.Data, msg))
|
||||
require.Equal(t, stateroot.RootT, msg.Type)
|
||||
|
||||
actual := msg.Payload.(*state.MPTRoot)
|
||||
require.Equal(t, r.Index, actual.Index)
|
||||
require.Equal(t, r.Version, actual.Version)
|
||||
require.Equal(t, r.Root, actual.Root)
|
||||
}
|
||||
|
||||
func checkVoteBroadcasted(t *testing.T, bc *Blockchain, p *payload.Extensible,
|
||||
height uint32, valIndex byte) {
|
||||
require.NotNil(t, p)
|
||||
m := new(stateroot.Message)
|
||||
require.NoError(t, testserdes.DecodeBinary(p.Data, m))
|
||||
require.Equal(t, stateroot.VoteT, m.Type)
|
||||
vote := m.Payload.(*stateroot.Vote)
|
||||
|
||||
srv := bc.GetStateModule()
|
||||
r, err := srv.GetStateRoot(bc.BlockHeight())
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, height, vote.Height)
|
||||
require.Equal(t, int32(valIndex), vote.ValidatorIndex)
|
||||
|
||||
pubs, _, err := bc.contracts.Designate.GetDesignatedByRole(bc.dao, native.RoleStateValidator, bc.BlockHeight())
|
||||
require.True(t, len(pubs) > int(valIndex))
|
||||
require.True(t, pubs[valIndex].Verify(vote.Signature, r.GetSignedHash().BytesBE()))
|
||||
}
|
||||
|
|
|
@ -173,7 +173,11 @@ func newServerFromConstructors(config ServerConfig, chain blockchainer.Blockchai
|
|||
}
|
||||
})
|
||||
|
||||
sr, err := stateroot.New(chain.GetStateModule())
|
||||
if config.StateRootCfg.Enabled && chain.GetConfig().StateRootInHeader {
|
||||
return nil, errors.New("`StateRootInHeader` should be disabled when state service is enabled")
|
||||
}
|
||||
|
||||
sr, err := stateroot.New(config.StateRootCfg, s.log, chain.GetStateModule())
|
||||
if err != nil {
|
||||
return nil, fmt.Errorf("can't initialize StateRoot service: %w", err)
|
||||
}
|
||||
|
@ -216,6 +220,10 @@ func newServerFromConstructors(config ServerConfig, chain blockchainer.Blockchai
|
|||
|
||||
s.consensus = srv
|
||||
|
||||
if config.StateRootCfg.Enabled {
|
||||
s.stateRoot.SetRelayCallback(s.handleNewPayload)
|
||||
}
|
||||
|
||||
if s.MinPeers < 0 {
|
||||
s.log.Info("bad MinPeers configured, using the default value",
|
||||
zap.Int("configured", s.MinPeers),
|
||||
|
@ -1052,9 +1060,14 @@ func (s *Server) handleNewPayload(p *payload.Extensible) {
|
|||
}
|
||||
|
||||
msg := NewMessage(CMDInv, payload.NewInventory(payload.ExtensibleType, []util.Uint256{p.Hash()}))
|
||||
// It's high priority because it directly affects consensus process,
|
||||
// even though it's just an inv.
|
||||
s.broadcastHPMessage(msg)
|
||||
switch p.Category {
|
||||
case consensus.Category:
|
||||
// It's high priority because it directly affects consensus process,
|
||||
// even though it's just an inv.
|
||||
s.broadcastHPMessage(msg)
|
||||
default:
|
||||
s.broadcastMessage(msg)
|
||||
}
|
||||
}
|
||||
|
||||
func (s *Server) requestTx(hashes ...util.Uint256) {
|
||||
|
|
|
@ -72,6 +72,9 @@ type (
|
|||
|
||||
// P2PNotaryCfg is notary module configuration.
|
||||
P2PNotaryCfg config.P2PNotary
|
||||
|
||||
// StateRootCfg is stateroot module configuration.
|
||||
StateRootCfg config.StateRoot
|
||||
}
|
||||
)
|
||||
|
||||
|
@ -104,5 +107,6 @@ func NewServerConfig(cfg config.Config) ServerConfig {
|
|||
TimePerBlock: time.Duration(protoConfig.SecondsPerBlock) * time.Second,
|
||||
OracleCfg: appConfig.Oracle,
|
||||
P2PNotaryCfg: appConfig.P2PNotary,
|
||||
StateRootCfg: appConfig.StateRoot,
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@ type (
|
|||
|
||||
// Various message types.
|
||||
const (
|
||||
VoteT MessageType = 0
|
||||
RootT MessageType = 1
|
||||
)
|
||||
|
||||
|
@ -40,6 +41,8 @@ func (m *Message) EncodeBinary(w *io.BinWriter) {
|
|||
// DecodeBinary implements io.Serializable interface.
|
||||
func (m *Message) DecodeBinary(r *io.BinReader) {
|
||||
switch m.Type = MessageType(r.ReadB()); m.Type {
|
||||
case VoteT:
|
||||
m.Payload = new(Vote)
|
||||
case RootT:
|
||||
m.Payload = new(state.MPTRoot)
|
||||
default:
|
||||
|
|
98
pkg/services/stateroot/network.go
Normal file
98
pkg/services/stateroot/network.go
Normal file
|
@ -0,0 +1,98 @@
|
|||
package stateroot
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"fmt"
|
||||
|
||||
"github.com/nspcc-dev/neo-go/pkg/config"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/state"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/transaction"
|
||||
"github.com/nspcc-dev/neo-go/pkg/io"
|
||||
"github.com/nspcc-dev/neo-go/pkg/network/payload"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
// RelayCallback represents callback for sending validated state roots.
|
||||
type RelayCallback = func(*payload.Extensible)
|
||||
|
||||
// AddSignature adds state root signature.
|
||||
func (s *service) AddSignature(height uint32, validatorIndex int32, sig []byte) error {
|
||||
if !s.MainCfg.Enabled {
|
||||
return nil
|
||||
}
|
||||
|
||||
pubs := s.GetStateValidators(height)
|
||||
if validatorIndex < 0 || int(validatorIndex) >= len(pubs) {
|
||||
return errors.New("invalid validator index")
|
||||
}
|
||||
pub := pubs[validatorIndex]
|
||||
|
||||
incRoot := s.getIncompleteRoot(height)
|
||||
if incRoot == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
incRoot.Lock()
|
||||
if incRoot.root != nil {
|
||||
ok := pub.Verify(sig, incRoot.root.GetSignedHash().BytesBE())
|
||||
if !ok {
|
||||
incRoot.Unlock()
|
||||
return fmt.Errorf("invalid state root signature for %d", validatorIndex)
|
||||
}
|
||||
}
|
||||
incRoot.addSignature(pub, sig)
|
||||
sr, ready := incRoot.finalize(pubs)
|
||||
incRoot.Unlock()
|
||||
|
||||
if ready {
|
||||
err := s.AddStateRoot(sr)
|
||||
if err != nil {
|
||||
s.log.Error("can't add validated state root", zap.Error(err))
|
||||
}
|
||||
s.sendValidatedRoot(sr)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
// GetConfig returns service configuration.
|
||||
func (s *service) GetConfig() config.StateRoot {
|
||||
return s.MainCfg
|
||||
}
|
||||
|
||||
func (s *service) getIncompleteRoot(height uint32) *incompleteRoot {
|
||||
s.srMtx.Lock()
|
||||
defer s.srMtx.Unlock()
|
||||
if incRoot, ok := s.incompleteRoots[height]; ok {
|
||||
return incRoot
|
||||
}
|
||||
incRoot := &incompleteRoot{sigs: make(map[string]*rootSig)}
|
||||
s.incompleteRoots[height] = incRoot
|
||||
return incRoot
|
||||
}
|
||||
|
||||
func (s *service) sendValidatedRoot(r *state.MPTRoot) {
|
||||
w := io.NewBufBinWriter()
|
||||
m := NewMessage(RootT, r)
|
||||
m.EncodeBinary(w.BinWriter)
|
||||
ep := &payload.Extensible{
|
||||
Network: s.Network,
|
||||
ValidBlockStart: r.Index,
|
||||
ValidBlockEnd: r.Index + transaction.MaxValidUntilBlockIncrement,
|
||||
Sender: s.getAccount().PrivateKey().GetScriptHash(),
|
||||
Data: w.Bytes(),
|
||||
}
|
||||
s.getRelayCallback()(ep)
|
||||
}
|
||||
|
||||
func (s *service) getRelayCallback() RelayCallback {
|
||||
s.cbMtx.RLock()
|
||||
defer s.cbMtx.RUnlock()
|
||||
return s.onValidatedRoot
|
||||
}
|
||||
|
||||
// SetRelayCallback sets callback to pool and broadcast tx.
|
||||
func (s *service) SetRelayCallback(cb RelayCallback) {
|
||||
s.cbMtx.Lock()
|
||||
defer s.cbMtx.Unlock()
|
||||
s.onValidatedRoot = cb
|
||||
}
|
|
@ -1,10 +1,18 @@
|
|||
package stateroot
|
||||
|
||||
import (
|
||||
"errors"
|
||||
"sync"
|
||||
|
||||
"github.com/nspcc-dev/neo-go/pkg/config"
|
||||
"github.com/nspcc-dev/neo-go/pkg/config/netmode"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/blockchainer"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/state"
|
||||
"github.com/nspcc-dev/neo-go/pkg/crypto/keys"
|
||||
"github.com/nspcc-dev/neo-go/pkg/io"
|
||||
"github.com/nspcc-dev/neo-go/pkg/network/payload"
|
||||
"github.com/nspcc-dev/neo-go/pkg/wallet"
|
||||
"go.uber.org/zap"
|
||||
)
|
||||
|
||||
type (
|
||||
|
@ -12,10 +20,28 @@ type (
|
|||
Service interface {
|
||||
blockchainer.StateRoot
|
||||
OnPayload(p *payload.Extensible) error
|
||||
AddSignature(height uint32, validatorIndex int32, sig []byte) error
|
||||
GetConfig() config.StateRoot
|
||||
SetRelayCallback(RelayCallback)
|
||||
}
|
||||
|
||||
service struct {
|
||||
blockchainer.StateRoot
|
||||
|
||||
MainCfg config.StateRoot
|
||||
Network netmode.Magic
|
||||
|
||||
log *zap.Logger
|
||||
accMtx sync.RWMutex
|
||||
myIndex byte
|
||||
wallet *wallet.Wallet
|
||||
acc *wallet.Account
|
||||
|
||||
srMtx sync.Mutex
|
||||
incompleteRoots map[uint32]*incompleteRoot
|
||||
|
||||
cbMtx sync.RWMutex
|
||||
onValidatedRoot RelayCallback
|
||||
}
|
||||
)
|
||||
|
||||
|
@ -25,10 +51,36 @@ const (
|
|||
)
|
||||
|
||||
// New returns new state root service instance using underlying module.
|
||||
func New(mod blockchainer.StateRoot) (Service, error) {
|
||||
return &service{
|
||||
StateRoot: mod,
|
||||
}, nil
|
||||
func New(cfg config.StateRoot, log *zap.Logger, mod blockchainer.StateRoot) (Service, error) {
|
||||
s := &service{
|
||||
StateRoot: mod,
|
||||
log: log,
|
||||
incompleteRoots: make(map[uint32]*incompleteRoot),
|
||||
}
|
||||
|
||||
s.MainCfg = cfg
|
||||
if cfg.Enabled {
|
||||
var err error
|
||||
w := cfg.UnlockWallet
|
||||
if s.wallet, err = wallet.NewWalletFromFile(w.Path); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
haveAccount := false
|
||||
for _, acc := range s.wallet.Accounts {
|
||||
if err := acc.Decrypt(w.Password); err == nil {
|
||||
haveAccount = true
|
||||
break
|
||||
}
|
||||
}
|
||||
if !haveAccount {
|
||||
return nil, errors.New("no wallet account could be unlocked")
|
||||
}
|
||||
|
||||
s.SetUpdateValidatorsCallback(s.updateValidators)
|
||||
s.SetSignAndSendCallback(s.signAndSend)
|
||||
}
|
||||
return s, nil
|
||||
}
|
||||
|
||||
// OnPayload implements Service interface.
|
||||
|
@ -46,6 +98,26 @@ func (s *service) OnPayload(ep *payload.Extensible) error {
|
|||
return nil
|
||||
}
|
||||
return s.AddStateRoot(sr)
|
||||
case VoteT:
|
||||
v := m.Payload.(*Vote)
|
||||
return s.AddSignature(v.Height, v.ValidatorIndex, v.Signature)
|
||||
}
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *service) updateValidators(pubs keys.PublicKeys) {
|
||||
s.accMtx.Lock()
|
||||
defer s.accMtx.Unlock()
|
||||
|
||||
s.acc = nil
|
||||
for i := range pubs {
|
||||
if acc := s.wallet.GetAccount(pubs[i].GetScriptHash()); acc != nil {
|
||||
err := acc.Decrypt(s.MainCfg.UnlockWallet.Password)
|
||||
if err == nil {
|
||||
s.acc = acc
|
||||
s.myIndex = byte(i)
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
88
pkg/services/stateroot/signature.go
Normal file
88
pkg/services/stateroot/signature.go
Normal file
|
@ -0,0 +1,88 @@
|
|||
package stateroot
|
||||
|
||||
import (
|
||||
"sync"
|
||||
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/state"
|
||||
"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/io"
|
||||
"github.com/nspcc-dev/neo-go/pkg/smartcontract"
|
||||
"github.com/nspcc-dev/neo-go/pkg/vm/emit"
|
||||
)
|
||||
|
||||
type (
|
||||
incompleteRoot struct {
|
||||
sync.RWMutex
|
||||
// isSent is true state root was already broadcasted.
|
||||
isSent bool
|
||||
// request is oracle request.
|
||||
root *state.MPTRoot
|
||||
// sigs contains signature from every oracle node.
|
||||
sigs map[string]*rootSig
|
||||
}
|
||||
|
||||
rootSig struct {
|
||||
// pub is cached public key.
|
||||
pub *keys.PublicKey
|
||||
// ok is true if signature was verified.
|
||||
ok bool
|
||||
// sig is state root signature.
|
||||
sig []byte
|
||||
}
|
||||
)
|
||||
|
||||
func newIncompleteRoot() *incompleteRoot {
|
||||
return &incompleteRoot{
|
||||
sigs: make(map[string]*rootSig),
|
||||
}
|
||||
}
|
||||
|
||||
func (r *incompleteRoot) reverify() {
|
||||
txHash := r.root.GetSignedHash()
|
||||
for _, sig := range r.sigs {
|
||||
if !sig.ok {
|
||||
sig.ok = sig.pub.Verify(sig.sig, txHash.BytesBE())
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (r *incompleteRoot) addSignature(pub *keys.PublicKey, sig []byte) {
|
||||
r.sigs[string(pub.Bytes())] = &rootSig{
|
||||
pub: pub,
|
||||
ok: r.root != nil,
|
||||
sig: sig,
|
||||
}
|
||||
}
|
||||
|
||||
// finalize checks is either main or backup tx has sufficient number of signatures and returns
|
||||
// tx and bool value indicating if it is ready to be broadcasted.
|
||||
func (r *incompleteRoot) finalize(stateValidators keys.PublicKeys) (*state.MPTRoot, bool) {
|
||||
if r.root == nil {
|
||||
return nil, false
|
||||
}
|
||||
|
||||
m := smartcontract.GetDefaultHonestNodeCount(len(stateValidators))
|
||||
sigs := make([][]byte, 0, m)
|
||||
for _, pub := range stateValidators {
|
||||
sig, ok := r.sigs[string(pub.Bytes())]
|
||||
if ok && sig.ok {
|
||||
sigs = append(sigs, sig.sig)
|
||||
if len(sigs) == m {
|
||||
break
|
||||
}
|
||||
}
|
||||
}
|
||||
if len(sigs) != m {
|
||||
return nil, false
|
||||
}
|
||||
|
||||
w := io.NewBufBinWriter()
|
||||
for i := range sigs {
|
||||
emit.Bytes(w.BinWriter, sigs[i])
|
||||
}
|
||||
r.root.Witness = &transaction.Witness{
|
||||
InvocationScript: w.Bytes(),
|
||||
}
|
||||
return r.root, true
|
||||
}
|
55
pkg/services/stateroot/validators.go
Normal file
55
pkg/services/stateroot/validators.go
Normal file
|
@ -0,0 +1,55 @@
|
|||
package stateroot
|
||||
|
||||
import (
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/state"
|
||||
"github.com/nspcc-dev/neo-go/pkg/core/transaction"
|
||||
"github.com/nspcc-dev/neo-go/pkg/io"
|
||||
"github.com/nspcc-dev/neo-go/pkg/network/payload"
|
||||
"github.com/nspcc-dev/neo-go/pkg/wallet"
|
||||
)
|
||||
|
||||
func (s *service) signAndSend(r *state.MPTRoot) error {
|
||||
if !s.MainCfg.Enabled {
|
||||
return nil
|
||||
}
|
||||
|
||||
acc := s.getAccount()
|
||||
if acc == nil {
|
||||
return nil
|
||||
}
|
||||
|
||||
sig := acc.PrivateKey().SignHash(r.GetSignedHash())
|
||||
incRoot := s.getIncompleteRoot(r.Index)
|
||||
incRoot.root = r
|
||||
incRoot.addSignature(acc.PrivateKey().PublicKey(), sig)
|
||||
incRoot.reverify()
|
||||
|
||||
s.accMtx.RLock()
|
||||
myIndex := s.myIndex
|
||||
s.accMtx.RUnlock()
|
||||
msg := NewMessage(VoteT, &Vote{
|
||||
ValidatorIndex: int32(myIndex),
|
||||
Height: r.Index,
|
||||
Signature: sig,
|
||||
})
|
||||
|
||||
w := io.NewBufBinWriter()
|
||||
msg.EncodeBinary(w.BinWriter)
|
||||
if w.Err != nil {
|
||||
return w.Err
|
||||
}
|
||||
s.getRelayCallback()(&payload.Extensible{
|
||||
Network: s.Network,
|
||||
ValidBlockStart: r.Index,
|
||||
ValidBlockEnd: r.Index + transaction.MaxValidUntilBlockIncrement,
|
||||
Sender: s.getAccount().PrivateKey().GetScriptHash(),
|
||||
Data: w.Bytes(),
|
||||
})
|
||||
return nil
|
||||
}
|
||||
|
||||
func (s *service) getAccount() *wallet.Account {
|
||||
s.accMtx.RLock()
|
||||
defer s.accMtx.RUnlock()
|
||||
return s.acc
|
||||
}
|
27
pkg/services/stateroot/vote.go
Normal file
27
pkg/services/stateroot/vote.go
Normal file
|
@ -0,0 +1,27 @@
|
|||
package stateroot
|
||||
|
||||
import (
|
||||
"github.com/nspcc-dev/neo-go/pkg/crypto/keys"
|
||||
"github.com/nspcc-dev/neo-go/pkg/io"
|
||||
)
|
||||
|
||||
// Vote represents vote message.
|
||||
type Vote struct {
|
||||
ValidatorIndex int32
|
||||
Height uint32
|
||||
Signature []byte
|
||||
}
|
||||
|
||||
// EncodeBinary implements io.Serializable interface.
|
||||
func (p *Vote) EncodeBinary(w *io.BinWriter) {
|
||||
w.WriteU32LE(uint32(p.ValidatorIndex))
|
||||
w.WriteU32LE(p.Height)
|
||||
w.WriteVarBytes(p.Signature)
|
||||
}
|
||||
|
||||
// DecodeBinary implements io.Serializable interface.
|
||||
func (p *Vote) DecodeBinary(r *io.BinReader) {
|
||||
p.ValidatorIndex = int32(r.ReadU32LE())
|
||||
p.Height = r.ReadU32LE()
|
||||
p.Signature = r.ReadVarBytes(keys.SignatureLen)
|
||||
}
|
Loading…
Reference in a new issue