Part 3 of Aligning Beacon Chain with Latest 2.1 - Processing Attestations (#423)

This commit is contained in:
terence tsao
2018-08-24 09:07:23 -07:00
committed by GitHub
parent 2ee9ec2120
commit 1598ae8605
16 changed files with 367 additions and 180 deletions

View File

@@ -20,6 +20,7 @@ go_library(
"@com_github_ethereum_go_ethereum//event:go_default_library", "@com_github_ethereum_go_ethereum//event:go_default_library",
"@com_github_gogo_protobuf//proto:go_default_library", "@com_github_gogo_protobuf//proto:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library", "@com_github_sirupsen_logrus//:go_default_library",
"@org_golang_x_crypto//blake2b:go_default_library",
], ],
) )

View File

@@ -2,6 +2,8 @@ package blockchain
import ( import (
"context" "context"
"encoding/binary"
"errors"
"fmt" "fmt"
"sync" "sync"
"time" "time"
@@ -15,6 +17,7 @@ import (
"github.com/prysmaticlabs/prysm/beacon-chain/utils" "github.com/prysmaticlabs/prysm/beacon-chain/utils"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1" pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
"github.com/sirupsen/logrus" "github.com/sirupsen/logrus"
"golang.org/x/crypto/blake2b"
) )
var canonicalHeadKey = "latest-canonical-head" var canonicalHeadKey = "latest-canonical-head"
@@ -194,7 +197,7 @@ func (b *BeaconChain) PersistCrystallizedState() error {
} }
// IsCycleTransition checks if a new cycle has been reached. At that point, // IsCycleTransition checks if a new cycle has been reached. At that point,
// a new state transition will occur. // a new crystallized state transition will occur.
func (b *BeaconChain) IsCycleTransition(slotNumber uint64) bool { func (b *BeaconChain) IsCycleTransition(slotNumber uint64) bool {
return slotNumber >= b.CrystallizedState().LastStateRecalc()+params.CycleLength return slotNumber >= b.CrystallizedState().LastStateRecalc()+params.CycleLength
} }
@@ -215,22 +218,6 @@ func (b *BeaconChain) CanProcessBlock(fetcher types.POWBlockFetcher, block *type
if !canProcess { if !canProcess {
return false, fmt.Errorf("time stamp verification for beacon block %v failed", block.SlotNumber()) return false, fmt.Errorf("time stamp verification for beacon block %v failed", block.SlotNumber())
} }
canProcess, err = b.verifyBlockActiveHash(block)
if err != nil {
return false, fmt.Errorf("unable to process block: %v", err)
}
if !canProcess {
return false, fmt.Errorf("active state verification for beacon block %v failed", block.SlotNumber())
}
canProcess, err = b.verifyBlockCrystallizedHash(block)
if err != nil {
return false, fmt.Errorf("unable to process block: %v", err)
}
if !canProcess {
return false, fmt.Errorf("crystallized verification for beacon block %v failed", block.SlotNumber())
}
return canProcess, nil return canProcess, nil
} }
@@ -327,6 +314,107 @@ func (b *BeaconChain) saveBlock(block *types.Block) error {
return b.db.Put(hash[:], encodedState) return b.db.Put(hash[:], encodedState)
} }
// processAttestations processes the attestations of an incoming block.
func (b *BeaconChain) processAttestations(block *types.Block) error {
// Validate attestation's slot number has is within range of incoming block number.
slotNumber := int(block.SlotNumber())
for _, attestation := range block.Attestations() {
if int(attestation.Slot) > slotNumber {
return fmt.Errorf("attestation slot number can't be higher than block slot number. Found: %v, Needed lower than: %v",
attestation.Slot,
slotNumber)
}
if int(attestation.Slot) < slotNumber-params.CycleLength {
return fmt.Errorf("attestation slot number can't be lower than block slot number by one CycleLength. Found: %v, Needed greater than: %v",
attestation.Slot,
slotNumber-params.CycleLength)
}
// Get all the block hashes up to cycle length.
parentHashes := b.getSignedParentHashes(block, attestation)
attesterIndices, err := b.getAttesterIndices(attestation)
if err != nil {
return err
}
// Verify attester bitfields matches crystallized state's prev computed bitfield.
if err := b.validateAttesterBitfields(attestation, attesterIndices); err != nil {
return err
}
// TODO: Generate validators aggregated pub key.
// Hash parentHashes + shardID + slotNumber + shardBlockHash into a message to use to
// to verify with aggregated public key and aggregated attestation signature.
msg := make([]byte, binary.MaxVarintLen64)
var signedHashesStr []byte
for _, parentHash := range parentHashes {
signedHashesStr = append(signedHashesStr, parentHash.Bytes()...)
signedHashesStr = append(signedHashesStr, byte(' '))
}
binary.PutUvarint(msg, attestation.Slot%params.CycleLength)
msg = append(msg, signedHashesStr...)
binary.PutUvarint(msg, attestation.ShardId)
msg = append(msg, attestation.ShardBlockHash...)
msgHash := blake2b.Sum512(msg)
log.Debugf("Attestation message for shard: %v, slot %v, block hash %v is: %v",
attestation.ShardId, attestation.Slot, attestation.ShardBlockHash, msgHash)
// TODO: Verify msgHash against aggregated pub key and aggregated signature.
}
return nil
}
// getSignedParentHashes returns all the parent hashes stored in active state up to last ycle length.
func (b *BeaconChain) getSignedParentHashes(block *types.Block, attestation *pb.AttestationRecord) []*common.Hash {
var signedParentHashes []*common.Hash
start := block.SlotNumber() - attestation.Slot
end := block.SlotNumber() - attestation.Slot - uint64(len(attestation.ObliqueParentHashes)) + params.CycleLength
for _, hashes := range b.ActiveState().RecentBlockHashes()[start:end] {
signedParentHashes = append(signedParentHashes, &hashes)
}
for _, obliqueParentHashes := range attestation.ObliqueParentHashes {
hashes := common.BytesToHash(obliqueParentHashes)
signedParentHashes = append(signedParentHashes, &hashes)
}
return signedParentHashes
}
// getAttesterIndices returns the attester committee of based from attestation's shard ID and slot number.
func (b *BeaconChain) getAttesterIndices(attestation *pb.AttestationRecord) ([]uint32, error) {
lastStateRecalc := b.CrystallizedState().LastStateRecalc()
shardCommitteeArray := b.CrystallizedState().IndicesForHeights()
shardCommittee := shardCommitteeArray[attestation.Slot-lastStateRecalc+params.CycleLength].ArrayShardAndCommittee
for i := 0; i < len(shardCommittee); i++ {
if attestation.ShardId == shardCommittee[i].ShardId {
return shardCommittee[i].Committee, nil
}
}
return nil, fmt.Errorf("unable to find attestation based on slot: %v, shardID: %v", attestation.Slot, attestation.ShardId)
}
// validateAttesterBitfields validates the attester bitfields are equal between attestation and crystallized state's calculation.
func (b *BeaconChain) validateAttesterBitfields(attestation *pb.AttestationRecord, attesterIndices []uint32) error {
// Validate attester bit field has the correct length.
if utils.BitLength(len(attesterIndices)) != len(attestation.AttesterBitfield) {
return fmt.Errorf("attestation has incorrect bitfield length. Found %v, expected %v",
len(attestation.AttesterBitfield), utils.BitLength(len(attesterIndices)))
}
// Valid attestation can not have non-zero trailing bits.
lastBit := len(attesterIndices)
if lastBit%8 != 0 {
for i := 0; i < 8-lastBit%8; i++ {
if utils.CheckBit(attestation.AttesterBitfield, lastBit+i) {
return errors.New("attestation has non-zero trailing bits")
}
}
}
return nil
}
// saveCanonical puts the passed block into the beacon chain db // saveCanonical puts the passed block into the beacon chain db
// and also saves a "latest-head" key mapping to the block in the db. // and also saves a "latest-head" key mapping to the block in the db.
func (b *BeaconChain) saveCanonical(block *types.Block) error { func (b *BeaconChain) saveCanonical(block *types.Block) error {

View File

@@ -286,49 +286,20 @@ func TestCanProcessBlock(t *testing.T) {
canProcess, err := beaconChain.CanProcessBlock(&mockFetcher{}, block, true) canProcess, err := beaconChain.CanProcessBlock(&mockFetcher{}, block, true)
if err != nil { if err != nil {
t.Fatalf("CanProcessBlocks failed: %v", err) t.Fatalf("canProcessBlocks failed: %v", err)
} }
if !canProcess { if !canProcess {
t.Error("Should be able to process block, could not") t.Error("Should be able to process block, could not")
} }
// Negative scenario #1, invalid active hash // Negative scenario, invalid timestamp
block = NewBlock(t, &pb.BeaconBlock{
SlotNumber: 2,
ActiveStateHash: []byte{'A'},
CrystallizedStateHash: crystallizedHash[:],
ParentHash: parentHash[:],
})
canProcess, err = beaconChain.CanProcessBlock(&mockFetcher{}, block, true)
if err == nil {
t.Fatalf("CanProcessBlocks failed: %v", err)
}
if canProcess {
t.Error("Should not be able to process block with invalid active hash")
}
// Negative scenario #2, invalid crystallized hash
block = NewBlock(t, &pb.BeaconBlock{
SlotNumber: 2,
ActiveStateHash: activeHash[:],
CrystallizedStateHash: []byte{'A'},
ParentHash: parentHash[:],
})
canProcess, err = beaconChain.CanProcessBlock(&mockFetcher{}, block, true)
if err == nil {
t.Fatalf("CanProcessBlocks failed: %v", err)
}
if canProcess {
t.Error("Should not be able to process block with invalid crystallied hash")
}
// Negative scenario #3, invalid timestamp
block = NewBlock(t, &pb.BeaconBlock{ block = NewBlock(t, &pb.BeaconBlock{
SlotNumber: 1000000, SlotNumber: 1000000,
ActiveStateHash: activeHash[:], ActiveStateHash: activeHash[:],
CrystallizedStateHash: crystallizedHash[:], CrystallizedStateHash: crystallizedHash[:],
ParentHash: parentHash[:], ParentHash: parentHash[:],
}) })
canProcess, err = beaconChain.CanProcessBlock(&mockFetcher{}, block, true) canProcess, err = beaconChain.CanProcessBlock(&mockFetcher{}, block, true)
if err == nil { if err == nil {
t.Fatalf("CanProcessBlocks failed: %v", err) t.Fatalf("CanProcessBlocks failed: %v", err)
@@ -338,63 +309,6 @@ func TestCanProcessBlock(t *testing.T) {
} }
} }
func TestProcessBlockWithBadHashes(t *testing.T) {
beaconChain, db := startInMemoryBeaconChain(t)
defer db.Close()
// Test negative scenario where active state hash is different than node's compute.
parentBlock := NewBlock(t, nil)
parentHash, err := parentBlock.Hash()
if err != nil {
t.Fatalf("Failed to compute parent block's hash: %v", err)
}
if err = db.DB().Put(parentHash[:], []byte{}); err != nil {
t.Fatalf("Failed to put parent block on db: %v", err)
}
// Initialize state.
active := types.NewActiveState(&pb.ActiveState{RecentBlockHashes: [][]byte{{'A'}}})
activeStateHash, err := active.Hash()
if err != nil {
t.Fatalf("Cannot hash active state: %v", err)
}
crystallized := types.NewCrystallizedState(&pb.CrystallizedState{LastStateRecalc: 10000})
crystallizedStateHash, err := crystallized.Hash()
if err != nil {
t.Fatalf("Cannot hash crystallized state: %v", err)
}
block := NewBlock(t, &pb.BeaconBlock{
SlotNumber: 1,
ActiveStateHash: activeStateHash[:],
CrystallizedStateHash: crystallizedStateHash[:],
ParentHash: parentHash[:],
})
// Test negative scenario where active state hash is different than node's compute.
beaconChain.state.ActiveState = types.NewActiveState(&pb.ActiveState{RecentBlockHashes: [][]byte{{'B'}}})
canProcess, err := beaconChain.CanProcessBlock(&mockFetcher{}, block, true)
if err == nil {
t.Fatalf("CanProcessBlocks failed: %v", err)
}
if canProcess {
t.Error("CanProcessBlocks should have returned false with diff state hashes")
}
// Test negative scenario where crystallized state hash is different than node's compute.
beaconChain.state.CrystallizedState = types.NewCrystallizedState(&pb.CrystallizedState{LastStateRecalc: 9999})
canProcess, err = beaconChain.CanProcessBlock(&mockFetcher{}, block, true)
if err == nil {
t.Fatalf("CanProcessBlocks failed: %v", err)
}
if canProcess {
t.Error("CanProcessBlocks should have returned false with diff state hashes")
}
}
func TestIsSlotTransition(t *testing.T) { func TestIsSlotTransition(t *testing.T) {
beaconChain, db := startInMemoryBeaconChain(t) beaconChain, db := startInMemoryBeaconChain(t)
defer db.Close() defer db.Close()
@@ -459,37 +373,7 @@ func TestCanProcessBlockObserver(t *testing.T) {
t.Error("Should be able to process block, could not") t.Error("Should be able to process block, could not")
} }
// Negative scenario #1, invalid crystallized state hash // Negative scenario, invalid timestamp
block = NewBlock(t, &pb.BeaconBlock{
SlotNumber: 2,
ActiveStateHash: activeHash[:],
CrystallizedStateHash: []byte{'A'},
ParentHash: parentHash[:],
})
canProcess, err = beaconChain.CanProcessBlock(nil, block, false)
if err == nil {
t.Fatalf("CanProcessBlocks failed: %v", err)
}
if canProcess {
t.Error("Should not be able to process block with invalid crystallized hash")
}
// Negative scenario #2, invalid active sate hash
block = NewBlock(t, &pb.BeaconBlock{
SlotNumber: 2,
ActiveStateHash: []byte{'A'},
CrystallizedStateHash: crystallizedHash[:],
ParentHash: parentHash[:],
})
canProcess, err = beaconChain.CanProcessBlock(nil, block, false)
if err == nil {
t.Fatalf("CanProcessBlocks failed: %v", err)
}
if canProcess {
t.Error("Should not be able to process block with invalid active hash")
}
// Negative scenario #3, invalid timestamp
block = NewBlock(t, &pb.BeaconBlock{ block = NewBlock(t, &pb.BeaconBlock{
SlotNumber: 1000000, SlotNumber: 1000000,
ActiveStateHash: activeHash[:], ActiveStateHash: activeHash[:],
@@ -564,6 +448,116 @@ func TestComputeActiveState(t *testing.T) {
} }
} }
func TestCanProcessAttestations(t *testing.T) {
bc, db := startInMemoryBeaconChain(t)
defer db.Close()
// Process attestation on this block should fail because AttestationRecord's slot # > than block's slot #.
block := NewBlock(t, &pb.BeaconBlock{
SlotNumber: 1,
Attestations: []*pb.AttestationRecord{
{Slot: 2, ShardId: 0},
},
})
if err := bc.processAttestations(block); err == nil {
t.Error("Process attestation should have failed because attestation slot # > block #")
}
// Process attestation on this should fail because AttestationRecord's slot # > than block's slot #.
block = NewBlock(t, &pb.BeaconBlock{
SlotNumber: 2 + params.CycleLength,
Attestations: []*pb.AttestationRecord{
{Slot: 1, ShardId: 0},
},
})
if err := bc.processAttestations(block); err == nil {
t.Error("Process attestation should have failed because attestation slot # < block # + cycle length")
}
block = NewBlock(t, &pb.BeaconBlock{
SlotNumber: 0,
Attestations: []*pb.AttestationRecord{
{Slot: 0, ShardId: 0, ObliqueParentHashes: [][]byte{{'A'}, {'B'}, {'C'}}},
},
})
var recentBlockHashes [][]byte
for i := 0; i < params.CycleLength; i++ {
recentBlockHashes = append(recentBlockHashes, []byte{'X'})
}
active := types.NewActiveState(&pb.ActiveState{RecentBlockHashes: recentBlockHashes})
if err := bc.SetActiveState(active); err != nil {
t.Fatalf("unable to mutate active state: %v", err)
}
// Process attestation on this crystallized state should fail because only committee is in shard 1.
crystallized := types.NewCrystallizedState(&pb.CrystallizedState{
LastStateRecalc: 64,
IndicesForHeights: []*pb.ShardAndCommitteeArray{
{
ArrayShardAndCommittee: []*pb.ShardAndCommittee{
{ShardId: 1, Committee: []uint32{0, 1, 2, 3, 4, 5}},
},
},
},
})
if err := bc.SetCrystallizedState(crystallized); err != nil {
t.Fatalf("unable to mutate crystallized state: %v", err)
}
if err := bc.processAttestations(block); err == nil {
t.Error("Process attestation should have failed, there's no committee in shard 0")
}
// Process attestation should work now, there's a committee in shard 0.
crystallized = types.NewCrystallizedState(&pb.CrystallizedState{
LastStateRecalc: 64,
IndicesForHeights: []*pb.ShardAndCommitteeArray{
{
ArrayShardAndCommittee: []*pb.ShardAndCommittee{
{ShardId: 0, Committee: []uint32{0, 1, 2, 3, 4, 5}},
},
},
},
})
if err := bc.SetCrystallizedState(crystallized); err != nil {
t.Fatalf("unable to mutate crystallized state: %v", err)
}
// Process attestation should fail because attester bit field has incorrect length.
block = NewBlock(t, &pb.BeaconBlock{
SlotNumber: 0,
Attestations: []*pb.AttestationRecord{
{Slot: 0, ShardId: 0, AttesterBitfield: []byte{'A', 'B', 'C'}},
},
})
if err := bc.processAttestations(block); err == nil {
t.Error("Process attestation should have failed, incorrect attester bit field length")
}
// Set attester bitfield to the right length.
block = NewBlock(t, &pb.BeaconBlock{
SlotNumber: 0,
Attestations: []*pb.AttestationRecord{
{Slot: 0, ShardId: 0, AttesterBitfield: []byte{'a'}},
},
})
// Process attestation should fail because the non-zero leading bits for votes.
// a is 01100001
if err := bc.processAttestations(block); err == nil {
t.Error("Process attestation should have failed, incorrect attester bit field length")
}
// Process attestation should work with correct bitfield bits.
block = NewBlock(t, &pb.BeaconBlock{
SlotNumber: 0,
Attestations: []*pb.AttestationRecord{
{Slot: 0, ShardId: 0, AttesterBitfield: []byte{'0'}},
},
})
if err := bc.processAttestations(block); err != nil {
t.Error(err)
}
}
// NewBlock is a helper method to create blocks with valid defaults. // NewBlock is a helper method to create blocks with valid defaults.
// For a generic block, use NewBlock(t, nil). // For a generic block, use NewBlock(t, nil).
func NewBlock(t *testing.T, b *pb.BeaconBlock) *types.Block { func NewBlock(t *testing.T, b *pb.BeaconBlock) *types.Block {

View File

@@ -295,6 +295,13 @@ func (c *ChainService) blockProcessing(done <-chan struct{}) {
log.Debugf("Incoming block failed validity conditions: %v", err) log.Debugf("Incoming block failed validity conditions: %v", err)
} }
// Process attestation as a validator.
if err := c.chain.processAttestations(block); err != nil {
// We might receive a lot of blocks that fail attestation processing,
// so we create a debug level log instead of an error log.
log.Debugf("could not process attestation: %v", err)
}
// If we cannot process this block, we keep listening. // If we cannot process this block, we keep listening.
if !canProcess { if !canProcess {
continue continue

View File

@@ -419,3 +419,98 @@ func TestUpdateHead(t *testing.T) {
chainService.lastSlot = 100 chainService.lastSlot = 100
chainService.updateHead(101) chainService.updateHead(101)
} }
func TestProcessingBlockWithAttestations(t *testing.T) {
ctx := context.Background()
config := &database.DBConfig{DataDir: "", Name: "", InMemory: true}
db, err := database.NewDB(config)
if err != nil {
t.Fatalf("could not setup beaconDB: %v", err)
}
endpoint := "ws://127.0.0.1"
client := &mockClient{}
web3Service, err := powchain.NewWeb3Service(ctx, &powchain.Web3ServiceConfig{Endpoint: endpoint, Pubkey: "", VrcAddr: common.Address{}}, client, client, client)
if err != nil {
t.Fatalf("unable to set up web3 service: %v", err)
}
beaconChain, err := NewBeaconChain(db.DB())
if err != nil {
t.Fatalf("could not register blockchain service: %v", err)
}
var validators []*pb.ValidatorRecord
for i := 0; i < 40; i++ {
validator := &pb.ValidatorRecord{Balance: 32, StartDynasty: 1, EndDynasty: 10}
validators = append(validators, validator)
}
crystallized := types.NewCrystallizedState(&pb.CrystallizedState{
LastStateRecalc: 64,
Validators: validators,
CurrentDynasty: 5,
IndicesForHeights: []*pb.ShardAndCommitteeArray{
{
ArrayShardAndCommittee: []*pb.ShardAndCommittee{
{ShardId: 0, Committee: []uint32{0, 1, 2, 3, 4, 5}},
},
},
},
})
crystallizedStateHash, err := crystallized.Hash()
if err != nil {
t.Fatalf("Cannot hash crystallized state: %v", err)
}
if err := beaconChain.SetCrystallizedState(crystallized); err != nil {
t.Fatalf("unable to mutate crystallized state: %v", err)
}
var recentBlockHashes [][]byte
for i := 0; i < params.CycleLength+1; i++ {
recentBlockHashes = append(recentBlockHashes, []byte{'X'})
}
active := types.NewActiveState(&pb.ActiveState{RecentBlockHashes: recentBlockHashes})
if err := beaconChain.SetActiveState(active); err != nil {
t.Fatalf("unable to mutate active state: %v", err)
}
cfg := &Config{
BeaconBlockBuf: 0,
BeaconDB: db.DB(),
Chain: beaconChain,
Web3Service: web3Service,
}
chainService, _ := NewChainService(ctx, cfg)
exitRoutine := make(chan bool)
go func() {
chainService.blockProcessing(chainService.ctx.Done())
<-exitRoutine
}()
parentBlock := NewBlock(t, nil)
if err := chainService.SaveBlock(parentBlock); err != nil {
t.Fatal(err)
}
parentHash, _ := parentBlock.Hash()
activeStateHash, err := active.Hash()
if err != nil {
t.Fatalf("Cannot hash active state: %v", err)
}
block := NewBlock(t, &pb.BeaconBlock{
SlotNumber: 1,
ActiveStateHash: activeStateHash[:],
CrystallizedStateHash: crystallizedStateHash[:],
ParentHash: parentHash[:],
PowChainRef: []byte("a"),
Attestations: []*pb.AttestationRecord{
{Slot: 0, ShardId: 0, AttesterBitfield: []byte{'0'}},
},
})
chainService.incomingBlockChan <- block
chainService.cancel()
exitRoutine <- true
}

View File

@@ -27,6 +27,7 @@ go_test(
"validator_test.go", "validator_test.go",
], ],
embed = [":go_default_library"], embed = [":go_default_library"],
race = "off", # TODO(#434): fix issues with tests failing with race on.
deps = [ deps = [
"//beacon-chain/params:go_default_library", "//beacon-chain/params:go_default_library",
"//beacon-chain/types:go_default_library", "//beacon-chain/types:go_default_library",

View File

@@ -1,8 +1,6 @@
package casper package casper
import ( import (
"fmt"
"github.com/prysmaticlabs/prysm/beacon-chain/params" "github.com/prysmaticlabs/prysm/beacon-chain/params"
"github.com/prysmaticlabs/prysm/beacon-chain/types" "github.com/prysmaticlabs/prysm/beacon-chain/types"
"github.com/prysmaticlabs/prysm/beacon-chain/utils" "github.com/prysmaticlabs/prysm/beacon-chain/utils"
@@ -31,10 +29,7 @@ func CalculateRewards(active *types.ActiveState, crystallized *types.Crystallize
log.Debug("Applying rewards and penalties for the validators from last cycle") log.Debug("Applying rewards and penalties for the validators from last cycle")
for i, attesterIndex := range activeValidators { for i, attesterIndex := range activeValidators {
voted, err := utils.CheckBit(latestPendingAtt.AttesterBitfield, attesterIndex) voted := utils.CheckBit(latestPendingAtt.AttesterBitfield, attesterIndex)
if err != nil {
return fmt.Errorf("exiting calculate rewards FFG due to %v", err)
}
if voted { if voted {
validators[i].Balance += params.AttesterReward validators[i].Balance += params.AttesterReward
} else { } else {

View File

@@ -89,6 +89,7 @@ func SampleAttestersAndProposers(seed common.Hash, crystallized *types.Crystalli
if crystallized.ValidatorsLength() < params.MinCommiteeSize { if crystallized.ValidatorsLength() < params.MinCommiteeSize {
attesterCount = crystallized.ValidatorsLength() attesterCount = crystallized.ValidatorsLength()
} }
fmt.Println()
indices, err := utils.ShuffleIndices(seed, ActiveValidatorIndices(crystallized)) indices, err := utils.ShuffleIndices(seed, ActiveValidatorIndices(crystallized))
if err != nil { if err != nil {
return nil, -1, err return nil, -1, err

View File

@@ -81,10 +81,7 @@ func TestHasVoted(t *testing.T) {
active.NewPendingAttestation(pendingAttestation) active.NewPendingAttestation(pendingAttestation)
for i := 0; i < len(active.LatestPendingAttestation().AttesterBitfield); i++ { for i := 0; i < len(active.LatestPendingAttestation().AttesterBitfield); i++ {
voted, err := utils.CheckBit(active.LatestPendingAttestation().AttesterBitfield, i) voted := utils.CheckBit(active.LatestPendingAttestation().AttesterBitfield, i)
if err != nil {
t.Errorf("checking bitfield for vote failed: %v", err)
}
if !voted { if !voted {
t.Error("validator voted but received didn't vote") t.Error("validator voted but received didn't vote")
} }
@@ -97,10 +94,7 @@ func TestHasVoted(t *testing.T) {
active.NewPendingAttestation(pendingAttestation) active.NewPendingAttestation(pendingAttestation)
for i := 0; i < len(active.LatestPendingAttestation().AttesterBitfield); i++ { for i := 0; i < len(active.LatestPendingAttestation().AttesterBitfield); i++ {
voted, err := utils.CheckBit(active.LatestPendingAttestation().AttesterBitfield, i) voted := utils.CheckBit(active.LatestPendingAttestation().AttesterBitfield, i)
if err != nil {
t.Errorf("checking bitfield for vote failed: %v", err)
}
if i%2 == 0 && voted { if i%2 == 0 && voted {
t.Error("validator didn't vote but received voted") t.Error("validator didn't vote but received voted")
} }

View File

@@ -285,5 +285,6 @@ func (b *BeaconNode) registerRPCService(ctx *cli.Context) error {
KeyFlag: key, KeyFlag: key,
SubscriptionBuf: 100, SubscriptionBuf: 100,
}, chainService) }, chainService)
return b.services.RegisterService(rpcService) return b.services.RegisterService(rpcService)
} }

View File

@@ -119,6 +119,11 @@ func (b *Block) AttestationCount() int {
return len(b.data.Attestations) return len(b.data.Attestations)
} }
// Attestations returns an array of attestations in the block.
func (b *Block) Attestations() []*pb.AttestationRecord {
return b.data.Attestations
}
// Timestamp returns the Go type time.Time from the protobuf type contained in the block. // Timestamp returns the Go type time.Time from the protobuf type contained in the block.
func (b *Block) Timestamp() (time.Time, error) { func (b *Block) Timestamp() (time.Time, error) {
return ptypes.Timestamp(b.data.Timestamp) return ptypes.Timestamp(b.data.Timestamp)

View File

@@ -1,11 +1,7 @@
package utils package utils
import (
"errors"
)
// CheckBit checks if a bit in a bit field is one. // CheckBit checks if a bit in a bit field is one.
func CheckBit(bitfield []byte, index int) (bool, error) { func CheckBit(bitfield []byte, index int) bool {
chunkLocation := (index + 1) / 8 chunkLocation := (index + 1) / 8
indexLocation := (index + 1) % 8 indexLocation := (index + 1) % 8
if indexLocation == 0 { if indexLocation == 0 {
@@ -14,15 +10,8 @@ func CheckBit(bitfield []byte, index int) (bool, error) {
chunkLocation++ chunkLocation++
} }
if len(bitfield) < chunkLocation {
return false, errors.New("attester index does not exist")
}
field := bitfield[chunkLocation-1] >> (8 - uint(indexLocation)) field := bitfield[chunkLocation-1] >> (8 - uint(indexLocation))
if field%2 != 0 { return field%2 != 0
return true, nil
}
return false, nil
} }
// BitSetCount counts the number of 1s in a byte using the following algo: // BitSetCount counts the number of 1s in a byte using the following algo:
@@ -32,3 +21,8 @@ func BitSetCount(v byte) byte {
v = (v & 0x33) + ((v >> 2) & 0x33) v = (v & 0x33) + ((v >> 2) & 0x33)
return (v + (v >> 4)) & 0xF return (v + (v >> 4)) & 0xF
} }
// BitLength returns the length of the bitfield for a given number of attesters in bytes.
func BitLength(b int) int {
return (b + 7) / 8
}

View File

@@ -15,20 +15,13 @@ func TestCheckBit(t *testing.T) {
{a: []byte{146}, b: 4, c: false}, //10010010 {a: []byte{146}, b: 4, c: false}, //10010010
{a: []byte{179}, b: 7, c: true}, //10110011 {a: []byte{179}, b: 7, c: true}, //10110011
{a: []byte{49}, b: 6, c: false}, //00110001 {a: []byte{49}, b: 6, c: false}, //00110001
} }
for _, tt := range tests { for _, tt := range tests {
set, err := CheckBit(tt.a, tt.b) set := CheckBit(tt.a, tt.b)
if err != nil {
t.Fatalf("Call check bit failed: %v", err)
}
if set != tt.c { if set != tt.c {
t.Errorf("Test check bit set failed with %v and location %v", tt.a, tt.b) t.Errorf("Test check bit set failed with %v and location %v", tt.a, tt.b)
} }
} }
if _, err := CheckBit([]byte{1}, 8); err == nil {
t.Errorf("Call check bit should have failed with invalid index")
}
} }
func TestBitSetCount(t *testing.T) { func TestBitSetCount(t *testing.T) {
@@ -44,7 +37,25 @@ func TestBitSetCount(t *testing.T) {
} }
for _, tt := range tests { for _, tt := range tests {
if int(BitSetCount(tt.a)) != tt.b { if int(BitSetCount(tt.a)) != tt.b {
t.Errorf("Expected %v, Got %v", tt.b, int(BitSetCount(tt.a))) t.Errorf("BitSetCount(%d) = %v, want = %d", tt.a, int(BitSetCount(tt.a)), tt.b)
}
}
}
func TestByteLength(t *testing.T) {
tests := []struct {
a int
b int
}{
{a: 200, b: 25}, //11001000
{a: 34324, b: 4291}, //10010100
{a: 146, b: 19}, //10010010
{a: 179, b: 23}, //10110011
{a: 49, b: 7}, //00110001
}
for _, tt := range tests {
if BitLength(tt.a) != tt.b {
t.Errorf("BitLength(%d) = %d, want = %d", tt.a, BitLength(tt.a), tt.b)
} }
} }
} }

View File

@@ -38,8 +38,8 @@ func DefaultConfig() *Config {
return &Config{AttesterChanBuf: 5, ProposerChanBuf: 5} return &Config{AttesterChanBuf: 5, ProposerChanBuf: 5}
} }
// NewBeaconClient instantiates a service that interacts with a beacon node. // NewBeaconValidator instantiates a service that interacts with a beacon node.
func NewBeaconClient(ctx context.Context, cfg *Config, rpcClient types.RPCClient) *Service { func NewBeaconValidator(ctx context.Context, cfg *Config, rpcClient types.RPCClient) *Service {
ctx, cancel := context.WithCancel(ctx) ctx, cancel := context.WithCancel(ctx)
return &Service{ return &Service{
ctx: ctx, ctx: ctx,

View File

@@ -49,7 +49,7 @@ func (fc *mockClient) BeaconServiceClient() pb.BeaconServiceClient {
func TestChannelGetters(t *testing.T) { func TestChannelGetters(t *testing.T) {
ctrl := gomock.NewController(t) ctrl := gomock.NewController(t)
defer ctrl.Finish() defer ctrl.Finish()
b := NewBeaconClient(context.Background(), &Config{AttesterChanBuf: 1, ProposerChanBuf: 1}, &mockClient{ctrl}) b := NewBeaconValidator(context.Background(), &Config{AttesterChanBuf: 1, ProposerChanBuf: 1}, &mockClient{ctrl})
b.proposerChan <- false b.proposerChan <- false
proposerVal := <-b.ProposerAssignment() proposerVal := <-b.ProposerAssignment()
if proposerVal { if proposerVal {
@@ -66,7 +66,7 @@ func TestLifecycle(t *testing.T) {
hook := logTest.NewGlobal() hook := logTest.NewGlobal()
ctrl := gomock.NewController(t) ctrl := gomock.NewController(t)
defer ctrl.Finish() defer ctrl.Finish()
b := NewBeaconClient(context.Background(), &Config{AttesterChanBuf: 0, ProposerChanBuf: 0}, &mockClient{ctrl}) b := NewBeaconValidator(context.Background(), &Config{AttesterChanBuf: 0, ProposerChanBuf: 0}, &mockClient{ctrl})
// Testing default config values. // Testing default config values.
cfg := DefaultConfig() cfg := DefaultConfig()
@@ -88,7 +88,7 @@ func TestFetchBeaconBlocks(t *testing.T) {
ctrl := gomock.NewController(t) ctrl := gomock.NewController(t)
defer ctrl.Finish() defer ctrl.Finish()
b := NewBeaconClient(context.Background(), &Config{AttesterChanBuf: 1, ProposerChanBuf: 1}, &mockClient{ctrl}) b := NewBeaconValidator(context.Background(), &Config{AttesterChanBuf: 1, ProposerChanBuf: 1}, &mockClient{ctrl})
// Create mock for the stream returned by LatestBeaconBlock. // Create mock for the stream returned by LatestBeaconBlock.
stream := internal.NewMockBeaconService_LatestBeaconBlockClient(ctrl) stream := internal.NewMockBeaconService_LatestBeaconBlockClient(ctrl)
@@ -163,7 +163,7 @@ func TestFetchCrystallizedState(t *testing.T) {
ctrl := gomock.NewController(t) ctrl := gomock.NewController(t)
defer ctrl.Finish() defer ctrl.Finish()
b := NewBeaconClient(context.Background(), &Config{AttesterChanBuf: 0, ProposerChanBuf: 0}, &mockClient{ctrl}) b := NewBeaconValidator(context.Background(), &Config{AttesterChanBuf: 0, ProposerChanBuf: 0}, &mockClient{ctrl})
// Creating a faulty stream will trigger error. // Creating a faulty stream will trigger error.
stream := internal.NewMockBeaconService_LatestCrystallizedStateClient(ctrl) stream := internal.NewMockBeaconService_LatestCrystallizedStateClient(ctrl)

View File

@@ -168,7 +168,7 @@ func (s *ShardEthereum) registerBeaconService() error {
if err := s.services.FetchService(&rpcService); err != nil { if err := s.services.FetchService(&rpcService); err != nil {
return err return err
} }
b := beacon.NewBeaconClient(context.TODO(), beacon.DefaultConfig(), rpcService) b := beacon.NewBeaconValidator(context.TODO(), beacon.DefaultConfig(), rpcService)
return s.services.RegisterService(b) return s.services.RegisterService(b)
} }