Use libp2p gossipsub upstream validator framework (#4318)

* add reject all pubsub validator to stop automatic propagation of messages
* gaz
* Merge branch 'master' of github.com:prysmaticlabs/prysm into pubsub-validator
* refactor p2p validator pipeline
* add sanity check
* Merge branch 'pubsub-validator' of github.com:prysmaticlabs/prysm into pubsub-validator
* fixed up test
* rem
* gaz
* Merge refs/heads/master into pubsub-validator
* fix from self test
* ensure validator data is set
* resolve todo
* Merge refs/heads/master into pubsub-validator
* gaz
* Merge refs/heads/master into pubsub-validator
* Merge branch 'pubsub-validator' of github.com:prysmaticlabs/prysm into pubsub-validator
* Merge refs/heads/master into pubsub-validator
* remove all of the 'from self' logic. filed https://github.com/libp2p/go-libp2p-pubsub/issues/250
* Merge branch 'pubsub-validator' of github.com:prysmaticlabs/prysm into pubsub-validator
* gaz
* update comment
* Merge refs/heads/master into pubsub-validator
* rename "VaidatorData"
* Merge branch 'pubsub-validator' of github.com:prysmaticlabs/prysm into pubsub-validator
* refactor
* one more bit of refactoring
* Update beacon-chain/sync/validate_beacon_attestation.go

Co-Authored-By: terence tsao <terence@prysmaticlabs.com>
* skip validation on self messages, add @nisdas feedback to increment failure counter
* Merge branch 'pubsub-validator' of github.com:prysmaticlabs/prysm into pubsub-validator
* remove flakey
This commit is contained in:
Preston Van Loon
2019-12-19 19:18:08 -08:00
committed by prylabs-bulldozer[bot]
parent 2e4908e7c4
commit 961dd21554
16 changed files with 571 additions and 691 deletions

View File

@@ -12,10 +12,10 @@ http_archive(
http_archive(
name = "bazel_gazelle",
sha256 = "7fc87f4170011201b1690326e8c16c5d802836e3a0d617d8f75c3af2b23180c4",
sha256 = "86c6d481b3f7aedc1d60c1c211c6f76da282ae197c3b3160f54bd3a8f847896f",
urls = [
"https://storage.googleapis.com/bazel-mirror/github.com/bazelbuild/bazel-gazelle/releases/download/0.18.2/bazel-gazelle-0.18.2.tar.gz",
"https://github.com/bazelbuild/bazel-gazelle/releases/download/0.18.2/bazel-gazelle-0.18.2.tar.gz",
"https://storage.googleapis.com/bazel-mirror/github.com/bazelbuild/bazel-gazelle/releases/download/v0.19.1/bazel-gazelle-v0.19.1.tar.gz",
"https://github.com/bazelbuild/bazel-gazelle/releases/download/v0.19.1/bazel-gazelle-v0.19.1.tar.gz",
],
)
@@ -746,8 +746,8 @@ go_repository(
name = "com_github_libp2p_go_libp2p_pubsub",
build_file_proto_mode = "disable_global",
importpath = "github.com/libp2p/go-libp2p-pubsub",
sum = "h1:tPKbkjAUI0xLGN3KKTKKy9TQEviVfrP++zJgH5Muke4=",
version = "v0.2.5",
sum = "h1:+Iz8zeI1KO6HX8cexU9g98cCGjae52Vujeg087SkuME=",
version = "v0.2.6-0.20191219233527-97846b574895",
)
go_repository(

View File

@@ -4,6 +4,7 @@ go_library(
name = "go_default_library",
srcs = [
"deadlines.go",
"decode_pubsub.go",
"doc.go",
"error.go",
"log.go",
@@ -45,16 +46,15 @@ go_library(
"//shared:go_default_library",
"//shared/bls:go_default_library",
"//shared/bytesutil:go_default_library",
"//shared/hashutil:go_default_library",
"//shared/params:go_default_library",
"//shared/roughtime:go_default_library",
"//shared/runutil:go_default_library",
"//shared/traceutil:go_default_library",
"@com_github_dgraph_io_ristretto//:go_default_library",
"@com_github_gogo_protobuf//proto:go_default_library",
"@com_github_libp2p_go_libp2p_core//:go_default_library",
"@com_github_libp2p_go_libp2p_core//network:go_default_library",
"@com_github_libp2p_go_libp2p_core//peer:go_default_library",
"@com_github_libp2p_go_libp2p_pubsub//:go_default_library",
"@com_github_pkg_errors//:go_default_library",
"@com_github_prometheus_client_golang//prometheus:go_default_library",
"@com_github_prometheus_client_golang//prometheus/promauto:go_default_library",
@@ -86,7 +86,6 @@ go_test(
"validate_voluntary_exit_test.go",
],
embed = [":go_default_library"],
flaky = True, # libp2p hosts are flaky upstream.
deps = [
"//beacon-chain/blockchain/testing:go_default_library",
"//beacon-chain/core/feed:go_default_library",
@@ -94,6 +93,7 @@ go_test(
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/state:go_default_library",
"//beacon-chain/db/testing:go_default_library",
"//beacon-chain/p2p:go_default_library",
"//beacon-chain/p2p/encoder:go_default_library",
"//beacon-chain/p2p/peers:go_default_library",
"//beacon-chain/p2p/testing:go_default_library",
@@ -107,7 +107,10 @@ go_test(
"@com_github_gogo_protobuf//proto:go_default_library",
"@com_github_libp2p_go_libp2p_core//:go_default_library",
"@com_github_libp2p_go_libp2p_core//network:go_default_library",
"@com_github_libp2p_go_libp2p_core//peer:go_default_library",
"@com_github_libp2p_go_libp2p_core//protocol:go_default_library",
"@com_github_libp2p_go_libp2p_pubsub//:go_default_library",
"@com_github_libp2p_go_libp2p_pubsub//pb:go_default_library",
"@com_github_prysmaticlabs_ethereumapis//eth/v1alpha1:go_default_library",
"@com_github_prysmaticlabs_go_ssz//:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",

View File

@@ -0,0 +1,28 @@
package sync
import (
"errors"
"fmt"
"strings"
"github.com/gogo/protobuf/proto"
pubsub "github.com/libp2p/go-libp2p-pubsub"
"github.com/prysmaticlabs/prysm/beacon-chain/p2p"
)
func (r *Service) decodePubsubMessage(msg *pubsub.Message) (proto.Message, error) {
if msg == nil || msg.TopicIDs == nil || len(msg.TopicIDs) == 0 {
return nil, errors.New("nil pubsub message")
}
topic := msg.TopicIDs[0]
topic = strings.TrimSuffix(topic, r.p2p.Encoding().ProtocolSuffix())
base, ok := p2p.GossipTopicMappings[topic]
if !ok {
return nil, fmt.Errorf("no message mapped for topic %s", topic)
}
m := proto.Clone(base)
if err := r.p2p.Encoding().Decode(msg.Data, m); err != nil {
return nil, err
}
return m, nil
}

View File

@@ -7,34 +7,30 @@ import (
"time"
"github.com/gogo/protobuf/proto"
"github.com/libp2p/go-libp2p-core/peer"
pubsub "github.com/libp2p/go-libp2p-pubsub"
"github.com/prysmaticlabs/prysm/beacon-chain/core/feed"
statefeed "github.com/prysmaticlabs/prysm/beacon-chain/core/feed/state"
"github.com/prysmaticlabs/prysm/beacon-chain/p2p"
"github.com/prysmaticlabs/prysm/shared/hashutil"
"github.com/prysmaticlabs/prysm/shared/roughtime"
"github.com/prysmaticlabs/prysm/shared/traceutil"
"go.opencensus.io/trace"
)
const oneYear = 365 * 24 * time.Hour
const pubsubMessageTimeout = 10 * time.Second
// prefix to add to keys, so that we can represent invalid objects
const invalid = "invalidObject"
// subHandler represents handler for a given subscription.
type subHandler func(context.Context, proto.Message) error
// validator should verify the contents of the message, propagate the message
// as expected, and return true or false to continue the message processing
// pipeline. FromSelf indicates whether or not this is a message received from our
// node in pubsub.
type validator func(ctx context.Context, msg proto.Message, broadcaster p2p.Broadcaster, fromSelf bool) (bool, error)
// noopValidator is a no-op that always returns true and does not propagate any
// message.
func noopValidator(_ context.Context, _ proto.Message, _ p2p.Broadcaster, _ bool) (bool, error) {
return true, nil
// noopValidator is a no-op that only decodes the message, but does not check its contents.
func (r *Service) noopValidator(ctx context.Context, _ peer.ID, msg *pubsub.Message) bool {
m, err := r.decodePubsubMessage(msg)
if err != nil {
log.WithError(err).Error("Failed to decode message")
return false
}
msg.ValidatorData = m
return true
}
// Register PubSub subscribers
@@ -94,7 +90,7 @@ func (r *Service) registerSubscribers() {
// subscribe to a given topic with a given validator and subscription handler.
// The base protobuf message is used to initialize new messages for decoding.
func (r *Service) subscribe(topic string, validate validator, handle subHandler) {
func (r *Service) subscribe(topic string, validator pubsub.Validator, handle subHandler) {
base := p2p.GossipTopicMappings[topic]
if base == nil {
panic(fmt.Sprintf("%s is not mapped to any message in GossipTopicMappings", topic))
@@ -103,6 +99,12 @@ func (r *Service) subscribe(topic string, validate validator, handle subHandler)
topic += r.p2p.Encoding().ProtocolSuffix()
log := log.WithField("topic", topic)
if err := r.p2p.PubSub().RegisterTopicValidator(wrapAndReportValidation(topic, validator)); err != nil {
// Configuring a topic validator would only return an error as a result of misconfiguration
// and is not a runtime concern.
panic(err)
}
sub, err := r.p2p.PubSub().Subscribe(topic)
if err != nil {
// Any error subscribing to a PubSub topic would be the result of a misconfiguration of
@@ -113,7 +115,7 @@ func (r *Service) subscribe(topic string, validate validator, handle subHandler)
// Pipeline decodes the incoming subscription data, runs the validation, and handles the
// message.
pipeline := func(data []byte, fromSelf bool) {
pipeline := func(msg *pubsub.Message) {
ctx, _ := context.WithTimeout(context.Background(), pubsubMessageTimeout)
ctx, span := trace.StartSpan(ctx, "sync.pubsub")
defer span.End()
@@ -127,39 +129,14 @@ func (r *Service) subscribe(topic string, validate validator, handle subHandler)
}()
span.AddAttributes(trace.StringAttribute("topic", topic))
span.AddAttributes(trace.BoolAttribute("fromSelf", fromSelf))
if data == nil {
log.Warn("Received nil message on pubsub")
if msg.ValidatorData == nil {
log.Error("Received nil message on pubsub")
messageFailedProcessingCounter.WithLabelValues(topic).Inc()
return
}
if span.IsRecordingEvents() {
id := hashutil.FastSum64(data)
messageLen := int64(len(data))
span.AddMessageReceiveEvent(int64(id), messageLen /*uncompressed*/, messageLen /*compressed*/)
}
msg := proto.Clone(base)
if err := r.p2p.Encoding().Decode(data, msg); err != nil {
traceutil.AnnotateError(span, err)
log.WithError(err).Warn("Failed to decode pubsub message")
return
}
valid, err := validate(ctx, msg, r.p2p, fromSelf)
if err != nil {
if !fromSelf {
log.WithError(err).Error("Message failed to verify")
messageFailedValidationCounter.WithLabelValues(topic).Inc()
}
return
}
if !valid {
return
}
if err := handle(ctx, msg); err != nil {
if err := handle(ctx, msg.ValidatorData.(proto.Message)); err != nil {
traceutil.AnnotateError(span, err)
log.WithError(err).Error("Failed to handle p2p pubsub")
messageFailedProcessingCounter.WithLabelValues(topic).Inc()
@@ -180,14 +157,24 @@ func (r *Service) subscribe(topic string, validate validator, handle subHandler)
messageReceivedBeforeChainStartCounter.WithLabelValues(topic + r.p2p.Encoding().ProtocolSuffix()).Inc()
continue
}
// Special validation occurs on messages received from ourselves.
fromSelf := msg.GetFrom() == r.p2p.PeerID()
messageReceivedCounter.WithLabelValues(topic + r.p2p.Encoding().ProtocolSuffix()).Inc()
go pipeline(msg.Data, fromSelf)
go pipeline(msg)
}
}
go messageLoop()
}
// Wrap the pubsub validator with a metric monitoring function. This function increments the
// appropriate counter if the particular message fails to validate.
func wrapAndReportValidation(topic string, v pubsub.Validator) (string, pubsub.Validator) {
return topic, func(ctx context.Context, pid peer.ID, msg *pubsub.Message) bool {
b := v(ctx, pid, msg)
if !b {
messageFailedValidationCounter.WithLabelValues(topic).Inc()
}
return b
}
}

View File

@@ -2,6 +2,7 @@ package sync
import (
"context"
"reflect"
"sync"
"testing"
"time"
@@ -11,7 +12,9 @@ import (
mockChain "github.com/prysmaticlabs/prysm/beacon-chain/blockchain/testing"
"github.com/prysmaticlabs/prysm/beacon-chain/core/feed"
statefeed "github.com/prysmaticlabs/prysm/beacon-chain/core/feed/state"
"github.com/prysmaticlabs/prysm/beacon-chain/p2p"
p2ptest "github.com/prysmaticlabs/prysm/beacon-chain/p2p/testing"
mockSync "github.com/prysmaticlabs/prysm/beacon-chain/sync/initial-sync/testing"
"github.com/prysmaticlabs/prysm/shared/bls"
"github.com/prysmaticlabs/prysm/shared/bytesutil"
"github.com/prysmaticlabs/prysm/shared/testutil"
@@ -22,12 +25,13 @@ func TestSubscribe_ReceivesValidMessage(t *testing.T) {
r := Service{
ctx: context.Background(),
p2p: p2p,
initialSync: &mockSync.Sync{IsSyncing: false},
}
topic := "/eth2/voluntary_exit"
var wg sync.WaitGroup
wg.Add(1)
r.subscribe(topic, noopValidator, func(_ context.Context, msg proto.Message) error {
r.subscribe(topic, r.noopValidator, func(_ context.Context, msg proto.Message) error {
m := msg.(*pb.VoluntaryExit)
if m.Epoch != 55 {
t.Errorf("Unexpected incoming message: %+v", m)
@@ -52,6 +56,7 @@ func TestSubscribe_WaitToSync(t *testing.T) {
p2p: p2p,
chain: chainService,
stateNotifier: chainService.StateNotifier(),
initialSync: &mockSync.Sync{IsSyncing: false},
}
topic := "/eth2/beacon_block"
@@ -86,22 +91,22 @@ func TestSubscribe_WaitToSync(t *testing.T) {
}
func TestSubscribe_HandlesPanic(t *testing.T) {
p2p := p2ptest.NewTestP2P(t)
p := p2ptest.NewTestP2P(t)
r := Service{
ctx: context.Background(),
p2p: p2p,
p2p: p,
}
topic := "/eth2/voluntary_exit"
topic := p2p.GossipTypeMapping[reflect.TypeOf(&pb.VoluntaryExit{})]
var wg sync.WaitGroup
wg.Add(1)
r.subscribe(topic, noopValidator, func(_ context.Context, msg proto.Message) error {
r.subscribe(topic, r.noopValidator, func(_ context.Context, msg proto.Message) error {
defer wg.Done()
panic("bad")
})
r.chainStarted = true
p2p.ReceivePubSub(topic, &pb.VoluntaryExit{Epoch: 55})
p.ReceivePubSub(topic, &pb.VoluntaryExit{Epoch: 55})
if testutil.WaitTimeout(&wg, time.Second) {
t.Fatal("Did not receive PubSub in 1 second")

View File

@@ -3,96 +3,66 @@ package sync
import (
"context"
"github.com/dgraph-io/ristretto"
"github.com/gogo/protobuf/proto"
"github.com/pkg/errors"
"github.com/libp2p/go-libp2p-core/peer"
pubsub "github.com/libp2p/go-libp2p-pubsub"
ethpb "github.com/prysmaticlabs/ethereumapis/eth/v1alpha1"
"github.com/prysmaticlabs/prysm/beacon-chain/core/blocks"
"github.com/prysmaticlabs/prysm/beacon-chain/core/state"
"github.com/prysmaticlabs/prysm/beacon-chain/p2p"
"github.com/prysmaticlabs/prysm/shared/hashutil"
"github.com/prysmaticlabs/prysm/shared/params"
"github.com/prysmaticlabs/prysm/shared/traceutil"
"go.opencensus.io/trace"
)
var seenAttSlashingCacheSize = int64(1 << 10)
// seenAttesterSlashings represents a cache of all the seen slashings
var seenAttesterSlashings, _ = ristretto.NewCache(&ristretto.Config{
NumCounters: seenAttSlashingCacheSize,
MaxCost: seenAttSlashingCacheSize,
BufferItems: 64,
})
func attSlashingCacheKey(slashing *ethpb.AttesterSlashing) (string, error) {
hash, err := hashutil.HashProto(slashing)
if err != nil {
return "", err
}
return string(hash[:]), nil
}
// Clients who receive an attester slashing on this topic MUST validate the conditions within VerifyAttesterSlashing before
// forwarding it across the network.
func (r *Service) validateAttesterSlashing(ctx context.Context, msg proto.Message, p p2p.Broadcaster, fromSelf bool) (bool, error) {
func (r *Service) validateAttesterSlashing(ctx context.Context, pid peer.ID, msg *pubsub.Message) bool {
// Validation runs on publish (not just subscriptions), so we should approve any message from
// ourselves.
if pid == r.p2p.PeerID() {
return true
}
// The head state will be too far away to validate any slashing.
if r.initialSync.Syncing() {
return false, nil
return false
}
ctx, span := trace.StartSpan(ctx, "sync.validateAttesterSlashing")
defer span.End()
slashing, ok := msg.(*ethpb.AttesterSlashing)
if !ok {
return false, nil
}
cacheKey, err := attSlashingCacheKey(slashing)
m, err := r.decodePubsubMessage(msg)
if err != nil {
return false, errors.Wrapf(err, "could not hash attestation slashing")
log.WithError(err).Error("Failed to decode message")
traceutil.AnnotateError(span, err)
return false
}
invalidKey := invalid + cacheKey
if _, ok := seenAttesterSlashings.Get(invalidKey); ok {
return false, errors.New("previously seen invalid attester slashing received")
}
if _, ok := seenAttesterSlashings.Get(cacheKey); ok {
return false, nil
slashing, ok := m.(*ethpb.AttesterSlashing)
if !ok {
return false
}
// Retrieve head state, advance state to the epoch slot used specified in slashing message.
s, err := r.chain.HeadState(ctx)
if err != nil {
return false, err
return false
}
slashSlot := slashing.Attestation_1.Data.Target.Epoch * params.BeaconConfig().SlotsPerEpoch
if s.Slot < slashSlot {
if ctx.Err() != nil {
return false, errors.Wrapf(ctx.Err(),
"Failed to advance state to slot %d to process attester slashing", slashSlot)
return false
}
var err error
s, err = state.ProcessSlots(ctx, s, slashSlot)
if err != nil {
return false, errors.Wrapf(err, "Failed to advance state to slot %d", slashSlot)
return false
}
}
if err := blocks.VerifyAttesterSlashing(ctx, s, slashing); err != nil {
seenAttesterSlashings.Set(invalidKey, true /*value*/, 1 /*cost*/)
return false, errors.Wrap(err, "Received invalid attester slashing")
}
seenAttesterSlashings.Set(cacheKey, true /*value*/, 1 /*cost*/)
if fromSelf {
return false, nil
return false
}
if err := p.Broadcast(ctx, slashing); err != nil {
log.WithError(err).Error("Failed to propagate attester slashing")
}
return true, nil
msg.ValidatorData = slashing // Used in downstream subscriber
return true
}

View File

@@ -1,15 +1,21 @@
package sync
import (
"bytes"
"context"
"math/rand"
"reflect"
"testing"
"time"
"github.com/libp2p/go-libp2p-core/peer"
pubsub "github.com/libp2p/go-libp2p-pubsub"
pubsubpb "github.com/libp2p/go-libp2p-pubsub/pb"
ethpb "github.com/prysmaticlabs/ethereumapis/eth/v1alpha1"
"github.com/prysmaticlabs/go-ssz"
mock "github.com/prysmaticlabs/prysm/beacon-chain/blockchain/testing"
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/beacon-chain/p2p"
p2ptest "github.com/prysmaticlabs/prysm/beacon-chain/p2p/testing"
mockSync "github.com/prysmaticlabs/prysm/beacon-chain/sync/initial-sync/testing"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
@@ -82,68 +88,43 @@ func setupValidAttesterSlashing(t *testing.T) (*ethpb.AttesterSlashing, *pb.Beac
}
func TestValidateAttesterSlashing_ValidSlashing(t *testing.T) {
p2p := p2ptest.NewTestP2P(t)
p := p2ptest.NewTestP2P(t)
ctx := context.Background()
slashing, s := setupValidAttesterSlashing(t)
r := &Service{
p2p: p2p,
p2p: p,
chain: &mock.ChainService{State: s},
initialSync: &mockSync.Sync{IsSyncing: false},
}
valid, err := r.validateAttesterSlashing(ctx, slashing, p2p, false /*fromSelf*/)
if err != nil {
t.Errorf("Failed validation: %v", err)
buf := new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, slashing); err != nil {
t.Fatal(err)
}
msg := &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(slashing)],
},
},
}
valid := r.validateAttesterSlashing(ctx, peer.ID("foobar"), msg)
if !valid {
t.Error("Failed Validation")
}
if !p2p.BroadcastCalled {
t.Error("Broadcast was not called")
}
time.Sleep(100 * time.Millisecond)
// A second message with the same information should not be valid for processing or
// propagation.
p2p.BroadcastCalled = false
valid, _ = r.validateAttesterSlashing(ctx, slashing, p2p, false /*fromSelf*/)
if valid {
t.Error("Passed validation when should have failed")
}
if p2p.BroadcastCalled {
t.Error("broadcast was called when it should not have been called")
}
}
func TestValidateAttesterSlashing_ValidSlashing_FromSelf(t *testing.T) {
p2p := p2ptest.NewTestP2P(t)
ctx := context.Background()
slashing, s := setupValidAttesterSlashing(t)
r := &Service{
p2p: p2p,
chain: &mock.ChainService{State: s},
initialSync: &mockSync.Sync{IsSyncing: false},
}
valid, _ := r.validateAttesterSlashing(ctx, slashing, p2p, true /*fromSelf*/)
if valid {
t.Error("Passed validation")
}
if p2p.BroadcastCalled {
t.Error("Broadcast was called")
if msg.ValidatorData == nil {
t.Error("Decoded message was not set on the message validator data")
}
}
func TestValidateAttesterSlashing_ContextTimeout(t *testing.T) {
p2p := p2ptest.NewTestP2P(t)
p := p2ptest.NewTestP2P(t)
slashing, state := setupValidAttesterSlashing(t)
slashing.Attestation_1.Data.Target.Epoch = 100000000
@@ -151,35 +132,57 @@ func TestValidateAttesterSlashing_ContextTimeout(t *testing.T) {
ctx, _ := context.WithTimeout(context.Background(), 100*time.Millisecond)
r := &Service{
p2p: p2p,
p2p: p,
chain: &mock.ChainService{State: state},
initialSync: &mockSync.Sync{IsSyncing: false},
}
valid, _ := r.validateAttesterSlashing(ctx, slashing, p2p, false /*fromSelf*/)
buf := new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, slashing); err != nil {
t.Fatal(err)
}
msg := &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(slashing)],
},
},
}
valid := r.validateAttesterSlashing(ctx, "", msg)
if valid {
t.Error("slashing from the far distant future should have timed out and returned false")
}
}
func TestValidateAttesterSlashing_Syncing(t *testing.T) {
p2p := p2ptest.NewTestP2P(t)
p := p2ptest.NewTestP2P(t)
ctx := context.Background()
slashing, s := setupValidAttesterSlashing(t)
r := &Service{
p2p: p2p,
p2p: p,
chain: &mock.ChainService{State: s},
initialSync: &mockSync.Sync{IsSyncing: true},
}
valid, _ := r.validateAttesterSlashing(ctx, slashing, p2p, false /*fromSelf*/)
buf := new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, slashing); err != nil {
t.Fatal(err)
}
msg := &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(slashing)],
},
},
}
valid := r.validateAttesterSlashing(ctx, "", msg)
if valid {
t.Error("Passed validation")
}
if p2p.BroadcastCalled {
t.Error("Broadcast was called")
}
}

View File

@@ -4,11 +4,10 @@ import (
"context"
"fmt"
"github.com/gogo/protobuf/proto"
"github.com/libp2p/go-libp2p-core/peer"
pubsub "github.com/libp2p/go-libp2p-pubsub"
"github.com/pkg/errors"
ethpb "github.com/prysmaticlabs/ethereumapis/eth/v1alpha1"
"github.com/prysmaticlabs/go-ssz"
"github.com/prysmaticlabs/prysm/beacon-chain/p2p"
"github.com/prysmaticlabs/prysm/shared/bytesutil"
"github.com/prysmaticlabs/prysm/shared/traceutil"
"github.com/sirupsen/logrus"
@@ -19,11 +18,17 @@ var errPointsToBlockNotInDatabase = errors.New("attestation points to a block wh
// validateBeaconAttestation validates that the block being voted for passes validation before forwarding to the
// network.
func (r *Service) validateBeaconAttestation(ctx context.Context, msg proto.Message, p p2p.Broadcaster, fromSelf bool) (bool, error) {
func (r *Service) validateBeaconAttestation(ctx context.Context, pid peer.ID, msg *pubsub.Message) bool {
// Validation runs on publish (not just subscriptions), so we should approve any message from
// ourselves.
if pid == r.p2p.PeerID() {
return true
}
// Attestation processing requires the target block to be present in the database, so we'll skip
// validating or processing attestations until fully synced.
if r.initialSync.Syncing() {
return false, nil
return false
}
ctx, span := trace.StartSpan(ctx, "sync.validateBeaconAttestation")
@@ -32,27 +37,23 @@ func (r *Service) validateBeaconAttestation(ctx context.Context, msg proto.Messa
// TODO(1332): Add blocks.VerifyAttestation before processing further.
// Discussion: https://github.com/ethereum/eth2.0-specs/issues/1332
att, ok := msg.(*ethpb.Attestation)
if !ok {
return false, nil
}
attRoot, err := ssz.HashTreeRoot(att)
m, err := r.decodePubsubMessage(msg)
if err != nil {
return false, errors.Wrap(err, "could not hash attestation")
log.WithError(err).Error("Failed to decode message")
traceutil.AnnotateError(span, err)
return false
}
att, ok := m.(*ethpb.Attestation)
if !ok {
traceutil.AnnotateError(span, errors.New("wrong proto message type"))
log.Error("Wrong proto message type")
return false
}
span.AddAttributes(
trace.StringAttribute("blockRoot", fmt.Sprintf("%#x", att.Data.BeaconBlockRoot)),
trace.StringAttribute("attRoot", fmt.Sprintf("%#x", attRoot)),
)
if _, ok := recentlySeenRoots.Get(string(attRoot[:])); ok {
return false, nil
}
recentlySeenRoots.Set(string(attRoot[:]), true /*value*/, 1 /*cost*/)
// Only valid blocks are saved in the database.
if !r.db.HasBlock(ctx, bytesutil.ToBytes32(att.Data.BeaconBlockRoot)) {
log.WithField(
@@ -60,27 +61,21 @@ func (r *Service) validateBeaconAttestation(ctx context.Context, msg proto.Messa
fmt.Sprintf("%#x", att.Data.BeaconBlockRoot),
).WithError(errPointsToBlockNotInDatabase).Debug("Ignored incoming attestation that points to a block which is not in the database")
traceutil.AnnotateError(span, errPointsToBlockNotInDatabase)
return false, nil
}
if fromSelf {
return false, nil
return false
}
finalizedEpoch := r.chain.FinalizedCheckpt().Epoch
attestationDataEpochOld := finalizedEpoch >= att.Data.Source.Epoch || finalizedEpoch >= att.Data.Target.Epoch
if finalizedEpoch != 0 && attestationDataEpochOld {
traceutil.AnnotateError(span, errors.New("wrong proto message type"))
log.WithFields(logrus.Fields{
"AttestationRoot": fmt.Sprintf("%#x", attRoot),
"TargetEpoch": att.Data.Target.Epoch,
"SourceEpoch": att.Data.Source.Epoch,
}).Debug("Rejecting old attestation")
return false, nil
return false
}
if err := p.Broadcast(ctx, msg); err != nil {
log.WithError(err).Error("Failed to broadcast message")
traceutil.AnnotateError(span, err)
}
return true, nil
msg.ValidatorData = att // Used in downstream subscriber
return true
}

View File

@@ -1,14 +1,18 @@
package sync
import (
"bytes"
"context"
"reflect"
"testing"
"time"
pubsub "github.com/libp2p/go-libp2p-pubsub"
pubsubpb "github.com/libp2p/go-libp2p-pubsub/pb"
ethpb "github.com/prysmaticlabs/ethereumapis/eth/v1alpha1"
"github.com/prysmaticlabs/go-ssz"
mockChain "github.com/prysmaticlabs/prysm/beacon-chain/blockchain/testing"
dbtest "github.com/prysmaticlabs/prysm/beacon-chain/db/testing"
"github.com/prysmaticlabs/prysm/beacon-chain/p2p"
p2ptest "github.com/prysmaticlabs/prysm/beacon-chain/p2p/testing"
mockSync "github.com/prysmaticlabs/prysm/beacon-chain/sync/initial-sync/testing"
"github.com/prysmaticlabs/prysm/shared/testutil"
@@ -21,6 +25,7 @@ func TestValidateBeaconAttestation_ValidBlock(t *testing.T) {
ctx := context.Background()
rs := &Service{
p2p: p,
db: db,
chain: &mockChain.ChainService{
FinalizedCheckPoint: &ethpb.Checkpoint{
@@ -54,27 +59,27 @@ func TestValidateBeaconAttestation_ValidBlock(t *testing.T) {
},
}
valid, err := rs.validateBeaconAttestation(ctx, msg, p, false /*fromSelf*/)
if err != nil {
t.Errorf("Beacon attestation failed validation: %v", err)
buf := new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, msg); err != nil {
t.Fatal(err)
}
m := &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(msg)],
},
},
}
valid := rs.validateBeaconAttestation(ctx, "", m)
if !valid {
t.Error("Beacon attestation failed validation")
}
if !p.BroadcastCalled {
t.Error("No message was broadcast")
}
time.Sleep(100 * time.Millisecond)
// It should ignore duplicate identical attestations.
p.BroadcastCalled = false
valid, _ = rs.validateBeaconAttestation(ctx, msg, p, false /*fromSelf*/)
if valid {
t.Error("Second identical beacon attestation passed validation when it should not have")
}
if p.BroadcastCalled {
t.Error("Second identcial beacon attestation was re-broadcast")
if m.ValidatorData == nil {
t.Error("Decoded message was not set on the message validator data")
}
}
@@ -85,6 +90,7 @@ func TestValidateBeaconAttestation_InvalidBlock(t *testing.T) {
ctx := context.Background()
rs := &Service{
p2p: p,
db: db,
initialSync: &mockSync.Sync{IsSyncing: false},
}
@@ -95,51 +101,21 @@ func TestValidateBeaconAttestation_InvalidBlock(t *testing.T) {
},
}
valid, _ := rs.validateBeaconAttestation(ctx, msg, p, false /*fromSelf*/)
if valid {
t.Error("Invalid beacon attestation passed validation when it should not have")
}
if p.BroadcastCalled {
t.Error("Invalid beacon attestation was broadcast")
}
}
func TestValidateBeaconAttestation_ValidBlock_FromSelf(t *testing.T) {
db := dbtest.SetupDB(t)
defer dbtest.TeardownDB(t, db)
p := p2ptest.NewTestP2P(t)
ctx := context.Background()
rs := &Service{
db: db,
initialSync: &mockSync.Sync{IsSyncing: false},
}
blk := &ethpb.BeaconBlock{
Slot: 55,
}
if err := db.SaveBlock(ctx, blk); err != nil {
buf := new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, msg); err != nil {
t.Fatal(err)
}
blockRoot, err := ssz.SigningRoot(blk)
if err != nil {
t.Fatal(err)
}
msg := &ethpb.Attestation{
Data: &ethpb.AttestationData{
BeaconBlockRoot: blockRoot[:],
m := &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(msg)],
},
},
}
valid, _ := rs.validateBeaconAttestation(ctx, msg, p, true /*fromSelf*/)
valid := rs.validateBeaconAttestation(ctx, "", m)
if valid {
t.Error("Beacon attestation passed validation")
}
if p.BroadcastCalled {
t.Error("Message was broadcast")
t.Error("Invalid beacon attestation passed validation when it should not have")
}
}
@@ -150,6 +126,7 @@ func TestValidateBeaconAttestation_Syncing(t *testing.T) {
ctx := context.Background()
rs := &Service{
p2p: p,
db: db,
initialSync: &mockSync.Sync{IsSyncing: true},
}
@@ -172,7 +149,19 @@ func TestValidateBeaconAttestation_Syncing(t *testing.T) {
},
}
valid, err := rs.validateBeaconAttestation(ctx, msg, p, false /*fromSelf*/)
buf := new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, msg); err != nil {
t.Fatal(err)
}
m := &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(msg)],
},
},
}
valid := rs.validateBeaconAttestation(ctx, "", m)
if valid {
t.Error("Beacon attestation passed validation")
}
@@ -185,6 +174,7 @@ func TestValidateBeaconAttestation_OldAttestation(t *testing.T) {
ctx := context.Background()
rs := &Service{
p2p: p,
db: db,
chain: &mockChain.ChainService{
FinalizedCheckPoint: &ethpb.Checkpoint{
@@ -218,17 +208,23 @@ func TestValidateBeaconAttestation_OldAttestation(t *testing.T) {
},
}
valid, err := rs.validateBeaconAttestation(ctx, msg, p, false /*fromSelf*/)
if err != nil {
t.Errorf("Beacon attestation failed validation: %v", err)
buf := new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, msg); err != nil {
t.Fatal(err)
}
m := &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(msg)],
},
},
}
valid := rs.validateBeaconAttestation(ctx, "", m)
if valid {
t.Error("Beacon attestation passed validation when it should have failed")
}
if p.BroadcastCalled {
t.Error("Message was broadcasted")
}
// source and target epoch same as finalized checkpoint
msg = &ethpb.Attestation{
Data: &ethpb.AttestationData{
@@ -242,17 +238,22 @@ func TestValidateBeaconAttestation_OldAttestation(t *testing.T) {
},
}
valid, err = rs.validateBeaconAttestation(ctx, msg, p, false /*fromSelf*/)
if err != nil {
t.Errorf("Beacon attestation failed validation: %v", err)
buf = new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, msg); err != nil {
t.Fatal(err)
}
m = &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(msg)],
},
},
}
valid = rs.validateBeaconAttestation(ctx, "", m)
if valid {
t.Error("Beacon attestation passed validation when it should have failed")
}
if p.BroadcastCalled {
t.Error("Message was broadcasted")
}
}
func TestValidateBeaconAttestation_FirstEpoch(t *testing.T) {
@@ -263,6 +264,7 @@ func TestValidateBeaconAttestation_FirstEpoch(t *testing.T) {
rs := &Service{
db: db,
p2p: p,
chain: &mockChain.ChainService{
FinalizedCheckPoint: &ethpb.Checkpoint{
Epoch: 0,
@@ -296,15 +298,20 @@ func TestValidateBeaconAttestation_FirstEpoch(t *testing.T) {
},
}
valid, err := rs.validateBeaconAttestation(ctx, msg, p, false /*fromSelf*/)
if err != nil {
t.Errorf("Beacon attestation failed validation: %v", err)
buf := new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, msg); err != nil {
t.Fatal(err)
}
m := &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(msg)],
},
},
}
valid := rs.validateBeaconAttestation(ctx, "", m)
if !valid {
t.Error("Beacon attestation did not pass validation")
}
if !p.BroadcastCalled {
t.Error("Message was not broadcasted")
}
}

View File

@@ -3,81 +3,75 @@ package sync
import (
"context"
"github.com/dgraph-io/ristretto"
"github.com/gogo/protobuf/proto"
"github.com/pkg/errors"
"github.com/libp2p/go-libp2p-core/peer"
pubsub "github.com/libp2p/go-libp2p-pubsub"
ethpb "github.com/prysmaticlabs/ethereumapis/eth/v1alpha1"
"github.com/prysmaticlabs/go-ssz"
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/beacon-chain/p2p"
"github.com/prysmaticlabs/prysm/shared/bls"
"github.com/prysmaticlabs/prysm/shared/traceutil"
"go.opencensus.io/trace"
)
var recentlySeenRootsSize = int64(1 << 16)
// recentlySeenBlockRoots cache with max size of ~2Mib ( including keys)
var recentlySeenRoots, _ = ristretto.NewCache(&ristretto.Config{
NumCounters: recentlySeenRootsSize,
MaxCost: recentlySeenRootsSize,
BufferItems: 64,
})
// validateBeaconBlockPubSub checks that the incoming block has a valid BLS signature.
// Blocks that have already been seen are ignored. If the BLS signature is any valid signature,
// this method rebroadcasts the message.
func (r *Service) validateBeaconBlockPubSub(ctx context.Context, msg proto.Message, p p2p.Broadcaster, fromSelf bool) (bool, error) {
func (r *Service) validateBeaconBlockPubSub(ctx context.Context, pid peer.ID, msg *pubsub.Message) bool {
// Validation runs on publish (not just subscriptions), so we should approve any message from
// ourselves.
if pid == r.p2p.PeerID() {
return true
}
// We should not attempt to process blocks until fully synced, but propagation is OK.
if r.initialSync.Syncing() {
return false
}
ctx, span := trace.StartSpan(ctx, "sync.validateBeaconBlockPubSub")
defer span.End()
m, err := r.decodePubsubMessage(msg)
if err != nil {
log.WithError(err).Error("Failed to decode message")
traceutil.AnnotateError(span, err)
return false
}
r.validateBlockLock.Lock()
defer r.validateBlockLock.Unlock()
m, ok := msg.(*ethpb.BeaconBlock)
blk, ok := m.(*ethpb.BeaconBlock)
if !ok {
return false, nil
return false
}
blockRoot, err := ssz.SigningRoot(m)
blockRoot, err := ssz.SigningRoot(blk)
if err != nil {
return false, errors.Wrap(err, "could not get signing root of beacon block")
return false
}
r.pendingQueueLock.RLock()
if r.seenPendingBlocks[blockRoot] {
r.pendingQueueLock.RUnlock()
return false, nil
return false
}
r.pendingQueueLock.RUnlock()
if _, ok := recentlySeenRoots.Get(string(blockRoot[:])); ok || r.db.HasBlock(ctx, blockRoot) {
return false, nil
}
recentlySeenRoots.Set(string(blockRoot[:]), true /*value*/, 1 /*cost*/)
if fromSelf {
return false, nil
if err := helpers.VerifySlotTime(uint64(r.chain.GenesisTime().Unix()), blk.Slot); err != nil {
log.WithError(err).WithField("blockSlot", blk.Slot).Warn("Rejecting incoming block.")
return false
}
if err := helpers.VerifySlotTime(uint64(r.chain.GenesisTime().Unix()), m.Slot); err != nil {
log.WithError(err).WithField("blockSlot", m.Slot).Warn("Rejecting incoming block.")
return false, err
}
if r.chain.FinalizedCheckpt().Epoch > helpers.SlotToEpoch(m.Slot) {
if r.chain.FinalizedCheckpt().Epoch > helpers.SlotToEpoch(blk.Slot) {
log.Debug("Block older than finalized checkpoint received,rejecting it")
return false, nil
return false
}
_, err = bls.SignatureFromBytes(m.Signature)
if err == nil {
p.Broadcast(ctx, m)
if _, err = bls.SignatureFromBytes(blk.Signature); err != nil {
return false
}
// We should not attempt to process blocks until fully synced, but propagation is OK.
if r.initialSync.Syncing() {
return false, nil
}
return err == nil, err
msg.ValidatorData = blk // Used in downstream subscriber
return true
}

View File

@@ -1,14 +1,18 @@
package sync
import (
"bytes"
"context"
"strings"
"reflect"
"testing"
"time"
pubsub "github.com/libp2p/go-libp2p-pubsub"
pubsubpb "github.com/libp2p/go-libp2p-pubsub/pb"
ethpb "github.com/prysmaticlabs/ethereumapis/eth/v1alpha1"
mock "github.com/prysmaticlabs/prysm/beacon-chain/blockchain/testing"
dbtest "github.com/prysmaticlabs/prysm/beacon-chain/db/testing"
"github.com/prysmaticlabs/prysm/beacon-chain/p2p"
p2ptest "github.com/prysmaticlabs/prysm/beacon-chain/p2p/testing"
mockSync "github.com/prysmaticlabs/prysm/beacon-chain/sync/initial-sync/testing"
"github.com/prysmaticlabs/prysm/shared/bls"
@@ -21,6 +25,7 @@ import (
// tests in this package.
func TestValidateBeaconBlockPubSub_InvalidSignature(t *testing.T) {
ctx := context.Background()
db := dbtest.SetupDB(t)
defer dbtest.TeardownDB(t, db)
msg := &ethpb.BeaconBlock{
@@ -29,38 +34,43 @@ func TestValidateBeaconBlockPubSub_InvalidSignature(t *testing.T) {
Signature: []byte("fake"),
}
mockBroadcaster := &p2ptest.MockBroadcaster{}
p := p2ptest.NewTestP2P(t)
r := &Service{
db: db,
p2p: p,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: &mock.ChainService{Genesis: time.Now(),
FinalizedCheckPoint: &ethpb.Checkpoint{
Epoch: 0,
}},
}
result, err := r.validateBeaconBlockPubSub(
context.Background(),
msg,
mockBroadcaster,
false, // fromSelf
)
if err == nil {
t.Error("expected an error")
buf := new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, msg); err != nil {
t.Fatal(err)
}
m := &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(msg)],
},
},
}
result := r.validateBeaconBlockPubSub(ctx, "", m)
if result {
t.Error("Expected false result, got true")
}
if mockBroadcaster.BroadcastCalled {
t.Error("Broadcast was called when it should not have been called")
}
}
func TestValidateBeaconBlockPubSub_BlockAlreadyPresentInDB(t *testing.T) {
db := dbtest.SetupDB(t)
defer dbtest.TeardownDB(t, db)
ctx := context.Background()
p := p2ptest.NewTestP2P(t)
msg := &ethpb.BeaconBlock{
Slot: 100,
ParentRoot: testutil.Random32Bytes(t),
@@ -69,89 +79,38 @@ func TestValidateBeaconBlockPubSub_BlockAlreadyPresentInDB(t *testing.T) {
t.Fatal(err)
}
mockBroadcaster := &p2ptest.MockBroadcaster{}
r := &Service{
db: db,
p2p: p,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: &mock.ChainService{Genesis: time.Now()},
}
result, _ := r.validateBeaconBlockPubSub(
context.Background(),
msg,
mockBroadcaster,
false, // fromSelf
)
if result {
t.Error("Expected false result, got true")
}
if mockBroadcaster.BroadcastCalled {
t.Error("Broadcast was called when it should not have been called")
}
}
func TestValidateBeaconBlockPubSub_BlockAlreadyPresentInCache(t *testing.T) {
db := dbtest.SetupDB(t)
defer dbtest.TeardownDB(t, db)
b := []byte("sk")
b32 := bytesutil.ToBytes32(b)
sk, err := bls.SecretKeyFromBytes(b32[:])
if err != nil {
buf := new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, msg); err != nil {
t.Fatal(err)
}
msg := &ethpb.BeaconBlock{
ParentRoot: testutil.Random32Bytes(t),
Signature: sk.Sign([]byte("data"), 0).Marshal(),
}
mockBroadcaster := &p2ptest.MockBroadcaster{}
r := &Service{
db: db,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: &mock.ChainService{Genesis: time.Now(),
FinalizedCheckPoint: &ethpb.Checkpoint{
Epoch: 0,
}},
m := &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(msg)],
},
},
}
result, err := r.validateBeaconBlockPubSub(
context.Background(),
msg,
mockBroadcaster,
false, // fromSelf
)
result := r.validateBeaconBlockPubSub(ctx, "", m)
if err != nil {
t.Errorf("Expected no error, got: %v", err)
}
if !result {
t.Error("Expected true result, got false")
}
if !mockBroadcaster.BroadcastCalled {
t.Error("Broadcast was not called when it should have been called")
}
// The value should be cached now so the second request will fail.
mockBroadcaster.BroadcastCalled = false
result, _ = r.validateBeaconBlockPubSub(
context.Background(),
msg,
mockBroadcaster,
false, // fromSelf
)
if result {
t.Error("Expected false result, got true")
}
if mockBroadcaster.BroadcastCalled {
t.Error("Broadcast was called when it should not have been called")
}
}
func TestValidateBeaconBlockPubSub_ValidSignature(t *testing.T) {
db := dbtest.SetupDB(t)
defer dbtest.TeardownDB(t, db)
p := p2ptest.NewTestP2P(t)
ctx := context.Background()
b := []byte("sk")
b32 := bytesutil.ToBytes32(b)
sk, err := bls.SecretKeyFromBytes(b32[:])
@@ -163,71 +122,43 @@ func TestValidateBeaconBlockPubSub_ValidSignature(t *testing.T) {
Signature: sk.Sign([]byte("data"), 0).Marshal(),
}
mockBroadcaster := &p2ptest.MockBroadcaster{}
r := &Service{
db: db,
p2p: p,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: &mock.ChainService{Genesis: time.Now(),
FinalizedCheckPoint: &ethpb.Checkpoint{
Epoch: 0,
}},
}
result, _ := r.validateBeaconBlockPubSub(
context.Background(),
msg,
mockBroadcaster,
false, // fromSelf
)
buf := new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, msg); err != nil {
t.Fatal(err)
}
m := &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(msg)],
},
},
}
result := r.validateBeaconBlockPubSub(ctx, "", m)
if !result {
t.Error("Expected true result, got false")
}
if !mockBroadcaster.BroadcastCalled {
t.Error("Broadcast was not called when it should have been called")
}
}
func TestValidateBeaconBlockPubSub_ValidSignature_FromSelf(t *testing.T) {
db := dbtest.SetupDB(t)
defer dbtest.TeardownDB(t, db)
b := []byte("sk")
b32 := bytesutil.ToBytes32(b)
sk, err := bls.SecretKeyFromBytes(b32[:])
if err != nil {
t.Fatal(err)
}
msg := &ethpb.BeaconBlock{
Slot: 1,
ParentRoot: testutil.Random32Bytes(t),
Signature: sk.Sign([]byte("data"), 0).Marshal(),
}
mockBroadcaster := &p2ptest.MockBroadcaster{}
r := &Service{
db: db,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: &mock.ChainService{Genesis: time.Now()},
}
result, _ := r.validateBeaconBlockPubSub(
context.Background(),
msg,
mockBroadcaster,
true, // fromSelf
)
if result {
t.Error("Expected false result, got true")
}
if mockBroadcaster.BroadcastCalled {
t.Error("Broadcast was called when it should not have been called")
if m.ValidatorData == nil {
t.Error("Decoded message was not set on the message validator data")
}
}
func TestValidateBeaconBlockPubSub_Syncing(t *testing.T) {
db := dbtest.SetupDB(t)
defer dbtest.TeardownDB(t, db)
p := p2ptest.NewTestP2P(t)
ctx := context.Background()
b := []byte("sk")
b32 := bytesutil.ToBytes32(b)
sk, err := bls.SecretKeyFromBytes(b32[:])
@@ -239,10 +170,9 @@ func TestValidateBeaconBlockPubSub_Syncing(t *testing.T) {
Signature: sk.Sign([]byte("data"), 0).Marshal(),
}
mockBroadcaster := &p2ptest.MockBroadcaster{}
r := &Service{
db: db,
p2p: p,
initialSync: &mockSync.Sync{IsSyncing: true},
chain: &mock.ChainService{
Genesis: time.Now(),
@@ -250,24 +180,30 @@ func TestValidateBeaconBlockPubSub_Syncing(t *testing.T) {
Epoch: 0,
}},
}
result, _ := r.validateBeaconBlockPubSub(
context.Background(),
msg,
mockBroadcaster,
false, // fromSelf
)
buf := new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, msg); err != nil {
t.Fatal(err)
}
m := &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(msg)],
},
},
}
result := r.validateBeaconBlockPubSub(ctx, "", m)
if result {
t.Error("Expected false result, got true")
}
if !mockBroadcaster.BroadcastCalled {
t.Error("Broadcast was not called when it should have been called")
}
}
func TestValidateBeaconBlockPubSub_RejectBlocksFromFuture(t *testing.T) {
db := dbtest.SetupDB(t)
defer dbtest.TeardownDB(t, db)
p := p2ptest.NewTestP2P(t)
ctx := context.Background()
b := []byte("sk")
b32 := bytesutil.ToBytes32(b)
sk, err := bls.SecretKeyFromBytes(b32[:])
@@ -280,30 +216,29 @@ func TestValidateBeaconBlockPubSub_RejectBlocksFromFuture(t *testing.T) {
Slot: 1000,
}
mockBroadcaster := &p2ptest.MockBroadcaster{}
r := &Service{
p2p: p,
db: db,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: &mock.ChainService{Genesis: time.Now()},
}
result, err := r.validateBeaconBlockPubSub(
context.Background(),
msg,
mockBroadcaster,
false, // fromSelf
)
if err == nil || !strings.Contains(err.Error(), "could not process slot from the future") {
t.Errorf("Err = %v, wanted substring %s", err, "could not process slot from the future")
buf := new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, msg); err != nil {
t.Fatal(err)
}
m := &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(msg)],
},
},
}
result := r.validateBeaconBlockPubSub(ctx, "", m)
if result {
t.Error("Expected false result, got true")
}
if mockBroadcaster.BroadcastCalled {
t.Error("Broadcast was called when it should not have been called")
}
}
func TestValidateBeaconBlockPubSub_RejectBlocksFromThePast(t *testing.T) {
@@ -311,6 +246,8 @@ func TestValidateBeaconBlockPubSub_RejectBlocksFromThePast(t *testing.T) {
defer dbtest.TeardownDB(t, db)
b := []byte("sk")
b32 := bytesutil.ToBytes32(b)
p := p2ptest.NewTestP2P(t)
ctx := context.Background()
sk, err := bls.SecretKeyFromBytes(b32[:])
if err != nil {
t.Fatal(err)
@@ -321,10 +258,10 @@ func TestValidateBeaconBlockPubSub_RejectBlocksFromThePast(t *testing.T) {
Slot: 10,
}
mockBroadcaster := &p2ptest.MockBroadcaster{}
genesisTime := time.Now()
r := &Service{
db: db,
p2p: p,
initialSync: &mockSync.Sync{IsSyncing: false},
chain: &mock.ChainService{
Genesis: time.Unix(genesisTime.Unix()-1000, 0),
@@ -332,16 +269,22 @@ func TestValidateBeaconBlockPubSub_RejectBlocksFromThePast(t *testing.T) {
Epoch: 1,
}},
}
result, err := r.validateBeaconBlockPubSub(
context.Background(),
msg,
mockBroadcaster,
false, // fromSelf
)
buf := new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, msg); err != nil {
t.Fatal(err)
}
m := &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(msg)],
},
},
}
result := r.validateBeaconBlockPubSub(ctx, "", m)
if result {
t.Error("Expected false result, got true")
}
if mockBroadcaster.BroadcastCalled {
t.Error("Broadcast was called when it should not have been called")
}
}

View File

@@ -3,93 +3,65 @@ package sync
import (
"context"
"github.com/dgraph-io/ristretto"
"github.com/gogo/protobuf/proto"
"github.com/pkg/errors"
"github.com/libp2p/go-libp2p-core/peer"
pubsub "github.com/libp2p/go-libp2p-pubsub"
ethpb "github.com/prysmaticlabs/ethereumapis/eth/v1alpha1"
"github.com/prysmaticlabs/prysm/beacon-chain/core/blocks"
"github.com/prysmaticlabs/prysm/beacon-chain/core/state"
"github.com/prysmaticlabs/prysm/beacon-chain/p2p"
"github.com/prysmaticlabs/prysm/shared/hashutil"
"github.com/prysmaticlabs/prysm/shared/traceutil"
"go.opencensus.io/trace"
)
var seenProposerSlashingCacheSize = int64(1 << 10)
// seenProposerSlashings represents a cache of all the seen slashings
var seenProposerSlashings, _ = ristretto.NewCache(&ristretto.Config{
NumCounters: seenProposerSlashingCacheSize,
MaxCost: seenProposerSlashingCacheSize,
BufferItems: 64,
})
func propSlashingCacheKey(slashing *ethpb.ProposerSlashing) (string, error) {
hash, err := hashutil.HashProto(slashing)
if err != nil {
return "", err
}
return string(hash[:]), nil
}
// Clients who receive a proposer slashing on this topic MUST validate the conditions within VerifyProposerSlashing before
// forwarding it across the network.
func (r *Service) validateProposerSlashing(ctx context.Context, msg proto.Message, p p2p.Broadcaster, fromSelf bool) (bool, error) {
func (r *Service) validateProposerSlashing(ctx context.Context, pid peer.ID, msg *pubsub.Message) bool {
// Validation runs on publish (not just subscriptions), so we should approve any message from
// ourselves.
if pid == r.p2p.PeerID() {
return true
}
// The head state will be too far away to validate any slashing.
if r.initialSync.Syncing() {
return false, nil
return false
}
ctx, span := trace.StartSpan(ctx, "sync.validateProposerSlashing")
defer span.End()
slashing, ok := msg.(*ethpb.ProposerSlashing)
if !ok {
return false, nil
}
cacheKey, err := propSlashingCacheKey(slashing)
m, err := r.decodePubsubMessage(msg)
if err != nil {
return false, errors.Wrapf(err, "could not hash proposer slashing")
log.WithError(err).Error("Failed to decode message")
traceutil.AnnotateError(span, err)
return false
}
invalidKey := invalid + cacheKey
if _, ok := seenProposerSlashings.Get(invalidKey); ok {
return false, errors.New("previously seen invalid proposer slashing received")
}
if _, ok := seenProposerSlashings.Get(cacheKey); ok {
return false, nil
slashing, ok := m.(*ethpb.ProposerSlashing)
if !ok {
return false
}
// Retrieve head state, advance state to the epoch slot used specified in slashing message.
s, err := r.chain.HeadState(ctx)
if err != nil {
return false, errors.Wrap(err, "Could not get head state")
return false
}
slashSlot := slashing.Header_1.Slot
if s.Slot < slashSlot {
if ctx.Err() != nil {
return false, errors.Wrapf(ctx.Err(),
"Failed to advance state to slot %d to process proposer slashing", slashSlot)
return false
}
var err error
s, err = state.ProcessSlots(ctx, s, slashSlot)
if err != nil {
return false, errors.Wrapf(err, "Failed to advance state to slot %d", slashSlot)
return false
}
}
if err := blocks.VerifyProposerSlashing(s, slashing); err != nil {
seenProposerSlashings.Set(invalidKey, true /*value*/, 1 /*cost*/)
return false, errors.Wrap(err, "Received invalid proposer slashing")
}
seenProposerSlashings.Set(cacheKey, true /*value*/, 1 /*cost*/)
if fromSelf {
return false, nil
return false
}
if err := p.Broadcast(ctx, slashing); err != nil {
log.WithError(err).Error("Failed to propagate proposer slashing")
}
return true, nil
msg.ValidatorData = slashing // Used in downstream subscriber
return true
}

View File

@@ -1,15 +1,20 @@
package sync
import (
"bytes"
"context"
"crypto/rand"
"reflect"
"testing"
"time"
pubsub "github.com/libp2p/go-libp2p-pubsub"
pubsubpb "github.com/libp2p/go-libp2p-pubsub/pb"
ethpb "github.com/prysmaticlabs/ethereumapis/eth/v1alpha1"
"github.com/prysmaticlabs/go-ssz"
mock "github.com/prysmaticlabs/prysm/beacon-chain/blockchain/testing"
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/beacon-chain/p2p"
p2ptest "github.com/prysmaticlabs/prysm/beacon-chain/p2p/testing"
mockSync "github.com/prysmaticlabs/prysm/beacon-chain/sync/initial-sync/testing"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
@@ -58,9 +63,13 @@ func setupValidProposerSlashing(t *testing.T) (*ethpb.ProposerSlashing, *pb.Beac
)
privKey := bls.RandKey()
someRoot := [32]byte{1, 2, 3}
someRoot2 := [32]byte{4, 5, 6}
header1 := &ethpb.BeaconBlockHeader{
Slot: 0,
StateRoot: []byte("A"),
ParentRoot: someRoot[:],
StateRoot: someRoot[:],
BodyRoot: someRoot[:],
}
signingRoot, err := ssz.SigningRoot(header1)
if err != nil {
@@ -70,7 +79,9 @@ func setupValidProposerSlashing(t *testing.T) (*ethpb.ProposerSlashing, *pb.Beac
header2 := &ethpb.BeaconBlockHeader{
Slot: 0,
StateRoot: []byte("B"),
ParentRoot: someRoot2[:],
StateRoot: someRoot2[:],
BodyRoot: someRoot2[:],
}
signingRoot, err = ssz.SigningRoot(header2)
if err != nil {
@@ -95,67 +106,42 @@ func setupValidProposerSlashing(t *testing.T) (*ethpb.ProposerSlashing, *pb.Beac
}
func TestValidateProposerSlashing_ValidSlashing(t *testing.T) {
p2p := p2ptest.NewTestP2P(t)
p := p2ptest.NewTestP2P(t)
ctx := context.Background()
slashing, s := setupValidProposerSlashing(t)
r := &Service{
p2p: p2p,
p2p: p,
chain: &mock.ChainService{State: s},
initialSync: &mockSync.Sync{IsSyncing: false},
}
valid, err := r.validateProposerSlashing(ctx, slashing, p2p, false /*fromSelf*/)
if err != nil {
t.Errorf("Failed validation: %v", err)
buf := new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, slashing); err != nil {
t.Fatal(err)
}
m := &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(slashing)],
},
},
}
valid := r.validateProposerSlashing(ctx, "", m)
if !valid {
t.Error("Failed validation")
}
if !p2p.BroadcastCalled {
t.Error("Broadcast was not called")
}
time.Sleep(100 * time.Millisecond)
// A second message with the same information should not be valid for processing or
// propagation.
p2p.BroadcastCalled = false
valid, _ = r.validateProposerSlashing(ctx, slashing, p2p, false /*fromSelf*/)
if valid {
t.Error("Passed validation when should have failed")
}
if p2p.BroadcastCalled {
t.Error("broadcast was called when it should not have been called")
}
}
func TestValidateProposerSlashing_ValidSlashing_FromSelf(t *testing.T) {
p2p := p2ptest.NewTestP2P(t)
ctx := context.Background()
slashing, s := setupValidProposerSlashing(t)
r := &Service{
p2p: p2p,
chain: &mock.ChainService{State: s},
initialSync: &mockSync.Sync{IsSyncing: false},
}
valid, _ := r.validateProposerSlashing(ctx, slashing, p2p, true /*fromSelf*/)
if valid {
t.Error("Did not fail validation")
}
if p2p.BroadcastCalled {
t.Error("Broadcast was called")
if m.ValidatorData == nil {
t.Error("Decoded message was not set on the message validator data")
}
}
func TestValidateProposerSlashing_ContextTimeout(t *testing.T) {
p2p := p2ptest.NewTestP2P(t)
p := p2ptest.NewTestP2P(t)
slashing, state := setupValidProposerSlashing(t)
slashing.Header_1.Slot = 100000000
@@ -163,35 +149,56 @@ func TestValidateProposerSlashing_ContextTimeout(t *testing.T) {
ctx, _ := context.WithTimeout(context.Background(), 100*time.Millisecond)
r := &Service{
p2p: p2p,
p2p: p,
chain: &mock.ChainService{State: state},
initialSync: &mockSync.Sync{IsSyncing: false},
}
valid, _ := r.validateProposerSlashing(ctx, slashing, p2p, false /*fromSelf*/)
buf := new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, slashing); err != nil {
t.Fatal(err)
}
m := &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(slashing)],
},
},
}
valid := r.validateProposerSlashing(ctx, "", m)
if valid {
t.Error("slashing from the far distant future should have timed out and returned false")
}
}
func TestValidateProposerSlashing_Syncing(t *testing.T) {
p2p := p2ptest.NewTestP2P(t)
p := p2ptest.NewTestP2P(t)
ctx := context.Background()
slashing, s := setupValidProposerSlashing(t)
r := &Service{
p2p: p2p,
p2p: p,
chain: &mock.ChainService{State: s},
initialSync: &mockSync.Sync{IsSyncing: true},
}
valid, _ := r.validateProposerSlashing(ctx, slashing, p2p, false /*fromSelf*/)
buf := new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, slashing); err != nil {
t.Fatal(err)
}
m := &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(slashing)],
},
},
}
valid := r.validateProposerSlashing(ctx, "", m)
if valid {
t.Error("Did not fail validation")
}
if p2p.BroadcastCalled {
t.Error("Broadcast was called")
}
}

View File

@@ -2,61 +2,50 @@ package sync
import (
"context"
"fmt"
"github.com/dgraph-io/ristretto"
"github.com/gogo/protobuf/proto"
"github.com/pkg/errors"
"github.com/libp2p/go-libp2p-core/peer"
pubsub "github.com/libp2p/go-libp2p-pubsub"
ethpb "github.com/prysmaticlabs/ethereumapis/eth/v1alpha1"
"github.com/prysmaticlabs/prysm/beacon-chain/core/blocks"
"github.com/prysmaticlabs/prysm/beacon-chain/core/state"
"github.com/prysmaticlabs/prysm/beacon-chain/p2p"
"github.com/prysmaticlabs/prysm/shared/params"
"github.com/prysmaticlabs/prysm/shared/traceutil"
"go.opencensus.io/trace"
)
var seenExitsCacheSize = int64(1 << 10)
// seenExits tracks exits we've already seen to prevent feedback loop.
var seenExits, _ = ristretto.NewCache(&ristretto.Config{
NumCounters: seenExitsCacheSize,
MaxCost: seenExitsCacheSize,
BufferItems: 64,
})
func exitCacheKey(exit *ethpb.VoluntaryExit) string {
return fmt.Sprintf("%d-%d", exit.Epoch, exit.ValidatorIndex)
}
// Clients who receive a voluntary exit on this topic MUST validate the conditions within process_voluntary_exit before
// forwarding it across the network.
func (r *Service) validateVoluntaryExit(ctx context.Context, msg proto.Message, p p2p.Broadcaster, fromSelf bool) (bool, error) {
func (r *Service) validateVoluntaryExit(ctx context.Context, pid peer.ID, msg *pubsub.Message) bool {
// Validation runs on publish (not just subscriptions), so we should approve any message from
// ourselves.
if pid == r.p2p.PeerID() {
return true
}
// The head state will be too far away to validate any voluntary exit.
if r.initialSync.Syncing() {
return false, nil
return false
}
ctx, span := trace.StartSpan(ctx, "sync.validateVoluntaryExit")
defer span.End()
exit, ok := msg.(*ethpb.VoluntaryExit)
if !ok {
return false, nil
m, err := r.decodePubsubMessage(msg)
if err != nil {
log.WithError(err).Error("Failed to decode message")
traceutil.AnnotateError(span, err)
return false
}
cacheKey := exitCacheKey(exit)
invalidKey := invalid + cacheKey
if _, ok := seenExits.Get(invalidKey); ok {
return false, errors.New("previously seen invalid validator exit received")
}
if _, ok := seenExits.Get(cacheKey); ok {
return false, nil
exit, ok := m.(*ethpb.VoluntaryExit)
if !ok {
return false
}
// Retrieve head state, advance state to the epoch slot used specified in exit message.
s, err := r.chain.HeadState(ctx)
if err != nil {
return false, err
return false
}
exitedEpochSlot := exit.Epoch * params.BeaconConfig().SlotsPerEpoch
@@ -64,22 +53,15 @@ func (r *Service) validateVoluntaryExit(ctx context.Context, msg proto.Message,
var err error
s, err = state.ProcessSlots(ctx, s, exitedEpochSlot)
if err != nil {
return false, errors.Wrapf(err, "Failed to advance state to slot %d", exitedEpochSlot)
return false
}
}
if err := blocks.VerifyExit(s, exit); err != nil {
seenExits.Set(invalidKey, true /*value*/, 1 /*cost*/)
return false, errors.Wrap(err, "Received invalid validator exit")
}
seenExits.Set(cacheKey, true /*value*/, 1 /*cost*/)
if fromSelf {
return false, nil
return false
}
if err := p.Broadcast(ctx, exit); err != nil {
log.WithError(err).Error("Failed to propagate voluntary exit")
}
return true, nil
msg.ValidatorData = exit // Used in downstream subscriber
return true
}

View File

@@ -1,15 +1,19 @@
package sync
import (
"bytes"
"context"
"crypto/rand"
"reflect"
"testing"
"time"
pubsub "github.com/libp2p/go-libp2p-pubsub"
pubsubpb "github.com/libp2p/go-libp2p-pubsub/pb"
ethpb "github.com/prysmaticlabs/ethereumapis/eth/v1alpha1"
"github.com/prysmaticlabs/go-ssz"
mock "github.com/prysmaticlabs/prysm/beacon-chain/blockchain/testing"
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/beacon-chain/p2p"
p2ptest "github.com/prysmaticlabs/prysm/beacon-chain/p2p/testing"
mockSync "github.com/prysmaticlabs/prysm/beacon-chain/sync/initial-sync/testing"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
@@ -57,89 +61,68 @@ func setupValidExit(t *testing.T) (*ethpb.VoluntaryExit, *pb.BeaconState) {
}
func TestValidateVoluntaryExit_ValidExit(t *testing.T) {
p2p := p2ptest.NewTestP2P(t)
p := p2ptest.NewTestP2P(t)
ctx := context.Background()
exit, s := setupValidExit(t)
r := &Service{
p2p: p2p,
p2p: p,
chain: &mock.ChainService{
State: s,
},
initialSync: &mockSync.Sync{IsSyncing: false},
}
valid, err := r.validateVoluntaryExit(ctx, exit, p2p, false /*fromSelf*/)
if err != nil {
t.Errorf("Failed validation: %v", err)
buf := new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, exit); err != nil {
t.Fatal(err)
}
m := &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(exit)],
},
},
}
valid := r.validateVoluntaryExit(ctx, "", m)
if !valid {
t.Error("Failed validation")
}
if !p2p.BroadcastCalled {
t.Error("Broadcast was not called")
}
time.Sleep(100 * time.Millisecond)
// A second message with the same information should not be valid for processing or
// propagation.
p2p.BroadcastCalled = false
valid, _ = r.validateVoluntaryExit(ctx, exit, p2p, false /*fromSelf*/)
if valid {
t.Error("Passed validation when should have failed")
}
if p2p.BroadcastCalled {
t.Error("broadcast was called when it should not have been called")
}
}
func TestValidateVoluntaryExit_ValidExit_FromSelf(t *testing.T) {
p2p := p2ptest.NewTestP2P(t)
ctx := context.Background()
exit, s := setupValidExit(t)
r := &Service{
p2p: p2p,
chain: &mock.ChainService{
State: s,
},
initialSync: &mockSync.Sync{IsSyncing: false},
}
valid, _ := r.validateVoluntaryExit(ctx, exit, p2p, true /*fromSelf*/)
if valid {
t.Error("Validation should have failed")
}
if p2p.BroadcastCalled {
t.Error("Broadcast was called")
if m.ValidatorData == nil {
t.Error("Decoded message was not set on the message validator data")
}
}
func TestValidateVoluntaryExit_ValidExit_Syncing(t *testing.T) {
p2p := p2ptest.NewTestP2P(t)
p := p2ptest.NewTestP2P(t)
ctx := context.Background()
exit, s := setupValidExit(t)
r := &Service{
p2p: p2p,
p2p: p,
chain: &mock.ChainService{
State: s,
},
initialSync: &mockSync.Sync{IsSyncing: true},
}
valid, _ := r.validateVoluntaryExit(ctx, exit, p2p, false /*fromSelf*/)
buf := new(bytes.Buffer)
if _, err := p.Encoding().Encode(buf, exit); err != nil {
t.Fatal(err)
}
m := &pubsub.Message{
Message: &pubsubpb.Message{
Data: buf.Bytes(),
TopicIDs: []string{
p2p.GossipTypeMapping[reflect.TypeOf(exit)],
},
},
}
valid := r.validateVoluntaryExit(ctx, "", m)
if valid {
t.Error("Validation should have failed")
}
if p2p.BroadcastCalled {
t.Error("Broadcast was called")
}
}

View File

@@ -260,7 +260,8 @@ func (d *db) KeyMap() ([][]byte, map[[48]byte]keyMap, error) {
for i, pk := range pks.PrivateKeys {
seckey, err := bls.SecretKeyFromBytes(pk)
if err != nil {
return err
log.WithError(err).Warn("Could not deserialize secret key... removing")
return tx.Bucket(assignedPkBucket).Delete(k)
}
keytoSet := bytesutil.ToBytes48(seckey.PublicKey().Marshal())