Refactor to Create DB Package (#582)

This commit is contained in:
Yutaro Mori
2018-10-06 02:14:50 +09:00
committed by Raul Jordan
parent 6bc655d9c1
commit ce99bd7605
39 changed files with 1467 additions and 2236 deletions

View File

@@ -2,37 +2,25 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
go_library(
name = "go_default_library",
srcs = [
"core.go",
"service.go",
],
srcs = ["service.go"],
importpath = "github.com/prysmaticlabs/prysm/beacon-chain/attestation",
visibility = ["//beacon-chain:__subpackages__"],
deps = [
"//beacon-chain/blockchain:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/types:go_default_library",
"//proto/beacon/p2p/v1:go_default_library",
"//shared/event:go_default_library",
"@com_github_ethereum_go_ethereum//ethdb:go_default_library",
"@com_github_gogo_protobuf//proto:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
],
)
go_test(
name = "go_default_test",
srcs = [
"core_test.go",
"service_test.go",
],
srcs = ["service_test.go"],
embed = [":go_default_library"],
deps = [
"//beacon-chain/blockchain:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/types:go_default_library",
"//proto/beacon/p2p/v1:go_default_library",
"//shared/database:go_default_library",
"//shared/testutil:go_default_library",
"@com_github_gogo_protobuf//proto:go_default_library",
"@com_github_sirupsen_logrus//hooks/test:go_default_library",
],
)

View File

@@ -1,144 +0,0 @@
package attestation
import (
"bytes"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/gogo/protobuf/proto"
"github.com/prysmaticlabs/prysm/beacon-chain/blockchain"
"github.com/prysmaticlabs/prysm/beacon-chain/types"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
)
// Handler represents the core attestation object
// containing a db.
type Handler struct {
db ethdb.Database
}
// NewHandler initializes an attestation handler for db interaction and more.
func NewHandler(db ethdb.Database) (*Handler, error) {
handler := &Handler{
db: db,
}
return handler, nil
}
// hasAttestation checks an attestation exists in beacon chain db by inputting its hash.
func (h *Handler) hasAttestation(attestationHash [32]byte) (bool, error) {
return h.db.Has(blockchain.AttestationKey(attestationHash))
}
// saveAttestation puts the attestation record into the beacon chain db.
func (h *Handler) saveAttestation(attestation *types.Attestation) error {
hash := attestation.Key()
key := blockchain.AttestationKey(hash)
encodedState, err := attestation.Marshal()
if err != nil {
return err
}
return h.db.Put(key, encodedState)
}
// getAttestation retrieves an attestation record from the db using its hash.
func (h *Handler) getAttestation(hash [32]byte) (*types.Attestation, error) {
key := blockchain.AttestationKey(hash)
enc, err := h.db.Get(key)
if err != nil {
return nil, err
}
attestation := &pb.AggregatedAttestation{}
err = proto.Unmarshal(enc, attestation)
return types.NewAttestation(attestation), err
}
// removeAttestation removes the attestation from the db.
func (h *Handler) removeAttestation(blockHash [32]byte) error {
return h.db.Delete(blockchain.AttestationKey(blockHash))
}
// hasAttestationHash checks if the beacon block has the attestation.
func (h *Handler) hasAttestationHash(blockHash [32]byte, attestationHash [32]byte) (bool, error) {
enc, err := h.db.Get(blockchain.AttestationHashListKey(blockHash))
if err != nil {
return false, err
}
attestationHashes := &pb.AttestationHashes{}
if err := proto.Unmarshal(enc, attestationHashes); err != nil {
return false, err
}
for _, hash := range attestationHashes.AttestationHash {
if bytes.Equal(hash, attestationHash[:]) {
return true, nil
}
}
return false, nil
}
// hasAttestationHashList checks if the attestation hash list is available.
func (h *Handler) hasAttestationHashList(blockHash [32]byte) (bool, error) {
key := blockchain.AttestationHashListKey(blockHash)
hasKey, err := h.db.Has(key)
if err != nil {
return false, err
}
return hasKey, nil
}
// getAttestationHashList gets the attestation hash list of the beacon block from the db.
func (h *Handler) getAttestationHashList(blockHash [32]byte) ([][]byte, error) {
key := blockchain.AttestationHashListKey(blockHash)
hasList, err := h.hasAttestationHashList(blockHash)
if err != nil {
return [][]byte{}, err
}
if !hasList {
if err := h.db.Put(key, []byte{}); err != nil {
return [][]byte{}, err
}
}
enc, err := h.db.Get(key)
if err != nil {
return [][]byte{}, err
}
attestationHashes := &pb.AttestationHashes{}
if err := proto.Unmarshal(enc, attestationHashes); err != nil {
return [][]byte{}, err
}
return attestationHashes.AttestationHash, nil
}
// removeAttestationHashList removes the attestation hash list of the beacon block from the db.
func (h *Handler) removeAttestationHashList(blockHash [32]byte) error {
return h.db.Delete(blockchain.AttestationHashListKey(blockHash))
}
// saveAttestationHash saves the attestation hash into the attestation hash list of the corresponding beacon block.
func (h *Handler) saveAttestationHash(blockHash [32]byte, attestationHash [32]byte) error {
key := blockchain.AttestationHashListKey(blockHash)
hashes, err := h.getAttestationHashList(blockHash)
if err != nil {
return err
}
hashes = append(hashes, attestationHash[:])
attestationHashes := &pb.AttestationHashes{}
attestationHashes.AttestationHash = hashes
encodedState, err := proto.Marshal(attestationHashes)
if err != nil {
return err
}
return h.db.Put(key, encodedState)
}

View File

@@ -1,140 +0,0 @@
package attestation
import (
"bytes"
"testing"
"github.com/gogo/protobuf/proto"
"github.com/prysmaticlabs/prysm/beacon-chain/blockchain"
"github.com/prysmaticlabs/prysm/beacon-chain/types"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
"github.com/prysmaticlabs/prysm/shared/database"
)
func startInMemoryAttestationDB(t *testing.T) (*Handler, *database.DB) {
config := &database.DBConfig{DataDir: "", Name: "", InMemory: true}
db, err := database.NewDB(config)
if err != nil {
t.Fatalf("unable to setup db: %v", err)
}
handler, err := NewHandler(db.DB())
if err != nil {
t.Fatalf("unable to setup beacon chain: %v", err)
}
return handler, db
}
func TestSaveAndRemoveAttestations(t *testing.T) {
b, db := startInMemoryAttestationDB(t)
defer db.Close()
attestation := types.NewAttestation(&pb.AggregatedAttestation{
Slot: 1,
ShardId: 1,
AttesterBitfield: []byte{'A'},
})
hash := attestation.Key()
if err := b.saveAttestation(attestation); err != nil {
t.Fatalf("unable to save attestation %v", err)
}
exist, err := b.hasAttestation(hash)
if err != nil {
t.Fatalf("unable to check attestation %v", err)
}
if !exist {
t.Fatal("saved attestation does not exist")
}
// Adding a different attestation with the same key
newAttestation := types.NewAttestation(&pb.AggregatedAttestation{
Slot: 2,
ShardId: 2,
AttesterBitfield: []byte{'B'},
})
key := blockchain.AttestationKey(hash)
marshalled, err := proto.Marshal(newAttestation.Proto())
if err != nil {
t.Fatal(err)
}
if err := b.db.Put(key, marshalled); err != nil {
t.Fatal(err)
}
returnedAttestation, err := b.getAttestation(hash)
if err != nil {
t.Fatalf("attestation is unable to be retrieved")
}
if returnedAttestation.SlotNumber() != newAttestation.SlotNumber() {
t.Errorf("slotnumber does not match for saved and retrieved attestation")
}
if !bytes.Equal(returnedAttestation.AttesterBitfield(), newAttestation.AttesterBitfield()) {
t.Errorf("attester bitfield does not match for saved and retrieved attester")
}
if err := b.removeAttestation(hash); err != nil {
t.Fatalf("error removing attestation %v", err)
}
if _, err := b.getAttestation(hash); err == nil {
t.Fatalf("attestation is able to be retrieved")
}
}
func TestSaveAndRemoveAttestationHashList(t *testing.T) {
b, db := startInMemoryAttestationDB(t)
defer db.Close()
block := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 0,
})
blockHash, err := block.Hash()
if err != nil {
t.Error(err)
}
attestation := types.NewAttestation(&pb.AggregatedAttestation{
Slot: 1,
ShardId: 1,
AttesterBitfield: []byte{'A'},
})
attestationHash := attestation.Key()
if err := b.saveAttestationHash(blockHash, attestationHash); err != nil {
t.Fatalf("unable to save attestation hash %v", err)
}
exist, err := b.hasAttestationHash(blockHash, attestationHash)
if err != nil {
t.Fatalf("unable to check for attestation hash %v", err)
}
if !exist {
t.Error("saved attestation hash does not exist")
}
// Negative test case: try with random attestation, exist should be false.
exist, err = b.hasAttestationHash(blockHash, [32]byte{'A'})
if err != nil {
t.Fatalf("unable to check for attestation hash %v", err)
}
if exist {
t.Error("attestation hash shouldn't have existed")
}
// Remove attestation list by deleting the block hash key.
if err := b.removeAttestationHashList(blockHash); err != nil {
t.Fatalf("remove attestation hash list failed %v", err)
}
// Negative test case: try with deleted block hash, this should fail.
_, err = b.hasAttestationHash(blockHash, attestationHash)
if err == nil {
t.Error("attestation hash should't have existed in DB")
}
}

View File

@@ -3,8 +3,8 @@ package attestation
import (
"context"
"fmt"
"github.com/prysmaticlabs/prysm/beacon-chain/db"
"github.com/prysmaticlabs/prysm/beacon-chain/types"
"github.com/prysmaticlabs/prysm/shared/event"
"github.com/sirupsen/logrus"
@@ -17,7 +17,7 @@ var log = logrus.WithField("prefix", "attestation")
type Service struct {
ctx context.Context
cancel context.CancelFunc
handler *Handler
beaconDB *db.BeaconDB
broadcastFeed *event.Feed
broadcastChan chan *types.Attestation
incomingFeed *event.Feed
@@ -26,19 +26,19 @@ type Service struct {
// Config options for the service.
type Config struct {
Handler *Handler
BeaconDB *db.BeaconDB
ReceiveAttestationBuf int
BroadcastAttestationBuf int
}
// NewAttestService instantiates a new service instance that will
// NewAttestationService instantiates a new service instance that will
// be registered into a running beacon node.
func NewAttestService(ctx context.Context, cfg *Config) *Service {
func NewAttestationService(ctx context.Context, cfg *Config) *Service {
ctx, cancel := context.WithCancel(ctx)
return &Service{
ctx: ctx,
cancel: cancel,
handler: cfg.Handler,
beaconDB: cfg.BeaconDB,
broadcastFeed: new(event.Feed),
broadcastChan: make(chan *types.Attestation, cfg.BroadcastAttestationBuf),
incomingFeed: new(event.Feed),
@@ -65,21 +65,6 @@ func (a *Service) IncomingAttestationFeed() *event.Feed {
return a.incomingFeed
}
// ContainsAttestation checks if an attestation has already been received and aggregated.
func (a *Service) ContainsAttestation(bitfield []byte, hash [32]byte) (bool, error) {
attestation, err := a.handler.getAttestation(hash)
if err != nil {
return false, fmt.Errorf("could not get attestation from DB: %v", err)
}
savedAttestationBitfield := attestation.AttesterBitfield()
for i := 0; i < len(bitfield); i++ {
if bitfield[i]&savedAttestationBitfield[i] != 0 {
return true, nil
}
}
return false, nil
}
// aggregateAttestations aggregates the newly broadcasted attestation that was
// received from sync service.
func (a *Service) aggregateAttestations() {
@@ -98,12 +83,12 @@ func (a *Service) aggregateAttestations() {
log.Errorf("Could not hash incoming attestation: %v", err)
continue
}
if err := a.handler.saveAttestation(attestation); err != nil {
if err := a.beaconDB.SaveAttestation(attestation); err != nil {
log.Errorf("Could not save attestation: %v", err)
continue
}
log.Infof("Forwarding aggregated attestation 0x%x to proposers through grpc", h)
log.Infof("Forwarding aggregated attestation %x to proposers through grpc", h)
}
}
}

View File

@@ -4,130 +4,41 @@ import (
"context"
"testing"
"github.com/prysmaticlabs/prysm/beacon-chain/db"
"github.com/prysmaticlabs/prysm/beacon-chain/types"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
"github.com/prysmaticlabs/prysm/shared/database"
"github.com/prysmaticlabs/prysm/shared/testutil"
logTest "github.com/sirupsen/logrus/hooks/test"
)
func TestStartStop(t *testing.T) {
hook := logTest.NewGlobal()
func setupService(t *testing.T) *Service {
ctx := context.Background()
config := &database.DBConfig{DataDir: "", Name: "", InMemory: true}
db, err := database.NewDB(config)
config := db.Config{Path: "", Name: "", InMemory: true}
db, err := db.NewDB(config)
if err != nil {
t.Fatalf("could not setup beaconDB: %v", err)
}
AttestHandler, err := NewHandler(db.DB())
if err != nil {
t.Fatalf("could not get attestation handler: %v", err)
}
cfg := &Config{
Handler: AttestHandler,
BeaconDB: db,
}
attestService := NewAttestService(ctx, cfg)
attestService.IncomingAttestationFeed()
attestService.ContainsAttestation([]byte{}, [32]byte{})
// Test the start function.
attestService.Start()
// Test the stop function.
if err := attestService.Stop(); err != nil {
t.Fatalf("unable to attest chain service: %v", err)
}
// The context should have been canceled.
if attestService.ctx.Err() == nil {
t.Error("context was not canceled")
}
testutil.AssertLogsContain(t, hook, "Starting service")
testutil.AssertLogsContain(t, hook, "Stopping service")
return NewAttestationService(ctx, cfg)
}
func TestIncomingAttestations(t *testing.T) {
hook := logTest.NewGlobal()
config := &database.DBConfig{DataDir: "", Name: "", InMemory: true}
db, err := database.NewDB(config)
if err != nil {
t.Fatalf("could not setup beaconDB: %v", err)
}
attestationHandler, err := NewHandler(db.DB())
if err != nil {
t.Fatalf("could not register blockchain service: %v", err)
}
cfg := &Config{
ReceiveAttestationBuf: 0,
Handler: attestationHandler,
}
attestationService := NewAttestService(context.Background(), cfg)
service := setupService(t)
exitRoutine := make(chan bool)
go func() {
attestationService.aggregateAttestations()
service.aggregateAttestations()
<-exitRoutine
}()
attestationService.incomingChan <- types.NewAttestation(nil)
attestationService.cancel()
service.incomingChan <- types.NewAttestation(nil)
service.cancel()
exitRoutine <- true
testutil.AssertLogsContain(t, hook, "Forwarding aggregated attestation")
}
func TestContainsAttestations(t *testing.T) {
config := &database.DBConfig{DataDir: "", Name: "", InMemory: true}
db, err := database.NewDB(config)
if err != nil {
t.Fatalf("could not setup beaconDB: %v", err)
}
attestationHandler, err := NewHandler(db.DB())
if err != nil {
t.Fatalf("could not register blockchain service: %v", err)
}
cfg := &Config{
ReceiveAttestationBuf: 0,
Handler: attestationHandler,
}
attestationService := NewAttestService(context.Background(), cfg)
attestation := types.NewAttestation(&pb.AggregatedAttestation{
Slot: 0,
ShardId: 0,
AttesterBitfield: []byte{7}, // 0000 0111
})
if err := attestationService.handler.saveAttestation(attestation); err != nil {
t.Fatalf("can not save attestation %v", err)
}
// Check if attestation exists for atteser bitfield 0000 0100
exists, err := attestationService.ContainsAttestation([]byte{4}, attestation.Key())
if err != nil {
t.Fatalf("can not call ContainsAttestation %v", err)
}
if !exists {
t.Error("Attestation should have existed")
}
// Check if attestation exists for atteser bitfield 0000 1000
exists, err = attestationService.ContainsAttestation([]byte{8}, attestation.Key())
if err != nil {
t.Fatalf("can not call ContainsAttestation %v", err)
}
if exists {
t.Error("Attestation shouldn't have existed")
}
}

View File

@@ -2,50 +2,37 @@ load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
go_library(
name = "go_default_library",
srcs = [
"core.go",
"schema.go",
"service.go",
],
srcs = ["service.go"],
importpath = "github.com/prysmaticlabs/prysm/beacon-chain/blockchain",
visibility = ["//beacon-chain:__subpackages__"],
deps = [
"//beacon-chain/db:go_default_library",
"//beacon-chain/params:go_default_library",
"//beacon-chain/powchain:go_default_library",
"//beacon-chain/types:go_default_library",
"//beacon-chain/utils:go_default_library",
"//proto/beacon/p2p/v1:go_default_library",
"//shared/event:go_default_library",
"@com_github_ethereum_go_ethereum//ethdb:go_default_library",
"@com_github_gogo_protobuf//proto:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
],
)
go_test(
name = "go_default_test",
srcs = [
"core_test.go",
"schema_test.go",
"service_test.go",
],
srcs = ["service_test.go"],
embed = [":go_default_library"],
race = "off", # TODO(#412): fix issues with tests failing with race on.
deps = [
"//beacon-chain/db:go_default_library",
"//beacon-chain/params:go_default_library",
"//beacon-chain/powchain:go_default_library",
"//beacon-chain/types:go_default_library",
"//proto/beacon/p2p/v1:go_default_library",
"//shared/database:go_default_library",
"//shared/event:go_default_library",
"//shared/testutil:go_default_library",
"@com_github_ethereum_go_ethereum//:go_default_library",
"@com_github_ethereum_go_ethereum//common:go_default_library",
"@com_github_ethereum_go_ethereum//core/types:go_default_library",
"@com_github_ethereum_go_ethereum//ethdb:go_default_library",
"@com_github_gogo_protobuf//proto:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
"@com_github_sirupsen_logrus//hooks/test:go_default_library",
"@io_bazel_rules_go//proto/wkt:timestamp_go_proto",
],
)

View File

@@ -1,277 +0,0 @@
package blockchain
import (
"errors"
"fmt"
"sync"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/gogo/protobuf/proto"
"github.com/prysmaticlabs/prysm/beacon-chain/types"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
)
// BeaconChain represents the core PoS blockchain object containing
// both a crystallized and active state.
type BeaconChain struct {
state *beaconState
lock sync.Mutex
db ethdb.Database
}
type beaconState struct {
// ActiveState captures the beacon state at block processing level,
// it focuses on verifying aggregated signatures and pending attestations.
ActiveState *types.ActiveState
// CrystallizedState captures the beacon state at cycle transition level,
// it focuses on changes to the validator set, processing cross links and
// setting up FFG checkpoints.
CrystallizedState *types.CrystallizedState
}
// NewBeaconChain initializes a beacon chain using genesis state parameters if
// none provided.
func NewBeaconChain(genesisJSON string, db ethdb.Database) (*BeaconChain, error) {
beaconChain := &BeaconChain{
db: db,
state: &beaconState{},
}
hasCrystallized, err := db.Has(crystallizedStateLookupKey)
if err != nil {
return nil, err
}
hasGenesis, err := db.Has(genesisLookupKey)
if err != nil {
return nil, err
}
active := types.NewGenesisActiveState()
crystallized, err := types.NewGenesisCrystallizedState(genesisJSON)
if err != nil {
return nil, err
}
beaconChain.state.ActiveState = active
if !hasGenesis {
log.Info("No genesis block found on disk, initializing genesis block")
// Active state hash is predefined so error can be safely ignored
// #nosec G104
activeStateHash, _ := active.Hash()
// Crystallized state hash is predefined so error can be safely ignored
// #nosec G104
crystallizedStateHash, _ := crystallized.Hash()
genesisBlock := types.NewGenesisBlock(activeStateHash, crystallizedStateHash)
genesisMarshall, err := proto.Marshal(genesisBlock.Proto())
if err != nil {
return nil, err
}
if err := beaconChain.db.Put(genesisLookupKey, genesisMarshall); err != nil {
return nil, err
}
if err := beaconChain.saveBlock(genesisBlock); err != nil {
return nil, err
}
}
if !hasCrystallized {
log.Info("No chainstate found on disk, initializing beacon from genesis")
beaconChain.state.CrystallizedState = crystallized
return beaconChain, nil
}
enc, err := db.Get(crystallizedStateLookupKey)
if err != nil {
return nil, err
}
crystallizedData := &pb.CrystallizedState{}
err = proto.Unmarshal(enc, crystallizedData)
if err != nil {
return nil, err
}
beaconChain.state.CrystallizedState = types.NewCrystallizedState(crystallizedData)
return beaconChain, nil
}
// genesisBlock returns the canonical, genesis block.
func (b *BeaconChain) genesisBlock() (*types.Block, error) {
genesisExists, err := b.db.Has(genesisLookupKey)
if err != nil {
return nil, err
}
if genesisExists {
bytes, err := b.db.Get(genesisLookupKey)
if err != nil {
return nil, err
}
block := &pb.BeaconBlock{}
if err := proto.Unmarshal(bytes, block); err != nil {
return nil, err
}
return types.NewBlock(block), nil
}
active := types.NewGenesisActiveState()
// Active state hash is predefined so error can be safely ignored
// #nosec G104
activeStateHash, _ := active.Hash()
// Crystallized state hash is predefined so error can be safely ignored
// #nosec G104
crystallizedStateHash, _ := b.CrystallizedState().Hash()
return types.NewGenesisBlock(activeStateHash, crystallizedStateHash), nil
}
// CanonicalHead fetches the latest head stored in persistent storage.
func (b *BeaconChain) CanonicalHead() (*types.Block, error) {
has, err := b.db.Has(canonicalHeadLookupKey)
if err != nil {
return nil, err
}
// If there has not been a canonical head stored yet, we
// return the genesis block of the chain.
if !has {
return b.genesisBlock()
}
bytes, err := b.db.Get(canonicalHeadLookupKey)
if err != nil {
return nil, err
}
block := &pb.BeaconBlock{}
if err := proto.Unmarshal(bytes, block); err != nil {
return nil, fmt.Errorf("cannot unmarshal proto: %v", err)
}
return types.NewBlock(block), nil
}
// ActiveState contains the current state of attestations and changes every block.
func (b *BeaconChain) ActiveState() *types.ActiveState {
return b.state.ActiveState
}
// CrystallizedState contains cycle dependent validator information, changes every cycle.
func (b *BeaconChain) CrystallizedState() *types.CrystallizedState {
return b.state.CrystallizedState
}
// SetActiveState is a convenience method which sets and persists the active state on the beacon chain.
func (b *BeaconChain) SetActiveState(activeState *types.ActiveState) error {
b.lock.Lock()
defer b.lock.Unlock()
b.state.ActiveState = activeState
return b.PersistActiveState()
}
// SetCrystallizedState is a convenience method which sets and persists the crystallized state on the beacon chain.
func (b *BeaconChain) SetCrystallizedState(crystallizedState *types.CrystallizedState) error {
b.lock.Lock()
defer b.lock.Unlock()
b.state.CrystallizedState = crystallizedState
return b.PersistCrystallizedState()
}
// PersistActiveState stores proto encoding of the current beacon chain active state into the db.
func (b *BeaconChain) PersistActiveState() error {
encodedState, err := b.ActiveState().Marshal()
if err != nil {
return err
}
return b.db.Put(activeStateLookupKey, encodedState)
}
// PersistCrystallizedState stores proto encoding of the current beacon chain crystallized state into the db.
func (b *BeaconChain) PersistCrystallizedState() error {
encodedState, err := b.CrystallizedState().Marshal()
if err != nil {
return err
}
return b.db.Put(crystallizedStateLookupKey, encodedState)
}
func (b *BeaconChain) hasBlock(blockhash [32]byte) (bool, error) {
return b.db.Has(blockKey(blockhash))
}
// saveBlock puts the passed block into the beacon chain db.
func (b *BeaconChain) saveBlock(block *types.Block) error {
hash, err := block.Hash()
if err != nil {
return err
}
key := blockKey(hash)
encodedState, err := block.Marshal()
if err != nil {
return err
}
return b.db.Put(key, encodedState)
}
// saveCanonicalSlotNumber saves the slotnumber and blockhash of a canonical block
// saved in the db. This will alow for canonical blocks to be retrieved from the db
// by using their slotnumber as a key, and then using the retrieved blockhash to get
// the block from the db.
// prefix + slotNumber -> blockhash
// prefix + blockHash -> block
func (b *BeaconChain) saveCanonicalSlotNumber(slotNumber uint64, hash [32]byte) error {
return b.db.Put(canonicalBlockKey(slotNumber), hash[:])
}
// saveCanonicalBlock puts the passed block into the beacon chain db
// and also saves a "latest-head" key mapping to the block in the db.
func (b *BeaconChain) saveCanonicalBlock(block *types.Block) error {
enc, err := block.Marshal()
if err != nil {
return err
}
return b.db.Put(canonicalHeadLookupKey, enc)
}
// getBlock retrieves a block from the db using its hash.
func (b *BeaconChain) getBlock(hash [32]byte) (*types.Block, error) {
key := blockKey(hash)
has, err := b.db.Has(key)
if err != nil {
return nil, err
}
if !has {
return nil, errors.New("block not found")
}
enc, err := b.db.Get(key)
if err != nil {
return nil, err
}
block := &pb.BeaconBlock{}
err = proto.Unmarshal(enc, block)
return types.NewBlock(block), err
}
// removeBlock removes the block from the db.
func (b *BeaconChain) removeBlock(hash [32]byte) error {
return b.db.Delete(blockKey(hash))
}
// hasCanonicalBlockForSlot checks the db if the canonical block for
// this slot exists.
func (b *BeaconChain) hasCanonicalBlockForSlot(slotNumber uint64) (bool, error) {
return b.db.Has(canonicalBlockKey(slotNumber))
}
// canonicalBlockForSlot retrieves the canonical block which is saved in the db
// for that required slot number.
func (b *BeaconChain) canonicalBlockForSlot(slotNumber uint64) (*types.Block, error) {
enc, err := b.db.Get(canonicalBlockKey(slotNumber))
if err != nil {
return nil, err
}
var blockhash [32]byte
copy(blockhash[:], enc)
block, err := b.getBlock(blockhash)
return block, err
}

View File

@@ -1,388 +0,0 @@
package blockchain
import (
"bytes"
"reflect"
"testing"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/gogo/protobuf/proto"
"github.com/golang/protobuf/ptypes/timestamp"
"github.com/prysmaticlabs/prysm/beacon-chain/types"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
"github.com/prysmaticlabs/prysm/shared/database"
logTest "github.com/sirupsen/logrus/hooks/test"
)
type faultyDB struct{}
func (f *faultyDB) Get(k []byte) ([]byte, error) {
return []byte{}, nil
}
func (f *faultyDB) Has(k []byte) (bool, error) {
return true, nil
}
func (f *faultyDB) Put(k []byte, v []byte) error {
return nil
}
func (f *faultyDB) Delete(k []byte) error {
return nil
}
func (f *faultyDB) Close() {}
func (f *faultyDB) NewBatch() ethdb.Batch {
return nil
}
func startInMemoryBeaconChain(t *testing.T) (*BeaconChain, *database.DB) {
config := &database.DBConfig{DataDir: "", Name: "", InMemory: true}
db, err := database.NewDB(config)
if err != nil {
t.Fatalf("unable to setup db: %v", err)
}
beaconChain, err := NewBeaconChain("", db.DB())
if err != nil {
t.Fatalf("unable to setup beacon chain: %v", err)
}
return beaconChain, db
}
func TestNewBeaconChain(t *testing.T) {
hook := logTest.NewGlobal()
beaconChain, db := startInMemoryBeaconChain(t)
defer db.Close()
msg := hook.LastEntry().Message
want := "No chainstate found on disk, initializing beacon from genesis"
if msg != want {
t.Errorf("incorrect log, expected %s, got %s", want, msg)
}
hook.Reset()
aState := types.NewGenesisActiveState()
cState, err := types.NewGenesisCrystallizedState("")
if err != nil {
t.Errorf("Creating new genesis state failed %v", err)
}
if !proto.Equal(beaconChain.ActiveState().Proto(), aState.Proto()) {
t.Errorf("active states not equal. received: %v, wanted: %v", beaconChain.ActiveState(), aState)
}
if !proto.Equal(beaconChain.CrystallizedState().Proto(), cState.Proto()) {
t.Errorf("crystallized states not equal. received: %v, wanted: %v", beaconChain.CrystallizedState(), cState)
}
if _, err := beaconChain.genesisBlock(); err != nil {
t.Errorf("Getting new beaconchain genesis failed: %v", err)
}
}
func TestGetGenesisBlock(t *testing.T) {
beaconChain, db := startInMemoryBeaconChain(t)
defer db.Close()
block := &pb.BeaconBlock{
ParentHash: make([]byte, 32),
Timestamp: &timestamp.Timestamp{
Seconds: 13000000,
},
}
bytes, err := proto.Marshal(block)
if err != nil {
t.Errorf("unable to Marshal genesis block: %v", err)
}
if err := db.DB().Put([]byte("genesis"), bytes); err != nil {
t.Errorf("unable to save key value of genesis: %v", err)
}
genesisBlock, err := beaconChain.genesisBlock()
if err != nil {
t.Errorf("unable to get key value of genesis: %v", err)
}
time, err := genesisBlock.Timestamp()
if err != nil {
t.Errorf("Timestamp could not be retrieved: %v", err)
}
if time.Second() != 40 {
t.Errorf("Timestamp was not saved properly: %v", time.Second())
}
}
func TestGetGenesisBlock_GenesisNotExist(t *testing.T) {
beaconChain, db := startInMemoryBeaconChain(t)
defer db.Close()
if err := db.DB().Delete([]byte("genesis")); err != nil {
t.Errorf("unable to delete key value of genesis: %v", err)
}
genesisBlock, err := beaconChain.genesisBlock()
if err != nil {
t.Errorf("unable to get key value of genesis: %v", err)
}
time, err := genesisBlock.Timestamp()
if err != nil {
t.Errorf("Timestamp could not be retrieved: %v", err)
}
if time.Unix() != 1535673600 {
t.Errorf("Timestamp was not saved properly: %v", time.Second())
}
}
func TestCanonicalHead(t *testing.T) {
chain, err := NewBeaconChain("", &faultyDB{})
if err != nil {
t.Fatalf("unable to setup second beacon chain: %v", err)
}
// Using a faultydb that returns true on has, but nil on get should cause
// proto.Unmarshal to throw error.
block, err := chain.CanonicalHead()
if err != nil {
t.Fatal("expected canonical head to throw error")
}
expectedBlock := types.NewBlock(&pb.BeaconBlock{})
if !reflect.DeepEqual(block, expectedBlock) {
t.Errorf("mismatched canonical head: expected %v, received %v", expectedBlock, block)
}
}
func TestSaveCanonicalBlock(t *testing.T) {
block := types.NewBlock(&pb.BeaconBlock{})
chain, err := NewBeaconChain("", &faultyDB{})
if err != nil {
t.Fatalf("unable to setup second beacon chain: %v", err)
}
if err := chain.saveCanonicalBlock(block); err != nil {
t.Errorf("save canonical should pass: %v", err)
}
}
func TestSetActiveState(t *testing.T) {
beaconChain, db := startInMemoryBeaconChain(t)
defer db.Close()
data := &pb.ActiveState{
PendingAttestations: []*pb.AggregatedAttestation{
{Slot: 0, ShardBlockHash: []byte{1}}, {Slot: 1, ShardBlockHash: []byte{2}},
},
RecentBlockHashes: [][]byte{
{'A'}, {'B'}, {'C'}, {'D'},
},
}
active := types.NewActiveState(data, make(map[[32]byte]*types.VoteCache))
if err := beaconChain.SetActiveState(active); err != nil {
t.Fatalf("unable to mutate active state: %v", err)
}
if !reflect.DeepEqual(beaconChain.state.ActiveState, active) {
t.Errorf("active state was not updated. wanted %v, got %v", active, beaconChain.state.ActiveState)
}
}
func TestSetCrystallizedState(t *testing.T) {
beaconChain, db := startInMemoryBeaconChain(t)
defer db.Close()
data := &pb.CrystallizedState{
CurrentDynasty: 3,
DynastySeed: []byte{'A'},
}
crystallized := types.NewCrystallizedState(data)
if err := beaconChain.SetCrystallizedState(crystallized); err != nil {
t.Fatalf("unable to mutate crystallized state: %v", err)
}
if !reflect.DeepEqual(beaconChain.state.CrystallizedState, crystallized) {
t.Errorf("crystallized state was not updated. wanted %v, got %v", crystallized, beaconChain.state.CrystallizedState)
}
// Initializing a new beacon chain should deserialize persisted state from disk.
newBeaconChain, err := NewBeaconChain("", db.DB())
if err != nil {
t.Fatalf("unable to setup second beacon chain: %v", err)
}
// The crystallized state should still be the one we mutated and persited earlier.
if crystallized.CurrentDynasty() != newBeaconChain.state.CrystallizedState.CurrentDynasty() {
t.Errorf("crystallized state dynasty incorrect. wanted %v, got %v", crystallized.CurrentDynasty(), newBeaconChain.state.CrystallizedState.CurrentDynasty())
}
if crystallized.DynastySeed() != newBeaconChain.state.CrystallizedState.DynastySeed() {
t.Errorf("crystallized state current checkpoint incorrect. wanted %v, got %v", crystallized.DynastySeed(), newBeaconChain.state.CrystallizedState.DynastySeed())
}
}
func TestSaveAndRemoveBlocks(t *testing.T) {
b, db := startInMemoryBeaconChain(t)
defer db.Close()
block := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 64,
PowChainRef: []byte("a"),
})
hash, err := block.Hash()
if err != nil {
t.Fatalf("unable to generate hash of block %v", err)
}
if err := b.saveBlock(block); err != nil {
t.Fatalf("unable to save block %v", err)
}
// Adding a different block with the same key
newblock := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 4,
PowChainRef: []byte("b"),
})
key := blockKey(hash)
marshalled, err := proto.Marshal(newblock.Proto())
if err != nil {
t.Fatal(err)
}
if err := b.db.Put(key, marshalled); err != nil {
t.Fatal(err)
}
retblock, err := b.getBlock(hash)
if err != nil {
t.Fatalf("block is unable to be retrieved")
}
if retblock.SlotNumber() != newblock.SlotNumber() {
t.Errorf("slotnumber does not match for saved and retrieved blocks")
}
if !bytes.Equal(retblock.PowChainRef().Bytes(), newblock.PowChainRef().Bytes()) {
t.Errorf("POW chain ref does not match for saved and retrieved blocks")
}
if err := b.removeBlock(hash); err != nil {
t.Fatalf("error removing block %v", err)
}
if _, err := b.getBlock(hash); err == nil {
t.Fatalf("block is able to be retrieved")
}
if err := b.removeBlock(hash); err != nil {
t.Fatalf("unable to remove block a second time %v", err)
}
}
func TestCheckBlockBySlotNumber(t *testing.T) {
beaconChain, db := startInMemoryBeaconChain(t)
defer db.Close()
block := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 64,
PowChainRef: []byte("a"),
})
hash, err := block.Hash()
if err != nil {
t.Error(err)
}
if err := beaconChain.saveCanonicalSlotNumber(block.SlotNumber(), hash); err != nil {
t.Fatalf("unable to save canonical slot %v", err)
}
if err := beaconChain.saveBlock(block); err != nil {
t.Fatalf("unable to save block %v", err)
}
slotExists, err := beaconChain.hasCanonicalBlockForSlot(block.SlotNumber())
if err != nil {
t.Fatalf("unable to check for block by slot %v", err)
}
if !slotExists {
t.Error("slot does not exist despite blockhash of canonical block being saved in the db")
}
alternateblock := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 64,
PowChainRef: []byte("d"),
})
althash, err := alternateblock.Hash()
if err != nil {
t.Fatalf("unable to hash block %v", err)
}
if err := beaconChain.saveCanonicalSlotNumber(block.SlotNumber(), althash); err != nil {
t.Fatalf("unable to save canonical slot %v", err)
}
retrievedHash, err := beaconChain.db.Get(canonicalBlockKey(block.SlotNumber()))
if err != nil {
t.Fatalf("unable to retrieve blockhash %v", err)
}
if !bytes.Equal(retrievedHash, althash[:]) {
t.Errorf("unequal hashes between what was saved and what was retrieved %v, %v", retrievedHash, althash)
}
}
func TestGetBlockBySlotNumber(t *testing.T) {
beaconChain, db := startInMemoryBeaconChain(t)
defer db.Close()
block := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 64,
PowChainRef: []byte("a"),
})
hash, err := block.Hash()
if err != nil {
t.Error(err)
}
if err := beaconChain.saveCanonicalSlotNumber(block.SlotNumber(), hash); err != nil {
t.Fatalf("unable to save canonical slot %v", err)
}
if err := beaconChain.saveBlock(block); err != nil {
t.Fatalf("unable to save block %v", err)
}
retblock, err := beaconChain.canonicalBlockForSlot(block.SlotNumber())
if err != nil {
t.Fatalf("unable to get block from db %v", err)
}
if !bytes.Equal(retblock.PowChainRef().Bytes(), block.PowChainRef().Bytes()) {
t.Error("canonical block saved different from block retrieved")
}
alternateblock := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 64,
PowChainRef: []byte("d"),
})
althash, err := alternateblock.Hash()
if err != nil {
t.Fatalf("unable to hash block %v", err)
}
if err := beaconChain.saveCanonicalSlotNumber(block.SlotNumber(), althash); err != nil {
t.Fatalf("unable to save canonical slot %v", err)
}
if _, err = beaconChain.canonicalBlockForSlot(block.SlotNumber()); err == nil {
t.Fatal("there should be an error because block does not exist in the db")
}
}

View File

@@ -4,11 +4,10 @@ package blockchain
import (
"context"
"fmt"
"math"
"sync"
"time"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/prysmaticlabs/prysm/beacon-chain/db"
"github.com/prysmaticlabs/prysm/beacon-chain/params"
"github.com/prysmaticlabs/prysm/beacon-chain/powchain"
"github.com/prysmaticlabs/prysm/beacon-chain/types"
@@ -24,8 +23,7 @@ var log = logrus.WithField("prefix", "blockchain")
type ChainService struct {
ctx context.Context
cancel context.CancelFunc
beaconDB ethdb.Database
chain *BeaconChain
beaconDB *db.BeaconDB
web3Service *powchain.Web3Service
incomingBlockFeed *event.Feed
incomingBlockChan chan *types.Block
@@ -45,9 +43,8 @@ type ChainService struct {
type Config struct {
BeaconBlockBuf int
IncomingBlockBuf int
Chain *BeaconChain
Web3Service *powchain.Web3Service
BeaconDB ethdb.Database
BeaconDB *db.BeaconDB
DevMode bool
EnableCrossLinks bool
EnableRewardChecking bool
@@ -61,7 +58,6 @@ func NewChainService(ctx context.Context, cfg *Config) (*ChainService, error) {
ctx, cancel := context.WithCancel(ctx)
return &ChainService{
ctx: ctx,
chain: cfg.Chain,
cancel: cancel,
beaconDB: cfg.BeaconDB,
web3Service: cfg.Web3Service,
@@ -84,7 +80,7 @@ func (c *ChainService) Start() {
// to truly continue across sessions.
log.Info("Starting service")
genesis, err := c.GenesisBlock()
genesis, err := c.beaconDB.GetCanonicalBlockForSlot(0)
if err != nil {
log.Fatalf("Could not get genesis block: %v", err)
}
@@ -111,82 +107,15 @@ func (c *ChainService) Start() {
func (c *ChainService) Stop() error {
defer c.cancel()
log.Info("Stopping service")
log.Info("Persisting current active and crystallized states before closing")
if err := c.chain.PersistActiveState(); err != nil {
return fmt.Errorf("Error persisting active state: %v", err)
}
if err := c.chain.PersistCrystallizedState(); err != nil {
return fmt.Errorf("Error persisting crystallized state: %v", err)
}
return nil
}
// CurrentBeaconSlot based on the seconds since genesis.
func (c *ChainService) CurrentBeaconSlot() uint64 {
secondsSinceGenesis := time.Since(c.genesisTimestamp).Seconds()
if secondsSinceGenesis-float64(params.GetConfig().SlotDuration) < 0 {
return 0
}
return uint64(math.Floor(secondsSinceGenesis/float64(params.GetConfig().SlotDuration))) - 1
}
// CanonicalHead of the current beacon chain.
func (c *ChainService) CanonicalHead() (*types.Block, error) {
return c.chain.CanonicalHead()
}
// CanonicalCrystallizedState of the current beacon chain's head.
func (c *ChainService) CanonicalCrystallizedState() *types.CrystallizedState {
return c.chain.CrystallizedState()
}
// IncomingBlockFeed returns a feed that any service can send incoming p2p blocks into.
// The chain service will subscribe to this feed in order to process incoming blocks.
func (c *ChainService) IncomingBlockFeed() *event.Feed {
return c.incomingBlockFeed
}
// HasStoredState checks if there is any Crystallized/Active State or blocks(not implemented) are
// persisted to the db.
func (c *ChainService) HasStoredState() (bool, error) {
hasCrystallized, err := c.beaconDB.Has(crystallizedStateLookupKey)
if err != nil {
return false, err
}
return hasCrystallized, nil
}
// SaveBlock is a mock which saves a block to the local db using the
// blockhash as the key.
func (c *ChainService) SaveBlock(block *types.Block) error {
return c.chain.saveBlock(block)
}
// ContainsBlock checks if a block for the hash exists in the chain.
// This method must be safe to call from a goroutine.
func (c *ChainService) ContainsBlock(h [32]byte) (bool, error) {
return c.chain.hasBlock(h)
}
// BlockSlotNumberByHash returns the slot number of a block.
func (c *ChainService) BlockSlotNumberByHash(h [32]byte) (uint64, error) {
block, err := c.chain.getBlock(h)
if err != nil {
return 0, fmt.Errorf("could not get block from DB: %v", err)
}
return block.SlotNumber(), nil
}
// CurrentCrystallizedState of the canonical chain.
func (c *ChainService) CurrentCrystallizedState() *types.CrystallizedState {
return c.chain.CrystallizedState()
}
// CurrentActiveState of the canonical chain.
func (c *ChainService) CurrentActiveState() *types.ActiveState {
return c.chain.ActiveState()
}
// CanonicalBlockFeed returns a channel that is written to
// whenever a new block is determined to be canonical in the chain.
func (c *ChainService) CanonicalBlockFeed() *event.Feed {
@@ -199,23 +128,6 @@ func (c *ChainService) CanonicalCrystallizedStateFeed() *event.Feed {
return c.canonicalCrystallizedStateFeed
}
// CheckForCanonicalBlockBySlot checks if the canonical block for that slot exists
// in the db.
func (c *ChainService) CheckForCanonicalBlockBySlot(slotNumber uint64) (bool, error) {
return c.chain.hasCanonicalBlockForSlot(slotNumber)
}
// CanonicalBlockBySlotNumber retrieves the canonical block for that slot which
// has been saved in the db.
func (c *ChainService) CanonicalBlockBySlotNumber(slotNumber uint64) (*types.Block, error) {
return c.chain.canonicalBlockForSlot(slotNumber)
}
// GenesisBlock returns the contents of the genesis block.
func (c *ChainService) GenesisBlock() (*types.Block, error) {
return c.chain.genesisBlock()
}
// doesPoWBlockExist checks if the referenced PoW block exists.
func (c *ChainService) doesPoWBlockExist(block *types.Block) bool {
powBlock, err := c.web3Service.Client().BlockByHash(context.Background(), block.PowChainRef())
@@ -238,7 +150,7 @@ func (c *ChainService) updateHead(slotInterval <-chan time.Time) {
case <-c.ctx.Done():
return
case <-slotInterval:
log.WithField("slotNumber", c.CurrentBeaconSlot()).Info("New beacon slot")
log.WithField("slotNumber", utils.CurrentSlot(c.genesisTimestamp)).Info("New beacon slot")
// First, we check if there were any blocks processed in the previous slot.
// If there is, we fetch the first one from the DB.
@@ -248,7 +160,7 @@ func (c *ChainService) updateHead(slotInterval <-chan time.Time) {
// Naive fork choice rule: we pick the first block we processed for the previous slot
// as canonical.
block, err := c.chain.getBlock(c.blocksPendingProcessing[0])
block, err := c.beaconDB.GetBlock(c.blocksPendingProcessing[0])
if err != nil {
log.Errorf("Could not get block: %v", err)
continue
@@ -262,14 +174,14 @@ func (c *ChainService) updateHead(slotInterval <-chan time.Time) {
log.Info("Applying fork choice rule")
parentBlock, err := c.chain.getBlock(block.ParentHash())
parentBlock, err := c.beaconDB.GetBlock(block.ParentHash())
if err != nil {
log.Errorf("Failed to get parent of block 0x%x", h)
log.Errorf("Failed to get parent of block %x", h)
continue
}
cState := c.chain.CrystallizedState()
aState := c.chain.ActiveState()
cState := c.beaconDB.GetCrystallizedState()
aState := c.beaconDB.GetActiveState()
var stateTransitioned bool
for cState.IsCycleTransition(parentBlock.SlotNumber()) {
@@ -297,26 +209,26 @@ func (c *ChainService) updateHead(slotInterval <-chan time.Time) {
continue
}
if err := c.chain.SetActiveState(aState); err != nil {
if err := c.beaconDB.SaveActiveState(aState); err != nil {
log.Errorf("Write active state to disk failed: %v", err)
continue
}
if stateTransitioned {
if err := c.chain.SetCrystallizedState(cState); err != nil {
if err := c.beaconDB.SaveCrystallizedState(cState); err != nil {
log.Errorf("Write crystallized state to disk failed: %v", err)
continue
}
}
// Save canonical block hash with slot number to DB.
if err := c.chain.saveCanonicalSlotNumber(block.SlotNumber(), h); err != nil {
if err := c.beaconDB.SaveCanonicalSlotNumber(block.SlotNumber(), h); err != nil {
log.Errorf("Unable to save slot number to db: %v", err)
continue
}
// Save canonical block to DB.
if err := c.chain.saveCanonicalBlock(block); err != nil {
if err := c.beaconDB.SaveCanonicalBlock(block); err != nil {
log.Errorf("Unable to save block to db: %v", err)
continue
}
@@ -363,7 +275,7 @@ func (c *ChainService) blockProcessing() {
}
// Check if we have received the parent block.
parentExists, err := c.chain.hasBlock(block.ParentHash())
parentExists, err := c.beaconDB.HasBlock(block.ParentHash())
if err != nil {
log.Errorf("Could not check existence of parent: %v", err)
continue
@@ -372,17 +284,17 @@ func (c *ChainService) blockProcessing() {
log.Debugf("Block points to nil parent: %v", err)
continue
}
parent, err := c.chain.getBlock(block.ParentHash())
parent, err := c.beaconDB.GetBlock(block.ParentHash())
if err != nil {
log.Debugf("Could not get parent block: %v", err)
continue
}
aState := c.chain.ActiveState()
cState := c.chain.CrystallizedState()
aState := c.beaconDB.GetActiveState()
cState := c.beaconDB.GetCrystallizedState()
if valid := block.IsValid(
c,
c.beaconDB,
aState,
cState,
parent.SlotNumber(),
@@ -393,7 +305,7 @@ func (c *ChainService) blockProcessing() {
continue
}
if err := c.chain.saveBlock(block); err != nil {
if err := c.beaconDB.SaveBlock(block); err != nil {
log.Errorf("Failed to save block: %v", err)
continue
}

View File

@@ -3,21 +3,21 @@ package blockchain
import (
"context"
"errors"
"fmt"
"io/ioutil"
"math"
"math/big"
"strings"
"testing"
"time"
ethereum "github.com/ethereum/go-ethereum"
"github.com/ethereum/go-ethereum/common"
gethTypes "github.com/ethereum/go-ethereum/core/types"
"github.com/prysmaticlabs/prysm/beacon-chain/db"
"github.com/prysmaticlabs/prysm/beacon-chain/params"
"github.com/prysmaticlabs/prysm/beacon-chain/powchain"
"github.com/prysmaticlabs/prysm/beacon-chain/types"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
"github.com/prysmaticlabs/prysm/shared/database"
"github.com/prysmaticlabs/prysm/shared/event"
"github.com/prysmaticlabs/prysm/shared/testutil"
"github.com/sirupsen/logrus"
@@ -70,26 +70,31 @@ func init() {
logrus.SetLevel(logrus.DebugLevel)
}
func TestStartStop(t *testing.T) {
ctx := context.Background()
config := &database.DBConfig{DataDir: "", Name: "", InMemory: true}
db, err := database.NewDB(config)
func setupBeaconChain(t *testing.T, faultyPoWClient bool) *ChainService {
config := db.Config{Path: "", Name: "", InMemory: true}
db, err := db.NewDB(config)
if err != nil {
t.Fatalf("could not setup beaconDB: %v", err)
}
endpoint := "ws://127.0.0.1"
client := &mockClient{}
web3Service, err := powchain.NewWeb3Service(ctx, &powchain.Web3ServiceConfig{Endpoint: endpoint, Pubkey: "", VrcAddr: common.Address{}}, client, client, client)
ctx := context.Background()
var web3Service *powchain.Web3Service
if faultyPoWClient {
client := &faultyClient{}
web3Service, err = powchain.NewWeb3Service(ctx, &powchain.Web3ServiceConfig{Endpoint: endpoint, Pubkey: "", VrcAddr: common.Address{}}, client, client, client)
} else {
client := &mockClient{}
web3Service, err = powchain.NewWeb3Service(ctx, &powchain.Web3ServiceConfig{Endpoint: endpoint, Pubkey: "", VrcAddr: common.Address{}}, client, client, client)
}
if err != nil {
t.Fatalf("unable to set up web3 service: %v", err)
}
beaconChain, err := NewBeaconChain("", db.DB())
cfg := &Config{
BeaconBlockBuf: 0,
BeaconDB: db.DB(),
Chain: beaconChain,
BeaconDB: db,
Web3Service: web3Service,
EnablePOWChain: true,
}
if err != nil {
t.Fatalf("could not register blockchain service: %v", err)
@@ -99,95 +104,18 @@ func TestStartStop(t *testing.T) {
t.Fatalf("unable to setup chain service: %v", err)
}
chainService.slotAlignmentDuration = 0
return chainService
}
func TestStartStop(t *testing.T) {
chainService := setupBeaconChain(t, false)
defer chainService.beaconDB.Close()
chainService.IncomingBlockFeed()
chainService.CanonicalBlockBySlotNumber(0)
chainService.CheckForCanonicalBlockBySlot(0)
chainService.CanonicalHead()
chainService.CanonicalCrystallizedState()
// Test the start function.
chainService.Start()
cfg = &Config{
BeaconBlockBuf: 0,
BeaconDB: db.DB(),
Chain: beaconChain,
Web3Service: web3Service,
}
chainService, err = NewChainService(ctx, cfg)
if err != nil {
t.Fatalf("unable to setup chain service: %v", err)
}
chainService.slotAlignmentDuration = 0
chainService.Start()
if len(chainService.CurrentActiveState().RecentBlockHashes()) != 128 {
t.Errorf("incorrect recent block hashes")
}
if len(chainService.CurrentCrystallizedState().Validators()) != params.GetConfig().BootstrappedValidatorsCount {
t.Errorf("incorrect default validator size")
}
blockExists, err := chainService.ContainsBlock([32]byte{})
if err != nil {
t.Fatalf("unable to check if block exists: %v", err)
}
if blockExists {
t.Errorf("chain is not empty")
}
hasState, err := chainService.HasStoredState()
if err != nil {
t.Fatalf("calling HasStoredState failed")
}
if hasState {
t.Errorf("has stored state should return false")
}
chainService.CanonicalBlockFeed()
chainService.CanonicalCrystallizedStateFeed()
chainService, _ = NewChainService(ctx, cfg)
active := types.NewActiveState(&pb.ActiveState{RecentBlockHashes: [][]byte{{'A'}}}, make(map[[32]byte]*types.VoteCache))
activeStateHash, err := active.Hash()
if err != nil {
t.Fatalf("Cannot hash active state: %v", err)
}
chainService.chain.SetActiveState(active)
crystallized := types.NewCrystallizedState(&pb.CrystallizedState{LastStateRecalc: 10000})
crystallizedStateHash, err := crystallized.Hash()
if err != nil {
t.Fatalf("Cannot hash crystallized state: %v", err)
}
chainService.chain.SetCrystallizedState(crystallized)
parentBlock := types.NewBlock(nil)
parentHash, _ := parentBlock.Hash()
block := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 2,
ActiveStateHash: activeStateHash[:],
CrystallizedStateHash: crystallizedStateHash[:],
ParentHash: parentHash[:],
PowChainRef: []byte("a"),
})
if err := chainService.SaveBlock(block); err != nil {
t.Errorf("save block should have failed")
}
// Save states so HasStoredState state should return true.
chainService.chain.SetActiveState(types.NewActiveState(&pb.ActiveState{}, make(map[[32]byte]*types.VoteCache)))
chainService.chain.SetCrystallizedState(types.NewCrystallizedState(&pb.CrystallizedState{}))
hasState, _ = chainService.HasStoredState()
if !hasState {
t.Errorf("has stored state should return false")
}
if err := chainService.Stop(); err != nil {
t.Fatalf("unable to stop chain service: %v", err)
}
@@ -198,126 +126,10 @@ func TestStartStop(t *testing.T) {
}
}
func TestFaultyStop(t *testing.T) {
ctx := context.Background()
config := &database.DBConfig{DataDir: "", Name: "", InMemory: true}
db, err := database.NewDB(config)
if err != nil {
t.Fatalf("could not setup beaconDB: %v", err)
}
endpoint := "ws://127.0.0.1"
client := &mockClient{}
web3Service, err := powchain.NewWeb3Service(ctx, &powchain.Web3ServiceConfig{Endpoint: endpoint, Pubkey: "", VrcAddr: common.Address{}}, client, client, client)
if err != nil {
t.Fatalf("unable to set up web3 service: %v", err)
}
beaconChain, err := NewBeaconChain("", db.DB())
if err != nil {
t.Fatalf("could not register blockchain service: %v", err)
}
cfg := &Config{
BeaconBlockBuf: 0,
BeaconDB: db.DB(),
Chain: beaconChain,
Web3Service: web3Service,
}
chainService, err := NewChainService(ctx, cfg)
if err != nil {
t.Fatalf("unable to setup chain service: %v", err)
}
chainService.slotAlignmentDuration = 0
chainService.Start()
chainService.chain.SetActiveState(types.NewActiveState(nil, make(map[[32]byte]*types.VoteCache)))
err = chainService.Stop()
if err == nil {
t.Errorf("chain stop should have failed with persist active state")
}
chainService.chain.SetActiveState(types.NewActiveState(&pb.ActiveState{}, make(map[[32]byte]*types.VoteCache)))
chainService.chain.SetCrystallizedState(types.NewCrystallizedState(nil))
err = chainService.Stop()
if err == nil {
t.Errorf("chain stop should have failed with persist crystallized state")
}
}
func TestCurrentBeaconSlot(t *testing.T) {
ctx := context.Background()
config := &database.DBConfig{DataDir: "", Name: "", InMemory: true}
db, err := database.NewDB(config)
if err != nil {
t.Fatalf("could not setup beaconDB: %v", err)
}
endpoint := "ws://127.0.0.1"
client := &faultyClient{}
web3Service, err := powchain.NewWeb3Service(
ctx,
&powchain.Web3ServiceConfig{
Endpoint: endpoint,
Pubkey: "",
VrcAddr: common.Address{},
},
client,
client,
client,
)
if err != nil {
t.Fatalf("unable to set up web3 service: %v", err)
}
beaconChain, err := NewBeaconChain("", db.DB())
if err != nil {
t.Fatalf("could not register blockchain service: %v", err)
}
cfg := &Config{
BeaconBlockBuf: 0,
BeaconDB: db.DB(),
Chain: beaconChain,
Web3Service: web3Service,
}
chainService, _ := NewChainService(ctx, cfg)
chainService.genesisTimestamp = time.Now()
if chainService.CurrentBeaconSlot() != 0 {
t.Errorf("Expected us to be in the 0th slot, received %v", chainService.CurrentBeaconSlot())
}
}
func TestRunningChainServiceFaultyPOWChain(t *testing.T) {
hook := logTest.NewGlobal()
ctx := context.Background()
config := &database.DBConfig{DataDir: "", Name: "", InMemory: true}
db, err := database.NewDB(config)
if err != nil {
t.Fatalf("could not setup beaconDB: %v", err)
}
endpoint := "ws://127.0.0.1"
client := &faultyClient{}
web3Service, err := powchain.NewWeb3Service(ctx, &powchain.Web3ServiceConfig{Endpoint: endpoint, Pubkey: "", VrcAddr: common.Address{}}, client, client, client)
if err != nil {
t.Fatalf("unable to set up web3 service: %v", err)
}
beaconChain, err := NewBeaconChain("", db.DB())
if err != nil {
t.Fatalf("could not register blockchain service: %v", err)
}
cfg := &Config{
BeaconBlockBuf: 0,
BeaconDB: db.DB(),
Chain: beaconChain,
Web3Service: web3Service,
EnablePOWChain: true,
}
chainService, _ := NewChainService(ctx, cfg)
chainService := setupBeaconChain(t, true)
defer chainService.beaconDB.Close()
block := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 1,
@@ -330,7 +142,7 @@ func TestRunningChainServiceFaultyPOWChain(t *testing.T) {
<-exitRoutine
}()
if err := chainService.SaveBlock(block); err != nil {
if err := chainService.beaconDB.SaveBlock(block); err != nil {
t.Fatal(err)
}
@@ -343,23 +155,9 @@ func TestRunningChainServiceFaultyPOWChain(t *testing.T) {
func TestRunningChainService(t *testing.T) {
hook := logTest.NewGlobal()
ctx := context.Background()
config := &database.DBConfig{DataDir: "", Name: "", InMemory: true}
db, err := database.NewDB(config)
if err != nil {
t.Fatalf("could not setup beaconDB: %v", err)
}
endpoint := "ws://127.0.0.1"
client := &mockClient{}
web3Service, err := powchain.NewWeb3Service(ctx, &powchain.Web3ServiceConfig{Endpoint: endpoint, Pubkey: "", VrcAddr: common.Address{}}, client, client, client)
if err != nil {
t.Fatalf("unable to set up web3 service: %v", err)
}
beaconChain, err := NewBeaconChain("", db.DB())
if err != nil {
t.Fatalf("could not register blockchain service: %v", err)
}
chainService := setupBeaconChain(t, false)
defer chainService.beaconDB.Close()
active := types.NewGenesisActiveState()
crystallized, err := types.NewGenesisCrystallizedState("")
@@ -370,19 +168,8 @@ func TestRunningChainService(t *testing.T) {
activeStateHash, _ := active.Hash()
crystallizedStateHash, _ := crystallized.Hash()
cfg := &Config{
BeaconBlockBuf: 0,
BeaconDB: db.DB(),
Chain: beaconChain,
Web3Service: web3Service,
}
chainService, _ := NewChainService(ctx, cfg)
genesis, err := beaconChain.genesisBlock()
if err != nil {
t.Fatalf("unable to get canonical head: %v", err)
}
beaconChain.saveBlock(genesis)
genesis := types.NewGenesisBlock([32]byte{}, [32]byte{})
chainService.beaconDB.SaveBlock(genesis)
parentHash, err := genesis.Hash()
if err != nil {
t.Fatalf("unable to get hash of canonical head: %v", err)
@@ -420,7 +207,7 @@ func TestRunningChainService(t *testing.T) {
<-exitRoutine
}()
if err := chainService.SaveBlock(block); err != nil {
if err := chainService.beaconDB.SaveBlock(block); err != nil {
t.Fatal(err)
}
@@ -433,84 +220,11 @@ func TestRunningChainService(t *testing.T) {
testutil.AssertLogsContain(t, hook, "Finished processing received block")
}
func TestBlockSlotNumberByHash(t *testing.T) {
ctx := context.Background()
config := &database.DBConfig{DataDir: "", Name: "", InMemory: true}
db, err := database.NewDB(config)
if err != nil {
t.Fatalf("could not setup beaconDB: %v", err)
}
endpoint := "ws://127.0.0.1"
client := &mockClient{}
web3Service, err := powchain.NewWeb3Service(ctx, &powchain.Web3ServiceConfig{Endpoint: endpoint, Pubkey: "", VrcAddr: common.Address{}}, client, client, client)
if err != nil {
t.Fatalf("unable to set up web3 service: %v", err)
}
beaconChain, err := NewBeaconChain("", db.DB())
if err != nil {
t.Fatalf("could not register blockchain service: %v", err)
}
cfg := &Config{
BeaconBlockBuf: 0,
BeaconDB: db.DB(),
Chain: beaconChain,
Web3Service: web3Service,
}
chainService, _ := NewChainService(ctx, cfg)
block := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 1,
})
hash, err := block.Hash()
if err != nil {
t.Fatal(err)
}
if err := chainService.SaveBlock(block); err != nil {
t.Fatal(err)
}
slot, err := chainService.BlockSlotNumberByHash(hash)
if err != nil {
t.Errorf("Unexpected error: %v", err)
}
if slot != 1 {
t.Errorf("Expected slot 1, received %d", slot)
}
_, err = chainService.BlockSlotNumberByHash([32]byte{})
if !strings.Contains(err.Error(), "could not get block from DB") {
t.Errorf("Received incorrect error, expected could not get block from DB, received %v", err)
}
}
func TestDoesPOWBlockExist(t *testing.T) {
hook := logTest.NewGlobal()
ctx := context.Background()
config := &database.DBConfig{DataDir: "", Name: "", InMemory: true}
db, err := database.NewDB(config)
if err != nil {
t.Fatalf("could not setup beaconDB: %v", err)
chainService := setupBeaconChain(t, true)
defer chainService.beaconDB.Close()
}
endpoint := "ws://127.0.0.1"
client := &faultyClient{}
web3Service, err := powchain.NewWeb3Service(ctx, &powchain.Web3ServiceConfig{Endpoint: endpoint, Pubkey: "", VrcAddr: common.Address{}}, client, client, client)
if err != nil {
t.Fatalf("unable to set up web3 service: %v", err)
}
beaconChain, err := NewBeaconChain("", db.DB())
if err != nil {
t.Fatalf("could not register blockchain service: %v", err)
}
cfg := &Config{
BeaconBlockBuf: 0,
IncomingBlockBuf: 0,
BeaconDB: db.DB(),
Chain: beaconChain,
Web3Service: web3Service,
}
chainService, _ := NewChainService(ctx, cfg)
block := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 10,
})
@@ -523,33 +237,122 @@ func TestDoesPOWBlockExist(t *testing.T) {
testutil.AssertLogsContain(t, hook, "fetching PoW block corresponding to mainchain reference failed")
}
func TestUpdateHeadEmpty(t *testing.T) {
hook := logTest.NewGlobal()
chainService := setupBeaconChain(t, false)
defer chainService.beaconDB.Close()
active := types.NewGenesisActiveState()
crystallized, err := types.NewGenesisCrystallizedState("")
if err != nil {
t.Fatalf("Can't generate genesis state: %v", err)
}
activeStateHash, _ := active.Hash()
crystallizedStateHash, _ := crystallized.Hash()
genesis := types.NewGenesisBlock(activeStateHash, crystallizedStateHash)
genesisHash, err := genesis.Hash()
if err != nil {
t.Fatalf("Could not get genesis block hash: %v", err)
}
block := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 64,
ActiveStateHash: activeStateHash[:],
CrystallizedStateHash: crystallizedStateHash[:],
ParentHash: genesisHash[:],
PowChainRef: []byte("a"),
})
exitRoutine := make(chan bool)
timeChan := make(chan time.Time)
go func() {
chainService.updateHead(timeChan)
<-exitRoutine
}()
if err := chainService.beaconDB.SaveBlock(block); err != nil {
t.Fatal(err)
}
// If blocks pending processing is empty, the updateHead routine does nothing.
chainService.blocksPendingProcessing = [][32]byte{}
timeChan <- time.Now()
chainService.cancel()
exitRoutine <- true
testutil.AssertLogsDoNotContain(t, hook, "Applying fork choice rule")
}
func TestUpdateHeadNoBlock(t *testing.T) {
hook := logTest.NewGlobal()
chainService := setupBeaconChain(t, false)
defer chainService.beaconDB.Close()
exitRoutine := make(chan bool)
timeChan := make(chan time.Time)
go func() {
chainService.updateHead(timeChan)
<-exitRoutine
}()
// If blocks pending processing contains a hash of a block that does not exist
// in persistent storage, we expect an error log to be thrown as
// that is unexpected behavior given the block should have been saved during
// processing.
fakeBlock := types.NewBlock(&pb.BeaconBlock{SlotNumber: 100})
fakeBlockHash, err := fakeBlock.Hash()
if err != nil {
t.Fatal(err)
}
chainService.blocksPendingProcessing = [][32]byte{}
chainService.blocksPendingProcessing = append(chainService.blocksPendingProcessing, fakeBlockHash)
timeChan <- time.Now()
chainService.cancel()
exitRoutine <- true
testutil.AssertLogsContain(t, hook, "Could not get block")
}
func TestUpdateHeadNoParent(t *testing.T) {
hook := logTest.NewGlobal()
chainService := setupBeaconChain(t, false)
defer chainService.beaconDB.Close()
exitRoutine := make(chan bool)
timeChan := make(chan time.Time)
go func() {
chainService.updateHead(timeChan)
<-exitRoutine
}()
// non-existent parent hash should log an error in updateHead.
noParentBlock := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 64,
})
noParentBlockHash, err := noParentBlock.Hash()
if err != nil {
t.Fatalf("Could not hash block: %v", err)
}
if err := chainService.beaconDB.SaveBlock(noParentBlock); err != nil {
t.Fatal(err)
}
chainService.blocksPendingProcessing = [][32]byte{}
chainService.blocksPendingProcessing = append(chainService.blocksPendingProcessing, noParentBlockHash)
timeChan <- time.Now()
chainService.cancel()
exitRoutine <- true
testutil.AssertLogsContain(t, hook, fmt.Sprintf("Failed to get parent of block %x", noParentBlockHash))
}
func TestUpdateHead(t *testing.T) {
hook := logTest.NewGlobal()
ctx := context.Background()
config := &database.DBConfig{DataDir: "", Name: "", InMemory: true}
db, err := database.NewDB(config)
if err != nil {
t.Fatalf("could not setup beaconDB: %v", err)
}
endpoint := "ws://127.0.0.1"
client := &mockClient{}
web3Service, err := powchain.NewWeb3Service(ctx, &powchain.Web3ServiceConfig{Endpoint: endpoint, Pubkey: "", VrcAddr: common.Address{}}, client, client, client)
if err != nil {
t.Fatalf("unable to set up web3 service: %v", err)
}
beaconChain, err := NewBeaconChain("", db.DB())
if err != nil {
t.Fatalf("could not register blockchain service: %v", err)
}
cfg := &Config{
BeaconBlockBuf: 0,
IncomingBlockBuf: 0,
BeaconDB: db.DB(),
Chain: beaconChain,
Web3Service: web3Service,
}
chainService, _ := NewChainService(ctx, cfg)
chainService := setupBeaconChain(t, false)
defer chainService.beaconDB.Close()
active := types.NewGenesisActiveState()
crystallized, err := types.NewGenesisCrystallizedState("")
@@ -584,108 +387,23 @@ func TestUpdateHead(t *testing.T) {
<-exitRoutine
}()
if err := chainService.SaveBlock(block); err != nil {
if err := chainService.beaconDB.SaveBlock(block); err != nil {
t.Fatal(err)
}
// If blocks pending processing is empty, the updateHead routine does nothing.
chainService.blocksPendingProcessing = [][32]byte{}
timeChan <- time.Now()
chainService.cancel()
exitRoutine <- true
chainService, _ = NewChainService(ctx, cfg)
go func() {
chainService.updateHead(timeChan)
<-exitRoutine
}()
// If blocks pending processing contains a hash of a block that does not exist
// in persistent storage, we expect an error log to be thrown as
// that is unexpected behavior given the block should have been saved during
// processing.
fakeBlock := types.NewBlock(&pb.BeaconBlock{SlotNumber: 100})
fakeBlockHash, err := fakeBlock.Hash()
if err != nil {
t.Fatal(err)
}
chainService.blocksPendingProcessing = [][32]byte{}
chainService.blocksPendingProcessing = append(chainService.blocksPendingProcessing, fakeBlockHash)
timeChan <- time.Now()
chainService.cancel()
exitRoutine <- true
// Inexistent parent hash should log an error in updateHead.
noParentBlock := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 64,
})
noParentBlockHash, err := noParentBlock.Hash()
if err != nil {
t.Fatalf("Could not hash block: %v", err)
}
chainService, _ = NewChainService(ctx, cfg)
go func() {
chainService.updateHead(timeChan)
<-exitRoutine
}()
if err := chainService.SaveBlock(noParentBlock); err != nil {
t.Fatal(err)
}
chainService.blocksPendingProcessing = [][32]byte{}
chainService.blocksPendingProcessing = append(chainService.blocksPendingProcessing, noParentBlockHash)
timeChan <- time.Now()
chainService.cancel()
exitRoutine <- true
// Now we test the correct, end-to-end updateHead functionality.
chainService, _ = NewChainService(ctx, cfg)
go func() {
chainService.updateHead(timeChan)
<-exitRoutine
}()
chainService.blocksPendingProcessing = [][32]byte{}
chainService.blocksPendingProcessing = append(chainService.blocksPendingProcessing, hash)
timeChan <- time.Now()
chainService.cancel()
exitRoutine <- true
testutil.AssertLogsContain(t, hook, "Could not get block")
testutil.AssertLogsContain(t, hook, "Failed to get parent of block")
testutil.AssertLogsContain(t, hook, "Canonical block determined")
}
func TestProcessBlocksWithCorrectAttestations(t *testing.T) {
ctx := context.Background()
config := &database.DBConfig{DataDir: "", Name: "", InMemory: true}
db, err := database.NewDB(config)
if err != nil {
t.Fatalf("could not setup beaconDB: %v", err)
}
endpoint := "ws://127.0.0.1"
client := &mockClient{}
web3Service, err := powchain.NewWeb3Service(ctx, &powchain.Web3ServiceConfig{Endpoint: endpoint, Pubkey: "", VrcAddr: common.Address{}}, client, client, client)
if err != nil {
t.Fatalf("unable to set up web3 service: %v", err)
}
beaconChain, err := NewBeaconChain("", db.DB())
if err != nil {
t.Fatalf("could not register blockchain service: %v", err)
}
cfg := &Config{
BeaconBlockBuf: 0,
BeaconDB: db.DB(),
Chain: beaconChain,
Web3Service: web3Service,
}
chainService, _ := NewChainService(ctx, cfg)
chainService := setupBeaconChain(t, false)
defer chainService.beaconDB.Close()
active := types.NewGenesisActiveState()
crystallized, err := types.NewGenesisCrystallizedState("")
@@ -696,16 +414,10 @@ func TestProcessBlocksWithCorrectAttestations(t *testing.T) {
activeStateHash, _ := active.Hash()
crystallizedStateHash, _ := crystallized.Hash()
exitRoutine := make(chan bool)
go func() {
chainService.blockProcessing()
<-exitRoutine
}()
block0 := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 0,
})
if saveErr := beaconChain.saveBlock(block0); saveErr != nil {
if saveErr := chainService.beaconDB.SaveBlock(block0); saveErr != nil {
t.Fatalf("Cannot save block: %v", saveErr)
}
block0Hash, err := block0.Hash()
@@ -728,7 +440,7 @@ func TestProcessBlocksWithCorrectAttestations(t *testing.T) {
}},
})
exitRoutine = make(chan bool)
exitRoutine := make(chan bool)
go func() {
chainService.blockProcessing()
<-exitRoutine

View File

@@ -0,0 +1,39 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
go_library(
name = "go_default_library",
srcs = [
"attestation.go",
"block.go",
"db.go",
"schema.go",
"state.go",
],
importpath = "github.com/prysmaticlabs/prysm/beacon-chain/db",
visibility = ["//beacon-chain:__subpackages__"],
deps = [
"//beacon-chain/types:go_default_library",
"//proto/beacon/p2p/v1:go_default_library",
"//shared/database:go_default_library",
"@com_github_gogo_protobuf//proto:go_default_library",
"@com_github_sirupsen_logrus//:go_default_library",
],
)
go_test(
name = "go_default_test",
srcs = [
"attestation_test.go",
"block_test.go",
"db_test.go",
"schema_test.go",
"state_test.go",
],
embed = [":go_default_library"],
deps = [
"//beacon-chain/types:go_default_library",
"//proto/beacon/p2p/v1:go_default_library",
"@com_github_gogo_protobuf//proto:go_default_library",
"@com_github_sirupsen_logrus//hooks/test:go_default_library",
],
)

View File

@@ -0,0 +1,127 @@
package db
import (
"bytes"
"github.com/gogo/protobuf/proto"
"github.com/prysmaticlabs/prysm/beacon-chain/types"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
)
// HasAttestation checks an attestation exists in beacon chain db by inputting its hash.
func (db *BeaconDB) HasAttestation(attestationHash [32]byte) (bool, error) {
return db.has(attestationKey(attestationHash))
}
// SaveAttestation puts the attestation record into the beacon chain db.
func (db *BeaconDB) SaveAttestation(attestation *types.Attestation) error {
hash := attestation.Key()
key := attestationKey(hash)
encodedState, err := attestation.Marshal()
if err != nil {
return err
}
return db.put(key, encodedState)
}
// GetAttestation retrieves an attestation record from the db using its hash.
func (db *BeaconDB) GetAttestation(hash [32]byte) (*types.Attestation, error) {
key := attestationKey(hash)
enc, err := db.get(key)
if err != nil {
return nil, err
}
attestation := &pb.AggregatedAttestation{}
err = proto.Unmarshal(enc, attestation)
return types.NewAttestation(attestation), err
}
// RemoveAttestation removes the attestation from the db.
func (db *BeaconDB) RemoveAttestation(blockHash [32]byte) error {
return db.delete(attestationKey(blockHash))
}
// HasAttestationHash checks if the beacon block has the attestation.
func (db *BeaconDB) HasAttestationHash(blockHash [32]byte, attestationHash [32]byte) (bool, error) {
enc, err := db.get(attestationHashListKey(blockHash))
if err != nil {
return false, err
}
attestationHashes := &pb.AttestationHashes{}
if err := proto.Unmarshal(enc, attestationHashes); err != nil {
return false, err
}
for _, hash := range attestationHashes.AttestationHash {
if bytes.Equal(hash, attestationHash[:]) {
return true, nil
}
}
return false, nil
}
// HasAttestationHashList checks if the attestation hash list is available.
func (db *BeaconDB) HasAttestationHashList(blockHash [32]byte) (bool, error) {
key := attestationHashListKey(blockHash)
hasKey, err := db.has(key)
if err != nil {
return false, err
}
return hasKey, nil
}
// GetAttestationHashList gets the attestation hash list of the beacon block from the db.
func (db *BeaconDB) GetAttestationHashList(blockHash [32]byte) ([][]byte, error) {
key := attestationHashListKey(blockHash)
hasList, err := db.HasAttestationHashList(blockHash)
if err != nil {
return [][]byte{}, err
}
if !hasList {
if err := db.put(key, []byte{}); err != nil {
return [][]byte{}, err
}
}
enc, err := db.get(key)
if err != nil {
return [][]byte{}, err
}
attestationHashes := &pb.AttestationHashes{}
if err := proto.Unmarshal(enc, attestationHashes); err != nil {
return [][]byte{}, err
}
return attestationHashes.AttestationHash, nil
}
// RemoveAttestationHashList removes the attestation hash list of the beacon block from the db.
func (db *BeaconDB) RemoveAttestationHashList(blockHash [32]byte) error {
return db.delete(attestationHashListKey(blockHash))
}
// SaveAttestationHash saves the attestation hash into the attestation hash list of the corresponding beacon block.
func (db *BeaconDB) SaveAttestationHash(blockHash [32]byte, attestationHash [32]byte) error {
key := attestationHashListKey(blockHash)
hashes, err := db.GetAttestationHashList(blockHash)
if err != nil {
return err
}
hashes = append(hashes, attestationHash[:])
attestationHashes := &pb.AttestationHashes{}
attestationHashes.AttestationHash = hashes
encodedState, err := proto.Marshal(attestationHashes)
if err != nil {
return err
}
return db.put(key, encodedState)
}

View File

@@ -0,0 +1,92 @@
package db
import (
"testing"
"github.com/prysmaticlabs/prysm/beacon-chain/types"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
)
func TestSaveAndRemoveAttestations(t *testing.T) {
db := startInMemoryBeaconDB(t)
defer db.Close()
attestation := types.NewAttestation(&pb.AggregatedAttestation{
Slot: 1,
ShardId: 1,
AttesterBitfield: []byte{'A'},
})
hash := attestation.Key()
if err := db.SaveAttestation(attestation); err != nil {
t.Fatalf("unable to save attestation %v", err)
}
exist, err := db.HasAttestation(hash)
if err != nil {
t.Fatalf("unable to check attestation %v", err)
}
if !exist {
t.Fatal("saved attestation does not exist")
}
if err := db.RemoveAttestation(hash); err != nil {
t.Fatalf("error removing attestation %v", err)
}
if _, err := db.GetAttestation(hash); err == nil {
t.Fatalf("attestation is able to be retrieved")
}
}
func TestSaveAndRemoveAttestationHashList(t *testing.T) {
db := startInMemoryBeaconDB(t)
defer db.Close()
block := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 0,
})
blockHash, err := block.Hash()
if err != nil {
t.Error(err)
}
attestation := types.NewAttestation(&pb.AggregatedAttestation{
Slot: 1,
ShardId: 1,
AttesterBitfield: []byte{'A'},
})
attestationHash := attestation.Key()
if err := db.SaveAttestationHash(blockHash, attestationHash); err != nil {
t.Fatalf("unable to save attestation hash %v", err)
}
exist, err := db.HasAttestationHash(blockHash, attestationHash)
if err != nil {
t.Fatalf("unable to check for attestation hash %v", err)
}
if !exist {
t.Error("saved attestation hash does not exist")
}
// Negative test case: try with random attestation, exist should be false.
exist, err = db.HasAttestationHash(blockHash, [32]byte{'A'})
if err != nil {
t.Fatalf("unable to check for attestation hash %v", err)
}
if exist {
t.Error("attestation hash shouldn't have existed")
}
// Remove attestation list by deleting the block hash key.
if err := db.RemoveAttestationHashList(blockHash); err != nil {
t.Fatalf("remove attestation hash list failed %v", err)
}
// Negative test case: try with deleted block hash, this should fail.
_, err = db.HasAttestationHash(blockHash, attestationHash)
if err == nil {
t.Error("attestation hash should't have existed in DB")
}
}

144
beacon-chain/db/block.go Normal file
View File

@@ -0,0 +1,144 @@
package db
import (
"errors"
"fmt"
"github.com/gogo/protobuf/proto"
"github.com/prysmaticlabs/prysm/beacon-chain/types"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
)
// GetCanonicalBlock fetches the latest head stored in persistent storage.
func (db *BeaconDB) GetCanonicalBlock() (*types.Block, error) {
bytes, err := db.get(canonicalHeadLookupKey)
if err != nil {
return nil, err
}
block := &pb.BeaconBlock{}
if err := proto.Unmarshal(bytes, block); err != nil {
return nil, fmt.Errorf("cannot unmarshal proto: %v", err)
}
return types.NewBlock(block), nil
}
// HasBlock returns true if the block for the given hash exists.
func (db *BeaconDB) HasBlock(blockhash [32]byte) (bool, error) {
return db.has(blockKey(blockhash))
}
// SaveBlock puts the passed block into the beacon chain db.
func (db *BeaconDB) SaveBlock(block *types.Block) error {
hash, err := block.Hash()
if err != nil {
return err
}
key := blockKey(hash)
encodedState, err := block.Marshal()
if err != nil {
return err
}
return db.put(key, encodedState)
}
// SaveCanonicalSlotNumber saves the slotnumber and blockhash of a canonical block
// saved in the db. This will alow for canonical blocks to be retrieved from the db
// by using their slotnumber as a key, and then using the retrieved blockhash to get
// the block from the db.
// prefix + slotNumber -> blockhash
// prefix + blockHash -> block
func (db *BeaconDB) SaveCanonicalSlotNumber(slotNumber uint64, hash [32]byte) error {
return db.put(canonicalBlockKey(slotNumber), hash[:])
}
// SaveCanonicalBlock puts the passed block into the beacon chain db
// and also saves a "latest-head" key mapping to the block in the db.
func (db *BeaconDB) SaveCanonicalBlock(block *types.Block) error {
enc, err := block.Marshal()
if err != nil {
return err
}
return db.put(canonicalHeadLookupKey, enc)
}
// GetBlock retrieves a block from the db using its hash.
func (db *BeaconDB) GetBlock(hash [32]byte) (*types.Block, error) {
key := blockKey(hash)
has, err := db.has(key)
if err != nil {
return nil, err
}
if !has {
return nil, errors.New("block not found")
}
enc, err := db.get(key)
if err != nil {
return nil, err
}
block := &pb.BeaconBlock{}
err = proto.Unmarshal(enc, block)
return types.NewBlock(block), err
}
// removeBlock removes the block from the db.
func (db *BeaconDB) removeBlock(hash [32]byte) error {
return db.delete(blockKey(hash))
}
// HasCanonicalBlockForSlot checks the db if the canonical block for
// this slot exists.
func (db *BeaconDB) HasCanonicalBlockForSlot(slotNumber uint64) (bool, error) {
return db.has(canonicalBlockKey(slotNumber))
}
// GetCanonicalBlockForSlot retrieves the canonical block which is saved in the db
// for that required slot number.
func (db *BeaconDB) GetCanonicalBlockForSlot(slotNumber uint64) (*types.Block, error) {
enc, err := db.get(canonicalBlockKey(slotNumber))
if err != nil {
return nil, err
}
var blockhash [32]byte
copy(blockhash[:], enc)
block, err := db.GetBlock(blockhash)
return block, err
}
// GetSimulatedBlock retrieves the last block broadcast by the simulator.
func (db *BeaconDB) GetSimulatedBlock() (*types.Block, error) {
enc, err := db.get(simulatedBlockKey)
if err != nil {
return nil, err
}
protoBlock := &pb.BeaconBlock{}
err = proto.Unmarshal(enc, protoBlock)
if err != nil {
return nil, err
}
return types.NewBlock(protoBlock), nil
}
// SaveSimulatedBlock saves the last broadcast block to the database.
func (db *BeaconDB) SaveSimulatedBlock(block *types.Block) error {
enc, err := block.Marshal()
if err != nil {
return err
}
return db.put(simulatedBlockKey, enc)
}
// HasSimulatedBlock checks if a block was broadcast by the simulator.
func (db *BeaconDB) HasSimulatedBlock() (bool, error) {
return db.has(simulatedBlockKey)
}

View File

@@ -0,0 +1,175 @@
package db
import (
"bytes"
"testing"
"github.com/gogo/protobuf/proto"
"github.com/prysmaticlabs/prysm/beacon-chain/types"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
)
func TestSaveAndRemoveBlocks(t *testing.T) {
beaconDB := startInMemoryBeaconDB(t)
defer beaconDB.Close()
block := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 64,
PowChainRef: []byte("a"),
})
hash, err := block.Hash()
if err != nil {
t.Fatalf("unable to generate hash of block %v", err)
}
if err := beaconDB.SaveBlock(block); err != nil {
t.Fatalf("unable to save block %v", err)
}
// Adding a different block with the same key
newblock := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 4,
PowChainRef: []byte("b"),
})
key := blockKey(hash)
marshalled, err := proto.Marshal(newblock.Proto())
if err != nil {
t.Fatal(err)
}
if err := beaconDB.put(key, marshalled); err != nil {
t.Fatal(err)
}
retblock, err := beaconDB.GetBlock(hash)
if err != nil {
t.Fatalf("block is unable to be retrieved")
}
if retblock.SlotNumber() != newblock.SlotNumber() {
t.Errorf("slotnumber does not match for saved and retrieved blocks")
}
if !bytes.Equal(retblock.PowChainRef().Bytes(), newblock.PowChainRef().Bytes()) {
t.Errorf("POW chain ref does not match for saved and retrieved blocks")
}
if err := beaconDB.removeBlock(hash); err != nil {
t.Fatalf("error removing block %v", err)
}
if _, err := beaconDB.GetBlock(hash); err == nil {
t.Fatalf("block is able to be retrieved")
}
if err := beaconDB.removeBlock(hash); err != nil {
t.Fatalf("unable to remove block a second time %v", err)
}
}
func TestCheckBlockBySlotNumber(t *testing.T) {
beaconDB := startInMemoryBeaconDB(t)
defer beaconDB.Close()
block := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 64,
PowChainRef: []byte("a"),
})
hash, err := block.Hash()
if err != nil {
t.Error(err)
}
if err := beaconDB.SaveCanonicalSlotNumber(block.SlotNumber(), hash); err != nil {
t.Fatalf("unable to save canonical slot %v", err)
}
if err := beaconDB.SaveBlock(block); err != nil {
t.Fatalf("unable to save block %v", err)
}
slotExists, err := beaconDB.HasCanonicalBlockForSlot(block.SlotNumber())
if err != nil {
t.Fatalf("unable to check for block by slot %v", err)
}
if !slotExists {
t.Error("slot does not exist despite blockhash of canonical block being saved in the db")
}
alternateblock := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 64,
PowChainRef: []byte("d"),
})
althash, err := alternateblock.Hash()
if err != nil {
t.Fatalf("unable to hash block %v", err)
}
if err := beaconDB.SaveCanonicalSlotNumber(block.SlotNumber(), althash); err != nil {
t.Fatalf("unable to save canonical slot %v", err)
}
retrievedHash, err := beaconDB.get(canonicalBlockKey(block.SlotNumber()))
if err != nil {
t.Fatalf("unable to retrieve blockhash %v", err)
}
if !bytes.Equal(retrievedHash, althash[:]) {
t.Errorf("unequal hashes between what was saved and what was retrieved %v, %v", retrievedHash, althash)
}
}
func TestGetBlockBySlotNumber(t *testing.T) {
beaconDB := startInMemoryBeaconDB(t)
defer beaconDB.Close()
block := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 64,
PowChainRef: []byte("a"),
})
hash, err := block.Hash()
if err != nil {
t.Error(err)
}
if err := beaconDB.SaveCanonicalSlotNumber(block.SlotNumber(), hash); err != nil {
t.Fatalf("unable to save canonical slot %v", err)
}
if err := beaconDB.SaveBlock(block); err != nil {
t.Fatalf("unable to save block %v", err)
}
retblock, err := beaconDB.GetCanonicalBlockForSlot(block.SlotNumber())
if err != nil {
t.Fatalf("unable to get block from db %v", err)
}
if !bytes.Equal(retblock.PowChainRef().Bytes(), block.PowChainRef().Bytes()) {
t.Error("canonical block saved different from block retrieved")
}
alternateblock := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 64,
PowChainRef: []byte("d"),
})
althash, err := alternateblock.Hash()
if err != nil {
t.Fatalf("unable to hash block %v", err)
}
if err := beaconDB.SaveCanonicalSlotNumber(block.SlotNumber(), althash); err != nil {
t.Fatalf("unable to save canonical slot %v", err)
}
if _, err = beaconDB.GetCanonicalBlockForSlot(block.SlotNumber()); err == nil {
t.Fatal("there should be an error because block does not exist in the db")
}
}

135
beacon-chain/db/db.go Normal file
View File

@@ -0,0 +1,135 @@
package db
import (
"fmt"
"github.com/gogo/protobuf/proto"
"github.com/prysmaticlabs/prysm/beacon-chain/types"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
"github.com/prysmaticlabs/prysm/shared/database"
"github.com/sirupsen/logrus"
)
var log = logrus.WithField("prefix", "blockchain")
// BeaconDB manages the data layer of the beacon chain implementation.
// The exposed methods do not have an opinion of the underlying data engine,
// but instead reflect the beacon chain logic.
// For example, instead of defining get, put, remove
// This defines methods such as getBlock, saveBlocksAndAttestations, etc.
type BeaconDB struct {
db *database.DB
state *beaconState
}
// Config exposes relevant config options for starting a database.
type Config struct {
Path string
Name string
InMemory bool
GenesisJSON string
}
type beaconState struct {
// aState captures the beacon state at block processing level,
// it focuses on verifying aggregated signatures and pending attestations.
aState *types.ActiveState
// cState captures the beacon state at cycle transition level,
// it focuses on changes to the validator set, processing cross links and
// setting up FFG checkpoints.
cState *types.CrystallizedState
}
func (db *BeaconDB) has(key []byte) (bool, error) {
return db.db.DB().Has(key)
}
func (db *BeaconDB) get(key []byte) ([]byte, error) {
return db.db.DB().Get(key)
}
func (db *BeaconDB) put(key []byte, val []byte) error {
return db.db.DB().Put(key, val)
}
func (db *BeaconDB) delete(key []byte) error {
return db.db.DB().Delete(key)
}
// Close closes the underlying leveldb database.
func (db *BeaconDB) Close() {
db.db.Close()
}
// NewDB initializes a new DB. If the genesis block and states do not exist, this method creates it.
func NewDB(cfg Config) (*BeaconDB, error) {
config := &database.DBConfig{DataDir: cfg.Path, Name: cfg.Name, InMemory: cfg.InMemory}
db, err := database.NewDB(config)
if err != nil {
return nil, fmt.Errorf("failed to start db: %v", err)
}
beaconDB := &BeaconDB{
db: db,
state: &beaconState{},
}
hasCrystallized, err := beaconDB.has(crystallizedStateLookupKey)
if err != nil {
return nil, err
}
hasGenesis, err := beaconDB.HasCanonicalBlockForSlot(0)
if err != nil {
return nil, err
}
active := types.NewGenesisActiveState()
crystallized, err := types.NewGenesisCrystallizedState(cfg.GenesisJSON)
if err != nil {
return nil, err
}
beaconDB.state.aState = active
if !hasGenesis {
log.Info("No genesis block found on disk, initializing genesis block")
// Active state hash is predefined so error can be safely ignored
// #nosec G104
activeStateHash, _ := active.Hash()
// Crystallized state hash is predefined so error can be safely ignored
// #nosec G104
crystallizedStateHash, _ := crystallized.Hash()
genesisBlock := types.NewGenesisBlock(activeStateHash, crystallizedStateHash)
if err := beaconDB.SaveBlock(genesisBlock); err != nil {
return nil, err
}
if err := beaconDB.SaveCanonicalBlock(genesisBlock); err != nil {
return nil, err
}
genesisBlockHash, err := genesisBlock.Hash()
if err != nil {
return nil, err
}
if err := beaconDB.SaveCanonicalSlotNumber(0, genesisBlockHash); err != nil {
return nil, err
}
}
if !hasCrystallized {
log.Info("No chainstate found on disk, initializing beacon from genesis")
beaconDB.state.cState = crystallized
return beaconDB, nil
}
enc, err := beaconDB.get(crystallizedStateLookupKey)
if err != nil {
return nil, err
}
crystallizedData := &pb.CrystallizedState{}
err = proto.Unmarshal(enc, crystallizedData)
if err != nil {
return nil, err
}
beaconDB.state.cState = types.NewCrystallizedState(crystallizedData)
return beaconDB, nil
}

View File

@@ -0,0 +1,46 @@
package db
import (
"testing"
"github.com/gogo/protobuf/proto"
"github.com/prysmaticlabs/prysm/beacon-chain/types"
logTest "github.com/sirupsen/logrus/hooks/test"
)
func startInMemoryBeaconDB(t *testing.T) *BeaconDB {
config := Config{Path: "", Name: "", InMemory: true}
db, err := NewDB(config)
if err != nil {
t.Fatalf("unable to setup db: %v", err)
}
return db
}
func TestNewDB(t *testing.T) {
hook := logTest.NewGlobal()
beaconDB := startInMemoryBeaconDB(t)
defer beaconDB.Close()
msg := hook.LastEntry().Message
want := "No chainstate found on disk, initializing beacon from genesis"
if msg != want {
t.Errorf("incorrect log, expected %s, got %s", want, msg)
}
hook.Reset()
aState := types.NewGenesisActiveState()
cState, err := types.NewGenesisCrystallizedState("")
if err != nil {
t.Errorf("Creating new genesis state failed %v", err)
}
if !proto.Equal(beaconDB.GetActiveState().Proto(), aState.Proto()) {
t.Errorf("active states not equal. received: %v, wanted: %v", beaconDB.GetActiveState(), aState)
}
if !proto.Equal(beaconDB.GetCrystallizedState().Proto(), cState.Proto()) {
t.Errorf("crystallized states not equal. received: %v, wanted: %v", beaconDB.GetCrystallizedState(), cState)
}
}

View File

@@ -1,4 +1,4 @@
package blockchain
package db
import (
"encoding/binary"
@@ -15,17 +15,16 @@ import (
// The fields below define the suffix of keys in the db.
var (
// CanonicalHeadLookupKey tracks the latest canonical head.
// canonicalHeadLookupKey tracks the latest canonical head.
canonicalHeadLookupKey = []byte("latest-canonical-head")
// ActiveStateLookupKey tracks the current active state.
// activeStateLookupKey tracks the current active state.
activeStateLookupKey = []byte("beacon-active-state")
// CrystallizedStateLookupKey tracks the current crystallized state.
// crystallizedStateLookupKey tracks the current crystallized state.
crystallizedStateLookupKey = []byte("beacon-crystallized-state")
// GenesisLookupKey tracks the genesis block.
genesisLookupKey = []byte("genesis")
simulatedBlockKey = []byte("last-simulated-block")
// Data item suffixes.
// TODO(#514): Change suffixes back to prefixes as originally designed after issue 514 is solved or after BoltDB migration
@@ -52,12 +51,12 @@ func canonicalBlockKey(slotnumber uint64) []byte {
return append(encodeSlotNumber(slotnumber)[:], canonicalSuffix...)
}
// AttestationKey = attestationPrefix + attestationHash.
func AttestationKey(hash [32]byte) []byte {
// attestationKey = attestationPrefix + attestationHash.
func attestationKey(hash [32]byte) []byte {
return append(hash[:], attestationSuffix...)
}
// AttestationHashListKey = attestationHashesPrefix + blockHash.
func AttestationHashListKey(hash [32]byte) []byte {
func attestationHashListKey(hash [32]byte) []byte {
return append(hash[:], attestationHashesSuffix...)
}

View File

@@ -1,4 +1,4 @@
package blockchain
package db
import (
"bytes"

45
beacon-chain/db/state.go Normal file
View File

@@ -0,0 +1,45 @@
package db
import (
"github.com/prysmaticlabs/prysm/beacon-chain/types"
)
// GetActiveState contains the current state of attestations and changes every block.
func (db *BeaconDB) GetActiveState() *types.ActiveState {
return db.state.aState
}
// GetCrystallizedState contains cycle dependent validator information, changes every cycle.
func (db *BeaconDB) GetCrystallizedState() *types.CrystallizedState {
return db.state.cState
}
// SaveActiveState is a convenience method which sets and persists the active state on the beacon chain.
func (db *BeaconDB) SaveActiveState(activeState *types.ActiveState) error {
db.state.aState = activeState
encodedState, err := db.GetCrystallizedState().Marshal()
if err != nil {
return err
}
return db.put(activeStateLookupKey, encodedState)
}
// SaveCrystallizedState is a convenience method which sets and persists the crystallized state on the beacon chain.
func (db *BeaconDB) SaveCrystallizedState(crystallizedState *types.CrystallizedState) error {
db.state.cState = crystallizedState
encodedState, err := db.GetActiveState().Marshal()
if err != nil {
return err
}
return db.put(crystallizedStateLookupKey, encodedState)
}
// HasStoredState checks if state has been stored to the database.
func (db *BeaconDB) HasStoredState() (bool, error) {
hasState, err := db.has(crystallizedStateLookupKey)
if err != nil {
return false, err
}
return hasState, nil
}

View File

@@ -0,0 +1,49 @@
package db
import (
"reflect"
"testing"
"github.com/prysmaticlabs/prysm/beacon-chain/types"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
)
func TestSaveActiveState(t *testing.T) {
beaconDB := startInMemoryBeaconDB(t)
defer beaconDB.Close()
data := &pb.ActiveState{
PendingAttestations: []*pb.AggregatedAttestation{
{Slot: 0, ShardBlockHash: []byte{1}}, {Slot: 1, ShardBlockHash: []byte{2}},
},
RecentBlockHashes: [][]byte{
{'A'}, {'B'}, {'C'}, {'D'},
},
}
active := types.NewActiveState(data, make(map[[32]byte]*types.VoteCache))
if err := beaconDB.SaveActiveState(active); err != nil {
t.Fatalf("unable to mutate active state: %v", err)
}
if !reflect.DeepEqual(beaconDB.GetActiveState(), active) {
t.Errorf("active state was not updated. wanted %v, got %v", active, beaconDB.state.aState)
}
}
func TestSaveCrystallizedState(t *testing.T) {
beaconDB := startInMemoryBeaconDB(t)
defer beaconDB.Close()
data := &pb.CrystallizedState{
CurrentDynasty: 3,
DynastySeed: []byte{'A'},
}
crystallized := types.NewCrystallizedState(data)
if err := beaconDB.SaveCrystallizedState(crystallized); err != nil {
t.Fatalf("unable to mutate crystallized state: %v", err)
}
if !reflect.DeepEqual(beaconDB.state.cState, crystallized) {
t.Errorf("crystallized state was not updated. wanted %v, got %v", crystallized, beaconDB.state.cState)
}
}

View File

@@ -11,6 +11,7 @@ go_library(
deps = [
"//beacon-chain/attestation:go_default_library",
"//beacon-chain/blockchain:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/params:go_default_library",
"//beacon-chain/powchain:go_default_library",
"//beacon-chain/rpc:go_default_library",
@@ -21,7 +22,6 @@ go_library(
"//proto/beacon/p2p/v1:go_default_library",
"//shared:go_default_library",
"//shared/cmd:go_default_library",
"//shared/database:go_default_library",
"//shared/debug:go_default_library",
"//shared/p2p:go_default_library",
"//shared/p2p/adapter/tracer:go_default_library",

View File

@@ -14,6 +14,7 @@ import (
gethRPC "github.com/ethereum/go-ethereum/rpc"
"github.com/prysmaticlabs/prysm/beacon-chain/attestation"
"github.com/prysmaticlabs/prysm/beacon-chain/blockchain"
"github.com/prysmaticlabs/prysm/beacon-chain/db"
"github.com/prysmaticlabs/prysm/beacon-chain/params"
"github.com/prysmaticlabs/prysm/beacon-chain/powchain"
"github.com/prysmaticlabs/prysm/beacon-chain/rpc"
@@ -23,7 +24,6 @@ import (
"github.com/prysmaticlabs/prysm/beacon-chain/utils"
"github.com/prysmaticlabs/prysm/shared"
"github.com/prysmaticlabs/prysm/shared/cmd"
"github.com/prysmaticlabs/prysm/shared/database"
"github.com/prysmaticlabs/prysm/shared/debug"
"github.com/prysmaticlabs/prysm/shared/p2p"
"github.com/sirupsen/logrus"
@@ -41,7 +41,7 @@ type BeaconNode struct {
services *shared.ServiceRegistry
lock sync.RWMutex
stop chan struct{} // Channel to wait for termination notifications.
db *database.DB
db *db.BeaconDB
}
// NewBeaconNode creates a new node instance, sets up configuration options, and registers
@@ -144,8 +144,13 @@ func (b *BeaconNode) Close() {
func (b *BeaconNode) startDB(ctx *cli.Context) error {
path := ctx.GlobalString(cmd.DataDirFlag.Name)
config := &database.DBConfig{DataDir: path, Name: beaconChainDBName, InMemory: false}
db, err := database.NewDB(config)
var genesisJSON string
if ctx.GlobalIsSet(utils.GenesisJSON.Name) {
genesisJSON = ctx.GlobalString(utils.GenesisJSON.Name)
}
config := db.Config{Path: path, Name: beaconChainDBName, InMemory: false, GenesisJSON: genesisJSON}
db, err := db.NewDB(config)
if err != nil {
return err
}
@@ -164,11 +169,6 @@ func (b *BeaconNode) registerP2P(ctx *cli.Context) error {
}
func (b *BeaconNode) registerBlockchainService(ctx *cli.Context) error {
var genesisJSON string
if ctx.GlobalIsSet(utils.GenesisJSON.Name) {
genesisJSON = ctx.GlobalString(utils.GenesisJSON.Name)
}
var web3Service *powchain.Web3Service
enablePOWChain := ctx.GlobalBool(utils.EnablePOWChain.Name)
if enablePOWChain {
@@ -181,15 +181,9 @@ func (b *BeaconNode) registerBlockchainService(ctx *cli.Context) error {
enableRewardChecking := ctx.GlobalBool(utils.EnableRewardChecking.Name)
enableAttestationValidity := ctx.GlobalBool(utils.EnableAttestationValidity.Name)
beaconChain, err := blockchain.NewBeaconChain(genesisJSON, b.db.DB())
if err != nil {
return fmt.Errorf("could not register blockchain service: %v", err)
}
blockchainService, err := blockchain.NewChainService(context.TODO(), &blockchain.Config{
BeaconDB: b.db.DB(),
BeaconDB: b.db,
Web3Service: web3Service,
Chain: beaconChain,
BeaconBlockBuf: 10,
IncomingBlockBuf: 100, // Big buffer to accommodate other feed subscribers.
EnablePOWChain: enablePOWChain,
@@ -204,13 +198,8 @@ func (b *BeaconNode) registerBlockchainService(ctx *cli.Context) error {
}
func (b *BeaconNode) registerService() error {
handler, err := attestation.NewHandler(b.db.DB())
if err != nil {
return fmt.Errorf("could not register attestation service: %v", err)
}
attestationService := attestation.NewAttestService(context.TODO(), &attestation.Config{
Handler: handler,
attestationService := attestation.NewAttestationService(context.TODO(), &attestation.Config{
BeaconDB: b.db,
})
return b.services.RegisterService(attestationService)
@@ -257,8 +246,10 @@ func (b *BeaconNode) registerSyncService() error {
cfg := rbcsync.DefaultConfig()
cfg.ChainService = chainService
cfg.AttestService = attestationService
cfg.P2P = p2pService
cfg.BeaconDB = b.db
syncService := rbcsync.NewSyncService(context.Background(), cfg, p2pService)
syncService := rbcsync.NewSyncService(context.Background(), cfg)
return b.services.RegisterService(syncService)
}
@@ -278,7 +269,11 @@ func (b *BeaconNode) registerInitialSyncService() error {
return err
}
initialSyncService := initialsync.NewInitialSyncService(context.Background(), initialsync.DefaultConfig(), p2pService, chainService, syncService)
cfg := initialsync.DefaultConfig()
cfg.P2P = p2pService
cfg.SyncService = syncService
cfg.BeaconDB = b.db
initialSyncService := initialsync.NewInitialSyncService(context.Background(), cfg)
return b.services.RegisterService(initialSyncService)
}
@@ -308,10 +303,9 @@ func (b *BeaconNode) registerSimulatorService(ctx *cli.Context) error {
cfg := &simulator.Config{
Delay: defaultConf.Delay,
BlockRequestBuf: defaultConf.BlockRequestBuf,
BeaconDB: b.db.DB(),
BeaconDB: b.db,
P2P: p2pService,
Web3Service: web3Service,
ChainService: chainService,
EnablePOWChain: enablePOWChain,
}
simulatorService := simulator.NewSimulator(context.TODO(), cfg)
@@ -345,7 +339,7 @@ func (b *BeaconNode) registerRPCService(ctx *cli.Context) error {
CertFlag: cert,
KeyFlag: key,
SubscriptionBuf: 100,
CanonicalFetcher: chainService,
BeaconDB: b.db,
ChainService: chainService,
AttestationService: attestationService,
POWChainService: web3Service,

View File

@@ -24,17 +24,16 @@ import (
var log = logrus.WithField("prefix", "rpc")
// canonicalFetcher defines a struct with methods that can be
// called on-demand to fetch the latest canonical head
// and crystallized state as well as methods that stream
// latest canonical head events to clients
// These functions are called by a validator client upon
// establishing an initial connection to a beacon node via gRPC.
type canonicalFetcher interface {
type beaconDB interface {
// These methods can be called on-demand by a validator
// to fetch canonical head and state.
CanonicalHead() (*types.Block, error)
CanonicalCrystallizedState() *types.CrystallizedState
GetCanonicalBlock() (*types.Block, error)
GetCanonicalBlockForSlot(uint64) (*types.Block, error)
GetCrystallizedState() *types.CrystallizedState
}
type chainService interface {
IncomingBlockFeed() *event.Feed
// These methods are not called on-demand by a validator
// but instead streamed to connected validators every
// time the canonical head changes in the chain service.
@@ -42,12 +41,6 @@ type canonicalFetcher interface {
CanonicalCrystallizedStateFeed() *event.Feed
}
type chainService interface {
IncomingBlockFeed() *event.Feed
CurrentCrystallizedState() *types.CrystallizedState
GenesisBlock() (*types.Block, error)
}
type attestationService interface {
IncomingAttestationFeed() *event.Feed
}
@@ -60,7 +53,7 @@ type powChainService interface {
type Service struct {
ctx context.Context
cancel context.CancelFunc
fetcher canonicalFetcher
beaconDB beaconDB
chainService chainService
powChainService powChainService
attestationService attestationService
@@ -82,7 +75,7 @@ type Config struct {
CertFlag string
KeyFlag string
SubscriptionBuf int
CanonicalFetcher canonicalFetcher
BeaconDB beaconDB
ChainService chainService
POWChainService powChainService
AttestationService attestationService
@@ -96,7 +89,7 @@ func NewRPCService(ctx context.Context, cfg *Config) *Service {
return &Service{
ctx: ctx,
cancel: cancel,
fetcher: cfg.CanonicalFetcher,
beaconDB: cfg.BeaconDB,
chainService: cfg.ChainService,
powChainService: cfg.POWChainService,
attestationService: cfg.AttestationService,
@@ -160,7 +153,7 @@ func (s *Service) Stop() error {
// CanonicalHead of the current beacon chain. This method is requested on-demand
// by a validator when it is their time to propose or attest.
func (s *Service) CanonicalHead(ctx context.Context, req *empty.Empty) (*pbp2p.BeaconBlock, error) {
block, err := s.fetcher.CanonicalHead()
block, err := s.beaconDB.GetCanonicalBlock()
if err != nil {
return nil, fmt.Errorf("could not get canonical head block: %v", err)
}
@@ -179,12 +172,11 @@ func (s *Service) CurrentAssignmentsAndGenesisTime(ctx context.Context, req *pb.
// from a constant value (genesis time is constant in the protocol
// and defined in the params.GetConfig().package).
// Get the genesis timestamp from persistent storage.
genesis, err := s.chainService.GenesisBlock()
genesis, err := s.beaconDB.GetCanonicalBlockForSlot(0)
if err != nil {
return nil, fmt.Errorf("could not get genesis block: %v", err)
}
cState := s.chainService.CurrentCrystallizedState()
cState := s.beaconDB.GetCrystallizedState()
var keys []*pb.PublicKey
if req.AllValidators {
for _, val := range cState.Validators() {
@@ -219,7 +211,7 @@ func (s *Service) ProposeBlock(ctx context.Context, req *pb.ProposeRequest) (*pb
//TODO(#589) The attestation should be aggregated in the validator client side not in the beacon node.
parentSlot := req.GetSlotNumber() - 1
cState := s.chainService.CurrentCrystallizedState()
cState := s.beaconDB.GetCrystallizedState()
_, prevProposerIndex, err := casper.ProposerShardAndIndex(
cState.ShardAndCommitteesForSlots(),
@@ -296,7 +288,7 @@ func (s *Service) LatestAttestation(req *empty.Empty, stream pb.BeaconService_La
// ValidatorShardID is called by a validator to get the shard ID of where it's suppose
// to proposer or attest.
func (s *Service) ValidatorShardID(ctx context.Context, req *pb.PublicKey) (*pb.ShardIDResponse, error) {
cState := s.chainService.CurrentCrystallizedState()
cState := s.beaconDB.GetCrystallizedState()
shardID, err := casper.ValidatorShardID(
req.PublicKey,
@@ -314,7 +306,7 @@ func (s *Service) ValidatorShardID(ctx context.Context, req *pb.PublicKey) (*pb.
// ValidatorSlotAndResponsibility fetches a validator's assigned slot number
// and whether it should act as a proposer/attester.
func (s *Service) ValidatorSlotAndResponsibility(ctx context.Context, req *pb.PublicKey) (*pb.SlotResponsibilityResponse, error) {
cState := s.chainService.CurrentCrystallizedState()
cState := s.beaconDB.GetCrystallizedState()
slot, responsibility, err := casper.ValidatorSlotAndResponsibility(
req.PublicKey,
@@ -339,7 +331,7 @@ func (s *Service) ValidatorSlotAndResponsibility(ctx context.Context, req *pb.Pu
// ValidatorIndex is called by a validator to get its index location that corresponds
// to the attestation bit fields.
func (s *Service) ValidatorIndex(ctx context.Context, req *pb.PublicKey) (*pb.IndexResponse, error) {
cState := s.chainService.CurrentCrystallizedState()
cState := s.beaconDB.GetCrystallizedState()
index, err := casper.ValidatorIndex(
req.PublicKey,
@@ -359,7 +351,7 @@ func (s *Service) ValidatorIndex(ctx context.Context, req *pb.PublicKey) (*pb.In
func (s *Service) ValidatorAssignments(
req *pb.ValidatorAssignmentRequest,
stream pb.BeaconService_ValidatorAssignmentsServer) error {
sub := s.fetcher.CanonicalCrystallizedStateFeed().Subscribe(s.canonicalStateChan)
sub := s.chainService.CanonicalCrystallizedStateFeed().Subscribe(s.canonicalStateChan)
defer sub.Unsubscribe()
for {
select {

View File

@@ -32,50 +32,16 @@ func (m *mockPOWChainService) LatestBlockHash() common.Hash {
return common.BytesToHash([]byte{})
}
type faultyChainService struct{}
func (f *faultyChainService) CanonicalHead() (*types.Block, error) {
return nil, errors.New("failed")
}
func (f *faultyChainService) CanonicalCrystallizedState() *types.CrystallizedState {
return nil
}
func (f *faultyChainService) CanonicalBlockFeed() *event.Feed {
return nil
}
func (f *faultyChainService) CanonicalCrystallizedStateFeed() *event.Feed {
return nil
}
func (f *faultyChainService) GenesisBlock() (*types.Block, error) {
return nil, errors.New("failed")
}
type mockChainService struct {
blockFeed *event.Feed
stateFeed *event.Feed
attestationFeed *event.Feed
}
type mockAttestationService struct{}
func (m *mockAttestationService) IncomingAttestationFeed() *event.Feed {
return new(event.Feed)
}
func (m *mockAttestationService) ContainsAttestation(bitfield []byte, h [32]byte) (bool, error) {
return true, nil
}
func (m *mockChainService) CurrentCrystallizedState() *types.CrystallizedState {
cState, err := types.NewGenesisCrystallizedState("")
if err != nil {
return nil
}
return cState
type mockChainService struct {
blockFeed *event.Feed
stateFeed *event.Feed
attestationFeed *event.Feed
}
func (m *mockChainService) IncomingBlockFeed() *event.Feed {
@@ -90,20 +56,6 @@ func (m *mockChainService) CanonicalCrystallizedStateFeed() *event.Feed {
return m.stateFeed
}
func (m *mockChainService) CanonicalHead() (*types.Block, error) {
data := &pbp2p.BeaconBlock{SlotNumber: 5}
return types.NewBlock(data), nil
}
func (m *mockChainService) CanonicalCrystallizedState() *types.CrystallizedState {
data := &pbp2p.CrystallizedState{}
return types.NewCrystallizedState(data)
}
func (m *mockChainService) GenesisBlock() (*types.Block, error) {
return types.NewGenesisBlock([32]byte{}, [32]byte{}), nil
}
func newMockChainService() *mockChainService {
return &mockChainService{
blockFeed: new(event.Feed),
@@ -112,14 +64,30 @@ func newMockChainService() *mockChainService {
}
}
type mockDB struct {
block *types.Block
genesis *types.Block
cState *types.CrystallizedState
}
func (m *mockDB) GetCanonicalBlock() (*types.Block, error) {
return m.block, nil
}
func (m *mockDB) GetCrystallizedState() *types.CrystallizedState {
return m.cState
}
func (m *mockDB) GetCanonicalBlockForSlot(uint64) (*types.Block, error) {
return m.genesis, nil
}
func TestLifecycle(t *testing.T) {
hook := logTest.NewGlobal()
cs := newMockChainService()
rpcService := NewRPCService(context.Background(), &Config{
Port: "7348",
CertFlag: "alice.crt",
KeyFlag: "alice.key",
CanonicalFetcher: cs,
Port: "7348",
CertFlag: "alice.crt",
KeyFlag: "alice.key",
})
rpcService.Start()
@@ -133,10 +101,8 @@ func TestLifecycle(t *testing.T) {
func TestBadEndpoint(t *testing.T) {
hook := logTest.NewGlobal()
cs := newMockChainService()
rpcService := NewRPCService(context.Background(), &Config{
Port: "ralph merkle!!!",
CanonicalFetcher: cs,
Port: "ralph merkle!!!",
})
rpcService.Start()
@@ -150,10 +116,8 @@ func TestBadEndpoint(t *testing.T) {
func TestInsecureEndpoint(t *testing.T) {
hook := logTest.NewGlobal()
cs := newMockChainService()
rpcService := NewRPCService(context.Background(), &Config{
Port: "7777",
CanonicalFetcher: cs,
Port: "7777",
})
rpcService.Start()
@@ -166,36 +130,20 @@ func TestInsecureEndpoint(t *testing.T) {
testutil.AssertLogsContain(t, hook, "Stopping service")
}
func TestCanonicalHead(t *testing.T) {
mockChain := &mockChainService{}
rpcService := NewRPCService(context.Background(), &Config{
Port: "6372",
CanonicalFetcher: mockChain,
ChainService: mockChain,
POWChainService: &mockPOWChainService{},
})
if _, err := rpcService.CanonicalHead(context.Background(), &empty.Empty{}); err != nil {
t.Errorf("Could not call CanonicalHead correctly: %v", err)
}
rpcService = NewRPCService(context.Background(), &Config{
Port: "6372",
CanonicalFetcher: &faultyChainService{},
ChainService: &mockChainService{},
POWChainService: &mockPOWChainService{},
})
if _, err := rpcService.CanonicalHead(context.Background(), &empty.Empty{}); err == nil {
t.Error("Expected error from faulty chain service, received nil")
}
}
func TestCurrentAssignmentsAndGenesisTime(t *testing.T) {
mockChain := &mockChainService{}
mockDB := &mockDB{}
mockDB.genesis = types.NewGenesisBlock([32]byte{}, [32]byte{})
var err error
mockDB.cState, err = types.NewGenesisCrystallizedState("")
if err != nil {
t.Fatalf("Could not instantiate initial crystallized state: %v", err)
}
rpcService := NewRPCService(context.Background(), &Config{
Port: "6372",
CanonicalFetcher: mockChain,
ChainService: mockChain,
POWChainService: &mockPOWChainService{},
Port: "6372",
BeaconDB: mockDB,
ChainService: mockChain,
POWChainService: &mockPOWChainService{},
})
key := &pb.PublicKey{PublicKey: []byte{}}
@@ -216,11 +164,13 @@ func TestCurrentAssignmentsAndGenesisTime(t *testing.T) {
func TestProposeBlock(t *testing.T) {
mockChain := &mockChainService{}
db := &mockDB{}
db.cState, _ = types.NewGenesisCrystallizedState("")
rpcService := NewRPCService(context.Background(), &Config{
Port: "6372",
CanonicalFetcher: mockChain,
ChainService: mockChain,
POWChainService: &mockPOWChainService{},
Port: "6372",
ChainService: mockChain,
BeaconDB: db,
POWChainService: &mockPOWChainService{},
})
req := &pb.ProposeRequest{
SlotNumber: 5,
@@ -319,9 +269,17 @@ func TestLatestAttestation(t *testing.T) {
func TestValidatorSlotAndResponsibility(t *testing.T) {
mockChain := &mockChainService{}
mockDB := &mockDB{}
cState, err := types.NewGenesisCrystallizedState("")
if err != nil {
t.Fatalf("Failed to instantiate genesis state: %v", err)
}
mockDB.cState = cState
rpcService := NewRPCService(context.Background(), &Config{
Port: "6372",
ChainService: mockChain,
BeaconDB: mockDB,
})
req := &pb.PublicKey{
PublicKey: []byte{},
@@ -333,9 +291,17 @@ func TestValidatorSlotAndResponsibility(t *testing.T) {
func TestValidatorIndex(t *testing.T) {
mockChain := &mockChainService{}
mockDB := &mockDB{}
cState, err := types.NewGenesisCrystallizedState("")
if err != nil {
t.Fatalf("Failed to instantiate genesis state: %v", err)
}
mockDB.cState = cState
rpcService := NewRPCService(context.Background(), &Config{
Port: "6372",
ChainService: mockChain,
BeaconDB: mockDB,
})
req := &pb.PublicKey{
PublicKey: []byte{},
@@ -347,9 +313,17 @@ func TestValidatorIndex(t *testing.T) {
func TestValidatorShardID(t *testing.T) {
mockChain := &mockChainService{}
mockDB := &mockDB{}
cState, err := types.NewGenesisCrystallizedState("")
if err != nil {
t.Fatalf("Failed to instantiate genesis state: %v", err)
}
mockDB.cState = cState
rpcService := NewRPCService(context.Background(), &Config{
Port: "6372",
ChainService: mockChain,
BeaconDB: mockDB,
})
req := &pb.PublicKey{
PublicKey: []byte{},
@@ -363,10 +337,11 @@ func TestValidatorAssignments(t *testing.T) {
hook := logTest.NewGlobal()
mockChain := newMockChainService()
mockDB := &mockDB{}
rpcService := NewRPCService(context.Background(), &Config{
Port: "6372",
ChainService: mockChain,
CanonicalFetcher: mockChain,
Port: "6372",
ChainService: mockChain,
BeaconDB: mockDB,
})
ctrl := gomock.NewController(t)

View File

@@ -8,11 +8,10 @@ go_library(
deps = [
"//beacon-chain/params:go_default_library",
"//beacon-chain/types:go_default_library",
"//beacon-chain/utils:go_default_library",
"//proto/beacon/p2p/v1:go_default_library",
"//shared:go_default_library",
"//shared/p2p:go_default_library",
"@com_github_ethereum_go_ethereum//ethdb:go_default_library",
"@com_github_gogo_protobuf//proto:go_default_library",
"@com_github_golang_protobuf//ptypes:go_default_library_gen",
"@com_github_sirupsen_logrus//:go_default_library",
],
@@ -23,10 +22,9 @@ go_test(
srcs = ["service_test.go"],
embed = [":go_default_library"],
deps = [
"//beacon-chain/params:go_default_library",
"//beacon-chain/db:go_default_library",
"//beacon-chain/types:go_default_library",
"//proto/beacon/p2p/v1:go_default_library",
"//shared/database:go_default_library",
"//shared/event:go_default_library",
"//shared/p2p:go_default_library",
"//shared/testutil:go_default_library",

View File

@@ -6,11 +6,10 @@ import (
"fmt"
"time"
"github.com/ethereum/go-ethereum/ethdb"
"github.com/gogo/protobuf/proto"
"github.com/golang/protobuf/ptypes"
"github.com/prysmaticlabs/prysm/beacon-chain/params"
"github.com/prysmaticlabs/prysm/beacon-chain/types"
"github.com/prysmaticlabs/prysm/beacon-chain/utils"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
"github.com/prysmaticlabs/prysm/shared"
"github.com/prysmaticlabs/prysm/shared/p2p"
@@ -25,11 +24,11 @@ type Simulator struct {
cancel context.CancelFunc
p2p shared.P2P
web3Service types.POWChainService
chainService types.StateFetcher
beaconDB ethdb.Database
beaconDB beaconDB
enablePOWChain bool
delay time.Duration
slotNum uint64
genesisTimestamp time.Time
broadcastedBlocks map[[32]byte]*types.Block
broadcastedBlockHashes [][32]byte
blockRequestChan chan p2p.Message
@@ -41,11 +40,20 @@ type Config struct {
BlockRequestBuf int
P2P shared.P2P
Web3Service types.POWChainService
ChainService types.StateFetcher
BeaconDB ethdb.Database
BeaconDB beaconDB
EnablePOWChain bool
}
type beaconDB interface {
HasSimulatedBlock() (bool, error)
GetSimulatedBlock() (*types.Block, error)
SaveSimulatedBlock(*types.Block) error
GetActiveState() *types.ActiveState
GetCrystallizedState() *types.CrystallizedState
GetCanonicalBlockForSlot(uint64) (*types.Block, error)
GetCanonicalBlock() (*types.Block, error)
}
// DefaultConfig options for the simulator.
func DefaultConfig() *Config {
return &Config{
@@ -62,7 +70,6 @@ func NewSimulator(ctx context.Context, cfg *Config) *Simulator {
cancel: cancel,
p2p: cfg.P2P,
web3Service: cfg.Web3Service,
chainService: cfg.ChainService,
beaconDB: cfg.BeaconDB,
delay: cfg.Delay,
enablePOWChain: cfg.EnablePOWChain,
@@ -76,7 +83,16 @@ func NewSimulator(ctx context.Context, cfg *Config) *Simulator {
// Start the sim.
func (sim *Simulator) Start() {
log.Info("Starting service")
go sim.run(time.NewTicker(sim.delay).C, sim.ctx.Done())
genesis, err := sim.beaconDB.GetCanonicalBlockForSlot(0)
if err != nil {
log.Fatalf("Could not get genesis block: %v", err)
}
sim.genesisTimestamp, err = genesis.Timestamp()
if err != nil {
log.Fatalf("Could not get genesis timestamp: %v", err)
}
go sim.run(time.NewTicker(sim.delay).C)
}
// Stop the sim.
@@ -88,35 +104,28 @@ func (sim *Simulator) Stop() error {
if len(sim.broadcastedBlockHashes) > 0 {
lastBlockHash := sim.broadcastedBlockHashes[len(sim.broadcastedBlockHashes)-1]
lastBlock := sim.broadcastedBlocks[lastBlockHash]
encoded, err := lastBlock.Marshal()
if err != nil {
return err
}
return sim.beaconDB.Put([]byte("last-simulated-block"), encoded)
return sim.beaconDB.SaveSimulatedBlock(lastBlock)
}
return nil
}
func (sim *Simulator) lastSimulatedSessionBlock() (*types.Block, error) {
hasSimulated, err := sim.beaconDB.Has([]byte("last-simulated-block"))
hasBlock, err := sim.beaconDB.HasSimulatedBlock()
if err != nil {
return nil, fmt.Errorf("Could not determine if a previous simulation occurred: %v", err)
}
if !hasSimulated {
if !hasBlock {
return nil, nil
}
enc, err := sim.beaconDB.Get([]byte("last-simulated-block"))
simulatedBlock, err := sim.beaconDB.GetSimulatedBlock()
if err != nil {
return nil, fmt.Errorf("Could not fetch simulated block from db: %v", err)
}
lastSimulatedBlockProto := &pb.BeaconBlock{}
if err = proto.Unmarshal(enc, lastSimulatedBlockProto); err != nil {
return nil, fmt.Errorf("Could not unmarshal simulated block from db: %v", err)
}
return types.NewBlock(lastSimulatedBlockProto), nil
return simulatedBlock, nil
}
func (sim *Simulator) run(delayChan <-chan time.Time, done <-chan struct{}) {
func (sim *Simulator) run(delayChan <-chan time.Time) {
blockReqSub := sim.p2p.Subscribe(&pb.BeaconBlockRequest{}, sim.blockRequestChan)
defer blockReqSub.Unsubscribe()
@@ -138,16 +147,16 @@ func (sim *Simulator) run(delayChan <-chan time.Time, done <-chan struct{}) {
for {
select {
case <-done:
case <-sim.ctx.Done():
log.Debug("Simulator context closed, exiting goroutine")
return
case <-delayChan:
activeStateHash, err := sim.chainService.CurrentActiveState().Hash()
activeStateHash, err := sim.beaconDB.GetActiveState().Hash()
if err != nil {
log.Errorf("Could not fetch active state hash: %v", err)
continue
}
crystallizedStateHash, err := sim.chainService.CurrentCrystallizedState().Hash()
crystallizedStateHash, err := sim.beaconDB.GetCrystallizedState().Hash()
if err != nil {
log.Errorf("Could not fetch crystallized state hash: %v", err)
continue
@@ -157,7 +166,7 @@ func (sim *Simulator) run(delayChan <-chan time.Time, done <-chan struct{}) {
// to the genesis block.
var hash [32]byte
if sim.slotNum == 1 {
genesisBlock, err := sim.chainService.GenesisBlock()
genesisBlock, err := sim.beaconDB.GetCanonicalBlockForSlot(0)
if err != nil {
log.Errorf("Failed to retrieve genesis block: %v", err)
continue
@@ -181,12 +190,10 @@ func (sim *Simulator) run(delayChan <-chan time.Time, done <-chan struct{}) {
powChainRef = []byte{byte(sim.slotNum)}
}
var blockSlot uint64
if sim.chainService.CurrentBeaconSlot() == 0 {
blockSlot := utils.CurrentSlot(sim.genesisTimestamp)
if blockSlot == 0 {
// cannot process a genesis block, so we start from 1
blockSlot = 1
} else {
blockSlot = sim.chainService.CurrentBeaconSlot()
}
block := types.NewBlock(&pb.BeaconBlock{

View File

@@ -9,10 +9,9 @@ import (
"github.com/ethereum/go-ethereum/common"
"github.com/golang/protobuf/proto"
"github.com/prysmaticlabs/prysm/beacon-chain/params"
"github.com/prysmaticlabs/prysm/beacon-chain/db"
"github.com/prysmaticlabs/prysm/beacon-chain/types"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
"github.com/prysmaticlabs/prysm/shared/database"
"github.com/prysmaticlabs/prysm/shared/event"
"github.com/prysmaticlabs/prysm/shared/p2p"
"github.com/prysmaticlabs/prysm/shared/testutil"
@@ -41,37 +40,30 @@ func (mpow *mockPOWChainService) LatestBlockHash() common.Hash {
return common.BytesToHash([]byte{})
}
type mockChainService struct{}
func setupSimulator(t *testing.T) *Simulator {
ctx := context.Background()
func (mc *mockChainService) CurrentActiveState() *types.ActiveState {
return types.NewActiveState(&pb.ActiveState{}, make(map[[32]byte]*types.VoteCache))
}
config := db.Config{Path: "", Name: "", InMemory: true}
db, err := db.NewDB(config)
if err != nil {
t.Fatalf("could not setup beaconDB: %v", err)
}
func (mc *mockChainService) CurrentCrystallizedState() *types.CrystallizedState {
return types.NewCrystallizedState(&pb.CrystallizedState{})
}
func (mc *mockChainService) GenesisBlock() (*types.Block, error) {
return types.NewGenesisBlock([32]byte{}, [32]byte{}), nil
}
func (mc *mockChainService) CurrentBeaconSlot() uint64 {
return 0
}
func TestLifecycle(t *testing.T) {
hook := logTest.NewGlobal()
db := database.NewKVStore()
cfg := &Config{
Delay: time.Second,
BlockRequestBuf: 0,
P2P: &mockP2P{},
Web3Service: &mockPOWChainService{},
ChainService: &mockChainService{},
BeaconDB: db,
EnablePOWChain: true,
}
sim := NewSimulator(context.Background(), cfg)
return NewSimulator(ctx, cfg)
}
func TestLifecycle(t *testing.T) {
hook := logTest.NewGlobal()
sim := setupSimulator(t)
sim.Start()
testutil.AssertLogsContain(t, hook, "Starting service")
@@ -86,34 +78,22 @@ func TestLifecycle(t *testing.T) {
func TestBroadcastBlockHash(t *testing.T) {
hook := logTest.NewGlobal()
db := database.NewKVStore()
cfg := &Config{
Delay: time.Second,
BlockRequestBuf: 0,
P2P: &mockP2P{},
Web3Service: &mockPOWChainService{},
ChainService: &mockChainService{},
BeaconDB: db,
EnablePOWChain: false,
}
sim := NewSimulator(context.Background(), cfg)
sim := setupSimulator(t)
delayChan := make(chan time.Time)
doneChan := make(chan struct{})
exitRoutine := make(chan bool)
go func() {
sim.run(delayChan, doneChan)
sim.run(delayChan)
<-exitRoutine
}()
delayChan <- time.Time{}
doneChan <- struct{}{}
sim.cancel()
exitRoutine <- true
testutil.AssertLogsContain(t, hook, "Announcing block hash")
exitRoutine <- true
if len(sim.broadcastedBlockHashes) != 1 {
t.Error("Did not store the broadcasted block hash")
}
@@ -122,24 +102,13 @@ func TestBroadcastBlockHash(t *testing.T) {
func TestBlockRequest(t *testing.T) {
hook := logTest.NewGlobal()
db := database.NewKVStore()
cfg := &Config{
Delay: time.Second,
BlockRequestBuf: 0,
P2P: &mockP2P{},
Web3Service: &mockPOWChainService{},
ChainService: &mockChainService{},
BeaconDB: db,
EnablePOWChain: false,
}
sim := NewSimulator(context.Background(), cfg)
sim := setupSimulator(t)
delayChan := make(chan time.Time)
doneChan := make(chan struct{})
exitRoutine := make(chan bool)
go func() {
sim.run(delayChan, doneChan)
sim.run(delayChan)
<-exitRoutine
}()
@@ -161,37 +130,8 @@ func TestBlockRequest(t *testing.T) {
sim.broadcastedBlocks[h] = block
sim.blockRequestChan <- msg
doneChan <- struct{}{}
sim.cancel()
exitRoutine <- true
testutil.AssertLogsContain(t, hook, fmt.Sprintf("Responding to full block request for hash: 0x%x", h))
}
func TestLastSimulatedSession(t *testing.T) {
db := database.NewKVStore()
cfg := &Config{
Delay: time.Second,
BlockRequestBuf: 0,
P2P: &mockP2P{},
Web3Service: &mockPOWChainService{},
ChainService: &mockChainService{},
BeaconDB: db,
EnablePOWChain: false,
}
sim := NewSimulator(context.Background(), cfg)
if err := db.Put([]byte("last-simulated-block"), []byte{}); err != nil {
t.Fatalf("Could not store last simulated block: %v", err)
}
if _, err := sim.lastSimulatedSessionBlock(); err != nil {
t.Errorf("could not fetch last simulated session block: %v", err)
}
}
func TestDefaultConfig(t *testing.T) {
if DefaultConfig().BlockRequestBuf != 100 {
t.Errorf("incorrect default config for block request buffer")
}
if DefaultConfig().Delay != time.Second*time.Duration(params.GetConfig().SlotDuration) {
t.Errorf("incorrect default config for delay")
}
}

View File

@@ -23,6 +23,7 @@ go_test(
embed = [":go_default_library"],
race = "off", # TODO(#377): fix issues with race detection testing.
deps = [
"//beacon-chain/db:go_default_library",
"//beacon-chain/types:go_default_library",
"//proto/beacon/p2p/v1:go_default_library",
"//shared/event:go_default_library",

View File

@@ -31,6 +31,9 @@ type Config struct {
SyncPollingInterval time.Duration
BlockBufferSize int
CrystallizedStateBufferSize int
BeaconDB beaconDB
P2P shared.P2P
SyncService syncService
}
// DefaultConfig provides the default configuration for a sync service.
@@ -45,15 +48,14 @@ func DefaultConfig() Config {
}
}
// ChainService is the interface for the blockchain package's ChainService struct.
type ChainService interface {
type beaconDB interface {
HasStoredState() (bool, error)
SaveBlock(*types.Block) error
}
// SyncService is the interface for the Sync service.
// InitialSync calls `Start` when initial sync completes.
type SyncService interface {
type syncService interface {
Start()
}
@@ -63,8 +65,8 @@ type InitialSync struct {
ctx context.Context
cancel context.CancelFunc
p2p shared.P2P
chainService ChainService
syncService SyncService
syncService syncService
db beaconDB
blockBuf chan p2p.Message
crystallizedStateBuf chan p2p.Message
currentSlotNumber uint64
@@ -76,9 +78,6 @@ type InitialSync struct {
// This method is normally called by the main node.
func NewInitialSyncService(ctx context.Context,
cfg Config,
beaconp2p shared.P2P,
chainService ChainService,
syncService SyncService,
) *InitialSync {
ctx, cancel := context.WithCancel(ctx)
@@ -88,9 +87,9 @@ func NewInitialSyncService(ctx context.Context,
return &InitialSync{
ctx: ctx,
cancel: cancel,
p2p: beaconp2p,
chainService: chainService,
syncService: syncService,
p2p: cfg.P2P,
syncService: cfg.SyncService,
db: cfg.BeaconDB,
blockBuf: blockBuf,
crystallizedStateBuf: crystallizedStateBuf,
syncPollingInterval: cfg.SyncPollingInterval,
@@ -99,7 +98,7 @@ func NewInitialSyncService(ctx context.Context,
// Start begins the goroutine.
func (s *InitialSync) Start() {
stored, err := s.chainService.HasStoredState()
stored, err := s.db.HasStoredState()
if err != nil {
log.Errorf("error retrieving stored state: %v", err)
return
@@ -262,5 +261,5 @@ func (s *InitialSync) validateAndSaveNextBlock(data *pb.BeaconBlockResponse) err
// writeBlockToDB saves the corresponding block to the local DB.
func (s *InitialSync) writeBlockToDB(block *types.Block) error {
return s.chainService.SaveBlock(block)
return s.db.SaveBlock(block)
}

View File

@@ -26,22 +26,6 @@ func (mp *mockP2P) Broadcast(msg proto.Message) {}
func (mp *mockP2P) Send(msg proto.Message, peer p2p.Peer) {
}
type mockChainService struct {
hasStoredState bool
}
func (mcs *mockChainService) HasStoredState() (bool, error) {
return mcs.hasStoredState, nil
}
func (mcs *mockChainService) setState(flag bool) {
mcs.hasStoredState = flag
}
func (mcs *mockChainService) SaveBlock(*types.Block) error {
return nil
}
type mockSyncService struct {
hasStarted bool
}
@@ -50,10 +34,26 @@ func (ms *mockSyncService) Start() {
ms.hasStarted = true
}
type mockDB struct{}
func (m *mockDB) HasStoredState() (bool, error) {
return true, nil
}
func (m *mockDB) SaveBlock(*types.Block) error {
return nil
}
func TestSetBlockForInitialSync(t *testing.T) {
hook := logTest.NewGlobal()
ss := NewInitialSyncService(context.Background(), Config{}, &mockP2P{}, &mockChainService{}, &mockSyncService{})
cfg := Config{
P2P: &mockP2P{},
SyncService: &mockSyncService{},
BeaconDB: &mockDB{},
}
ss := NewInitialSyncService(context.Background(), cfg)
exitRoutine := make(chan bool)
delayChan := make(chan time.Time)
@@ -102,7 +102,12 @@ func TestSetBlockForInitialSync(t *testing.T) {
func TestSavingBlocksInSync(t *testing.T) {
hook := logTest.NewGlobal()
ss := NewInitialSyncService(context.Background(), Config{}, &mockP2P{}, &mockChainService{}, &mockSyncService{})
cfg := Config{
P2P: &mockP2P{},
SyncService: &mockSyncService{},
BeaconDB: &mockDB{},
}
ss := NewInitialSyncService(context.Background(), cfg)
exitRoutine := make(chan bool)
delayChan := make(chan time.Time)
@@ -206,7 +211,12 @@ func TestSavingBlocksInSync(t *testing.T) {
func TestDelayChan(t *testing.T) {
hook := logTest.NewGlobal()
ss := NewInitialSyncService(context.Background(), Config{}, &mockP2P{}, &mockChainService{}, &mockSyncService{})
cfg := Config{
P2P: &mockP2P{},
SyncService: &mockSyncService{},
BeaconDB: &mockDB{},
}
ss := NewInitialSyncService(context.Background(), cfg)
exitRoutine := make(chan bool)
delayChan := make(chan time.Time)
@@ -276,23 +286,3 @@ func TestDelayChan(t *testing.T) {
hook.Reset()
}
func TestStartEmptyState(t *testing.T) {
hook := logTest.NewGlobal()
cfg := DefaultConfig()
mcs := &mockChainService{}
ss := NewInitialSyncService(context.Background(), cfg, &mockP2P{}, mcs, &mockSyncService{})
mcs.setState(true)
ss.Start()
testutil.AssertLogsContain(t, hook, "Chain state detected, exiting initial sync")
hook.Reset()
mcs.setState(false)
ss.Start()
testutil.AssertLogsDoNotContain(t, hook, "Chain state detected, exiting initial sync")
ss.cancel()
}

View File

@@ -18,20 +18,24 @@ import (
var log = logrus.WithField("prefix", "sync")
type chainService interface {
ContainsBlock(h [32]byte) (bool, error)
HasStoredState() (bool, error)
IncomingBlockFeed() *event.Feed
CheckForCanonicalBlockBySlot(slotNumber uint64) (bool, error)
CanonicalBlockBySlotNumber(slotNumber uint64) (*types.Block, error)
BlockSlotNumberByHash(h [32]byte) (uint64, error)
CurrentCrystallizedState() *types.CrystallizedState
}
type attestationService interface {
ContainsAttestation(bitfield []byte, h [32]byte) (bool, error)
IncomingAttestationFeed() *event.Feed
}
type beaconDB interface {
GetCrystallizedState() *types.CrystallizedState
GetBlock([32]byte) (*types.Block, error)
GetAttestation([32]byte) (*types.Attestation, error)
HasAttestation([32]byte) (bool, error)
HasStoredState() (bool, error)
HasBlock([32]byte) (bool, error)
HasCanonicalBlockForSlot(uint64) (bool, error)
GetCanonicalBlockForSlot(uint64) (*types.Block, error)
}
// Service is the gateway and the bridge between the p2p network and the local beacon chain.
// In broad terms, a new block is synced in 4 steps:
// 1. Receive a block hash from a peer
@@ -50,6 +54,7 @@ type Service struct {
p2p shared.P2P
chainService chainService
attestationService attestationService
db beaconDB
blockAnnouncementFeed *event.Feed
announceBlockHashBuf chan p2p.Message
blockBuf chan p2p.Message
@@ -66,6 +71,8 @@ type Config struct {
AttestationBufferSize int
ChainService chainService
AttestService attestationService
BeaconDB beaconDB
P2P shared.P2P
EnableAttestationValidity bool
}
@@ -80,13 +87,14 @@ func DefaultConfig() Config {
}
// NewSyncService accepts a context and returns a new Service.
func NewSyncService(ctx context.Context, cfg Config, beaconp2p shared.P2P) *Service {
func NewSyncService(ctx context.Context, cfg Config) *Service {
ctx, cancel := context.WithCancel(ctx)
return &Service{
ctx: ctx,
cancel: cancel,
p2p: beaconp2p,
p2p: cfg.P2P,
chainService: cfg.ChainService,
db: cfg.BeaconDB,
attestationService: cfg.AttestService,
blockAnnouncementFeed: new(event.Feed),
announceBlockHashBuf: make(chan p2p.Message, cfg.BlockHashBufferSize),
@@ -99,7 +107,7 @@ func NewSyncService(ctx context.Context, cfg Config, beaconp2p shared.P2P) *Serv
// Start begins the block processing goroutine.
func (ss *Service) Start() {
stored, err := ss.chainService.HasStoredState()
stored, err := ss.db.HasStoredState()
if err != nil {
log.Errorf("error retrieving stored state: %v", err)
return
@@ -169,7 +177,7 @@ func (ss *Service) receiveBlockHash(msg p2p.Message) {
copy(h[:], data.Hash[:32])
ctx, containsBlockSpan := trace.StartSpan(ctx, "containsBlock")
blockExists, err := ss.chainService.ContainsBlock(h)
blockExists, err := ss.db.HasBlock(h)
containsBlockSpan.End()
if err != nil {
log.Errorf("Received block hash failed: %v", err)
@@ -198,7 +206,7 @@ func (ss *Service) receiveBlock(msg p2p.Message) {
}
ctx, containsBlockSpan := trace.StartSpan(ctx, "containsBlock")
blockExists, err := ss.chainService.ContainsBlock(blockHash)
blockExists, err := ss.db.HasBlock(blockHash)
containsBlockSpan.End()
if err != nil {
log.Errorf("Can not check for block in DB: %v", err)
@@ -211,12 +219,13 @@ func (ss *Service) receiveBlock(msg p2p.Message) {
if ss.enableAttestationValidity {
// Verify attestation coming from proposer then forward block to the subscribers.
attestation := types.NewAttestation(response.Attestation)
cState := ss.chainService.CurrentCrystallizedState()
parentSlot, err := ss.chainService.BlockSlotNumberByHash(block.ParentHash())
cState := ss.db.GetCrystallizedState()
parentBlock, err := ss.db.GetBlock(block.ParentHash())
if err != nil {
log.Errorf("Failed to get parent slot: %v", err)
return
}
parentSlot := parentBlock.SlotNumber()
proposerShardID, _, err := casper.ProposerShardAndIndex(cState.ShardAndCommitteesForSlots(), cState.LastStateRecalc(), parentSlot)
if err != nil {
log.Errorf("Failed to get proposer shard ID: %v", err)
@@ -252,7 +261,7 @@ func (ss *Service) handleBlockRequestBySlot(msg p2p.Message) {
}
ctx, checkForBlockSpan := trace.StartSpan(ctx, "checkForBlockBySlot")
blockExists, err := ss.chainService.CheckForCanonicalBlockBySlot(request.GetSlotNumber())
blockExists, err := ss.db.HasCanonicalBlockForSlot(request.GetSlotNumber())
checkForBlockSpan.End()
if err != nil {
log.Errorf("Error checking db for block %v", err)
@@ -263,10 +272,10 @@ func (ss *Service) handleBlockRequestBySlot(msg p2p.Message) {
}
ctx, getBlockSpan := trace.StartSpan(ctx, "getBlockBySlot")
block, err := ss.chainService.CanonicalBlockBySlotNumber(request.GetSlotNumber())
block, err := ss.db.GetCanonicalBlockForSlot(request.GetSlotNumber())
getBlockSpan.End()
if err != nil {
log.Errorf("Error retrieving block from db %v", err)
log.Errorf("Error retrieving block from db: %v", err)
return
}
@@ -289,14 +298,21 @@ func (ss *Service) receiveAttestation(msg p2p.Message) {
_, containsAttestationSpan := trace.StartSpan(ctx, "containsAttestation")
containsAttestationSpan.End()
attestationExists, err := ss.attestationService.ContainsAttestation(a.AttesterBitfield(), h)
hasAttestation, err := ss.db.HasAttestation(h)
if err != nil {
log.Errorf("Can not check for attestation in DB: %v", err)
return
log.Errorf("Failed check for existence of attestation: %v", err)
}
if attestationExists {
log.Debugf("Received attestation 0x%v already", h)
return
if hasAttestation {
attestation, err := ss.db.GetAttestation(h)
if err != nil {
log.Errorf("Can not check for attestation in DB: %v", err)
return
}
validatorExists := attestation.ContainsValidator(a.AttesterBitfield())
if validatorExists {
log.Debugf("Received attestation 0x%v already", h)
return
}
}
log.WithField("attestationHash", fmt.Sprintf("0x%x", h)).Debug("Forwarding attestation to subscribed services")

View File

@@ -2,12 +2,11 @@ package sync
import (
"context"
"errors"
"fmt"
"io/ioutil"
"testing"
"github.com/golang/protobuf/proto"
"github.com/prysmaticlabs/prysm/beacon-chain/db"
"github.com/prysmaticlabs/prysm/beacon-chain/types"
pb "github.com/prysmaticlabs/prysm/proto/beacon/p2p/v1"
"github.com/prysmaticlabs/prysm/shared/event"
@@ -35,79 +34,52 @@ func (mp *mockP2P) Broadcast(msg proto.Message) {}
func (mp *mockP2P) Send(msg proto.Message, peer p2p.Peer) {
}
type mockChainService struct {
slotExists bool
checkError bool
getError bool
}
func (ms *mockChainService) ContainsBlock(h [32]byte) (bool, error) {
return false, nil
}
func (ms *mockChainService) HasStoredState() (bool, error) {
return false, nil
}
type mockChainService struct{}
func (ms *mockChainService) IncomingBlockFeed() *event.Feed {
return new(event.Feed)
}
func (ms *mockChainService) CurrentCrystallizedState() *types.CrystallizedState {
cState, err := types.NewGenesisCrystallizedState("")
if err != nil {
fmt.Println(err)
}
return cState
}
func (ms *mockChainService) BlockSlotNumberByHash(h [32]byte) (uint64, error) {
return 0, nil
}
func (ms *mockChainService) CheckForCanonicalBlockBySlot(slotnumber uint64) (bool, error) {
if ms.checkError {
return ms.slotExists, errors.New("mock check canonical block error")
}
return ms.slotExists, nil
}
func (ms *mockChainService) CanonicalBlockBySlotNumber(slotnumber uint64) (*types.Block, error) {
if ms.getError {
return nil, errors.New("mock get canonical block error")
}
if !ms.slotExists {
return nil, errors.New("invalid key")
}
return types.NewBlock(&pb.BeaconBlock{SlotNumber: slotnumber}), nil
}
type faultyMockAttestService struct{}
func (ms *faultyMockAttestService) IncomingAttestationFeed() *event.Feed {
return new(event.Feed)
}
func (ms *faultyMockAttestService) ContainsAttestation(bitfield []byte, h [32]byte) (bool, error) {
return false, errors.New("Can not check for attestation in DB")
}
type mockAttestService struct{}
func (ms *mockAttestService) IncomingAttestationFeed() *event.Feed {
return new(event.Feed)
}
func (ms *mockAttestService) ContainsAttestation(bitfield []byte, h [32]byte) (bool, error) {
return false, nil
func setupDB(t *testing.T) *db.BeaconDB {
dbConfig := db.Config{Path: "", Name: "", InMemory: true}
db, err := db.NewDB(dbConfig)
if err != nil {
t.Fatalf("could not setup beaconDB: %v", err)
}
return db
}
func setupService(t *testing.T) *Service {
db := setupDB(t)
cfg := Config{
BlockHashBufferSize: 0,
BlockBufferSize: 0,
ChainService: &mockChainService{},
P2P: &mockP2P{},
BeaconDB: db,
}
return NewSyncService(context.Background(), cfg)
}
func TestProcessBlockHash(t *testing.T) {
hook := logTest.NewGlobal()
// set the channel's buffer to 0 to make channel interactions blocking
cfg := Config{BlockHashBufferSize: 0, BlockBufferSize: 0, ChainService: &mockChainService{}}
ss := NewSyncService(context.Background(), cfg, &mockP2P{})
cfg := Config{
BlockHashBufferSize: 0,
BlockBufferSize: 0,
ChainService: &mockChainService{},
P2P: &mockP2P{},
BeaconDB: setupDB(t),
}
ss := NewSyncService(context.Background(), cfg)
exitRoutine := make(chan bool)
@@ -140,19 +112,37 @@ func TestProcessBlockHash(t *testing.T) {
func TestProcessBlock(t *testing.T) {
hook := logTest.NewGlobal()
cfg := Config{BlockHashBufferSize: 0, BlockBufferSize: 0, ChainService: &mockChainService{}, AttestService: &mockAttestService{}}
ss := NewSyncService(context.Background(), cfg, &mockP2P{})
db := setupDB(t)
cfg := Config{
BlockHashBufferSize: 0,
BlockBufferSize: 0,
ChainService: &mockChainService{},
P2P: &mockP2P{},
BeaconDB: db,
AttestService: &mockAttestService{},
}
ss := NewSyncService(context.Background(), cfg)
exitRoutine := make(chan bool)
go func() {
ss.run()
exitRoutine <- true
}()
parentBlock := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 0,
})
if err := db.SaveBlock(parentBlock); err != nil {
t.Fatalf("failed to save block: %v", err)
}
parentHash, err := parentBlock.Hash()
if err != nil {
t.Fatalf("failed to get parent hash: %v", err)
}
data := &pb.BeaconBlock{
PowChainRef: []byte{1, 2, 3, 4, 5},
ParentHash: make([]byte, 32),
ParentHash: parentHash[:],
}
attestation := &pb.AggregatedAttestation{
Slot: 0,
@@ -182,8 +172,16 @@ func TestProcessBlock(t *testing.T) {
func TestProcessMultipleBlocks(t *testing.T) {
hook := logTest.NewGlobal()
cfg := Config{BlockHashBufferSize: 0, BlockBufferSize: 0, ChainService: &mockChainService{}, AttestService: &mockAttestService{}}
ss := NewSyncService(context.Background(), cfg, &mockP2P{})
db := setupDB(t)
cfg := Config{
BlockHashBufferSize: 0,
BlockBufferSize: 0,
ChainService: &mockChainService{},
P2P: &mockP2P{},
BeaconDB: db,
AttestService: &mockAttestService{},
}
ss := NewSyncService(context.Background(), cfg)
exitRoutine := make(chan bool)
@@ -192,9 +190,20 @@ func TestProcessMultipleBlocks(t *testing.T) {
exitRoutine <- true
}()
parentBlock := types.NewBlock(&pb.BeaconBlock{
SlotNumber: 0,
})
if err := db.SaveBlock(parentBlock); err != nil {
t.Fatalf("failed to save block: %v", err)
}
parentHash, err := parentBlock.Hash()
if err != nil {
t.Fatalf("failed to get parent hash: %v", err)
}
data1 := &pb.BeaconBlock{
PowChainRef: []byte{1, 2, 3, 4, 5},
ParentHash: make([]byte, 32),
ParentHash: parentHash[:],
}
responseBlock1 := &pb.BeaconBlockResponse{
@@ -236,8 +245,7 @@ func TestProcessMultipleBlocks(t *testing.T) {
func TestBlockRequestErrors(t *testing.T) {
hook := logTest.NewGlobal()
cfg := Config{BlockHashBufferSize: 0, BlockBufferSize: 0, BlockRequestBufferSize: 0, ChainService: &mockChainService{}}
ss := NewSyncService(context.Background(), cfg, &mockP2P{})
ss := setupService(t)
exitRoutine := make(chan bool)
@@ -275,112 +283,20 @@ func TestBlockRequestErrors(t *testing.T) {
}
func TestBlockRequestGetCanonicalError(t *testing.T) {
hook := logTest.NewGlobal()
ms := &mockChainService{}
cfg := Config{BlockHashBufferSize: 0, BlockBufferSize: 0, BlockRequestBufferSize: 0, ChainService: ms}
ss := NewSyncService(context.Background(), cfg, &mockP2P{})
exitRoutine := make(chan bool)
go func() {
ss.run()
exitRoutine <- true
}()
request1 := &pb.BeaconBlockRequestBySlotNumber{
SlotNumber: 20,
}
msg1 := p2p.Message{
Ctx: context.Background(),
Data: request1,
Peer: p2p.Peer{},
}
ms.slotExists = true
ms.getError = true
ss.blockRequestBySlot <- msg1
testutil.AssertLogsContain(t, hook, "Error retrieving block from db mock get canonical block error")
hook.Reset()
}
func TestBlockRequestBySlot(t *testing.T) {
hook := logTest.NewGlobal()
ms := &mockChainService{}
cfg := Config{BlockHashBufferSize: 0, BlockBufferSize: 0, BlockRequestBufferSize: 0, ChainService: ms}
ss := NewSyncService(context.Background(), cfg, &mockP2P{})
exitRoutine := make(chan bool)
go func() {
ss.run()
exitRoutine <- true
}()
request1 := &pb.BeaconBlockRequestBySlotNumber{
SlotNumber: 20,
}
msg1 := p2p.Message{
Ctx: context.Background(),
Data: request1,
Peer: p2p.Peer{},
}
ms.checkError = true
ms.slotExists = true
ss.blockRequestBySlot <- msg1
testutil.AssertLogsContain(t, hook, "Error checking db for block mock check canonical block error")
ms.checkError = false
ss.blockRequestBySlot <- msg1
ss.cancel()
<-exitRoutine
testutil.AssertLogsContain(t, hook, "Sending requested block to peer")
hook.Reset()
}
func TestFaultyReceiveAttestationService(t *testing.T) {
hook := logTest.NewGlobal()
ms := &mockChainService{}
as := &faultyMockAttestService{}
cfg := Config{BlockHashBufferSize: 0, BlockBufferSize: 0, BlockRequestBufferSize: 0, ChainService: ms, AttestService: as}
ss := NewSyncService(context.Background(), cfg, &mockP2P{})
exitRoutine := make(chan bool)
go func() {
ss.run()
exitRoutine <- true
}()
request1 := &pb.AggregatedAttestation{
Slot: 0,
AttesterBitfield: []byte{99},
}
msg1 := p2p.Message{
Ctx: context.Background(),
Data: request1,
Peer: p2p.Peer{},
}
ss.attestationBuf <- msg1
ss.cancel()
<-exitRoutine
testutil.AssertLogsContain(t, hook, "Can not check for attestation in DB")
}
func TestReceiveAttestation(t *testing.T) {
hook := logTest.NewGlobal()
ms := &mockChainService{}
as := &mockAttestService{}
cfg := Config{BlockHashBufferSize: 0, BlockBufferSize: 0, BlockRequestBufferSize: 0, ChainService: ms, AttestService: as}
ss := NewSyncService(context.Background(), cfg, &mockP2P{})
cfg := Config{
BlockHashBufferSize: 0,
BlockBufferSize: 0,
BlockRequestBufferSize: 0,
ChainService: ms,
AttestService: as,
P2P: &mockP2P{},
BeaconDB: setupDB(t),
}
ss := NewSyncService(context.Background(), cfg)
exitRoutine := make(chan bool)
@@ -406,58 +322,21 @@ func TestReceiveAttestation(t *testing.T) {
testutil.AssertLogsContain(t, hook, "Forwarding attestation to subscribed services")
}
type mockEmptyChainService struct {
hasStoredState bool
}
func (ms *mockEmptyChainService) ContainsBlock(h [32]byte) (bool, error) {
return false, nil
}
func (ms *mockEmptyChainService) HasStoredState() (bool, error) {
return ms.hasStoredState, nil
}
func (ms *mockEmptyChainService) IncomingBlockFeed() *event.Feed {
return new(event.Feed)
}
func (ms *mockEmptyChainService) IncomingAttestationFeed() *event.Feed {
return new(event.Feed)
}
func (ms *mockEmptyChainService) setState(flag bool) {
ms.hasStoredState = flag
}
func (ms *mockEmptyChainService) CheckForCanonicalBlockBySlot(slotnumber uint64) (bool, error) {
return false, nil
}
func (ms *mockEmptyChainService) CanonicalBlockBySlotNumber(slotnumber uint64) (*types.Block, error) {
return nil, nil
}
func (ms *mockEmptyChainService) CurrentCrystallizedState() *types.CrystallizedState {
return types.NewCrystallizedState(nil)
}
func (ms *mockEmptyChainService) BlockSlotNumberByHash(h [32]byte) (uint64, error) {
return 0, nil
}
func TestStartEmptyState(t *testing.T) {
hook := logTest.NewGlobal()
ms := &mockEmptyChainService{}
db := setupDB(t)
cfg := DefaultConfig()
cfg.ChainService = ms
ss := NewSyncService(context.Background(), cfg, &mockP2P{})
cfg.ChainService = &mockChainService{}
cfg.P2P = &mockP2P{}
cfg.BeaconDB = db
ss := NewSyncService(context.Background(), cfg)
ss.Start()
testutil.AssertLogsContain(t, hook, "Empty chain state, but continue sync")
hook.Reset()
ms.setState(true)
db.SaveCrystallizedState(db.GetCrystallizedState())
ss.Start()
testutil.AssertLogsDoNotContain(t, hook, "Empty chain state, but continue sync")

View File

@@ -163,3 +163,15 @@ func AttestationMsg(parentHashes [][32]byte, blockHash []byte, slot uint64, shar
copy(hashMsg[:], h[:32])
return hashMsg
}
// ContainsValidator checks if the validator is included in the attestation.
// TODO(#569): Modify method to accept a single index rather than a bitfield.
func (a *Attestation) ContainsValidator(bitfield []byte) bool {
savedAttestationBitfield := a.AttesterBitfield()
for i := 0; i < len(bitfield); i++ {
if bitfield[i]&savedAttestationBitfield[i] != 0 {
return true
}
}
return false
}

View File

@@ -56,3 +56,19 @@ func TestAttestation(t *testing.T) {
t.Errorf("verify attestation failed: %v", err)
}
}
func TestContainsValidator(t *testing.T) {
attestation := NewAttestation(&pb.AggregatedAttestation{
Slot: 0,
ShardId: 0,
AttesterBitfield: []byte{7}, // 0000 0111
})
if !attestation.ContainsValidator([]byte{4}) {
t.Error("Attestation should contain validator")
}
if attestation.ContainsValidator([]byte{8}) {
t.Error("Attestation should not contain validator")
}
}

View File

@@ -26,8 +26,8 @@ type Block struct {
data *pb.BeaconBlock
}
type chainSearchService interface {
ContainsBlock(h [32]byte) (bool, error)
type beaconDB interface {
HasBlock(h [32]byte) (bool, error)
}
// NewBlock explicitly sets the data field of a block.
@@ -148,7 +148,7 @@ func (b *Block) isSlotValid(genesisTimestamp time.Time) bool {
// 1.) Ensure local time is large enough to process this block's slot.
// 2.) Verify that the parent block's proposer's attestation is included.
func (b *Block) IsValid(
chain chainSearchService,
db beaconDB,
aState *ActiveState,
cState *CrystallizedState,
parentSlot uint64,
@@ -180,16 +180,14 @@ func (b *Block) IsValid(
log.Errorf("Can not get proposer index %v", err)
return false
}
log.Infof("Proposer index: %v", proposerIndex)
if !shared.CheckBit(b.Attestations()[0].AttesterBitfield, int(proposerIndex)) {
log.Errorf("Can not locate proposer in the first attestation of AttestionRecord %v", err)
return false
}
log.Debugf("Checking block validity. Recent block hash is %d",
aState.data.RecentBlockHashes[0],
)
for index, attestation := range b.Attestations() {
if !b.isAttestationValid(index, chain, aState, cState, parentSlot) {
if !b.isAttestationValid(index, db, aState, cState, parentSlot) {
log.Debugf("attestation invalid: %v", attestation)
return false
}
@@ -202,7 +200,7 @@ func (b *Block) IsValid(
// isAttestationValid validates an attestation in a block.
// Attestations are cross-checked against validators in CrystallizedState.ShardAndCommitteesForSlots.
// In addition, the signature is verified by constructing the list of parent hashes using ActiveState.RecentBlockHashes.
func (b *Block) isAttestationValid(attestationIndex int, chain chainSearchService, aState *ActiveState, cState *CrystallizedState, parentSlot uint64) bool {
func (b *Block) isAttestationValid(attestationIndex int, db beaconDB, aState *ActiveState, cState *CrystallizedState, parentSlot uint64) bool {
// Validate attestation's slot number has is within range of incoming block number.
attestation := b.Attestations()[attestationIndex]
if !isAttestationSlotNumberValid(attestation.Slot, parentSlot) {
@@ -218,7 +216,7 @@ func (b *Block) isAttestationValid(attestationIndex int, chain chainSearchServic
hash := [32]byte{}
copy(hash[:], attestation.JustifiedBlockHash)
blockInChain, err := chain.ContainsBlock(hash)
blockInChain, err := db.HasBlock(hash)
if err != nil {
log.Errorf("unable to determine if attestation justified block is in the DB: %s", err)
return false
@@ -256,7 +254,6 @@ func (b *Block) isAttestationValid(attestationIndex int, chain chainSearchServic
attestation.ShardId, attestation.Slot, attestation.ShardBlockHash, attestationMsg)
// TODO(#258): Verify msgHash against aggregated pub key and aggregated signature.
return true
}

View File

@@ -14,9 +14,9 @@ func init() {
logrus.SetLevel(logrus.DebugLevel)
}
type mockChainService struct{}
type mockDB struct{}
func (f *mockChainService) ContainsBlock(h [32]byte) (bool, error) {
func (f *mockDB) HasBlock(h [32]byte) (bool, error) {
return true, nil
}
@@ -101,17 +101,17 @@ func TestBlockValidity(t *testing.T) {
})
parentSlot := uint64(1)
chainService := &mockChainService{}
db := &mockDB{}
if !b.isAttestationValid(0, chainService, aState, cState, parentSlot) {
if !b.isAttestationValid(0, db, aState, cState, parentSlot) {
t.Fatalf("failed attestation validation")
}
genesisTime := params.GetConfig().GenesisTime
if !b.IsValid(chainService, aState, cState, parentSlot, false, genesisTime) {
if !b.IsValid(db, aState, cState, parentSlot, false, genesisTime) {
t.Fatalf("failed block validation")
}
if !b.IsValid(chainService, aState, cState, parentSlot, true, genesisTime) {
if !b.IsValid(db, aState, cState, parentSlot, true, genesisTime) {
t.Fatalf("failed block validation")
}
}

View File

@@ -6,6 +6,7 @@ go_library(
"clock.go",
"flags.go",
"shuffle.go",
"slot_interval.go",
],
importpath = "github.com/prysmaticlabs/prysm/beacon-chain/utils",
visibility = ["//beacon-chain:__subpackages__"],

View File

@@ -0,0 +1,17 @@
package utils
import (
"time"
"github.com/prysmaticlabs/prysm/beacon-chain/params"
)
// CurrentSlot returns slot number based on the genesis timestamp.
func CurrentSlot(genesisTime time.Time) uint64 {
secondsSinceGenesis := uint64(time.Since(genesisTime).Seconds())
currentSlot := secondsSinceGenesis / params.GetConfig().SlotDuration
if currentSlot < 1 {
return 0
}
return currentSlot - 1
}