Merge branch 'develop' into jae/literefactor4

This commit is contained in:
Ethan Buchman
2018-07-23 23:28:14 -04:00
342 changed files with 30691 additions and 6539 deletions

View File

@ -17,8 +17,8 @@ import (
// TODO: add Version byte
type Block struct {
mtx sync.Mutex
*Header `json:"header"`
*Data `json:"data"`
Header `json:"header"`
Data `json:"data"`
Evidence EvidenceData `json:"evidence"`
LastCommit *Commit `json:"last_commit"`
}
@ -27,15 +27,15 @@ type Block struct {
// It populates the same set of fields validated by ValidateBasic
func MakeBlock(height int64, txs []Tx, commit *Commit) *Block {
block := &Block{
Header: &Header{
Header: Header{
Height: height,
Time: time.Now(),
NumTxs: int64(len(txs)),
},
LastCommit: commit,
Data: &Data{
Data: Data{
Txs: txs,
},
LastCommit: commit,
}
block.fillHeader()
return block
@ -43,6 +43,9 @@ func MakeBlock(height int64, txs []Tx, commit *Commit) *Block {
// AddEvidence appends the given evidence to the block
func (b *Block) AddEvidence(evidence []Evidence) {
if b == nil {
return
}
b.Evidence.Evidence = append(b.Evidence.Evidence, evidence...)
}
@ -98,7 +101,7 @@ func (b *Block) Hash() cmn.HexBytes {
b.mtx.Lock()
defer b.mtx.Unlock()
if b == nil || b.Header == nil || b.Data == nil || b.LastCommit == nil {
if b == nil || b.LastCommit == nil {
return nil
}
b.fillHeader()
@ -107,6 +110,7 @@ func (b *Block) Hash() cmn.HexBytes {
// MakePartSet returns a PartSet containing parts of a serialized block.
// This is the form in which the block is gossipped to peers.
// CONTRACT: partSize is greater than zero.
func (b *Block) MakePartSet(partSize int) *PartSet {
if b == nil {
return nil
@ -209,7 +213,7 @@ type Header struct {
// Hash returns the hash of the header.
// Returns nil if ValidatorHash is missing,
// since a Header is not valid unless there is
// a ValidaotrsHash (corresponding to the validator set).
// a ValidatorsHash (corresponding to the validator set).
func (h *Header) Hash() cmn.HexBytes {
if h == nil || len(h.ValidatorsHash) == 0 {
return nil
@ -397,6 +401,9 @@ func (commit *Commit) ValidateBasic() error {
// Hash returns the hash of the commit
func (commit *Commit) Hash() cmn.HexBytes {
if commit == nil {
return nil
}
if commit.hash == nil {
bs := make([]merkle.Hasher, len(commit.Precommits))
for i, precommit := range commit.Precommits {

View File

@ -3,7 +3,7 @@ package types
// BlockMeta contains meta information about a block - namely, it's ID and Header.
type BlockMeta struct {
BlockID BlockID `json:"block_id"` // the block hash and partsethash
Header *Header `json:"header"` // The block's Header
Header Header `json:"header"` // The block's Header
}
// NewBlockMeta returns a new BlockMeta from the block and its blockParts.

View File

@ -10,7 +10,25 @@ import (
cmn "github.com/tendermint/tendermint/libs/common"
)
func TestValidateBlock(t *testing.T) {
func TestBlockAddEvidence(t *testing.T) {
txs := []Tx{Tx("foo"), Tx("bar")}
lastID := makeBlockIDRandom()
h := int64(3)
voteSet, valSet, vals := randVoteSet(h-1, 1, VoteTypePrecommit, 10, 1)
commit, err := MakeCommit(lastID, h-1, 1, voteSet, vals)
require.NoError(t, err)
block := MakeBlock(h, txs, commit)
require.NotNil(t, block)
ev := NewMockGoodEvidence(h, 0, valSet.Validators[0].Address)
block.AddEvidence([]Evidence{ev})
}
func TestBlockValidateBasic(t *testing.T) {
require.Error(t, (*Block)(nil).ValidateBasic())
txs := []Tx{Tx("foo"), Tx("bar")}
lastID := makeBlockIDRandom()
h := int64(3)
@ -57,6 +75,59 @@ func TestValidateBlock(t *testing.T) {
block.DataHash = cmn.RandBytes(len(block.DataHash))
err = block.ValidateBasic()
require.Error(t, err)
// tamper with evidence
block = MakeBlock(h, txs, commit)
block.EvidenceHash = []byte("something else")
err = block.ValidateBasic()
require.Error(t, err)
}
func TestBlockHash(t *testing.T) {
assert.Nil(t, (*Block)(nil).Hash())
assert.Nil(t, MakeBlock(int64(3), []Tx{Tx("Hello World")}, nil).Hash())
}
func TestBlockMakePartSet(t *testing.T) {
assert.Nil(t, (*Block)(nil).MakePartSet(2))
partSet := MakeBlock(int64(3), []Tx{Tx("Hello World")}, nil).MakePartSet(1024)
assert.NotNil(t, partSet)
assert.Equal(t, 1, partSet.Total())
}
func TestBlockHashesTo(t *testing.T) {
assert.False(t, (*Block)(nil).HashesTo(nil))
lastID := makeBlockIDRandom()
h := int64(3)
voteSet, valSet, vals := randVoteSet(h-1, 1, VoteTypePrecommit, 10, 1)
commit, err := MakeCommit(lastID, h-1, 1, voteSet, vals)
require.NoError(t, err)
block := MakeBlock(h, []Tx{Tx("Hello World")}, commit)
block.ValidatorsHash = valSet.Hash()
assert.False(t, block.HashesTo([]byte{}))
assert.False(t, block.HashesTo([]byte("something else")))
assert.True(t, block.HashesTo(block.Hash()))
}
func TestBlockSize(t *testing.T) {
size := MakeBlock(int64(3), []Tx{Tx("Hello World")}, nil).Size()
if size <= 0 {
t.Fatal("Size of the block is zero or negative")
}
}
func TestBlockString(t *testing.T) {
assert.Equal(t, "nil-Block", (*Block)(nil).String())
assert.Equal(t, "nil-Block", (*Block)(nil).StringIndented(""))
assert.Equal(t, "nil-Block", (*Block)(nil).StringShort())
block := MakeBlock(int64(3), []Tx{Tx("Hello World")}, nil)
assert.NotEqual(t, "nil-Block", block.String())
assert.NotEqual(t, "nil-Block", block.StringIndented(""))
assert.NotEqual(t, "nil-Block", block.StringShort())
}
func makeBlockIDRandom() BlockID {
@ -86,3 +157,61 @@ func TestNilDataHashDoesntCrash(t *testing.T) {
assert.Equal(t, []byte((*Data)(nil).Hash()), nilBytes)
assert.Equal(t, []byte(new(Data).Hash()), nilBytes)
}
func TestCommit(t *testing.T) {
lastID := makeBlockIDRandom()
h := int64(3)
voteSet, _, vals := randVoteSet(h-1, 1, VoteTypePrecommit, 10, 1)
commit, err := MakeCommit(lastID, h-1, 1, voteSet, vals)
require.NoError(t, err)
assert.NotNil(t, commit.FirstPrecommit())
assert.Equal(t, h-1, commit.Height())
assert.Equal(t, 1, commit.Round())
assert.Equal(t, VoteTypePrecommit, commit.Type())
if commit.Size() <= 0 {
t.Fatalf("commit %v has a zero or negative size: %d", commit, commit.Size())
}
require.NotNil(t, commit.BitArray())
assert.Equal(t, cmn.NewBitArray(10).Size(), commit.BitArray().Size())
assert.Equal(t, voteSet.GetByIndex(0), commit.GetByIndex(0))
assert.True(t, commit.IsCommit())
}
func TestCommitValidateBasic(t *testing.T) {
commit := randCommit()
assert.NoError(t, commit.ValidateBasic())
// nil precommit is OK
commit = randCommit()
commit.Precommits[0] = nil
assert.NoError(t, commit.ValidateBasic())
// tamper with types
commit = randCommit()
commit.Precommits[0].Type = VoteTypePrevote
assert.Error(t, commit.ValidateBasic())
// tamper with height
commit = randCommit()
commit.Precommits[0].Height = int64(100)
assert.Error(t, commit.ValidateBasic())
// tamper with round
commit = randCommit()
commit.Precommits[0].Round = 100
assert.Error(t, commit.ValidateBasic())
}
func randCommit() *Commit {
lastID := makeBlockIDRandom()
h := int64(3)
voteSet, _, vals := randVoteSet(h-1, 1, VoteTypePrecommit, 10, 1)
commit, err := MakeCommit(lastID, h-1, 1, voteSet, vals)
if err != nil {
panic(err)
}
return commit
}

View File

@ -1,50 +0,0 @@
package types
// Interface assertions
var _ TxEventPublisher = (*TxEventBuffer)(nil)
// TxEventBuffer is a buffer of events, which uses a slice to temporarily store
// events.
type TxEventBuffer struct {
next TxEventPublisher
capacity int
events []EventDataTx
}
// NewTxEventBuffer accepts a TxEventPublisher and returns a new buffer with the given
// capacity.
func NewTxEventBuffer(next TxEventPublisher, capacity int) *TxEventBuffer {
return &TxEventBuffer{
next: next,
capacity: capacity,
events: make([]EventDataTx, 0, capacity),
}
}
// Len returns the number of events cached.
func (b TxEventBuffer) Len() int {
return len(b.events)
}
// PublishEventTx buffers an event to be fired upon finality.
func (b *TxEventBuffer) PublishEventTx(e EventDataTx) error {
b.events = append(b.events, e)
return nil
}
// Flush publishes events by running next.PublishWithTags on all cached events.
// Blocks. Clears cached events.
func (b *TxEventBuffer) Flush() error {
for _, e := range b.events {
err := b.next.PublishEventTx(e)
if err != nil {
return err
}
}
// Clear out the elements and set the length to 0
// but maintain the underlying slice's capacity.
// See Issue https://github.com/tendermint/tendermint/issues/1189
b.events = b.events[:0]
return nil
}

View File

@ -1,21 +0,0 @@
package types
import (
"testing"
"github.com/stretchr/testify/assert"
)
type eventBusMock struct{}
func (eventBusMock) PublishEventTx(e EventDataTx) error {
return nil
}
func TestEventBuffer(t *testing.T) {
b := NewTxEventBuffer(eventBusMock{}, 1)
b.PublishEventTx(EventDataTx{})
assert.Equal(t, 1, b.Len())
b.Flush()
assert.Equal(t, 0, b.Len())
}

View File

@ -4,9 +4,9 @@ import (
"context"
"fmt"
tmpubsub "github.com/tendermint/tendermint/libs/pubsub"
cmn "github.com/tendermint/tendermint/libs/common"
"github.com/tendermint/tendermint/libs/log"
tmpubsub "github.com/tendermint/tendermint/libs/pubsub"
)
const defaultCapacity = 0
@ -49,7 +49,7 @@ func (b *EventBus) OnStart() error {
}
func (b *EventBus) OnStop() {
b.pubsub.OnStop()
b.pubsub.Stop()
}
func (b *EventBus) Subscribe(ctx context.Context, subscriber string, query tmpubsub.Query, out chan<- interface{}) error {

View File

@ -3,7 +3,6 @@ package types
import (
"context"
"fmt"
"math/rand"
"testing"
"time"
@ -11,9 +10,9 @@ import (
"github.com/stretchr/testify/require"
abci "github.com/tendermint/tendermint/abci/types"
cmn "github.com/tendermint/tendermint/libs/common"
tmpubsub "github.com/tendermint/tendermint/libs/pubsub"
tmquery "github.com/tendermint/tendermint/libs/pubsub/query"
cmn "github.com/tendermint/tendermint/libs/common"
)
func TestEventBusPublishEventTx(t *testing.T) {
@ -59,6 +58,64 @@ func TestEventBusPublishEventTx(t *testing.T) {
}
}
func TestEventBusPublish(t *testing.T) {
eventBus := NewEventBus()
err := eventBus.Start()
require.NoError(t, err)
defer eventBus.Stop()
eventsCh := make(chan interface{})
err = eventBus.Subscribe(context.Background(), "test", tmquery.Empty{}, eventsCh)
require.NoError(t, err)
const numEventsExpected = 14
done := make(chan struct{})
go func() {
numEvents := 0
for range eventsCh {
numEvents++
if numEvents >= numEventsExpected {
close(done)
}
}
}()
err = eventBus.Publish(EventNewBlockHeader, EventDataNewBlockHeader{})
require.NoError(t, err)
err = eventBus.PublishEventNewBlock(EventDataNewBlock{})
require.NoError(t, err)
err = eventBus.PublishEventNewBlockHeader(EventDataNewBlockHeader{})
require.NoError(t, err)
err = eventBus.PublishEventVote(EventDataVote{})
require.NoError(t, err)
err = eventBus.PublishEventProposalHeartbeat(EventDataProposalHeartbeat{})
require.NoError(t, err)
err = eventBus.PublishEventNewRoundStep(EventDataRoundState{})
require.NoError(t, err)
err = eventBus.PublishEventTimeoutPropose(EventDataRoundState{})
require.NoError(t, err)
err = eventBus.PublishEventTimeoutWait(EventDataRoundState{})
require.NoError(t, err)
err = eventBus.PublishEventNewRound(EventDataRoundState{})
require.NoError(t, err)
err = eventBus.PublishEventCompleteProposal(EventDataRoundState{})
require.NoError(t, err)
err = eventBus.PublishEventPolka(EventDataRoundState{})
require.NoError(t, err)
err = eventBus.PublishEventUnlock(EventDataRoundState{})
require.NoError(t, err)
err = eventBus.PublishEventRelock(EventDataRoundState{})
require.NoError(t, err)
err = eventBus.PublishEventLock(EventDataRoundState{})
require.NoError(t, err)
select {
case <-done:
case <-time.After(1 * time.Second):
t.Fatalf("expected to receive %d events after 1 sec.", numEventsExpected)
}
}
func BenchmarkEventBus(b *testing.B) {
benchmarks := []struct {
name string
@ -92,7 +149,7 @@ func BenchmarkEventBus(b *testing.B) {
func benchmarkEventBus(numClients int, randQueries bool, randEvents bool, b *testing.B) {
// for random* functions
rand.Seed(time.Now().Unix())
cmn.Seed(time.Now().Unix())
eventBus := NewEventBusWithBufferCapacity(0) // set buffer capacity to 0 so we are not testing cache
eventBus.Start()
@ -126,11 +183,7 @@ func benchmarkEventBus(numClients int, randQueries bool, randEvents bool, b *tes
}
}
var events = []string{EventBond,
EventUnbond,
EventRebond,
EventDupeout,
EventFork,
var events = []string{
EventNewBlock,
EventNewBlockHeader,
EventNewRound,
@ -145,14 +198,10 @@ var events = []string{EventBond,
EventVote}
func randEvent() string {
return events[rand.Intn(len(events))]
return events[cmn.RandIntn(len(events))]
}
var queries = []tmpubsub.Query{EventQueryBond,
EventQueryUnbond,
EventQueryRebond,
EventQueryDupeout,
EventQueryFork,
var queries = []tmpubsub.Query{
EventQueryNewBlock,
EventQueryNewBlockHeader,
EventQueryNewRound,
@ -167,5 +216,5 @@ var queries = []tmpubsub.Query{EventQueryBond,
EventQueryVote}
func randQuery() tmpubsub.Query {
return queries[rand.Intn(len(queries))]
return queries[cmn.RandIntn(len(queries))]
}

View File

@ -10,22 +10,17 @@ import (
// Reserved event types
const (
EventBond = "Bond"
EventCompleteProposal = "CompleteProposal"
EventDupeout = "Dupeout"
EventFork = "Fork"
EventLock = "Lock"
EventNewBlock = "NewBlock"
EventNewBlockHeader = "NewBlockHeader"
EventNewRound = "NewRound"
EventNewRoundStep = "NewRoundStep"
EventPolka = "Polka"
EventRebond = "Rebond"
EventRelock = "Relock"
EventTimeoutPropose = "TimeoutPropose"
EventTimeoutWait = "TimeoutWait"
EventTx = "Tx"
EventUnbond = "Unbond"
EventUnlock = "Unlock"
EventVote = "Vote"
EventProposalHeartbeat = "ProposalHeartbeat"
@ -69,7 +64,7 @@ type EventDataNewBlock struct {
// light weight event for benchmarking
type EventDataNewBlockHeader struct {
Header *Header `json:"header"`
Header Header `json:"header"`
}
// All txs fire EventDataTx
@ -113,11 +108,6 @@ const (
)
var (
EventQueryBond = QueryForEvent(EventBond)
EventQueryUnbond = QueryForEvent(EventUnbond)
EventQueryRebond = QueryForEvent(EventRebond)
EventQueryDupeout = QueryForEvent(EventDupeout)
EventQueryFork = QueryForEvent(EventFork)
EventQueryNewBlock = QueryForEvent(EventNewBlock)
EventQueryNewBlockHeader = QueryForEvent(EventNewBlockHeader)
EventQueryNewRound = QueryForEvent(EventNewRound)

23
types/events_test.go Normal file
View File

@ -0,0 +1,23 @@
package types
import (
"fmt"
"testing"
"github.com/stretchr/testify/assert"
)
func TestQueryTxFor(t *testing.T) {
tx := Tx("foo")
assert.Equal(t,
fmt.Sprintf("tm.event='Tx' AND tx.hash='%X'", tx.Hash()),
EventQueryTxFor(tx).String(),
)
}
func TestQueryForEvent(t *testing.T) {
assert.Equal(t,
"tm.event='NewBlock'",
QueryForEvent(EventNewBlock).String(),
)
}

View File

@ -4,7 +4,7 @@ import (
"bytes"
"fmt"
"github.com/tendermint/go-amino"
amino "github.com/tendermint/go-amino"
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto/merkle"

View File

@ -36,7 +36,7 @@ func TestEvidence(t *testing.T) {
blockID3 := makeBlockID("blockhash", 10000, "partshash")
blockID4 := makeBlockID("blockhash", 10000, "partshash2")
chainID := "mychain"
const chainID = "mychain"
vote1 := makeVote(val, chainID, 0, 10, 2, 1, blockID)
badVote := makeVote(val, chainID, 0, 10, 2, 1, blockID)
@ -72,3 +72,30 @@ func TestEvidence(t *testing.T) {
}
}
}
func TestDuplicatedVoteEvidence(t *testing.T) {
ev := randomDuplicatedVoteEvidence()
assert.True(t, ev.Equal(ev))
assert.False(t, ev.Equal(&DuplicateVoteEvidence{}))
}
func TestEvidenceList(t *testing.T) {
ev := randomDuplicatedVoteEvidence()
evl := EvidenceList([]Evidence{ev})
assert.NotNil(t, evl.Hash())
assert.True(t, evl.Has(ev))
assert.False(t, evl.Has(&DuplicateVoteEvidence{}))
}
func randomDuplicatedVoteEvidence() *DuplicateVoteEvidence {
val := NewMockPV()
blockID := makeBlockID("blockhash", 1000, "partshash")
blockID2 := makeBlockID("blockhash2", 1000, "partshash")
const chainID = "mychain"
return &DuplicateVoteEvidence{
VoteA: makeVote(val, chainID, 0, 10, 2, 1, blockID),
VoteB: makeVote(val, chainID, 0, 10, 2, 1, blockID2),
}
}

View File

@ -26,17 +26,7 @@ type GenesisDoc struct {
ConsensusParams *ConsensusParams `json:"consensus_params,omitempty"`
Validators []GenesisValidator `json:"validators"`
AppHash cmn.HexBytes `json:"app_hash"`
AppStateJSON json.RawMessage `json:"app_state,omitempty"`
AppOptions json.RawMessage `json:"app_options,omitempty"` // DEPRECATED
}
// AppState returns raw application state.
// TODO: replace with AppState field during next breaking release (0.18)
func (genDoc *GenesisDoc) AppState() json.RawMessage {
if len(genDoc.AppOptions) > 0 {
return genDoc.AppOptions
}
return genDoc.AppStateJSON
AppState json.RawMessage `json:"app_state,omitempty"`
}
// SaveAs is a utility method for saving GenensisDoc as a JSON file.

View File

@ -1,10 +1,14 @@
package types
import (
"io/ioutil"
"os"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/tendermint/tendermint/crypto"
"github.com/stretchr/testify/require"
"github.com/tendermint/tendermint/crypto/ed25519"
)
func TestGenesisBad(t *testing.T) {
@ -36,7 +40,7 @@ func TestGenesisGood(t *testing.T) {
// create a base gendoc from struct
baseGenDoc := &GenesisDoc{
ChainID: "abc",
Validators: []GenesisValidator{{crypto.GenPrivKeyEd25519().PubKey(), 10, "myval"}},
Validators: []GenesisValidator{{ed25519.GenPrivKey().PubKey(), 10, "myval"}},
}
genDocBytes, err = cdc.MarshalJSON(baseGenDoc)
assert.NoError(t, err, "error marshalling genDoc")
@ -59,3 +63,44 @@ func TestGenesisGood(t *testing.T) {
genDoc, err = GenesisDocFromJSON(genDocBytes)
assert.Error(t, err, "expected error for genDoc json with block size of 0")
}
func TestGenesisSaveAs(t *testing.T) {
tmpfile, err := ioutil.TempFile("", "genesis")
require.NoError(t, err)
defer os.Remove(tmpfile.Name())
genDoc := randomGenesisDoc()
// save
genDoc.SaveAs(tmpfile.Name())
stat, err := tmpfile.Stat()
require.NoError(t, err)
if err != nil && stat.Size() <= 0 {
t.Fatalf("SaveAs failed to write any bytes to %v", tmpfile.Name())
}
err = tmpfile.Close()
require.NoError(t, err)
// load
genDoc2, err := GenesisDocFromFile(tmpfile.Name())
require.NoError(t, err)
// fails to unknown reason
// assert.EqualValues(t, genDoc2, genDoc)
assert.Equal(t, genDoc2.Validators, genDoc.Validators)
}
func TestGenesisValidatorHash(t *testing.T) {
genDoc := randomGenesisDoc()
assert.NotEmpty(t, genDoc.ValidatorHash())
}
func randomGenesisDoc() *GenesisDoc {
return &GenesisDoc{
GenesisTime: time.Now().UTC(),
ChainID: "abc",
Validators: []GenesisValidator{{ed25519.GenPrivKey().PubKey(), 10, "myval"}},
ConsensusParams: DefaultConsensusParams(),
}
}

View File

@ -4,8 +4,7 @@ import (
"testing"
"github.com/stretchr/testify/require"
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto/ed25519"
)
func TestHeartbeatCopy(t *testing.T) {
@ -27,7 +26,7 @@ func TestHeartbeatString(t *testing.T) {
hb := &Heartbeat{ValidatorIndex: 1, Height: 11, Round: 2}
require.Equal(t, hb.String(), "Heartbeat{1:000000000000 11/02 (0) <nil>}")
var key crypto.PrivKeyEd25519
var key ed25519.PrivKeyEd25519
sig, err := key.Sign([]byte("Tendermint"))
require.NoError(t, err)
hb.Signature = sig

View File

@ -6,6 +6,7 @@ import (
"testing"
"github.com/stretchr/testify/assert"
abci "github.com/tendermint/tendermint/abci/types"
)
func newConsensusParams(blockSize, partSize int) ConsensusParams {
@ -86,3 +87,59 @@ func TestConsensusParamsHash(t *testing.T) {
assert.NotEqual(t, hashes[i], hashes[i+1])
}
}
func TestConsensusParamsUpdate(t *testing.T) {
testCases := []struct {
params ConsensusParams
updates *abci.ConsensusParams
updatedParams ConsensusParams
}{
// empty updates
{
makeParams(1, 2, 3, 4, 5, 6),
&abci.ConsensusParams{},
makeParams(1, 2, 3, 4, 5, 6),
},
// negative BlockPartSizeBytes
{
makeParams(1, 2, 3, 4, 5, 6),
&abci.ConsensusParams{
BlockSize: &abci.BlockSize{
MaxBytes: -100,
MaxTxs: -200,
MaxGas: -300,
},
TxSize: &abci.TxSize{
MaxBytes: -400,
MaxGas: -500,
},
BlockGossip: &abci.BlockGossip{
BlockPartSizeBytes: -600,
},
},
makeParams(1, 2, 3, 4, 5, 6),
},
// fine updates
{
makeParams(1, 2, 3, 4, 5, 6),
&abci.ConsensusParams{
BlockSize: &abci.BlockSize{
MaxBytes: 100,
MaxTxs: 200,
MaxGas: 300,
},
TxSize: &abci.TxSize{
MaxBytes: 400,
MaxGas: 500,
},
BlockGossip: &abci.BlockGossip{
BlockPartSizeBytes: 600,
},
},
makeParams(100, 200, 300, 400, 500, 600),
},
}
for _, tc := range testCases {
assert.Equal(t, tc.updatedParams, tc.params.Update(tc.updates))
}
}

View File

@ -1,10 +1,12 @@
package types
import (
"bytes"
"io/ioutil"
"testing"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
cmn "github.com/tendermint/tendermint/libs/common"
)
@ -13,24 +15,21 @@ const (
)
func TestBasicPartSet(t *testing.T) {
// Construct random data of size partSize * 100
data := cmn.RandBytes(testPartSize * 100)
partSet := NewPartSetFromData(data, testPartSize)
if len(partSet.Hash()) == 0 {
t.Error("Expected to get hash")
}
if partSet.Total() != 100 {
t.Errorf("Expected to get 100 parts, but got %v", partSet.Total())
}
if !partSet.IsComplete() {
t.Errorf("PartSet should be complete")
}
assert.NotEmpty(t, partSet.Hash())
assert.Equal(t, 100, partSet.Total())
assert.Equal(t, 100, partSet.BitArray().Size())
assert.True(t, partSet.HashesTo(partSet.Hash()))
assert.True(t, partSet.IsComplete())
assert.Equal(t, 100, partSet.Count())
// Test adding parts to a new partSet.
partSet2 := NewPartSetFromHeader(partSet.Header())
assert.True(t, partSet2.HasHeader(partSet.Header()))
for i := 0; i < partSet.Total(); i++ {
part := partSet.GetPart(i)
//t.Logf("\n%v", part)
@ -39,31 +38,28 @@ func TestBasicPartSet(t *testing.T) {
t.Errorf("Failed to add part %v, error: %v", i, err)
}
}
// adding part with invalid index
added, err := partSet2.AddPart(&Part{Index: 10000})
assert.False(t, added)
assert.Error(t, err)
// adding existing part
added, err = partSet2.AddPart(partSet2.GetPart(0))
assert.False(t, added)
assert.Nil(t, err)
if !bytes.Equal(partSet.Hash(), partSet2.Hash()) {
t.Error("Expected to get same hash")
}
if partSet2.Total() != 100 {
t.Errorf("Expected to get 100 parts, but got %v", partSet2.Total())
}
if !partSet2.IsComplete() {
t.Errorf("Reconstructed PartSet should be complete")
}
assert.Equal(t, partSet.Hash(), partSet2.Hash())
assert.Equal(t, 100, partSet2.Total())
assert.True(t, partSet2.IsComplete())
// Reconstruct data, assert that they are equal.
data2Reader := partSet2.GetReader()
data2, err := ioutil.ReadAll(data2Reader)
if err != nil {
t.Errorf("Error reading data2Reader: %v", err)
}
if !bytes.Equal(data, data2) {
t.Errorf("Got wrong data.")
}
require.NoError(t, err)
assert.Equal(t, data, data2)
}
func TestWrongProof(t *testing.T) {
// Construct random data of size partSize * 100
data := cmn.RandBytes(testPartSize * 100)
partSet := NewPartSetFromData(data, testPartSize)
@ -86,5 +82,4 @@ func TestWrongProof(t *testing.T) {
if added || err == nil {
t.Errorf("Expected to fail adding a part with bad bytes.")
}
}

View File

@ -5,6 +5,7 @@ import (
"fmt"
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto/ed25519"
)
// PrivValidator defines the functionality of a local Tendermint validator
@ -47,7 +48,7 @@ type MockPV struct {
}
func NewMockPV() *MockPV {
return &MockPV{crypto.GenPrivKeyEd25519()}
return &MockPV{ed25519.GenPrivKey()}
}
// Implements PrivValidator.

261
types/proto3/block.pb.go Normal file
View File

@ -0,0 +1,261 @@
// Code generated by protoc-gen-go. DO NOT EDIT.
// source: block.proto
/*
Package proto3 is a generated protocol buffer package.
It is generated from these files:
block.proto
It has these top-level messages:
PartSetHeader
BlockID
Header
Timestamp
*/
package proto3
import proto "github.com/golang/protobuf/proto"
import fmt "fmt"
import math "math"
// Reference imports to suppress errors if they are not otherwise used.
var _ = proto.Marshal
var _ = fmt.Errorf
var _ = math.Inf
// This is a compile-time assertion to ensure that this generated file
// is compatible with the proto package it is being compiled against.
// A compilation error at this line likely means your copy of the
// proto package needs to be updated.
const _ = proto.ProtoPackageIsVersion2 // please upgrade the proto package
type PartSetHeader struct {
Total int32 `protobuf:"zigzag32,1,opt,name=Total" json:"Total,omitempty"`
Hash []byte `protobuf:"bytes,2,opt,name=Hash,proto3" json:"Hash,omitempty"`
}
func (m *PartSetHeader) Reset() { *m = PartSetHeader{} }
func (m *PartSetHeader) String() string { return proto.CompactTextString(m) }
func (*PartSetHeader) ProtoMessage() {}
func (*PartSetHeader) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{0} }
func (m *PartSetHeader) GetTotal() int32 {
if m != nil {
return m.Total
}
return 0
}
func (m *PartSetHeader) GetHash() []byte {
if m != nil {
return m.Hash
}
return nil
}
type BlockID struct {
Hash []byte `protobuf:"bytes,1,opt,name=Hash,proto3" json:"Hash,omitempty"`
PartsHeader *PartSetHeader `protobuf:"bytes,2,opt,name=PartsHeader" json:"PartsHeader,omitempty"`
}
func (m *BlockID) Reset() { *m = BlockID{} }
func (m *BlockID) String() string { return proto.CompactTextString(m) }
func (*BlockID) ProtoMessage() {}
func (*BlockID) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{1} }
func (m *BlockID) GetHash() []byte {
if m != nil {
return m.Hash
}
return nil
}
func (m *BlockID) GetPartsHeader() *PartSetHeader {
if m != nil {
return m.PartsHeader
}
return nil
}
type Header struct {
// basic block info
ChainID string `protobuf:"bytes,1,opt,name=ChainID" json:"ChainID,omitempty"`
Height int64 `protobuf:"zigzag64,2,opt,name=Height" json:"Height,omitempty"`
Time *Timestamp `protobuf:"bytes,3,opt,name=Time" json:"Time,omitempty"`
NumTxs int64 `protobuf:"zigzag64,4,opt,name=NumTxs" json:"NumTxs,omitempty"`
// prev block info
LastBlockID *BlockID `protobuf:"bytes,5,opt,name=LastBlockID" json:"LastBlockID,omitempty"`
TotalTxs int64 `protobuf:"zigzag64,6,opt,name=TotalTxs" json:"TotalTxs,omitempty"`
// hashes of block data
LastCommitHash []byte `protobuf:"bytes,7,opt,name=LastCommitHash,proto3" json:"LastCommitHash,omitempty"`
DataHash []byte `protobuf:"bytes,8,opt,name=DataHash,proto3" json:"DataHash,omitempty"`
// hashes from the app output from the prev block
ValidatorsHash []byte `protobuf:"bytes,9,opt,name=ValidatorsHash,proto3" json:"ValidatorsHash,omitempty"`
ConsensusHash []byte `protobuf:"bytes,10,opt,name=ConsensusHash,proto3" json:"ConsensusHash,omitempty"`
AppHash []byte `protobuf:"bytes,11,opt,name=AppHash,proto3" json:"AppHash,omitempty"`
LastResultsHash []byte `protobuf:"bytes,12,opt,name=LastResultsHash,proto3" json:"LastResultsHash,omitempty"`
// consensus info
EvidenceHash []byte `protobuf:"bytes,13,opt,name=EvidenceHash,proto3" json:"EvidenceHash,omitempty"`
}
func (m *Header) Reset() { *m = Header{} }
func (m *Header) String() string { return proto.CompactTextString(m) }
func (*Header) ProtoMessage() {}
func (*Header) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{2} }
func (m *Header) GetChainID() string {
if m != nil {
return m.ChainID
}
return ""
}
func (m *Header) GetHeight() int64 {
if m != nil {
return m.Height
}
return 0
}
func (m *Header) GetTime() *Timestamp {
if m != nil {
return m.Time
}
return nil
}
func (m *Header) GetNumTxs() int64 {
if m != nil {
return m.NumTxs
}
return 0
}
func (m *Header) GetLastBlockID() *BlockID {
if m != nil {
return m.LastBlockID
}
return nil
}
func (m *Header) GetTotalTxs() int64 {
if m != nil {
return m.TotalTxs
}
return 0
}
func (m *Header) GetLastCommitHash() []byte {
if m != nil {
return m.LastCommitHash
}
return nil
}
func (m *Header) GetDataHash() []byte {
if m != nil {
return m.DataHash
}
return nil
}
func (m *Header) GetValidatorsHash() []byte {
if m != nil {
return m.ValidatorsHash
}
return nil
}
func (m *Header) GetConsensusHash() []byte {
if m != nil {
return m.ConsensusHash
}
return nil
}
func (m *Header) GetAppHash() []byte {
if m != nil {
return m.AppHash
}
return nil
}
func (m *Header) GetLastResultsHash() []byte {
if m != nil {
return m.LastResultsHash
}
return nil
}
func (m *Header) GetEvidenceHash() []byte {
if m != nil {
return m.EvidenceHash
}
return nil
}
// Timestamp wraps how amino encodes time. Note that this is different from the protobuf well-known type
// protobuf/timestamp.proto in the sense that there seconds and nanos are varint encoded. See:
// https://github.com/google/protobuf/blob/d2980062c859649523d5fd51d6b55ab310e47482/src/google/protobuf/timestamp.proto#L123-L135
// Also nanos do not get skipped if they are zero in amino.
type Timestamp struct {
Seconds int64 `protobuf:"fixed64,1,opt,name=seconds" json:"seconds,omitempty"`
Nanos int32 `protobuf:"fixed32,2,opt,name=nanos" json:"nanos,omitempty"`
}
func (m *Timestamp) Reset() { *m = Timestamp{} }
func (m *Timestamp) String() string { return proto.CompactTextString(m) }
func (*Timestamp) ProtoMessage() {}
func (*Timestamp) Descriptor() ([]byte, []int) { return fileDescriptor0, []int{3} }
func (m *Timestamp) GetSeconds() int64 {
if m != nil {
return m.Seconds
}
return 0
}
func (m *Timestamp) GetNanos() int32 {
if m != nil {
return m.Nanos
}
return 0
}
func init() {
proto.RegisterType((*PartSetHeader)(nil), "proto3.PartSetHeader")
proto.RegisterType((*BlockID)(nil), "proto3.BlockID")
proto.RegisterType((*Header)(nil), "proto3.Header")
proto.RegisterType((*Timestamp)(nil), "proto3.Timestamp")
}
func init() { proto.RegisterFile("block.proto", fileDescriptor0) }
var fileDescriptor0 = []byte{
// 372 bytes of a gzipped FileDescriptorProto
0x1f, 0x8b, 0x08, 0x00, 0x00, 0x00, 0x00, 0x00, 0x02, 0xff, 0x5c, 0x92, 0x4f, 0x6b, 0xe3, 0x30,
0x10, 0xc5, 0xf1, 0xe6, 0xff, 0x38, 0xd9, 0x6c, 0x86, 0xdd, 0xc5, 0xf4, 0x14, 0x4c, 0x5b, 0x72,
0x0a, 0xb4, 0x39, 0x94, 0xd2, 0x53, 0x9b, 0x14, 0x12, 0x28, 0xa5, 0xa8, 0x21, 0x77, 0x25, 0x16,
0x8d, 0xa9, 0x2d, 0x19, 0x4b, 0x29, 0xfd, 0x7c, 0xfd, 0x64, 0x45, 0x23, 0xdb, 0x8d, 0x73, 0x4a,
0xde, 0x9b, 0x37, 0xbf, 0x91, 0x47, 0x02, 0x7f, 0x9b, 0xa8, 0xdd, 0xfb, 0x34, 0xcb, 0x95, 0x51,
0xd8, 0xa6, 0x9f, 0x59, 0x78, 0x0b, 0x83, 0x17, 0x9e, 0x9b, 0x57, 0x61, 0x96, 0x82, 0x47, 0x22,
0xc7, 0xbf, 0xd0, 0x5a, 0x2b, 0xc3, 0x93, 0xc0, 0x1b, 0x7b, 0x93, 0x11, 0x73, 0x02, 0x11, 0x9a,
0x4b, 0xae, 0xf7, 0xc1, 0xaf, 0xb1, 0x37, 0xe9, 0x33, 0xfa, 0x1f, 0x6e, 0xa0, 0xf3, 0x60, 0x89,
0xab, 0x45, 0x55, 0xf6, 0x7e, 0xca, 0x78, 0x03, 0xbe, 0x25, 0x6b, 0xc7, 0xa5, 0x4e, 0xff, 0xfa,
0x9f, 0x1b, 0x3f, 0x9b, 0xd6, 0x86, 0xb2, 0xe3, 0x64, 0xf8, 0xd5, 0x80, 0x76, 0x71, 0x98, 0x00,
0x3a, 0xf3, 0x3d, 0x8f, 0xe5, 0x6a, 0x41, 0xe8, 0x1e, 0x2b, 0x25, 0xfe, 0xb7, 0x99, 0xf8, 0x6d,
0x6f, 0x08, 0x8c, 0xac, 0x50, 0x78, 0x01, 0xcd, 0x75, 0x9c, 0x8a, 0xa0, 0x41, 0xe3, 0x46, 0xe5,
0x38, 0xeb, 0x69, 0xc3, 0xd3, 0x8c, 0x51, 0xd9, 0xb6, 0x3f, 0x1f, 0xd2, 0xf5, 0xa7, 0x0e, 0x9a,
0xae, 0xdd, 0x29, 0xbc, 0x02, 0xff, 0x89, 0x6b, 0x53, 0x7c, 0x57, 0xd0, 0x22, 0xca, 0xb0, 0xa4,
0x14, 0x36, 0x3b, 0xce, 0xe0, 0x19, 0x74, 0x69, 0x47, 0x16, 0xd6, 0x26, 0x58, 0xa5, 0xf1, 0x12,
0x7e, 0xdb, 0xe8, 0x5c, 0xa5, 0x69, 0x6c, 0x68, 0x43, 0x1d, 0xda, 0xd0, 0x89, 0x6b, 0x19, 0x0b,
0x6e, 0x38, 0x25, 0xba, 0x94, 0xa8, 0xb4, 0x65, 0x6c, 0x78, 0x12, 0x47, 0xdc, 0xa8, 0x5c, 0x53,
0xa2, 0xe7, 0x18, 0x75, 0x17, 0xcf, 0x61, 0x30, 0x57, 0x52, 0x0b, 0xa9, 0x0f, 0x2e, 0x06, 0x14,
0xab, 0x9b, 0x76, 0xa3, 0xf7, 0x59, 0x46, 0x75, 0x9f, 0xea, 0xa5, 0xc4, 0x09, 0x0c, 0xed, 0xa9,
0x98, 0xd0, 0x87, 0xc4, 0x38, 0x42, 0x9f, 0x12, 0xa7, 0x36, 0x86, 0xd0, 0x7f, 0xfc, 0x88, 0x23,
0x21, 0x77, 0x82, 0x62, 0x03, 0x8a, 0xd5, 0xbc, 0xf0, 0x0e, 0x7a, 0xd5, 0xce, 0xed, 0x50, 0x2d,
0x76, 0x4a, 0x46, 0x9a, 0xae, 0xf1, 0x0f, 0x2b, 0xa5, 0x7d, 0x6d, 0x92, 0x4b, 0xa5, 0xe9, 0x16,
0x87, 0xcc, 0x89, 0x6d, 0xf1, 0x38, 0xbf, 0x03, 0x00, 0x00, 0xff, 0xff, 0x6b, 0x74, 0x2f, 0xbd,
0xb2, 0x02, 0x00, 0x00,
}

48
types/proto3/block.proto Normal file
View File

@ -0,0 +1,48 @@
syntax = "proto3";
package proto3;
message PartSetHeader {
sint32 Total = 1;
bytes Hash = 2;
}
message BlockID {
bytes Hash = 1;
PartSetHeader PartsHeader = 2;
}
message Header {
// basic block info
string ChainID = 1;
sint64 Height = 2;
Timestamp Time = 3;
sint64 NumTxs = 4;
// prev block info
BlockID LastBlockID = 5;
sint64 TotalTxs = 6;
// hashes of block data
bytes LastCommitHash = 7; // commit from validators from the last block
bytes DataHash = 8; // transactions
// hashes from the app output from the prev block
bytes ValidatorsHash = 9; // validators for the current block
bytes ConsensusHash = 10; // consensus params for current block
bytes AppHash = 11; // state after txs from the previous block
bytes LastResultsHash = 12; // root hash of all results from the txs from the previous block
// consensus info
bytes EvidenceHash = 13; // evidence included in the block
}
// Timestamp wraps how amino encodes time. Note that this is different from the protobuf well-known type
// protobuf/timestamp.proto in the sense that there seconds and nanos are varint encoded. See:
// https://github.com/google/protobuf/blob/d2980062c859649523d5fd51d6b55ab310e47482/src/google/protobuf/timestamp.proto#L123-L135
// Also nanos do not get skipped if they are zero in amino.
message Timestamp {
sfixed64 seconds = 1;
sfixed32 nanos = 2;
}

115
types/proto3_test.go Normal file
View File

@ -0,0 +1,115 @@
package types
import (
"testing"
"time"
"github.com/golang/protobuf/proto"
"github.com/stretchr/testify/assert"
"github.com/tendermint/tendermint/types/proto3"
)
func TestProto3Compatibility(t *testing.T) {
tm, err := time.Parse("Mon Jan 2 15:04:05 -0700 MST 2006", "Mon Jan 2 15:04:05 -0700 MST 2006")
assert.NoError(t, err)
// add some nanos, otherwise protobuf will skip over this while amino (still) won't!
tm = tm.Add(50000 * time.Nanosecond)
seconds := tm.Unix()
nanos := int32(tm.Nanosecond())
t.Log("seconds", seconds)
t.Log("nanos", nanos)
pbHeader := proto3.Header{
ChainID: "cosmos",
Height: 150,
Time: &proto3.Timestamp{Seconds: seconds, Nanos: nanos},
NumTxs: 7,
LastBlockID: &proto3.BlockID{
Hash: []byte("some serious hashing"),
PartsHeader: &proto3.PartSetHeader{
Total: 8,
Hash: []byte("some more serious hashing"),
},
},
TotalTxs: 100,
LastCommitHash: []byte("commit hash"),
DataHash: []byte("data hash"),
ValidatorsHash: []byte("validators hash"),
}
aminoHeader := Header{
ChainID: "cosmos",
Height: 150,
Time: tm,
NumTxs: 7,
LastBlockID: BlockID{
Hash: []byte("some serious hashing"),
PartsHeader: PartSetHeader{
Total: 8,
Hash: []byte("some more serious hashing"),
},
},
TotalTxs: 100,
LastCommitHash: []byte("commit hash"),
DataHash: []byte("data hash"),
ValidatorsHash: []byte("validators hash"),
}
ab, err := cdc.MarshalBinaryBare(aminoHeader)
assert.NoError(t, err, "unexpected error")
pb, err := proto.Marshal(&pbHeader)
assert.NoError(t, err, "unexpected error")
// This works:
assert.Equal(t, ab, pb, "encoding doesn't match")
emptyLastBlockPb := proto3.Header{
ChainID: "cosmos",
Height: 150,
Time: &proto3.Timestamp{Seconds: seconds, Nanos: nanos},
NumTxs: 7,
// This is not fully skipped in amino (yet) although it is empty:
LastBlockID: &proto3.BlockID{
PartsHeader: &proto3.PartSetHeader{
},
},
TotalTxs: 100,
LastCommitHash: []byte("commit hash"),
DataHash: []byte("data hash"),
ValidatorsHash: []byte("validators hash"),
}
emptyLastBlockAm := Header{
ChainID: "cosmos",
Height: 150,
Time: tm,
NumTxs: 7,
TotalTxs: 100,
LastCommitHash: []byte("commit hash"),
DataHash: []byte("data hash"),
ValidatorsHash: []byte("validators hash"),
}
ab, err = cdc.MarshalBinaryBare(emptyLastBlockAm)
assert.NoError(t, err, "unexpected error")
pb, err = proto.Marshal(&emptyLastBlockPb)
assert.NoError(t, err, "unexpected error")
// This works:
assert.Equal(t, ab, pb, "encoding doesn't match")
pb, err = proto.Marshal(&proto3.Header{})
assert.NoError(t, err, "unexpected error")
t.Log(pb)
// While in protobuf Header{} encodes to an empty byte slice it does not in amino:
ab, err = cdc.MarshalBinaryBare(Header{})
assert.NoError(t, err, "unexpected error")
t.Log(ab)
pb, err = proto.Marshal(&proto3.Timestamp{})
assert.NoError(t, err, "unexpected error")
t.Log(pb)
ab, err = cdc.MarshalBinaryBare(time.Time{})
assert.NoError(t, err, "unexpected error")
t.Log(ab)
}

View File

@ -7,7 +7,9 @@ import (
"time"
abci "github.com/tendermint/tendermint/abci/types"
crypto "github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto/ed25519"
"github.com/tendermint/tendermint/crypto/secp256k1"
)
//-------------------------------------------------------
@ -61,12 +63,12 @@ func (tm2pb) Validator(val *Validator) abci.Validator {
// TODO: add cases when new pubkey types are added to crypto
func (tm2pb) PubKey(pubKey crypto.PubKey) abci.PubKey {
switch pk := pubKey.(type) {
case crypto.PubKeyEd25519:
case ed25519.PubKeyEd25519:
return abci.PubKey{
Type: ABCIPubKeyTypeEd25519,
Data: pk[:],
}
case crypto.PubKeySecp256k1:
case secp256k1.PubKeySecp256k1:
return abci.PubKey{
Type: ABCIPubKeyTypeSecp256k1,
Data: pk[:],
@ -78,7 +80,7 @@ func (tm2pb) PubKey(pubKey crypto.PubKey) abci.PubKey {
// XXX: panics on nil or unknown pubkey type
func (tm2pb) Validators(vals *ValidatorSet) []abci.Validator {
validators := make([]abci.Validator, len(vals.Validators))
validators := make([]abci.Validator, vals.Size())
for i, val := range vals.Validators {
validators[i] = TM2PB.Validator(val)
}
@ -161,14 +163,14 @@ func (pb2tm) PubKey(pubKey abci.PubKey) (crypto.PubKey, error) {
if len(pubKey.Data) != sizeEd {
return nil, fmt.Errorf("Invalid size for PubKeyEd25519. Got %d, expected %d", len(pubKey.Data), sizeEd)
}
var pk crypto.PubKeyEd25519
var pk ed25519.PubKeyEd25519
copy(pk[:], pubKey.Data)
return pk, nil
case ABCIPubKeyTypeSecp256k1:
if len(pubKey.Data) != sizeSecp {
return nil, fmt.Errorf("Invalid size for PubKeyEd25519. Got %d, expected %d", len(pubKey.Data), sizeSecp)
}
var pk crypto.PubKeySecp256k1
var pk secp256k1.PubKeySecp256k1
copy(pk[:], pubKey.Data)
return pk, nil
default:

View File

@ -2,15 +2,18 @@ package types
import (
"testing"
"time"
"github.com/stretchr/testify/assert"
abci "github.com/tendermint/tendermint/abci/types"
crypto "github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto/ed25519"
"github.com/tendermint/tendermint/crypto/secp256k1"
)
func TestABCIPubKey(t *testing.T) {
pkEd := crypto.GenPrivKeyEd25519().PubKey()
pkSecp := crypto.GenPrivKeySecp256k1().PubKey()
pkEd := ed25519.GenPrivKey().PubKey()
pkSecp := secp256k1.GenPrivKey().PubKey()
testABCIPubKey(t, pkEd, ABCIPubKeyTypeEd25519)
testABCIPubKey(t, pkSecp, ABCIPubKeyTypeSecp256k1)
}
@ -23,7 +26,7 @@ func testABCIPubKey(t *testing.T, pk crypto.PubKey, typeStr string) {
}
func TestABCIValidators(t *testing.T) {
pkEd := crypto.GenPrivKeyEd25519().PubKey()
pkEd := ed25519.GenPrivKey().PubKey()
// correct validator
tmValExpected := &Validator{
@ -43,6 +46,9 @@ func TestABCIValidators(t *testing.T) {
assert.Nil(t, err)
assert.Equal(t, tmValExpected, tmVals[0])
abciVals := TM2PB.Validators(NewValidatorSet(tmVals))
assert.Equal(t, []abci.Validator{abciVal}, abciVals)
// val with address
tmVal.Address = pkEd.Address()
@ -67,3 +73,50 @@ func TestABCIConsensusParams(t *testing.T) {
assert.Equal(t, *cp, cp2)
}
func TestABCIHeader(t *testing.T) {
header := &Header{
Height: int64(3),
Time: time.Now(),
NumTxs: int64(10),
}
abciHeader := TM2PB.Header(header)
assert.Equal(t, int64(3), abciHeader.Height)
}
func TestABCIEvidence(t *testing.T) {
val := NewMockPV()
blockID := makeBlockID("blockhash", 1000, "partshash")
blockID2 := makeBlockID("blockhash2", 1000, "partshash")
const chainID = "mychain"
ev := &DuplicateVoteEvidence{
PubKey: val.GetPubKey(),
VoteA: makeVote(val, chainID, 0, 10, 2, 1, blockID),
VoteB: makeVote(val, chainID, 0, 10, 2, 1, blockID2),
}
abciEv := TM2PB.Evidence(
ev,
NewValidatorSet([]*Validator{NewValidator(val.GetPubKey(), 10)}),
time.Now(),
)
assert.Equal(t, "duplicate/vote", abciEv.Type)
}
type pubKeyEddie struct{}
func (pubKeyEddie) Address() Address { return []byte{} }
func (pubKeyEddie) Bytes() []byte { return []byte{} }
func (pubKeyEddie) VerifyBytes(msg []byte, sig crypto.Signature) bool { return false }
func (pubKeyEddie) Equals(crypto.PubKey) bool { return false }
func TestABCIValidatorFromPubKeyAndPower(t *testing.T) {
pubkey := ed25519.GenPrivKey().PubKey()
abciVal := TM2PB.ValidatorFromPubKeyAndPower(pubkey, 10)
assert.Equal(t, int64(10), abciVal.Power)
assert.Panics(t, func() { TM2PB.ValidatorFromPubKeyAndPower(nil, 10) })
assert.Panics(t, func() { TM2PB.ValidatorFromPubKeyAndPower(pubKeyEddie{}, 10) })
}

View File

@ -24,15 +24,16 @@ func (a ABCIResult) Hash() []byte {
// ABCIResults wraps the deliver tx results to return a proof
type ABCIResults []ABCIResult
// NewResults creates ABCIResults from ResponseDeliverTx
func NewResults(del []*abci.ResponseDeliverTx) ABCIResults {
res := make(ABCIResults, len(del))
for i, d := range del {
// NewResults creates ABCIResults from the list of ResponseDeliverTx.
func NewResults(responses []*abci.ResponseDeliverTx) ABCIResults {
res := make(ABCIResults, len(responses))
for i, d := range responses {
res[i] = NewResultFromResponse(d)
}
return res
}
// NewResultFromResponse creates ABCIResult from ResponseDeliverTx.
func NewResultFromResponse(response *abci.ResponseDeliverTx) ABCIResult {
return ABCIResult{
Code: response.Code,

View File

@ -5,6 +5,7 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
abci "github.com/tendermint/tendermint/abci/types"
)
func TestABCIResults(t *testing.T) {
@ -41,3 +42,14 @@ func TestABCIResults(t *testing.T) {
assert.True(t, valid, "%d", i)
}
}
func TestABCIBytes(t *testing.T) {
results := NewResults([]*abci.ResponseDeliverTx{
{Code: 0, Data: []byte{}},
{Code: 0, Data: []byte("one")},
{Code: 14, Data: nil},
{Code: 14, Data: []byte("foo")},
{Code: 14, Data: []byte("bar")},
})
assert.NotNil(t, results.Bytes())
}

View File

@ -24,21 +24,32 @@ func randInt(low, high int) int {
}
func TestTxIndex(t *testing.T) {
assert := assert.New(t)
for i := 0; i < 20; i++ {
txs := makeTxs(15, 60)
for j := 0; j < len(txs); j++ {
tx := txs[j]
idx := txs.Index(tx)
assert.Equal(j, idx)
assert.Equal(t, j, idx)
}
assert.Equal(-1, txs.Index(nil))
assert.Equal(-1, txs.Index(Tx("foodnwkf")))
assert.Equal(t, -1, txs.Index(nil))
assert.Equal(t, -1, txs.Index(Tx("foodnwkf")))
}
}
func TestTxIndexByHash(t *testing.T) {
for i := 0; i < 20; i++ {
txs := makeTxs(15, 60)
for j := 0; j < len(txs); j++ {
tx := txs[j]
idx := txs.IndexByHash(tx.Hash())
assert.Equal(t, j, idx)
}
assert.Equal(t, -1, txs.IndexByHash(nil))
assert.Equal(t, -1, txs.IndexByHash(Tx("foodnwkf").Hash()))
}
}
func TestValidTxProof(t *testing.T) {
assert := assert.New(t)
cases := []struct {
txs Txs
}{
@ -58,21 +69,21 @@ func TestValidTxProof(t *testing.T) {
leaf := txs[i]
leafHash := leaf.Hash()
proof := txs.Proof(i)
assert.Equal(i, proof.Index, "%d: %d", h, i)
assert.Equal(len(txs), proof.Total, "%d: %d", h, i)
assert.EqualValues(root, proof.RootHash, "%d: %d", h, i)
assert.EqualValues(leaf, proof.Data, "%d: %d", h, i)
assert.EqualValues(leafHash, proof.LeafHash(), "%d: %d", h, i)
assert.Nil(proof.Validate(root), "%d: %d", h, i)
assert.NotNil(proof.Validate([]byte("foobar")), "%d: %d", h, i)
assert.Equal(t, i, proof.Index, "%d: %d", h, i)
assert.Equal(t, len(txs), proof.Total, "%d: %d", h, i)
assert.EqualValues(t, root, proof.RootHash, "%d: %d", h, i)
assert.EqualValues(t, leaf, proof.Data, "%d: %d", h, i)
assert.EqualValues(t, leafHash, proof.LeafHash(), "%d: %d", h, i)
assert.Nil(t, proof.Validate(root), "%d: %d", h, i)
assert.NotNil(t, proof.Validate([]byte("foobar")), "%d: %d", h, i)
// read-write must also work
var p2 TxProof
bin, err := cdc.MarshalBinary(proof)
assert.Nil(err)
assert.Nil(t, err)
err = cdc.UnmarshalBinary(bin, &p2)
if assert.Nil(err, "%d: %d: %+v", h, i, err) {
assert.Nil(p2.Validate(root), "%d: %d", h, i)
if assert.Nil(t, err, "%d: %d: %+v", h, i, err) {
assert.Nil(t, p2.Validate(root), "%d: %d", h, i)
}
}
}
@ -86,8 +97,6 @@ func TestTxProofUnchangable(t *testing.T) {
}
func testTxProofUnchangable(t *testing.T) {
assert := assert.New(t)
// make some proof
txs := makeTxs(randInt(2, 100), randInt(16, 128))
root := txs.Hash()
@ -95,9 +104,9 @@ func testTxProofUnchangable(t *testing.T) {
proof := txs.Proof(i)
// make sure it is valid to start with
assert.Nil(proof.Validate(root))
assert.Nil(t, proof.Validate(root))
bin, err := cdc.MarshalBinary(proof)
assert.Nil(err)
assert.Nil(t, err)
// try mutating the data and make sure nothing breaks
for j := 0; j < 500; j++ {

View File

@ -41,7 +41,7 @@ func NewValidatorSet(valz []*Validator) *ValidatorSet {
vals := &ValidatorSet{
Validators: validators,
}
if valz != nil {
if len(valz) > 0 {
vals.IncrementAccum(1)
}
@ -55,7 +55,8 @@ func (vals *ValidatorSet) CopyIncrementAccum(times int) *ValidatorSet {
return copy
}
// Increment Accum and update the proposer.
// IncrementAccum increments accum of each validator and updates the
// proposer. Panics if validator set is empty.
func (vals *ValidatorSet) IncrementAccum(times int) {
// Add VotingPower * times to each validator and order into heap.

View File

@ -11,9 +11,64 @@ import (
"github.com/stretchr/testify/assert"
crypto "github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto/ed25519"
cmn "github.com/tendermint/tendermint/libs/common"
)
func TestValidatorSetBasic(t *testing.T) {
for _, vset := range []*ValidatorSet{NewValidatorSet([]*Validator{}), NewValidatorSet(nil)} {
assert.Panics(t, func() { vset.IncrementAccum(1) })
assert.EqualValues(t, vset, vset.Copy())
assert.False(t, vset.HasAddress([]byte("some val")))
idx, val := vset.GetByAddress([]byte("some val"))
assert.Equal(t, -1, idx)
assert.Nil(t, val)
addr, val := vset.GetByIndex(-100)
assert.Nil(t, addr)
assert.Nil(t, val)
addr, val = vset.GetByIndex(0)
assert.Nil(t, addr)
assert.Nil(t, val)
addr, val = vset.GetByIndex(100)
assert.Nil(t, addr)
assert.Nil(t, val)
assert.Zero(t, vset.Size())
assert.Equal(t, int64(0), vset.TotalVotingPower())
assert.Nil(t, vset.GetProposer())
assert.Nil(t, vset.Hash())
// add
val = randValidator_()
assert.True(t, vset.Add(val))
assert.True(t, vset.HasAddress(val.Address))
idx, val2 := vset.GetByAddress(val.Address)
assert.Equal(t, 0, idx)
assert.Equal(t, val, val2)
addr, val2 = vset.GetByIndex(0)
assert.Equal(t, []byte(val.Address), addr)
assert.Equal(t, val, val2)
assert.Equal(t, 1, vset.Size())
assert.Equal(t, val.VotingPower, vset.TotalVotingPower())
assert.Equal(t, val, vset.GetProposer())
assert.NotNil(t, vset.Hash())
assert.NotPanics(t, func() { vset.IncrementAccum(1) })
// update
assert.False(t, vset.Update(randValidator_()))
val.VotingPower = 100
assert.True(t, vset.Update(val))
// remove
val2, removed := vset.Remove(randValidator_().Address)
assert.Nil(t, val2)
assert.False(t, removed)
val2, removed = vset.Remove(val.Address)
assert.Equal(t, val.Address, val2.Address)
assert.True(t, removed)
}
}
func TestCopy(t *testing.T) {
vset := randValidatorSet(10)
vsetHash := vset.Hash()
@ -33,7 +88,7 @@ func BenchmarkValidatorSetCopy(b *testing.B) {
b.StopTimer()
vset := NewValidatorSet([]*Validator{})
for i := 0; i < 1000; i++ {
privKey := crypto.GenPrivKeyEd25519()
privKey := ed25519.GenPrivKey()
pubKey := privKey.PubKey()
val := NewValidator(pubKey, 0)
if !vset.Add(val) {
@ -197,7 +252,7 @@ func newValidator(address []byte, power int64) *Validator {
func randPubKey() crypto.PubKey {
var pubKey [32]byte
copy(pubKey[:], cmn.RandBytes(32))
return crypto.PubKeyEd25519(pubKey)
return ed25519.PubKeyEd25519(pubKey)
}
func randValidator_() *Validator {
@ -314,7 +369,7 @@ func TestSafeSubClip(t *testing.T) {
//-------------------------------------------------------------------
func TestValidatorSetVerifyCommit(t *testing.T) {
privKey := crypto.GenPrivKeyEd25519()
privKey := ed25519.GenPrivKey()
pubKey := privKey.PubKey()
v1 := NewValidator(pubKey, 1000)
vset := NewValidatorSet([]*Validator{v1})

View File

@ -4,7 +4,9 @@ import (
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/tendermint/tendermint/crypto/ed25519"
)
func examplePrevote() *Vote {
@ -50,29 +52,9 @@ func TestVoteSignable(t *testing.T) {
}
}
func TestVoteString(t *testing.T) {
tc := []struct {
name string
in string
out string
}{
{"Precommit", examplePrecommit().String(), `Vote{56789:616464720000 12345/02/2(Precommit) 686173680000 <nil> @ 2017-12-25T03:00:01.234Z}`},
{"Prevote", examplePrevote().String(), `Vote{56789:616464720000 12345/02/1(Prevote) 686173680000 <nil> @ 2017-12-25T03:00:01.234Z}`},
}
for _, tt := range tc {
tt := tt
t.Run(tt.name, func(st *testing.T) {
if tt.in != tt.out {
t.Errorf("Got unexpected string for Proposal. Expected:\n%v\nGot:\n%v", tt.in, tt.out)
}
})
}
}
func TestVoteVerifySignature(t *testing.T) {
privVal := NewMockPV()
pubKey := privVal.GetPubKey()
pubkey := privVal.GetPubKey()
vote := examplePrecommit()
signBytes := vote.SignBytes("test_chain_id")
@ -82,7 +64,7 @@ func TestVoteVerifySignature(t *testing.T) {
require.NoError(t, err)
// verify the same vote
valid := pubKey.VerifyBytes(vote.SignBytes("test_chain_id"), vote.Signature)
valid := pubkey.VerifyBytes(vote.SignBytes("test_chain_id"), vote.Signature)
require.True(t, valid)
// serialize, deserialize and verify again....
@ -95,7 +77,7 @@ func TestVoteVerifySignature(t *testing.T) {
// verify the transmitted vote
newSignBytes := precommit.SignBytes("test_chain_id")
require.Equal(t, string(signBytes), string(newSignBytes))
valid = pubKey.VerifyBytes(newSignBytes, precommit.Signature)
valid = pubkey.VerifyBytes(newSignBytes, precommit.Signature)
require.True(t, valid)
}
@ -119,3 +101,21 @@ func TestIsVoteTypeValid(t *testing.T) {
})
}
}
func TestVoteVerify(t *testing.T) {
privVal := NewMockPV()
pubkey := privVal.GetPubKey()
vote := examplePrevote()
vote.ValidatorAddress = pubkey.Address()
err := vote.Verify("test_chain_id", ed25519.GenPrivKey().PubKey())
if assert.Error(t, err) {
assert.Equal(t, ErrVoteInvalidValidatorAddress, err)
}
err = vote.Verify("test_chain_id", pubkey)
if assert.Error(t, err) {
assert.Equal(t, ErrVoteInvalidSignature, err)
}
}

View File

@ -2,11 +2,11 @@ package types
import (
"github.com/tendermint/go-amino"
"github.com/tendermint/tendermint/crypto"
cryptoAmino "github.com/tendermint/tendermint/crypto/encoding/amino"
)
var cdc = amino.NewCodec()
func init() {
crypto.RegisterAmino(cdc)
cryptoAmino.RegisterAmino(cdc)
}