Create and Verify Signatures for Attestations (#1908)

* Verify signatures of attestations

* Implement BLS Signing for attestations

* Remove custody bit 0 from the attestation for now

* Fixes tests for attestations

* Fix tests to ensure they use proper attester indice

* Run gazelle

* Goimports

* Test attestation sigs in block operations

* Change formatting and make sure signatures are actually verified

* Fix duplicate import

* Fix duplicate import from merge

* Organize attestation sig to be conssitent with codebase

* Update to comments

* Change signatures to use aggregation

* Run gazelle

* Change function to return err instead of bool
Also gofmt

* Fix for comments

* Move createAggregationSignature to a function in attestations.go
This commit is contained in:
Ivan Martinez
2019-03-08 01:30:01 -06:00
committed by Nishant Das
parent 938842be69
commit 23072983ff
20 changed files with 340 additions and 87 deletions

View File

@@ -8,6 +8,11 @@ go_library(
deps = [
"//beacon-chain/core/helpers:go_default_library",
"//proto/beacon/p2p/v1:go_default_library",
"//shared/bls:go_default_library",
"//shared/forkutils:go_default_library",
"//shared/hashutil:go_default_library",
"//shared/params:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
],
)

View File

@@ -6,8 +6,15 @@ package attestations
import (
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
"github.com/prysmaticlabs/prysm/shared/bls"
"github.com/prysmaticlabs/prysm/shared/forkutils"
"github.com/prysmaticlabs/prysm/shared/hashutil"
"github.com/prysmaticlabs/prysm/shared/params"
"github.com/sirupsen/logrus"
)
var log = logrus.WithField("prefix", "attestation")
// IsDoubleVote checks if both of the attestations have been used to vote for the same slot.
// Spec:
// def is_double_vote(attestation_data_1: AttestationData,
@@ -46,3 +53,21 @@ func IsSurroundVote(attestation1 *pb.AttestationData, attestation2 *pb.Attestati
return sourceEpoch1 < sourceEpoch2 && targetEpoch2 < targetEpoch1
}
// AggregateSignature returns the signature of the attestation from the beacon state, attestation
// and secret key provided.
func AggregateSignature(beaconState *pb.BeaconState, att *pb.Attestation, privKey *bls.SecretKey) []byte {
attestationDataHash, err := hashutil.HashProto(&pb.AttestationDataAndCustodyBit{
Data: att.Data,
CustodyBit: true,
})
if err != nil {
log.Errorf("could not hash attestation data: %v", err)
}
currentEpoch := helpers.SlotToEpoch(att.Data.Slot)
domain := forkutils.DomainVersion(beaconState.Fork, currentEpoch, params.BeaconConfig().DomainAttestation)
sig := privKey.Sign(attestationDataHash[:], domain)
return sig.Marshal()
}

View File

@@ -38,6 +38,7 @@ go_test(
],
embed = [":go_default_library"],
deps = [
"//beacon-chain/core/attestations:go_default_library",
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/state:go_default_library",
"//proto/beacon/p2p/v1:go_default_library",

View File

@@ -134,7 +134,7 @@ func verifyBlockRandao(beaconState *pb.BeaconState, block *pb.BeaconBlock, propo
"pubkey": fmt.Sprintf("%#x", proposer.Pubkey),
"epochSig": fmt.Sprintf("%#x", sig.Marshal()),
}).Info("Verifying randao")
if !sig.Verify(buf, pub, domain) {
if !sig.Verify(pub, buf, domain) {
return fmt.Errorf("block randao reveal signature did not verify")
}
return nil
@@ -440,6 +440,52 @@ func ProcessBlockAttestations(
return beaconState, nil
}
func verifyAttestationSig(beaconState *pb.BeaconState, att *pb.Attestation) error {
attesterIndices, err := helpers.AttestationParticipants(beaconState, att.Data, att.AggregationBitfield)
if err != nil {
return fmt.Errorf("could not get attestation participants: %v", err)
}
attestorPubKeys := make([]*bls.PublicKey, len(attesterIndices))
for idx, participantIndex := range attesterIndices {
pubkey, err := bls.PublicKeyFromBytes(beaconState.ValidatorRegistry[participantIndex].Pubkey)
if err != nil {
return fmt.Errorf("could not deserialize attestor public key: %v at index: %v", err, participantIndex)
}
attestorPubKeys[idx] = pubkey
}
aggregatePubkeys := []*bls.PublicKey{
bls.AggregatePublicKeys(attestorPubKeys),
}
attestationDataHash, err := hashutil.HashProto(&pb.AttestationDataAndCustodyBit{
Data: att.Data,
CustodyBit: true,
})
if err != nil {
return fmt.Errorf("could not hash attestation data: %v", err)
}
messageHashes := [][]byte{
attestationDataHash[:],
}
sig, err := bls.SignatureFromBytes(att.AggregateSignature)
currentEpoch := helpers.CurrentEpoch(beaconState)
domain := forkutils.DomainVersion(beaconState.Fork, currentEpoch, params.BeaconConfig().DomainAttestation)
log.WithFields(logrus.Fields{
"aggregatePubkeys": fmt.Sprintf("%#x", aggregatePubkeys),
}).Info("Verifying attestation")
if !sig.VerifyMultiple(aggregatePubkeys, messageHashes, domain) {
return errors.New("aggregate signature did not verify")
}
return nil
}
func verifyAttestation(beaconState *pb.BeaconState, att *pb.Attestation, verifySignatures bool) error {
if att.Data.Slot < params.BeaconConfig().GenesisSlot {
return fmt.Errorf(
@@ -532,20 +578,9 @@ func verifyAttestation(beaconState *pb.BeaconState, att *pb.Attestation, verifyS
)
}
if verifySignatures {
// TODO(#258): Integrate BLS signature verification for attestation.
// assert bls_verify_multiple(
// pubkeys=[
// bls_aggregate_pubkeys([state.validator_registry[i].pubkey for i in custody_bit_0_participants]),
// bls_aggregate_pubkeys([state.validator_registry[i].pubkey for i in custody_bit_1_participants]),
// ],
// message_hash=[
// hash_tree_root(AttestationDataAndCustodyBit(data=attestation.data, custody_bit=0b0)),
// hash_tree_root(AttestationDataAndCustodyBit(data=attestation.data, custody_bit=0b1)),
// ],
// signature=attestation.aggregate_signature,
// domain=get_domain(state.fork, slot_to_epoch(attestation.data.slot), DOMAIN_ATTESTATION),
// )
return nil
if err := verifyAttestationSig(beaconState, att); err != nil {
return fmt.Errorf("could not verify aggregate signature %v", err)
}
}
return nil
}

View File

@@ -11,6 +11,7 @@ import (
"testing"
"time"
atts "github.com/prysmaticlabs/prysm/beacon-chain/core/attestations"
"github.com/prysmaticlabs/prysm/beacon-chain/core/blocks"
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/beacon-chain/core/state"
@@ -1018,24 +1019,27 @@ func TestProcessBlockAttestations_ShardBlockRootEqualZeroHashFailure(t *testing.
}
func TestProcessBlockAttestations_CreatePendingAttestations(t *testing.T) {
deposits, privKeys := setupInitialDeposits(t, 100)
beaconState, err := state.GenesisBeaconState(deposits, uint64(0), &pb.Eth1Data{})
beaconState.Slot = params.BeaconConfig().GenesisSlot + 70
beaconState.PreviousJustifiedEpoch = params.BeaconConfig().GenesisEpoch
stateLatestCrosslinks := make([]*pb.Crosslink, 21)
stateLatestCrosslinks[20] = &pb.Crosslink{
CrosslinkDataRootHash32: []byte{1},
}
beaconState.LatestCrosslinks = stateLatestCrosslinks
var blockRoots [][]byte
for i := uint64(0); i < params.BeaconConfig().LatestBlockRootsLength; i++ {
blockRoots = append(blockRoots, []byte{byte(i)})
}
stateLatestCrosslinks := []*pb.Crosslink{
{
CrosslinkDataRootHash32: []byte{1},
},
}
state := &pb.BeaconState{
Slot: params.BeaconConfig().GenesisSlot + 70,
PreviousJustifiedEpoch: params.BeaconConfig().GenesisEpoch,
LatestBlockRootHash32S: blockRoots,
LatestCrosslinks: stateLatestCrosslinks,
}
beaconState.LatestBlockRootHash32S = blockRoots
att1 := &pb.Attestation{
Data: &pb.AttestationData{
Shard: 0,
Shard: 20,
Slot: params.BeaconConfig().GenesisSlot + 20,
JustifiedBlockRootHash32: blockRoots[0],
LatestCrosslink: &pb.Crosslink{CrosslinkDataRootHash32: []byte{1}},
@@ -1045,18 +1049,27 @@ func TestProcessBlockAttestations_CreatePendingAttestations(t *testing.T) {
AggregationBitfield: []byte{1},
CustodyBitfield: []byte{1},
}
attestorIndices, err := helpers.AttestationParticipants(beaconState, att1.Data, att1.AggregationBitfield)
if err != nil {
t.Errorf("could not get aggregation participants %v", err)
}
att1.AggregateSignature = atts.AggregateSignature(beaconState, att1, privKeys[attestorIndices[0]])
attestations := []*pb.Attestation{att1}
block := &pb.BeaconBlock{
Body: &pb.BeaconBlockBody{
Attestations: attestations,
},
}
newState, err := blocks.ProcessBlockAttestations(
context.Background(),
state,
beaconState,
block,
false,
true,
)
pendingAttestations := newState.LatestAttestations
if err != nil {
t.Fatalf("Could not produce pending attestations: %v", err)

View File

@@ -35,6 +35,7 @@ go_test(
],
embed = [":go_default_library"],
deps = [
"//beacon-chain/core/attestations:go_default_library",
"//beacon-chain/core/helpers:go_default_library",
"//proto/beacon/p2p/v1:go_default_library",
"//shared/bls:go_default_library",

View File

@@ -7,16 +7,14 @@ import (
"context"
"fmt"
"github.com/prysmaticlabs/prysm/shared/hashutil"
"github.com/prysmaticlabs/prysm/shared/params"
bal "github.com/prysmaticlabs/prysm/beacon-chain/core/balances"
b "github.com/prysmaticlabs/prysm/beacon-chain/core/blocks"
e "github.com/prysmaticlabs/prysm/beacon-chain/core/epoch"
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
v "github.com/prysmaticlabs/prysm/beacon-chain/core/validators"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
"github.com/prysmaticlabs/prysm/shared/hashutil"
"github.com/prysmaticlabs/prysm/shared/params"
"github.com/sirupsen/logrus"
"go.opencensus.io/trace"
)

View File

@@ -9,12 +9,12 @@ import (
"testing"
"time"
atts "github.com/prysmaticlabs/prysm/beacon-chain/core/attestations"
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/beacon-chain/core/state"
"github.com/prysmaticlabs/prysm/shared/forkutils"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
"github.com/prysmaticlabs/prysm/shared/bls"
"github.com/prysmaticlabs/prysm/shared/forkutils"
"github.com/prysmaticlabs/prysm/shared/params"
)
@@ -144,6 +144,106 @@ func TestProcessBlock_IncorrectAttesterSlashing(t *testing.T) {
}
}
func TestProcessBlock_IncorrectAggregateSig(t *testing.T) {
deposits, privKeys := setupInitialDeposits(t, params.BeaconConfig().SlotsPerEpoch)
beaconState, err := state.GenesisBeaconState(deposits, uint64(0), &pb.Eth1Data{})
if err != nil {
t.Fatal(err)
}
proposerSlashings := []*pb.ProposerSlashing{
{
ProposerIndex: 1,
ProposalData_1: &pb.ProposalSignedData{
Slot: 1,
Shard: 1,
BlockRootHash32: []byte{0, 1, 0},
},
ProposalData_2: &pb.ProposalSignedData{
Slot: 1,
Shard: 1,
BlockRootHash32: []byte{0, 1, 0},
},
},
}
att1 := &pb.AttestationData{
Slot: 5,
JustifiedEpoch: params.BeaconConfig().GenesisEpoch + 5,
}
att2 := &pb.AttestationData{
Slot: 5,
JustifiedEpoch: params.BeaconConfig().GenesisEpoch + 4,
}
attesterSlashings := []*pb.AttesterSlashing{
{
SlashableAttestation_1: &pb.SlashableAttestation{
Data: att1,
ValidatorIndices: []uint64{1, 2, 3, 4, 5, 6, 7, 8},
CustodyBitfield: []byte{0xFF},
},
SlashableAttestation_2: &pb.SlashableAttestation{
Data: att2,
ValidatorIndices: []uint64{1, 2, 3, 4, 5, 6, 7, 8},
CustodyBitfield: []byte{0xFF},
},
},
}
var blockRoots [][]byte
for i := uint64(0); i < params.BeaconConfig().LatestBlockRootsLength; i++ {
blockRoots = append(blockRoots, []byte{byte(i)})
}
beaconState.LatestBlockRootHash32S = blockRoots
beaconState.LatestCrosslinks = []*pb.Crosslink{
{
CrosslinkDataRootHash32: []byte{1},
},
}
beaconState.Slot = params.BeaconConfig().GenesisSlot + 10
blockAtt := &pb.Attestation{
Data: &pb.AttestationData{
Shard: 0,
Slot: params.BeaconConfig().GenesisSlot,
JustifiedEpoch: params.BeaconConfig().GenesisEpoch,
JustifiedBlockRootHash32: blockRoots[0],
LatestCrosslink: &pb.Crosslink{CrosslinkDataRootHash32: []byte{1}},
CrosslinkDataRootHash32: params.BeaconConfig().ZeroHash[:],
},
AggregationBitfield: []byte{1},
CustodyBitfield: []byte{1},
}
attestorIndices, err := helpers.AttestationParticipants(beaconState, blockAtt.Data, blockAtt.AggregationBitfield)
if err != nil {
t.Errorf("could not get aggergation participants %v", err)
}
blockAtt.AggregateSignature = atts.AggregateSignature(beaconState, blockAtt, privKeys[attestorIndices[0]+1])
attestations := []*pb.Attestation{blockAtt}
exits := []*pb.VoluntaryExit{
{
ValidatorIndex: 10,
Epoch: params.BeaconConfig().GenesisEpoch,
},
}
randaoReveal := createRandaoReveal(t, beaconState, privKeys)
block := &pb.BeaconBlock{
Slot: params.BeaconConfig().GenesisSlot + 10,
RandaoReveal: randaoReveal,
Eth1Data: &pb.Eth1Data{
DepositRootHash32: []byte{2},
BlockHash32: []byte{3},
},
Body: &pb.BeaconBlockBody{
ProposerSlashings: proposerSlashings,
AttesterSlashings: attesterSlashings,
Attestations: attestations,
VoluntaryExits: exits,
},
}
want := "aggregate signature did not verify"
if _, err := state.ProcessBlock(context.Background(), beaconState, block, true); !strings.Contains(err.Error(), want) {
t.Errorf("Expected %s, received %v", want, err)
}
}
func TestProcessBlock_IncorrectProcessBlockAttestations(t *testing.T) {
deposits, privKeys := setupInitialDeposits(t, params.BeaconConfig().SlotsPerEpoch)
beaconState, err := state.GenesisBeaconState(deposits, uint64(0), &pb.Eth1Data{})
@@ -370,6 +470,12 @@ func TestProcessBlock_PassesProcessingConditions(t *testing.T) {
AggregationBitfield: []byte{1},
CustodyBitfield: []byte{1},
}
attestorIndices, err := helpers.AttestationParticipants(beaconState, blockAtt.Data, blockAtt.AggregationBitfield)
if err != nil {
t.Errorf("could not get aggergation participants %v", err)
}
blockAtt.AggregateSignature = atts.AggregateSignature(beaconState, blockAtt, privKeys[attestorIndices[0]])
attestations := []*pb.Attestation{blockAtt}
exits := []*pb.VoluntaryExit{
{
@@ -392,7 +498,7 @@ func TestProcessBlock_PassesProcessingConditions(t *testing.T) {
VoluntaryExits: exits,
},
}
if _, err := state.ProcessBlock(context.Background(), beaconState, block, false); err != nil {
if _, err := state.ProcessBlock(context.Background(), beaconState, block, true); err != nil {
t.Errorf("Expected block to pass processing conditions: %v", err)
}
}

View File

@@ -4,11 +4,10 @@ import (
"errors"
"fmt"
"github.com/prysmaticlabs/prysm/shared/hashutil"
"github.com/boltdb/bolt"
"github.com/gogo/protobuf/proto"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
"github.com/prysmaticlabs/prysm/shared/hashutil"
)
func createBlock(enc []byte) (*pb.BeaconBlock, error) {

View File

@@ -10,11 +10,10 @@ import (
"sync"
"syscall"
"github.com/prysmaticlabs/prysm/beacon-chain/attestation"
"github.com/ethereum/go-ethereum/common"
"github.com/ethereum/go-ethereum/ethclient"
gethRPC "github.com/ethereum/go-ethereum/rpc"
"github.com/prysmaticlabs/prysm/beacon-chain/attestation"
"github.com/prysmaticlabs/prysm/beacon-chain/blockchain"
"github.com/prysmaticlabs/prysm/beacon-chain/db"
"github.com/prysmaticlabs/prysm/beacon-chain/operations"

View File

@@ -932,38 +932,6 @@ func TestHasChainStartLogOccurred_OK(t *testing.T) {
}
}
func TestStatus(t *testing.T) {
now := time.Now()
beforeFiveMinutesAgo := now.Add(-5*time.Minute - 30*time.Second)
afterFiveMinutesAgo := now.Add(-5*time.Minute + 30*time.Second)
testCases := map[*Web3Service]string{
// "status is ok" cases
{}: "",
{isRunning: true, blockTime: afterFiveMinutesAgo}: "",
{isRunning: false, blockTime: beforeFiveMinutesAgo}: "",
{isRunning: false, runError: errors.New("test runError")}: "",
// "status is error" cases
{isRunning: true, blockTime: beforeFiveMinutesAgo}: "web3 client is not syncing",
{isRunning: true}: "web3 client is not syncing",
{isRunning: true, runError: errors.New("test runError")}: "test runError",
}
for web3ServiceState, wantedErrorText := range testCases {
status := web3ServiceState.Status()
if status == nil {
if wantedErrorText != "" {
t.Errorf("Wanted: \"%v\", but Status() return nil", wantedErrorText)
}
} else {
if status.Error() != wantedErrorText {
t.Errorf("Wanted: \"%v\", but Status() return: \"%v\"", wantedErrorText, status.Error())
}
}
}
}
func TestBlockHashByHeight_ReturnsHash(t *testing.T) {
endpoint := "ws://127.0.0.1"
web3Service, err := NewWeb3Service(context.Background(), &Web3ServiceConfig{

View File

@@ -53,7 +53,7 @@ func (as *AttesterServer) AttestationDataAtSlot(ctx context.Context, req *pb.Att
}
for beaconState.Slot < req.Slot {
beaconState, err = state.ExecuteStateTransition(
ctx, beaconState, nil /* block */, blockRoot, false, /* verify signatures */
ctx, beaconState, nil /* block */, blockRoot, true, /* verify signatures */
)
if err != nil {
return nil, fmt.Errorf("could not execute head transition: %v", err)

View File

@@ -6,9 +6,10 @@ package prysm_internal_cluster
import (
context "context"
fmt "fmt"
math "math"
proto "github.com/golang/protobuf/proto"
grpc "google.golang.org/grpc"
math "math"
)
// Reference imports to suppress errors if they are not otherwise used.

View File

@@ -95,7 +95,7 @@ func (p *PublicKey) Aggregate(p2 *PublicKey) *PublicKey {
}
// Verify a bls signature given a public key, a message, and a domain.
func (s *Signature) Verify(msg []byte, pub *PublicKey, domain uint64) bool {
func (s *Signature) Verify(pub *PublicKey, msg []byte, domain uint64) bool {
return gobls.Verify(msg, pub.val, s.val, domain)
}
@@ -103,13 +103,22 @@ func (s *Signature) Verify(msg []byte, pub *PublicKey, domain uint64) bool {
// This is vulnerable to rogue public-key attack. Each user must
// provide a proof-of-knowledge of the public key.
func (s *Signature) VerifyAggregate(pubKeys []*PublicKey, msg []byte, domain uint64) bool {
var keys []*gobls.PublicKey
for _, v := range pubKeys {
keys = append(keys, v.val)
keys := make([]*gobls.PublicKey, len(pubKeys))
for idx, v := range pubKeys {
keys[idx] = v.val
}
return s.val.VerifyAggregateCommon(keys, msg, domain)
}
// VerifyMultiple verifies each public key against each message.
func (s *Signature) VerifyMultiple(pubKeys []*PublicKey, msgs [][]byte, domain uint64) bool {
keys := make([]*gobls.PublicKey, len(pubKeys))
for idx, v := range pubKeys {
keys[idx] = v.val
}
return s.val.VerifyAggregate(keys, msgs, domain)
}
// Marshal a signature into a byte slice.
func (s *Signature) Marshal() []byte {
k := s.val.Serialize()
@@ -118,9 +127,18 @@ func (s *Signature) Marshal() []byte {
// AggregateSignatures converts a list of signatures into a single, aggregated sig.
func AggregateSignatures(sigs []*Signature) *Signature {
var ss []*gobls.Signature
for _, v := range sigs {
ss = append(ss, v.val)
ss := make([]*gobls.Signature, len(sigs))
for idx, v := range sigs {
ss[idx] = v.val
}
return &Signature{val: gobls.AggregateSignatures(ss)}
}
// AggregatePublicKeys converts a list of public keys into a single, aggregated public key.
func AggregatePublicKeys(publicKeys []*PublicKey) *PublicKey {
ss := make([]*gobls.PublicKey, len(publicKeys))
for idx, pub := range publicKeys {
ss[idx] = pub.val
}
return &PublicKey{val: gobls.AggregatePublicKeys(ss)}
}

View File

@@ -30,7 +30,7 @@ func TestSignVerify(t *testing.T) {
pub := priv.PublicKey()
msg := []byte("hello")
sig := priv.Sign(msg, 0)
if !sig.Verify(msg, pub, 0) {
if !sig.Verify(pub, msg, 0) {
t.Error("Signature did not verify")
}
}

View File

@@ -44,7 +44,7 @@ func TestDepositInput_GeneratesPb(t *testing.T) {
t.Fatal(err)
}
if !sig.Verify(buf.Bytes(), k1.PublicKey, params.BeaconConfig().DomainDeposit) {
if !sig.Verify(k1.PublicKey, buf.Bytes(), params.BeaconConfig().DomainDeposit) {
t.Error("Invalid proof of proofOfPossession signature")
}
}

View File

@@ -44,6 +44,8 @@ go_test(
"//proto/beacon/p2p/v1:go_default_library",
"//proto/beacon/rpc/v1:go_default_library",
"//shared:go_default_library",
"//shared/forkutils:go_default_library",
"//shared/hashutil:go_default_library",
"//shared/keystore:go_default_library",
"//shared/params:go_default_library",
"//shared/testutil:go_default_library",

View File

@@ -5,8 +5,11 @@ import (
"fmt"
"time"
ptypes "github.com/gogo/protobuf/types"
pbp2p "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
pb "github.com/prysmaticlabs/prysm/proto/beacon/rpc/v1"
"github.com/prysmaticlabs/prysm/shared/forkutils"
"github.com/prysmaticlabs/prysm/shared/hashutil"
"github.com/prysmaticlabs/prysm/shared/params"
"go.opencensus.io/trace"
)
@@ -112,8 +115,28 @@ func (v *validator) AttestToBlockHead(ctx context.Context, slot uint64) {
// should return a list of length equal to 1, containing validator_index.
attestation.AggregationBitfield = aggregationBitfield
// TODO(#1366): Use BLS to generate an aggregate signature.
attestation.AggregateSignature = []byte("signed")
// Retrieve the current fork data from the beacon node.
fork, err := v.beaconClient.ForkData(ctx, &ptypes.Empty{})
if err != nil {
log.Errorf("Failed to get fork data from beacon node's state: %v", err)
return
}
epoch := slot / params.BeaconConfig().SlotsPerEpoch
attDataHash, err := hashutil.HashProto(&pbp2p.AttestationDataAndCustodyBit{
Data: attestation.Data,
CustodyBit: true,
})
if err != nil {
log.Errorf("Could not hash attestation data: %v", err)
return
}
log.Infof("Signing attestation for slot: %d", slot)
domain := forkutils.DomainVersion(fork, epoch, params.BeaconConfig().DomainAttestation)
aggregateSig := v.key.SecretKey.Sign(attDataHash[:], domain)
attestation.AggregateSignature = aggregateSig.Marshal()
log.Infof("Attestation signature: %#x", attestation.AggregateSignature)
duration := time.Duration(slot*params.BeaconConfig().SecondsPerSlot+delay) * time.Second
timeToBroadcast := time.Unix(int64(v.genesisTime), 0).Add(duration)

View File

@@ -8,9 +8,12 @@ import (
"time"
"github.com/gogo/protobuf/proto"
ptypes "github.com/gogo/protobuf/types"
"github.com/golang/mock/gomock"
pbp2p "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
pb "github.com/prysmaticlabs/prysm/proto/beacon/rpc/v1"
"github.com/prysmaticlabs/prysm/shared/forkutils"
"github.com/prysmaticlabs/prysm/shared/hashutil"
"github.com/prysmaticlabs/prysm/shared/params"
"github.com/prysmaticlabs/prysm/shared/testutil"
logTest "github.com/sirupsen/logrus/hooks/test"
@@ -100,6 +103,14 @@ func TestAttestToBlockHead_AttestHeadRequestFailure(t *testing.T) {
LatestCrosslink: &pbp2p.Crosslink{},
JustifiedEpoch: 0,
}, nil)
m.beaconClient.EXPECT().ForkData(
gomock.Any(), // ctx
gomock.Eq(&ptypes.Empty{}),
).Return(&pbp2p.Fork{
Epoch: params.BeaconConfig().GenesisEpoch,
CurrentVersion: 0,
PreviousVersion: 0,
}, nil /*err*/)
m.attesterClient.EXPECT().AttestHead(
gomock.Any(), // ctx
gomock.AssignableToTypeOf(&pbp2p.Attestation{}),
@@ -129,6 +140,14 @@ func TestAttestToBlockHead_AttestsCorrectly(t *testing.T) {
Shard: 5,
Committee: committee,
}, nil)
m.beaconClient.EXPECT().ForkData(
gomock.Any(), // ctx
gomock.Eq(&ptypes.Empty{}),
).Return(&pbp2p.Fork{
Epoch: params.BeaconConfig().GenesisEpoch,
CurrentVersion: 0,
PreviousVersion: 0,
}, nil /*err*/)
m.attesterClient.EXPECT().AttestationDataAtSlot(
gomock.Any(), // ctx
gomock.AssignableToTypeOf(&pb.AttestationDataRequest{}),
@@ -154,6 +173,7 @@ func TestAttestToBlockHead_AttestsCorrectly(t *testing.T) {
// Validator index is at index 4 in the mocked committee defined in this test.
indexIntoCommittee := uint64(4)
aggregationBitfield[indexIntoCommittee/8] |= 1 << (indexIntoCommittee % 8)
expectedAttestation := &pbp2p.Attestation{
Data: &pbp2p.AttestationData{
Slot: 30,
@@ -167,8 +187,28 @@ func TestAttestToBlockHead_AttestsCorrectly(t *testing.T) {
},
CustodyBitfield: make([]byte, (len(committee)+7)/8),
AggregationBitfield: aggregationBitfield,
AggregateSignature: []byte("signed"),
}
// Retrieve the current fork data from the beacon node.
fork := &pbp2p.Fork{
Epoch: params.BeaconConfig().GenesisEpoch,
CurrentVersion: 0,
PreviousVersion: 0,
}
epoch := 30 / params.BeaconConfig().SlotsPerEpoch
attestationHash, err := hashutil.HashProto(&pbp2p.AttestationDataAndCustodyBit{
Data: expectedAttestation.Data,
CustodyBit: true,
})
if err != nil {
log.Fatalf("Could not hash attestation data: %v", err)
return
}
domain := forkutils.DomainVersion(fork, epoch, params.BeaconConfig().DomainAttestation)
expectedAttestation.AggregateSignature = validator.key.SecretKey.Sign(attestationHash[:], domain).Marshal()
if !proto.Equal(generatedAttestation, expectedAttestation) {
t.Errorf("Incorrectly attested head, wanted %v, received %v", expectedAttestation, generatedAttestation)
}
@@ -219,13 +259,24 @@ func TestAttestToBlockHead_DoesNotAttestBeforeDelay(t *testing.T) {
wg.Done()
})
m.beaconClient.EXPECT().ForkData(
gomock.Any(), // ctx
gomock.Eq(&ptypes.Empty{}),
).Return(&pbp2p.Fork{
Epoch: params.BeaconConfig().GenesisEpoch,
CurrentVersion: 0,
PreviousVersion: 0,
}, nil /*err*/).Times(1)
m.attesterClient.EXPECT().AttestHead(
gomock.Any(), // ctx
gomock.AssignableToTypeOf(&pbp2p.Attestation{}),
).Return(&pb.AttestResponse{}, nil /* error */).Times(0)
delay = 2
timer := time.NewTimer(time.Duration(1 * time.Second))
go validator.AttestToBlockHead(context.Background(), 0)
<-timer.C
}
func TestAttestToBlockHead_DoesAttestAfterDelay(t *testing.T) {
@@ -271,6 +322,15 @@ func TestAttestToBlockHead_DoesAttestAfterDelay(t *testing.T) {
wg.Done()
})
m.beaconClient.EXPECT().ForkData(
gomock.Any(), // ctx
gomock.Eq(&ptypes.Empty{}),
).Return(&pbp2p.Fork{
Epoch: params.BeaconConfig().GenesisEpoch,
CurrentVersion: 0,
PreviousVersion: 0,
}, nil /*err*/).Times(1)
m.attesterClient.EXPECT().AttestHead(
gomock.Any(), // ctx
gomock.Any(),

View File

@@ -82,7 +82,6 @@ func (v *validator) ProposeBlock(ctx context.Context, slot uint64) {
// Fetch pending attestations seen by the beacon node.
attResp, err := v.proposerClient.PendingAttestations(ctx, &pb.PendingAttestationsRequest{
FilterReadyForInclusion: true,
ProposalBlockSlot: slot,
})
if err != nil {
log.Errorf("Failed to fetch pending attestations from the beacon node: %v", err)