Slasher Simulator Code for Testing Optimized Slasher Behavior (#9695)

* slashing simulator

* add in necessary items for slasher sim

* sim item

* fix up

* fixed build

* rev

* slasher sim in testing

* testonly

* gaz

* gaz

* fix viz

Co-authored-by: prylabs-bulldozer[bot] <58059840+prylabs-bulldozer[bot]@users.noreply.github.com>
Co-authored-by: prestonvanloon <preston@prysmaticlabs.com>
This commit is contained in:
Raul Jordan
2021-09-28 21:27:21 -05:00
committed by GitHub
parent 0edb3b9e65
commit 9edba29f64
27 changed files with 938 additions and 31 deletions

View File

@@ -23,6 +23,7 @@ go_library(
visibility = [
"//beacon-chain:__subpackages__",
"//testing/fuzz:__pkg__",
"//testing/slasher/simulator:__pkg__",
],
deps = [
"//async:go_default_library",

View File

@@ -7,6 +7,7 @@ go_library(
importpath = "github.com/prysmaticlabs/prysm/beacon-chain/blockchain/testing",
visibility = [
"//beacon-chain:__subpackages__",
"//testing:__subpackages__",
"//testing/fuzz:__pkg__",
],
deps = [

View File

@@ -18,6 +18,7 @@ go_library(
"//testing/benchmark/benchmark_files:__subpackages__",
"//testing/endtoend/evaluators:__pkg__",
"//testing/fuzz:__pkg__",
"//testing/slasher/simulator:__subpackages__",
"//testing/spectest:__subpackages__",
"//testing/util:__pkg__",
"//tools:__subpackages__",

View File

@@ -72,7 +72,7 @@ func ProcessProposerSlashings(
// VerifyProposerSlashing verifies that the data provided from slashing is valid.
func VerifyProposerSlashing(
beaconState state.BeaconState,
beaconState state.ReadOnlyBeaconState,
slashing *ethpb.ProposerSlashing,
) error {
if slashing.Header_1 == nil || slashing.Header_1.Header == nil || slashing.Header_2 == nil || slashing.Header_2.Header == nil {

View File

@@ -6,6 +6,6 @@ go_library(
importpath = "github.com/prysmaticlabs/prysm/beacon-chain/core/feed",
visibility = [
"//beacon-chain:__subpackages__",
"//shared:__subpackages__",
"//testing/slasher/simulator:__subpackages__",
],
)

View File

@@ -7,7 +7,10 @@ go_library(
"notifier.go",
],
importpath = "github.com/prysmaticlabs/prysm/beacon-chain/core/feed/state",
visibility = ["//beacon-chain:__subpackages__"],
visibility = [
"//beacon-chain:__subpackages__",
"//testing/slasher/simulator:__subpackages__",
],
deps = [
"//async/event:go_default_library",
"//proto/prysm/v1alpha1/block:go_default_library",

View File

@@ -29,6 +29,7 @@ go_library(
"//testing/benchmark/benchmark_files:__subpackages__",
"//testing/endtoend/evaluators:__pkg__",
"//testing/fuzz:__pkg__",
"//testing/slasher/simulator:__pkg__",
"//testing/spectest:__subpackages__",
"//testing/util:__pkg__",
"//tools:__subpackages__",

View File

@@ -64,7 +64,7 @@ func signingData(rootFunc func() ([32]byte, error), domain []byte) ([32]byte, er
}
// ComputeDomainVerifySigningRoot computes domain and verifies signing root of an object given the beacon state, validator index and signature.
func ComputeDomainVerifySigningRoot(st state.BeaconState, index types.ValidatorIndex, epoch types.Epoch, obj fssz.HashRoot, domain [4]byte, sig []byte) error {
func ComputeDomainVerifySigningRoot(st state.ReadOnlyBeaconState, index types.ValidatorIndex, epoch types.Epoch, obj fssz.HashRoot, domain [4]byte, sig []byte) error {
v, err := st.ValidatorAtIndex(index)
if err != nil {
return err

View File

@@ -13,6 +13,7 @@ go_library(
"//beacon-chain:__subpackages__",
"//cmd/beacon-chain:__subpackages__",
"//testing/fuzz:__pkg__",
"//testing/slasher/simulator:__pkg__",
"//tools:__subpackages__",
],
deps = [

View File

@@ -7,7 +7,7 @@ go_library(
importpath = "github.com/prysmaticlabs/prysm/beacon-chain/db/testing",
visibility = [
"//beacon-chain:__subpackages__",
"//testing/endtoend:__subpackages__",
"//testing:__subpackages__",
],
deps = [
"//beacon-chain/db:go_default_library",

View File

@@ -13,7 +13,9 @@ go_library(
importpath = "github.com/prysmaticlabs/prysm/beacon-chain/operations/slashings",
visibility = [
"//beacon-chain:__subpackages__",
"//testing/endtoend:__subpackages__",
"//testing/fuzz:__pkg__",
"//testing/slasher/simulator:__pkg__",
],
deps = [
"//beacon-chain/core:go_default_library",

View File

@@ -30,17 +30,17 @@ func (m *PoolMock) InsertAttesterSlashing(_ context.Context, _ state.ReadOnlyBea
}
// InsertProposerSlashing --
func (m *PoolMock) InsertProposerSlashing(_ context.Context, _ state.BeaconState, slashing *ethpb.ProposerSlashing) error {
func (m *PoolMock) InsertProposerSlashing(_ context.Context, _ state.ReadOnlyBeaconState, slashing *ethpb.ProposerSlashing) error {
m.PendingPropSlashings = append(m.PendingPropSlashings, slashing)
return nil
}
// MarkIncludedAttesterSlashing --
func (m *PoolMock) MarkIncludedAttesterSlashing(_ *ethpb.AttesterSlashing) {
func (_ *PoolMock) MarkIncludedAttesterSlashing(_ *ethpb.AttesterSlashing) {
panic("implement me")
}
// MarkIncludedProposerSlashing --
func (m *PoolMock) MarkIncludedProposerSlashing(_ *ethpb.ProposerSlashing) {
func (_ *PoolMock) MarkIncludedProposerSlashing(_ *ethpb.ProposerSlashing) {
panic("implement me")
}

View File

@@ -175,7 +175,7 @@ func (p *Pool) InsertAttesterSlashing(
// has been included recently, the validator is already exited, or the validator was already slashed.
func (p *Pool) InsertProposerSlashing(
ctx context.Context,
state state.BeaconState,
state state.ReadOnlyBeaconState,
slashing *ethpb.ProposerSlashing,
) error {
p.lock.Lock()

View File

@@ -6,6 +6,13 @@ import (
"github.com/prysmaticlabs/prysm/testing/require"
)
var (
_ = PoolManager(&Pool{})
_ = PoolInserter(&Pool{})
_ = PoolManager(&PoolMock{})
_ = PoolInserter(&PoolMock{})
)
func TestPool_validatorSlashingPreconditionCheck_requiresLock(t *testing.T) {
p := &Pool{}
_, err := p.validatorSlashingPreconditionCheck(nil, 0)

View File

@@ -9,11 +9,8 @@ import (
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
)
// PoolManager maintains a pool of pending and recently included attester and proposer slashings.
// This pool is used by proposers to insert data into new blocks.
type PoolManager interface {
PendingAttesterSlashings(ctx context.Context, state state.ReadOnlyBeaconState, noLimit bool) []*ethpb.AttesterSlashing
PendingProposerSlashings(ctx context.Context, state state.ReadOnlyBeaconState, noLimit bool) []*ethpb.ProposerSlashing
// PoolInserter is capable of inserting new slashing objects into the operations pool.
type PoolInserter interface {
InsertAttesterSlashing(
ctx context.Context,
state state.ReadOnlyBeaconState,
@@ -21,9 +18,17 @@ type PoolManager interface {
) error
InsertProposerSlashing(
ctx context.Context,
state state.BeaconState,
state state.ReadOnlyBeaconState,
slashing *ethpb.ProposerSlashing,
) error
}
// PoolManager maintains a pool of pending and recently included attester and proposer slashings.
// This pool is used by proposers to insert data into new blocks.
type PoolManager interface {
PoolInserter
PendingAttesterSlashings(ctx context.Context, state state.ReadOnlyBeaconState, noLimit bool) []*ethpb.AttesterSlashing
PendingProposerSlashings(ctx context.Context, state state.ReadOnlyBeaconState, noLimit bool) []*ethpb.ProposerSlashing
MarkIncludedAttesterSlashing(as *ethpb.AttesterSlashing)
MarkIncludedProposerSlashing(ps *ethpb.ProposerSlashing)
}

View File

@@ -8,6 +8,7 @@ go_library(
"detect_blocks.go",
"doc.go",
"helpers.go",
"log.go",
"metrics.go",
"params.go",
"process_slashings.go",
@@ -16,18 +17,23 @@ go_library(
"service.go",
],
importpath = "github.com/prysmaticlabs/prysm/beacon-chain/slasher",
visibility = ["//beacon-chain:__subpackages__"],
visibility = [
"//beacon-chain:__subpackages__",
"//testing/slasher/simulator:__subpackages__",
],
deps = [
"//async/event:go_default_library",
"//beacon-chain/blockchain:go_default_library",
"//beacon-chain/core:go_default_library",
"//beacon-chain/core/blocks:go_default_library",
"//beacon-chain/core/feed:go_default_library",
"//beacon-chain/core/feed/state:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/operations/slashings:go_default_library",
"//beacon-chain/slasher/types:go_default_library",
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/stategen:go_default_library",
"//beacon-chain/sync:go_default_library",
"//config/params:go_default_library",
"//container/slice:go_default_library",
"//encoding/bytesutil:go_default_library",

View File

@@ -0,0 +1,5 @@
package slasher
import "github.com/sirupsen/logrus"
var log = logrus.WithField("prefix", "slasher")

View File

@@ -1,36 +1,157 @@
// Package slasher implements slashing detection for eth2, able to catch slashable attestations
// and proposals that it receives via two event feeds, respectively. Any found slashings
// are then submitted to the beacon node's slashing operations pool. See the design document
// here https://hackmd.io/@prysmaticlabs/slasher.
package slasher
import (
"context"
"time"
"github.com/prysmaticlabs/prysm/async/event"
"github.com/prysmaticlabs/prysm/beacon-chain/blockchain"
"github.com/prysmaticlabs/prysm/beacon-chain/core/feed"
statefeed "github.com/prysmaticlabs/prysm/beacon-chain/core/feed/state"
"github.com/prysmaticlabs/prysm/beacon-chain/db"
"github.com/prysmaticlabs/prysm/beacon-chain/operations/slashings"
"github.com/prysmaticlabs/prysm/beacon-chain/state/stategen"
"github.com/sirupsen/logrus"
"github.com/prysmaticlabs/prysm/beacon-chain/sync"
"github.com/prysmaticlabs/prysm/config/params"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/time/slots"
)
var log = logrus.WithField("prefix", "slasher")
// ServiceConfig contains service dependencies for slasher.
// ServiceConfig for the slasher service in the beacon node.
// This struct allows us to specify required dependencies and
// parameters for slasher to function as needed.
type ServiceConfig struct {
Database db.SlasherDatabase
AttestationStateFetcher blockchain.AttestationStateFetcher
IndexedAttestationsFeed *event.Feed
BeaconBlockHeadersFeed *event.Feed
StateGen stategen.StateManager
SlashingPoolInserter slashings.PoolManager
Database db.SlasherDatabase
StateNotifier statefeed.Notifier
AttestationStateFetcher blockchain.AttestationStateFetcher
StateGen stategen.StateManager
SlashingPoolInserter slashings.PoolInserter
HeadStateFetcher blockchain.HeadFetcher
SyncChecker sync.Checker
}
// Service for running slasher mode in a beacon node.
type Service struct {
params *Parameters
serviceCfg *ServiceConfig
blksQueue *blocksQueue
attsQueue *attestationsQueue
genesisTime time.Time
// SlashingChecker is an interface for defining services that the beacon node may interact with to provide slashing data.
type SlashingChecker interface {
IsSlashableBlock(ctx context.Context, proposal *ethpb.SignedBeaconBlockHeader) (*ethpb.ProposerSlashing, error)
IsSlashableAttestation(ctx context.Context, attestation *ethpb.IndexedAttestation) ([]*ethpb.AttesterSlashing, error)
}
// Service defining a slasher implementation as part of
// the beacon node, able to detect eth2 slashable offenses.
type Service struct {
params *Parameters
serviceCfg *ServiceConfig
indexedAttsChan chan *ethpb.IndexedAttestation
beaconBlockHeadersChan chan *ethpb.SignedBeaconBlockHeader
attsQueue *attestationsQueue
blksQueue *blocksQueue
ctx context.Context
cancel context.CancelFunc
slotTicker *slots.SlotTicker
genesisTime time.Time
}
// New instantiates a new slasher from configuration values.
func New(ctx context.Context, srvCfg *ServiceConfig) (*Service, error) {
ctx, cancel := context.WithCancel(ctx)
return &Service{
params: DefaultParams(),
serviceCfg: srvCfg,
indexedAttsChan: make(chan *ethpb.IndexedAttestation, 1),
beaconBlockHeadersChan: make(chan *ethpb.SignedBeaconBlockHeader, 1),
attsQueue: newAttestationsQueue(),
blksQueue: newBlocksQueue(),
ctx: ctx,
cancel: cancel,
}, nil
}
// Start listening for received indexed attestations and blocks
// and perform slashing detection on them.
func (s *Service) Start() {
go s.run()
}
func (s *Service) run() {
stateChannel := make(chan *feed.Event, 1)
stateSub := s.serviceCfg.StateNotifier.StateFeed().Subscribe(stateChannel)
stateEvent := <-stateChannel
// Wait for us to receive the genesis time via a chain started notification.
if stateEvent.Type == statefeed.ChainStarted {
data, ok := stateEvent.Data.(*statefeed.ChainStartedData)
if !ok {
log.Error("Could not receive chain start notification, want *statefeed.ChainStartedData")
return
}
s.genesisTime = data.StartTime
log.WithField("genesisTime", s.genesisTime).Info("Starting slasher, received chain start event")
} else if stateEvent.Type == statefeed.Initialized {
// Alternatively, if the chain has already started, we then read the genesis
// time value from this data.
data, ok := stateEvent.Data.(*statefeed.InitializedData)
if !ok {
log.Error("Could not receive chain start notification, want *statefeed.ChainStartedData")
return
}
s.genesisTime = data.StartTime
log.WithField("genesisTime", s.genesisTime).Info("Starting slasher, chain already initialized")
} else {
// This should not happen.
log.Error("Could start slasher, could not receive chain start event")
return
}
stateSub.Unsubscribe()
secondsPerSlot := params.BeaconConfig().SecondsPerSlot
s.slotTicker = slots.NewSlotTicker(s.genesisTime, secondsPerSlot)
s.waitForSync(s.genesisTime)
indexedAttsChan := make(chan *ethpb.IndexedAttestation, 1)
beaconBlockHeadersChan := make(chan *ethpb.SignedBeaconBlockHeader, 1)
log.Info("Completed chain sync, starting slashing detection")
go s.processQueuedAttestations(s.ctx, s.slotTicker.C())
go s.processQueuedBlocks(s.ctx, s.slotTicker.C())
go s.receiveAttestations(s.ctx, indexedAttsChan)
go s.receiveBlocks(s.ctx, beaconBlockHeadersChan)
go s.pruneSlasherData(s.ctx, s.slotTicker.C())
}
// Stop the slasher service.
func (s *Service) Stop() error {
s.cancel()
if s.slotTicker != nil {
s.slotTicker.Done()
}
return nil
}
// Status of the slasher service.
func (s *Service) Status() error {
return nil
}
func (s *Service) waitForSync(genesisTime time.Time) {
if slots.SlotsSinceGenesis(genesisTime) == 0 || !s.serviceCfg.SyncChecker.Syncing() {
return
}
for {
select {
case <-s.slotTicker.C():
// If node is still syncing, do not operate slasher.
if s.serviceCfg.SyncChecker.Syncing() {
continue
}
return
case <-s.ctx.Done():
return
}
}
}

View File

@@ -15,6 +15,7 @@ go_library(
"//slasher/rpc:__subpackages__",
"//testing/benchmark:__pkg__",
"//testing/fuzz:__pkg__",
"//testing/slasher/simulator:__pkg__",
"//testing/spectest:__subpackages__",
"//testing/util:__pkg__",
"//tools/benchmark-files-gen:__pkg__",

View File

@@ -19,6 +19,7 @@ go_library(
visibility = [
"//beacon-chain:__subpackages__",
"//testing/fuzz:__pkg__",
"//testing/slasher/simulator:__pkg__",
],
deps = [
"//beacon-chain/core:go_default_library",

View File

@@ -0,0 +1,59 @@
load("@prysm//tools/go:def.bzl", "go_library", "go_test")
go_library(
name = "go_default_library",
testonly = True,
srcs = [
"attestation_generator.go",
"block_generator.go",
"simulator.go",
],
importpath = "github.com/prysmaticlabs/prysm/testing/slasher/simulator",
visibility = [
"//endtoend:__subpackages__",
],
deps = [
"//async/event:go_default_library",
"//beacon-chain/blockchain:go_default_library",
"//beacon-chain/core:go_default_library",
"//beacon-chain/core/feed:go_default_library",
"//beacon-chain/core/feed/state:go_default_library",
"//beacon-chain/core/helpers:go_default_library",
"//beacon-chain/core/signing:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/operations/slashings:go_default_library",
"//beacon-chain/slasher:go_default_library",
"//beacon-chain/state:go_default_library",
"//beacon-chain/state/stategen:go_default_library",
"//config/params:go_default_library",
"//crypto/bls:go_default_library",
"//crypto/rand:go_default_library",
"//encoding/bytesutil:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//time/slots:go_default_library",
"@com_github_prysmaticlabs_eth2_types//:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
],
)
go_test(
name = "go_default_test",
srcs = [
"attestation_generator_test.go",
"block_generator_test.go",
"simulator_test.go",
],
embed = [":go_default_library"],
deps = [
"//beacon-chain/blockchain/testing:go_default_library",
"//beacon-chain/db/testing:go_default_library",
"//beacon-chain/state/stategen:go_default_library",
"//config/params:go_default_library",
"//crypto/bls:go_default_library",
"//proto/prysm/v1alpha1:go_default_library",
"//proto/prysm/v1alpha1/slashings:go_default_library",
"//testing/require:go_default_library",
"//testing/util:go_default_library",
"@com_github_prysmaticlabs_eth2_types//:go_default_library",
],
)

View File

@@ -0,0 +1,177 @@
package simulator
import (
"context"
"math"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/core"
"github.com/prysmaticlabs/prysm/beacon-chain/core/helpers"
"github.com/prysmaticlabs/prysm/beacon-chain/core/signing"
"github.com/prysmaticlabs/prysm/beacon-chain/state"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/crypto/bls"
"github.com/prysmaticlabs/prysm/crypto/rand"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/sirupsen/logrus"
)
func (s *Simulator) generateAttestationsForSlot(
ctx context.Context, slot types.Slot,
) ([]*ethpb.IndexedAttestation, []*ethpb.AttesterSlashing, error) {
attestations := make([]*ethpb.IndexedAttestation, 0)
slashings := make([]*ethpb.AttesterSlashing, 0)
currentEpoch := core.SlotToEpoch(slot)
committeesPerSlot := helpers.SlotCommitteeCount(s.srvConfig.Params.NumValidators)
valsPerCommittee := s.srvConfig.Params.NumValidators /
(committeesPerSlot * uint64(s.srvConfig.Params.SlotsPerEpoch))
valsPerSlot := committeesPerSlot * valsPerCommittee
var sourceEpoch types.Epoch = 0
if currentEpoch != 0 {
sourceEpoch = currentEpoch - 1
}
var slashedIndices []uint64
startIdx := valsPerSlot * uint64(slot%s.srvConfig.Params.SlotsPerEpoch)
endIdx := startIdx + valsPerCommittee
for c := types.CommitteeIndex(0); uint64(c) < committeesPerSlot; c++ {
attData := &ethpb.AttestationData{
Slot: slot,
CommitteeIndex: c,
BeaconBlockRoot: bytesutil.PadTo([]byte("block"), 32),
Source: &ethpb.Checkpoint{
Epoch: sourceEpoch,
Root: bytesutil.PadTo([]byte("source"), 32),
},
Target: &ethpb.Checkpoint{
Epoch: currentEpoch,
Root: bytesutil.PadTo([]byte("target"), 32),
},
}
valsPerAttestation := uint64(math.Floor(s.srvConfig.Params.AggregationPercent * float64(valsPerCommittee)))
for i := startIdx; i < endIdx; i += valsPerAttestation {
attEndIdx := i + valsPerAttestation
if attEndIdx >= endIdx {
attEndIdx = endIdx
}
indices := make([]uint64, 0, valsPerAttestation)
for idx := i; idx < attEndIdx; idx++ {
indices = append(indices, idx)
}
att := &ethpb.IndexedAttestation{
AttestingIndices: indices,
Data: attData,
Signature: params.BeaconConfig().EmptySignature[:],
}
beaconState, err := s.srvConfig.AttestationStateFetcher.AttestationTargetState(ctx, att.Data.Target)
if err != nil {
return nil, nil, err
}
// Sign the attestation with a valid signature.
aggSig, err := s.aggregateSigForAttestation(beaconState, att)
if err != nil {
return nil, nil, err
}
att.Signature = aggSig.Marshal()
attestations = append(attestations, att)
if rand.NewGenerator().Float64() < s.srvConfig.Params.AttesterSlashingProbab {
slashableAtt := makeSlashableFromAtt(att, []uint64{indices[0]})
aggSig, err := s.aggregateSigForAttestation(beaconState, slashableAtt)
if err != nil {
return nil, nil, err
}
slashableAtt.Signature = aggSig.Marshal()
slashedIndices = append(slashedIndices, slashableAtt.AttestingIndices...)
slashings = append(slashings, &ethpb.AttesterSlashing{
Attestation_1: att,
Attestation_2: slashableAtt,
})
attestations = append(attestations, slashableAtt)
}
}
startIdx += valsPerCommittee
endIdx += valsPerCommittee
}
if len(slashedIndices) > 0 {
log.WithFields(logrus.Fields{
"amount": len(slashedIndices),
"indices": slashedIndices,
}).Infof("Slashable attestation made")
}
return attestations, slashings, nil
}
func (s *Simulator) aggregateSigForAttestation(
beaconState state.BeaconState, att *ethpb.IndexedAttestation,
) (bls.Signature, error) {
domain, err := signing.Domain(
beaconState.Fork(),
att.Data.Target.Epoch,
params.BeaconConfig().DomainBeaconAttester,
beaconState.GenesisValidatorRoot(),
)
if err != nil {
return nil, err
}
signingRoot, err := signing.ComputeSigningRoot(att.Data, domain)
if err != nil {
return nil, err
}
sigs := make([]bls.Signature, len(att.AttestingIndices))
for i, validatorIndex := range att.AttestingIndices {
privKey := s.srvConfig.PrivateKeysByValidatorIndex[types.ValidatorIndex(validatorIndex)]
sigs[i] = privKey.Sign(signingRoot[:])
}
return bls.AggregateSignatures(sigs), nil
}
func makeSlashableFromAtt(att *ethpb.IndexedAttestation, indices []uint64) *ethpb.IndexedAttestation {
if att.Data.Source.Epoch <= 2 {
return makeDoubleVoteFromAtt(att, indices)
}
attData := &ethpb.AttestationData{
Slot: att.Data.Slot,
CommitteeIndex: att.Data.CommitteeIndex,
BeaconBlockRoot: att.Data.BeaconBlockRoot,
Source: &ethpb.Checkpoint{
Epoch: att.Data.Source.Epoch - 3,
Root: att.Data.Source.Root,
},
Target: &ethpb.Checkpoint{
Epoch: att.Data.Target.Epoch,
Root: att.Data.Target.Root,
},
}
return &ethpb.IndexedAttestation{
AttestingIndices: indices,
Data: attData,
Signature: params.BeaconConfig().EmptySignature[:],
}
}
func makeDoubleVoteFromAtt(att *ethpb.IndexedAttestation, indices []uint64) *ethpb.IndexedAttestation {
attData := &ethpb.AttestationData{
Slot: att.Data.Slot,
CommitteeIndex: att.Data.CommitteeIndex,
BeaconBlockRoot: bytesutil.PadTo([]byte("slash me"), 32),
Source: &ethpb.Checkpoint{
Epoch: att.Data.Source.Epoch,
Root: att.Data.Source.Root,
},
Target: &ethpb.Checkpoint{
Epoch: att.Data.Target.Epoch,
Root: att.Data.Target.Root,
},
}
return &ethpb.IndexedAttestation{
AttestingIndices: indices,
Data: attData,
Signature: params.BeaconConfig().EmptySignature[:],
}
}

View File

@@ -0,0 +1,64 @@
package simulator
import (
"context"
"testing"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1/slashings"
"github.com/prysmaticlabs/prysm/testing/require"
)
func TestGenerateAttestationsForSlot_Slashing(t *testing.T) {
ctx := context.Background()
simParams := &Parameters{
SecondsPerSlot: params.BeaconConfig().SecondsPerSlot,
SlotsPerEpoch: params.BeaconConfig().SlotsPerEpoch,
AggregationPercent: 1,
NumValidators: 64,
AttesterSlashingProbab: 1,
}
srv := setupService(t, simParams)
epoch3Atts, _, err := srv.generateAttestationsForSlot(ctx, params.BeaconConfig().SlotsPerEpoch*3)
require.NoError(t, err)
epoch4Atts, _, err := srv.generateAttestationsForSlot(ctx, params.BeaconConfig().SlotsPerEpoch*4)
require.NoError(t, err)
for i := 0; i < len(epoch3Atts); i += 2 {
goodAtt := epoch3Atts[i]
surroundAtt := epoch4Atts[i+1]
require.Equal(t, true, slashings.IsSurround(surroundAtt, goodAtt))
}
}
func TestGenerateAttestationsForSlot_CorrectIndices(t *testing.T) {
ctx := context.Background()
simParams := &Parameters{
SecondsPerSlot: params.BeaconConfig().SecondsPerSlot,
SlotsPerEpoch: params.BeaconConfig().SlotsPerEpoch,
AggregationPercent: 1,
NumValidators: 16384,
AttesterSlashingProbab: 0,
}
srv := setupService(t, simParams)
slot0Atts, _, err := srv.generateAttestationsForSlot(ctx, 0)
require.NoError(t, err)
slot1Atts, _, err := srv.generateAttestationsForSlot(ctx, 1)
require.NoError(t, err)
slot2Atts, _, err := srv.generateAttestationsForSlot(ctx, 2)
require.NoError(t, err)
var validatorIndices []uint64
for _, att := range append(slot0Atts, slot1Atts...) {
validatorIndices = append(validatorIndices, att.AttestingIndices...)
}
for _, att := range slot2Atts {
validatorIndices = append(validatorIndices, att.AttestingIndices...)
}
// Making sure indices are one after the other for attestations.
var validatorIndex uint64
for _, ii := range validatorIndices {
require.Equal(t, validatorIndex, ii)
validatorIndex++
}
}

View File

@@ -0,0 +1,98 @@
package simulator
import (
"context"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/beacon-chain/core/signing"
"github.com/prysmaticlabs/prysm/beacon-chain/state"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/crypto/bls"
"github.com/prysmaticlabs/prysm/crypto/rand"
"github.com/prysmaticlabs/prysm/encoding/bytesutil"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
)
func (s *Simulator) generateBlockHeadersForSlot(
ctx context.Context, slot types.Slot,
) ([]*ethpb.SignedBeaconBlockHeader, []*ethpb.ProposerSlashing, error) {
blocks := make([]*ethpb.SignedBeaconBlockHeader, 0)
slashings := make([]*ethpb.ProposerSlashing, 0)
proposer := rand.NewGenerator().Uint64() % s.srvConfig.Params.NumValidators
parentRoot := [32]byte{}
beaconState, err := s.srvConfig.StateGen.StateByRoot(ctx, parentRoot)
if err != nil {
return nil, nil, err
}
block := &ethpb.SignedBeaconBlockHeader{
Header: &ethpb.BeaconBlockHeader{
Slot: slot,
ProposerIndex: types.ValidatorIndex(proposer),
ParentRoot: bytesutil.PadTo([]byte{}, 32),
StateRoot: bytesutil.PadTo([]byte{}, 32),
BodyRoot: bytesutil.PadTo([]byte("good block"), 32),
},
}
sig, err := s.signBlockHeader(beaconState, block)
if err != nil {
return nil, nil, err
}
block.Signature = sig.Marshal()
blocks = append(blocks, block)
if rand.NewGenerator().Float64() < s.srvConfig.Params.ProposerSlashingProbab {
log.WithField("proposerIndex", proposer).Infof("Slashable block made")
slashableBlock := &ethpb.SignedBeaconBlockHeader{
Header: &ethpb.BeaconBlockHeader{
Slot: slot,
ProposerIndex: types.ValidatorIndex(proposer),
ParentRoot: bytesutil.PadTo([]byte{}, 32),
StateRoot: bytesutil.PadTo([]byte{}, 32),
BodyRoot: bytesutil.PadTo([]byte("bad block"), 32),
},
Signature: sig.Marshal(),
}
sig, err = s.signBlockHeader(beaconState, slashableBlock)
if err != nil {
return nil, nil, err
}
slashableBlock.Signature = sig.Marshal()
blocks = append(blocks, slashableBlock)
slashings = append(slashings, &ethpb.ProposerSlashing{
Header_1: block,
Header_2: slashableBlock,
})
}
return blocks, slashings, nil
}
func (s *Simulator) signBlockHeader(
beaconState state.BeaconState,
header *ethpb.SignedBeaconBlockHeader,
) (bls.Signature, error) {
domain, err := signing.Domain(
beaconState.Fork(),
0,
params.BeaconConfig().DomainBeaconProposer,
beaconState.GenesisValidatorRoot(),
)
if err != nil {
return nil, err
}
htr, err := header.Header.HashTreeRoot()
if err != nil {
return nil, err
}
container := &ethpb.SigningData{
ObjectRoot: htr[:],
Domain: domain,
}
signingRoot, err := container.HashTreeRoot()
if err != nil {
return nil, err
}
validatorPrivKey := s.srvConfig.PrivateKeysByValidatorIndex[header.Header.ProposerIndex]
return validatorPrivKey.Sign(signingRoot[:]), nil
}

View File

@@ -0,0 +1,31 @@
package simulator
import (
"bytes"
"context"
"testing"
"github.com/prysmaticlabs/prysm/testing/require"
)
func TestGenerateBlockHeadersForSlot_Slashing(t *testing.T) {
ctx := context.Background()
simParams := &Parameters{
AggregationPercent: 1,
NumValidators: 64,
ProposerSlashingProbab: 1,
}
srv := setupService(t, simParams)
slot1Blocks, _, err := srv.generateBlockHeadersForSlot(ctx, 1)
require.NoError(t, err)
require.Equal(t, 2, len(slot1Blocks))
block1Root, err := slot1Blocks[0].HashTreeRoot()
require.NoError(t, err)
block2Root, err := slot1Blocks[1].HashTreeRoot()
require.NoError(t, err)
if slot1Blocks[0].Header.ProposerIndex == slot1Blocks[1].Header.ProposerIndex && bytes.Equal(block1Root[:], block2Root[:]) {
t.Error("Blocks received were not slashable")
}
}

View File

@@ -0,0 +1,274 @@
package simulator
import (
"context"
"time"
types "github.com/prysmaticlabs/eth2-types"
"github.com/prysmaticlabs/prysm/async/event"
"github.com/prysmaticlabs/prysm/beacon-chain/blockchain"
"github.com/prysmaticlabs/prysm/beacon-chain/core"
"github.com/prysmaticlabs/prysm/beacon-chain/core/feed"
statefeed "github.com/prysmaticlabs/prysm/beacon-chain/core/feed/state"
"github.com/prysmaticlabs/prysm/beacon-chain/db"
"github.com/prysmaticlabs/prysm/beacon-chain/operations/slashings"
"github.com/prysmaticlabs/prysm/beacon-chain/slasher"
"github.com/prysmaticlabs/prysm/beacon-chain/state/stategen"
"github.com/prysmaticlabs/prysm/config/params"
"github.com/prysmaticlabs/prysm/crypto/bls"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/time/slots"
"github.com/sirupsen/logrus"
)
var log = logrus.WithField("prefix", "simulator")
// ServiceConfig for the simulator.
type ServiceConfig struct {
Params *Parameters
Database db.SlasherDatabase
StateNotifier statefeed.Notifier
AttestationStateFetcher blockchain.AttestationStateFetcher
HeadStateFetcher blockchain.HeadFetcher
StateGen stategen.StateManager
SlashingsPool slashings.PoolManager
PrivateKeysByValidatorIndex map[types.ValidatorIndex]bls.SecretKey
}
// Parameters for a slasher simulator.
type Parameters struct {
SecondsPerSlot uint64
SlotsPerEpoch types.Slot
AggregationPercent float64
ProposerSlashingProbab float64
AttesterSlashingProbab float64
NumValidators uint64
NumEpochs uint64
}
// Simulator defines a struct which can launch a slasher simulation
// at scale using configuration parameters.
type Simulator struct {
ctx context.Context
slasher *slasher.Service
srvConfig *ServiceConfig
indexedAttsFeed *event.Feed
beaconBlocksFeed *event.Feed
sentAttSlashingFeed *event.Feed
sentBlockSlashingFeed *event.Feed
sentProposerSlashings map[[32]byte]*ethpb.ProposerSlashing
sentAttesterSlashings map[[32]byte]*ethpb.AttesterSlashing
genesisTime time.Time
}
// DefaultParams for launching a slasher simulator.
func DefaultParams() *Parameters {
return &Parameters{
SecondsPerSlot: params.BeaconConfig().SecondsPerSlot,
SlotsPerEpoch: 4,
AggregationPercent: 1.0,
ProposerSlashingProbab: 0.3,
AttesterSlashingProbab: 0.3,
NumValidators: params.BeaconConfig().MinGenesisActiveValidatorCount,
NumEpochs: 4,
}
}
// New initializes a slasher simulator from a beacon database
// and configuration parameters.
func New(ctx context.Context, srvConfig *ServiceConfig) (*Simulator, error) {
indexedAttsFeed := new(event.Feed)
beaconBlocksFeed := new(event.Feed)
sentBlockSlashingFeed := new(event.Feed)
sentAttSlashingFeed := new(event.Feed)
slasherSrv, err := slasher.New(ctx, &slasher.ServiceConfig{
IndexedAttestationsFeed: indexedAttsFeed,
BeaconBlockHeadersFeed: beaconBlocksFeed,
Database: srvConfig.Database,
StateNotifier: srvConfig.StateNotifier,
HeadStateFetcher: srvConfig.HeadStateFetcher,
AttestationStateFetcher: srvConfig.AttestationStateFetcher,
StateGen: srvConfig.StateGen,
SlashingPoolInserter: srvConfig.SlashingsPool,
})
if err != nil {
return nil, err
}
return &Simulator{
ctx: ctx,
slasher: slasherSrv,
srvConfig: srvConfig,
indexedAttsFeed: indexedAttsFeed,
beaconBlocksFeed: beaconBlocksFeed,
sentAttSlashingFeed: sentAttSlashingFeed,
sentBlockSlashingFeed: sentBlockSlashingFeed,
sentProposerSlashings: make(map[[32]byte]*ethpb.ProposerSlashing),
sentAttesterSlashings: make(map[[32]byte]*ethpb.AttesterSlashing),
}, nil
}
// Start a simulator.
func (s *Simulator) Start() {
log.WithFields(logrus.Fields{
"numValidators": s.srvConfig.Params.NumValidators,
"numEpochs": s.srvConfig.Params.NumEpochs,
"secondsPerSlot": s.srvConfig.Params.SecondsPerSlot,
"proposerSlashingProbab": s.srvConfig.Params.ProposerSlashingProbab,
"attesterSlashingProbab": s.srvConfig.Params.AttesterSlashingProbab,
}).Info("Starting slasher simulator")
// Override global configuration for simulation purposes.
config := params.BeaconConfig().Copy()
config.SecondsPerSlot = s.srvConfig.Params.SecondsPerSlot
config.SlotsPerEpoch = s.srvConfig.Params.SlotsPerEpoch
params.OverrideBeaconConfig(config)
defer params.OverrideBeaconConfig(params.BeaconConfig())
// Start slasher in the background.
go s.slasher.Start()
// Wait some time and then send a "chain started" event over a notifier
// for slasher to pick up a genesis time.
time.Sleep(time.Second)
s.genesisTime = time.Now()
s.srvConfig.StateNotifier.StateFeed().Send(&feed.Event{
Type: statefeed.ChainStarted,
Data: &statefeed.ChainStartedData{StartTime: s.genesisTime},
})
// We simulate blocks and attestations for N epochs.
s.simulateBlocksAndAttestations(s.ctx)
// Verify the slashings we detected are the same as those the
// simulator produced, effectively checking slasher caught all slashable offenses.
s.verifySlashingsWereDetected(s.ctx)
}
// Stop the simulator.
func (s *Simulator) Stop() error {
return s.slasher.Stop()
}
func (s *Simulator) simulateBlocksAndAttestations(ctx context.Context) {
// Add a small offset to producing blocks and attestations a little bit after a slot starts.
ticker := slots.NewSlotTicker(s.genesisTime.Add(time.Millisecond*500), params.BeaconConfig().SecondsPerSlot)
defer ticker.Done()
for {
select {
case slot := <-ticker.C():
// We only run the simulator for a specified number of epochs.
totalEpochs := types.Epoch(s.srvConfig.Params.NumEpochs)
if core.SlotToEpoch(slot) >= totalEpochs {
return
}
// Since processing slashings requires at least one slot, we do nothing
// if we are a few slots from the end of the simulation.
endSlot, err := core.StartSlot(totalEpochs)
if err != nil {
log.WithError(err).Fatal("Could not get epoch start slot")
}
if slot+3 > endSlot {
continue
}
blockHeaders, propSlashings, err := s.generateBlockHeadersForSlot(ctx, slot)
if err != nil {
log.WithError(err).Fatal("Could not generate block headers for slot")
}
log.WithFields(logrus.Fields{
"numBlocks": len(blockHeaders),
"numSlashable": len(propSlashings),
}).Infof("Producing blocks for slot %d", slot)
for _, sl := range propSlashings {
slashingRoot, err := sl.HashTreeRoot()
if err != nil {
log.WithError(err).Fatal("Could not hash tree root slashing")
}
s.sentProposerSlashings[slashingRoot] = sl
}
for _, bb := range blockHeaders {
s.beaconBlocksFeed.Send(bb)
}
atts, attSlashings, err := s.generateAttestationsForSlot(ctx, slot)
if err != nil {
log.WithError(err).Fatal("Could not generate block headers for slot")
}
log.WithFields(logrus.Fields{
"numAtts": len(atts),
"numSlashable": len(propSlashings),
}).Infof("Producing attestations for slot %d", slot)
for _, sl := range attSlashings {
slashingRoot, err := sl.HashTreeRoot()
if err != nil {
log.WithError(err).Fatal("Could not hash tree root slashing")
}
s.sentAttesterSlashings[slashingRoot] = sl
}
for _, aa := range atts {
s.indexedAttsFeed.Send(aa)
}
case <-ctx.Done():
return
}
}
}
func (s *Simulator) verifySlashingsWereDetected(ctx context.Context) {
poolProposerSlashings := s.srvConfig.SlashingsPool.PendingProposerSlashings(
ctx, nil, true, /* no limit */
)
poolAttesterSlashings := s.srvConfig.SlashingsPool.PendingAttesterSlashings(
ctx, nil, true, /* no limit */
)
detectedProposerSlashings := make(map[[32]byte]*ethpb.ProposerSlashing)
detectedAttesterSlashings := make(map[[32]byte]*ethpb.AttesterSlashing)
for _, slashing := range poolProposerSlashings {
slashingRoot, err := slashing.HashTreeRoot()
if err != nil {
log.WithError(err).Error("Could not determine slashing root")
}
detectedProposerSlashings[slashingRoot] = slashing
}
for _, slashing := range poolAttesterSlashings {
slashingRoot, err := slashing.HashTreeRoot()
if err != nil {
log.WithError(err).Error("Could not determine slashing root")
}
detectedAttesterSlashings[slashingRoot] = slashing
}
// Check if the sent slashings made it into the slashings pool.
for slashingRoot, slashing := range s.sentProposerSlashings {
if _, ok := detectedProposerSlashings[slashingRoot]; !ok {
log.WithFields(logrus.Fields{
"slot": slashing.Header_1.Header.Slot,
"proposerIndex": slashing.Header_1.Header.ProposerIndex,
}).Errorf("Did not detect simulated proposer slashing")
continue
}
log.WithFields(logrus.Fields{
"slot": slashing.Header_1.Header.Slot,
"proposerIndex": slashing.Header_1.Header.ProposerIndex,
}).Info("Correctly detected simulated proposer slashing")
}
for slashingRoot, slashing := range s.sentAttesterSlashings {
if _, ok := detectedAttesterSlashings[slashingRoot]; !ok {
log.WithFields(logrus.Fields{
"targetEpoch": slashing.Attestation_1.Data.Target.Epoch,
"prevTargetEpoch": slashing.Attestation_2.Data.Target.Epoch,
"sourceEpoch": slashing.Attestation_1.Data.Source.Epoch,
"prevSourceEpoch": slashing.Attestation_2.Data.Source.Epoch,
}).Errorf("Did not detect simulated attester slashing")
continue
}
log.WithFields(logrus.Fields{
"targetEpoch": slashing.Attestation_1.Data.Target.Epoch,
"prevTargetEpoch": slashing.Attestation_2.Data.Target.Epoch,
"sourceEpoch": slashing.Attestation_1.Data.Source.Epoch,
"prevSourceEpoch": slashing.Attestation_2.Data.Source.Epoch,
}).Info("Correctly detected simulated attester slashing")
}
}

View File

@@ -0,0 +1,48 @@
package simulator
import (
"testing"
types "github.com/prysmaticlabs/eth2-types"
mock "github.com/prysmaticlabs/prysm/beacon-chain/blockchain/testing"
dbtest "github.com/prysmaticlabs/prysm/beacon-chain/db/testing"
"github.com/prysmaticlabs/prysm/beacon-chain/state/stategen"
"github.com/prysmaticlabs/prysm/crypto/bls"
ethpb "github.com/prysmaticlabs/prysm/proto/prysm/v1alpha1"
"github.com/prysmaticlabs/prysm/testing/require"
"github.com/prysmaticlabs/prysm/testing/util"
)
func setupService(t *testing.T, params *Parameters) *Simulator {
slasherDB := dbtest.SetupSlasherDB(t)
beaconState, err := util.NewBeaconState()
require.NoError(t, err)
// We setup validators in the beacon state along with their
// private keys used to generate valid signatures in generated objects.
validators := make([]*ethpb.Validator, params.NumValidators)
privKeys := make(map[types.ValidatorIndex]bls.SecretKey)
for valIdx := range validators {
privKey, err := bls.RandKey()
require.NoError(t, err)
privKeys[types.ValidatorIndex(valIdx)] = privKey
validators[valIdx] = &ethpb.Validator{
PublicKey: privKey.PublicKey().Marshal(),
WithdrawalCredentials: make([]byte, 32),
}
}
err = beaconState.SetValidators(validators)
require.NoError(t, err)
gen := stategen.NewMockService()
gen.AddStateForRoot(beaconState, [32]byte{})
return &Simulator{
srvConfig: &ServiceConfig{
Params: params,
Database: slasherDB,
AttestationStateFetcher: &mock.ChainService{State: beaconState},
PrivateKeysByValidatorIndex: privKeys,
StateGen: gen,
},
}
}