Do massive overhaul of consensus part due to new architecture with blockchain, implement chain selection

This commit is contained in:
ChronosX88 2021-06-09 00:30:23 +03:00
parent 012d4a68c3
commit 4e7294e046
Signed by: ChronosXYZ
GPG Key ID: 085A69A82C8C511A
26 changed files with 712 additions and 808 deletions

View File

@ -17,7 +17,7 @@ type BeaconResult struct {
type BeaconNetworks []BeaconNetwork type BeaconNetworks []BeaconNetwork
func (bn BeaconNetworks) BeaconNetworkForRound(e types.DrandRound) BeaconAPI { func (bn BeaconNetworks) BeaconNetworkForRound(e uint64) BeaconAPI {
for i := len(bn) - 1; i >= 0; i-- { for i := len(bn) - 1; i >= 0; i-- {
bp := bn[i] bp := bn[i]
if e >= bp.Start { if e >= bp.Start {
@ -28,7 +28,7 @@ func (bn BeaconNetworks) BeaconNetworkForRound(e types.DrandRound) BeaconAPI {
} }
type BeaconNetwork struct { type BeaconNetwork struct {
Start types.DrandRound Start uint64
Beacon BeaconAPI Beacon BeaconAPI
} }

View File

@ -6,6 +6,10 @@ import (
"fmt" "fmt"
"sync" "sync"
"github.com/Arceliar/phony"
"github.com/Secured-Finance/dione/consensus"
"github.com/Secured-Finance/dione/beacon" "github.com/Secured-Finance/dione/beacon"
"github.com/drand/drand/chain" "github.com/drand/drand/chain"
"github.com/drand/drand/client" "github.com/drand/drand/client"
@ -30,16 +34,17 @@ var log = logrus.WithFields(logrus.Fields{
}) })
type DrandBeacon struct { type DrandBeacon struct {
phony.Inbox
DrandClient client.Client DrandClient client.Client
PublicKey kyber.Point PublicKey kyber.Point
drandResultChannel <-chan client.Result drandResultChannel <-chan client.Result
cacheLock sync.Mutex
cacheLock sync.Mutex localCache map[uint64]types.BeaconEntry
localCache map[uint64]types.BeaconEntry latestDrandRound uint64
latestDrandRound uint64 consensusManager *consensus.PBFTConsensusManager
} }
func NewDrandBeacon(ps *pubsub.PubSub) (*DrandBeacon, error) { func NewDrandBeacon(ps *pubsub.PubSub, pcm *consensus.PBFTConsensusManager) (*DrandBeacon, error) {
cfg := config.NewDrandConfig() cfg := config.NewDrandConfig()
drandChain, err := chain.InfoFromJSON(bytes.NewReader([]byte(cfg.ChainInfo))) drandChain, err := chain.InfoFromJSON(bytes.NewReader([]byte(cfg.ChainInfo)))
@ -78,8 +83,9 @@ func NewDrandBeacon(ps *pubsub.PubSub) (*DrandBeacon, error) {
} }
db := &DrandBeacon{ db := &DrandBeacon{
DrandClient: drandClient, DrandClient: drandClient,
localCache: make(map[uint64]types.BeaconEntry), localCache: make(map[uint64]types.BeaconEntry),
consensusManager: pcm,
} }
db.PublicKey = drandChain.PublicKey db.PublicKey = drandChain.PublicKey
@ -116,6 +122,7 @@ func (db *DrandBeacon) loop(ctx context.Context) {
{ {
db.cacheValue(newBeaconResultFromDrandResult(res)) db.cacheValue(newBeaconResultFromDrandResult(res))
db.updateLatestDrandRound(res.Round()) db.updateLatestDrandRound(res.Round())
db.consensusManager.NewDrandRound(db, res)
} }
} }
} }

View File

@ -0,0 +1,57 @@
package pool
import (
"encoding/hex"
"github.com/Secured-Finance/dione/blockchain/types"
"github.com/Secured-Finance/dione/cache"
)
// BlockPool is pool for blocks that isn't not validated or committed yet
type BlockPool struct {
knownBlocks cache.Cache
acceptedBlocks cache.Cache
}
func NewBlockPool() (*BlockPool, error) {
bp := &BlockPool{
acceptedBlocks: cache.NewInMemoryCache(), // here we need to use separate cache
}
return bp, nil
}
func (bp *BlockPool) AddBlock(block *types.Block) error {
return bp.knownBlocks.Store(hex.EncodeToString(block.Header.Hash), block)
}
func (bp *BlockPool) GetBlock(blockhash []byte) (*types.Block, error) {
var block *types.Block
return block, bp.knownBlocks.Get(hex.EncodeToString(blockhash), &block)
}
// PruneBlocks cleans known blocks list. It is called when new consensus round starts.
func (bp *BlockPool) PruneBlocks() {
for k := range bp.knownBlocks.Items() {
bp.knownBlocks.Delete(k)
}
}
func (bp *BlockPool) AddAcceptedBlock(block *types.Block) error {
return bp.acceptedBlocks.Store(hex.EncodeToString(block.Header.Hash), block)
}
func (bp *BlockPool) GetAllAcceptedBlocks() []*types.Block {
var blocks []*types.Block
for _, v := range bp.acceptedBlocks.Items() {
blocks = append(blocks, v.(*types.Block))
}
return blocks
}
// PruneAcceptedBlocks cleans accepted blocks list. It is called when new consensus round starts.
func (bp *BlockPool) PruneAcceptedBlocks() {
for k := range bp.acceptedBlocks.Items() {
bp.acceptedBlocks.Delete(k)
}
}

View File

@ -3,10 +3,13 @@ package types
import ( import (
"time" "time"
"github.com/libp2p/go-libp2p-core/crypto"
"github.com/ethereum/go-ethereum/common"
"github.com/wealdtech/go-merkletree" "github.com/wealdtech/go-merkletree"
"github.com/wealdtech/go-merkletree/keccak256" "github.com/wealdtech/go-merkletree/keccak256"
"github.com/Secured-Finance/dione/wallet"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p-core/peer"
) )
@ -22,6 +25,7 @@ type BlockHeader struct {
LastHash []byte LastHash []byte
LastHashProof *merkletree.Proof LastHashProof *merkletree.Proof
Proposer peer.ID Proposer peer.ID
ProposerEth common.Address
Signature []byte Signature []byte
} }
@ -36,12 +40,8 @@ func GenesisBlock() *Block {
} }
} }
func CreateBlock(lastBlockHeader *BlockHeader, txs []*Transaction, wallet *wallet.LocalWallet) (*Block, error) { func CreateBlock(lastBlockHeader *BlockHeader, txs []*Transaction, minerEth common.Address, privateKey crypto.PrivKey) (*Block, error) {
timestamp := time.Now().Unix() timestamp := time.Now().Unix()
proposer, err := wallet.GetDefault()
if err != nil {
return nil, err
}
// extract hashes from transactions // extract hashes from transactions
var txHashes [][]byte var txHashes [][]byte
@ -58,8 +58,8 @@ func CreateBlock(lastBlockHeader *BlockHeader, txs []*Transaction, wallet *walle
// fetch merkle tree root hash (block hash) // fetch merkle tree root hash (block hash)
blockHash := tree.Root() blockHash := tree.Root()
// sign this block hash // sign the block hash
s, err := wallet.Sign(proposer, blockHash) s, err := privateKey.Sign(blockHash)
if err != nil { if err != nil {
return nil, err return nil, err
} }
@ -69,12 +69,18 @@ func CreateBlock(lastBlockHeader *BlockHeader, txs []*Transaction, wallet *walle
return nil, err return nil, err
} }
proposer, err := peer.IDFromPrivateKey(privateKey)
if err != nil {
return nil, err
}
block := &Block{ block := &Block{
Header: &BlockHeader{ Header: &BlockHeader{
Timestamp: timestamp, Timestamp: timestamp,
Height: lastBlockHeader.Height + 1, Height: lastBlockHeader.Height + 1,
Proposer: proposer, Proposer: proposer,
Signature: s.Data, ProposerEth: minerEth,
Signature: s,
Hash: blockHash, Hash: blockHash,
LastHash: lastBlockHeader.Hash, LastHash: lastBlockHeader.Hash,
LastHashProof: lastHashProof, LastHashProof: lastHashProof,

View File

@ -28,3 +28,5 @@ func NewDrandConfig() *DrandConfig {
} }
return cfg return cfg
} }
var DrandChainGenesisTime = uint64(1603603302)

View File

@ -1,9 +0,0 @@
package config
var ExpectedLeadersPerEpoch = int64(5)
var TasksPerEpoch = uint64(ExpectedLeadersPerEpoch)
var ChainGenesis = uint64(1603603302)
var TaskEpochInterval = uint64(15)

5
config/win_config.go Normal file
View File

@ -0,0 +1,5 @@
package config
import "math/big"
var ExpectedLeadersPerEpoch = big.NewInt(5)

View File

@ -1,11 +1,21 @@
package consensus package consensus
import ( import (
"fmt" "errors"
"math/big" "math/big"
"sync" "sync"
"github.com/Secured-Finance/dione/cache" "github.com/Secured-Finance/dione/blockchain"
"github.com/drand/drand/client"
"github.com/Arceliar/phony"
types3 "github.com/Secured-Finance/dione/blockchain/types"
"github.com/libp2p/go-libp2p-core/crypto"
"github.com/Secured-Finance/dione/blockchain/pool"
"github.com/Secured-Finance/dione/consensus/types" "github.com/Secured-Finance/dione/consensus/types"
@ -13,65 +23,87 @@ import (
"github.com/sirupsen/logrus" "github.com/sirupsen/logrus"
"github.com/Secured-Finance/dione/pubsub" "github.com/Secured-Finance/dione/pubsub"
types2 "github.com/Secured-Finance/dione/types" )
type StateStatus uint8
const (
StateStatusUnknown = iota
StateStatusPrePrepared
StateStatusPrepared
StateStatusCommited
) )
type PBFTConsensusManager struct { type PBFTConsensusManager struct {
phony.Inbox
psb *pubsub.PubSubRouter psb *pubsub.PubSubRouter
minApprovals int minApprovals int // FIXME
privKey []byte privKey crypto.PrivKey
msgLog *MessageLog msgLog *ConsensusMessageLog
validator *ConsensusValidator validator *ConsensusValidator
consensusMap map[string]*Consensus
ethereumClient *ethclient.EthereumClient ethereumClient *ethclient.EthereumClient
miner *Miner miner *Miner
cache cache.Cache blockPool pool.BlockPool
blockchain blockchain.BlockChain
state *State
} }
type Consensus struct { type State struct {
mutex sync.Mutex mutex sync.Mutex
Finished bool drandRound uint64
IsCurrentMinerLeader bool randomness []byte
Task *types2.DioneTask blockHeight uint64
status StateStatus
} }
func NewPBFTConsensusManager(psb *pubsub.PubSubRouter, minApprovals int, privKey []byte, ethereumClient *ethclient.EthereumClient, miner *Miner, evc cache.Cache) *PBFTConsensusManager { func NewPBFTConsensusManager(psb *pubsub.PubSubRouter, minApprovals int, privKey crypto.PrivKey, ethereumClient *ethclient.EthereumClient, miner *Miner) *PBFTConsensusManager {
pcm := &PBFTConsensusManager{} pcm := &PBFTConsensusManager{}
pcm.psb = psb pcm.psb = psb
pcm.miner = miner pcm.miner = miner
pcm.validator = NewConsensusValidator(evc, miner) pcm.validator = NewConsensusValidator(miner)
pcm.msgLog = NewMessageLog() pcm.msgLog = NewConsensusMessageLog()
pcm.minApprovals = minApprovals pcm.minApprovals = minApprovals
pcm.privKey = privKey pcm.privKey = privKey
pcm.ethereumClient = ethereumClient pcm.ethereumClient = ethereumClient
pcm.cache = evc pcm.state = &State{}
pcm.consensusMap = map[string]*Consensus{} pcm.psb.Hook(pubsub.PrePrepareMessageType, pcm.handlePrePrepare, types.PrePrepareMessage{})
pcm.psb.Hook(pubsub.PrePrepareMessageType, pcm.handlePrePrepare, types2.DioneTask{}) pcm.psb.Hook(pubsub.PrepareMessageType, pcm.handlePrepare, types.PrepareMessage{})
pcm.psb.Hook(pubsub.PrepareMessageType, pcm.handlePrepare, types2.DioneTask{}) pcm.psb.Hook(pubsub.CommitMessageType, pcm.handleCommit, types.CommitMessage{})
pcm.psb.Hook(pubsub.CommitMessageType, pcm.handleCommit, types2.DioneTask{})
return pcm return pcm
} }
func (pcm *PBFTConsensusManager) Propose(task types2.DioneTask) error { func (pcm *PBFTConsensusManager) propose(blk *types3.Block) error {
pcm.createConsensusInfo(&task, true) pcm.state.mutex.Lock()
defer pcm.state.mutex.Unlock()
prePrepareMsg, err := CreatePrePrepareWithTaskSignature(&task, pcm.privKey) prePrepareMsg, err := NewMessage(types.ConsensusMessage{Block: blk}, types.ConsensusMessageTypePrePrepare, pcm.privKey)
if err != nil { if err != nil {
return err return err
} }
pcm.psb.BroadcastToServiceTopic(prePrepareMsg) pcm.psb.BroadcastToServiceTopic(prePrepareMsg)
pcm.state.status = StateStatusPrePrepared
return nil return nil
} }
func (pcm *PBFTConsensusManager) handlePrePrepare(message *pubsub.GenericMessage) { func (pcm *PBFTConsensusManager) handlePrePrepare(message *pubsub.GenericMessage) {
cmsg, err := unmarshalPayload(message) pcm.state.mutex.Lock()
if err != nil { defer pcm.state.mutex.Unlock()
prePrepare, ok := message.Payload.(types.PrePrepareMessage)
if !ok {
logrus.Warn("failed to convert payload to PrePrepare message")
return return
} }
if cmsg.Task.Miner == pcm.miner.address { if prePrepare.Block.Header.Proposer == pcm.miner.address {
return return
} }
cmsg := types.ConsensusMessage{
Type: types.ConsensusMessageTypePrePrepare,
From: message.From,
Block: prePrepare.Block,
}
if pcm.msgLog.Exists(cmsg) { if pcm.msgLog.Exists(cmsg) {
logrus.Debugf("received existing pre_prepare msg, dropping...") logrus.Debugf("received existing pre_prepare msg, dropping...")
return return
@ -82,21 +114,43 @@ func (pcm *PBFTConsensusManager) handlePrePrepare(message *pubsub.GenericMessage
} }
pcm.msgLog.AddMessage(cmsg) pcm.msgLog.AddMessage(cmsg)
pcm.blockPool.AddBlock(cmsg.Block)
prepareMsg, err := NewMessage(message, pubsub.PrepareMessageType) prepareMsg, err := NewMessage(cmsg, types.ConsensusMessageTypePrepare, pcm.privKey)
if err != nil { if err != nil {
logrus.Errorf("failed to create prepare message: %v", err) logrus.Errorf("failed to create prepare message: %v", err)
return return
} }
pcm.createConsensusInfo(&cmsg.Task, false) pcm.psb.BroadcastToServiceTopic(prepareMsg)
pcm.state.status = StateStatusPrePrepared
pcm.psb.BroadcastToServiceTopic(&prepareMsg)
} }
func (pcm *PBFTConsensusManager) handlePrepare(message *pubsub.GenericMessage) { func (pcm *PBFTConsensusManager) handlePrepare(message *pubsub.GenericMessage) {
cmsg, err := unmarshalPayload(message) pcm.state.mutex.Lock()
defer pcm.state.mutex.Unlock()
prepare, ok := message.Payload.(types.PrepareMessage)
if !ok {
logrus.Warn("failed to convert payload to Prepare message")
return
}
cmsg := types.ConsensusMessage{
Type: types.ConsensusMessageTypePrepare,
From: message.From,
Blockhash: prepare.Blockhash,
Signature: prepare.Signature, // TODO check the signature
}
pk, _ := message.From.ExtractPublicKey()
ok, err := pk.Verify(cmsg.Blockhash, cmsg.Signature)
if err != nil { if err != nil {
logrus.Warnf("Failed to verify PREPARE message signature: %s", err.Error())
return
}
if !ok {
logrus.Errorf("Signature of PREPARE message of peer %s isn't valid!", cmsg.From)
return return
} }
@ -111,18 +165,41 @@ func (pcm *PBFTConsensusManager) handlePrepare(message *pubsub.GenericMessage) {
pcm.msgLog.AddMessage(cmsg) pcm.msgLog.AddMessage(cmsg)
if len(pcm.msgLog.Get(types.MessageTypePrepare, cmsg.Task.ConsensusID)) >= pcm.minApprovals { if len(pcm.msgLog.Get(types.ConsensusMessageTypePrepare, cmsg.Blockhash)) >= pcm.minApprovals {
commitMsg, err := NewMessage(message, pubsub.CommitMessageType) commitMsg, err := NewMessage(cmsg, types.ConsensusMessageTypeCommit, pcm.privKey)
if err != nil { if err != nil {
logrus.Errorf("failed to create commit message: %w", err) logrus.Errorf("failed to create commit message: %v", err)
} }
pcm.psb.BroadcastToServiceTopic(&commitMsg) pcm.psb.BroadcastToServiceTopic(commitMsg)
pcm.state.status = StateStatusPrepared
} }
} }
func (pcm *PBFTConsensusManager) handleCommit(message *pubsub.GenericMessage) { func (pcm *PBFTConsensusManager) handleCommit(message *pubsub.GenericMessage) {
cmsg, err := unmarshalPayload(message) pcm.state.mutex.Lock()
defer pcm.state.mutex.Unlock()
commit, ok := message.Payload.(types.CommitMessage)
if !ok {
logrus.Warn("failed to convert payload to Prepare message")
return
}
cmsg := types.ConsensusMessage{
Type: types.ConsensusMessageTypeCommit,
From: message.From,
Blockhash: commit.Blockhash,
Signature: commit.Signature, // TODO check the signature
}
pk, _ := message.From.ExtractPublicKey()
ok, err := pk.Verify(cmsg.Blockhash, cmsg.Signature)
if err != nil { if err != nil {
logrus.Warnf("Failed to verify COMMIT message signature: %s", err.Error())
return
}
if !ok {
logrus.Errorf("Signature of COMMIT message of peer %s isn't valid!", cmsg.From)
return return
} }
@ -137,80 +214,70 @@ func (pcm *PBFTConsensusManager) handleCommit(message *pubsub.GenericMessage) {
pcm.msgLog.AddMessage(cmsg) pcm.msgLog.AddMessage(cmsg)
if len(pcm.msgLog.Get(types.MessageTypeCommit, cmsg.Task.ConsensusID)) >= pcm.minApprovals { if len(pcm.msgLog.Get(types.ConsensusMessageTypeCommit, cmsg.Blockhash)) >= pcm.minApprovals {
info := pcm.GetConsensusInfo(cmsg.Task.ConsensusID) block, err := pcm.blockPool.GetBlock(cmsg.Blockhash)
if info == nil { if err != nil {
logrus.Debugf("consensus doesn't exist in our consensus map - skipping...") logrus.Debug(err)
return return
} }
info.mutex.Lock() pcm.blockPool.AddAcceptedBlock(block)
defer info.mutex.Unlock() pcm.state.status = StateStatusCommited
if info.Finished { }
return }
}
if info.IsCurrentMinerLeader {
logrus.Infof("Submitting on-chain result for consensus ID: %s", cmsg.Task.ConsensusID)
reqID, ok := new(big.Int).SetString(cmsg.Task.RequestID, 10)
if !ok {
logrus.Errorf("Failed to parse request ID: %v", cmsg.Task.RequestID)
}
err := pcm.ethereumClient.SubmitRequestAnswer(reqID, cmsg.Task.Payload) func (pcm *PBFTConsensusManager) NewDrandRound(from phony.Actor, res client.Result) {
pcm.Act(from, func() {
pcm.state.mutex.Lock()
defer pcm.state.mutex.Unlock()
block, err := pcm.commitAcceptedBlocks()
if err != nil {
logrus.Errorf("Failed to select the block in consensus round %d: %s", pcm.state.blockHeight, err.Error())
return
}
minedBlock, err := pcm.miner.MineBlock(res.Randomness(), res.Round(), block.Header)
if err != nil {
logrus.Errorf("Failed to mine the block: %s", err.Error())
return
}
pcm.state.drandRound = res.Round()
pcm.state.randomness = res.Randomness()
pcm.state.blockHeight = pcm.state.blockHeight + 1
// if we are round winner
if minedBlock != nil {
err = pcm.propose(minedBlock)
if err != nil { if err != nil {
logrus.Errorf("Failed to submit on-chain result: %v", err) logrus.Errorf("Failed to propose the block: %s", err.Error())
return
} }
} }
})
info.Finished = true
}
} }
func (pcm *PBFTConsensusManager) createConsensusInfo(task *types2.DioneTask, isLeader bool) { func (pcm *PBFTConsensusManager) commitAcceptedBlocks() (*types3.Block, error) {
if _, ok := pcm.consensusMap[task.ConsensusID]; !ok { blocks := pcm.blockPool.GetAllAcceptedBlocks()
pcm.consensusMap[task.ConsensusID] = &Consensus{ if blocks == nil {
IsCurrentMinerLeader: isLeader, return nil, errors.New("there is no accepted blocks")
Task: task,
Finished: false,
}
} }
} var maxStake *big.Int
var selectedBlock *types3.Block
for _, v := range blocks {
stake, err := pcm.ethereumClient.GetMinerStake(v.Header.ProposerEth)
if err != nil {
return nil, err
}
func (pcm *PBFTConsensusManager) GetConsensusInfo(consensusID string) *Consensus { if maxStake != nil {
c, ok := pcm.consensusMap[consensusID] if stake.Cmp(maxStake) == -1 {
if !ok { continue
return nil }
}
maxStake = stake
selectedBlock = v
} }
logrus.Debugf("Selected block of miner %s", selectedBlock.Header.ProposerEth.Hex())
return c pcm.blockPool.PruneAcceptedBlocks()
} return selectedBlock, pcm.blockchain.StoreBlock(selectedBlock)
func unmarshalPayload(msg *pubsub.GenericMessage) (types.ConsensusMessage, error) {
task, ok := msg.Payload.(types2.DioneTask)
if !ok {
return types.ConsensusMessage{}, fmt.Errorf("cannot convert payload to DioneTask")
}
var consensusMessageType types.MessageType
switch msg.Type {
case pubsub.PrePrepareMessageType:
{
consensusMessageType = types.MessageTypePrePrepare
break
}
case pubsub.PrepareMessageType:
{
consensusMessageType = types.MessageTypePrepare
break
}
case pubsub.CommitMessageType:
{
consensusMessageType = types.MessageTypeCommit
break
}
}
cmsg := types.ConsensusMessage{
Type: consensusMessageType,
From: msg.From,
Task: task,
}
return cmsg, nil
} }

View File

@ -1,153 +1,127 @@
package consensus package consensus
import ( import (
"github.com/Secured-Finance/dione/cache"
types2 "github.com/Secured-Finance/dione/consensus/types" types2 "github.com/Secured-Finance/dione/consensus/types"
"github.com/Secured-Finance/dione/consensus/validation"
"github.com/Secured-Finance/dione/contracts/dioneOracle"
"github.com/Secured-Finance/dione/types"
"github.com/ethereum/go-ethereum/common"
"github.com/filecoin-project/go-state-types/crypto"
"github.com/sirupsen/logrus"
) )
type ConsensusValidator struct { type ConsensusValidator struct {
validationFuncMap map[types2.MessageType]func(msg types2.ConsensusMessage) bool validationFuncMap map[types2.ConsensusMessageType]func(msg types2.ConsensusMessage) bool
cache cache.Cache
miner *Miner miner *Miner
} }
func NewConsensusValidator(ec cache.Cache, miner *Miner) *ConsensusValidator { func NewConsensusValidator(miner *Miner) *ConsensusValidator {
cv := &ConsensusValidator{ cv := &ConsensusValidator{
cache: ec,
miner: miner, miner: miner,
} }
cv.validationFuncMap = map[types2.MessageType]func(msg types2.ConsensusMessage) bool{ cv.validationFuncMap = map[types2.ConsensusMessageType]func(msg types2.ConsensusMessage) bool{
types2.MessageTypePrePrepare: func(msg types2.ConsensusMessage) bool { // FIXME it all
// TODO here we need to do validation of tx itself //types2.ConsensusMessageTypePrePrepare: func(msg types2.PrePrepareMessage) bool {
// // TODO here we need to do validation of block itself
// === verify task signature === //
err := VerifyTaskSignature(msg.Task) // // === verify task signature ===
if err != nil { // err := VerifyTaskSignature(msg.Task)
logrus.Errorf("unable to verify signature: %v", err) // if err != nil {
return false // logrus.Errorf("unable to verify signature: %v", err)
} // return false
///////////////////////////////// // }
// /////////////////////////////////
// === verify if request exists in cache === //
var requestEvent *dioneOracle.DioneOracleNewOracleRequest // // === verify if request exists in cache ===
err = cv.cache.Get("request_"+msg.Task.RequestID, &requestEvent) // var requestEvent *dioneOracle.DioneOracleNewOracleRequest
if err != nil { // err = cv.cache.Get("request_"+msg.Task.RequestID, &requestEvent)
logrus.Errorf("the request doesn't exist in the cache or has been failed to decode: %v", err) // if err != nil {
return false // logrus.Errorf("the request doesn't exist in the cache or has been failed to decode: %v", err)
} // return false
// }
if requestEvent.OriginChain != msg.Task.OriginChain || //
requestEvent.RequestType != msg.Task.RequestType || // if requestEvent.OriginChain != msg.Task.OriginChain ||
requestEvent.RequestParams != msg.Task.RequestParams { // requestEvent.RequestType != msg.Task.RequestType ||
// requestEvent.RequestParams != msg.Task.RequestParams {
logrus.Errorf("the incoming task and cached request requestEvent don't match!") //
return false // logrus.Errorf("the incoming task and cached request requestEvent don't match!")
} // return false
///////////////////////////////// // }
// /////////////////////////////////
// === verify election proof wincount preliminarily === //
if msg.Task.ElectionProof.WinCount < 1 { // // === verify election proof wincount preliminarily ===
logrus.Error("miner isn't a winner!") // if msg.Task.ElectionProof.WinCount < 1 {
return false // logrus.Error("miner isn't a winner!")
} // return false
///////////////////////////////// // }
// /////////////////////////////////
// === verify miner's eligibility to propose this task === //
err = cv.miner.IsMinerEligibleToProposeTask(common.HexToAddress(msg.Task.MinerEth)) // // === verify miner's eligibility to propose this task ===
if err != nil { // err = cv.miner.IsMinerEligibleToProposeBlock(common.HexToAddress(msg.Task.MinerEth))
logrus.Errorf("miner is not eligible to propose task: %v", err) // if err != nil {
return false // logrus.Errorf("miner is not eligible to propose task: %v", err)
} // return false
///////////////////////////////// // }
// /////////////////////////////////
// === verify election proof vrf === //
minerAddressMarshalled, err := msg.Task.Miner.MarshalBinary() // // === verify election proof vrf ===
if err != nil { // minerAddressMarshalled, err := msg.Task.Miner.MarshalBinary()
logrus.Errorf("failed to marshal miner address: %v", err) // if err != nil {
return false // logrus.Errorf("failed to marshal miner address: %v", err)
} // return false
electionProofRandomness, err := DrawRandomness( // }
msg.Task.BeaconEntries[1].Data, // electionProofRandomness, err := DrawRandomness(
crypto.DomainSeparationTag_ElectionProofProduction, // msg.Task.BeaconEntries[1].Data,
msg.Task.DrandRound, // crypto.DomainSeparationTag_ElectionProofProduction,
minerAddressMarshalled, // msg.Task.DrandRound,
) // minerAddressMarshalled,
if err != nil { // )
logrus.Errorf("failed to draw electionProofRandomness: %v", err) // if err != nil {
return false // logrus.Errorf("failed to draw electionProofRandomness: %v", err)
} // return false
err = VerifyVRF(msg.Task.Miner, electionProofRandomness, msg.Task.ElectionProof.VRFProof) // }
if err != nil { // err = VerifyVRF(msg.Task.Miner, electionProofRandomness, msg.Task.ElectionProof.VRFProof)
logrus.Errorf("failed to verify election proof vrf: %v", err) // if err != nil {
} // logrus.Errorf("failed to verify election proof vrf: %v", err)
////////////////////////////////////// // }
// //////////////////////////////////////
// === verify ticket vrf === //
ticketRandomness, err := DrawRandomness( // // === compute wincount locally and verify values ===
msg.Task.BeaconEntries[1].Data, // mStake, nStake, err := cv.miner.GetStakeInfo(common.HexToAddress(msg.Task.MinerEth))
crypto.DomainSeparationTag_TicketProduction, // if err != nil {
msg.Task.DrandRound-types.TicketRandomnessLookback, // logrus.Errorf("failed to get miner stake: %v", err)
minerAddressMarshalled, // return false
) // }
if err != nil { // actualWinCount := msg.Task.ElectionProof.ComputeWinCount(*mStake, *nStake)
logrus.Errorf("failed to draw ticket electionProofRandomness: %v", err) // if msg.Task.ElectionProof.WinCount != actualWinCount {
return false // logrus.Errorf("locally computed wincount isn't matching received value!", err)
} // return false
// }
err = VerifyVRF(msg.Task.Miner, ticketRandomness, msg.Task.Ticket.VRFProof) // //////////////////////////////////////
if err != nil { //
logrus.Errorf("failed to verify ticket vrf: %v", err) // // === validate payload by specific-chain checks ===
} // if validationFunc := validation.GetValidationMethod(msg.Task.OriginChain, msg.Task.RequestType); validationFunc != nil {
////////////////////////////////////// // err := validationFunc(msg.Task.Payload)
// if err != nil {
// === compute wincount locally and verify values === // logrus.Errorf("payload validation has failed: %v", err)
mStake, nStake, err := cv.miner.GetStakeInfo(common.HexToAddress(msg.Task.MinerEth)) // return false
if err != nil { // }
logrus.Errorf("failed to get miner stake: %v", err) // } else {
return false // logrus.Debugf("Origin chain [%v]/request type[%v] doesn't have any payload validation!", msg.Task.OriginChain, msg.Task.RequestType)
} // }
actualWinCount := msg.Task.ElectionProof.ComputeWinCount(*mStake, *nStake) // /////////////////////////////////
if msg.Task.ElectionProof.WinCount != actualWinCount { //
logrus.Errorf("locally computed wincount isn't matching received value!", err) // return true
return false //},
} //types2.ConsensusMessageTypePrepare: func(msg types2.PrePrepareMessage) bool {
////////////////////////////////////// // err := VerifyTaskSignature(msg.Task)
// if err != nil {
// === validate payload by specific-chain checks === // return false
if validationFunc := validation.GetValidationMethod(msg.Task.OriginChain, msg.Task.RequestType); validationFunc != nil { // }
err := validationFunc(msg.Task.Payload) // return true
if err != nil { //},
logrus.Errorf("payload validation has failed: %v", err) //types2.ConsensusMessageTypeCommit: func(msg types2.PrePrepareMessage) bool {
return false // err := VerifyTaskSignature(msg.Task)
} // if err != nil {
} else { // return false
logrus.Debugf("Origin chain [%v]/request type[%v] doesn't have any payload validation!", msg.Task.OriginChain, msg.Task.RequestType) // }
} // return true
///////////////////////////////// //},
return true
},
types2.MessageTypePrepare: func(msg types2.ConsensusMessage) bool {
err := VerifyTaskSignature(msg.Task)
if err != nil {
return false
}
return true
},
types2.MessageTypeCommit: func(msg types2.ConsensusMessage) bool {
err := VerifyTaskSignature(msg.Task)
if err != nil {
return false
}
return true
},
} }
return cv return cv

View File

@ -2,17 +2,11 @@ package consensus
import ( import (
"context" "context"
"encoding/hex"
"time" "time"
"math/big"
"github.com/Secured-Finance/dione/contracts/dioneDispute" "github.com/Secured-Finance/dione/contracts/dioneDispute"
"github.com/Secured-Finance/dione/contracts/dioneOracle" "github.com/Secured-Finance/dione/contracts/dioneOracle"
"github.com/Secured-Finance/dione/ethclient" "github.com/Secured-Finance/dione/ethclient"
"github.com/ethereum/go-ethereum/common"
"github.com/sirupsen/logrus"
"golang.org/x/crypto/sha3"
) )
type DisputeManager struct { type DisputeManager struct {
@ -69,92 +63,94 @@ func NewDisputeManager(ctx context.Context, ethClient *ethclient.EthereumClient,
} }
func (dm *DisputeManager) onNewSubmission(submittion *dioneOracle.DioneOracleSubmittedOracleRequest) { func (dm *DisputeManager) onNewSubmission(submittion *dioneOracle.DioneOracleSubmittedOracleRequest) {
c := dm.pcm.GetConsensusInfo(submittion.ReqID.String()) //c := dm.pcm.GetConsensusInfo(submittion.ReqID.String())
if c == nil { //if c == nil {
// todo: warn // // todo: warn
return // return
} //}
//
dm.submissionMap[submittion.ReqID.String()] = submittion //dm.submissionMap[submittion.ReqID.String()] = submittion
//
submHashBytes := sha3.Sum256(submittion.Data) //submHashBytes := sha3.Sum256(submittion.Data)
localHashBytes := sha3.Sum256(c.Task.Payload) //localHashBytes := sha3.Sum256(c.Task.Payload)
submHash := hex.EncodeToString(submHashBytes[:]) //submHash := hex.EncodeToString(submHashBytes[:])
localHash := hex.EncodeToString(localHashBytes[:]) //localHash := hex.EncodeToString(localHashBytes[:])
if submHash != localHash { //if submHash != localHash {
logrus.Debugf("submission of request id %s isn't valid - beginning dispute", c.Task.RequestID) // logrus.Debugf("submission of request id %s isn't valid - beginning dispute", c.Task.RequestID)
addr := common.HexToAddress(c.Task.MinerEth) // addr := common.HexToAddress(c.Task.MinerEth)
reqID, ok := big.NewInt(0).SetString(c.Task.RequestID, 10) // reqID, ok := big.NewInt(0).SetString(c.Task.RequestID, 10)
if !ok { // if !ok {
logrus.Errorf("cannot parse request id: %s", c.Task.RequestID) // logrus.Errorf("cannot parse request id: %s", c.Task.RequestID)
return // return
} // }
err := dm.ethClient.BeginDispute(addr, reqID) // err := dm.ethClient.BeginDispute(addr, reqID)
if err != nil { // if err != nil {
logrus.Errorf(err.Error()) // logrus.Errorf(err.Error())
return // return
} // }
disputeFinishTimer := time.NewTimer(dm.voteWindow) // disputeFinishTimer := time.NewTimer(dm.voteWindow)
go func() { // go func() {
for { // for {
select { // select {
case <-dm.ctx.Done(): // case <-dm.ctx.Done():
return // return
case <-disputeFinishTimer.C: // case <-disputeFinishTimer.C:
{ // {
d, ok := dm.disputeMap[reqID.String()] // d, ok := dm.disputeMap[reqID.String()]
if !ok { // if !ok {
logrus.Error("cannot finish dispute: it doesn't exist in manager's dispute map!") // logrus.Error("cannot finish dispute: it doesn't exist in manager's dispute map!")
return // return
} // }
err := dm.ethClient.FinishDispute(d.Dhash) // err := dm.ethClient.FinishDispute(d.Dhash)
if err != nil { // if err != nil {
logrus.Errorf(err.Error()) // logrus.Errorf(err.Error())
return // return
} // }
disputeFinishTimer.Stop() // disputeFinishTimer.Stop()
return // return
} // }
} // }
} // }
}() // }()
} //}
// TODO refactor due to new architecture with blockchain
} }
func (dm *DisputeManager) onNewDispute(dispute *dioneDispute.DioneDisputeNewDispute) { func (dm *DisputeManager) onNewDispute(dispute *dioneDispute.DioneDisputeNewDispute) {
c := dm.pcm.GetConsensusInfo(dispute.RequestID.String()) //c := dm.pcm.GetConsensusInfo(dispute.RequestID.String())
if c == nil { //if c == nil {
// todo: warn // // todo: warn
return // return
} //}
//
subm, ok := dm.submissionMap[dispute.RequestID.String()] //subm, ok := dm.submissionMap[dispute.RequestID.String()]
if !ok { //if !ok {
// todo: warn // // todo: warn
return // return
} //}
//
dm.disputeMap[dispute.RequestID.String()] = dispute //dm.disputeMap[dispute.RequestID.String()] = dispute
//
if dispute.DisputeInitiator.Hex() == dm.ethClient.GetEthAddress().Hex() { //if dispute.DisputeInitiator.Hex() == dm.ethClient.GetEthAddress().Hex() {
return // return
} //}
//
submHashBytes := sha3.Sum256(subm.Data) //submHashBytes := sha3.Sum256(subm.Data)
localHashBytes := sha3.Sum256(c.Task.Payload) //localHashBytes := sha3.Sum256(c.Task.Payload)
submHash := hex.EncodeToString(submHashBytes[:]) //submHash := hex.EncodeToString(submHashBytes[:])
localHash := hex.EncodeToString(localHashBytes[:]) //localHash := hex.EncodeToString(localHashBytes[:])
if submHash == localHash { //if submHash == localHash {
err := dm.ethClient.VoteDispute(dispute.Dhash, false) // err := dm.ethClient.VoteDispute(dispute.Dhash, false)
if err != nil { // if err != nil {
logrus.Errorf(err.Error()) // logrus.Errorf(err.Error())
return // return
} // }
} //}
//
err := dm.ethClient.VoteDispute(dispute.Dhash, true) //err := dm.ethClient.VoteDispute(dispute.Dhash, true)
if err != nil { //if err != nil {
logrus.Errorf(err.Error()) // logrus.Errorf(err.Error())
return // return
} //}
// TODO refactor due to new architecture with blockchain
} }

View File

@ -1,25 +1,23 @@
package consensus package consensus
import ( import (
"context" "errors"
"fmt"
"math/big"
"sync" "sync"
big2 "github.com/filecoin-project/go-state-types/big" "github.com/Secured-Finance/dione/blockchain/pool"
"github.com/Secured-Finance/dione/sigs" "github.com/libp2p/go-libp2p-core/crypto"
"github.com/Secured-Finance/dione/rpc" types2 "github.com/Secured-Finance/dione/blockchain/types"
"github.com/Secured-Finance/dione/beacon" "github.com/Secured-Finance/dione/beacon"
"github.com/Secured-Finance/dione/contracts/dioneOracle"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p-core/peer"
"github.com/Secured-Finance/dione/ethclient" "github.com/Secured-Finance/dione/ethclient"
"github.com/Secured-Finance/dione/types"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
"github.com/filecoin-project/go-state-types/crypto"
"github.com/sirupsen/logrus" "github.com/sirupsen/logrus"
"golang.org/x/xerrors"
) )
type Miner struct { type Miner struct {
@ -28,9 +26,10 @@ type Miner struct {
mutex sync.Mutex mutex sync.Mutex
beacon beacon.BeaconNetworks beacon beacon.BeaconNetworks
ethClient *ethclient.EthereumClient ethClient *ethclient.EthereumClient
minerStake types.BigInt minerStake *big.Int
networkStake types.BigInt networkStake *big.Int
privateKey []byte privateKey crypto.PrivKey
mempool *pool.Mempool
} }
func NewMiner( func NewMiner(
@ -38,7 +37,8 @@ func NewMiner(
ethAddress common.Address, ethAddress common.Address,
beacon beacon.BeaconNetworks, beacon beacon.BeaconNetworks,
ethClient *ethclient.EthereumClient, ethClient *ethclient.EthereumClient,
privateKey []byte, privateKey crypto.PrivKey,
mempool *pool.Mempool,
) *Miner { ) *Miner {
return &Miner{ return &Miner{
address: address, address: address,
@ -46,6 +46,7 @@ func NewMiner(
beacon: beacon, beacon: beacon,
ethClient: ethClient, ethClient: ethClient,
privateKey: privateKey, privateKey: privateKey,
mempool: mempool,
} }
} }
@ -64,13 +65,13 @@ func (m *Miner) UpdateCurrentStakeInfo() error {
return err return err
} }
m.minerStake = *mStake m.minerStake = mStake
m.networkStake = *nStake m.networkStake = nStake
return nil return nil
} }
func (m *Miner) GetStakeInfo(miner common.Address) (*types.BigInt, *types.BigInt, error) { func (m *Miner) GetStakeInfo(miner common.Address) (*big.Int, *big.Int, error) {
mStake, err := m.ethClient.GetMinerStake(miner) mStake, err := m.ethClient.GetMinerStake(miner)
if err != nil { if err != nil {
@ -88,100 +89,59 @@ func (m *Miner) GetStakeInfo(miner common.Address) (*types.BigInt, *types.BigInt
return mStake, nStake, nil return mStake, nStake, nil
} }
func (m *Miner) MineTask(ctx context.Context, event *dioneOracle.DioneOracleNewOracleRequest) (*types.DioneTask, error) { func (m *Miner) MineBlock(randomness []byte, drandRound uint64, lastBlockHeader *types2.BlockHeader) (*types2.Block, error) {
beaconValues, err := beacon.BeaconEntriesForTask(ctx, m.beacon) logrus.Debug("attempting to mine the block at epoch: ", drandRound)
if err != nil {
return nil, xerrors.Errorf("failed to get beacon entries: %w", err)
}
logrus.Debug("attempting to mine the task at epoch: ", beaconValues[1].Round)
randomBase := beaconValues[1]
if err := m.UpdateCurrentStakeInfo(); err != nil { if err := m.UpdateCurrentStakeInfo(); err != nil {
return nil, xerrors.Errorf("failed to update miner stake: %w", err) return nil, fmt.Errorf("failed to update miner stake: %w", err)
}
ticket, err := m.computeTicket(&randomBase)
if err != nil {
return nil, xerrors.Errorf("scratching ticket failed: %w", err)
} }
winner, err := IsRoundWinner( winner, err := IsRoundWinner(
types.DrandRound(randomBase.Round), drandRound,
m.address, m.address,
randomBase, randomness,
m.minerStake, m.minerStake,
m.networkStake, m.networkStake,
func(id peer.ID, bytes []byte) (*types.Signature, error) { m.privateKey,
return sigs.Sign(types.SigTypeEd25519, m.privateKey, bytes)
},
) )
if err != nil { if err != nil {
return nil, xerrors.Errorf("failed to check if we win next round: %w", err) return nil, fmt.Errorf("failed to check if we winned in next round: %w", err)
} }
if winner == nil { if winner == nil {
return nil, nil return nil, nil
} }
rpcMethod := rpc.GetRPCMethod(event.OriginChain, event.RequestType) // TODO get rpc responses for oracle requests
if rpcMethod == nil { //rpcMethod := rpc.GetRPCMethod(event.OriginChain, event.RequestType)
return nil, xerrors.Errorf("invalid rpc method name/type") //if rpcMethod == nil {
} // return nil, xerrors.Errorf("invalid rpc method name/type")
res, err := rpcMethod(event.RequestParams) //}
if err != nil { //res, err := rpcMethod(event.RequestParams)
return nil, xerrors.Errorf("couldn't do rpc request: %w", err) //if err != nil {
// return nil, xerrors.Errorf("couldn't do rpc request: %w", err)
//}
txs := m.mempool.GetAllTransactions()
if txs == nil {
return nil, fmt.Errorf("there is no txes for processing") // skip new consensus round because there is no transaction for processing
} }
return &types.DioneTask{ newBlock, err := types2.CreateBlock(lastBlockHeader, txs, m.ethAddress, m.privateKey)
OriginChain: event.OriginChain, if err != nil {
RequestType: event.RequestType, return nil, fmt.Errorf("failed to create new block: %w", err)
RequestParams: event.RequestParams, }
RequestID: event.ReqID.String(),
ConsensusID: event.ReqID.String(), return newBlock, nil
Miner: m.address,
MinerEth: m.ethAddress.Hex(),
Ticket: ticket,
ElectionProof: winner,
BeaconEntries: beaconValues,
Payload: res,
DrandRound: types.DrandRound(randomBase.Round),
}, nil
} }
func (m *Miner) computeTicket(brand *types.BeaconEntry) (*types.Ticket, error) { func (m *Miner) IsMinerEligibleToProposeBlock(ethAddress common.Address) error {
buf, err := m.address.MarshalBinary()
if err != nil {
return nil, xerrors.Errorf("failed to marshal address: %w", err)
}
round := types.DrandRound(brand.Round)
input, err := DrawRandomness(brand.Data, crypto.DomainSeparationTag_TicketProduction, round-types.TicketRandomnessLookback, buf)
if err != nil {
return nil, err
}
vrfOut, err := ComputeVRF(func(id peer.ID, bytes []byte) (*types.Signature, error) {
return sigs.Sign(types.SigTypeEd25519, m.privateKey, bytes)
}, m.address, input)
if err != nil {
return nil, err
}
return &types.Ticket{
VRFProof: vrfOut,
}, nil
}
func (m *Miner) IsMinerEligibleToProposeTask(ethAddress common.Address) error {
mStake, err := m.ethClient.GetMinerStake(ethAddress) mStake, err := m.ethClient.GetMinerStake(ethAddress)
if err != nil { if err != nil {
return err return err
} }
ok := mStake.GreaterThanEqual(big2.NewInt(ethclient.MinMinerStake)) if mStake.Cmp(big.NewInt(ethclient.MinMinerStake)) == -1 {
if !ok { return errors.New("miner doesn't have enough staked tokens")
return xerrors.Errorf("miner doesn't have enough staked tokens")
} }
return nil return nil
} }

View File

@ -1,18 +1,19 @@
package consensus package consensus
import ( import (
"bytes"
types2 "github.com/Secured-Finance/dione/consensus/types" types2 "github.com/Secured-Finance/dione/consensus/types"
mapset "github.com/Secured-Finance/golang-set" mapset "github.com/Secured-Finance/golang-set"
) )
type MessageLog struct { type ConsensusMessageLog struct {
messages mapset.Set messages mapset.Set
maxLogSize int maxLogSize int
validationFuncMap map[types2.MessageType]func(message types2.ConsensusMessage)
} }
func NewMessageLog() *MessageLog { func NewConsensusMessageLog() *ConsensusMessageLog {
msgLog := &MessageLog{ msgLog := &ConsensusMessageLog{
messages: mapset.NewSet(), messages: mapset.NewSet(),
maxLogSize: 0, // TODO maxLogSize: 0, // TODO
} }
@ -20,21 +21,32 @@ func NewMessageLog() *MessageLog {
return msgLog return msgLog
} }
func (ml *MessageLog) AddMessage(msg types2.ConsensusMessage) { func (ml *ConsensusMessageLog) AddMessage(msg types2.ConsensusMessage) {
ml.messages.Add(msg) ml.messages.Add(msg)
} }
func (ml *MessageLog) Exists(msg types2.ConsensusMessage) bool { func (ml *ConsensusMessageLog) Exists(msg types2.ConsensusMessage) bool {
return ml.messages.Contains(msg) return ml.messages.Contains(msg)
} }
func (ml *MessageLog) Get(typ types2.MessageType, consensusID string) []*types2.ConsensusMessage { func (ml *ConsensusMessageLog) Get(typ types2.ConsensusMessageType, blockhash []byte) []*types2.ConsensusMessage {
var result []*types2.ConsensusMessage var result []*types2.ConsensusMessage
for v := range ml.messages.Iter() { for v := range ml.messages.Iter() {
msg := v.(types2.ConsensusMessage) msg := v.(types2.ConsensusMessage)
if msg.Type == typ && msg.Task.ConsensusID == consensusID { if msg.Block != nil {
result = append(result, &msg)
}
if msg.Type == typ {
var msgBlockHash []byte
if msg.Block != nil {
msgBlockHash = msg.Block.Header.Hash
} else {
msgBlockHash = msg.Blockhash
}
if bytes.Compare(msgBlockHash, blockhash) == 0 {
result = append(result, &msg)
}
} }
} }

View File

@ -0,0 +1,25 @@
package types
import (
types3 "github.com/Secured-Finance/dione/blockchain/types"
"github.com/libp2p/go-libp2p-core/peer"
)
type ConsensusMessageType uint8
const (
ConsensusMessageTypeUnknown = ConsensusMessageType(iota)
ConsensusMessageTypePrePrepare
ConsensusMessageTypePrepare
ConsensusMessageTypeCommit
)
// ConsensusMessage is common struct for various consensus message types. It is stored in consensus message log.
type ConsensusMessage struct {
Type ConsensusMessageType
Blockhash []byte
Signature []byte
Block *types3.Block // it is optional, because not all message types have block included
From peer.ID
}

View File

@ -1,22 +1,16 @@
package types package types
import ( import (
"github.com/Secured-Finance/dione/types" types2 "github.com/Secured-Finance/dione/blockchain/types"
"github.com/libp2p/go-libp2p-core/peer"
) )
type MessageType uint8 type PrePrepareMessage struct {
Block *types2.Block
const (
MessageTypeUnknown = MessageType(iota)
MessageTypePrePrepare
MessageTypePrepare
MessageTypeCommit
)
type ConsensusMessage struct {
Task types.DioneTask
From peer.ID
Type MessageType
} }
type PrepareMessage struct {
Blockhash []byte
Signature []byte
}
type CommitMessage PrepareMessage

View File

@ -3,8 +3,7 @@ package consensus
import ( import (
"encoding/binary" "encoding/binary"
"fmt" "fmt"
"math/big"
"github.com/fxamacker/cbor/v2"
"github.com/Secured-Finance/dione/pubsub" "github.com/Secured-Finance/dione/pubsub"
@ -18,23 +17,15 @@ import (
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p-core/peer"
"github.com/Secured-Finance/dione/types" "github.com/Secured-Finance/dione/types"
"github.com/filecoin-project/go-state-types/crypto" crypto2 "github.com/filecoin-project/go-state-types/crypto"
"github.com/libp2p/go-libp2p-core/crypto"
"golang.org/x/xerrors" "golang.org/x/xerrors"
) )
type SignFunc func(peer.ID, []byte) (*types.Signature, error) type SignFunc func(peer.ID, []byte) (*types.Signature, error)
func ComputeVRF(sign SignFunc, worker peer.ID, sigInput []byte) ([]byte, error) { func ComputeVRF(privKey crypto.PrivKey, sigInput []byte) ([]byte, error) {
sig, err := sign(worker, sigInput) return privKey.Sign(sigInput)
if err != nil {
return nil, err
}
if sig.Type != types.SigTypeEd25519 {
return nil, fmt.Errorf("miner worker address was not a Ed25519 key")
}
return sig.Data, nil
} }
func VerifyVRF(worker peer.ID, vrfBase, vrfproof []byte) error { func VerifyVRF(worker peer.ID, vrfBase, vrfproof []byte) error {
@ -46,20 +37,20 @@ func VerifyVRF(worker peer.ID, vrfBase, vrfproof []byte) error {
return nil return nil
} }
func IsRoundWinner(round types.DrandRound, func IsRoundWinner(round uint64,
worker peer.ID, brand types.BeaconEntry, minerStake, networkStake types.BigInt, sign SignFunc) (*types.ElectionProof, error) { worker peer.ID, randomness []byte, minerStake, networkStake *big.Int, privKey crypto.PrivKey) (*types.ElectionProof, error) {
buf, err := worker.MarshalBinary() buf, err := worker.MarshalBinary()
if err != nil { if err != nil {
return nil, xerrors.Errorf("failed to marshal address: %w", err) return nil, xerrors.Errorf("failed to marshal address: %w", err)
} }
electionRand, err := DrawRandomness(brand.Data, crypto.DomainSeparationTag_ElectionProofProduction, round, buf) electionRand, err := DrawRandomness(randomness, crypto2.DomainSeparationTag_ElectionProofProduction, round, buf)
if err != nil { if err != nil {
return nil, xerrors.Errorf("failed to draw randomness: %w", err) return nil, xerrors.Errorf("failed to draw randomness: %w", err)
} }
vrfout, err := ComputeVRF(sign, worker, electionRand) vrfout, err := ComputeVRF(privKey, electionRand)
if err != nil { if err != nil {
return nil, xerrors.Errorf("failed to compute VRF: %w", err) return nil, xerrors.Errorf("failed to compute VRF: %w", err)
} }
@ -74,7 +65,7 @@ func IsRoundWinner(round types.DrandRound,
return ep, nil return ep, nil
} }
func DrawRandomness(rbase []byte, pers crypto.DomainSeparationTag, round types.DrandRound, entropy []byte) ([]byte, error) { func DrawRandomness(rbase []byte, pers crypto2.DomainSeparationTag, round uint64, entropy []byte) ([]byte, error) {
h := blake2b.New256() h := blake2b.New256()
if err := binary.Write(h, binary.BigEndian, int64(pers)); err != nil { if err := binary.Write(h, binary.BigEndian, int64(pers)); err != nil {
return nil, xerrors.Errorf("deriving randomness: %v", err) return nil, xerrors.Errorf("deriving randomness: %v", err)
@ -111,31 +102,46 @@ func VerifyTaskSignature(task types.DioneTask) error {
return nil return nil
} }
func NewMessage(msg *pubsub.GenericMessage, typ pubsub.PubSubMessageType) (pubsub.GenericMessage, error) { func NewMessage(cmsg types2.ConsensusMessage, typ types2.ConsensusMessageType, privKey crypto.PrivKey) (*pubsub.GenericMessage, error) {
var newMsg pubsub.GenericMessage
newMsg.Type = typ
newCMsg := msg.Payload
newMsg.Payload = newCMsg
return newMsg, nil
}
func CreatePrePrepareWithTaskSignature(task *types.DioneTask, privateKey []byte) (*pubsub.GenericMessage, error) {
var message pubsub.GenericMessage var message pubsub.GenericMessage
message.Type = pubsub.PrePrepareMessageType switch typ {
case types2.ConsensusMessageTypePrePrepare:
{
message.Type = pubsub.PrePrepareMessageType
message.Payload = types2.PrePrepareMessage{
Block: cmsg.Block,
}
break
}
case types2.ConsensusMessageTypePrepare:
{
message.Type = pubsub.PrepareMessageType
pm := types2.PrepareMessage{
Blockhash: cmsg.Blockhash,
}
signature, err := privKey.Sign(cmsg.Blockhash)
if err != nil {
return nil, fmt.Errorf("failed to create signature: %v", err)
}
pm.Signature = signature
message.Payload = pm
break
}
case types2.ConsensusMessageTypeCommit:
{
message.Type = pubsub.CommitMessageType
pm := types2.CommitMessage{
Blockhash: cmsg.Blockhash,
}
signature, err := privKey.Sign(cmsg.Blockhash)
if err != nil {
return nil, fmt.Errorf("failed to create signature: %v", err)
}
pm.Signature = signature
message.Payload = pm
break
}
}
cHash, err := hashstructure.Hash(task, hashstructure.FormatV2, nil)
if err != nil {
return nil, err
}
signature, err := sigs.Sign(types.SigTypeEd25519, privateKey, []byte(fmt.Sprintf("%v", cHash)))
if err != nil {
return nil, err
}
task.Signature = signature.Data
data, err := cbor.Marshal(types2.ConsensusMessage{Task: *task})
if err != nil {
return nil, err
}
message.Payload = data
return &message, nil return &message, nil
} }

View File

@ -1,7 +1,8 @@
package ethclient package ethclient
import ( import (
"github.com/Secured-Finance/dione/types" "math/big"
"github.com/ethereum/go-ethereum/common" "github.com/ethereum/go-ethereum/common"
) )
@ -9,32 +10,22 @@ const (
MinMinerStake = 1000 MinMinerStake = 1000
) )
// Getting total stake in DioneStaking contract, this function could // GetTotalStake for getting total stake in DioneStaking contract
// be used for storing the total stake and veryfing the stake tokens func (c *EthereumClient) GetTotalStake() (*big.Int, error) {
// on new tasks
func (c *EthereumClient) GetTotalStake() (*types.BigInt, error) {
var b types.BigInt
totalStake, err := c.dioneStaking.TotalStake() totalStake, err := c.dioneStaking.TotalStake()
if err != nil { if err != nil {
return nil, err return nil, err
} }
b.Int = totalStake return totalStake, nil
return &b, nil
} }
// Getting miner stake in DioneStaking contract, this function could // GetMinerStake for getting specified miner stake in DioneStaking contract
// be used for storing the miner's stake and veryfing the stake tokens func (c *EthereumClient) GetMinerStake(minerAddress common.Address) (*big.Int, error) {
// on new tasks
func (c *EthereumClient) GetMinerStake(minerAddress common.Address) (*types.BigInt, error) {
var b types.BigInt
minerStake, err := c.dioneStaking.MinerStake(minerAddress) minerStake, err := c.dioneStaking.MinerStake(minerAddress)
if err != nil { if err != nil {
return nil, err return nil, err
} }
b.Int = minerStake return minerStake, nil
return &b, nil
} }

1
go.mod
View File

@ -3,6 +3,7 @@ module github.com/Secured-Finance/dione
go 1.14 go 1.14
require ( require (
github.com/Arceliar/phony v0.0.0-20210209235338-dde1a8dca979 // indirect
github.com/Secured-Finance/go-libp2p-pex v1.1.0 github.com/Secured-Finance/go-libp2p-pex v1.1.0
github.com/Secured-Finance/golang-set v1.8.0 github.com/Secured-Finance/golang-set v1.8.0
github.com/aristanetworks/goarista v0.0.0-20210308203447-b196d8410f1d // indirect github.com/aristanetworks/goarista v0.0.0-20210308203447-b196d8410f1d // indirect

2
go.sum
View File

@ -28,6 +28,8 @@ git.apache.org/thrift.git v0.0.0-20180902110319-2566ecd5d999/go.mod h1:fPE2ZNJGy
github.com/AndreasBriese/bbloom v0.0.0-20180913140656-343706a395b7/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= github.com/AndreasBriese/bbloom v0.0.0-20180913140656-343706a395b7/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8=
github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9 h1:HD8gA2tkByhMAwYaFAX9w2l7vxvBQ5NMoxDrkhqhtn4= github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9 h1:HD8gA2tkByhMAwYaFAX9w2l7vxvBQ5NMoxDrkhqhtn4=
github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8=
github.com/Arceliar/phony v0.0.0-20210209235338-dde1a8dca979 h1:WndgpSW13S32VLQ3ugUxx2EnnWmgba1kCqPkd4Gk1yQ=
github.com/Arceliar/phony v0.0.0-20210209235338-dde1a8dca979/go.mod h1:6Lkn+/zJilRMsKmbmG1RPoamiArC6HS73xbwRyp3UyI=
github.com/Azure/azure-pipeline-go v0.2.1/go.mod h1:UGSo8XybXnIGZ3epmeBw7Jdz+HiUVpqIlpz/HKHylF4= github.com/Azure/azure-pipeline-go v0.2.1/go.mod h1:UGSo8XybXnIGZ3epmeBw7Jdz+HiUVpqIlpz/HKHylF4=
github.com/Azure/azure-pipeline-go v0.2.2/go.mod h1:4rQ/NZncSvGqNkkOsNpOU1tgoNuIlp9AfUH5G1tvCHc= github.com/Azure/azure-pipeline-go v0.2.2/go.mod h1:4rQ/NZncSvGqNkkOsNpOU1tgoNuIlp9AfUH5G1tvCHc=
github.com/Azure/azure-storage-blob-go v0.7.0/go.mod h1:f9YQKtsG1nMisotuTPpO0tjNuEjKRYAcJU8/ydDI++4= github.com/Azure/azure-storage-blob-go v0.7.0/go.mod h1:f9YQKtsG1nMisotuTPpO0tjNuEjKRYAcJU8/ydDI++4=

View File

@ -13,15 +13,12 @@ import (
types2 "github.com/Secured-Finance/dione/blockchain/types" types2 "github.com/Secured-Finance/dione/blockchain/types"
"github.com/fxamacker/cbor/v2"
gorpc "github.com/libp2p/go-libp2p-gorpc" gorpc "github.com/libp2p/go-libp2p-gorpc"
"github.com/Secured-Finance/dione/blockchain/pool" "github.com/Secured-Finance/dione/blockchain/pool"
"github.com/Secured-Finance/dione/blockchain/sync" "github.com/Secured-Finance/dione/blockchain/sync"
"github.com/Secured-Finance/dione/cache"
"github.com/Secured-Finance/dione/consensus" "github.com/Secured-Finance/dione/consensus"
pubsub2 "github.com/Secured-Finance/dione/pubsub" pubsub2 "github.com/Secured-Finance/dione/pubsub"
@ -34,8 +31,6 @@ import (
"github.com/Secured-Finance/dione/rpc/filecoin" "github.com/Secured-Finance/dione/rpc/filecoin"
"github.com/Secured-Finance/dione/wallet"
"golang.org/x/xerrors" "golang.org/x/xerrors"
"github.com/Secured-Finance/dione/beacon" "github.com/Secured-Finance/dione/beacon"
@ -62,14 +57,14 @@ type Node struct {
ConsensusManager *consensus.PBFTConsensusManager ConsensusManager *consensus.PBFTConsensusManager
Miner *consensus.Miner Miner *consensus.Miner
Beacon beacon.BeaconNetworks Beacon beacon.BeaconNetworks
Wallet *wallet.LocalWallet
Cache cache.Cache
DisputeManager *consensus.DisputeManager DisputeManager *consensus.DisputeManager
BlockPool *blockchain.BlockChain BlockPool *blockchain.BlockChain
MemPool *pool.Mempool MemPool *pool.Mempool
SyncManager sync.SyncManager SyncManager sync.SyncManager
NetworkService *NetworkService NetworkService *NetworkService
NetworkRPCHost *gorpc.Server NetworkRPCHost *gorpc.Server
//Cache cache.Cache
//Wallet *wallet.LocalWallet
} }
func NewNode(config *config.Config, prvKey crypto.PrivKey, pexDiscoveryUpdateTime time.Duration) (*Node, error) { func NewNode(config *config.Config, prvKey crypto.PrivKey, pexDiscoveryUpdateTime time.Duration) (*Node, error) {
@ -119,24 +114,10 @@ func NewNode(config *config.Config, prvKey crypto.PrivKey, pexDiscoveryUpdateTim
n.PeerDiscovery = peerDiscovery n.PeerDiscovery = peerDiscovery
logrus.Info("Peer discovery subsystem has been initialized!") logrus.Info("Peer discovery subsystem has been initialized!")
// get private key of libp2p host
rawPrivKey, err := prvKey.Raw()
if err != nil {
logrus.Fatal(err)
}
// initialize random beacon network subsystem
randomBeaconNetwork, err := provideBeacon(psb.Pubsub)
if err != nil {
logrus.Fatal(err)
}
n.Beacon = randomBeaconNetwork
logrus.Info("Random beacon subsystem has been initialized!")
// initialize event log cache subsystem // initialize event log cache subsystem
c := provideCache(config) //c := provideCache(config)
n.Cache = c //n.Cache = c
logrus.Info("Event cache subsystem has initialized!") //logrus.Info("Event cache subsystem has initialized!")
// == initialize blockchain modules // == initialize blockchain modules
@ -177,17 +158,25 @@ func NewNode(config *config.Config, prvKey crypto.PrivKey, pexDiscoveryUpdateTim
logrus.Info("Blockchain synchronization subsystem has been successfully initialized!") logrus.Info("Blockchain synchronization subsystem has been successfully initialized!")
// initialize mining subsystem // initialize mining subsystem
miner := provideMiner(n.Host.ID(), *n.Ethereum.GetEthAddress(), n.Beacon, n.Ethereum, rawPrivKey) miner := provideMiner(n.Host.ID(), *n.Ethereum.GetEthAddress(), n.Beacon, n.Ethereum, prvKey, mp)
n.Miner = miner n.Miner = miner
logrus.Info("Mining subsystem has initialized!") logrus.Info("Mining subsystem has initialized!")
// initialize consensus subsystem // initialize consensus subsystem
cManager := provideConsensusManager(psb, miner, ethClient, rawPrivKey, n.Config.ConsensusMinApprovals, c) consensusManager := provideConsensusManager(psb, miner, ethClient, prvKey, n.Config.ConsensusMinApprovals)
n.ConsensusManager = cManager n.ConsensusManager = consensusManager
logrus.Info("Consensus subsystem has initialized!") logrus.Info("Consensus subsystem has initialized!")
// initialize random beacon network subsystem
randomBeaconNetwork, err := provideBeacon(psb.Pubsub, consensusManager)
if err != nil {
logrus.Fatal(err)
}
n.Beacon = randomBeaconNetwork
logrus.Info("Random beacon subsystem has been initialized!")
// initialize dispute subsystem // initialize dispute subsystem
disputeManager, err := provideDisputeManager(context.TODO(), ethClient, cManager, config) disputeManager, err := provideDisputeManager(context.TODO(), ethClient, consensusManager, config)
if err != nil { if err != nil {
logrus.Fatal(err) logrus.Fatal(err)
} }
@ -195,11 +184,11 @@ func NewNode(config *config.Config, prvKey crypto.PrivKey, pexDiscoveryUpdateTim
logrus.Info("Dispute subsystem has initialized!") logrus.Info("Dispute subsystem has initialized!")
// initialize internal eth wallet // initialize internal eth wallet
w, err := provideWallet(n.Host.ID(), rawPrivKey) //w, err := provideWallet(n.Host.ID(), rawPrivKey)
if err != nil { //if err != nil {
logrus.Fatal(err) // logrus.Fatal(err)
} //}
n.Wallet = w //n.Wallet = w
return n, nil return n, nil
} }
@ -272,35 +261,18 @@ func (n *Node) subscribeOnEthContractsAsync(ctx context.Context) {
EventLoop: EventLoop:
for { for {
select { select {
case event := <-eventChan: case <-eventChan:
{ {
logrus.Info("Let's wait a little so that all nodes have time to receive the request") logrus.Info("Let's wait a little so that all nodes have time to receive the request")
time.Sleep(5 * time.Second) time.Sleep(5 * time.Second)
task, err := n.Miner.MineTask(context.TODO(), event) // TODO make the rpc request and save response as tx payload
if err != nil { tx := types2.CreateTransaction([]byte{})
logrus.Errorf("Failed to mine task: %v", err)
}
if task == nil {
logrus.Warnf("Task is nil!")
continue
}
payload, err := cbor.Marshal(task)
if err != nil {
logrus.Errorf("Failed to marshal request event")
continue
}
tx := types2.CreateTransaction(payload)
err = n.MemPool.StoreTx(tx) err = n.MemPool.StoreTx(tx)
if err != nil { if err != nil {
logrus.Errorf("Failed to store tx in mempool: %s", err.Error()) logrus.Errorf("Failed to store tx in mempool: %s", err.Error())
continue continue
} }
//logrus.Infof("Proposed new Dione task with ID: %s", event.ReqID.String())
//err = n.ConsensusManager.Propose(*task)
//if err != nil {
// logrus.Errorf("Failed to propose task: %v", err)
//}
} }
case <-ctx.Done(): case <-ctx.Done():
break EventLoop break EventLoop

View File

@ -58,17 +58,17 @@ func provideDisputeManager(ctx context.Context, ethClient *ethclient.EthereumCli
return consensus.NewDisputeManager(ctx, ethClient, pcm, cfg.Ethereum.DisputeVoteWindow) return consensus.NewDisputeManager(ctx, ethClient, pcm, cfg.Ethereum.DisputeVoteWindow)
} }
func provideMiner(peerID peer.ID, ethAddress common.Address, beacon beacon.BeaconNetworks, ethClient *ethclient.EthereumClient, privateKey []byte) *consensus.Miner { func provideMiner(peerID peer.ID, ethAddress common.Address, beacon beacon.BeaconNetworks, ethClient *ethclient.EthereumClient, privateKey crypto.PrivKey, mempool *pool.Mempool) *consensus.Miner {
return consensus.NewMiner(peerID, ethAddress, beacon, ethClient, privateKey) return consensus.NewMiner(peerID, ethAddress, beacon, ethClient, privateKey, mempool)
} }
func provideBeacon(ps *pubsub2.PubSub) (beacon.BeaconNetworks, error) { func provideBeacon(ps *pubsub2.PubSub, pcm *consensus.PBFTConsensusManager) (beacon.BeaconNetworks, error) {
networks := beacon.BeaconNetworks{} networks := beacon.BeaconNetworks{}
bc, err := drand2.NewDrandBeacon(ps) bc, err := drand2.NewDrandBeacon(ps, pcm)
if err != nil { if err != nil {
return nil, fmt.Errorf("failed to setup drand beacon: %w", err) return nil, fmt.Errorf("failed to setup drand beacon: %w", err)
} }
networks = append(networks, beacon.BeaconNetwork{Start: types.DrandRound(config.ChainGenesis), Beacon: bc}) networks = append(networks, beacon.BeaconNetwork{Start: config.DrandChainGenesisTime, Beacon: bc})
// NOTE: currently we use only one network // NOTE: currently we use only one network
return networks, nil return networks, nil
} }
@ -103,8 +103,8 @@ func providePubsubRouter(lhost host.Host, config *config.Config) *pubsub.PubSubR
return pubsub.NewPubSubRouter(lhost, config.PubSub.ServiceTopicName, config.IsBootstrap) return pubsub.NewPubSubRouter(lhost, config.PubSub.ServiceTopicName, config.IsBootstrap)
} }
func provideConsensusManager(psb *pubsub.PubSubRouter, miner *consensus.Miner, ethClient *ethclient.EthereumClient, privateKey []byte, minApprovals int, evc cache.Cache) *consensus.PBFTConsensusManager { func provideConsensusManager(psb *pubsub.PubSubRouter, miner *consensus.Miner, ethClient *ethclient.EthereumClient, privateKey crypto.PrivKey, minApprovals int) *consensus.PBFTConsensusManager {
return consensus.NewPBFTConsensusManager(psb, minApprovals, privateKey, ethClient, miner, evc) return consensus.NewPBFTConsensusManager(psb, minApprovals, privateKey, ethClient, miner)
} }
func provideLibp2pHost(config *config.Config, privateKey crypto.PrivKey) (host.Host, error) { func provideLibp2pHost(config *config.Config, privateKey crypto.PrivKey) (host.Host, error) {

View File

@ -1,96 +0,0 @@
package store
import (
"time"
"github.com/Secured-Finance/dione/ethclient"
"github.com/Secured-Finance/dione/lib"
"github.com/Secured-Finance/dione/types"
"github.com/ethereum/go-ethereum/common"
validation "github.com/go-ozzo/ozzo-validation"
)
type DioneStakeInfo struct {
ID int
MinerStake *types.BigInt
TotalStake *types.BigInt
MinerAddress string
MinerEthWallet string
Timestamp time.Time
Ethereum *ethclient.EthereumClient
}
func NewDioneStakeInfo(minerStake, totalStake *types.BigInt, minerWallet, minerEthWallet string, ethereumClient *ethclient.EthereumClient) *DioneStakeInfo {
return &DioneStakeInfo{
MinerStake: minerStake,
TotalStake: totalStake,
MinerAddress: minerWallet,
MinerEthWallet: minerEthWallet,
Ethereum: ethereumClient,
}
}
func (d *DioneStakeInfo) UpdateMinerStake(minerEthAddress common.Address) error {
minerStake, err := d.Ethereum.GetMinerStake(minerEthAddress)
if err != nil {
return err
}
d.MinerStake = minerStake
return nil
}
func (d *DioneStakeInfo) UpdateTotalStake() error {
totalStake, err := d.Ethereum.GetTotalStake()
if err != nil {
return err
}
d.TotalStake = totalStake
return nil
}
// Put miner's staking information into the database
func (s *Store) CreateDioneStakeInfo(stakeStore *DioneStakeInfo) error {
if err := stakeStore.Validate(); err != nil {
return err
}
now := lib.Clock.Now()
return s.db.QueryRow(
"INSERT INTO staking (miner_stake, total_stake, miner_address, miner_eth_wallet, timestamp) VALUES ($1, $2, $3, $4, $5) RETURNING id",
stakeStore.MinerStake,
stakeStore.TotalStake,
stakeStore.MinerAddress,
stakeStore.MinerEthWallet,
now,
).Scan(&stakeStore.ID)
}
func (s *Store) GetLastStakeInfo(wallet, ethWallet string) (*DioneStakeInfo, error) {
var stake *DioneStakeInfo
if err := s.db.Select(&stake,
`SELECT miner_stake, total_stake, miner_address, miner_eth_wallet, timestamp FROM staking ORDER BY TIMESTAMP DESC LIMIT 1 WHERE miner_address=$1, miner_eth_wallet=$2`,
wallet,
ethWallet,
); err != nil {
return nil, err
}
return stake, nil
}
// Before puting the data into the database validating all required fields
func (s *DioneStakeInfo) Validate() error {
return validation.ValidateStruct(
s,
validation.Field(&s.MinerStake, validation.Required, validation.By(types.ValidateBigInt(s.MinerStake.Int))),
validation.Field(&s.TotalStake, validation.Required, validation.By(types.ValidateBigInt(s.TotalStake.Int))),
validation.Field(&s.MinerAddress, validation.Required),
validation.Field(&s.MinerEthWallet, validation.Required),
)
}

View File

@ -1,46 +1,44 @@
package store package store
import ( //import (
"github.com/Secured-Finance/dione/node" // "github.com/Secured-Finance/dione/node"
"github.com/jmoiron/sqlx" // "github.com/jmoiron/sqlx"
_ "github.com/mattn/go-sqlite3" // _ "github.com/mattn/go-sqlite3"
) //)
//
type Store struct { //type Store struct {
db *sqlx.DB // db *sqlx.DB
node *node.Node // node *node.Node
genesisTs uint64 // genesisTs uint64
StakeStorage DioneStakeInfo //}
// genesisTask *types.DioneTask //
} //func NewStore(node *node.Node, genesisTs uint64) (*Store, error) {
// db, err := newDB(node.Config.Store.DatabaseURL)
func NewStore(node *node.Node, genesisTs uint64) (*Store, error) { // if err != nil {
db, err := newDB(node.Config.Store.DatabaseURL) // return nil, err
if err != nil { // }
return nil, err //
} // defer db.Close()
//
defer db.Close() // return &Store{
// db: db,
return &Store{ // node: node,
db: db, // genesisTs: genesisTs,
node: node, // }, nil
genesisTs: genesisTs, //}
}, nil //
} //func newDB(databaseURL string) (*sqlx.DB, error) {
// db, err := sqlx.Connect("sqlite3", databaseURL)
func newDB(databaseURL string) (*sqlx.DB, error) { // if err != nil {
db, err := sqlx.Connect("sqlite3", databaseURL) // return nil, err
if err != nil { // }
return nil, err //
} // if err := db.Ping(); err != nil {
// return nil, err
if err := db.Ping(); err != nil { // }
return nil, err //
} // return db, nil
//}
return db, nil //
}
// TODO: Discuss with ChronosX88 about using custom database to decrease I/O bound // TODO: Discuss with ChronosX88 about using custom database to decrease I/O bound
// specify the migrations for stake storage; // specify the migrations for stake storage;

View File

@ -1,32 +0,0 @@
package types
import (
"errors"
"math/big"
big2 "github.com/filecoin-project/go-state-types/big"
validation "github.com/go-ozzo/ozzo-validation"
)
var EmptyInt = BigInt{}
type BigInt = big2.Int
func NewInt(i uint64) BigInt {
return BigInt{Int: big.NewInt(0).SetUint64(i)}
}
func BigFromBytes(b []byte) BigInt {
i := big.NewInt(0).SetBytes(b)
return BigInt{Int: i}
}
func ValidateBigInt(i *big.Int) validation.RuleFunc {
return func(value interface{}) error {
bigInt := i.IsInt64()
if !bigInt {
return errors.New("expected big integer")
}
return nil
}
}

View File

@ -99,13 +99,13 @@ func polyval(p []*big.Int, x *big.Int) *big.Int {
// computes lambda in Q.256 // computes lambda in Q.256
func lambda(power, totalPower *big.Int) *big.Int { func lambda(power, totalPower *big.Int) *big.Int {
lam := new(big.Int).Mul(power, tasksPerEpoch.Int) // Q.0 lam := new(big.Int).Mul(power, config.ExpectedLeadersPerEpoch) // Q.0
lam = lam.Lsh(lam, precision) // Q.256 lam = lam.Lsh(lam, precision) // Q.256
lam = lam.Div(lam /* Q.256 */, totalPower /* Q.0 */) // Q.256 lam = lam.Div(lam /* Q.256 */, totalPower /* Q.0 */) // Q.256
return lam return lam
} }
var MaxWinCount = 3 * int64(config.TasksPerEpoch) var MaxWinCount = 3 * config.ExpectedLeadersPerEpoch.Int64()
type poiss struct { type poiss struct {
lam *big.Int lam *big.Int
@ -175,10 +175,10 @@ func (p *poiss) next() *big.Int {
// ComputeWinCount uses VRFProof to compute number of wins // ComputeWinCount uses VRFProof to compute number of wins
// The algorithm is based on Algorand's Sortition with Binomial distribution // The algorithm is based on Algorand's Sortition with Binomial distribution
// replaced by Poisson distribution. // replaced by Poisson distribution.
func (ep *ElectionProof) ComputeWinCount(power BigInt, totalPower BigInt) int64 { func (ep *ElectionProof) ComputeWinCount(power *big.Int, totalPower *big.Int) int64 {
h := blake2b.Sum256(ep.VRFProof) h := blake2b.Sum256(ep.VRFProof)
lhs := BigFromBytes(h[:]).Int // 256bits, assume Q.256 so [0, 1) lhs := big.NewInt(0).SetBytes(h[:]) // 256bits, assume Q.256 so [0, 1)
// We are calculating upside-down CDF of Poisson distribution with // We are calculating upside-down CDF of Poisson distribution with
// rate λ=power*E/totalPower // rate λ=power*E/totalPower
@ -191,7 +191,7 @@ func (ep *ElectionProof) ComputeWinCount(power BigInt, totalPower BigInt) int64
// rhs = 1 - pmf // rhs = 1 - pmf
// for h(vrf) < rhs: j++; pmf = pmf * lam / j; rhs = rhs - pmf // for h(vrf) < rhs: j++; pmf = pmf * lam / j; rhs = rhs - pmf
lam := lambda(power.Int, totalPower.Int) // Q.256 lam := lambda(power, totalPower) // Q.256
p, rhs := newPoiss(lam) p, rhs := newPoiss(lam)

View File

@ -1,33 +1,24 @@
package types package types
import ( import (
"strconv" "github.com/ethereum/go-ethereum/common"
"github.com/libp2p/go-libp2p-core/peer" "github.com/libp2p/go-libp2p-core/peer"
"github.com/Secured-Finance/dione/config"
) )
// DrandRound represents the round number in DRAND const TicketRandomnessLookback = 1
type DrandRound int64
const TicketRandomnessLookback = DrandRound(1)
func (e DrandRound) String() string {
return strconv.FormatInt(int64(e), 10)
}
// DioneTask represents the values of task computation // DioneTask represents the values of task computation
// DEPRECATED!
type DioneTask struct { type DioneTask struct {
OriginChain uint8 OriginChain uint8
RequestType string RequestType string
RequestParams string RequestParams string
Miner peer.ID Miner peer.ID
MinerEth string MinerEth common.Address
Ticket *Ticket
ElectionProof *ElectionProof ElectionProof *ElectionProof
BeaconEntries []BeaconEntry BeaconEntries []BeaconEntry
DrandRound DrandRound DrandRound uint64
Payload []byte Payload []byte
RequestID string RequestID string
ConsensusID string ConsensusID string
@ -39,10 +30,9 @@ func NewDioneTask(
requestType string, requestType string,
requestParams string, requestParams string,
miner peer.ID, miner peer.ID,
ticket *Ticket,
electionProof *ElectionProof, electionProof *ElectionProof,
beacon []BeaconEntry, beacon []BeaconEntry,
drand DrandRound, drandRound uint64,
payload []byte, payload []byte,
) *DioneTask { ) *DioneTask {
return &DioneTask{ return &DioneTask{
@ -50,12 +40,9 @@ func NewDioneTask(
RequestType: requestType, RequestType: requestType,
RequestParams: requestParams, RequestParams: requestParams,
Miner: miner, Miner: miner,
Ticket: ticket,
ElectionProof: electionProof, ElectionProof: electionProof,
BeaconEntries: beacon, BeaconEntries: beacon,
DrandRound: drand, DrandRound: drandRound,
Payload: payload, Payload: payload,
} }
} }
var tasksPerEpoch = NewInt(config.TasksPerEpoch)

View File

@ -1,21 +0,0 @@
package types
import (
"math/big"
"github.com/minio/blake2b-simd"
)
type Ticket struct {
VRFProof []byte
}
func (t *Ticket) Quality() float64 {
ticketHash := blake2b.Sum256(t.VRFProof)
ticketNum := BigFromBytes(ticketHash[:]).Int
ticketDenu := big.NewInt(1)
ticketDenu.Lsh(ticketDenu, 256)
tv, _ := new(big.Rat).SetFrac(ticketNum, ticketDenu).Float64()
tq := 1 - tv
return tq
}