Browse Source

Merge pull request #716 from tendermint/unstable

Unstable
pull/702/merge
Ethan Buchman 7 years ago
committed by GitHub
parent
commit
ddb8430341
46 changed files with 316 additions and 316 deletions
  1. +0
    -1
      Makefile
  2. +4
    -3
      benchmarks/map_test.go
  3. +2
    -2
      benchmarks/os_test.go
  4. +1
    -2
      benchmarks/simu/counter.go
  5. +12
    -12
      blockchain/pool.go
  6. +2
    -2
      blockchain/pool_test.go
  7. +34
    -14
      blockchain/reactor.go
  8. +158
    -0
      blockchain/reactor_test.go
  9. +2
    -2
      cmd/tendermint/commands/root_test.go
  10. +5
    -5
      consensus/byzantine_test.go
  11. +5
    -5
      consensus/common_test.go
  12. +5
    -5
      consensus/height_vote_set.go
  13. +2
    -2
      consensus/height_vote_set_test.go
  14. +4
    -4
      consensus/mempool_test.go
  15. +4
    -4
      consensus/state_test.go
  16. +2
    -2
      consensus/version.go
  17. +7
    -7
      consensus/wal.go
  18. +2
    -1
      node/id.go
  19. +1
    -1
      p2p/listener.go
  20. +1
    -0
      p2p/peer.go
  21. +0
    -5
      p2p/upnp/README.md
  22. +4
    -6
      p2p/upnp/upnp.go
  23. +1
    -0
      rpc/client/httpclient.go
  24. +2
    -2
      rpc/client/mock/abci_test.go
  25. +2
    -2
      rpc/client/mock/status_test.go
  26. +2
    -1
      rpc/client/rpc_test.go
  27. +4
    -4
      rpc/core/blocks.go
  28. +1
    -2
      rpc/core/pipe.go
  29. +0
    -1
      rpc/core/types/responses.go
  30. +1
    -0
      rpc/core/types/responses_test.go
  31. +2
    -3
      rpc/grpc/api.go
  32. +2
    -2
      rpc/grpc/client_server.go
  33. +0
    -12
      rpc/lib/Dockerfile
  34. +0
    -18
      rpc/lib/Makefile
  35. +0
    -121
      rpc/lib/README.md
  36. +0
    -21
      rpc/lib/circle.yml
  37. +1
    -0
      rpc/lib/rpc_test.go
  38. +1
    -0
      state/execution_test.go
  39. +3
    -5
      state/state_test.go
  40. +1
    -0
      types/genesis_test.go
  41. +3
    -3
      types/part_set_test.go
  42. +0
    -1
      types/proposal.go
  43. +2
    -2
      types/signable.go
  44. +1
    -0
      types/tx_test.go
  45. +19
    -19
      types/vote_set.go
  46. +11
    -12
      types/vote_set_test.go

+ 0
- 1
Makefile View File

@ -2,7 +2,6 @@ GOTOOLS = \
github.com/mitchellh/gox \
github.com/tcnksm/ghr \
github.com/Masterminds/glide \
honnef.co/go/tools/cmd/megacheck
PACKAGES=$(shell go list ./... | grep -v '/vendor/')
BUILD_TAGS?=tendermint


+ 4
- 3
benchmarks/map_test.go View File

@ -1,8 +1,9 @@
package benchmarks
import (
. "github.com/tendermint/tmlibs/common"
"testing"
cmn "github.com/tendermint/tmlibs/common"
)
func BenchmarkSomething(b *testing.B) {
@ -11,11 +12,11 @@ func BenchmarkSomething(b *testing.B) {
numChecks := 100000
keys := make([]string, numItems)
for i := 0; i < numItems; i++ {
keys[i] = RandStr(100)
keys[i] = cmn.RandStr(100)
}
txs := make([]string, numChecks)
for i := 0; i < numChecks; i++ {
txs[i] = RandStr(100)
txs[i] = cmn.RandStr(100)
}
b.StartTimer()


+ 2
- 2
benchmarks/os_test.go View File

@ -4,7 +4,7 @@ import (
"os"
"testing"
. "github.com/tendermint/tmlibs/common"
cmn "github.com/tendermint/tmlibs/common"
)
func BenchmarkFileWrite(b *testing.B) {
@ -14,7 +14,7 @@ func BenchmarkFileWrite(b *testing.B) {
if err != nil {
b.Error(err)
}
testString := RandStr(200) + "\n"
testString := cmn.RandStr(200) + "\n"
b.StartTimer()
for i := 0; i < b.N; i++ {


+ 1
- 2
benchmarks/simu/counter.go View File

@ -3,9 +3,8 @@ package main
import (
"context"
"encoding/binary"
"time"
//"encoding/hex"
"fmt"
"time"
rpcclient "github.com/tendermint/tendermint/rpc/lib/client"
cmn "github.com/tendermint/tmlibs/common"


+ 12
- 12
blockchain/pool.go View File

@ -6,7 +6,7 @@ import (
"time"
"github.com/tendermint/tendermint/types"
. "github.com/tendermint/tmlibs/common"
cmn "github.com/tendermint/tmlibs/common"
flow "github.com/tendermint/tmlibs/flowrate"
"github.com/tendermint/tmlibs/log"
)
@ -33,7 +33,7 @@ var peerTimeoutSeconds = time.Duration(15) // not const so we can override with
*/
type BlockPool struct {
BaseService
cmn.BaseService
startTime time.Time
mtx sync.Mutex
@ -59,7 +59,7 @@ func NewBlockPool(start int, requestsCh chan<- BlockRequest, timeoutsCh chan<- s
requestsCh: requestsCh,
timeoutsCh: timeoutsCh,
}
bp.BaseService = *NewBaseService(nil, "BlockPool", bp)
bp.BaseService = *cmn.NewBaseService(nil, "BlockPool", bp)
return bp
}
@ -137,14 +137,14 @@ func (pool *BlockPool) IsCaughtUp() bool {
maxPeerHeight := 0
for _, peer := range pool.peers {
maxPeerHeight = MaxInt(maxPeerHeight, peer.height)
maxPeerHeight = cmn.MaxInt(maxPeerHeight, peer.height)
}
// some conditions to determine if we're caught up
receivedBlockOrTimedOut := (pool.height > 0 || time.Since(pool.startTime) > 5*time.Second)
ourChainIsLongestAmongPeers := maxPeerHeight == 0 || pool.height >= maxPeerHeight
isCaughtUp := receivedBlockOrTimedOut && ourChainIsLongestAmongPeers
pool.Logger.Info(Fmt("IsCaughtUp: %v", isCaughtUp), "height", pool.height, "maxPeerHeight", maxPeerHeight)
pool.Logger.Info(cmn.Fmt("IsCaughtUp: %v", isCaughtUp), "height", pool.height, "maxPeerHeight", maxPeerHeight)
return isCaughtUp
}
@ -180,7 +180,7 @@ func (pool *BlockPool) PopRequest() {
delete(pool.requesters, pool.height)
pool.height++
} else {
PanicSanity(Fmt("Expected requester to pop, got nothing at height %v", pool.height))
cmn.PanicSanity(cmn.Fmt("Expected requester to pop, got nothing at height %v", pool.height))
}
}
@ -192,7 +192,7 @@ func (pool *BlockPool) RedoRequest(height int) {
pool.mtx.Unlock()
if request.block == nil {
PanicSanity("Expected block to be non-nil")
cmn.PanicSanity("Expected block to be non-nil")
}
// RemovePeer will redo all requesters associated with this peer.
// TODO: record this malfeasance
@ -311,10 +311,10 @@ func (pool *BlockPool) debug() string {
str := ""
for h := pool.height; h < pool.height+len(pool.requesters); h++ {
if pool.requesters[h] == nil {
str += Fmt("H(%v):X ", h)
str += cmn.Fmt("H(%v):X ", h)
} else {
str += Fmt("H(%v):", h)
str += Fmt("B?(%v) ", pool.requesters[h].block != nil)
str += cmn.Fmt("H(%v):", h)
str += cmn.Fmt("B?(%v) ", pool.requesters[h].block != nil)
}
}
return str
@ -394,7 +394,7 @@ func (peer *bpPeer) onTimeout() {
//-------------------------------------
type bpRequester struct {
BaseService
cmn.BaseService
pool *BlockPool
height int
gotBlockCh chan struct{}
@ -415,7 +415,7 @@ func newBPRequester(pool *BlockPool, height int) *bpRequester {
peerID: "",
block: nil,
}
bpr.BaseService = *NewBaseService(nil, "bpRequester", bpr)
bpr.BaseService = *cmn.NewBaseService(nil, "bpRequester", bpr)
return bpr
}


+ 2
- 2
blockchain/pool_test.go View File

@ -6,7 +6,7 @@ import (
"time"
"github.com/tendermint/tendermint/types"
. "github.com/tendermint/tmlibs/common"
cmn "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tmlibs/log"
)
@ -22,7 +22,7 @@ type testPeer struct {
func makePeers(numPeers int, minHeight, maxHeight int) map[string]testPeer {
peers := make(map[string]testPeer, numPeers)
for i := 0; i < numPeers; i++ {
peerID := RandStr(12)
peerID := cmn.RandStr(12)
height := minHeight + rand.Intn(maxHeight-minHeight)
peers[peerID] = testPeer{peerID, height}
}


+ 34
- 14
blockchain/reactor.go View File

@ -121,6 +121,24 @@ func (bcR *BlockchainReactor) RemovePeer(peer p2p.Peer, reason interface{}) {
bcR.pool.RemovePeer(peer.Key())
}
// respondToPeer loads a block and sends it to the requesting peer,
// if we have it. Otherwise, we'll respond saying we don't have it.
// According to the Tendermint spec, if all nodes are honest,
// no node should be requesting for a block that's non-existent.
func (bcR *BlockchainReactor) respondToPeer(msg *bcBlockRequestMessage, src p2p.Peer) (queued bool) {
block := bcR.store.LoadBlock(msg.Height)
if block != nil {
msg := &bcBlockResponseMessage{Block: block}
return src.TrySend(BlockchainChannel, struct{ BlockchainMessage }{msg})
}
bcR.Logger.Info("Peer asking for a block we don't have", "src", src, "height", msg.Height)
return src.TrySend(BlockchainChannel, struct{ BlockchainMessage }{
&bcNoBlockResponseMessage{Height: msg.Height},
})
}
// Receive implements Reactor by handling 4 types of messages (look below).
func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) {
_, msg, err := DecodeMessage(msgBytes, bcR.maxMsgSize())
@ -134,16 +152,8 @@ func (bcR *BlockchainReactor) Receive(chID byte, src p2p.Peer, msgBytes []byte)
// TODO: improve logic to satisfy megacheck
switch msg := msg.(type) {
case *bcBlockRequestMessage:
// Got a request for a block. Respond with block if we have it.
block := bcR.store.LoadBlock(msg.Height)
if block != nil {
msg := &bcBlockResponseMessage{Block: block}
queued := src.TrySend(BlockchainChannel, struct{ BlockchainMessage }{msg})
if !queued {
// queue is full, just ignore.
}
} else {
// TODO peer is asking for things we don't have.
if queued := bcR.respondToPeer(msg, src); !queued {
// Unfortunately not queued since the queue is full.
}
case *bcBlockResponseMessage:
// Got a block.
@ -276,10 +286,11 @@ func (bcR *BlockchainReactor) SetEventSwitch(evsw types.EventSwitch) {
// Messages
const (
msgTypeBlockRequest = byte(0x10)
msgTypeBlockResponse = byte(0x11)
msgTypeStatusResponse = byte(0x20)
msgTypeStatusRequest = byte(0x21)
msgTypeBlockRequest = byte(0x10)
msgTypeBlockResponse = byte(0x11)
msgTypeNoBlockResponse = byte(0x12)
msgTypeStatusResponse = byte(0x20)
msgTypeStatusRequest = byte(0x21)
)
// BlockchainMessage is a generic message for this reactor.
@ -289,6 +300,7 @@ var _ = wire.RegisterInterface(
struct{ BlockchainMessage }{},
wire.ConcreteType{&bcBlockRequestMessage{}, msgTypeBlockRequest},
wire.ConcreteType{&bcBlockResponseMessage{}, msgTypeBlockResponse},
wire.ConcreteType{&bcNoBlockResponseMessage{}, msgTypeNoBlockResponse},
wire.ConcreteType{&bcStatusResponseMessage{}, msgTypeStatusResponse},
wire.ConcreteType{&bcStatusRequestMessage{}, msgTypeStatusRequest},
)
@ -316,6 +328,14 @@ func (m *bcBlockRequestMessage) String() string {
return cmn.Fmt("[bcBlockRequestMessage %v]", m.Height)
}
type bcNoBlockResponseMessage struct {
Height int
}
func (brm *bcNoBlockResponseMessage) String() string {
return cmn.Fmt("[bcNoBlockResponseMessage %d]", brm.Height)
}
//-------------------------------------
// NOTE: keep up-to-date with maxBlockchainResponseSize


+ 158
- 0
blockchain/reactor_test.go View File

@ -0,0 +1,158 @@
package blockchain
import (
"bytes"
"testing"
wire "github.com/tendermint/go-wire"
cmn "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tmlibs/db"
"github.com/tendermint/tmlibs/log"
cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/p2p"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
)
func newBlockchainReactor(logger log.Logger, maxBlockHeight int) *BlockchainReactor {
config := cfg.ResetTestRoot("node_node_test")
blockStoreDB := db.NewDB("blockstore", config.DBBackend, config.DBDir())
blockStore := NewBlockStore(blockStoreDB)
stateLogger := logger.With("module", "state")
// Get State
stateDB := db.NewDB("state", config.DBBackend, config.DBDir())
state, _ := sm.GetState(stateDB, config.GenesisFile())
state.SetLogger(stateLogger)
state.Save()
// Make the blockchainReactor itself
fastSync := true
bcReactor := NewBlockchainReactor(state.Copy(), nil, blockStore, fastSync)
// Next: we need to set a switch in order for peers to be added in
bcReactor.Switch = p2p.NewSwitch(cfg.DefaultP2PConfig())
bcReactor.SetLogger(logger.With("module", "blockchain"))
// Lastly: let's add some blocks in
for blockHeight := 1; blockHeight <= maxBlockHeight; blockHeight++ {
firstBlock := makeBlock(blockHeight, state)
secondBlock := makeBlock(blockHeight+1, state)
firstParts := firstBlock.MakePartSet(state.Params().BlockGossipParams.BlockPartSizeBytes)
blockStore.SaveBlock(firstBlock, firstParts, secondBlock.LastCommit)
}
return bcReactor
}
func TestNoBlockMessageResponse(t *testing.T) {
logBuf := new(bytes.Buffer)
logger := log.NewTMLogger(logBuf)
maxBlockHeight := 20
bcr := newBlockchainReactor(logger, maxBlockHeight)
go bcr.OnStart()
defer bcr.Stop()
// Add some peers in
peer := newbcrTestPeer(cmn.RandStr(12))
bcr.AddPeer(peer)
chID := byte(0x01)
tests := []struct {
height int
existent bool
}{
{maxBlockHeight + 2, false},
{10, true},
{1, true},
{100, false},
}
for _, tt := range tests {
reqBlockMsg := &bcBlockRequestMessage{tt.height}
reqBlockBytes := wire.BinaryBytes(struct{ BlockchainMessage }{reqBlockMsg})
bcr.Receive(chID, peer, reqBlockBytes)
value := peer.lastValue()
msg := value.(struct{ BlockchainMessage }).BlockchainMessage
if tt.existent {
if blockMsg, ok := msg.(*bcBlockResponseMessage); !ok {
t.Fatalf("Expected to receive a block response for height %d", tt.height)
} else if blockMsg.Block.Height != tt.height {
t.Fatalf("Expected response to be for height %d, got %d", tt.height, blockMsg.Block.Height)
}
} else {
if noBlockMsg, ok := msg.(*bcNoBlockResponseMessage); !ok {
t.Fatalf("Expected to receive a no block response for height %d", tt.height)
} else if noBlockMsg.Height != tt.height {
t.Fatalf("Expected response to be for height %d, got %d", tt.height, noBlockMsg.Height)
}
}
}
}
//----------------------------------------------
// utility funcs
func makeTxs(blockNumber int) (txs []types.Tx) {
for i := 0; i < 10; i++ {
txs = append(txs, types.Tx([]byte{byte(blockNumber), byte(i)}))
}
return txs
}
func makeBlock(blockNumber int, state *sm.State) *types.Block {
prevHash := state.LastBlockID.Hash
prevParts := types.PartSetHeader{}
valHash := state.Validators.Hash()
prevBlockID := types.BlockID{prevHash, prevParts}
block, _ := types.MakeBlock(blockNumber, "test_chain", makeTxs(blockNumber),
new(types.Commit), prevBlockID, valHash, state.AppHash, state.Params().BlockGossipParams.BlockPartSizeBytes)
return block
}
// The Test peer
type bcrTestPeer struct {
cmn.Service
key string
ch chan interface{}
}
var _ p2p.Peer = (*bcrTestPeer)(nil)
func newbcrTestPeer(key string) *bcrTestPeer {
return &bcrTestPeer{
Service: cmn.NewBaseService(nil, "bcrTestPeer", nil),
key: key,
ch: make(chan interface{}, 2),
}
}
func (tp *bcrTestPeer) lastValue() interface{} { return <-tp.ch }
func (tp *bcrTestPeer) TrySend(chID byte, value interface{}) bool {
if _, ok := value.(struct{ BlockchainMessage }).BlockchainMessage.(*bcStatusResponseMessage); ok {
// Discard status response messages since they skew our results
// We only want to deal with:
// + bcBlockResponseMessage
// + bcNoBlockResponseMessage
} else {
tp.ch <- value
}
return true
}
func (tp *bcrTestPeer) Send(chID byte, data interface{}) bool { return tp.TrySend(chID, data) }
func (tp *bcrTestPeer) NodeInfo() *p2p.NodeInfo { return nil }
func (tp *bcrTestPeer) Status() p2p.ConnectionStatus { return p2p.ConnectionStatus{} }
func (tp *bcrTestPeer) Key() string { return tp.key }
func (tp *bcrTestPeer) IsOutbound() bool { return false }
func (tp *bcrTestPeer) IsPersistent() bool { return true }
func (tp *bcrTestPeer) Get(s string) interface{} { return s }
func (tp *bcrTestPeer) Set(string, interface{}) {}

+ 2
- 2
cmd/tendermint/commands/root_test.go View File

@ -3,15 +3,15 @@ package commands
import (
"os"
"strconv"
"testing"
"github.com/spf13/cobra"
"github.com/spf13/viper"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tmlibs/cli"
"testing"
)
var (


+ 5
- 5
consensus/byzantine_test.go View File

@ -9,7 +9,7 @@ import (
data "github.com/tendermint/go-wire/data"
"github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/types"
. "github.com/tendermint/tmlibs/common"
cmn "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tmlibs/events"
)
@ -149,8 +149,8 @@ func TestByzantine(t *testing.T) {
case <-done:
case <-tick.C:
for i, reactor := range reactors {
t.Log(Fmt("Consensus Reactor %v", i))
t.Log(Fmt("%v", reactor))
t.Log(cmn.Fmt("Consensus Reactor %v", i))
t.Log(cmn.Fmt("%v", reactor))
}
t.Fatalf("Timed out waiting for all validators to commit first block")
}
@ -220,7 +220,7 @@ func sendProposalAndParts(height, round int, cs *ConsensusState, peer p2p.Peer,
// byzantine consensus reactor
type ByzantineReactor struct {
Service
cmn.Service
reactor *ConsensusReactor
}
@ -296,5 +296,5 @@ func (privVal *ByzantinePrivValidator) SignHeartbeat(chainID string, heartbeat *
}
func (privVal *ByzantinePrivValidator) String() string {
return Fmt("PrivValidator{%X}", privVal.GetAddress())
return cmn.Fmt("PrivValidator{%X}", privVal.GetAddress())
}

+ 5
- 5
consensus/common_test.go View File

@ -19,7 +19,7 @@ import (
"github.com/tendermint/tendermint/p2p"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
. "github.com/tendermint/tmlibs/common"
cmn "github.com/tendermint/tmlibs/common"
dbm "github.com/tendermint/tmlibs/db"
"github.com/tendermint/tmlibs/log"
@ -34,7 +34,7 @@ var config *cfg.Config // NOTE: must be reset for each _test.go file
var ensureTimeout = time.Second * 2
func ensureDir(dir string, mode os.FileMode) {
if err := EnsureDir(dir, mode); err != nil {
if err := cmn.EnsureDir(dir, mode); err != nil {
panic(err)
}
}
@ -341,7 +341,7 @@ func randConsensusNet(nValidators int, testName string, tickerFunc func() Timeou
state, _ := sm.MakeGenesisState(db, genDoc)
state.SetLogger(logger.With("module", "state", "validator", i))
state.Save()
thisConfig := ResetConfig(Fmt("%s_%d", testName, i))
thisConfig := ResetConfig(cmn.Fmt("%s_%d", testName, i))
for _, opt := range configOpts {
opt(thisConfig)
}
@ -362,13 +362,13 @@ func randConsensusNetWithPeers(nValidators, nPeers int, testName string, tickerF
state, _ := sm.MakeGenesisState(db, genDoc)
state.SetLogger(log.TestingLogger().With("module", "state"))
state.Save()
thisConfig := ResetConfig(Fmt("%s_%d", testName, i))
thisConfig := ResetConfig(cmn.Fmt("%s_%d", testName, i))
ensureDir(path.Dir(thisConfig.Consensus.WalFile()), 0700) // dir for wal
var privVal types.PrivValidator
if i < nValidators {
privVal = privVals[i]
} else {
_, tempFilePath := Tempfile("priv_validator_")
_, tempFilePath := cmn.Tempfile("priv_validator_")
privVal = types.GenPrivValidatorFS(tempFilePath)
}


+ 5
- 5
consensus/height_vote_set.go View File

@ -5,7 +5,7 @@ import (
"sync"
"github.com/tendermint/tendermint/types"
. "github.com/tendermint/tmlibs/common"
cmn "github.com/tendermint/tmlibs/common"
)
type RoundVoteSet struct {
@ -76,7 +76,7 @@ func (hvs *HeightVoteSet) SetRound(round int) {
hvs.mtx.Lock()
defer hvs.mtx.Unlock()
if hvs.round != 0 && (round < hvs.round+1) {
PanicSanity("SetRound() must increment hvs.round")
cmn.PanicSanity("SetRound() must increment hvs.round")
}
for r := hvs.round + 1; r <= round; r++ {
if _, ok := hvs.roundVoteSets[r]; ok {
@ -89,7 +89,7 @@ func (hvs *HeightVoteSet) SetRound(round int) {
func (hvs *HeightVoteSet) addRound(round int) {
if _, ok := hvs.roundVoteSets[round]; ok {
PanicSanity("addRound() for an existing round")
cmn.PanicSanity("addRound() for an existing round")
}
// log.Debug("addRound(round)", "round", round)
prevotes := types.NewVoteSet(hvs.chainID, hvs.height, round, types.VoteTypePrevote, hvs.valSet)
@ -164,7 +164,7 @@ func (hvs *HeightVoteSet) getVoteSet(round int, type_ byte) *types.VoteSet {
case types.VoteTypePrecommit:
return rvs.Precommits
default:
PanicSanity(Fmt("Unexpected vote type %X", type_))
cmn.PanicSanity(cmn.Fmt("Unexpected vote type %X", type_))
return nil
}
}
@ -194,7 +194,7 @@ func (hvs *HeightVoteSet) StringIndented(indent string) string {
voteSetString = roundVoteSet.Precommits.StringShort()
vsStrings = append(vsStrings, voteSetString)
}
return Fmt(`HeightVoteSet{H:%v R:0~%v
return cmn.Fmt(`HeightVoteSet{H:%v R:0~%v
%s %v
%s}`,
hvs.height, hvs.round,


+ 2
- 2
consensus/height_vote_set_test.go View File

@ -4,7 +4,7 @@ import (
"testing"
"github.com/tendermint/tendermint/types"
. "github.com/tendermint/tmlibs/common"
cmn "github.com/tendermint/tmlibs/common"
)
func init() {
@ -57,7 +57,7 @@ func makeVoteHR(t *testing.T, height, round int, privVals []*types.PrivValidator
chainID := config.ChainID
err := privVal.SignVote(chainID, vote)
if err != nil {
panic(Fmt("Error signing vote: %v", err))
panic(cmn.Fmt("Error signing vote: %v", err))
return nil
}
return vote


+ 4
- 4
consensus/mempool_test.go View File

@ -8,7 +8,7 @@ import (
abci "github.com/tendermint/abci/types"
"github.com/tendermint/tendermint/types"
. "github.com/tendermint/tmlibs/common"
cmn "github.com/tendermint/tmlibs/common"
)
func init() {
@ -86,7 +86,7 @@ func deliverTxsRange(cs *ConsensusState, start, end int) {
binary.BigEndian.PutUint64(txBytes, uint64(i))
err := cs.mempool.CheckTx(txBytes, nil)
if err != nil {
panic(Fmt("Error after CheckTx: %v", err))
panic(cmn.Fmt("Error after CheckTx: %v", err))
}
}
}
@ -184,7 +184,7 @@ func NewCounterApplication() *CounterApplication {
}
func (app *CounterApplication) Info(req abci.RequestInfo) abci.ResponseInfo {
return abci.ResponseInfo{Data: Fmt("txs:%v", app.txCount)}
return abci.ResponseInfo{Data: cmn.Fmt("txs:%v", app.txCount)}
}
func (app *CounterApplication) DeliverTx(tx []byte) abci.Result {
@ -201,7 +201,7 @@ func runTx(tx []byte, countPtr *int) abci.Result {
copy(tx8[len(tx8)-len(tx):], tx)
txValue := binary.BigEndian.Uint64(tx8)
if txValue != uint64(count) {
return abci.ErrBadNonce.AppendLog(Fmt("Invalid nonce. Expected %v, got %v", count, txValue))
return abci.ErrBadNonce.AppendLog(cmn.Fmt("Invalid nonce. Expected %v, got %v", count, txValue))
}
*countPtr += 1
return abci.OK


+ 4
- 4
consensus/state_test.go View File

@ -7,7 +7,7 @@ import (
"time"
"github.com/tendermint/tendermint/types"
. "github.com/tendermint/tmlibs/common"
cmn "github.com/tendermint/tmlibs/common"
)
func init() {
@ -80,7 +80,7 @@ func TestProposerSelection0(t *testing.T) {
prop = cs1.GetRoundState().Validators.GetProposer()
if !bytes.Equal(prop.Address, vss[1].GetAddress()) {
panic(Fmt("expected proposer to be validator %d. Got %X", 1, prop.Address))
panic(cmn.Fmt("expected proposer to be validator %d. Got %X", 1, prop.Address))
}
}
@ -101,7 +101,7 @@ func TestProposerSelection2(t *testing.T) {
for i := 0; i < len(vss); i++ {
prop := cs1.GetRoundState().Validators.GetProposer()
if !bytes.Equal(prop.Address, vss[(i+2)%len(vss)].GetAddress()) {
panic(Fmt("expected proposer to be validator %d. Got %X", (i+2)%len(vss), prop.Address))
panic(cmn.Fmt("expected proposer to be validator %d. Got %X", (i+2)%len(vss), prop.Address))
}
rs := cs1.GetRoundState()
@ -432,7 +432,7 @@ func TestLockNoPOL(t *testing.T) {
// now we're on a new round and are the proposer
if !bytes.Equal(rs.ProposalBlock.Hash(), rs.LockedBlock.Hash()) {
panic(Fmt("Expected proposal block to be locked block. Got %v, Expected %v", rs.ProposalBlock, rs.LockedBlock))
panic(cmn.Fmt("Expected proposal block to be locked block. Got %v, Expected %v", rs.ProposalBlock, rs.LockedBlock))
}
<-voteCh // prevote


+ 2
- 2
consensus/version.go View File

@ -1,7 +1,7 @@
package consensus
import (
. "github.com/tendermint/tmlibs/common"
cmn "github.com/tendermint/tmlibs/common"
)
// kind of arbitrary
@ -10,4 +10,4 @@ var Major = "0" //
var Minor = "2" // replay refactor
var Revision = "2" // validation -> commit
var Version = Fmt("v%s/%s.%s.%s", Spec, Major, Minor, Revision)
var Version = cmn.Fmt("v%s/%s.%s.%s", Spec, Major, Minor, Revision)

+ 7
- 7
consensus/wal.go View File

@ -6,7 +6,7 @@ import (
wire "github.com/tendermint/go-wire"
"github.com/tendermint/tendermint/types"
auto "github.com/tendermint/tmlibs/autofile"
. "github.com/tendermint/tmlibs/common"
cmn "github.com/tendermint/tmlibs/common"
)
//--------------------------------------------------------
@ -34,7 +34,7 @@ var _ = wire.RegisterInterface(
// TODO: currently the wal is overwritten during replay catchup
// give it a mode so it's either reading or appending - must read to end to start appending again
type WAL struct {
BaseService
cmn.BaseService
group *auto.Group
light bool // ignore block parts
@ -49,7 +49,7 @@ func NewWAL(walFile string, light bool) (*WAL, error) {
group: group,
light: light,
}
wal.BaseService = *NewBaseService(nil, "WAL", wal)
wal.BaseService = *cmn.NewBaseService(nil, "WAL", wal)
return wal, nil
}
@ -86,19 +86,19 @@ func (wal *WAL) Save(wmsg WALMessage) {
var wmsgBytes = wire.JSONBytes(TimedWALMessage{time.Now(), wmsg})
err := wal.group.WriteLine(string(wmsgBytes))
if err != nil {
PanicQ(Fmt("Error writing msg to consensus wal. Error: %v \n\nMessage: %v", err, wmsg))
cmn.PanicQ(cmn.Fmt("Error writing msg to consensus wal. Error: %v \n\nMessage: %v", err, wmsg))
}
// TODO: only flush when necessary
if err := wal.group.Flush(); err != nil {
PanicQ(Fmt("Error flushing consensus wal buf to file. Error: %v \n", err))
cmn.PanicQ(cmn.Fmt("Error flushing consensus wal buf to file. Error: %v \n", err))
}
}
func (wal *WAL) writeEndHeight(height int) {
wal.group.WriteLine(Fmt("#ENDHEIGHT: %v", height))
wal.group.WriteLine(cmn.Fmt("#ENDHEIGHT: %v", height))
// TODO: only flush when necessary
if err := wal.group.Flush(); err != nil {
PanicQ(Fmt("Error flushing consensus wal buf to file. Error: %v \n", err))
cmn.PanicQ(cmn.Fmt("Error flushing consensus wal buf to file. Error: %v \n", err))
}
}

+ 2
- 1
node/id.go View File

@ -1,8 +1,9 @@
package node
import (
"github.com/tendermint/go-crypto"
"time"
"github.com/tendermint/go-crypto"
)
type NodeID struct {


+ 1
- 1
p2p/listener.go View File

@ -196,7 +196,7 @@ func getUPNPExternalAddress(externalPort, internalPort int, logger log.Logger) *
return NewNetAddressIPPort(ext, uint16(externalPort))
}
// TODO: use syscalls: http://pastebin.com/9exZG4rh
// TODO: use syscalls: see issue #712
func getNaiveExternalAddress(port int, settleForLocal bool, logger log.Logger) *NetAddress {
addrs, err := net.InterfaceAddrs()
if err != nil {


+ 1
- 0
p2p/peer.go View File

@ -7,6 +7,7 @@ import (
"time"
"github.com/pkg/errors"
crypto "github.com/tendermint/go-crypto"
wire "github.com/tendermint/go-wire"
cmn "github.com/tendermint/tmlibs/common"


+ 0
- 5
p2p/upnp/README.md View File

@ -1,5 +0,0 @@
# `tendermint/p2p/upnp`
## Resources
* http://www.upnp-hacks.org/upnp.html

+ 4
- 6
p2p/upnp/upnp.go View File

@ -1,11 +1,9 @@
/*
Taken from taipei-torrent
Just enough UPnP to be able to forward ports
*/
// Taken from taipei-torrent.
// Just enough UPnP to be able to forward ports
// For more information, see: http://www.upnp-hacks.org/upnp.html
package upnp
// BUG(jae): TODO: use syscalls to get actual ourIP. http://pastebin.com/9exZG4rh
// TODO: use syscalls to get actual ourIP, see issue #712
import (
"bytes"


+ 1
- 0
rpc/client/httpclient.go View File

@ -6,6 +6,7 @@ import (
"fmt"
"github.com/pkg/errors"
data "github.com/tendermint/go-wire/data"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
rpcclient "github.com/tendermint/tendermint/rpc/lib/client"


+ 2
- 2
rpc/client/mock/abci_test.go View File

@ -8,13 +8,13 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/tendermint/abci/example/dummy"
abci "github.com/tendermint/abci/types"
data "github.com/tendermint/go-wire/data"
"github.com/tendermint/tendermint/rpc/client/mock"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
"github.com/tendermint/tendermint/types"
"github.com/tendermint/tendermint/rpc/client/mock"
)
func TestABCIMock(t *testing.T) {


+ 2
- 2
rpc/client/mock/status_test.go View File

@ -5,10 +5,10 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
data "github.com/tendermint/go-wire/data"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
data "github.com/tendermint/go-wire/data"
"github.com/tendermint/tendermint/rpc/client/mock"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
)
func TestStatus(t *testing.T) {


+ 2
- 1
rpc/client/rpc_test.go View File

@ -6,7 +6,8 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/tendermint/merkleeyes/iavl"
"github.com/tendermint/merkleeyes/iavl" //TODO use tendermint/iavl ?
"github.com/tendermint/tendermint/rpc/client"
rpctest "github.com/tendermint/tendermint/rpc/test"
"github.com/tendermint/tendermint/types"


+ 4
- 4
rpc/core/blocks.go View File

@ -5,7 +5,7 @@ import (
ctypes "github.com/tendermint/tendermint/rpc/core/types"
"github.com/tendermint/tendermint/types"
. "github.com/tendermint/tmlibs/common"
cmn "github.com/tendermint/tmlibs/common"
)
// Get block headers for minHeight <= height <= maxHeight.
@ -65,12 +65,12 @@ func BlockchainInfo(minHeight, maxHeight int) (*ctypes.ResultBlockchainInfo, err
if maxHeight == 0 {
maxHeight = blockStore.Height()
} else {
maxHeight = MinInt(blockStore.Height(), maxHeight)
maxHeight = cmn.MinInt(blockStore.Height(), maxHeight)
}
if minHeight == 0 {
minHeight = MaxInt(1, maxHeight-20)
minHeight = cmn.MaxInt(1, maxHeight-20)
} else {
minHeight = MaxInt(minHeight, maxHeight-20)
minHeight = cmn.MaxInt(minHeight, maxHeight-20)
}
logger.Debug("BlockchainInfoHandler", "maxHeight", maxHeight, "minHeight", minHeight)


+ 1
- 2
rpc/core/pipe.go View File

@ -2,14 +2,13 @@ package core
import (
crypto "github.com/tendermint/go-crypto"
"github.com/tendermint/tmlibs/log"
"github.com/tendermint/tendermint/consensus"
p2p "github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/proxy"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/state/txindex"
"github.com/tendermint/tendermint/types"
"github.com/tendermint/tmlibs/log"
)
//----------------------------------------------


+ 0
- 1
rpc/core/types/responses.go View File

@ -6,7 +6,6 @@ import (
abci "github.com/tendermint/abci/types"
"github.com/tendermint/go-crypto"
"github.com/tendermint/go-wire/data"
"github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/types"
)


+ 1
- 0
rpc/core/types/responses_test.go View File

@ -4,6 +4,7 @@ import (
"testing"
"github.com/stretchr/testify/assert"
"github.com/tendermint/tendermint/p2p"
)


+ 2
- 3
rpc/grpc/api.go View File

@ -1,11 +1,10 @@
package core_grpc
import (
core "github.com/tendermint/tendermint/rpc/core"
context "golang.org/x/net/context"
abci "github.com/tendermint/abci/types"
context "golang.org/x/net/context"
core "github.com/tendermint/tendermint/rpc/core"
)
type broadcastAPI struct {


+ 2
- 2
rpc/grpc/client_server.go View File

@ -8,7 +8,7 @@ import (
"google.golang.org/grpc"
. "github.com/tendermint/tmlibs/common"
cmn "github.com/tendermint/tmlibs/common"
)
// Start the grpcServer in a go routine
@ -40,5 +40,5 @@ func StartGRPCClient(protoAddr string) BroadcastAPIClient {
}
func dialerFunc(addr string, timeout time.Duration) (net.Conn, error) {
return Connect(addr)
return cmn.Connect(addr)
}

+ 0
- 12
rpc/lib/Dockerfile View File

@ -1,12 +0,0 @@
FROM golang:latest
RUN mkdir -p /go/src/github.com/tendermint/tendermint/rpc/lib
WORKDIR /go/src/github.com/tendermint/tendermint/rpc/lib
COPY Makefile /go/src/github.com/tendermint/tendermint/rpc/lib/
# COPY glide.yaml /go/src/github.com/tendermint/tendermint/rpc/lib/
# COPY glide.lock /go/src/github.com/tendermint/tendermint/rpc/lib/
COPY . /go/src/github.com/tendermint/tendermint/rpc/lib
RUN make get_deps

+ 0
- 18
rpc/lib/Makefile View File

@ -1,18 +0,0 @@
PACKAGES=$(shell go list ./... | grep -v "test")
all: get_deps test
test:
@echo "--> Running go test --race"
@go test --race $(PACKAGES)
@echo "--> Running integration tests"
@bash ./test/integration_test.sh
get_deps:
@echo "--> Running go get"
@go get -v -d $(PACKAGES)
@go list -f '{{join .TestImports "\n"}}' ./... | \
grep -v /vendor/ | sort | uniq | \
xargs go get -v -d
.PHONY: all test get_deps

+ 0
- 121
rpc/lib/README.md View File

@ -1,121 +0,0 @@
# tendermint/rpc/lib
[![CircleCI](https://circleci.com/gh/tendermint/tendermint/rpc/lib.svg?style=svg)](https://circleci.com/gh/tendermint/tendermint/rpc/lib)
HTTP RPC server supporting calls via uri params, jsonrpc, and jsonrpc over websockets
# Client Requests
Suppose we want to expose the rpc function `HelloWorld(name string, num int)`.
## GET (URI)
As a GET request, it would have URI encoded parameters, and look like:
```
curl 'http://localhost:8008/hello_world?name="my_world"&num=5'
```
Note the `'` around the url, which is just so bash doesn't ignore the quotes in `"my_world"`.
This should also work:
```
curl http://localhost:8008/hello_world?name=\"my_world\"&num=5
```
A GET request to `/` returns a list of available endpoints.
For those which take arguments, the arguments will be listed in order, with `_` where the actual value should be.
## POST (JSONRPC)
As a POST request, we use JSONRPC. For instance, the same request would have this as the body:
```
{
"jsonrpc": "2.0",
"id": "anything",
"method": "hello_world",
"params": {
"name": "my_world",
"num": 5
}
}
```
With the above saved in file `data.json`, we can make the request with
```
curl --data @data.json http://localhost:8008
```
## WebSocket (JSONRPC)
All requests are exposed over websocket in the same form as the POST JSONRPC.
Websocket connections are available at their own endpoint, typically `/websocket`,
though this is configurable when starting the server.
# Server Definition
Define some types and routes:
```
type ResultStatus struct {
Value string
}
// Define some routes
var Routes = map[string]*rpcserver.RPCFunc{
"status": rpcserver.NewRPCFunc(Status, "arg"),
}
// an rpc function
func Status(v string) (*ResultStatus, error) {
return &ResultStatus{v}, nil
}
```
Now start the server:
```
mux := http.NewServeMux()
rpcserver.RegisterRPCFuncs(mux, Routes)
wm := rpcserver.NewWebsocketManager(Routes, nil)
mux.HandleFunc("/websocket", wm.WebsocketHandler)
logger := log.NewTMLogger(log.NewSyncWriter(os.Stdout))
go func() {
_, err := rpcserver.StartHTTPServer("0.0.0.0:8008", mux, logger)
if err != nil {
panic(err)
}
}()
```
Note that unix sockets are supported as well (eg. `/path/to/socket` instead of `0.0.0.0:8008`)
Now see all available endpoints by sending a GET request to `0.0.0.0:8008`.
Each route is available as a GET request, as a JSONRPCv2 POST request, and via JSONRPCv2 over websockets.
# Examples
* [Tendermint](https://github.com/tendermint/tendermint/blob/master/rpc/core/routes.go)
* [tm-monitor](https://github.com/tendermint/tools/blob/master/tm-monitor/rpc.go)
## CHANGELOG
### 0.7.0
BREAKING CHANGES:
- removed `Client` empty interface
- `ClientJSONRPC#Call` `params` argument became a map
- rename `ClientURI` -> `URIClient`, `ClientJSONRPC` -> `JSONRPCClient`
IMPROVEMENTS:
- added `HTTPClient` interface, which can be used for both `ClientURI`
and `ClientJSONRPC`
- all params are now optional (Golang's default will be used if some param is missing)
- added `Call` method to `WSClient` (see method's doc for details)

+ 0
- 21
rpc/lib/circle.yml View File

@ -1,21 +0,0 @@
machine:
environment:
GOPATH: /home/ubuntu/.go_workspace
REPO: $GOPATH/src/github.com/$CIRCLE_PROJECT_USERNAME/$CIRCLE_PROJECT_REPONAME
hosts:
circlehost: 127.0.0.1
localhost: 127.0.0.1
checkout:
post:
- rm -rf $REPO
- mkdir -p $HOME/.go_workspace/src/github.com/$CIRCLE_PROJECT_USERNAME
- mv $HOME/$CIRCLE_PROJECT_REPONAME $REPO
dependencies:
override:
- "cd $REPO && make get_deps"
test:
override:
- "cd $REPO && make test"

+ 1
- 0
rpc/lib/rpc_test.go View File

@ -16,6 +16,7 @@ import (
"github.com/go-kit/kit/log/term"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/tendermint/go-wire/data"
client "github.com/tendermint/tendermint/rpc/lib/client"
server "github.com/tendermint/tendermint/rpc/lib/server"


+ 1
- 0
state/execution_test.go View File

@ -5,6 +5,7 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"github.com/tendermint/abci/example/dummy"
crypto "github.com/tendermint/go-crypto"
"github.com/tendermint/tendermint/proxy"


+ 3
- 5
state/state_test.go View File

@ -7,16 +7,14 @@ import (
"github.com/stretchr/testify/assert"
abci "github.com/tendermint/abci/types"
cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/types"
abci "github.com/tendermint/abci/types"
crypto "github.com/tendermint/go-crypto"
cmn "github.com/tendermint/tmlibs/common"
dbm "github.com/tendermint/tmlibs/db"
"github.com/tendermint/tmlibs/log"
cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/types"
)
// setupTestCase does setup common to all test cases


+ 1
- 0
types/genesis_test.go View File

@ -5,6 +5,7 @@ import (
"testing"
"github.com/stretchr/testify/assert"
crypto "github.com/tendermint/go-crypto"
)


+ 3
- 3
types/part_set_test.go View File

@ -5,7 +5,7 @@ import (
"io/ioutil"
"testing"
. "github.com/tendermint/tmlibs/common"
cmn "github.com/tendermint/tmlibs/common"
)
const (
@ -15,7 +15,7 @@ const (
func TestBasicPartSet(t *testing.T) {
// Construct random data of size partSize * 100
data := RandBytes(testPartSize * 100)
data := cmn.RandBytes(testPartSize * 100)
partSet := NewPartSetFromData(data, testPartSize)
if len(partSet.Hash()) == 0 {
@ -65,7 +65,7 @@ func TestBasicPartSet(t *testing.T) {
func TestWrongProof(t *testing.T) {
// Construct random data of size partSize * 100
data := RandBytes(testPartSize * 100)
data := cmn.RandBytes(testPartSize * 100)
partSet := NewPartSetFromData(data, testPartSize)
// Test adding a part with wrong data.


+ 0
- 1
types/proposal.go View File

@ -5,7 +5,6 @@ import (
"fmt"
"io"
//. "github.com/tendermint/tmlibs/common"
"github.com/tendermint/go-crypto"
"github.com/tendermint/go-wire"
)


+ 2
- 2
types/signable.go View File

@ -4,7 +4,7 @@ import (
"bytes"
"io"
. "github.com/tendermint/tmlibs/common"
cmn "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tmlibs/merkle"
)
@ -19,7 +19,7 @@ func SignBytes(chainID string, o Signable) []byte {
buf, n, err := new(bytes.Buffer), new(int), new(error)
o.WriteSignBytes(chainID, buf, n, err)
if *err != nil {
PanicCrisis(err)
cmn.PanicCrisis(err)
}
return buf.Bytes()
}


+ 1
- 0
types/tx_test.go View File

@ -5,6 +5,7 @@ import (
"testing"
"github.com/stretchr/testify/assert"
wire "github.com/tendermint/go-wire"
cmn "github.com/tendermint/tmlibs/common"
ctest "github.com/tendermint/tmlibs/test"


+ 19
- 19
types/vote_set.go View File

@ -6,7 +6,7 @@ import (
"strings"
"sync"
. "github.com/tendermint/tmlibs/common"
cmn "github.com/tendermint/tmlibs/common"
)
/*
@ -51,7 +51,7 @@ type VoteSet struct {
mtx sync.Mutex
valSet *ValidatorSet
votesBitArray *BitArray
votesBitArray *cmn.BitArray
votes []*Vote // Primary votes to share
sum int64 // Sum of voting power for seen votes, discounting conflicts
maj23 *BlockID // First 2/3 majority seen
@ -62,7 +62,7 @@ type VoteSet struct {
// Constructs a new VoteSet struct used to accumulate votes for given height/round.
func NewVoteSet(chainID string, height int, round int, type_ byte, valSet *ValidatorSet) *VoteSet {
if height == 0 {
PanicSanity("Cannot make VoteSet for height == 0, doesn't make sense.")
cmn.PanicSanity("Cannot make VoteSet for height == 0, doesn't make sense.")
}
return &VoteSet{
chainID: chainID,
@ -70,7 +70,7 @@ func NewVoteSet(chainID string, height int, round int, type_ byte, valSet *Valid
round: round,
type_: type_,
valSet: valSet,
votesBitArray: NewBitArray(valSet.Size()),
votesBitArray: cmn.NewBitArray(valSet.Size()),
votes: make([]*Vote, valSet.Size()),
sum: 0,
maj23: nil,
@ -125,7 +125,7 @@ func (voteSet *VoteSet) Size() int {
// NOTE: VoteSet must not be nil
func (voteSet *VoteSet) AddVote(vote *Vote) (added bool, err error) {
if voteSet == nil {
PanicSanity("AddVote() on nil VoteSet")
cmn.PanicSanity("AddVote() on nil VoteSet")
}
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
@ -188,7 +188,7 @@ func (voteSet *VoteSet) addVote(vote *Vote) (added bool, err error) {
}
} else {
if !added {
PanicSanity("Expected to add non-conflicting vote")
cmn.PanicSanity("Expected to add non-conflicting vote")
}
return added, nil
}
@ -214,7 +214,7 @@ func (voteSet *VoteSet) addVerifiedVote(vote *Vote, blockKey string, votingPower
// Already exists in voteSet.votes?
if existing := voteSet.votes[valIndex]; existing != nil {
if existing.BlockID.Equals(vote.BlockID) {
PanicSanity("addVerifiedVote does not expect duplicate votes")
cmn.PanicSanity("addVerifiedVote does not expect duplicate votes")
} else {
conflicting = existing
}
@ -285,7 +285,7 @@ func (voteSet *VoteSet) addVerifiedVote(vote *Vote, blockKey string, votingPower
// NOTE: VoteSet must not be nil
func (voteSet *VoteSet) SetPeerMaj23(peerID string, blockID BlockID) {
if voteSet == nil {
PanicSanity("SetPeerMaj23() on nil VoteSet")
cmn.PanicSanity("SetPeerMaj23() on nil VoteSet")
}
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
@ -318,7 +318,7 @@ func (voteSet *VoteSet) SetPeerMaj23(peerID string, blockID BlockID) {
}
}
func (voteSet *VoteSet) BitArray() *BitArray {
func (voteSet *VoteSet) BitArray() *cmn.BitArray {
if voteSet == nil {
return nil
}
@ -327,7 +327,7 @@ func (voteSet *VoteSet) BitArray() *BitArray {
return voteSet.votesBitArray.Copy()
}
func (voteSet *VoteSet) BitArrayByBlockID(blockID BlockID) *BitArray {
func (voteSet *VoteSet) BitArrayByBlockID(blockID BlockID) *cmn.BitArray {
if voteSet == nil {
return nil
}
@ -358,7 +358,7 @@ func (voteSet *VoteSet) GetByAddress(address []byte) *Vote {
defer voteSet.mtx.Unlock()
valIndex, val := voteSet.valSet.GetByAddress(address)
if val == nil {
PanicSanity("GetByAddress(address) returned nil")
cmn.PanicSanity("GetByAddress(address) returned nil")
}
return voteSet.votes[valIndex]
}
@ -456,14 +456,14 @@ func (voteSet *VoteSet) StringShort() string {
func (voteSet *VoteSet) MakeCommit() *Commit {
if voteSet.type_ != VoteTypePrecommit {
PanicSanity("Cannot MakeCommit() unless VoteSet.Type is VoteTypePrecommit")
cmn.PanicSanity("Cannot MakeCommit() unless VoteSet.Type is VoteTypePrecommit")
}
voteSet.mtx.Lock()
defer voteSet.mtx.Unlock()
// Make sure we have a 2/3 majority
if voteSet.maj23 == nil {
PanicSanity("Cannot MakeCommit() unless a blockhash has +2/3")
cmn.PanicSanity("Cannot MakeCommit() unless a blockhash has +2/3")
}
// For every validator, get the precommit
@ -484,16 +484,16 @@ func (voteSet *VoteSet) MakeCommit() *Commit {
2. A peer claims to have a 2/3 majority w/ blockKey (peerMaj23=true)
*/
type blockVotes struct {
peerMaj23 bool // peer claims to have maj23
bitArray *BitArray // valIndex -> hasVote?
votes []*Vote // valIndex -> *Vote
sum int64 // vote sum
peerMaj23 bool // peer claims to have maj23
bitArray *cmn.BitArray // valIndex -> hasVote?
votes []*Vote // valIndex -> *Vote
sum int64 // vote sum
}
func newBlockVotes(peerMaj23 bool, numValidators int) *blockVotes {
return &blockVotes{
peerMaj23: peerMaj23,
bitArray: NewBitArray(numValidators),
bitArray: cmn.NewBitArray(numValidators),
votes: make([]*Vote, numValidators),
sum: 0,
}
@ -523,7 +523,7 @@ type VoteSetReader interface {
Round() int
Type() byte
Size() int
BitArray() *BitArray
BitArray() *cmn.BitArray
GetByIndex(int) *Vote
IsCommit() bool
}

+ 11
- 12
types/vote_set_test.go View File

@ -2,12 +2,11 @@ package types
import (
"bytes"
"testing"
"github.com/tendermint/go-crypto"
. "github.com/tendermint/tmlibs/common"
. "github.com/tendermint/tmlibs/test"
"testing"
cmn "github.com/tendermint/tmlibs/common"
tst "github.com/tendermint/tmlibs/test"
)
// NOTE: privValidators are in order
@ -137,7 +136,7 @@ func Test2_3Majority(t *testing.T) {
// 7th validator voted for some blockhash
{
vote := withValidator(voteProto, privValidators[6].GetAddress(), 6)
signAddVote(privValidators[6], withBlockHash(vote, RandBytes(32)), voteSet)
signAddVote(privValidators[6], withBlockHash(vote, cmn.RandBytes(32)), voteSet)
blockID, ok = voteSet.TwoThirdsMajority()
if ok || !blockID.IsZero() {
t.Errorf("There should be no 2/3 majority")
@ -217,7 +216,7 @@ func Test2_3MajorityRedux(t *testing.T) {
// 70th validator voted for different BlockHash
{
vote := withValidator(voteProto, privValidators[69].GetAddress(), 69)
signAddVote(privValidators[69], withBlockHash(vote, RandBytes(32)), voteSet)
signAddVote(privValidators[69], withBlockHash(vote, cmn.RandBytes(32)), voteSet)
blockID, ok = voteSet.TwoThirdsMajority()
if ok || !blockID.IsZero() {
t.Errorf("There should be no 2/3 majority: last vote added had different BlockHash")
@ -260,7 +259,7 @@ func TestBadVotes(t *testing.T) {
// val0 votes again for some block.
{
vote := withValidator(voteProto, privValidators[0].GetAddress(), 0)
added, err := signAddVote(privValidators[0], withBlockHash(vote, RandBytes(32)), voteSet)
added, err := signAddVote(privValidators[0], withBlockHash(vote, cmn.RandBytes(32)), voteSet)
if added || err == nil {
t.Errorf("Expected VoteSet.Add to fail, conflicting vote.")
}
@ -297,8 +296,8 @@ func TestBadVotes(t *testing.T) {
func TestConflicts(t *testing.T) {
height, round := 1, 0
voteSet, _, privValidators := randVoteSet(height, round, VoteTypePrevote, 4, 1)
blockHash1 := RandBytes(32)
blockHash2 := RandBytes(32)
blockHash1 := cmn.RandBytes(32)
blockHash2 := cmn.RandBytes(32)
voteProto := &Vote{
ValidatorAddress: nil,
@ -444,13 +443,13 @@ func TestMakeCommit(t *testing.T) {
}
// MakeCommit should fail.
AssertPanics(t, "Doesn't have +2/3 majority", func() { voteSet.MakeCommit() })
tst.AssertPanics(t, "Doesn't have +2/3 majority", func() { voteSet.MakeCommit() })
// 7th voted for some other block.
{
vote := withValidator(voteProto, privValidators[6].GetAddress(), 6)
vote = withBlockHash(vote, RandBytes(32))
vote = withBlockPartsHeader(vote, PartSetHeader{123, RandBytes(32)})
vote = withBlockHash(vote, cmn.RandBytes(32))
vote = withBlockPartsHeader(vote, PartSetHeader{123, cmn.RandBytes(32)})
signAddVote(privValidators[6], vote, voteSet)
}


Loading…
Cancel
Save