block: replace Base with Header

This commit is contained in:
Evgeniy Stratonikov 2021-03-01 16:44:47 +03:00
parent 4df8a2ad36
commit f83b376181
25 changed files with 292 additions and 340 deletions

View file

@ -236,7 +236,7 @@ func (chain *FakeChain) GetHeader(hash util.Uint256) (*block.Header, error) {
if err != nil {
return nil, err
}
return b.Header(), nil
return &b.Header, nil
}
// GetNextBlockValidators implements Blockchainer interface.

View file

@ -163,7 +163,7 @@ func NewBlock(t *testing.T, bc blockchainer.Blockchainer, offset uint32, primary
hdr, err := bc.GetHeader(h)
require.NoError(t, err)
b := &block.Block{
Base: block.Base{
Header: block.Header{
PrevHash: hdr.Hash(),
Timestamp: (uint64(time.Now().UTC().Unix()) + uint64(hdr.Index)) * 1000,
Index: hdr.Index + offset,

View file

@ -20,7 +20,7 @@ var _ block.Block = (*neoBlock)(nil)
// Sign implements block.Block interface.
func (n *neoBlock) Sign(key crypto.PrivateKey) error {
data := n.Base.GetSignedPart()
data := n.Header.GetSignedPart()
sig, err := key.Sign(data[:])
if err != nil {
return err
@ -33,7 +33,7 @@ func (n *neoBlock) Sign(key crypto.PrivateKey) error {
// Verify implements block.Block interface.
func (n *neoBlock) Verify(key crypto.PublicKey, sign []byte) error {
data := n.Base.GetSignedPart()
data := n.Header.GetSignedPart()
return key.Verify(data, sign)
}

View file

@ -24,7 +24,7 @@ var ErrMaxContentsPerBlock = errors.New("the number of contents exceeds the maxi
// Block represents one block in the chain.
type Block struct {
// The base of the block.
Base
Header
// Transaction list.
Transactions []*transaction.Transaction
@ -43,13 +43,6 @@ type auxBlockIn struct {
Transactions []json.RawMessage `json:"tx"`
}
// Header returns the Header of the Block.
func (b *Block) Header() *Header {
return &Header{
Base: b.Base,
}
}
// ComputeMerkleRoot computes Merkle tree root hash based on actual block's data.
func (b *Block) ComputeMerkleRoot() util.Uint256 {
hashes := make([]util.Uint256, len(b.Transactions))
@ -71,7 +64,7 @@ func (b *Block) RebuildMerkleRoot() {
// set to true.
func NewBlockFromTrimmedBytes(network netmode.Magic, stateRootEnabled bool, b []byte) (*Block, error) {
block := &Block{
Base: Base{
Header: Header{
Network: network,
StateRootEnabled: stateRootEnabled,
},
@ -79,12 +72,7 @@ func NewBlockFromTrimmedBytes(network netmode.Magic, stateRootEnabled bool, b []
}
br := io.NewBinReaderFromBuf(b)
block.decodeHashableFields(br)
_ = br.ReadB()
block.Script.DecodeBinary(br)
block.Header.DecodeBinary(br)
lenHashes := br.ReadVarUint()
if lenHashes > MaxTransactionsPerBlock {
return nil, ErrMaxContentsPerBlock
@ -104,7 +92,7 @@ func NewBlockFromTrimmedBytes(network netmode.Magic, stateRootEnabled bool, b []
// New creates a new blank block tied to the specific network.
func New(network netmode.Magic, stateRootEnabled bool) *Block {
return &Block{
Base: Base{
Header: Header{
Network: network,
StateRootEnabled: stateRootEnabled,
},
@ -116,9 +104,7 @@ func New(network netmode.Magic, stateRootEnabled bool) *Block {
// Notice that only the hashes of the transactions are stored.
func (b *Block) Trim() ([]byte, error) {
buf := io.NewBufBinWriter()
b.encodeHashableFields(buf.BinWriter)
buf.WriteB(1)
b.Script.EncodeBinary(buf.BinWriter)
b.Header.EncodeBinary(buf.BinWriter)
buf.WriteVarUint(uint64(len(b.Transactions)))
for _, tx := range b.Transactions {
@ -136,7 +122,7 @@ func (b *Block) Trim() ([]byte, error) {
// DecodeBinary decodes the block from the given BinReader, implementing
// Serializable interface.
func (b *Block) DecodeBinary(br *io.BinReader) {
b.Base.DecodeBinary(br)
b.Header.DecodeBinary(br)
contentsCount := br.ReadVarUint()
if contentsCount > MaxTransactionsPerBlock {
br.Err = ErrMaxContentsPerBlock
@ -157,7 +143,7 @@ func (b *Block) DecodeBinary(br *io.BinReader) {
// EncodeBinary encodes the block to the given BinWriter, implementing
// Serializable interface.
func (b *Block) EncodeBinary(bw *io.BinWriter) {
b.Base.EncodeBinary(bw)
b.Header.EncodeBinary(bw)
bw.WriteVarUint(uint64(len(b.Transactions)))
for i := 0; i < len(b.Transactions); i++ {
b.Transactions[i].EncodeBinary(bw)
@ -185,7 +171,7 @@ func (b Block) MarshalJSON() ([]byte, error) {
if err != nil {
return nil, err
}
baseBytes, err := json.Marshal(b.Base)
baseBytes, err := json.Marshal(b.Header)
if err != nil {
return nil, err
}
@ -208,7 +194,7 @@ func (b *Block) UnmarshalJSON(data []byte) error {
if err != nil {
return err
}
err = json.Unmarshal(data, &b.Base)
err = json.Unmarshal(data, &b.Header)
if err != nil {
return err
}

View file

@ -1,223 +0,0 @@
package block
import (
"encoding/json"
"errors"
"github.com/nspcc-dev/neo-go/pkg/config/netmode"
"github.com/nspcc-dev/neo-go/pkg/core/transaction"
"github.com/nspcc-dev/neo-go/pkg/crypto/hash"
"github.com/nspcc-dev/neo-go/pkg/encoding/address"
"github.com/nspcc-dev/neo-go/pkg/io"
"github.com/nspcc-dev/neo-go/pkg/util"
)
// Base holds the base info of a block
type Base struct {
// Version of the block.
Version uint32
// hash of the previous block.
PrevHash util.Uint256
// Root hash of a transaction list.
MerkleRoot util.Uint256
// Timestamp is a millisecond-precision timestamp.
// The time stamp of each block must be later than previous block's time stamp.
// Generally the difference of two block's time stamp is about 15 seconds and imprecision is allowed.
// The height of the block must be exactly equal to the height of the previous block plus 1.
Timestamp uint64
// index/height of the block
Index uint32
// Contract address of the next miner
NextConsensus util.Uint160
// Script used to validate the block
Script transaction.Witness
// Network magic number this block belongs to. This one actually is not
// a part of the wire-representation of Block, but it's absolutely
// necessary for correct signing/verification.
Network netmode.Magic
// StateRootEnabled specifies if header contains state root.
StateRootEnabled bool
// PrevStateRoot is state root of the previous block.
PrevStateRoot util.Uint256
// PrimaryIndex is the index of primary consensus node for this block.
PrimaryIndex byte
// Hash of this block, created when binary encoded (double SHA256).
hash util.Uint256
// Hash of the block used to verify it (single SHA256).
verificationHash util.Uint256
}
// baseAux is used to marshal/unmarshal to/from JSON, it's almost the same
// as original Base, but with Nonce and NextConsensus fields differing and
// Hash added.
type baseAux struct {
Hash util.Uint256 `json:"hash"`
Version uint32 `json:"version"`
PrevHash util.Uint256 `json:"previousblockhash"`
MerkleRoot util.Uint256 `json:"merkleroot"`
Timestamp uint64 `json:"time"`
Index uint32 `json:"index"`
NextConsensus string `json:"nextconsensus"`
PrimaryIndex byte `json:"primary"`
PrevStateRoot *util.Uint256 `json:"previousstateroot,omitempty"`
Witnesses []transaction.Witness `json:"witnesses"`
}
// Hash returns the hash of the block.
func (b *Base) Hash() util.Uint256 {
if b.hash.Equals(util.Uint256{}) {
b.createHash()
}
return b.hash
}
// GetSignedHash returns a hash of the block used to verify it.
func (b *Base) GetSignedHash() util.Uint256 {
if b.verificationHash.Equals(util.Uint256{}) {
b.createHash()
}
return b.verificationHash
}
// DecodeBinary implements Serializable interface.
func (b *Base) DecodeBinary(br *io.BinReader) {
b.decodeHashableFields(br)
witnessCount := br.ReadVarUint()
if br.Err == nil && witnessCount != 1 {
br.Err = errors.New("wrong witness count")
return
}
b.Script.DecodeBinary(br)
}
// EncodeBinary implements Serializable interface
func (b *Base) EncodeBinary(bw *io.BinWriter) {
b.encodeHashableFields(bw)
bw.WriteVarUint(1)
b.Script.EncodeBinary(bw)
}
// GetSignedPart returns serialized hashable data of the block.
func (b *Base) GetSignedPart() []byte {
buf := io.NewBufBinWriter()
buf.WriteU32LE(uint32(b.Network))
// No error can occure while encoding hashable fields.
b.encodeHashableFields(buf.BinWriter)
return buf.Bytes()
}
// createHash creates the hash of the block.
// When calculating the hash value of the block, instead of calculating the entire block,
// only first seven fields in the block head will be calculated, which are
// version, PrevBlock, MerkleRoot, timestamp, and height, the nonce, NextMiner.
// Since MerkleRoot already contains the hash value of all transactions,
// the modification of transaction will influence the hash value of the block.
func (b *Base) createHash() {
bb := b.GetSignedPart()
b.verificationHash = hash.Sha256(bb)
b.hash = hash.Sha256(b.verificationHash.BytesBE())
}
// encodeHashableFields will only encode the fields used for hashing.
// see Hash() for more information about the fields.
func (b *Base) encodeHashableFields(bw *io.BinWriter) {
bw.WriteU32LE(b.Version)
bw.WriteBytes(b.PrevHash[:])
bw.WriteBytes(b.MerkleRoot[:])
bw.WriteU64LE(b.Timestamp)
bw.WriteU32LE(b.Index)
bw.WriteB(b.PrimaryIndex)
bw.WriteBytes(b.NextConsensus[:])
if b.StateRootEnabled {
bw.WriteBytes(b.PrevStateRoot[:])
}
}
// decodeHashableFields decodes the fields used for hashing.
// see Hash() for more information about the fields.
func (b *Base) decodeHashableFields(br *io.BinReader) {
b.Version = br.ReadU32LE()
br.ReadBytes(b.PrevHash[:])
br.ReadBytes(b.MerkleRoot[:])
b.Timestamp = br.ReadU64LE()
b.Index = br.ReadU32LE()
b.PrimaryIndex = br.ReadB()
br.ReadBytes(b.NextConsensus[:])
if b.StateRootEnabled {
br.ReadBytes(b.PrevStateRoot[:])
}
// Make the hash of the block here so we dont need to do this
// again.
if br.Err == nil {
b.createHash()
}
}
// MarshalJSON implements json.Marshaler interface.
func (b Base) MarshalJSON() ([]byte, error) {
aux := baseAux{
Hash: b.Hash(),
Version: b.Version,
PrevHash: b.PrevHash,
MerkleRoot: b.MerkleRoot,
Timestamp: b.Timestamp,
Index: b.Index,
PrimaryIndex: b.PrimaryIndex,
NextConsensus: address.Uint160ToString(b.NextConsensus),
Witnesses: []transaction.Witness{b.Script},
}
if b.StateRootEnabled {
aux.PrevStateRoot = &b.PrevStateRoot
}
return json.Marshal(aux)
}
// UnmarshalJSON implements json.Unmarshaler interface.
func (b *Base) UnmarshalJSON(data []byte) error {
var aux = new(baseAux)
var nextC util.Uint160
err := json.Unmarshal(data, aux)
if err != nil {
return err
}
nextC, err = address.StringToUint160(aux.NextConsensus)
if err != nil {
return err
}
if len(aux.Witnesses) != 1 {
return errors.New("wrong number of witnesses")
}
b.Version = aux.Version
b.PrevHash = aux.PrevHash
b.MerkleRoot = aux.MerkleRoot
b.Timestamp = aux.Timestamp
b.Index = aux.Index
b.PrimaryIndex = aux.PrimaryIndex
b.NextConsensus = nextC
b.Script = aux.Witnesses[0]
if b.StateRootEnabled {
if aux.PrevStateRoot == nil {
return errors.New("'previousstateroot' is empty")
}
b.PrevStateRoot = *aux.PrevStateRoot
}
if !aux.Hash.Equals(b.Hash()) {
return errors.New("json 'hash' doesn't match block hash")
}
return nil
}

View file

@ -79,7 +79,7 @@ func TestTrimmedBlock(t *testing.T) {
func newDumbBlock() *Block {
return &Block{
Base: Base{
Header: Header{
Version: 0,
PrevHash: hash.Sha256([]byte("a")),
MerkleRoot: hash.Sha256([]byte("b")),
@ -100,7 +100,7 @@ func newDumbBlock() *Block {
func TestHashBlockEqualsHashHeader(t *testing.T) {
block := newDumbBlock()
assert.Equal(t, block.Hash(), block.Header().Hash())
assert.Equal(t, block.Hash(), block.Header.Hash())
}
func TestBinBlockDecodeEncode(t *testing.T) {
@ -203,9 +203,9 @@ func TestBlockSizeCalculation(t *testing.T) {
}
func TestBlockCompare(t *testing.T) {
b1 := Block{Base: Base{Index: 1}}
b2 := Block{Base: Base{Index: 2}}
b3 := Block{Base: Base{Index: 3}}
b1 := Block{Header: Header{Index: 1}}
b2 := Block{Header: Header{Index: 2}}
b3 := Block{Header: Header{Index: 3}}
assert.Equal(t, 1, b2.Compare(&b1))
assert.Equal(t, 0, b2.Compare(&b2))
assert.Equal(t, -1, b2.Compare(&b3))

View file

@ -1,31 +1,223 @@
package block
import (
"fmt"
"encoding/json"
"errors"
"github.com/nspcc-dev/neo-go/pkg/config/netmode"
"github.com/nspcc-dev/neo-go/pkg/core/transaction"
"github.com/nspcc-dev/neo-go/pkg/crypto/hash"
"github.com/nspcc-dev/neo-go/pkg/encoding/address"
"github.com/nspcc-dev/neo-go/pkg/io"
"github.com/nspcc-dev/neo-go/pkg/util"
)
// Header holds the head info of a block.
// Header holds the base info of a block
type Header struct {
// Base of the block.
Base
// Version of the block.
Version uint32
// hash of the previous block.
PrevHash util.Uint256
// Root hash of a transaction list.
MerkleRoot util.Uint256
// Timestamp is a millisecond-precision timestamp.
// The time stamp of each block must be later than previous block's time stamp.
// Generally the difference of two block's time stamp is about 15 seconds and imprecision is allowed.
// The height of the block must be exactly equal to the height of the previous block plus 1.
Timestamp uint64
// index/height of the block
Index uint32
// Contract address of the next miner
NextConsensus util.Uint160
// Script used to validate the block
Script transaction.Witness
// Network magic number this block belongs to. This one actually is not
// a part of the wire-representation of Block, but it's absolutely
// necessary for correct signing/verification.
Network netmode.Magic
// StateRootEnabled specifies if header contains state root.
StateRootEnabled bool
// PrevStateRoot is state root of the previous block.
PrevStateRoot util.Uint256
// PrimaryIndex is the index of primary consensus node for this block.
PrimaryIndex byte
// Hash of this block, created when binary encoded (double SHA256).
hash util.Uint256
// Hash of the block used to verify it (single SHA256).
verificationHash util.Uint256
}
// baseAux is used to marshal/unmarshal to/from JSON, it's almost the same
// as original Base, but with Nonce and NextConsensus fields differing and
// Hash added.
type baseAux struct {
Hash util.Uint256 `json:"hash"`
Version uint32 `json:"version"`
PrevHash util.Uint256 `json:"previousblockhash"`
MerkleRoot util.Uint256 `json:"merkleroot"`
Timestamp uint64 `json:"time"`
Index uint32 `json:"index"`
NextConsensus string `json:"nextconsensus"`
PrimaryIndex byte `json:"primary"`
PrevStateRoot *util.Uint256 `json:"previousstateroot,omitempty"`
Witnesses []transaction.Witness `json:"witnesses"`
}
// Hash returns the hash of the block.
func (b *Header) Hash() util.Uint256 {
if b.hash.Equals(util.Uint256{}) {
b.createHash()
}
return b.hash
}
// GetSignedHash returns a hash of the block used to verify it.
func (b *Header) GetSignedHash() util.Uint256 {
if b.verificationHash.Equals(util.Uint256{}) {
b.createHash()
}
return b.verificationHash
}
// DecodeBinary implements Serializable interface.
func (h *Header) DecodeBinary(r *io.BinReader) {
h.Base.DecodeBinary(r)
func (b *Header) DecodeBinary(br *io.BinReader) {
b.decodeHashableFields(br)
witnessCount := br.ReadVarUint()
if br.Err == nil && witnessCount != 1 {
br.Err = errors.New("wrong witness count")
return
}
padding := []byte{0}
r.ReadBytes(padding)
b.Script.DecodeBinary(br)
}
if padding[0] != 0 {
r.Err = fmt.Errorf("format error: padding must equal 0 got %d", padding)
// EncodeBinary implements Serializable interface
func (b *Header) EncodeBinary(bw *io.BinWriter) {
b.encodeHashableFields(bw)
bw.WriteVarUint(1)
b.Script.EncodeBinary(bw)
}
// GetSignedPart returns serialized hashable data of the block.
func (b *Header) GetSignedPart() []byte {
buf := io.NewBufBinWriter()
buf.WriteU32LE(uint32(b.Network))
// No error can occure while encoding hashable fields.
b.encodeHashableFields(buf.BinWriter)
return buf.Bytes()
}
// createHash creates the hash of the block.
// When calculating the hash value of the block, instead of calculating the entire block,
// only first seven fields in the block head will be calculated, which are
// version, PrevBlock, MerkleRoot, timestamp, and height, the nonce, NextMiner.
// Since MerkleRoot already contains the hash value of all transactions,
// the modification of transaction will influence the hash value of the block.
func (b *Header) createHash() {
bb := b.GetSignedPart()
b.verificationHash = hash.Sha256(bb)
b.hash = hash.Sha256(b.verificationHash.BytesBE())
}
// encodeHashableFields will only encode the fields used for hashing.
// see Hash() for more information about the fields.
func (b *Header) encodeHashableFields(bw *io.BinWriter) {
bw.WriteU32LE(b.Version)
bw.WriteBytes(b.PrevHash[:])
bw.WriteBytes(b.MerkleRoot[:])
bw.WriteU64LE(b.Timestamp)
bw.WriteU32LE(b.Index)
bw.WriteB(b.PrimaryIndex)
bw.WriteBytes(b.NextConsensus[:])
if b.StateRootEnabled {
bw.WriteBytes(b.PrevStateRoot[:])
}
}
// EncodeBinary implements Serializable interface.
func (h *Header) EncodeBinary(w *io.BinWriter) {
h.Base.EncodeBinary(w)
w.WriteBytes([]byte{0})
// decodeHashableFields decodes the fields used for hashing.
// see Hash() for more information about the fields.
func (b *Header) decodeHashableFields(br *io.BinReader) {
b.Version = br.ReadU32LE()
br.ReadBytes(b.PrevHash[:])
br.ReadBytes(b.MerkleRoot[:])
b.Timestamp = br.ReadU64LE()
b.Index = br.ReadU32LE()
b.PrimaryIndex = br.ReadB()
br.ReadBytes(b.NextConsensus[:])
if b.StateRootEnabled {
br.ReadBytes(b.PrevStateRoot[:])
}
// Make the hash of the block here so we dont need to do this
// again.
if br.Err == nil {
b.createHash()
}
}
// MarshalJSON implements json.Marshaler interface.
func (b Header) MarshalJSON() ([]byte, error) {
aux := baseAux{
Hash: b.Hash(),
Version: b.Version,
PrevHash: b.PrevHash,
MerkleRoot: b.MerkleRoot,
Timestamp: b.Timestamp,
Index: b.Index,
PrimaryIndex: b.PrimaryIndex,
NextConsensus: address.Uint160ToString(b.NextConsensus),
Witnesses: []transaction.Witness{b.Script},
}
if b.StateRootEnabled {
aux.PrevStateRoot = &b.PrevStateRoot
}
return json.Marshal(aux)
}
// UnmarshalJSON implements json.Unmarshaler interface.
func (b *Header) UnmarshalJSON(data []byte) error {
var aux = new(baseAux)
var nextC util.Uint160
err := json.Unmarshal(data, aux)
if err != nil {
return err
}
nextC, err = address.StringToUint160(aux.NextConsensus)
if err != nil {
return err
}
if len(aux.Witnesses) != 1 {
return errors.New("wrong number of witnesses")
}
b.Version = aux.Version
b.PrevHash = aux.PrevHash
b.MerkleRoot = aux.MerkleRoot
b.Timestamp = aux.Timestamp
b.Index = aux.Index
b.PrimaryIndex = aux.PrimaryIndex
b.NextConsensus = nextC
b.Script = aux.Witnesses[0]
if b.StateRootEnabled {
if aux.PrevStateRoot == nil {
return errors.New("'previousstateroot' is empty")
}
b.PrevStateRoot = *aux.PrevStateRoot
}
if !aux.Hash.Equals(b.Hash()) {
return errors.New("json 'hash' doesn't match block hash")
}
return nil
}

View file

@ -13,7 +13,7 @@ import (
)
func testHeaderEncodeDecode(t *testing.T, stateRootEnabled bool) {
header := Header{Base: Base{
header := Header{
Version: 0,
PrevHash: hash.Sha256([]byte("prevhash")),
MerkleRoot: hash.Sha256([]byte("merkleroot")),
@ -24,14 +24,14 @@ func testHeaderEncodeDecode(t *testing.T, stateRootEnabled bool) {
InvocationScript: []byte{0x10},
VerificationScript: []byte{0x11},
},
}}
}
if stateRootEnabled {
header.StateRootEnabled = stateRootEnabled
header.PrevStateRoot = random.Uint256()
}
_ = header.Hash()
headerDecode := &Header{Base: Base{StateRootEnabled: stateRootEnabled}}
headerDecode := &Header{StateRootEnabled: stateRootEnabled}
testserdes.EncodeDecodeBinary(t, &header, headerDecode)
assert.Equal(t, header.Version, headerDecode.Version, "expected both versions to be equal")

View file

@ -491,7 +491,7 @@ func (bc *Blockchain) AddBlock(block *block.Block) error {
}
if block.Index == bc.HeaderHeight()+1 {
err := bc.addHeaders(bc.config.VerifyBlocks, block.Header())
err := bc.addHeaders(bc.config.VerifyBlocks, &block.Header)
if err != nil {
return err
}
@ -576,6 +576,7 @@ func (bc *Blockchain) addHeaders(verify bool, headers ...*block.Header) error {
}
bc.headerHashes = append(bc.headerHashes, h.Hash())
h.EncodeBinary(buf.BinWriter)
buf.BinWriter.WriteB(0)
if buf.Err != nil {
return buf.Err
}
@ -1137,14 +1138,14 @@ func (bc *Blockchain) GetHeader(hash util.Uint256) (*block.Header, error) {
if topBlock != nil {
tb := topBlock.(*block.Block)
if tb.Hash().Equals(hash) {
return tb.Header(), nil
return &tb.Header, nil
}
}
block, err := bc.dao.GetBlock(hash)
if err != nil {
return nil, err
}
return block.Header(), nil
return &block.Header, nil
}
// HasTransaction returns true if the blockchain contains he given

View file

@ -42,63 +42,63 @@ import (
func TestVerifyHeader(t *testing.T) {
bc := newTestChain(t)
prev := bc.topBlock.Load().(*block.Block).Header()
prev := bc.topBlock.Load().(*block.Block).Header
t.Run("Invalid", func(t *testing.T) {
t.Run("Hash", func(t *testing.T) {
h := prev.Hash()
h[0] = ^h[0]
hdr := newBlock(bc.config, 1, h).Header()
require.True(t, errors.Is(bc.verifyHeader(hdr, prev), ErrHdrHashMismatch))
hdr := newBlock(bc.config, 1, h).Header
require.True(t, errors.Is(bc.verifyHeader(&hdr, &prev), ErrHdrHashMismatch))
})
t.Run("Index", func(t *testing.T) {
hdr := newBlock(bc.config, 3, prev.Hash()).Header()
require.True(t, errors.Is(bc.verifyHeader(hdr, prev), ErrHdrIndexMismatch))
hdr := newBlock(bc.config, 3, prev.Hash()).Header
require.True(t, errors.Is(bc.verifyHeader(&hdr, &prev), ErrHdrIndexMismatch))
})
t.Run("Timestamp", func(t *testing.T) {
hdr := newBlock(bc.config, 1, prev.Hash()).Header()
hdr := newBlock(bc.config, 1, prev.Hash()).Header
hdr.Timestamp = 0
require.True(t, errors.Is(bc.verifyHeader(hdr, prev), ErrHdrInvalidTimestamp))
require.True(t, errors.Is(bc.verifyHeader(&hdr, &prev), ErrHdrInvalidTimestamp))
})
})
t.Run("Valid", func(t *testing.T) {
hdr := newBlock(bc.config, 1, prev.Hash()).Header()
require.NoError(t, bc.verifyHeader(hdr, prev))
hdr := newBlock(bc.config, 1, prev.Hash()).Header
require.NoError(t, bc.verifyHeader(&hdr, &prev))
})
}
func TestAddHeaders(t *testing.T) {
bc := newTestChain(t)
lastBlock := bc.topBlock.Load().(*block.Block)
h1 := newBlock(bc.config, 1, lastBlock.Hash()).Header()
h2 := newBlock(bc.config, 2, h1.Hash()).Header()
h3 := newBlock(bc.config, 3, h2.Hash()).Header()
h1 := newBlock(bc.config, 1, lastBlock.Hash()).Header
h2 := newBlock(bc.config, 2, h1.Hash()).Header
h3 := newBlock(bc.config, 3, h2.Hash()).Header
require.NoError(t, bc.AddHeaders())
require.NoError(t, bc.AddHeaders(h1, h2))
require.NoError(t, bc.AddHeaders(h2, h3))
require.NoError(t, bc.AddHeaders(&h1, &h2))
require.NoError(t, bc.AddHeaders(&h2, &h3))
assert.Equal(t, h3.Index, bc.HeaderHeight())
assert.Equal(t, uint32(0), bc.BlockHeight())
assert.Equal(t, h3.Hash(), bc.CurrentHeaderHash())
// Add them again, they should not be added.
require.NoError(t, bc.AddHeaders(h3, h2, h1))
require.NoError(t, bc.AddHeaders(&h3, &h2, &h1))
assert.Equal(t, h3.Index, bc.HeaderHeight())
assert.Equal(t, uint32(0), bc.BlockHeight())
assert.Equal(t, h3.Hash(), bc.CurrentHeaderHash())
h4 := newBlock(bc.config, 4, h3.Hash().Reverse()).Header()
h5 := newBlock(bc.config, 5, h4.Hash()).Header()
h4 := newBlock(bc.config, 4, h3.Hash().Reverse()).Header
h5 := newBlock(bc.config, 5, h4.Hash()).Header
assert.Error(t, bc.AddHeaders(h4, h5))
assert.Error(t, bc.AddHeaders(&h4, &h5))
assert.Equal(t, h3.Index, bc.HeaderHeight())
assert.Equal(t, uint32(0), bc.BlockHeight())
assert.Equal(t, h3.Hash(), bc.CurrentHeaderHash())
h6 := newBlock(bc.config, 4, h3.Hash()).Header()
h6 := newBlock(bc.config, 4, h3.Hash()).Header
h6.Script.InvocationScript = nil
assert.Error(t, bc.AddHeaders(h6))
assert.Error(t, bc.AddHeaders(&h6))
assert.Equal(t, h3.Index, bc.HeaderHeight())
assert.Equal(t, uint32(0), bc.BlockHeight())
assert.Equal(t, h3.Hash(), bc.CurrentHeaderHash())
@ -206,7 +206,7 @@ func TestGetHeader(t *testing.T) {
hash := block.Hash()
header, err := bc.GetHeader(hash)
require.NoError(t, err)
assert.Equal(t, block.Header(), header)
assert.Equal(t, &block.Header, header)
b2 := bc.newBlock()
_, err = bc.GetHeader(b2.Hash())

View file

@ -538,7 +538,8 @@ func (dao *Simple) DeleteBlock(h util.Uint256, w *io.BufBinWriter) error {
if w == nil {
w = io.NewBufBinWriter()
}
b.Header().EncodeBinary(w.BinWriter)
b.Header.EncodeBinary(w.BinWriter)
w.BinWriter.WriteB(0)
if w.Err != nil {
return w.Err
}

View file

@ -95,7 +95,7 @@ func TestGetBlock_NotExists(t *testing.T) {
func TestPutGetBlock(t *testing.T) {
dao := NewSimple(storage.NewMemoryStore(), netmode.UnitTestNet, false)
b := &block.Block{
Base: block.Base{
Header: block.Header{
Script: transaction.Witness{
VerificationScript: []byte{byte(opcode.PUSH1)},
InvocationScript: []byte{byte(opcode.NOP)},
@ -136,7 +136,7 @@ func TestGetCurrentHeaderHeight_NoHeader(t *testing.T) {
func TestGetCurrentHeaderHeight_Store(t *testing.T) {
dao := NewSimple(storage.NewMemoryStore(), netmode.UnitTestNet, false)
b := &block.Block{
Base: block.Base{
Header: block.Header{
Script: transaction.Witness{
VerificationScript: []byte{byte(opcode.PUSH1)},
InvocationScript: []byte{byte(opcode.NOP)},

View file

@ -108,7 +108,7 @@ func newBlockCustom(cfg config.ProtocolConfiguration, f func(b *block.Block),
VerificationScript: valScript,
}
b := &block.Block{
Base: block.Base{
Header: block.Header{
Network: testchain.Network(),
NextConsensus: witness.ScriptHash(),
Script: witness,
@ -205,7 +205,7 @@ func getBlockData(i int) (map[string]interface{}, error) {
func newDumbBlock() *block.Block {
return &block.Block{
Base: block.Base{
Header: block.Header{
Network: testchain.Network(),
Version: 0,
PrevHash: hash.Sha256([]byte("a")),
@ -325,7 +325,7 @@ func initBasicChain(t *testing.T, bc *Blockchain) {
// info for getblockheader rpc tests
t.Logf("header hash: %s", b.Hash().StringLE())
buf := io.NewBufBinWriter()
b.Header().EncodeBinary(buf.BinWriter)
b.Header.EncodeBinary(buf.BinWriter)
t.Logf("header: %s", hex.EncodeToString(buf.Bytes()))
acc0 := wallet.NewAccountFromPrivateKey(priv0)

View file

@ -92,7 +92,6 @@ func Log(ic *interop.Context) error {
// GetTime returns timestamp of the block being verified, or the latest
// one in the blockchain if no block is given to Context.
func GetTime(ic *interop.Context) error {
header := ic.Block.Header()
ic.VM.Estack().PushVal(header.Timestamp)
ic.VM.Estack().PushVal(ic.Block.Timestamp)
return nil
}

View file

@ -37,7 +37,7 @@ func createGenesisBlock(cfg config.ProtocolConfiguration) (*block.Block, error)
return nil, err
}
base := block.Base{
base := block.Header{
Version: 0,
PrevHash: util.Uint256{},
Timestamp: uint64(time.Date(2016, 7, 15, 15, 8, 21, 0, time.UTC).Unix()) * 1000, // Milliseconds.
@ -52,7 +52,7 @@ func createGenesisBlock(cfg config.ProtocolConfiguration) (*block.Block, error)
}
b := &block.Block{
Base: base,
Header: base,
Transactions: []*transaction.Transaction{},
}
b.RebuildMerkleRoot()

View file

@ -16,7 +16,7 @@ func TestBlockQueue(t *testing.T) {
bq := newBlockQueue(0, chain, zaptest.NewLogger(t), nil)
blocks := make([]*block.Block, 11)
for i := 1; i < 11; i++ {
blocks[i] = &block.Block{Base: block.Base{Index: uint32(i)}}
blocks[i] = &block.Block{Header: block.Header{Index: uint32(i)}}
}
// not the ones expected currently
for i := 3; i < 5; i++ {
@ -33,7 +33,7 @@ func TestBlockQueue(t *testing.T) {
assert.Equal(t, uint32(0), chain.BlockHeight())
assert.Equal(t, 4, bq.length())
// block with too big index is dropped
assert.NoError(t, bq.putBlock(&block.Block{Base: block.Base{Index: bq.chain.BlockHeight() + blockCacheSize + 1}}))
assert.NoError(t, bq.putBlock(&block.Block{Header: block.Header{Index: bq.chain.BlockHeight() + blockCacheSize + 1}}))
assert.Equal(t, 4, bq.length())
go bq.run()
// run() is asynchronous, so we need some kind of timeout anyway and this is the simplest one

View file

@ -57,12 +57,10 @@ func TestEncodeDecodeHeaders(t *testing.T) {
headers := &payload.Headers{Hdrs: make([]*block.Header, CompressionMinSize)}
for i := range headers.Hdrs {
h := &block.Header{
Base: block.Base{
Index: uint32(i + 1),
Script: transaction.Witness{
InvocationScript: []byte{0x0},
VerificationScript: []byte{0x1},
},
Index: uint32(i + 1),
Script: transaction.Witness{
InvocationScript: []byte{0x0},
VerificationScript: []byte{0x1},
},
}
h.Hash()
@ -199,7 +197,7 @@ func TestEncodeDecodeTransaction(t *testing.T) {
}
func TestEncodeDecodeMerkleBlock(t *testing.T) {
base := &block.Base{
base := &block.Header{
PrevHash: random.Uint256(),
Timestamp: rand.Uint64(),
Script: transaction.Witness{
@ -212,7 +210,7 @@ func TestEncodeDecodeMerkleBlock(t *testing.T) {
t.Run("good", func(t *testing.T) {
testEncodeDecode(t, CMDMerkleBlock, &payload.MerkleBlock{
Network: netmode.UnitTestNet,
Base: base,
Header: base,
TxCount: 1,
Hashes: []util.Uint256{random.Uint256()},
Flags: []byte{0},
@ -220,7 +218,7 @@ func TestEncodeDecodeMerkleBlock(t *testing.T) {
})
t.Run("bad, invalid TxCount", func(t *testing.T) {
testEncodeDecodeFail(t, CMDMerkleBlock, &payload.MerkleBlock{
Base: base,
Header: base,
TxCount: 2,
Hashes: []util.Uint256{random.Uint256()},
Flags: []byte{0},

View file

@ -35,12 +35,10 @@ func newTestHeaders(n int) *Headers {
for i := range headers.Hdrs {
headers.Hdrs[i] = &block.Header{
Base: block.Base{
Index: uint32(i + 1),
Script: transaction.Witness{
InvocationScript: []byte{0x0},
VerificationScript: []byte{0x1},
},
Index: uint32(i + 1),
Script: transaction.Witness{
InvocationScript: []byte{0x0},
VerificationScript: []byte{0x1},
},
}
}

View file

@ -11,7 +11,7 @@ import (
// MerkleBlock represents a merkle block packet payload.
type MerkleBlock struct {
*block.Base
*block.Header
Network netmode.Magic
TxCount int
Hashes []util.Uint256
@ -20,8 +20,8 @@ type MerkleBlock struct {
// DecodeBinary implements Serializable interface.
func (m *MerkleBlock) DecodeBinary(br *io.BinReader) {
m.Base = &block.Base{Network: m.Network}
m.Base.DecodeBinary(br)
m.Header = &block.Header{Network: m.Network}
m.Header.DecodeBinary(br)
txCount := int(br.ReadVarUint())
if txCount > block.MaxTransactionsPerBlock {
@ -38,7 +38,7 @@ func (m *MerkleBlock) DecodeBinary(br *io.BinReader) {
// EncodeBinary implements Serializable interface.
func (m *MerkleBlock) EncodeBinary(bw *io.BinWriter) {
m.Base.EncodeBinary(bw)
m.Header.EncodeBinary(bw)
bw.WriteVarUint(uint64(m.TxCount))
bw.WriteArray(m.Hashes)

View file

@ -13,8 +13,8 @@ import (
"github.com/nspcc-dev/neo-go/pkg/util"
)
func newDumbBlock() *block.Base {
return &block.Base{
func newDumbBlock() *block.Header {
return &block.Header{
Version: 0,
PrevHash: hash.Sha256([]byte("a")),
MerkleRoot: hash.Sha256([]byte("b")),
@ -33,7 +33,7 @@ func TestMerkleBlock_EncodeDecodeBinary(t *testing.T) {
b := newDumbBlock()
_ = b.Hash()
expected := &MerkleBlock{
Base: b,
Header: b,
TxCount: 0,
Hashes: []util.Uint256{},
Flags: []byte{},
@ -45,7 +45,7 @@ func TestMerkleBlock_EncodeDecodeBinary(t *testing.T) {
b := newDumbBlock()
_ = b.Hash()
expected := &MerkleBlock{
Base: b,
Header: b,
TxCount: block.MaxTransactionsPerBlock + 1,
Hashes: make([]util.Uint256, block.MaxTransactionsPerBlock),
Flags: []byte{},
@ -59,7 +59,7 @@ func TestMerkleBlock_EncodeDecodeBinary(t *testing.T) {
b := newDumbBlock()
_ = b.Hash()
expected := &MerkleBlock{
Base: b,
Header: b,
TxCount: 0,
Hashes: []util.Uint256{},
Flags: []byte{1, 2, 3, 4, 5},

View file

@ -640,7 +640,7 @@ func TestGetHeaders(t *testing.T) {
expected := make([]*block.Header, len(blocks))
for i := range blocks {
expected[i] = blocks[i].Header()
expected[i] = &blocks[i].Header
}
var actual *payload.Headers

View file

@ -246,7 +246,7 @@ var rpcClientTestCases = map[string][]rpcClientTestCase{
serverResponse: `{"id":1,"jsonrpc":"2.0","result":"` + base64Header1 + `"}`,
result: func(c *Client) interface{} {
b := getResultBlock1()
return b.Header()
return &b.Header
},
},
{
@ -884,7 +884,7 @@ var rpcClientTestCases = map[string][]rpcClientTestCase{
name: "positive",
invoke: func(c *Client) (interface{}, error) {
return c.SubmitBlock(block.Block{
Base: block.Base{},
Header: block.Header{},
Transactions: nil,
Trimmed: false,
})
@ -1011,7 +1011,7 @@ var rpcClientErrorCases = map[string][]rpcClientErrorCase{
name: "submitblock_bad_server_answer",
invoke: func(c *Client) (interface{}, error) {
return c.SubmitBlock(block.Block{
Base: block.Base{},
Header: block.Header{},
Transactions: nil,
Trimmed: false,
})
@ -1372,7 +1372,7 @@ var rpcClientErrorCases = map[string][]rpcClientErrorCase{
name: "submitblock_unmarshalling_error",
invoke: func(c *Client) (interface{}, error) {
return c.SubmitBlock(block.Block{
Base: block.Base{},
Header: block.Header{},
Transactions: nil,
Trimmed: false,
})

View file

@ -35,7 +35,7 @@ func NewTransactionOutputRaw(tx *transaction.Transaction, header *block.Header,
return result
}
// confirmations formula
confirmations := int(chain.BlockHeight() - header.Base.Index + 1)
confirmations := int(chain.BlockHeight() - header.Index + 1)
result.TransactionMetadata = TransactionMetadata{
Blockhash: header.Hash(),
Confirmations: confirmations,

View file

@ -1506,7 +1506,7 @@ func (e *executor) getHeader(s string) *block.Header {
if err != nil {
panic("unknown block (update block hash)")
}
return block.Header()
return &block.Header
}
func encodeBlock(t *testing.T, b *block.Block) string {

View file

@ -160,7 +160,7 @@ func addBlock(bc *core.Blockchain, lastBlock *block.Block, script []byte, txs ..
func newBlock(bc *core.Blockchain, lastBlock *block.Block, script []byte, txs ...*transaction.Transaction) (*block.Block, error) {
witness := transaction.Witness{VerificationScript: script}
b := &block.Block{
Base: block.Base{
Header: block.Header{
Network: netmode.UnitTestNet,
PrevHash: lastBlock.Hash(),
Timestamp: uint64(time.Now().UTC().Unix())*1000 + uint64(lastBlock.Index),