From ca674304c5316785fc9cdc24c73f66163f7f7966 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Fri, 24 Jun 2016 20:21:44 -0400 Subject: [PATCH 01/38] fix blockpool races. closes #188 --- Makefile | 3 +++ blockchain/pool.go | 60 +++++++++++++++++++++++++++---------------- blockchain/reactor.go | 2 +- 3 files changed, 42 insertions(+), 23 deletions(-) diff --git a/Makefile b/Makefile index d9b6fdf97..ac1956dde 100644 --- a/Makefile +++ b/Makefile @@ -20,6 +20,9 @@ build_race: test: build go test `${NOVENDOR}` + +test_race: build + go test -race `${NOVENDOR}` test100: build for i in {1..100}; do make test; done diff --git a/blockchain/pool.go b/blockchain/pool.go index f3bf5d204..3c06d669b 100644 --- a/blockchain/pool.go +++ b/blockchain/pool.go @@ -81,13 +81,13 @@ func (pool *BlockPool) makeRequestersRoutine() { if !pool.IsRunning() { break } - _, numPending := pool.GetStatus() + _, numPending, lenRequesters := pool.GetStatus() if numPending >= maxPendingRequests { // sleep for a bit. time.Sleep(requestIntervalMS * time.Millisecond) // check for timed out peers pool.removeTimedoutPeers() - } else if len(pool.requesters) >= maxTotalRequesters { + } else if lenRequesters >= maxTotalRequesters { // sleep for a bit. time.Sleep(requestIntervalMS * time.Millisecond) // check for timed out peers @@ -100,6 +100,8 @@ func (pool *BlockPool) makeRequestersRoutine() { } func (pool *BlockPool) removeTimedoutPeers() { + pool.peersMtx.Lock() // Lock + defer pool.peersMtx.Unlock() for _, peer := range pool.peers { if !peer.didTimeout && peer.numPending > 0 { curRate := peer.recvMonitor.Status().CurRate @@ -111,25 +113,24 @@ func (pool *BlockPool) removeTimedoutPeers() { } } if peer.didTimeout { - pool.peersMtx.Lock() // Lock pool.removePeer(peer.id) - pool.peersMtx.Unlock() } } } -func (pool *BlockPool) GetStatus() (height int, numPending int32) { +func (pool *BlockPool) GetStatus() (height int, numPending int32, lenRequesters int) { pool.mtx.Lock() // Lock defer pool.mtx.Unlock() - return pool.height, pool.numPending + return pool.height, pool.numPending, len(pool.requesters) } // TODO: relax conditions, prevent abuse. func (pool *BlockPool) IsCaughtUp() bool { - pool.mtx.Lock() - height := pool.height - pool.mtx.Unlock() + height, _, _ := pool.GetStatus() + + pool.peersMtx.Lock() + defer pool.peersMtx.Unlock() // Need at least 1 peer to be considered caught up. if len(pool.peers) == 0 { @@ -137,12 +138,10 @@ func (pool *BlockPool) IsCaughtUp() bool { return false } - pool.peersMtx.Lock() maxPeerHeight := 0 for _, peer := range pool.peers { maxPeerHeight = MaxInt(maxPeerHeight, peer.height) } - pool.peersMtx.Unlock() isCaughtUp := (height > 0 || time.Now().Sub(pool.startTime) > 5*time.Second) && (maxPeerHeight == 0 || height >= maxPeerHeight) log.Notice(Fmt("IsCaughtUp: %v", isCaughtUp), "height", height, "maxPeerHeight", maxPeerHeight) @@ -189,15 +188,15 @@ func (pool *BlockPool) PopRequest() { // Remove the peer and redo request from others. func (pool *BlockPool) RedoRequest(height int) { pool.mtx.Lock() // Lock - defer pool.mtx.Unlock() - request := pool.requesters[height] + pool.mtx.Unlock() + if request.block == nil { PanicSanity("Expected block to be non-nil") } // RemovePeer will redo all requesters associated with this peer. // TODO: record this malfeasance - pool.RemovePeer(request.peerID) // Lock on peersMtx. + pool.RemovePeer(request.peerID) // Lock on peersMtx and mtx } // TODO: ensure that blocks come in order for each peer. @@ -213,7 +212,10 @@ func (pool *BlockPool) AddBlock(peerID string, block *types.Block, blockSize int if requester.setBlock(block, peerID) { pool.numPending-- peer := pool.getPeer(peerID) - peer.decrPending(blockSize) + + pool.peersMtx.Lock() + peer.decrPending(blockSize, pool.onTimeout(peer)) + pool.peersMtx.Unlock() } else { // Bad peer? } @@ -241,6 +243,11 @@ func (pool *BlockPool) RemovePeer(peerID string) { } func (pool *BlockPool) removePeer(peerID string) { + // need to lock pool to access requesters and numPending. + // peersMtx should be locked by caller + pool.mtx.Lock() + defer pool.mtx.Unlock() + for _, requester := range pool.requesters { if requester.getPeerID() == peerID { pool.numPending++ @@ -276,7 +283,7 @@ func (pool *BlockPool) pickIncrAvailablePeer(minHeight int) *bpPeer { if peer.height < minHeight { continue } - peer.incrPending() + peer.incrPending(pool.onTimeout(peer)) return peer } return nil @@ -309,6 +316,14 @@ func (pool *BlockPool) sendTimeout(peerID string) { pool.timeoutsCh <- peerID } +func (pool *BlockPool) onTimeout(peer *bpPeer) func() { + return func() { + pool.peersMtx.Lock() + defer pool.peersMtx.Unlock() + peer.onTimeout() + } +} + func (pool *BlockPool) debug() string { pool.mtx.Lock() // Lock defer pool.mtx.Unlock() @@ -353,29 +368,30 @@ func (peer *bpPeer) resetMonitor() { peer.recvMonitor.SetREMA(initialValue) } -func (peer *bpPeer) resetTimeout() { +// needs the closure so we can lock the peersMtx +func (peer *bpPeer) resetTimeout(callback func()) { if peer.timeout == nil { - peer.timeout = time.AfterFunc(time.Second*peerTimeoutSeconds, peer.onTimeout) + peer.timeout = time.AfterFunc(time.Second*peerTimeoutSeconds, callback) } else { peer.timeout.Reset(time.Second * peerTimeoutSeconds) } } -func (peer *bpPeer) incrPending() { +func (peer *bpPeer) incrPending(onTimeout func()) { if peer.numPending == 0 { peer.resetMonitor() - peer.resetTimeout() + peer.resetTimeout(onTimeout) } peer.numPending++ } -func (peer *bpPeer) decrPending(recvSize int) { +func (peer *bpPeer) decrPending(recvSize int, onTimeout func()) { peer.numPending-- if peer.numPending == 0 { peer.timeout.Stop() } else { peer.recvMonitor.Update(recvSize) - peer.resetTimeout() + peer.resetTimeout(onTimeout) } } diff --git a/blockchain/reactor.go b/blockchain/reactor.go index a668e5f10..53aa311cc 100644 --- a/blockchain/reactor.go +++ b/blockchain/reactor.go @@ -196,7 +196,7 @@ FOR_LOOP: // ask for status updates go bcR.BroadcastStatusRequest() case _ = <-switchToConsensusTicker.C: - height, numPending := bcR.pool.GetStatus() + height, numPending, _ := bcR.pool.GetStatus() outbound, inbound, _ := bcR.Switch.NumPeers() log.Info("Consensus ticker", "numPending", numPending, "total", len(bcR.pool.requesters), "outbound", outbound, "inbound", inbound) From b25cfb0e0bae591e00c1c4a3d5a869fd16f5dee1 Mon Sep 17 00:00:00 2001 From: Jae Kwon Date: Tue, 28 Jun 2016 18:02:27 -0700 Subject: [PATCH 02/38] Unify blockpool mtxs --- blockchain/pool.go | 105 +++++++++++++++++---------------------------- 1 file changed, 40 insertions(+), 65 deletions(-) diff --git a/blockchain/pool.go b/blockchain/pool.go index 3c06d669b..86f67296d 100644 --- a/blockchain/pool.go +++ b/blockchain/pool.go @@ -35,15 +35,13 @@ type BlockPool struct { QuitService startTime time.Time + mtx sync.Mutex // block requests - mtx sync.Mutex requesters map[int]*bpRequester height int // the lowest key in requesters. numPending int32 // number of requests pending assignment or block response - // peers - peersMtx sync.Mutex - peers map[string]*bpPeer + peers map[string]*bpPeer requestsCh chan<- BlockRequest timeoutsCh chan<- string @@ -100,8 +98,9 @@ func (pool *BlockPool) makeRequestersRoutine() { } func (pool *BlockPool) removeTimedoutPeers() { - pool.peersMtx.Lock() // Lock - defer pool.peersMtx.Unlock() + pool.mtx.Lock() + defer pool.mtx.Unlock() + for _, peer := range pool.peers { if !peer.didTimeout && peer.numPending > 0 { curRate := peer.recvMonitor.Status().CurRate @@ -119,7 +118,7 @@ func (pool *BlockPool) removeTimedoutPeers() { } func (pool *BlockPool) GetStatus() (height int, numPending int32, lenRequesters int) { - pool.mtx.Lock() // Lock + pool.mtx.Lock() defer pool.mtx.Unlock() return pool.height, pool.numPending, len(pool.requesters) @@ -127,10 +126,10 @@ func (pool *BlockPool) GetStatus() (height int, numPending int32, lenRequesters // TODO: relax conditions, prevent abuse. func (pool *BlockPool) IsCaughtUp() bool { - height, _, _ := pool.GetStatus() + pool.mtx.Lock() + defer pool.mtx.Unlock() - pool.peersMtx.Lock() - defer pool.peersMtx.Unlock() + height := pool.height // Need at least 1 peer to be considered caught up. if len(pool.peers) == 0 { @@ -152,7 +151,7 @@ func (pool *BlockPool) IsCaughtUp() bool { // So we peek two blocks at a time. // The caller will verify the commit. func (pool *BlockPool) PeekTwoBlocks() (first *types.Block, second *types.Block) { - pool.mtx.Lock() // Lock + pool.mtx.Lock() defer pool.mtx.Unlock() if r := pool.requesters[pool.height]; r != nil { @@ -167,7 +166,7 @@ func (pool *BlockPool) PeekTwoBlocks() (first *types.Block, second *types.Block) // Pop the first block at pool.height // It must have been validated by 'second'.Commit from PeekTwoBlocks(). func (pool *BlockPool) PopRequest() { - pool.mtx.Lock() // Lock + pool.mtx.Lock() defer pool.mtx.Unlock() if r := pool.requesters[pool.height]; r != nil { @@ -187,7 +186,7 @@ func (pool *BlockPool) PopRequest() { // Invalidates the block at pool.height, // Remove the peer and redo request from others. func (pool *BlockPool) RedoRequest(height int) { - pool.mtx.Lock() // Lock + pool.mtx.Lock() request := pool.requesters[height] pool.mtx.Unlock() @@ -196,12 +195,12 @@ func (pool *BlockPool) RedoRequest(height int) { } // RemovePeer will redo all requesters associated with this peer. // TODO: record this malfeasance - pool.RemovePeer(request.peerID) // Lock on peersMtx and mtx + pool.RemovePeer(request.peerID) } // TODO: ensure that blocks come in order for each peer. func (pool *BlockPool) AddBlock(peerID string, block *types.Block, blockSize int) { - pool.mtx.Lock() // Lock + pool.mtx.Lock() defer pool.mtx.Unlock() requester := pool.requesters[block.Height] @@ -211,11 +210,8 @@ func (pool *BlockPool) AddBlock(peerID string, block *types.Block, blockSize int if requester.setBlock(block, peerID) { pool.numPending-- - peer := pool.getPeer(peerID) - - pool.peersMtx.Lock() - peer.decrPending(blockSize, pool.onTimeout(peer)) - pool.peersMtx.Unlock() + peer := pool.peers[peerID] + peer.decrPending(blockSize) } else { // Bad peer? } @@ -223,8 +219,8 @@ func (pool *BlockPool) AddBlock(peerID string, block *types.Block, blockSize int // Sets the peer's alleged blockchain height. func (pool *BlockPool) SetPeerHeight(peerID string, height int) { - pool.peersMtx.Lock() // Lock - defer pool.peersMtx.Unlock() + pool.mtx.Lock() + defer pool.mtx.Unlock() peer := pool.peers[peerID] if peer != nil { @@ -236,18 +232,13 @@ func (pool *BlockPool) SetPeerHeight(peerID string, height int) { } func (pool *BlockPool) RemovePeer(peerID string) { - pool.peersMtx.Lock() // Lock - defer pool.peersMtx.Unlock() + pool.mtx.Lock() + defer pool.mtx.Unlock() pool.removePeer(peerID) } func (pool *BlockPool) removePeer(peerID string) { - // need to lock pool to access requesters and numPending. - // peersMtx should be locked by caller - pool.mtx.Lock() - defer pool.mtx.Unlock() - for _, requester := range pool.requesters { if requester.getPeerID() == peerID { pool.numPending++ @@ -257,22 +248,14 @@ func (pool *BlockPool) removePeer(peerID string) { delete(pool.peers, peerID) } -func (pool *BlockPool) getPeer(peerID string) *bpPeer { - pool.peersMtx.Lock() // Lock - defer pool.peersMtx.Unlock() - - peer := pool.peers[peerID] - return peer -} - // Pick an available peer with at least the given minHeight. // If no peers are available, returns nil. func (pool *BlockPool) pickIncrAvailablePeer(minHeight int) *bpPeer { - pool.peersMtx.Lock() - defer pool.peersMtx.Unlock() + pool.mtx.Lock() + defer pool.mtx.Unlock() for _, peer := range pool.peers { - if peer.isBad() { + if peer.didTimeout { pool.removePeer(peer.id) continue } else { @@ -283,14 +266,14 @@ func (pool *BlockPool) pickIncrAvailablePeer(minHeight int) *bpPeer { if peer.height < minHeight { continue } - peer.incrPending(pool.onTimeout(peer)) + peer.incrPending() return peer } return nil } func (pool *BlockPool) makeNextRequester() { - pool.mtx.Lock() // Lock + pool.mtx.Lock() defer pool.mtx.Unlock() nextHeight := pool.height + len(pool.requesters) @@ -316,14 +299,6 @@ func (pool *BlockPool) sendTimeout(peerID string) { pool.timeoutsCh <- peerID } -func (pool *BlockPool) onTimeout(peer *bpPeer) func() { - return func() { - pool.peersMtx.Lock() - defer pool.peersMtx.Unlock() - peer.onTimeout() - } -} - func (pool *BlockPool) debug() string { pool.mtx.Lock() // Lock defer pool.mtx.Unlock() @@ -345,11 +320,13 @@ func (pool *BlockPool) debug() string { type bpPeer struct { pool *BlockPool id string - height int - numPending int32 recvMonitor *flow.Monitor - timeout *time.Timer - didTimeout bool + + mtx sync.Mutex + height int + numPending int32 + timeout *time.Timer + didTimeout bool } func newBPPeer(pool *BlockPool, peerID string, height int) *bpPeer { @@ -368,43 +345,41 @@ func (peer *bpPeer) resetMonitor() { peer.recvMonitor.SetREMA(initialValue) } -// needs the closure so we can lock the peersMtx -func (peer *bpPeer) resetTimeout(callback func()) { +func (peer *bpPeer) resetTimeout() { if peer.timeout == nil { - peer.timeout = time.AfterFunc(time.Second*peerTimeoutSeconds, callback) + peer.timeout = time.AfterFunc(time.Second*peerTimeoutSeconds, peer.onTimeout) } else { peer.timeout.Reset(time.Second * peerTimeoutSeconds) } } -func (peer *bpPeer) incrPending(onTimeout func()) { +func (peer *bpPeer) incrPending() { if peer.numPending == 0 { peer.resetMonitor() - peer.resetTimeout(onTimeout) + peer.resetTimeout() } peer.numPending++ } -func (peer *bpPeer) decrPending(recvSize int, onTimeout func()) { +func (peer *bpPeer) decrPending(recvSize int) { peer.numPending-- if peer.numPending == 0 { peer.timeout.Stop() } else { peer.recvMonitor.Update(recvSize) - peer.resetTimeout(onTimeout) + peer.resetTimeout() } } func (peer *bpPeer) onTimeout() { + peer.pool.mtx.Lock() + defer peer.pool.mtx.Unlock() + peer.pool.sendTimeout(peer.id) log.Warn("SendTimeout", "peer", peer.id, "reason", "onTimeout") peer.didTimeout = true } -func (peer *bpPeer) isBad() bool { - return peer.didTimeout -} - //------------------------------------- type bpRequester struct { From 5fb171edff3ab2dfb6cd5cd5a4991fb74179f42a Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Thu, 23 Jun 2016 20:53:11 -0400 Subject: [PATCH 03/38] mempool: remove bad txs from cacheMap --- mempool/mempool.go | 27 +++++++++++++++++++++++++-- 1 file changed, 25 insertions(+), 2 deletions(-) diff --git a/mempool/mempool.go b/mempool/mempool.go index 87a185ef2..5a629b159 100644 --- a/mempool/mempool.go +++ b/mempool/mempool.go @@ -144,6 +144,13 @@ func (mem *Mempool) CheckTx(tx types.Tx, cb func(*tmsp.Response)) (err error) { return nil } +func (mem *Mempool) removeTxFromCacheMap(tx []byte) { + mem.proxyMtx.Lock() + delete(mem.cacheMap, string(tx)) + mem.proxyMtx.Unlock() + +} + // TMSP callback function func (mem *Mempool) resCb(req *tmsp.Request, res *tmsp.Response) { if mem.recheckCursor == nil { @@ -165,8 +172,12 @@ func (mem *Mempool) resCbNormal(req *tmsp.Request, res *tmsp.Response) { } mem.txs.PushBack(memTx) } else { - log.Info("Bad Transaction", "res", r) // ignore bad transaction + log.Info("Bad Transaction", "res", r) + + // remove from cache (it might be good later) + mem.removeTxFromCacheMap(req.GetCheckTx().Tx) + // TODO: handle other retcodes } default: @@ -188,6 +199,9 @@ func (mem *Mempool) resCbRecheck(req *tmsp.Request, res *tmsp.Response) { // Tx became invalidated due to newly committed block. mem.txs.Remove(mem.recheckCursor) mem.recheckCursor.DetachPrev() + + // remove from cache (it might be good later) + mem.removeTxFromCacheMap(req.GetCheckTx().Tx) } if mem.recheckCursor == mem.recheckEnd { mem.recheckCursor = nil @@ -270,10 +284,19 @@ func (mem *Mempool) filterTxs(blockTxsMap map[string]struct{}) []types.Tx { goodTxs := make([]types.Tx, 0, mem.txs.Len()) for e := mem.txs.Front(); e != nil; e = e.Next() { memTx := e.Value.(*mempoolTx) + // Remove the tx if its alredy in a block. if _, ok := blockTxsMap[string(memTx.tx)]; ok { - // Remove the tx since already in block. + // remove from clist mem.txs.Remove(e) e.DetachPrev() + + // remove from mempool cache + // we only enforce "at-least once" semantics and + // leave it to the application to implement "only-once" + // via eg. sequence numbers, utxos, etc. + // NOTE: expects caller of filterTxs to hold the lock + // (so we can't use mem.removeTxFromCacheMap) + delete(mem.cacheMap, string(memTx.tx)) continue } // Good tx! From 65ebc344ac26b8d7ca8b5e96fa9a6f6c180814db Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Mon, 27 Jun 2016 20:43:09 -0400 Subject: [PATCH 04/38] broadcast_commit --- node/node.go | 1 + rpc/core/mempool.go | 70 +++++++++++++++++++++++++++++++++++++++++++-- rpc/core/pipe.go | 6 ++++ rpc/core/routes.go | 9 ++++++ state/execution.go | 8 ++++++ types/events.go | 2 ++ types/tx.go | 9 +++++- 7 files changed, 102 insertions(+), 3 deletions(-) diff --git a/node/node.go b/node/node.go index 3d2c91966..231b6b650 100644 --- a/node/node.go +++ b/node/node.go @@ -177,6 +177,7 @@ func (n *Node) AddListener(l p2p.Listener) { func (n *Node) StartRPC() ([]net.Listener, error) { rpccore.SetConfig(n.config) + rpccore.SetEventSwitch(n.evsw) rpccore.SetBlockStore(n.blockStore) rpccore.SetConsensusState(n.consensusState) rpccore.SetConsensusReactor(n.consensusReactor) diff --git a/rpc/core/mempool.go b/rpc/core/mempool.go index b0e5c0c43..f915ce426 100644 --- a/rpc/core/mempool.go +++ b/rpc/core/mempool.go @@ -2,14 +2,18 @@ package core import ( "fmt" + "time" + + "github.com/tendermint/go-events" ctypes "github.com/tendermint/tendermint/rpc/core/types" "github.com/tendermint/tendermint/types" tmsp "github.com/tendermint/tmsp/types" ) //----------------------------------------------------------------------------- +// NOTE: tx should be signed, but this is only checked at the app level (not by Tendermint!) -// NOTE: tx must be signed +// Returns right away, with no response func BroadcastTxAsync(tx types.Tx) (*ctypes.ResultBroadcastTx, error) { err := mempoolReactor.BroadcastTx(tx, nil) if err != nil { @@ -18,7 +22,7 @@ func BroadcastTxAsync(tx types.Tx) (*ctypes.ResultBroadcastTx, error) { return &ctypes.ResultBroadcastTx{}, nil } -// Note: tx must be signed +// Returns with the response from CheckTx func BroadcastTxSync(tx types.Tx) (*ctypes.ResultBroadcastTx, error) { resCh := make(chan *tmsp.Response, 1) err := mempoolReactor.BroadcastTx(tx, func(res *tmsp.Response) { @@ -36,6 +40,68 @@ func BroadcastTxSync(tx types.Tx) (*ctypes.ResultBroadcastTx, error) { }, nil } +// CONTRACT: returns error==nil iff the tx is included in a block. +// +// If CheckTx fails, return with the response from CheckTx AND an error. +// Else, block until the tx is included in a block, +// and return the result of AppendTx (with no error). +// Even if AppendTx fails, so long as the tx is included in a block this function +// will not return an error. +// The function times out after five minutes and returns the result of CheckTx and an error. +// TODO: smarter timeout logic or someway to cancel (tx not getting committed is a sign of a larger problem!) +func BroadcastTxCommit(tx types.Tx) (*ctypes.ResultBroadcastTx, error) { + + // subscribe to tx being committed in block + appendTxResCh := make(chan *tmsp.Response, 1) + eventSwitch.AddListenerForEvent("rpc", types.EventStringTx(tx), func(data events.EventData) { + appendTxResCh <- data.(*tmsp.Response) + }) + + // broadcast the tx and register checktx callback + checkTxResCh := make(chan *tmsp.Response, 1) + err := mempoolReactor.BroadcastTx(tx, func(res *tmsp.Response) { + checkTxResCh <- res + }) + if err != nil { + return nil, fmt.Errorf("Error broadcasting transaction: %v", err) + } + checkTxRes := <-checkTxResCh + checkTxR := checkTxRes.GetCheckTx() + if r := checkTxR; r.Code != tmsp.CodeType_OK { + // CheckTx failed! + return &ctypes.ResultBroadcastTx{ + Code: r.Code, + Data: r.Data, + Log: r.Log, + }, fmt.Errorf("Check tx failed with non-zero code: %s. Data: %X; Log: %s", r.Code.String(), r.Data, r.Log) + } + + // Wait for the tx to be included in a block, + // timeout after something reasonable. + timer := time.NewTimer(60 * 5 * time.Second) + select { + case appendTxRes := <-appendTxResCh: + // The tx was included in a block. + // NOTE we don't return an error regardless of the AppendTx code; + // clients must check this to see if they need to send a new tx! + r := appendTxRes.GetAppendTx() + return &ctypes.ResultBroadcastTx{ + Code: r.Code, + Data: r.Data, + Log: r.Log, + }, nil + case <-timer.C: + r := checkTxR + return &ctypes.ResultBroadcastTx{ + Code: r.Code, + Data: r.Data, + Log: r.Log, + }, fmt.Errorf("Timed out waiting for transaction to be included in a block") + } + + panic("Should never happen!") +} + func UnconfirmedTxs() (*ctypes.ResultUnconfirmedTxs, error) { txs := mempoolReactor.Mempool.Reap(0) return &ctypes.ResultUnconfirmedTxs{len(txs), txs}, nil diff --git a/rpc/core/pipe.go b/rpc/core/pipe.go index de196e813..464f7fda9 100644 --- a/rpc/core/pipe.go +++ b/rpc/core/pipe.go @@ -4,12 +4,14 @@ import ( cfg "github.com/tendermint/go-config" "github.com/tendermint/go-p2p" + "github.com/tendermint/go-events" bc "github.com/tendermint/tendermint/blockchain" "github.com/tendermint/tendermint/consensus" mempl "github.com/tendermint/tendermint/mempool" "github.com/tendermint/tendermint/types" ) +var eventSwitch *events.EventSwitch var blockStore *bc.BlockStore var consensusState *consensus.ConsensusState var consensusReactor *consensus.ConsensusReactor @@ -24,6 +26,10 @@ func SetConfig(c cfg.Config) { config = c } +func SetEventSwitch(evsw *events.EventSwitch) { + eventSwitch = evsw +} + func SetBlockStore(bs *bc.BlockStore) { blockStore = bs } diff --git a/rpc/core/routes.go b/rpc/core/routes.go index 585a18cd2..b8938f6bc 100644 --- a/rpc/core/routes.go +++ b/rpc/core/routes.go @@ -19,6 +19,7 @@ var Routes = map[string]*rpc.RPCFunc{ "block": rpc.NewRPCFunc(BlockResult, "height"), "validators": rpc.NewRPCFunc(ValidatorsResult, ""), "dump_consensus_state": rpc.NewRPCFunc(DumpConsensusStateResult, ""), + "broadcast_tx_commit": rpc.NewRPCFunc(BroadcastTxCommitResult, "tx"), "broadcast_tx_sync": rpc.NewRPCFunc(BroadcastTxSyncResult, "tx"), "broadcast_tx_async": rpc.NewRPCFunc(BroadcastTxAsyncResult, "tx"), "unconfirmed_txs": rpc.NewRPCFunc(UnconfirmedTxsResult, ""), @@ -126,6 +127,14 @@ func NumUnconfirmedTxsResult() (ctypes.TMResult, error) { } } +func BroadcastTxCommitResult(tx []byte) (ctypes.TMResult, error) { + if r, err := BroadcastTxCommit(tx); err != nil { + return nil, err + } else { + return r, nil + } +} + func BroadcastTxSyncResult(tx []byte) (ctypes.TMResult, error) { if r, err := BroadcastTxSync(tx); err != nil { return nil, err diff --git a/state/execution.go b/state/execution.go index 230bceb29..ad359f4d4 100644 --- a/state/execution.go +++ b/state/execution.go @@ -57,6 +57,8 @@ func (s *State) ExecBlock(evsw *events.EventSwitch, proxyAppConn proxy.AppConn, // TODO: Generate a bitmap or otherwise store tx validity in state. func (s *State) execBlockOnProxyApp(evsw *events.EventSwitch, proxyAppConn proxy.AppConn, block *types.Block) error { + eventCache := events.NewEventCache(evsw) + var validTxs, invalidTxs = 0, 0 // Execute transactions and get hash @@ -73,6 +75,9 @@ func (s *State) execBlockOnProxyApp(evsw *events.EventSwitch, proxyAppConn proxy log.Debug("Invalid tx", "code", r.AppendTx.Code, "log", r.AppendTx.Log) invalidTxs += 1 } + // NOTE: if we count we can access the tx from the block instead of + // pulling it from the req + eventCache.FireEvent(types.EventStringTx(req.GetAppendTx().Tx), res) } } proxyAppConn.SetResponseCallback(proxyCb) @@ -95,6 +100,9 @@ func (s *State) execBlockOnProxyApp(evsw *events.EventSwitch, proxyAppConn proxy log.Info("TODO: Do something with changedValidators", changedValidators) log.Info(Fmt("ExecBlock got %v valid txs and %v invalid txs", validTxs, invalidTxs)) + + // fire events + eventCache.Flush() return nil } diff --git a/types/events.go b/types/events.go index 3328911c1..68313ff25 100644 --- a/types/events.go +++ b/types/events.go @@ -2,6 +2,7 @@ package types import ( // for registering TMEventData as events.EventData + . "github.com/tendermint/go-common" "github.com/tendermint/go-events" "github.com/tendermint/go-wire" ) @@ -14,6 +15,7 @@ func EventStringUnbond() string { return "Unbond" } func EventStringRebond() string { return "Rebond" } func EventStringDupeout() string { return "Dupeout" } func EventStringFork() string { return "Fork" } +func EventStringTx(tx Tx) string { return Fmt("Tx:%X", tx.Hash()) } func EventStringNewBlock() string { return "NewBlock" } func EventStringNewBlockHeader() string { return "NewBlockHeader" } diff --git a/types/tx.go b/types/tx.go index 60699d534..40a042395 100644 --- a/types/tx.go +++ b/types/tx.go @@ -6,6 +6,13 @@ import ( type Tx []byte +// NOTE: this is the hash of the go-wire encoded Tx. +// Tx has no types at this level, so just length-prefixed. +// Maybe it should just be the hash of the bytes tho? +func (tx Tx) Hash() []byte { + return merkle.SimpleHashFromBinary(tx) +} + type Txs []Tx func (txs Txs) Hash() []byte { @@ -15,7 +22,7 @@ func (txs Txs) Hash() []byte { case 0: return nil case 1: - return merkle.SimpleHashFromBinary(txs[0]) + return txs[0].Hash() default: left := Txs(txs[:(len(txs)+1)/2]).Hash() right := Txs(txs[(len(txs)+1)/2:]).Hash() From 7e3e9ee9d2a63a6e60c65699599997307a766a8d Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Tue, 5 Jul 2016 14:41:50 -0400 Subject: [PATCH 05/38] rpc: broadcast tests. closes #219 --- mempool/mempool.go | 21 +++++++- rpc/core/mempool.go | 2 +- rpc/test/client_test.go | 105 ++++++++++++++++++++++++++++++---------- rpc/test/helpers.go | 1 - 4 files changed, 100 insertions(+), 29 deletions(-) diff --git a/mempool/mempool.go b/mempool/mempool.go index 5a629b159..e26302051 100644 --- a/mempool/mempool.go +++ b/mempool/mempool.go @@ -60,7 +60,7 @@ type Mempool struct { // Keep a cache of already-seen txs. // This reduces the pressure on the proxyApp. cacheMap map[string]struct{} - cacheList *list.List + cacheList *list.List // to remove oldest tx when cache gets too big } func NewMempool(config cfg.Config, proxyAppConn proxy.AppConn) *Mempool { @@ -81,6 +81,7 @@ func NewMempool(config cfg.Config, proxyAppConn proxy.AppConn) *Mempool { return mempool } +// consensus must be able to hold lock to safely update func (mem *Mempool) Lock() { mem.proxyMtx.Lock() } @@ -89,10 +90,25 @@ func (mem *Mempool) Unlock() { mem.proxyMtx.Unlock() } +// Number of transactions in the mempool clist func (mem *Mempool) Size() int { return mem.txs.Len() } +// Remove all transactions from mempool and cache +func (mem *Mempool) Flush() { + mem.proxyMtx.Lock() + defer mem.proxyMtx.Unlock() + + mem.cacheMap = make(map[string]struct{}, cacheSize) + mem.cacheList.Init() + + for e := mem.txs.Front(); e != nil; e = e.Next() { + mem.txs.Remove(e) + e.DetachPrev() + } +} + // Return the first element of mem.txs for peer goroutines to call .NextWait() on. // Blocks until txs has elements. func (mem *Mempool) TxsFrontWait() *clist.CElement { @@ -125,6 +141,8 @@ func (mem *Mempool) CheckTx(tx types.Tx, cb func(*tmsp.Response)) (err error) { if mem.cacheList.Len() >= cacheSize { popped := mem.cacheList.Front() poppedTx := popped.Value.(types.Tx) + // NOTE: the tx may have already been removed from the map + // but deleting a non-existant element is fine delete(mem.cacheMap, string(poppedTx)) mem.cacheList.Remove(popped) } @@ -146,6 +164,7 @@ func (mem *Mempool) CheckTx(tx types.Tx, cb func(*tmsp.Response)) (err error) { func (mem *Mempool) removeTxFromCacheMap(tx []byte) { mem.proxyMtx.Lock() + // NOTE tx not removed from cacheList delete(mem.cacheMap, string(tx)) mem.proxyMtx.Unlock() diff --git a/rpc/core/mempool.go b/rpc/core/mempool.go index f915ce426..cf3ec919c 100644 --- a/rpc/core/mempool.go +++ b/rpc/core/mempool.go @@ -46,7 +46,7 @@ func BroadcastTxSync(tx types.Tx) (*ctypes.ResultBroadcastTx, error) { // Else, block until the tx is included in a block, // and return the result of AppendTx (with no error). // Even if AppendTx fails, so long as the tx is included in a block this function -// will not return an error. +// will not return an error - it is the caller's responsibility to check res.Code. // The function times out after five minutes and returns the result of CheckTx and an error. // TODO: smarter timeout logic or someway to cancel (tx not getting committed is a sign of a larger problem!) func BroadcastTxCommit(tx types.Tx) (*ctypes.ResultBroadcastTx, error) { diff --git a/rpc/test/client_test.go b/rpc/test/client_test.go index a5b243bea..bb66965ed 100644 --- a/rpc/test/client_test.go +++ b/rpc/test/client_test.go @@ -1,11 +1,13 @@ package rpctest import ( + "bytes" "fmt" "testing" ctypes "github.com/tendermint/tendermint/rpc/core/types" "github.com/tendermint/tendermint/types" + tmsp "github.com/tendermint/tmsp/types" ) //-------------------------------------------------------------------------------- @@ -42,33 +44,84 @@ func testStatus(t *testing.T, statusI interface{}) { } } -// TODO -/* -func testBroadcastTx(t *testing.T, typ string) { - amt := int64(100) - toAddr := user[1].Address - tx := makeDefaultSendTxSigned(t, typ, toAddr, amt) - receipt := broadcastTx(t, typ, tx) - if receipt.CreatesContract > 0 { - t.Fatal("This tx does not create a contract") - } - if len(receipt.TxHash) == 0 { - t.Fatal("Failed to compute tx hash") - } - pool := node.MempoolReactor().Mempool - txs := pool.GetProposalTxs() - if len(txs) != mempoolCount { - t.Fatalf("The mem pool has %d txs. Expected %d", len(txs), mempoolCount) - } - tx2 := txs[mempoolCount-1].(*types.SendTx) - n, err := new(int64), new(error) - buf1, buf2 := new(bytes.Buffer), new(bytes.Buffer) - tx.WriteSignBytes(chainID, buf1, n, err) - tx2.WriteSignBytes(chainID, buf2, n, err) - if bytes.Compare(buf1.Bytes(), buf2.Bytes()) != 0 { - t.Fatal("inconsistent hashes for mempool tx and sent tx") +//-------------------------------------------------------------------------------- +// broadcast tx sync + +var testTx = []byte{0x1, 0x2, 0x3, 0x4, 0x5} + +func TestURIBroadcastTxSync(t *testing.T) { + config.Set("block_size", 0) + defer config.Set("block_size", -1) + tmResult := new(ctypes.TMResult) + _, err := clientURI.Call("broadcast_tx_sync", map[string]interface{}{"tx": testTx}, tmResult) + if err != nil { + t.Fatal(err) } -}*/ + testBroadcastTxSync(t, tmResult) +} + +func TestJSONBroadcastTxSync(t *testing.T) { + config.Set("block_size", 0) + defer config.Set("block_size", -1) + tmResult := new(ctypes.TMResult) + _, err := clientJSON.Call("broadcast_tx_sync", []interface{}{testTx}, tmResult) + if err != nil { + t.Fatal(err) + } + testBroadcastTxSync(t, tmResult) +} + +func testBroadcastTxSync(t *testing.T, resI interface{}) { + tmRes := resI.(*ctypes.TMResult) + res := (*tmRes).(*ctypes.ResultBroadcastTx) + if res.Code != tmsp.CodeType_OK { + t.Fatalf("BroadcastTxSync got non-zero exit code: %v. %X; %s", res.Code, res.Data, res.Log) + } + mem := node.MempoolReactor().Mempool + if mem.Size() != 1 { + t.Fatalf("Mempool size should have been 1. Got %d", mem.Size()) + } + + txs := mem.Reap(1) + if !bytes.Equal(txs[0], testTx) { + t.Fatalf("Tx in mempool does not match test tx. Got %X, expected %X", txs[0], testTx) + } + + mem.Flush() +} + +//-------------------------------------------------------------------------------- +// broadcast tx commit + +func TestURIBroadcastTxCommit(t *testing.T) { + tmResult := new(ctypes.TMResult) + _, err := clientURI.Call("broadcast_tx_commit", map[string]interface{}{"tx": testTx}, tmResult) + if err != nil { + t.Fatal(err) + } + testBroadcastTxCommit(t, tmResult) +} + +func TestJSONBroadcastTxCommit(t *testing.T) { + tmResult := new(ctypes.TMResult) + _, err := clientJSON.Call("broadcast_tx_commit", []interface{}{testTx}, tmResult) + if err != nil { + t.Fatal(err) + } + testBroadcastTxCommit(t, tmResult) +} + +func testBroadcastTxCommit(t *testing.T, resI interface{}) { + tmRes := resI.(*ctypes.TMResult) + res := (*tmRes).(*ctypes.ResultBroadcastTx) + if res.Code != tmsp.CodeType_OK { + t.Fatalf("BroadcastTxCommit got non-zero exit code: %v. %X; %s", res.Code, res.Data, res.Log) + } + mem := node.MempoolReactor().Mempool + if mem.Size() != 0 { + t.Fatalf("Mempool size should have been 0. Got %s", mem.Size()) + } +} //-------------------------------------------------------------------------------- // Test the websocket service diff --git a/rpc/test/helpers.go b/rpc/test/helpers.go index c209324b6..3713aa821 100644 --- a/rpc/test/helpers.go +++ b/rpc/test/helpers.go @@ -20,7 +20,6 @@ import ( var ( config cfg.Config node *nm.Node - mempoolCount = 0 chainID string rpcAddr string requestAddr string From bc0b52293e116d8ff7f8998aee4d1d7bc5c642b3 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Tue, 5 Jul 2016 15:40:53 -0400 Subject: [PATCH 06/38] update circle and glide --- Makefile | 4 ++++ circle.yml | 4 ++-- consensus/state_test.go | 2 +- glide.lock | 4 ++-- 4 files changed, 9 insertions(+), 5 deletions(-) diff --git a/Makefile b/Makefile index ac1956dde..ea66d8207 100644 --- a/Makefile +++ b/Makefile @@ -43,6 +43,10 @@ get_deps: grep -v /vendor/ | sort | uniq | \ xargs go get +get_vendor_deps: + go get github.com/Masterminds/glide + glide install + update_deps: go get -d -u github.com/tendermint/tendermint/... diff --git a/circle.yml b/circle.yml index f022d5a6b..98dd22283 100644 --- a/circle.yml +++ b/circle.yml @@ -16,8 +16,8 @@ checkout: dependencies: override: - - "cd $REPO && make get_deps" + - "cd $REPO && make get_vendor_deps" test: override: - - "cd $REPO && make test" + - "cd $REPO && make test_race" diff --git a/consensus/state_test.go b/consensus/state_test.go index 797e5d04b..c2ab54585 100644 --- a/consensus/state_test.go +++ b/consensus/state_test.go @@ -346,7 +346,7 @@ func TestLockNoPOL(t *testing.T) { // we should now be stuck in limbo forever, waiting for more prevotes // prevote arrives from cs2: - signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, cs1.ProposalBlock.Hash(), cs1.ProposalBlockParts.Header()) + signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header()) <-voteCh // prevote <-voteCh // precommit diff --git a/glide.lock b/glide.lock index 1cc00875b..1624d7441 100644 --- a/glide.lock +++ b/glide.lock @@ -56,7 +56,7 @@ imports: - name: github.com/tendermint/go-common version: dcfa46af1341d03b80d32e4901019d1668b978b9 - name: github.com/tendermint/go-config - version: cfcef384d64b94e50909596e39b32ffb3cc20573 + version: e64b424499acd0eb9856b88e10c0dff41628c0d6 - name: github.com/tendermint/go-crypto version: 41cfb7b677f4e16cdfd22b6ce0946c89919fbc7b - name: github.com/tendermint/go-db @@ -68,7 +68,7 @@ imports: - name: github.com/tendermint/go-merkle version: 05042c6ab9cad51d12e4cecf717ae68e3b1409a8 - name: github.com/tendermint/go-p2p - version: 5bd7692323ec60d6461678f09b5024a952164151 + version: 929cf433b9c8e987af5f7f3ca3ce717e1e3eda53 subpackages: - upnp - name: github.com/tendermint/go-rpc From 8ca615c30153322ad207ece2e791c7ee8c14c861 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Tue, 5 Jul 2016 17:03:09 -0400 Subject: [PATCH 07/38] wait until txs removed from mempool to fire tx events --- consensus/state.go | 8 +++++++- mempool/mempool.go | 1 - rpc/test/client_test.go | 2 +- state/execution.go | 11 +++-------- 4 files changed, 11 insertions(+), 11 deletions(-) diff --git a/consensus/state.go b/consensus/state.go index 37209f68d..338568b30 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -1214,10 +1214,13 @@ func (cs *ConsensusState) finalizeCommit(height int) { // Create a copy of the state for staging stateCopy := cs.state.Copy() + // event cache for txs + eventCache := events.NewEventCache(cs.evsw) + // Run the block on the State: // + update validator sets // + run txs on the proxyAppConn - err := stateCopy.ExecBlock(cs.evsw, cs.proxyAppConn, block, blockParts.Header()) + err := stateCopy.ExecBlock(eventCache, cs.proxyAppConn, block, blockParts.Header()) if err != nil { // TODO: handle this gracefully. PanicQ(Fmt("Exec failed for application: %v", err)) @@ -1230,6 +1233,9 @@ func (cs *ConsensusState) finalizeCommit(height int) { PanicQ(Fmt("Commit failed for application: %v", err)) } + // txs committed and removed from mempool, fire events + eventCache.Flush() + // Save to blockStore. if cs.blockStore.Height() < block.Height { precommits := cs.Votes.Precommits(cs.CommitRound) diff --git a/mempool/mempool.go b/mempool/mempool.go index e26302051..71ee65a32 100644 --- a/mempool/mempool.go +++ b/mempool/mempool.go @@ -167,7 +167,6 @@ func (mem *Mempool) removeTxFromCacheMap(tx []byte) { // NOTE tx not removed from cacheList delete(mem.cacheMap, string(tx)) mem.proxyMtx.Unlock() - } // TMSP callback function diff --git a/rpc/test/client_test.go b/rpc/test/client_test.go index bb66965ed..e29587fec 100644 --- a/rpc/test/client_test.go +++ b/rpc/test/client_test.go @@ -119,7 +119,7 @@ func testBroadcastTxCommit(t *testing.T, resI interface{}) { } mem := node.MempoolReactor().Mempool if mem.Size() != 0 { - t.Fatalf("Mempool size should have been 0. Got %s", mem.Size()) + t.Fatalf("Mempool size should have been 0. Got %d", mem.Size()) } } diff --git a/state/execution.go b/state/execution.go index ad359f4d4..1acd04930 100644 --- a/state/execution.go +++ b/state/execution.go @@ -18,7 +18,7 @@ func (s *State) ValidateBlock(block *types.Block) error { // Execute the block to mutate State. // Validates block and then executes Data.Txs in the block. -func (s *State) ExecBlock(evsw *events.EventSwitch, proxyAppConn proxy.AppConn, block *types.Block, blockPartsHeader types.PartSetHeader) error { +func (s *State) ExecBlock(eventCache events.Fireable, proxyAppConn proxy.AppConn, block *types.Block, blockPartsHeader types.PartSetHeader) error { // Validate the block. err := s.validateBlock(block) @@ -34,7 +34,7 @@ func (s *State) ExecBlock(evsw *events.EventSwitch, proxyAppConn proxy.AppConn, nextValSet := valSet.Copy() // Execute the block txs - err = s.execBlockOnProxyApp(evsw, proxyAppConn, block) + err = s.execBlockOnProxyApp(eventCache, proxyAppConn, block) if err != nil { // There was some error in proxyApp // TODO Report error and wait for proxyApp to be available. @@ -55,9 +55,7 @@ func (s *State) ExecBlock(evsw *events.EventSwitch, proxyAppConn proxy.AppConn, // Executes block's transactions on proxyAppConn. // TODO: Generate a bitmap or otherwise store tx validity in state. -func (s *State) execBlockOnProxyApp(evsw *events.EventSwitch, proxyAppConn proxy.AppConn, block *types.Block) error { - - eventCache := events.NewEventCache(evsw) +func (s *State) execBlockOnProxyApp(eventCache events.Fireable, proxyAppConn proxy.AppConn, block *types.Block) error { var validTxs, invalidTxs = 0, 0 @@ -100,9 +98,6 @@ func (s *State) execBlockOnProxyApp(evsw *events.EventSwitch, proxyAppConn proxy log.Info("TODO: Do something with changedValidators", changedValidators) log.Info(Fmt("ExecBlock got %v valid txs and %v invalid txs", validTxs, invalidTxs)) - - // fire events - eventCache.Flush() return nil } From 4e5cdd6abb29fc9d2066f46cf692f8789aa9178e Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Wed, 6 Jul 2016 13:46:37 -0400 Subject: [PATCH 08/38] bash tests for broadcast_tx through rpc --- test/rpc/clean.sh | 3 ++ test/rpc/counter_test.sh | 72 ++++++++++++++++++++++++++++++++++++++++ test/rpc/dummy_test.sh | 34 +++++++++++++++++++ test/rpc/test.sh | 57 +++++++++++++++++++++++++++++++ 4 files changed, 166 insertions(+) create mode 100644 test/rpc/clean.sh create mode 100644 test/rpc/counter_test.sh create mode 100644 test/rpc/dummy_test.sh create mode 100644 test/rpc/test.sh diff --git a/test/rpc/clean.sh b/test/rpc/clean.sh new file mode 100644 index 000000000..1b34033f2 --- /dev/null +++ b/test/rpc/clean.sh @@ -0,0 +1,3 @@ +killall tendermint +killall dummy +killall counter diff --git a/test/rpc/counter_test.sh b/test/rpc/counter_test.sh new file mode 100644 index 000000000..bece24019 --- /dev/null +++ b/test/rpc/counter_test.sh @@ -0,0 +1,72 @@ +#! /bin/bash + +##################### +# counter over socket +##################### +TESTNAME=$1 + +# Send some txs + +function sendTx() { + TX=$1 + RESPONSE=`curl -s localhost:46657/broadcast_tx_commit?tx=\"$TX\"` + CODE=`echo $RESPONSE | jq .result[1].code` + ERROR=`echo $RESPONSE | jq .error` + ERROR=$(echo "$ERROR" | tr -d '"') # remove surrounding quotes + echo "ERROR: $ERROR" + echo "CODE: $CODE" + echo "" +} + +# 0 should pass once and get in block, with no error +TX=00 +sendTx $TX +if [[ $CODE != 0 ]]; then + echo "Got non-zero exit code for $TX. $RESPONSE" + exit 1 +fi +if [[ "$ERROR" != "" ]]; then + echo "Unexpected error. Tx $TX should have been included in a block. $ERROR" + exit 1 +fi + + + +# second time should get rejected by the mempool (return error and non-zero code) +sendTx $TX +if [[ $CODE == 0 ]]; then + echo "Got zero exit code for $TX. Expected tx to be rejected by mempool. $RESPONSE" + exit 1 +fi +if [[ "$ERROR" == "" ]]; then + echo "Expected to get an error - tx $TX should have been rejected from mempool" + echo "$RESPONSE" + exit 1 +fi + + +# now, TX=01 should pass, with no error +TX=01 +sendTx $TX +if [[ $CODE != 0 ]]; then + echo "Got non-zero exit code for $TX. $RESPONSE" + exit 1 +fi +if [[ "$ERROR" != "" ]]; then + echo "Unexpected error. Tx $TX should have been accepted in block. $ERROR" + exit 1 +fi + +# now, TX=03 should get in a block (passes CheckTx, no error), but is invalid +TX=03 +sendTx $TX +if [[ $CODE == 0 ]]; then + echo "Got zero exit code for $TX. Should have been bad nonce. $RESPONSE" + exit 1 +fi +if [[ "$ERROR" != "" ]]; then + echo "Unexpected error. Tx $TX should have been included in a block. $ERROR" + exit 1 +fi + +echo "Passed Test: $TESTNAME" diff --git a/test/rpc/dummy_test.sh b/test/rpc/dummy_test.sh new file mode 100644 index 000000000..9410c88da --- /dev/null +++ b/test/rpc/dummy_test.sh @@ -0,0 +1,34 @@ +#! /bin/bash + +function toHex() { + echo -n $1 | hexdump -ve '1/1 "%.2X"' +} + +##################### +# dummy with curl +##################### +TESTNAME=$1 + +# store key value pair +KEY="abcd" +VALUE="dcba" +curl localhost:46657/broadcast_tx_commit?tx=\"$(toHex $KEY=$VALUE)\" +echo "" + +# we should be able to look up the key +RESPONSE=`tmsp-cli query $KEY` +A=`echo $RESPONSE | grep exists=true` +if [[ $? != 0 ]]; then + echo "Failed to find 'exists=true' for $KEY. Response:" + echo "$RESPONSE" +fi + +# we should not be able to look up the value +RESPONSE=`tmsp-cli query $VALUE` +A=`echo $RESPONSE | grep exists=true` +if [[ $? == 0 ]]; then + echo "Found 'exists=true' for $VALUE when we should not have. Response:" + echo "$RESPONSE" +fi + +echo "Passed Test: $TESTNAME" diff --git a/test/rpc/test.sh b/test/rpc/test.sh new file mode 100644 index 000000000..f31b19c55 --- /dev/null +++ b/test/rpc/test.sh @@ -0,0 +1,57 @@ +#! /bin/bash +set -e + +#- dummy over socket, curl +#- counter over socket, curl +#- counter over grpc, curl +#- counter over grpc, grpc + +# TODO: install everything + +function dummy_over_socket(){ + dummy > /dev/null & + tendermint node > tendermint.log & + sleep 3 + + bash dummy_test.sh "Dummy over Socket" + + killall dummy tendermint +} + + +function counter_over_socket() { + counter --serial > /dev/null & + tendermint node > tendermint.log & + sleep 3 + + bash counter_test.sh "Counter over Socket" + + killall counter tendermint +} + +function counter_over_grpc() { + counter --serial --tmsp grpc > /dev/null & + tendermint node --tmsp grpc > tendermint.log & + sleep 3 + + bash counter_test.sh "Counter over GRPC" + + killall counter tendermint +} + +case "$1" in + "dummy_over_socket") + dummy_over_socket + ;; + "counter_over_socket") + counter_over_socket + ;; + "counter_over_grpc") + counter_over_grpc + ;; + *) + dummy_over_socket + counter_over_socket + counter_over_grpc +esac + From e8271c8a4beebefc49e45d64c6110d7af0b04cef Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Wed, 6 Jul 2016 17:09:40 -0400 Subject: [PATCH 09/38] update tmsp glide --- glide.lock | 2 +- mempool/mempool.go | 2 ++ test/rpc/counter_test.sh | 3 --- 3 files changed, 3 insertions(+), 4 deletions(-) diff --git a/glide.lock b/glide.lock index 1624d7441..e99b9f854 100644 --- a/glide.lock +++ b/glide.lock @@ -84,7 +84,7 @@ imports: subpackages: - term - name: github.com/tendermint/tmsp - version: f41bc5f11969e22b357d94b4247403fd62d40445 + version: 49a67aee8a7984a68eabe2c45ff6eb0ff51e31f9 subpackages: - client - example/dummy diff --git a/mempool/mempool.go b/mempool/mempool.go index 71ee65a32..0833cb4a6 100644 --- a/mempool/mempool.go +++ b/mempool/mempool.go @@ -194,6 +194,8 @@ func (mem *Mempool) resCbNormal(req *tmsp.Request, res *tmsp.Response) { log.Info("Bad Transaction", "res", r) // remove from cache (it might be good later) + // note this is an async callback, + // so we need to grab the lock in removeTxFromCacheMap mem.removeTxFromCacheMap(req.GetCheckTx().Tx) // TODO: handle other retcodes diff --git a/test/rpc/counter_test.sh b/test/rpc/counter_test.sh index bece24019..809a22128 100644 --- a/test/rpc/counter_test.sh +++ b/test/rpc/counter_test.sh @@ -13,9 +13,6 @@ function sendTx() { CODE=`echo $RESPONSE | jq .result[1].code` ERROR=`echo $RESPONSE | jq .error` ERROR=$(echo "$ERROR" | tr -d '"') # remove surrounding quotes - echo "ERROR: $ERROR" - echo "CODE: $CODE" - echo "" } # 0 should pass once and get in block, with no error From f3f257a6260ac7ba9873c54602df28d7574cb1a6 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Wed, 6 Jul 2016 17:29:04 -0400 Subject: [PATCH 10/38] fix glide grpc --- glide.lock | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/glide.lock b/glide.lock index e99b9f854..26df8d0f2 100644 --- a/glide.lock +++ b/glide.lock @@ -115,7 +115,7 @@ imports: subpackages: - unix - name: google.golang.org/grpc - version: 88aeffff979aa77aa502cb011423d0a08fa12c5a + version: daeb9cc0f2607997cce611a1458e71b981ce5986 subpackages: - codes - credentials From cadb9e8bfe84c04b523abd85138caa307f4c2a61 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Mon, 11 Jul 2016 12:32:24 -0400 Subject: [PATCH 11/38] mempool: don't remove committed txs from cache --- mempool/mempool.go | 10 ++-------- 1 file changed, 2 insertions(+), 8 deletions(-) diff --git a/mempool/mempool.go b/mempool/mempool.go index 0833cb4a6..6cd2227c2 100644 --- a/mempool/mempool.go +++ b/mempool/mempool.go @@ -304,19 +304,13 @@ func (mem *Mempool) filterTxs(blockTxsMap map[string]struct{}) []types.Tx { goodTxs := make([]types.Tx, 0, mem.txs.Len()) for e := mem.txs.Front(); e != nil; e = e.Next() { memTx := e.Value.(*mempoolTx) - // Remove the tx if its alredy in a block. + // Remove the tx if it's alredy in a block. if _, ok := blockTxsMap[string(memTx.tx)]; ok { // remove from clist mem.txs.Remove(e) e.DetachPrev() - // remove from mempool cache - // we only enforce "at-least once" semantics and - // leave it to the application to implement "only-once" - // via eg. sequence numbers, utxos, etc. - // NOTE: expects caller of filterTxs to hold the lock - // (so we can't use mem.removeTxFromCacheMap) - delete(mem.cacheMap, string(memTx.tx)) + // NOTE: we don't remove committed txs from the cache. continue } // Good tx! From a43d53ac6ecb7364ec7b8b0aa1b4d74f22564f8a Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Mon, 11 Jul 2016 20:40:24 -0400 Subject: [PATCH 12/38] rpc/tests: panic dont t.Fatal. use random txs for broadcast --- rpc/test/client_test.go | 73 +++++++++++++++++++++++++---------------- rpc/test/helpers.go | 13 ++++---- 2 files changed, 50 insertions(+), 36 deletions(-) diff --git a/rpc/test/client_test.go b/rpc/test/client_test.go index e29587fec..47d2ead10 100644 --- a/rpc/test/client_test.go +++ b/rpc/test/client_test.go @@ -2,9 +2,11 @@ package rpctest import ( "bytes" + "crypto/rand" "fmt" "testing" + . "github.com/tendermint/go-common" ctypes "github.com/tendermint/tendermint/rpc/core/types" "github.com/tendermint/tendermint/types" tmsp "github.com/tendermint/tmsp/types" @@ -21,7 +23,7 @@ func TestURIStatus(t *testing.T) { tmResult := new(ctypes.TMResult) _, err := clientURI.Call("status", map[string]interface{}{}, tmResult) if err != nil { - t.Fatal(err) + panic(err) } testStatus(t, tmResult) } @@ -30,7 +32,7 @@ func TestJSONStatus(t *testing.T) { tmResult := new(ctypes.TMResult) _, err := clientJSON.Call("status", []interface{}{}, tmResult) if err != nil { - t.Fatal(err) + panic(err) } testStatus(t, tmResult) } @@ -39,7 +41,7 @@ func testStatus(t *testing.T, statusI interface{}) { tmRes := statusI.(*ctypes.TMResult) status := (*tmRes).(*ctypes.ResultStatus) if status.NodeInfo.Network != chainID { - t.Fatal(fmt.Errorf("ChainID mismatch: got %s expected %s", + panic(Fmt("ChainID mismatch: got %s expected %s", status.NodeInfo.Network, chainID)) } } @@ -47,44 +49,53 @@ func testStatus(t *testing.T, statusI interface{}) { //-------------------------------------------------------------------------------- // broadcast tx sync -var testTx = []byte{0x1, 0x2, 0x3, 0x4, 0x5} +func testTx() []byte { + buf := make([]byte, 16) + _, err := rand.Read(buf) + if err != nil { + panic(err) + } + return buf +} func TestURIBroadcastTxSync(t *testing.T) { config.Set("block_size", 0) defer config.Set("block_size", -1) tmResult := new(ctypes.TMResult) - _, err := clientURI.Call("broadcast_tx_sync", map[string]interface{}{"tx": testTx}, tmResult) + tx := testTx() + _, err := clientURI.Call("broadcast_tx_sync", map[string]interface{}{"tx": tx}, tmResult) if err != nil { - t.Fatal(err) + panic(err) } - testBroadcastTxSync(t, tmResult) + testBroadcastTxSync(t, tmResult, tx) } func TestJSONBroadcastTxSync(t *testing.T) { config.Set("block_size", 0) defer config.Set("block_size", -1) tmResult := new(ctypes.TMResult) - _, err := clientJSON.Call("broadcast_tx_sync", []interface{}{testTx}, tmResult) + tx := testTx() + _, err := clientJSON.Call("broadcast_tx_sync", []interface{}{tx}, tmResult) if err != nil { - t.Fatal(err) + panic(err) } - testBroadcastTxSync(t, tmResult) + testBroadcastTxSync(t, tmResult, tx) } -func testBroadcastTxSync(t *testing.T, resI interface{}) { +func testBroadcastTxSync(t *testing.T, resI interface{}, tx []byte) { tmRes := resI.(*ctypes.TMResult) res := (*tmRes).(*ctypes.ResultBroadcastTx) if res.Code != tmsp.CodeType_OK { - t.Fatalf("BroadcastTxSync got non-zero exit code: %v. %X; %s", res.Code, res.Data, res.Log) + panic(Fmt("BroadcastTxSync got non-zero exit code: %v. %X; %s", res.Code, res.Data, res.Log)) } mem := node.MempoolReactor().Mempool if mem.Size() != 1 { - t.Fatalf("Mempool size should have been 1. Got %d", mem.Size()) + panic(Fmt("Mempool size should have been 1. Got %d", mem.Size())) } txs := mem.Reap(1) - if !bytes.Equal(txs[0], testTx) { - t.Fatalf("Tx in mempool does not match test tx. Got %X, expected %X", txs[0], testTx) + if !bytes.Equal(txs[0], tx) { + panic(Fmt("Tx in mempool does not match test tx. Got %X, expected %X", txs[0], testTx)) } mem.Flush() @@ -95,32 +106,36 @@ func testBroadcastTxSync(t *testing.T, resI interface{}) { func TestURIBroadcastTxCommit(t *testing.T) { tmResult := new(ctypes.TMResult) - _, err := clientURI.Call("broadcast_tx_commit", map[string]interface{}{"tx": testTx}, tmResult) + tx := testTx() + _, err := clientURI.Call("broadcast_tx_commit", map[string]interface{}{"tx": tx}, tmResult) if err != nil { - t.Fatal(err) + panic(err) } - testBroadcastTxCommit(t, tmResult) + testBroadcastTxCommit(t, tmResult, tx) } func TestJSONBroadcastTxCommit(t *testing.T) { tmResult := new(ctypes.TMResult) - _, err := clientJSON.Call("broadcast_tx_commit", []interface{}{testTx}, tmResult) + tx := testTx() + _, err := clientJSON.Call("broadcast_tx_commit", []interface{}{tx}, tmResult) if err != nil { - t.Fatal(err) + panic(err) } - testBroadcastTxCommit(t, tmResult) + testBroadcastTxCommit(t, tmResult, tx) } -func testBroadcastTxCommit(t *testing.T, resI interface{}) { +func testBroadcastTxCommit(t *testing.T, resI interface{}, tx []byte) { tmRes := resI.(*ctypes.TMResult) res := (*tmRes).(*ctypes.ResultBroadcastTx) if res.Code != tmsp.CodeType_OK { - t.Fatalf("BroadcastTxCommit got non-zero exit code: %v. %X; %s", res.Code, res.Data, res.Log) + panic(Fmt("BroadcastTxCommit got non-zero exit code: %v. %X; %s", res.Code, res.Data, res.Log)) } mem := node.MempoolReactor().Mempool if mem.Size() != 0 { - t.Fatalf("Mempool size should have been 0. Got %d", mem.Size()) + panic(Fmt("Mempool size should have been 0. Got %d", mem.Size())) } + + // TODO: find tx in block } //-------------------------------------------------------------------------------- @@ -232,7 +247,7 @@ func TestURIUnsafeSetConfig(t *testing.T) { "value": testCase[2], }, tmResult) if err != nil { - t.Fatal(err) + panic(err) } } testUnsafeSetConfig(t) @@ -243,7 +258,7 @@ func TestJSONUnsafeSetConfig(t *testing.T) { tmResult := new(ctypes.TMResult) _, err := clientJSON.Call("unsafe_set_config", []interface{}{testCase[0], testCase[1], testCase[2]}, tmResult) if err != nil { - t.Fatal(err) + panic(err) } } testUnsafeSetConfig(t) @@ -252,16 +267,16 @@ func TestJSONUnsafeSetConfig(t *testing.T) { func testUnsafeSetConfig(t *testing.T) { s := config.GetString("key1") if s != stringVal { - t.Fatalf("got %v, expected %v", s, stringVal) + panic(Fmt("got %v, expected %v", s, stringVal)) } i := config.GetInt("key2") if i != intVal { - t.Fatalf("got %v, expected %v", i, intVal) + panic(Fmt("got %v, expected %v", i, intVal)) } b := config.GetBool("key3") if b != boolVal { - t.Fatalf("got %v, expected %v", b, boolVal) + panic(Fmt("got %v, expected %v", b, boolVal)) } } diff --git a/rpc/test/helpers.go b/rpc/test/helpers.go index 3713aa821..59709ada2 100644 --- a/rpc/test/helpers.go +++ b/rpc/test/helpers.go @@ -75,7 +75,7 @@ func newNode(ready chan struct{}) { func newWSClient(t *testing.T) *client.WSClient { wsc := client.NewWSClient(websocketAddr, websocketEndpoint) if _, err := wsc.Start(); err != nil { - t.Fatal(err) + panic(err) } return wsc } @@ -83,14 +83,14 @@ func newWSClient(t *testing.T) *client.WSClient { // subscribe to an event func subscribe(t *testing.T, wsc *client.WSClient, eventid string) { if err := wsc.Subscribe(eventid); err != nil { - t.Fatal(err) + panic(err) } } // unsubscribe from an event func unsubscribe(t *testing.T, wsc *client.WSClient, eventid string) { if err := wsc.Unsubscribe(eventid); err != nil { - t.Fatal(err) + panic(err) } } @@ -137,7 +137,7 @@ func waitForEvent(t *testing.T, wsc *client.WSClient, eventid string, dieOnTimeo case <-timeout.C: if dieOnTimeout { wsc.Stop() - t.Fatalf("%s event was not received in time", eventid) + panic(Fmt("%s event was not received in time", eventid)) } // else that's great, we didn't hear the event // and we shouldn't have @@ -146,14 +146,13 @@ func waitForEvent(t *testing.T, wsc *client.WSClient, eventid string, dieOnTimeo // message was received and expected // run the check if err := check(eventid, eventData); err != nil { - t.Fatal(err) // Show the stack trace. + panic(err) // Show the stack trace. } } else { wsc.Stop() - t.Fatalf("%s event was not expected", eventid) + panic(Fmt("%s event was not expected", eventid)) } case err := <-errCh: - t.Fatal(err) panic(err) // Show the stack trace. } From e4c795f15cc879a22e1a809058982d89aa341465 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Mon, 11 Jul 2016 20:40:48 -0400 Subject: [PATCH 13/38] consensus: fix more races in tests --- consensus/common_test.go | 34 ++++++++++++- consensus/state_test.go | 104 +++++++++++++++++---------------------- 2 files changed, 78 insertions(+), 60 deletions(-) diff --git a/consensus/common_test.go b/consensus/common_test.go index a7fce7d15..57bff3704 100644 --- a/consensus/common_test.go +++ b/consensus/common_test.go @@ -128,13 +128,16 @@ func addVoteToFromMany(to *ConsensusState, votes []*types.Vote, froms ...*valida if len(votes) != len(froms) { panic("len(votes) and len(froms) must match") } + for i, from := range froms { addVoteToFrom(to, from, votes[i]) } } func addVoteToFrom(to *ConsensusState, from *validatorStub, vote *types.Vote) { + to.mtx.Lock() // NOTE: wont need this when the vote comes with the index! valIndex, _ := to.Validators.GetByAddress(from.PrivValidator.Address) + to.mtx.Unlock() to.peerMsgQueue <- msgInfo{Msg: &VoteMessage{valIndex, vote}} // added, err := to.TryAddVote(valIndex, vote, "") @@ -158,16 +161,32 @@ func signVoteMany(voteType byte, hash []byte, header types.PartSetHeader, vss .. } // add vote to one cs from another -func signAddVoteToFromMany(voteType byte, to *ConsensusState, hash []byte, header types.PartSetHeader, froms ...*validatorStub) { +// if voteCh is not nil, read all votes +func signAddVoteToFromMany(voteType byte, to *ConsensusState, hash []byte, header types.PartSetHeader, voteCh chan interface{}, froms ...*validatorStub) { + var wg chan struct{} // when done reading all votes + if voteCh != nil { + wg = readVotes(voteCh, len(froms)) + } for _, from := range froms { vote := signVote(from, voteType, hash, header) addVoteToFrom(to, from, vote) } + + if voteCh != nil { + <-wg + } } -func signAddVoteToFrom(voteType byte, to *ConsensusState, from *validatorStub, hash []byte, header types.PartSetHeader) *types.Vote { +func signAddVoteToFrom(voteType byte, to *ConsensusState, from *validatorStub, hash []byte, header types.PartSetHeader, voteCh chan interface{}) *types.Vote { + var wg chan struct{} // when done reading all votes + if voteCh != nil { + wg = readVotes(voteCh, 1) + } vote := signVote(from, voteType, hash, header) addVoteToFrom(to, from, vote) + if voteCh != nil { + <-wg + } return vote } @@ -357,6 +376,17 @@ func subscribeToVoter(cs *ConsensusState, addr []byte) chan interface{} { return voteCh } +func readVotes(ch chan interface{}, reads int) chan struct{} { + wg := make(chan struct{}) + go func() { + for i := 0; i < reads; i++ { + <-ch // read the precommit event + } + close(wg) + }() + return wg +} + func randGenesisState(numValidators int, randPower bool, minPower int64) (*sm.State, []*types.PrivValidator) { db := dbm.NewMemDB() genDoc, privValidators := randGenesisDoc(numValidators, randPower, minPower) diff --git a/consensus/state_test.go b/consensus/state_test.go index c2ab54585..18c74230d 100644 --- a/consensus/state_test.go +++ b/consensus/state_test.go @@ -74,7 +74,7 @@ func TestProposerSelection0(t *testing.T) { <-proposalCh rs := cs1.GetRoundState() - signAddVoteToFromMany(types.VoteTypePrecommit, cs1, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), vss[1:]...) + signAddVoteToFromMany(types.VoteTypePrecommit, cs1, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), nil, vss[1:]...) // wait for new round so next validator is set <-newRoundCh @@ -106,7 +106,7 @@ func TestProposerSelection2(t *testing.T) { } rs := cs1.GetRoundState() - signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, rs.ProposalBlockParts.Header(), vss[1:]...) + signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, rs.ProposalBlockParts.Header(), nil, vss[1:]...) <-newRoundCh // wait for the new round event each round incrementRound(vss[1:]...) @@ -218,14 +218,13 @@ func TestBadProposal(t *testing.T) { validatePrevote(t, cs1, round, vss[0], nil) // add bad prevote from cs2 and wait for it - signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, propBlock.Hash(), propBlock.MakePartSet().Header()) - <-voteCh + signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, propBlock.Hash(), propBlock.MakePartSet().Header(), voteCh) // wait for precommit <-voteCh validatePrecommit(t, cs1, round, 0, vss[0], nil, nil) - signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, propBlock.Hash(), propBlock.MakePartSet().Header()) + signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, propBlock.Hash(), propBlock.MakePartSet().Header(), voteCh) } //---------------------------------------------------------------------------------------------------- @@ -292,12 +291,11 @@ func TestFullRound2(t *testing.T) { <-voteCh // prevote // we should be stuck in limbo waiting for more prevotes - - propBlockHash, propPartsHeader := cs1.ProposalBlock.Hash(), cs1.ProposalBlockParts.Header() + rs := cs1.GetRoundState() + propBlockHash, propPartsHeader := rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header() // prevote arrives from cs2: - signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, propBlockHash, propPartsHeader) - <-voteCh + signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, propBlockHash, propPartsHeader, voteCh) <-voteCh //precommit @@ -307,8 +305,7 @@ func TestFullRound2(t *testing.T) { // we should be stuck in limbo waiting for more precommits // precommit arrives from cs2: - signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, propBlockHash, propPartsHeader) - <-voteCh + signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, propBlockHash, propPartsHeader, voteCh) // wait to finish commit, propose in next height <-newBlockCh @@ -346,8 +343,7 @@ func TestLockNoPOL(t *testing.T) { // we should now be stuck in limbo forever, waiting for more prevotes // prevote arrives from cs2: - signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header()) - <-voteCh // prevote + signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), voteCh) <-voteCh // precommit @@ -360,8 +356,7 @@ func TestLockNoPOL(t *testing.T) { hash := make([]byte, len(theBlockHash)) copy(hash, theBlockHash) hash[0] = byte((hash[0] + 1) % 255) - signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, hash, rs.ProposalBlock.MakePartSet().Header()) - <-voteCh // precommit + signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, hash, rs.ProposalBlock.MakePartSet().Header(), voteCh) // (note we're entering precommit for a second time this round) // but with invalid args. then we enterPrecommitWait, and the timeout to new round @@ -392,8 +387,7 @@ func TestLockNoPOL(t *testing.T) { validatePrevote(t, cs1, 1, vss[0], rs.LockedBlock.Hash()) // add a conflicting prevote from the other validator - signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, hash, rs.ProposalBlock.MakePartSet().Header()) - <-voteCh + signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, hash, rs.ProposalBlock.MakePartSet().Header(), voteCh) // now we're going to enter prevote again, but with invalid args // and then prevote wait, which should timeout. then wait for precommit @@ -407,8 +401,7 @@ func TestLockNoPOL(t *testing.T) { // add conflicting precommit from cs2 // NOTE: in practice we should never get to a point where there are precommits for different blocks at the same round - signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, hash, rs.ProposalBlock.MakePartSet().Header()) - <-voteCh + signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, hash, rs.ProposalBlock.MakePartSet().Header(), voteCh) // (note we're entering precommit for a second time this round, but with invalid args // then we enterPrecommitWait and timeout into NewRound @@ -434,15 +427,13 @@ func TestLockNoPOL(t *testing.T) { validatePrevote(t, cs1, 2, vss[0], rs.LockedBlock.Hash()) - signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, hash, rs.ProposalBlock.MakePartSet().Header()) - <-voteCh + signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, hash, rs.ProposalBlock.MakePartSet().Header(), voteCh) <-timeoutWaitCh // prevote wait <-voteCh // precommit - validatePrecommit(t, cs1, 2, 0, vss[0], nil, theBlockHash) // precommit nil but be locked on proposal - signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, hash, rs.ProposalBlock.MakePartSet().Header()) // NOTE: conflicting precommits at same height - <-voteCh + validatePrecommit(t, cs1, 2, 0, vss[0], nil, theBlockHash) // precommit nil but be locked on proposal + signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, hash, rs.ProposalBlock.MakePartSet().Header(), voteCh) // NOTE: conflicting precommits at same height <-timeoutWaitCh @@ -470,15 +461,13 @@ func TestLockNoPOL(t *testing.T) { // prevote for locked block (not proposal) validatePrevote(t, cs1, 0, vss[0], cs1.LockedBlock.Hash()) - signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, propBlock.Hash(), propBlock.MakePartSet().Header()) - <-voteCh + signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, propBlock.Hash(), propBlock.MakePartSet().Header(), voteCh) <-timeoutWaitCh <-voteCh - validatePrecommit(t, cs1, 2, 0, vss[0], nil, theBlockHash) // precommit nil but locked on proposal - signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, propBlock.Hash(), propBlock.MakePartSet().Header()) // NOTE: conflicting precommits at same height - <-voteCh + validatePrecommit(t, cs1, 2, 0, vss[0], nil, theBlockHash) // precommit nil but locked on proposal + signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, propBlock.Hash(), propBlock.MakePartSet().Header(), voteCh) // NOTE: conflicting precommits at same height } // 4 vals, one precommits, other 3 polka at next round, so we unlock and precomit the polka @@ -510,20 +499,19 @@ func TestLockPOLRelock(t *testing.T) { re := <-proposalCh rs := re.(types.EventDataRoundState).RoundState.(*RoundState) theBlockHash := rs.ProposalBlock.Hash() + theBlockPartsHeader := rs.ProposalBlockParts.Header() <-voteCh // prevote - signAddVoteToFromMany(types.VoteTypePrevote, cs1, cs1.ProposalBlock.Hash(), cs1.ProposalBlockParts.Header(), cs2, cs3, cs4) - _, _, _ = <-voteCh, <-voteCh, <-voteCh // prevotes + signAddVoteToFromMany(types.VoteTypePrevote, cs1, theBlockHash, theBlockPartsHeader, voteCh, cs2, cs3, cs4) <-voteCh // our precommit // the proposed block should now be locked and our precommit added validatePrecommit(t, cs1, 0, 0, vss[0], theBlockHash, theBlockHash) // add precommits from the rest - signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, types.PartSetHeader{}, cs2, cs4) - signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs3, cs1.ProposalBlock.Hash(), cs1.ProposalBlockParts.Header()) - _, _, _ = <-voteCh, <-voteCh, <-voteCh // precommits + signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, types.PartSetHeader{}, voteCh, cs2, cs4) + signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs3, theBlockHash, theBlockPartsHeader, voteCh) // before we timeout to the new round set the new proposal prop, propBlock := decideProposal(cs1, cs2, cs2.Height, cs2.Round+1) @@ -560,8 +548,7 @@ func TestLockPOLRelock(t *testing.T) { validatePrevote(t, cs1, 0, vss[0], theBlockHash) // now lets add prevotes from everyone else for the new block - signAddVoteToFromMany(types.VoteTypePrevote, cs1, propBlockHash, propBlockParts.Header(), cs2, cs3, cs4) - _, _, _ = <-voteCh, <-voteCh, <-voteCh // prevotes + signAddVoteToFromMany(types.VoteTypePrevote, cs1, propBlockHash, propBlockParts.Header(), voteCh, cs2, cs3, cs4) // now either we go to PrevoteWait or Precommit select { @@ -573,8 +560,7 @@ func TestLockPOLRelock(t *testing.T) { // we should have unlocked and locked on the new block validatePrecommit(t, cs1, 1, 1, vss[0], propBlockHash, propBlockHash) - signAddVoteToFromMany(types.VoteTypePrecommit, cs1, propBlockHash, propBlockParts.Header(), cs2, cs3) - _, _ = <-voteCh, <-voteCh + signAddVoteToFromMany(types.VoteTypePrecommit, cs1, propBlockHash, propBlockParts.Header(), voteCh, cs2, cs3) be := <-newBlockCh b := be.(types.EventDataNewBlockHeader) @@ -618,16 +604,18 @@ func TestLockPOLUnlock(t *testing.T) { <-voteCh // prevote - signAddVoteToFromMany(types.VoteTypePrevote, cs1, cs1.ProposalBlock.Hash(), cs1.ProposalBlockParts.Header(), cs2, cs3, cs4) + signAddVoteToFromMany(types.VoteTypePrevote, cs1, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), nil, cs2, cs3, cs4) <-voteCh //precommit // the proposed block should now be locked and our precommit added validatePrecommit(t, cs1, 0, 0, vss[0], theBlockHash, theBlockHash) + rs = cs1.GetRoundState() + // add precommits from the rest - signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, types.PartSetHeader{}, cs2, cs4) - signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs3, cs1.ProposalBlock.Hash(), cs1.ProposalBlockParts.Header()) + signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, types.PartSetHeader{}, nil, cs2, cs4) + signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs3, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), nil) // before we time out into new round, set next proposal block prop, propBlock := decideProposal(cs1, cs2, cs2.Height, cs2.Round+1) @@ -663,7 +651,7 @@ func TestLockPOLUnlock(t *testing.T) { <-voteCh validatePrevote(t, cs1, 0, vss[0], lockedBlockHash) // now lets add prevotes from everyone else for nil (a polka!) - signAddVoteToFromMany(types.VoteTypePrevote, cs1, nil, types.PartSetHeader{}, cs2, cs3, cs4) + signAddVoteToFromMany(types.VoteTypePrevote, cs1, nil, types.PartSetHeader{}, nil, cs2, cs3, cs4) // the polka makes us unlock and precommit nil <-unlockCh @@ -673,7 +661,7 @@ func TestLockPOLUnlock(t *testing.T) { // NOTE: since we don't relock on nil, the lock round is 0 validatePrecommit(t, cs1, 1, 0, vss[0], nil, nil) - signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, types.PartSetHeader{}, cs2, cs3) + signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, types.PartSetHeader{}, nil, cs2, cs3) <-newRoundCh } @@ -717,7 +705,7 @@ func TestLockPOLSafety1(t *testing.T) { log.Warn("old prop", "hash", fmt.Sprintf("%X", propBlock.Hash())) // we do see them precommit nil - signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, types.PartSetHeader{}, cs2, cs3, cs4) + signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, types.PartSetHeader{}, nil, cs2, cs3, cs4) prop, propBlock := decideProposal(cs1, cs2, cs2.Height, cs2.Round+1) propBlockHash := propBlock.Hash() @@ -754,14 +742,14 @@ func TestLockPOLSafety1(t *testing.T) { validatePrevote(t, cs1, 1, vss[0], propBlockHash) // now we see the others prevote for it, so we should lock on it - signAddVoteToFromMany(types.VoteTypePrevote, cs1, propBlockHash, propBlockParts.Header(), cs2, cs3, cs4) + signAddVoteToFromMany(types.VoteTypePrevote, cs1, propBlockHash, propBlockParts.Header(), nil, cs2, cs3, cs4) <-voteCh // precommit // we should have precommitted validatePrecommit(t, cs1, 1, 1, vss[0], propBlockHash, propBlockHash) - signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, types.PartSetHeader{}, cs2, cs3) + signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, types.PartSetHeader{}, nil, cs2, cs3) <-timeoutWaitCh @@ -840,15 +828,15 @@ func TestLockPOLSafety2(t *testing.T) { <-voteCh // prevote - signAddVoteToFromMany(types.VoteTypePrevote, cs1, propBlockHash1, propBlockParts1.Header(), cs2, cs3, cs4) + signAddVoteToFromMany(types.VoteTypePrevote, cs1, propBlockHash1, propBlockParts1.Header(), nil, cs2, cs3, cs4) <-voteCh // precommit // the proposed block should now be locked and our precommit added validatePrecommit(t, cs1, 1, 1, vss[0], propBlockHash1, propBlockHash1) // add precommits from the rest - signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, types.PartSetHeader{}, cs2, cs4) - signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs3, propBlockHash1, propBlockParts1.Header()) + signAddVoteToFromMany(types.VoteTypePrecommit, cs1, nil, types.PartSetHeader{}, nil, cs2, cs4) + signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs3, propBlockHash1, propBlockParts1.Header(), nil) incrementRound(cs2, cs3, cs4) @@ -910,9 +898,9 @@ func TestSlashingPrevotes(t *testing.T) { // we should now be stuck in limbo forever, waiting for more prevotes // add one for a different block should cause us to go into prevote wait - hash := cs1.ProposalBlock.Hash() + hash := rs.ProposalBlock.Hash() hash[0] = byte(hash[0]+1) % 255 - signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, hash, rs.ProposalBlockParts.Header()) + signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, hash, rs.ProposalBlockParts.Header(), nil) <-timeoutWaitCh @@ -920,7 +908,7 @@ func TestSlashingPrevotes(t *testing.T) { // away and ignore more prevotes (and thus fail to slash!) // add the conflicting vote - signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header()) + signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(),nil) // XXX: Check for existence of Dupeout info } @@ -942,7 +930,7 @@ func TestSlashingPrecommits(t *testing.T) { <-voteCh // prevote // add prevote from cs2 - signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header()) + signAddVoteToFrom(types.VoteTypePrevote, cs1, cs2, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(), nil) <-voteCh // precommit @@ -950,13 +938,13 @@ func TestSlashingPrecommits(t *testing.T) { // add one for a different block should cause us to go into prevote wait hash := rs.ProposalBlock.Hash() hash[0] = byte(hash[0]+1) % 255 - signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, hash, rs.ProposalBlockParts.Header()) + signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, hash, rs.ProposalBlockParts.Header(),nil) // NOTE: we have to send the vote for different block first so we don't just go into precommit round right // away and ignore more prevotes (and thus fail to slash!) // add precommit from cs2 - signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header()) + signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, rs.ProposalBlock.Hash(), rs.ProposalBlockParts.Header(),nil) // XXX: Check for existence of Dupeout info } @@ -990,15 +978,15 @@ func TestHalt1(t *testing.T) { <-voteCh // prevote - signAddVoteToFromMany(types.VoteTypePrevote, cs1, propBlock.Hash(), propBlockParts.Header(), cs3, cs4) + signAddVoteToFromMany(types.VoteTypePrevote, cs1, propBlock.Hash(), propBlockParts.Header(), nil, cs3, cs4) <-voteCh // precommit // the proposed block should now be locked and our precommit added validatePrecommit(t, cs1, 0, 0, vss[0], propBlock.Hash(), propBlock.Hash()) // add precommits from the rest - signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, nil, types.PartSetHeader{}) // didnt receive proposal - signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs3, propBlock.Hash(), propBlockParts.Header()) + signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs2, nil, types.PartSetHeader{}, nil) // didnt receive proposal + signAddVoteToFrom(types.VoteTypePrecommit, cs1, cs3, propBlock.Hash(), propBlockParts.Header(), nil) // we receive this later, but cs3 might receive it earlier and with ours will go to commit! precommit4 := signVote(cs4, types.VoteTypePrecommit, propBlock.Hash(), propBlockParts.Header()) From dd788c563190da5b7d068ad51fa49f3d37072ccc Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Mon, 11 Jul 2016 20:54:32 -0400 Subject: [PATCH 14/38] consensus: fix race from OnStop accessing cs.Height --- consensus/state.go | 3 +++ 1 file changed, 3 insertions(+) diff --git a/consensus/state.go b/consensus/state.go index 338568b30..d76c82969 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -321,7 +321,10 @@ func (cs *ConsensusState) startRoutines(maxSteps int) { } func (cs *ConsensusState) OnStop() { + cs.mtx.Lock() // NOTE: OnStop prints the cs.Height, which might be concurrently updated ... cs.QuitService.OnStop() + cs.mtx.Unlock() + if cs.wal != nil && cs.IsRunning() { cs.wal.Wait() } From 47acada2cb0b70aad3370d78fd87c6a352634ac0 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Mon, 11 Jul 2016 21:10:05 -0400 Subject: [PATCH 15/38] consensus: t.Fatal -> panic --- consensus/common_test.go | 8 ++++---- consensus/height_vote_set_test.go | 7 ++++--- consensus/mempool_test.go | 4 ++-- consensus/replay_test.go | 11 ++++++----- consensus/state_test.go | 33 ++++++++++++++++--------------- consensus/wal_test.go | 16 ++++++++------- 6 files changed, 42 insertions(+), 37 deletions(-) diff --git a/consensus/common_test.go b/consensus/common_test.go index 57bff3704..693edbca1 100644 --- a/consensus/common_test.go +++ b/consensus/common_test.go @@ -99,18 +99,18 @@ func changeProposer(t *testing.T, perspectiveOf *ConsensusState, newProposer *va _, v1 := perspectiveOf.Validators.GetByAddress(perspectiveOf.privValidator.Address) v1.Accum, v1.VotingPower = 0, 0 if updated := perspectiveOf.Validators.Update(v1); !updated { - t.Fatal("failed to update validator") + panic("failed to update validator") } _, v2 := perspectiveOf.Validators.GetByAddress(newProposer.Address) v2.Accum, v2.VotingPower = 100, 100 if updated := perspectiveOf.Validators.Update(v2); !updated { - t.Fatal("failed to update validator") + panic("failed to update validator") } // make the proposal propBlock, _ := perspectiveOf.createProposalBlock() if propBlock == nil { - t.Fatal("Failed to create proposal block with cs2") + panic("Failed to create proposal block with cs2") } return propBlock } @@ -120,7 +120,7 @@ func fixVotingPower(t *testing.T, cs1 *ConsensusState, addr2 []byte) { _, v2 := cs1.Validators.GetByAddress(addr2) v1.Accum, v1.VotingPower = v2.Accum, v2.VotingPower if updated := cs1.Validators.Update(v1); !updated { - t.Fatal("failed to update validator") + panic("failed to update validator") } } diff --git a/consensus/height_vote_set_test.go b/consensus/height_vote_set_test.go index 285027162..b5153259d 100644 --- a/consensus/height_vote_set_test.go +++ b/consensus/height_vote_set_test.go @@ -1,10 +1,11 @@ package consensus import ( + "testing" + + . "github.com/tendermint/go-common" "github.com/tendermint/tendermint/config/tendermint_test" "github.com/tendermint/tendermint/types" - - "testing" ) func init() { @@ -45,7 +46,7 @@ func makeVoteHR(t *testing.T, height, round int, privVal *types.PrivValidator) * chainID := config.GetString("chain_id") err := privVal.SignVote(chainID, vote) if err != nil { - t.Fatalf("Error signing vote: %v", err) + panic(Fmt("Error signing vote: %v", err)) return nil } return vote diff --git a/consensus/mempool_test.go b/consensus/mempool_test.go index 14098b863..6eb957c25 100644 --- a/consensus/mempool_test.go +++ b/consensus/mempool_test.go @@ -31,7 +31,7 @@ func TestTxConcurrentWithCommit(t *testing.T) { binary.BigEndian.PutUint64(txBytes, uint64(i)) err := cs.mempool.CheckTx(txBytes, nil) if err != nil { - t.Fatal("Error after CheckTx: %v", err) + panic(Fmt("Error after CheckTx: %v", err)) } // time.Sleep(time.Microsecond * time.Duration(rand.Int63n(3000))) } @@ -47,7 +47,7 @@ func TestTxConcurrentWithCommit(t *testing.T) { case b := <-newBlockCh: nTxs += b.(types.EventDataNewBlock).Block.Header.NumTxs case <-ticker.C: - t.Fatal("Timed out waiting to commit blocks with transactions") + panic("Timed out waiting to commit blocks with transactions") } } } diff --git a/consensus/replay_test.go b/consensus/replay_test.go index d46f68cd1..46e862ed2 100644 --- a/consensus/replay_test.go +++ b/consensus/replay_test.go @@ -6,6 +6,7 @@ import ( "testing" "time" + . "github.com/tendermint/go-common" "github.com/tendermint/tendermint/types" ) @@ -59,12 +60,12 @@ func TestReplayCatchup(t *testing.T) { // write the needed wal to file f, err := ioutil.TempFile(os.TempDir(), "replay_test_") if err != nil { - t.Fatal(err) + panic(err) } name := f.Name() _, err = f.WriteString(testLog) if err != nil { - t.Fatal(err) + panic(err) } f.Close() @@ -83,14 +84,14 @@ func TestReplayCatchup(t *testing.T) { // open wal and run catchup messages openWAL(t, cs, name) if err := cs.catchupReplay(cs.Height); err != nil { - t.Fatalf("Error on catchup replay %v", err) + panic(Fmt("Error on catchup replay %v", err)) } after := time.After(time.Second * 15) select { case <-newBlockCh: case <-after: - t.Fatal("Timed out waiting for new block") + panic("Timed out waiting for new block") } } @@ -98,7 +99,7 @@ func openWAL(t *testing.T, cs *ConsensusState, file string) { // open the wal wal, err := NewWAL(file, config.GetBool("cswal_light")) if err != nil { - t.Fatal(err) + panic(err) } wal.exists = true cs.wal = wal diff --git a/consensus/state_test.go b/consensus/state_test.go index 18c74230d..d311512c2 100644 --- a/consensus/state_test.go +++ b/consensus/state_test.go @@ -8,6 +8,7 @@ import ( "github.com/tendermint/tendermint/config/tendermint_test" //"github.com/tendermint/go-events" + . "github.com/tendermint/go-common" "github.com/tendermint/tendermint/types" ) @@ -67,7 +68,7 @@ func TestProposerSelection0(t *testing.T) { // lets commit a block and ensure proposer for the next height is correct prop := cs1.GetRoundState().Validators.Proposer() if !bytes.Equal(prop.Address, cs1.privValidator.Address) { - t.Fatalf("expected proposer to be validator %d. Got %X", 0, prop.Address) + panic(Fmt("expected proposer to be validator %d. Got %X", 0, prop.Address)) } // wait for complete proposal @@ -81,7 +82,7 @@ func TestProposerSelection0(t *testing.T) { prop = cs1.GetRoundState().Validators.Proposer() if !bytes.Equal(prop.Address, vss[1].Address) { - t.Fatalf("expected proposer to be validator %d. Got %X", 1, prop.Address) + panic(Fmt("expected proposer to be validator %d. Got %X", 1, prop.Address)) } } @@ -102,7 +103,7 @@ func TestProposerSelection2(t *testing.T) { for i := 0; i < len(vss); i++ { prop := cs1.GetRoundState().Validators.Proposer() if !bytes.Equal(prop.Address, vss[(i+2)%len(vss)].Address) { - t.Fatalf("expected proposer to be validator %d. Got %X", (i+2)%len(vss), prop.Address) + panic(Fmt("expected proposer to be validator %d. Got %X", (i+2)%len(vss), prop.Address)) } rs := cs1.GetRoundState() @@ -130,7 +131,7 @@ func TestEnterProposeNoPrivValidator(t *testing.T) { select { case <-timeoutCh: case <-ticker.C: - t.Fatal("Expected EnterPropose to timeout") + panic("Expected EnterPropose to timeout") } @@ -170,7 +171,7 @@ func TestEnterProposeYesPrivValidator(t *testing.T) { ticker := time.NewTicker(cs.timeoutParams.ensureProposeTimeout()) select { case <-timeoutCh: - t.Fatal("Expected EnterPropose not to timeout") + panic("Expected EnterPropose not to timeout") case <-ticker.C: } @@ -200,7 +201,7 @@ func TestBadProposal(t *testing.T) { propBlockParts := propBlock.MakePartSet() proposal := types.NewProposal(cs2.Height, round, propBlockParts.Header(), -1) if err := cs2.SignProposal(config.GetString("chain_id"), proposal); err != nil { - t.Fatal("failed to sign bad proposal", err) + panic("failed to sign bad proposal: " + err.Error()) } // set the proposal block @@ -377,7 +378,7 @@ func TestLockNoPOL(t *testing.T) { rs = re.(types.EventDataRoundState).RoundState.(*RoundState) if rs.ProposalBlock != nil { - t.Fatal("Expected proposal block to be nil") + panic("Expected proposal block to be nil") } // wait to finish prevote @@ -420,7 +421,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()) { - t.Fatalf("Expected proposal block to be locked block. Got %v, Expected %v", rs.ProposalBlock, rs.LockedBlock) + panic(Fmt("Expected proposal block to be locked block. Got %v, Expected %v", rs.ProposalBlock, rs.LockedBlock)) } <-voteCh // prevote @@ -440,7 +441,7 @@ func TestLockNoPOL(t *testing.T) { // before we time out into new round, set next proposal block prop, propBlock := decideProposal(cs1, cs2, cs2.Height, cs2.Round+1) if prop == nil || propBlock == nil { - t.Fatal("Failed to create proposal block with cs2") + panic("Failed to create proposal block with cs2") } incrementRound(cs2) @@ -567,11 +568,11 @@ func TestLockPOLRelock(t *testing.T) { re = <-newRoundCh rs = re.(types.EventDataRoundState).RoundState.(*RoundState) if rs.Height != 2 { - t.Fatal("Expected height to increment") + panic("Expected height to increment") } if !bytes.Equal(b.Header.Hash(), propBlockHash) { - t.Fatal("Expected new block to be proposal block") + panic("Expected new block to be proposal block") } } @@ -699,7 +700,7 @@ func TestLockPOLSafety1(t *testing.T) { _, v1 := cs1.Validators.GetByAddress(vss[0].Address) v1.VotingPower = 1 if updated := cs1.Validators.Update(v1); !updated { - t.Fatal("failed to update validator") + panic("failed to update validator") }*/ log.Warn("old prop", "hash", fmt.Sprintf("%X", propBlock.Hash())) @@ -734,7 +735,7 @@ func TestLockPOLSafety1(t *testing.T) { rs = re.(types.EventDataRoundState).RoundState.(*RoundState) if rs.LockedBlock != nil { - t.Fatal("we should not be locked!") + panic("we should not be locked!") } log.Warn("new prop", "hash", fmt.Sprintf("%X", propBlockHash)) // go to prevote, prevote for proposal block @@ -846,7 +847,7 @@ func TestLockPOLSafety2(t *testing.T) { // in round 2 we see the polkad block from round 0 newProp := types.NewProposal(height, 2, propBlockParts0.Header(), 0) if err := cs3.SignProposal(config.GetString("chain_id"), newProp); err != nil { - t.Fatal(err) + panic(err) } cs1.SetProposalAndBlock(newProp, propBlock0, propBlockParts0, "some peer") addVoteToFromMany(cs1, prevotes, cs2, cs3, cs4) // add the pol votes @@ -865,7 +866,7 @@ func TestLockPOLSafety2(t *testing.T) { select { case <-unlockCh: - t.Fatal("validator unlocked using an old polka") + panic("validator unlocked using an old polka") case <-voteCh: // prevote our locked block } @@ -1016,6 +1017,6 @@ func TestHalt1(t *testing.T) { rs = re.(types.EventDataRoundState).RoundState.(*RoundState) if rs.Height != 2 { - t.Fatal("expected height to increment") + panic("expected height to increment") } } diff --git a/consensus/wal_test.go b/consensus/wal_test.go index 808a5e414..648692e40 100644 --- a/consensus/wal_test.go +++ b/consensus/wal_test.go @@ -6,6 +6,8 @@ import ( "path" "strings" "testing" + + . "github.com/tendermint/go-common" ) var testTxt = `{"time":"2016-01-16T04:42:00.390Z","msg":[1,{"height":28219,"round":0,"step":"RoundStepPrevote"}]} @@ -18,7 +20,7 @@ var testTxt = `{"time":"2016-01-16T04:42:00.390Z","msg":[1,{"height":28219,"roun func TestSeek(t *testing.T) { f, err := ioutil.TempFile(os.TempDir(), "seek_test_") if err != nil { - t.Fatal(err) + panic(err) } stat, _ := f.Stat() @@ -26,13 +28,13 @@ func TestSeek(t *testing.T) { _, err = f.WriteString(testTxt) if err != nil { - t.Fatal(err) + panic(err) } f.Close() wal, err := NewWAL(path.Join(os.TempDir(), name), config.GetBool("cswal_light")) if err != nil { - t.Fatal(err) + panic(err) } keyWord := "Precommit" @@ -43,7 +45,7 @@ func TestSeek(t *testing.T) { return false }) if err != nil { - t.Fatal(err) + panic(err) } // confirm n @@ -58,18 +60,18 @@ func TestSeek(t *testing.T) { // n is lines from the end. spl = spl[i:] if n != len(spl) { - t.Fatalf("Wrong nLines. Got %d, expected %d", n, len(spl)) + panic(Fmt("Wrong nLines. Got %d, expected %d", n, len(spl))) } b, err := ioutil.ReadAll(wal.fp) if err != nil { - t.Fatal(err) + panic(err) } // first char is a \n spl2 := strings.Split(strings.Trim(string(b), "\n"), "\n") for i, s := range spl { if s != spl2[i] { - t.Fatalf("Mismatch. Got %s, expected %s", spl2[i], s) + panic(Fmt("Mismatch. Got %s, expected %s", spl2[i], s)) } } From 33d987759992755680bf570ad6ff7e2115856bb0 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Mon, 11 Jul 2016 21:45:08 -0400 Subject: [PATCH 16/38] consensus: hvs.Reset(height, valSet) --- consensus/height_vote_set.go | 22 ++++++++++++++++------ 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/consensus/height_vote_set.go b/consensus/height_vote_set.go index 93fa94e86..583ef3f6f 100644 --- a/consensus/height_vote_set.go +++ b/consensus/height_vote_set.go @@ -38,18 +38,28 @@ type HeightVoteSet struct { func NewHeightVoteSet(chainID string, height int, valSet *types.ValidatorSet) *HeightVoteSet { hvs := &HeightVoteSet{ - chainID: chainID, - height: height, - valSet: valSet, - roundVoteSets: make(map[int]RoundVoteSet), - peerCatchupRounds: make(map[string]int), + chainID: chainID, } + hvs.Reset(height, valSet) + return hvs +} + +func (hvs *HeightVoteSet) Reset(height int, valSet *types.ValidatorSet) { + hvs.mtx.Lock() + defer hvs.mtx.Unlock() + + hvs.height = height + hvs.valSet = valSet + hvs.roundVoteSets = make(map[int]RoundVoteSet) + hvs.peerCatchupRounds = make(map[string]int) + hvs.addRound(0) hvs.round = 0 - return hvs } func (hvs *HeightVoteSet) Height() int { + hvs.mtx.Lock() + defer hvs.mtx.Unlock() return hvs.height } From adab114674b90daea7eced0952470f3312753982 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Mon, 11 Jul 2016 22:21:59 -0400 Subject: [PATCH 17/38] glide: update go-common --- glide.lock | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/glide.lock b/glide.lock index 26df8d0f2..03693aa4e 100644 --- a/glide.lock +++ b/glide.lock @@ -54,7 +54,7 @@ imports: - name: github.com/tendermint/go-clist version: 634527f5b60fd7c71ca811262493df2ad65ee0ca - name: github.com/tendermint/go-common - version: dcfa46af1341d03b80d32e4901019d1668b978b9 + version: dee6622bf7f811d3ba8638a3f5ffaf8d679aa9d9 - name: github.com/tendermint/go-config version: e64b424499acd0eb9856b88e10c0dff41628c0d6 - name: github.com/tendermint/go-crypto From a1a90fd41f1fb1a8ad6e31573f2f1980353104d3 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Mon, 11 Jul 2016 22:32:20 -0400 Subject: [PATCH 18/38] consensus: increase mempool_test timeout --- consensus/mempool_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/consensus/mempool_test.go b/consensus/mempool_test.go index 6eb957c25..2a063be75 100644 --- a/consensus/mempool_test.go +++ b/consensus/mempool_test.go @@ -41,7 +41,7 @@ func TestTxConcurrentWithCommit(t *testing.T) { go appendTxsRange(0, NTxs) startTestRound(cs, height, round) - ticker := time.NewTicker(time.Second * 5) + ticker := time.NewTicker(time.Second * 20) for nTxs := 0; nTxs < NTxs; { select { case b := <-newBlockCh: From f44336d57df2f1a8bfb26083402ee9bcd5ff7229 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Mon, 11 Jul 2016 23:07:21 -0400 Subject: [PATCH 19/38] consensus: don't print shared vars in cs.String() --- consensus/state.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/consensus/state.go b/consensus/state.go index d76c82969..4000b072f 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -268,7 +268,8 @@ func (cs *ConsensusState) SetEventSwitch(evsw *events.EventSwitch) { } func (cs *ConsensusState) String() string { - return Fmt("ConsensusState(H:%v R:%v S:%v", cs.Height, cs.Round, cs.Step) + // better not to access shared variables + return Fmt("ConsensusState") //(H:%v R:%v S:%v", cs.Height, cs.Round, cs.Step) } func (cs *ConsensusState) GetState() *sm.State { @@ -321,9 +322,7 @@ func (cs *ConsensusState) startRoutines(maxSteps int) { } func (cs *ConsensusState) OnStop() { - cs.mtx.Lock() // NOTE: OnStop prints the cs.Height, which might be concurrently updated ... cs.QuitService.OnStop() - cs.mtx.Unlock() if cs.wal != nil && cs.IsRunning() { cs.wal.Wait() From 54357bcf8f8ed968e446c3e0d665a428a6b16dab Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Tue, 12 Jul 2016 14:08:59 -0400 Subject: [PATCH 20/38] crank consensus timeout parameters for tests (see #236) --- config/tendermint_test/config.go | 14 +++++++------- consensus/state_test.go | 2 ++ 2 files changed, 9 insertions(+), 7 deletions(-) diff --git a/config/tendermint_test/config.go b/config/tendermint_test/config.go index 689ba794c..c53f8bfbb 100644 --- a/config/tendermint_test/config.go +++ b/config/tendermint_test/config.go @@ -86,13 +86,13 @@ func ResetConfig(localPath string) cfg.Config { mapConfig.SetDefault("block_size", 10000) mapConfig.SetDefault("disable_data_hash", false) - mapConfig.SetDefault("timeout_propose", 100) - mapConfig.SetDefault("timeout_propose_delta", 1) - mapConfig.SetDefault("timeout_prevote", 1) - mapConfig.SetDefault("timeout_prevote_delta", 1) - mapConfig.SetDefault("timeout_precommit", 1) - mapConfig.SetDefault("timeout_precommit_delta", 1) - mapConfig.SetDefault("timeout_commit", 1) + mapConfig.SetDefault("timeout_propose", 3000) + mapConfig.SetDefault("timeout_propose_delta", 1000) + mapConfig.SetDefault("timeout_prevote", 2000) + mapConfig.SetDefault("timeout_prevote_delta", 1000) + mapConfig.SetDefault("timeout_precommit", 2000) + mapConfig.SetDefault("timeout_precommit_delta", 1000) + mapConfig.SetDefault("timeout_commit", 1000) mapConfig.SetDefault("mempool_recheck", true) mapConfig.SetDefault("mempool_recheck_empty", true) mapConfig.SetDefault("mempool_broadcast", true) diff --git a/consensus/state_test.go b/consensus/state_test.go index d311512c2..9a8d60905 100644 --- a/consensus/state_test.go +++ b/consensus/state_test.go @@ -554,6 +554,8 @@ func TestLockPOLRelock(t *testing.T) { // now either we go to PrevoteWait or Precommit select { case <-timeoutWaitCh: // we're in PrevoteWait, go to Precommit + // XXX: there's no guarantee we see the polka, this might be a precommit for nil, + // in which case the test fails! <-voteCh case <-voteCh: // we went straight to Precommit } From 7bf34d1fae5c1c8b7364f34e2a53dc0d7660d4dc Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Tue, 12 Jul 2016 14:58:06 -0400 Subject: [PATCH 21/38] rpc: unsafe_flush_mempool. closes #190 --- rpc/core/dev.go | 5 +++++ rpc/core/mempool.go | 2 +- rpc/core/routes.go | 9 +++++++++ rpc/core/types/responses.go | 4 ++++ 4 files changed, 19 insertions(+), 1 deletion(-) diff --git a/rpc/core/dev.go b/rpc/core/dev.go index 2b8dfb8f3..6ae2014b2 100644 --- a/rpc/core/dev.go +++ b/rpc/core/dev.go @@ -9,6 +9,11 @@ import ( ctypes "github.com/tendermint/tendermint/rpc/core/types" ) +func UnsafeFlushMempool() (*ctypes.ResultUnsafeFlushMempool, error) { + mempoolReactor.Mempool.Flush() + return &ctypes.ResultUnsafeFlushMempool{}, nil +} + func UnsafeSetConfig(typ, key, value string) (*ctypes.ResultUnsafeSetConfig, error) { switch typ { case "string": diff --git a/rpc/core/mempool.go b/rpc/core/mempool.go index cf3ec919c..bef85c796 100644 --- a/rpc/core/mempool.go +++ b/rpc/core/mempool.go @@ -103,7 +103,7 @@ func BroadcastTxCommit(tx types.Tx) (*ctypes.ResultBroadcastTx, error) { } func UnconfirmedTxs() (*ctypes.ResultUnconfirmedTxs, error) { - txs := mempoolReactor.Mempool.Reap(0) + txs := mempoolReactor.Mempool.Reap(-1) return &ctypes.ResultUnconfirmedTxs{len(txs), txs}, nil } diff --git a/rpc/core/routes.go b/rpc/core/routes.go index b8938f6bc..78a0b6187 100644 --- a/rpc/core/routes.go +++ b/rpc/core/routes.go @@ -25,6 +25,7 @@ var Routes = map[string]*rpc.RPCFunc{ "unconfirmed_txs": rpc.NewRPCFunc(UnconfirmedTxsResult, ""), "num_unconfirmed_txs": rpc.NewRPCFunc(NumUnconfirmedTxsResult, ""), + "unsafe_flush_mempool": rpc.NewRPCFunc(UnsafeFlushMempool, ""), "unsafe_set_config": rpc.NewRPCFunc(UnsafeSetConfigResult, "type,key,value"), "unsafe_start_cpu_profiler": rpc.NewRPCFunc(UnsafeStartCPUProfilerResult, "filename"), "unsafe_stop_cpu_profiler": rpc.NewRPCFunc(UnsafeStopCPUProfilerResult, ""), @@ -151,6 +152,14 @@ func BroadcastTxAsyncResult(tx []byte) (ctypes.TMResult, error) { } } +func UnsafeFlushMempoolResult() (ctypes.TMResult, error) { + if r, err := UnsafeFlushMempool(); err != nil { + return nil, err + } else { + return r, nil + } +} + func UnsafeSetConfigResult(typ, key, value string) (ctypes.TMResult, error) { if r, err := UnsafeSetConfig(typ, key, value); err != nil { return nil, err diff --git a/rpc/core/types/responses.go b/rpc/core/types/responses.go index d08892e8c..c1eebb6e1 100644 --- a/rpc/core/types/responses.go +++ b/rpc/core/types/responses.go @@ -68,6 +68,8 @@ type ResultUnconfirmedTxs struct { Txs []types.Tx `json:"txs"` } +type ResultUnsafeFlushMempool struct{} + type ResultUnsafeSetConfig struct{} type ResultUnsafeProfile struct{} @@ -115,6 +117,7 @@ const ( ResultTypeUnsafeStartCPUProfiler = byte(0xa1) ResultTypeUnsafeStopCPUProfiler = byte(0xa2) ResultTypeUnsafeWriteHeapProfile = byte(0xa3) + ResultTypeUnsafeFlushMempool = byte(0xa4) ) type TMResult interface { @@ -141,4 +144,5 @@ var _ = wire.RegisterInterface( wire.ConcreteType{&ResultUnsafeProfile{}, ResultTypeUnsafeStartCPUProfiler}, wire.ConcreteType{&ResultUnsafeProfile{}, ResultTypeUnsafeStopCPUProfiler}, wire.ConcreteType{&ResultUnsafeProfile{}, ResultTypeUnsafeWriteHeapProfile}, + wire.ConcreteType{&ResultUnsafeFlushMempool{}, ResultTypeUnsafeFlushMempool}, ) From 05136ff09d76e6f3439fe0533c8006a01e0cc482 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Tue, 12 Jul 2016 14:58:16 -0400 Subject: [PATCH 22/38] update some comments --- consensus/state.go | 2 +- types/tx.go | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/consensus/state.go b/consensus/state.go index 4000b072f..50488f8d1 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -1235,7 +1235,7 @@ func (cs *ConsensusState) finalizeCommit(height int) { PanicQ(Fmt("Commit failed for application: %v", err)) } - // txs committed and removed from mempool, fire events + // txs committed, bad ones removed from mepool; fire events eventCache.Flush() // Save to blockStore. diff --git a/types/tx.go b/types/tx.go index 40a042395..a67206d4e 100644 --- a/types/tx.go +++ b/types/tx.go @@ -8,7 +8,8 @@ type Tx []byte // NOTE: this is the hash of the go-wire encoded Tx. // Tx has no types at this level, so just length-prefixed. -// Maybe it should just be the hash of the bytes tho? +// Alternatively, it may make sense to add types here and let +// []byte be type 0x1 so we can have versioned txs if need be in the future. func (tx Tx) Hash() []byte { return merkle.SimpleHashFromBinary(tx) } From eac43dc62b0d2db37b9f14a8b1850f034470482f Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Sat, 23 Jul 2016 12:13:41 -0400 Subject: [PATCH 23/38] glide: update lock and add util scripts --- glide.lock | 12 +++---- scripts/{ => debora}/unsafe_debug_net.sh | 0 scripts/{ => debora}/unsafe_reset_net.sh | 0 scripts/{ => debora}/unsafe_start_group.sh | 0 scripts/{ => debora}/unsafe_stop_group.sh | 0 scripts/{ => debora}/unsafe_upgrade_barak.sh | 0 scripts/{ => debora}/unsafe_upgrade_group.sh | 0 scripts/glide/glide_status.sh | 34 ++++++++++++++++++++ scripts/glide/glide_update.sh | 23 +++++++++++++ 9 files changed, 63 insertions(+), 6 deletions(-) rename scripts/{ => debora}/unsafe_debug_net.sh (100%) rename scripts/{ => debora}/unsafe_reset_net.sh (100%) rename scripts/{ => debora}/unsafe_start_group.sh (100%) rename scripts/{ => debora}/unsafe_stop_group.sh (100%) rename scripts/{ => debora}/unsafe_upgrade_barak.sh (100%) rename scripts/{ => debora}/unsafe_upgrade_group.sh (100%) create mode 100644 scripts/glide/glide_status.sh create mode 100644 scripts/glide/glide_update.sh diff --git a/glide.lock b/glide.lock index 03693aa4e..910814635 100644 --- a/glide.lock +++ b/glide.lock @@ -52,7 +52,7 @@ imports: - name: github.com/tendermint/flowcontrol version: 84d9671090430e8ec80e35b339907e0579b999eb - name: github.com/tendermint/go-clist - version: 634527f5b60fd7c71ca811262493df2ad65ee0ca + version: 3baa390bbaf7634251c42ad69a8682e7e3990552 - name: github.com/tendermint/go-common version: dee6622bf7f811d3ba8638a3f5ffaf8d679aa9d9 - name: github.com/tendermint/go-config @@ -62,9 +62,9 @@ imports: - name: github.com/tendermint/go-db version: 31fdd21c7eaeed53e0ea7ca597fb1e960e2988a5 - name: github.com/tendermint/go-events - version: 7b75ca7bb55aa25e9ef765eb8c0b69486b227357 + version: 48fa21511b259278b871a37b6951da2d5bef698d - name: github.com/tendermint/go-logger - version: 529efe50eab1a8a9c111d55f4de4ecd95f482761 + version: cefb3a45c0bf3c493a04e9bcd9b1540528be59f2 - name: github.com/tendermint/go-merkle version: 05042c6ab9cad51d12e4cecf717ae68e3b1409a8 - name: github.com/tendermint/go-p2p @@ -72,7 +72,7 @@ imports: subpackages: - upnp - name: github.com/tendermint/go-rpc - version: 1d9e89812adc202811b7fb8e9e0837e73adadb43 + version: dea910cd3e71bbfaf1973fd7ba295f0ee515a25f subpackages: - client - server @@ -80,11 +80,11 @@ imports: - name: github.com/tendermint/go-wire version: 3b0adbc86ed8425eaed98516165b6788d9f4de7a - name: github.com/tendermint/log15 - version: 9545b249b3aacafa97f79e0838b02b274adc6f5f + version: 6e460758f10ef42a4724b8e4a82fee59aaa0e41d subpackages: - term - name: github.com/tendermint/tmsp - version: 49a67aee8a7984a68eabe2c45ff6eb0ff51e31f9 + version: ba11348508939e9d273cdc1cc476c5c611e14e66 subpackages: - client - example/dummy diff --git a/scripts/unsafe_debug_net.sh b/scripts/debora/unsafe_debug_net.sh similarity index 100% rename from scripts/unsafe_debug_net.sh rename to scripts/debora/unsafe_debug_net.sh diff --git a/scripts/unsafe_reset_net.sh b/scripts/debora/unsafe_reset_net.sh similarity index 100% rename from scripts/unsafe_reset_net.sh rename to scripts/debora/unsafe_reset_net.sh diff --git a/scripts/unsafe_start_group.sh b/scripts/debora/unsafe_start_group.sh similarity index 100% rename from scripts/unsafe_start_group.sh rename to scripts/debora/unsafe_start_group.sh diff --git a/scripts/unsafe_stop_group.sh b/scripts/debora/unsafe_stop_group.sh similarity index 100% rename from scripts/unsafe_stop_group.sh rename to scripts/debora/unsafe_stop_group.sh diff --git a/scripts/unsafe_upgrade_barak.sh b/scripts/debora/unsafe_upgrade_barak.sh similarity index 100% rename from scripts/unsafe_upgrade_barak.sh rename to scripts/debora/unsafe_upgrade_barak.sh diff --git a/scripts/unsafe_upgrade_group.sh b/scripts/debora/unsafe_upgrade_group.sh similarity index 100% rename from scripts/unsafe_upgrade_group.sh rename to scripts/debora/unsafe_upgrade_group.sh diff --git a/scripts/glide/glide_status.sh b/scripts/glide/glide_status.sh new file mode 100644 index 000000000..11e573ae7 --- /dev/null +++ b/scripts/glide/glide_status.sh @@ -0,0 +1,34 @@ +#! /bin/bash + +# for every github.com/tendermint dependency, warn is if its not synced with origin/master + +GLIDE=$1 + +# make list of libs +LIBS=($(grep "github.com/tendermint" $GLIDE | awk '{print $3}')) + + +for lib in "${LIBS[@]}"; do + # get vendored commit + VENDORED=`grep -A1 $lib $GLIDE | grep -v $lib | awk '{print $2}'` + PWD=`pwd` + cd $GOPATH/src/$lib + MASTER=`git rev-parse origin/master` + HEAD=`git rev-parse HEAD` + cd $PWD + + if [[ "$VENDORED" != "$MASTER" ]]; then + echo "" + if [[ "$VENDORED" != "$HEAD" ]]; then + echo "Vendored version of $lib differs from origin/master and HEAD" + echo "Vendored: $VENDORED" + echo "Master: $MASTER" + echo "Head: $HEAD" + else + echo "Vendored version of $lib differs from origin/master but matches HEAD" + echo "Vendored: $VENDORED" + echo "Master: $MASTER" + fi + fi +done + diff --git a/scripts/glide/glide_update.sh b/scripts/glide/glide_update.sh new file mode 100644 index 000000000..2d78da4f3 --- /dev/null +++ b/scripts/glide/glide_update.sh @@ -0,0 +1,23 @@ +#! /bin/bash +set -euo pipefail +IFS=$'\n\t' + +# script to update the given dependency in the glide.lock file with the checked out branch on the local host + +GLIDE=$1 +LIB=$2 + +# get vendored commit for given lib +function parseGlide() { + cat $1 | grep -A1 $2 | grep -v $2 | awk '{print $2}' +} + +OLD_COMMIT=`parseGlide $GLIDE $LIB` + +PWD=`pwd` +cd $GOPATH/src/github.com/tendermint/$LIB + +NEW_COMMIT=$(git rev-parse HEAD) + +cd $PWD +sed -i "s/$OLD_COMMIT/$NEW_COMMIT/g" $GLIDE From cb6c3fb0b6541f1a7a483291049c18d494ba3c8b Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Sat, 23 Jul 2016 12:48:30 -0400 Subject: [PATCH 24/38] integrations tests and coverage badge --- Makefile | 3 +++ README.md | 1 + circle.yml | 7 ++++++ glide.lock | 2 +- test/Dockerfile | 8 ++++++ test/test.sh | 14 +++++++++++ test/test_libs.sh | 63 +++++++++++++++++++++++++++++++++++++++++++++++ 7 files changed, 97 insertions(+), 1 deletion(-) create mode 100644 test/Dockerfile create mode 100644 test/test.sh create mode 100644 test/test_libs.sh diff --git a/Makefile b/Makefile index ea66d8207..94736921b 100644 --- a/Makefile +++ b/Makefile @@ -24,6 +24,9 @@ test: build test_race: build go test -race `${NOVENDOR}` +test_integrations: + bash ./test/test.sh + test100: build for i in {1..100}; do make test; done diff --git a/README.md b/README.md index 654e68196..38e2443d9 100644 --- a/README.md +++ b/README.md @@ -2,6 +2,7 @@ Simple, Secure, Scalable Blockchain Platform [![CircleCI](https://circleci.com/gh/tendermint/tendermint.svg?style=svg)](https://circleci.com/gh/tendermint/tendermint) +[![codecov](https://codecov.io/gh/tendermint/tendermint/branch/develop/graph/badge.svg)](https://codecov.io/gh/tendermint/tendermint) _NOTE: This is yet pre-alpha non-production-quality software._ diff --git a/circle.yml b/circle.yml index 98dd22283..ab7c23b96 100644 --- a/circle.yml +++ b/circle.yml @@ -16,8 +16,15 @@ checkout: dependencies: override: + - sudo curl -sSL -o /usr/bin/docker http://s3-external-1.amazonaws.com/circle-downloads/docker-1.9.1-circleci; sudo chmod 0755 /usr/bin/docker + - sudo service docker start + - go version + - docker version - "cd $REPO && make get_vendor_deps" test: override: - "cd $REPO && make test_race" + - "cd $REPO && make test_integrations" + post: + - bash <(curl -s https://codecov.io/bash) diff --git a/glide.lock b/glide.lock index 910814635..4a4cbf9fd 100644 --- a/glide.lock +++ b/glide.lock @@ -80,7 +80,7 @@ imports: - name: github.com/tendermint/go-wire version: 3b0adbc86ed8425eaed98516165b6788d9f4de7a - name: github.com/tendermint/log15 - version: 6e460758f10ef42a4724b8e4a82fee59aaa0e41d + version: 9545b249b3aacafa97f79e0838b02b274adc6f5f subpackages: - term - name: github.com/tendermint/tmsp diff --git a/test/Dockerfile b/test/Dockerfile new file mode 100644 index 000000000..613780e99 --- /dev/null +++ b/test/Dockerfile @@ -0,0 +1,8 @@ +# Pull base image. +FROM golang:1.5 + +COPY test/test_libs.sh / +COPY glide.lock /glide.lock + +ENV GLIDE /glide.lock + diff --git a/test/test.sh b/test/test.sh new file mode 100644 index 000000000..b9d15d351 --- /dev/null +++ b/test/test.sh @@ -0,0 +1,14 @@ +#! /bin/bash + +# integrations test! +# if we pushed to STAGING or MASTER, +# run the integrations tests. + +BRANCH=`git rev-parse --abbrev-ref HEAD` +echo "Current branch: $BRANCH" + +if [[ "$BRANCH" == "master" || "$BRANCH" == "staging" ]]; then + docker build -t tester -f ./test/Dockerfile . + docker run -t tester bash /test_libs.sh +fi + diff --git a/test/test_libs.sh b/test/test_libs.sh new file mode 100644 index 000000000..6824e3141 --- /dev/null +++ b/test/test_libs.sh @@ -0,0 +1,63 @@ +#! /bin/bash + +# set glide.lock path +if [[ "$GLIDE" == "" ]]; then + GLIDE=$GOPATH/src/github.com/tendermint/tendermint/glide.lock +fi + +# get vendored commit for given lib +function parseGlide() { + cat $1 | grep -A1 $2 | grep -v $2 | awk '{print $2}' +} + +# fetch and checkout vendored dep +function getDep() { + lib=$1 + echo "----------------------------------" + echo "Getting $lib ..." + go get -t github.com/tendermint/$lib/... + + VENDORED=$(parseGlide $GLIDE $lib) + cd $GOPATH/src/github.com/tendermint/$lib + MASTER=$(git rev-parse origin/master) + + if [[ "$VENDORED" != "$MASTER" ]]; then + echo "... VENDORED != MASTER ($VENDORED != $MASTER)" + echo "... Checking out commit $VENDORED" + git checkout $VENDORED &> /dev/null + fi +} + +#################### +# libs we depend on +#################### + +LIBS_GO_TEST=(go-clist go-common go-config go-crypto go-db go-events go-merkle go-p2p) +LIBS_MAKE_TEST=(go-rpc go-wire tmsp) + +for lib in "${LIBS_GO_TEST[@]}"; do + getDep $lib + + echo "Testing $lib ..." + go test --race github.com/tendermint/$lib/... + if [[ "$?" != 0 ]]; then + echo "FAIL" + exit 1 + fi +done + + +for lib in "${LIBS_MAKE_TEST[@]}"; do + getDep $lib + + echo "Testing $lib ..." + cd $GOPATH/src/github.com/tendermint/$lib + make test + if [[ "$?" != 0 ]]; then + echo "FAIL" + exit 1 + fi +done + +echo "" +echo "PASS" From 8fd216b4f65bbe79ebf4c83a63e241a10e3f56d2 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Sat, 23 Jul 2016 19:38:44 -0400 Subject: [PATCH 25/38] run CI tests in docker container --- Makefile | 2 +- circle.yml | 2 -- glide.lock | 2 +- test/Dockerfile | 11 +++++------ test/run_test.sh | 14 ++++++++++++++ test/test.sh | 22 ++++++++++------------ 6 files changed, 31 insertions(+), 22 deletions(-) create mode 100644 test/run_test.sh diff --git a/Makefile b/Makefile index 94736921b..6a3b2381f 100644 --- a/Makefile +++ b/Makefile @@ -24,7 +24,7 @@ test: build test_race: build go test -race `${NOVENDOR}` -test_integrations: +test_integrations: bash ./test/test.sh test100: build diff --git a/circle.yml b/circle.yml index ab7c23b96..95d5bfabd 100644 --- a/circle.yml +++ b/circle.yml @@ -20,11 +20,9 @@ dependencies: - sudo service docker start - go version - docker version - - "cd $REPO && make get_vendor_deps" test: override: - - "cd $REPO && make test_race" - "cd $REPO && make test_integrations" post: - bash <(curl -s https://codecov.io/bash) diff --git a/glide.lock b/glide.lock index 4a4cbf9fd..ce3018940 100644 --- a/glide.lock +++ b/glide.lock @@ -45,7 +45,7 @@ imports: - leveldb/table - leveldb/util - name: github.com/tendermint/ed25519 - version: fdac6641497281ed1cc368687ec6377e96e02b24 + version: 1f52c6f8b8a5c7908aff4497c186af344b428925 subpackages: - extra25519 - edwards25519 diff --git a/test/Dockerfile b/test/Dockerfile index 613780e99..9978d3674 100644 --- a/test/Dockerfile +++ b/test/Dockerfile @@ -1,8 +1,7 @@ # Pull base image. -FROM golang:1.5 - -COPY test/test_libs.sh / -COPY glide.lock /glide.lock - -ENV GLIDE /glide.lock +FROM golang:1.6 +ENV TENDERMINT_ORG $GOPATH/src/github.com/tendermint/ +RUN mkdir -p $TENDERMINT_ORG +COPY . $TENDERMINT_ORG/tendermint +WORKDIR $TENDERMINT_ORG/tendermint diff --git a/test/run_test.sh b/test/run_test.sh new file mode 100644 index 000000000..67d1f23ca --- /dev/null +++ b/test/run_test.sh @@ -0,0 +1,14 @@ +#! /bin/bash + +echo `pwd` + +BRANCH=`git rev-parse --abbrev-ref HEAD` +echo "Current branch: $BRANCH" + +make get_vendor_deps + +make test_race + +if [[ "$BRANCH" == "master" || "$BRANCH" == "staging" ]]; then + bash test/test_libs.sh +fi diff --git a/test/test.sh b/test/test.sh index b9d15d351..5c646c27a 100644 --- a/test/test.sh +++ b/test/test.sh @@ -1,14 +1,12 @@ #! /bin/bash -# integrations test! -# if we pushed to STAGING or MASTER, -# run the integrations tests. - -BRANCH=`git rev-parse --abbrev-ref HEAD` -echo "Current branch: $BRANCH" - -if [[ "$BRANCH" == "master" || "$BRANCH" == "staging" ]]; then - docker build -t tester -f ./test/Dockerfile . - docker run -t tester bash /test_libs.sh -fi - +# integrations test +# this is the script run by eg CircleCI. +# It creates a docker container, +# installs the dependencies, +# and runs the tests. +# If we pushed to STAGING or MASTER, +# it will also run the tests for all dependencies + +docker build -t tester -f ./test/Dockerfile . +docker run -t tester bash test/run_test.sh From 8987d29beeb0d28d0999c5af3e7f2dd44350efd4 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Sun, 24 Jul 2016 14:08:47 -0400 Subject: [PATCH 26/38] tests: broadcast_tx with tmsp; p2p --- consensus/state.go | 1 + scripts/glide/parse.sh | 7 ++ scripts/glide/{glide_status.sh => status.sh} | 0 scripts/glide/{glide_update.sh => update.sh} | 7 +- scripts/install_tmsp_apps.sh | 11 +++ test/Dockerfile | 16 ++++ test/{rpc => broadcast_tx}/clean.sh | 0 test/{rpc => broadcast_tx}/counter_test.sh | 0 test/{rpc => broadcast_tx}/dummy_test.sh | 11 ++- test/{rpc => broadcast_tx}/test.sh | 37 +++++++-- test/p2p/clean.sh | 4 + test/p2p/data/app/init.sh | 9 +++ test/p2p/data/chain_config.json | 53 +++++++++++++ test/p2p/data/core/init.sh | 20 +++++ test/p2p/data/data/init.sh | 7 ++ test/p2p/data/mach1/core/genesis.json | 39 ++++++++++ test/p2p/data/mach1/core/priv_validator.json | 14 ++++ test/p2p/data/mach2/core/genesis.json | 39 ++++++++++ test/p2p/data/mach2/core/priv_validator.json | 14 ++++ test/p2p/data/mach3/core/genesis.json | 39 ++++++++++ test/p2p/data/mach3/core/priv_validator.json | 14 ++++ test/p2p/data/mach4/core/genesis.json | 39 ++++++++++ test/p2p/data/mach4/core/priv_validator.json | 14 ++++ test/p2p/local_testnet.sh | 28 +++++++ test/p2p/run_test.sh | 79 ++++++++++++++++++++ test/p2p/test.sh | 10 +++ test/p2p/test_client.sh | 16 ++++ test/run_test.sh | 12 ++- test/test.sh | 11 +++ 29 files changed, 536 insertions(+), 15 deletions(-) create mode 100644 scripts/glide/parse.sh rename scripts/glide/{glide_status.sh => status.sh} (100%) rename scripts/glide/{glide_update.sh => update.sh} (67%) create mode 100644 scripts/install_tmsp_apps.sh rename test/{rpc => broadcast_tx}/clean.sh (100%) rename test/{rpc => broadcast_tx}/counter_test.sh (100%) rename test/{rpc => broadcast_tx}/dummy_test.sh (84%) rename test/{rpc => broadcast_tx}/test.sh (55%) create mode 100644 test/p2p/clean.sh create mode 100755 test/p2p/data/app/init.sh create mode 100644 test/p2p/data/chain_config.json create mode 100755 test/p2p/data/core/init.sh create mode 100755 test/p2p/data/data/init.sh create mode 100644 test/p2p/data/mach1/core/genesis.json create mode 100644 test/p2p/data/mach1/core/priv_validator.json create mode 100644 test/p2p/data/mach2/core/genesis.json create mode 100644 test/p2p/data/mach2/core/priv_validator.json create mode 100644 test/p2p/data/mach3/core/genesis.json create mode 100644 test/p2p/data/mach3/core/priv_validator.json create mode 100644 test/p2p/data/mach4/core/genesis.json create mode 100644 test/p2p/data/mach4/core/priv_validator.json create mode 100644 test/p2p/local_testnet.sh create mode 100644 test/p2p/run_test.sh create mode 100644 test/p2p/test.sh create mode 100644 test/p2p/test_client.sh diff --git a/consensus/state.go b/consensus/state.go index 50488f8d1..f89de3921 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -1236,6 +1236,7 @@ func (cs *ConsensusState) finalizeCommit(height int) { } // txs committed, bad ones removed from mepool; fire events + // NOTE: the block.AppHash wont reflect these txs until the next block eventCache.Flush() // Save to blockStore. diff --git a/scripts/glide/parse.sh b/scripts/glide/parse.sh new file mode 100644 index 000000000..a2f68d540 --- /dev/null +++ b/scripts/glide/parse.sh @@ -0,0 +1,7 @@ +#! /bin/bash +set -euo pipefail + +GLIDE=$1 +LIB=$2 + +cat $GLIDE | grep -A1 $LIB | grep -v $LIB | awk '{print $2}' diff --git a/scripts/glide/glide_status.sh b/scripts/glide/status.sh similarity index 100% rename from scripts/glide/glide_status.sh rename to scripts/glide/status.sh diff --git a/scripts/glide/glide_update.sh b/scripts/glide/update.sh similarity index 67% rename from scripts/glide/glide_update.sh rename to scripts/glide/update.sh index 2d78da4f3..630859d84 100644 --- a/scripts/glide/glide_update.sh +++ b/scripts/glide/update.sh @@ -7,12 +7,7 @@ IFS=$'\n\t' GLIDE=$1 LIB=$2 -# get vendored commit for given lib -function parseGlide() { - cat $1 | grep -A1 $2 | grep -v $2 | awk '{print $2}' -} - -OLD_COMMIT=`parseGlide $GLIDE $LIB` +OLD_COMMIT=`bash scripts/glide/parse.sh $GLIDE $LIB` PWD=`pwd` cd $GOPATH/src/github.com/tendermint/$LIB diff --git a/scripts/install_tmsp_apps.sh b/scripts/install_tmsp_apps.sh new file mode 100644 index 000000000..035bc8d3e --- /dev/null +++ b/scripts/install_tmsp_apps.sh @@ -0,0 +1,11 @@ +#! /bin/bash + +go get github.com/tendermint/tmsp/... + +COMMIT=`bash scripts/glide/parse.sh $(pwd)/glide.lock tmsp` + +cd $GOPATH/src/github.com/tendermint/tmsp +git checkout $COMMIT +go install ./cmd/... + + diff --git a/test/Dockerfile b/test/Dockerfile index 9978d3674..26995de38 100644 --- a/test/Dockerfile +++ b/test/Dockerfile @@ -1,7 +1,23 @@ # Pull base image. FROM golang:1.6 +# Grab deps (jq, hexdump) +RUN apt-get update && \ + apt-get install -y --no-install-recommends \ + jq bsdmainutils && \ + rm -rf /var/lib/apt/lists/* + ENV TENDERMINT_ORG $GOPATH/src/github.com/tendermint/ RUN mkdir -p $TENDERMINT_ORG COPY . $TENDERMINT_ORG/tendermint WORKDIR $TENDERMINT_ORG/tendermint + +RUN make get_vendor_deps + +RUN go install ./cmd/tendermint + +RUN bash scripts/install_tmsp_apps.sh + + +EXPOSE 46656 +EXPOSE 46657 diff --git a/test/rpc/clean.sh b/test/broadcast_tx/clean.sh similarity index 100% rename from test/rpc/clean.sh rename to test/broadcast_tx/clean.sh diff --git a/test/rpc/counter_test.sh b/test/broadcast_tx/counter_test.sh similarity index 100% rename from test/rpc/counter_test.sh rename to test/broadcast_tx/counter_test.sh diff --git a/test/rpc/dummy_test.sh b/test/broadcast_tx/dummy_test.sh similarity index 84% rename from test/rpc/dummy_test.sh rename to test/broadcast_tx/dummy_test.sh index 9410c88da..d294e56f8 100644 --- a/test/rpc/dummy_test.sh +++ b/test/broadcast_tx/dummy_test.sh @@ -1,4 +1,5 @@ #! /bin/bash +set -e function toHex() { echo -n $1 | hexdump -ve '1/1 "%.2X"' @@ -12,23 +13,31 @@ TESTNAME=$1 # store key value pair KEY="abcd" VALUE="dcba" -curl localhost:46657/broadcast_tx_commit?tx=\"$(toHex $KEY=$VALUE)\" +curl 127.0.0.1:46657/broadcast_tx_commit?tx=\"$(toHex $KEY=$VALUE)\" +echo $? echo "" # we should be able to look up the key RESPONSE=`tmsp-cli query $KEY` + +set +e A=`echo $RESPONSE | grep exists=true` if [[ $? != 0 ]]; then echo "Failed to find 'exists=true' for $KEY. Response:" echo "$RESPONSE" + exit 1 fi +set -e # we should not be able to look up the value RESPONSE=`tmsp-cli query $VALUE` +set +e A=`echo $RESPONSE | grep exists=true` if [[ $? == 0 ]]; then echo "Found 'exists=true' for $VALUE when we should not have. Response:" echo "$RESPONSE" + exit 1 fi +set -e echo "Passed Test: $TESTNAME" diff --git a/test/rpc/test.sh b/test/broadcast_tx/test.sh similarity index 55% rename from test/rpc/test.sh rename to test/broadcast_tx/test.sh index f31b19c55..ee31d34da 100644 --- a/test/rpc/test.sh +++ b/test/broadcast_tx/test.sh @@ -8,37 +8,59 @@ set -e # TODO: install everything +export TMROOT=$HOME/.tendermint_broadcast_tx + function dummy_over_socket(){ + rm -rf $TMROOT + tendermint init + echo "Starting dummy and tendermint" dummy > /dev/null & + pid_dummy=$! tendermint node > tendermint.log & - sleep 3 + pid_tendermint=$! + sleep 5 + echo "running test" bash dummy_test.sh "Dummy over Socket" - killall dummy tendermint + kill -9 $pid_dummy $pid_tendermint } function counter_over_socket() { + rm -rf $TMROOT + tendermint init + echo "Starting counter and tendermint" counter --serial > /dev/null & + pid_counter=$! tendermint node > tendermint.log & - sleep 3 + pid_tendermint=$! + sleep 5 + echo "running test" bash counter_test.sh "Counter over Socket" - killall counter tendermint + kill -9 $pid_counter $pid_tendermint } function counter_over_grpc() { + rm -rf $TMROOT + tendermint init + echo "Starting counter and tendermint" counter --serial --tmsp grpc > /dev/null & + pid_counter=$! tendermint node --tmsp grpc > tendermint.log & - sleep 3 + pid_tendermint=$! + sleep 5 + echo "running test" bash counter_test.sh "Counter over GRPC" - killall counter tendermint + kill -9 $pid_counter $pid_tendermint } +cd $GOPATH/src/github.com/tendermint/tendermint/test/broadcast_tx + case "$1" in "dummy_over_socket") dummy_over_socket @@ -50,8 +72,11 @@ case "$1" in counter_over_grpc ;; *) + echo "Running all" dummy_over_socket + echo "" counter_over_socket + echo "" counter_over_grpc esac diff --git a/test/p2p/clean.sh b/test/p2p/clean.sh new file mode 100644 index 000000000..e0fb36956 --- /dev/null +++ b/test/p2p/clean.sh @@ -0,0 +1,4 @@ +#! /bin/bash + +docker rm -vf $(docker ps -aq) +docker network rm local_testnet diff --git a/test/p2p/data/app/init.sh b/test/p2p/data/app/init.sh new file mode 100755 index 000000000..abaccae43 --- /dev/null +++ b/test/p2p/data/app/init.sh @@ -0,0 +1,9 @@ +#! /bin/bash +# This is a sample bash script for a TMSP application + +cd app/ +git clone https://github.com/tendermint/nomnomcoin.git +cd nomnomcoin +npm install . + +node app.js --eyes="unix:///data/tendermint/data/data.sock" \ No newline at end of file diff --git a/test/p2p/data/chain_config.json b/test/p2p/data/chain_config.json new file mode 100644 index 000000000..54fcea524 --- /dev/null +++ b/test/p2p/data/chain_config.json @@ -0,0 +1,53 @@ +{ + "id": "", + "val_set_id": "anon", + "validators": [ + { + "validator": { + "id": "mach1", + "pub_key": [ + 1, + "BE8933DFF1600C026E34718F1785A4CDEAB90C35698B394E38B6947AE91DE116" + ] + }, + "p2p_addr": "", + "rpc_addr": "" + }, + { + "validator": { + "id": "mach2", + "pub_key": [ + 1, + "6DC534465323126587D2A2A93B59D689B717073B1DE968A25A6EF13D595318AD" + ] + }, + "p2p_addr": "", + "rpc_addr": "", + "index": 1 + }, + { + "validator": { + "id": "mach3", + "pub_key": [ + 1, + "AE67AC697D135AA0B4601EA57EAAB3FEBF4BAA4F229C45A598C2985B12FCD1A1" + ] + }, + "p2p_addr": "", + "rpc_addr": "", + "index": 2 + }, + { + "validator": { + "id": "mach4", + "pub_key": [ + 1, + "9EBC8F58CED4B46DCD5AB8ABA591DD253CD7CB5037273FDA32BC0B6461C4EFD9" + ] + }, + "p2p_addr": "", + "rpc_addr": "", + "index": 3 + } + ] +} \ No newline at end of file diff --git a/test/p2p/data/core/init.sh b/test/p2p/data/core/init.sh new file mode 100755 index 000000000..95db91191 --- /dev/null +++ b/test/p2p/data/core/init.sh @@ -0,0 +1,20 @@ +#! /bin/bash +# This is a sample bash script for tendermint core +# Edit this script before "mintnet start" to change +# the core blockchain engine. + +TMREPO="github.com/tendermint/tendermint" +BRANCH="master" + +go get -d $TMREPO/cmd/tendermint +### DEPENDENCIES (example) +# cd $GOPATH/src/github.com/tendermint/tmsp +# git fetch origin $BRANCH +# git checkout $BRANCH +### DEPENDENCIES END +cd $GOPATH/src/$TMREPO +git fetch origin $BRANCH +git checkout $BRANCH +make install + +tendermint node --seeds="$TMSEEDS" --moniker="$TMNAME" --proxy_app="$PROXYAPP" \ No newline at end of file diff --git a/test/p2p/data/data/init.sh b/test/p2p/data/data/init.sh new file mode 100755 index 000000000..41122a547 --- /dev/null +++ b/test/p2p/data/data/init.sh @@ -0,0 +1,7 @@ +#! /bin/bash +# This is a sample bash script for MerkleEyes. +# NOTE: mintnet expects data.sock to be created + +go get github.com/tendermint/merkleeyes/cmd/merkleeyes + +merkleeyes server --address="unix:///data/tendermint/data/data.sock" \ No newline at end of file diff --git a/test/p2p/data/mach1/core/genesis.json b/test/p2p/data/mach1/core/genesis.json new file mode 100644 index 000000000..3f6fbe5a2 --- /dev/null +++ b/test/p2p/data/mach1/core/genesis.json @@ -0,0 +1,39 @@ +{ + "app_hash": "", + "chain_id": "chain-9ujDWI", + "genesis_time": "2016-06-24T20:01:19.322Z", + "validators": [ + { + "amount": 1, + "name": "mach1", + "pub_key": [ + 1, + "BE8933DFF1600C026E34718F1785A4CDEAB90C35698B394E38B6947AE91DE116" + ] + }, + { + "amount": 1, + "name": "mach2", + "pub_key": [ + 1, + "6DC534465323126587D2A2A93B59D689B717073B1DE968A25A6EF13D595318AD" + ] + }, + { + "amount": 1, + "name": "mach3", + "pub_key": [ + 1, + "AE67AC697D135AA0B4601EA57EAAB3FEBF4BAA4F229C45A598C2985B12FCD1A1" + ] + }, + { + "amount": 1, + "name": "mach4", + "pub_key": [ + 1, + "9EBC8F58CED4B46DCD5AB8ABA591DD253CD7CB5037273FDA32BC0B6461C4EFD9" + ] + } + ] +} \ No newline at end of file diff --git a/test/p2p/data/mach1/core/priv_validator.json b/test/p2p/data/mach1/core/priv_validator.json new file mode 100644 index 000000000..242c7b9fc --- /dev/null +++ b/test/p2p/data/mach1/core/priv_validator.json @@ -0,0 +1,14 @@ +{ + "address": "0E6925C3EE4C599DFF1536A5071AF4A26DF33635", + "last_height": 0, + "last_round": 0, + "last_step": 0, + "priv_key": [ + 1, + "547AA07C7A8CE16C5CB2A40C6C26D15B0A32960410A9F1EA6E50B636F1AB389ABE8933DFF1600C026E34718F1785A4CDEAB90C35698B394E38B6947AE91DE116" + ], + "pub_key": [ + 1, + "BE8933DFF1600C026E34718F1785A4CDEAB90C35698B394E38B6947AE91DE116" + ] +} \ No newline at end of file diff --git a/test/p2p/data/mach2/core/genesis.json b/test/p2p/data/mach2/core/genesis.json new file mode 100644 index 000000000..3f6fbe5a2 --- /dev/null +++ b/test/p2p/data/mach2/core/genesis.json @@ -0,0 +1,39 @@ +{ + "app_hash": "", + "chain_id": "chain-9ujDWI", + "genesis_time": "2016-06-24T20:01:19.322Z", + "validators": [ + { + "amount": 1, + "name": "mach1", + "pub_key": [ + 1, + "BE8933DFF1600C026E34718F1785A4CDEAB90C35698B394E38B6947AE91DE116" + ] + }, + { + "amount": 1, + "name": "mach2", + "pub_key": [ + 1, + "6DC534465323126587D2A2A93B59D689B717073B1DE968A25A6EF13D595318AD" + ] + }, + { + "amount": 1, + "name": "mach3", + "pub_key": [ + 1, + "AE67AC697D135AA0B4601EA57EAAB3FEBF4BAA4F229C45A598C2985B12FCD1A1" + ] + }, + { + "amount": 1, + "name": "mach4", + "pub_key": [ + 1, + "9EBC8F58CED4B46DCD5AB8ABA591DD253CD7CB5037273FDA32BC0B6461C4EFD9" + ] + } + ] +} \ No newline at end of file diff --git a/test/p2p/data/mach2/core/priv_validator.json b/test/p2p/data/mach2/core/priv_validator.json new file mode 100644 index 000000000..ead45d5e6 --- /dev/null +++ b/test/p2p/data/mach2/core/priv_validator.json @@ -0,0 +1,14 @@ +{ + "address": "99DBBD2AFC28FB5BAC5574AFAF0D9C806CED3B55", + "last_height": 0, + "last_round": 0, + "last_step": 0, + "priv_key": [ + 1, + "D047889E60502FC3129D0AB7F334B1838ED9ED1ECD99CBB96B71AD5ABF5A81436DC534465323126587D2A2A93B59D689B717073B1DE968A25A6EF13D595318AD" + ], + "pub_key": [ + 1, + "6DC534465323126587D2A2A93B59D689B717073B1DE968A25A6EF13D595318AD" + ] +} \ No newline at end of file diff --git a/test/p2p/data/mach3/core/genesis.json b/test/p2p/data/mach3/core/genesis.json new file mode 100644 index 000000000..3f6fbe5a2 --- /dev/null +++ b/test/p2p/data/mach3/core/genesis.json @@ -0,0 +1,39 @@ +{ + "app_hash": "", + "chain_id": "chain-9ujDWI", + "genesis_time": "2016-06-24T20:01:19.322Z", + "validators": [ + { + "amount": 1, + "name": "mach1", + "pub_key": [ + 1, + "BE8933DFF1600C026E34718F1785A4CDEAB90C35698B394E38B6947AE91DE116" + ] + }, + { + "amount": 1, + "name": "mach2", + "pub_key": [ + 1, + "6DC534465323126587D2A2A93B59D689B717073B1DE968A25A6EF13D595318AD" + ] + }, + { + "amount": 1, + "name": "mach3", + "pub_key": [ + 1, + "AE67AC697D135AA0B4601EA57EAAB3FEBF4BAA4F229C45A598C2985B12FCD1A1" + ] + }, + { + "amount": 1, + "name": "mach4", + "pub_key": [ + 1, + "9EBC8F58CED4B46DCD5AB8ABA591DD253CD7CB5037273FDA32BC0B6461C4EFD9" + ] + } + ] +} \ No newline at end of file diff --git a/test/p2p/data/mach3/core/priv_validator.json b/test/p2p/data/mach3/core/priv_validator.json new file mode 100644 index 000000000..dd366d205 --- /dev/null +++ b/test/p2p/data/mach3/core/priv_validator.json @@ -0,0 +1,14 @@ +{ + "address": "4C5F061DAC28660853904A66705B12CA2B317572", + "last_height": 0, + "last_round": 0, + "last_step": 0, + "priv_key": [ + 1, + "C1A4E47F349FC5F556F4A9A27BA776B94424C312BAA6CF6EE44B867348D7C3F2AE67AC697D135AA0B4601EA57EAAB3FEBF4BAA4F229C45A598C2985B12FCD1A1" + ], + "pub_key": [ + 1, + "AE67AC697D135AA0B4601EA57EAAB3FEBF4BAA4F229C45A598C2985B12FCD1A1" + ] +} \ No newline at end of file diff --git a/test/p2p/data/mach4/core/genesis.json b/test/p2p/data/mach4/core/genesis.json new file mode 100644 index 000000000..3f6fbe5a2 --- /dev/null +++ b/test/p2p/data/mach4/core/genesis.json @@ -0,0 +1,39 @@ +{ + "app_hash": "", + "chain_id": "chain-9ujDWI", + "genesis_time": "2016-06-24T20:01:19.322Z", + "validators": [ + { + "amount": 1, + "name": "mach1", + "pub_key": [ + 1, + "BE8933DFF1600C026E34718F1785A4CDEAB90C35698B394E38B6947AE91DE116" + ] + }, + { + "amount": 1, + "name": "mach2", + "pub_key": [ + 1, + "6DC534465323126587D2A2A93B59D689B717073B1DE968A25A6EF13D595318AD" + ] + }, + { + "amount": 1, + "name": "mach3", + "pub_key": [ + 1, + "AE67AC697D135AA0B4601EA57EAAB3FEBF4BAA4F229C45A598C2985B12FCD1A1" + ] + }, + { + "amount": 1, + "name": "mach4", + "pub_key": [ + 1, + "9EBC8F58CED4B46DCD5AB8ABA591DD253CD7CB5037273FDA32BC0B6461C4EFD9" + ] + } + ] +} \ No newline at end of file diff --git a/test/p2p/data/mach4/core/priv_validator.json b/test/p2p/data/mach4/core/priv_validator.json new file mode 100644 index 000000000..4a73707e8 --- /dev/null +++ b/test/p2p/data/mach4/core/priv_validator.json @@ -0,0 +1,14 @@ +{ + "address": "86F6DA4B34F16D743D2D992B5ACB12F5E724CC2D", + "last_height": 0, + "last_round": 0, + "last_step": 0, + "priv_key": [ + 1, + "C4CC3ED28F020C2DBDA98BCDBF08C3CED370470E74F25E938D5D295E8E3D2B0C9EBC8F58CED4B46DCD5AB8ABA591DD253CD7CB5037273FDA32BC0B6461C4EFD9" + ], + "pub_key": [ + 1, + "9EBC8F58CED4B46DCD5AB8ABA591DD253CD7CB5037273FDA32BC0B6461C4EFD9" + ] +} \ No newline at end of file diff --git a/test/p2p/local_testnet.sh b/test/p2p/local_testnet.sh new file mode 100644 index 000000000..2b4183b95 --- /dev/null +++ b/test/p2p/local_testnet.sh @@ -0,0 +1,28 @@ +#! /bin/bash +set -eu + +DOCKER_IMAGE=$1 +NETWORK_NAME=$2 + +cd $GOPATH/src/github.com/tendermint/tendermint + +# create docker network +docker network create --driver bridge --subnet 172.57.0.0/16 $NETWORK_NAME + +N=4 +seeds="172.57.0.101:46656" +for i in `seq 2 $N`; do + seeds="$seeds,172.57.0.$((100+$i)):46656" +done +echo "Seeds: $seeds" + +for i in `seq 1 $N`; do + # start tendermint container + docker run -d \ + --net=$NETWORK_NAME \ + --ip=172.57.0.$((100+$i)) \ + --name local_testnet_$i \ + --entrypoint tendermint \ + -e TMROOT=/go/src/github.com/tendermint/tendermint/test/p2p/data/mach$i/core \ + $DOCKER_IMAGE node --seeds $seeds --proxy_app=dummy +done diff --git a/test/p2p/run_test.sh b/test/p2p/run_test.sh new file mode 100644 index 000000000..87ab8e1f7 --- /dev/null +++ b/test/p2p/run_test.sh @@ -0,0 +1,79 @@ +#! /bin/bash + +# wait for everyone to come online +echo "Waiting for nodes to come online" +for i in `seq 1 4`; do + addr="172.57.0.$((100+$i)):46657" + curl -s $addr/status > /dev/null + ERR=$? + while [ "$ERR" != 0 ]; do + sleep 1 + curl -s $addr/status > /dev/null + ERR=$? + done + echo "... node $i is up" +done + +echo "" +# run the test on each of them +for i in `seq 1 4`; do + addr="172.57.0.$((100+$i)):46657" + + # - assert everyone has 3 other peers + N_PEERS=`curl -s $addr/net_info | jq '.result[1].peers | length'` + while [ "$N_PEERS" != 3 ]; do + echo "Waiting for node $i to connect to all peers ..." + sleep 1 + N_PEERS=`curl -s $addr/net_info | jq '.result[1].peers | length'` + done + + # - assert block height is greater than 1 + BLOCK_HEIGHT=`curl -s $addr/status | jq .result[1].latest_block_height` + while [ "$BLOCK_HEIGHT" -le 1 ]; do + echo "Waiting for node $i to commit a block ..." + sleep 1 + BLOCK_HEIGHT=`curl -s $addr/status | jq .result[1].latest_block_height` + done + echo "Node $i is connected to all peers and at block $BLOCK_HEIGHT" + + # current state + HASH1=`curl -s $addr/status | jq .result[1].latest_app_hash` + + # - send a tx + TX=\"aadeadbeefbeefbeef0$i\" + echo "Broadcast Tx $TX" + curl -s $addr/broadcast_tx_commit?tx=$TX + echo "" + + # we need to wait another block to get the new app_hash + h1=`curl -s $addr/status | jq .result[1].latest_block_height` + h2=$h1 + while [ "$h2" == "$h1" ]; do + sleep 1 + h2=`curl -s $addr/status | jq .result[1].latest_block_height` + done + + # check that hash was updated + HASH2=`curl -s $addr/status | jq .result[1].latest_app_hash` + if [[ "$HASH1" == "$HASH2" ]]; then + echo "Expected state hash to update from $HASH1. Got $HASH2" + exit 1 + fi + + # check we get the same new hash on all other nodes + for j in `seq 1 4`; do + if [[ "$i" != "$j" ]]; then + HASH3=`curl -s 172.57.0.$((100+$j)):46657/status | jq .result[1].latest_app_hash` + + if [[ "$HASH2" != "$HASH3" ]]; then + echo "App hash for node $j doesn't match. Got $HASH3, expected $HASH2" + exit 1 + fi + fi + done + + echo "All nodes are up to date" +done + +echo "" +echo "PASS" diff --git a/test/p2p/test.sh b/test/p2p/test.sh new file mode 100644 index 000000000..f139fd675 --- /dev/null +++ b/test/p2p/test.sh @@ -0,0 +1,10 @@ +#! /bin/bash + +DOCKER_IMAGE=$1 +NETWORK_NAME=local_testnet + +# start the testnet on a local network +bash test/p2p/local_testnet.sh $DOCKER_IMAGE $NETWORK_NAME + +# run the test +bash test/p2p/test_client.sh $DOCKER_IMAGE $NETWORK_NAME test/p2p/run_test.sh diff --git a/test/p2p/test_client.sh b/test/p2p/test_client.sh new file mode 100644 index 000000000..d9fe64c5c --- /dev/null +++ b/test/p2p/test_client.sh @@ -0,0 +1,16 @@ +#! /bin/bash +set -eu + +DOCKER_IMAGE=$1 +NETWORK_NAME=$2 +CMD=$3 + +# run the test container on the local network +docker run -t \ + --rm \ + -v $GOPATH/src/github.com/tendermint/tendermint/test/p2p/:/go/src/github.com/tendermint/tendermint/test/p2p \ + --net=$NETWORK_NAME \ + --ip=172.57.0.99 \ + --name test_container \ + --entrypoint bash \ + $DOCKER_IMAGE $CMD diff --git a/test/run_test.sh b/test/run_test.sh index 67d1f23ca..a7de6a141 100644 --- a/test/run_test.sh +++ b/test/run_test.sh @@ -1,14 +1,22 @@ #! /bin/bash +set -e echo `pwd` BRANCH=`git rev-parse --abbrev-ref HEAD` echo "Current branch: $BRANCH" -make get_vendor_deps - +# go test --race github.com/tendermint/tendermint/... make test_race +# run the broadcast_tx tests +bash test/broadcast_tx/test.sh + if [[ "$BRANCH" == "master" || "$BRANCH" == "staging" ]]; then + echo "" + echo "* branch $BRANCH; testing libs" + # checkout every github.com/tendermint dir and run its tests bash test/test_libs.sh + + # TODO: mintnet/netmon fi diff --git a/test/test.sh b/test/test.sh index 5c646c27a..83787f82b 100644 --- a/test/test.sh +++ b/test/test.sh @@ -8,5 +8,16 @@ # If we pushed to STAGING or MASTER, # it will also run the tests for all dependencies +echo "" +echo "* building docker file" docker build -t tester -f ./test/Dockerfile . + +echo "" +echo "* running go tests and broadcast tests" docker run -t tester bash test/run_test.sh + +# test basic network connectivity +# by starting a local testnet and checking peers connect and make blocks +echo "" +echo "* running basic peer tests" +bash test/p2p/test.sh tester From f7086d0a4df41c04f2c0a9c22a7ca066d338536b Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Sun, 24 Jul 2016 15:39:24 -0400 Subject: [PATCH 27/38] circle: docker 1.10.0 --- circle.yml | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/circle.yml b/circle.yml index 95d5bfabd..b21e05b20 100644 --- a/circle.yml +++ b/circle.yml @@ -5,6 +5,10 @@ machine: hosts: circlehost: 127.0.0.1 localhost: 127.0.0.1 + pre: + - curl -sSL https://s3.amazonaws.com/circle-downloads/install-circleci-docker.sh | bash -s -- 1.10.0 + services: + - docker checkout: post: @@ -16,8 +20,6 @@ checkout: dependencies: override: - - sudo curl -sSL -o /usr/bin/docker http://s3-external-1.amazonaws.com/circle-downloads/docker-1.9.1-circleci; sudo chmod 0755 /usr/bin/docker - - sudo service docker start - go version - docker version From 6643e4969f196f0d9ac4e1c12f719d576b9fe0fc Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Fri, 5 Aug 2016 18:21:39 -0400 Subject: [PATCH 28/38] tests: add throughput benchmark using mintnet and netmon --- DOCKER/Dockerfile | 2 +- circle.yml | 5 ++++- test/{broadcast_tx => app}/clean.sh | 0 test/{broadcast_tx => app}/counter_test.sh | 0 test/{broadcast_tx => app}/dummy_test.sh | 0 test/{broadcast_tx => app}/test.sh | 4 ++-- test/net/test.sh | 23 ++++++++++++++++++++++ test/run_test.sh | 6 ++---- test/test.sh | 7 +++++++ 9 files changed, 39 insertions(+), 8 deletions(-) rename test/{broadcast_tx => app}/clean.sh (100%) rename test/{broadcast_tx => app}/counter_test.sh (100%) rename test/{broadcast_tx => app}/dummy_test.sh (100%) rename test/{broadcast_tx => app}/test.sh (92%) create mode 100644 test/net/test.sh diff --git a/DOCKER/Dockerfile b/DOCKER/Dockerfile index 34f5aa95d..356fa8bf9 100644 --- a/DOCKER/Dockerfile +++ b/DOCKER/Dockerfile @@ -1,5 +1,5 @@ # Pull base image. -FROM golang:1.4 +FROM golang:1.6 ENV USER tmuser ENV DATA_ROOT /data/tendermint diff --git a/circle.yml b/circle.yml index b21e05b20..2c5075d45 100644 --- a/circle.yml +++ b/circle.yml @@ -2,11 +2,14 @@ machine: environment: GOPATH: /home/ubuntu/.go_workspace REPO: $GOPATH/src/github.com/$CIRCLE_PROJECT_USERNAME/$CIRCLE_PROJECT_REPONAME + DOCKER_VERSION: 1.10.0 + DOCKER_MACHINE_VERSION: 0.6.0 hosts: circlehost: 127.0.0.1 localhost: 127.0.0.1 pre: - - curl -sSL https://s3.amazonaws.com/circle-downloads/install-circleci-docker.sh | bash -s -- 1.10.0 + - curl -sSL https://s3.amazonaws.com/circle-downloads/install-circleci-docker.sh | bash -s -- $DOCKER_VERSION + - curl -sSL -o /usr/bin/docker-machine https://github.com/docker/machine/releases/download/v$DOCKER_MACHINE_VERSION/docker-machine-linux-x86_64; sudo chmod 0755 /usr/bin/docker-machine services: - docker diff --git a/test/broadcast_tx/clean.sh b/test/app/clean.sh similarity index 100% rename from test/broadcast_tx/clean.sh rename to test/app/clean.sh diff --git a/test/broadcast_tx/counter_test.sh b/test/app/counter_test.sh similarity index 100% rename from test/broadcast_tx/counter_test.sh rename to test/app/counter_test.sh diff --git a/test/broadcast_tx/dummy_test.sh b/test/app/dummy_test.sh similarity index 100% rename from test/broadcast_tx/dummy_test.sh rename to test/app/dummy_test.sh diff --git a/test/broadcast_tx/test.sh b/test/app/test.sh similarity index 92% rename from test/broadcast_tx/test.sh rename to test/app/test.sh index ee31d34da..ce9fff8df 100644 --- a/test/broadcast_tx/test.sh +++ b/test/app/test.sh @@ -8,7 +8,7 @@ set -e # TODO: install everything -export TMROOT=$HOME/.tendermint_broadcast_tx +export TMROOT=$HOME/.tendermint_app function dummy_over_socket(){ rm -rf $TMROOT @@ -59,7 +59,7 @@ function counter_over_grpc() { kill -9 $pid_counter $pid_tendermint } -cd $GOPATH/src/github.com/tendermint/tendermint/test/broadcast_tx +cd $GOPATH/src/github.com/tendermint/tendermint/test/app case "$1" in "dummy_over_socket") diff --git a/test/net/test.sh b/test/net/test.sh new file mode 100644 index 000000000..c426e0ffa --- /dev/null +++ b/test/net/test.sh @@ -0,0 +1,23 @@ +#! /bin/bash +set -eu + +DATACENTER=single +VALSETSIZE=4 +BLOCKSIZE=8092 +TX_SIZE=200 +NTXS=$((BLOCKSIZE*4)) +MACH_PREFIX=mach +RESULTSDIR=results +CLOUD_PROVIDER=digitalocean + +export TMHEAD=`git rev-parse --abbrev-ref HEAD` +export TM_IMAGE="tendermint/tmbase" + +# not a go repo +set +e +go get github.com/tendermint/network_testing +set -e +cd $GOPATH/src/github.com/tendermint/network_testing +bash experiments/exp_throughput.sh $DATACENTER $VALSETSIZE $BLOCKSIZE $TX_SIZE $NTXS $MACH_PREFIX $RESULTSDIR $CLOUD_PROVIDER + +# TODO echo result! diff --git a/test/run_test.sh b/test/run_test.sh index a7de6a141..188c94f51 100644 --- a/test/run_test.sh +++ b/test/run_test.sh @@ -9,14 +9,12 @@ echo "Current branch: $BRANCH" # go test --race github.com/tendermint/tendermint/... make test_race -# run the broadcast_tx tests -bash test/broadcast_tx/test.sh +# run the app tests +bash test/app/test.sh if [[ "$BRANCH" == "master" || "$BRANCH" == "staging" ]]; then echo "" echo "* branch $BRANCH; testing libs" # checkout every github.com/tendermint dir and run its tests bash test/test_libs.sh - - # TODO: mintnet/netmon fi diff --git a/test/test.sh b/test/test.sh index 83787f82b..99e1fcdfe 100644 --- a/test/test.sh +++ b/test/test.sh @@ -21,3 +21,10 @@ docker run -t tester bash test/run_test.sh echo "" echo "* running basic peer tests" bash test/p2p/test.sh tester + +BRANCH=`git rev-parse --abbrev-ref HEAD` +if [[ "$BRANCH" == "master" || "$BRANCH" == "staging" ]]; then + echo "" + echo "* branch $BRANCH; running mintnet/netmon throughput benchmark" + bash tests/net/test.sh +fi From 6fef31442383d7b2a36a66b77156987c6f15df7c Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Fri, 5 Aug 2016 19:15:14 -0400 Subject: [PATCH 29/38] version bump and release branch --- consensus/version.go | 2 +- test/run_test.sh | 2 +- test/test.sh | 2 +- version/version.go | 2 +- 4 files changed, 4 insertions(+), 4 deletions(-) diff --git a/consensus/version.go b/consensus/version.go index 2acc3b77b..34886db3c 100644 --- a/consensus/version.go +++ b/consensus/version.go @@ -8,6 +8,6 @@ import ( var Spec = "1" // async var Major = "0" // var Minor = "2" // replay refactor -var Revision = "1" // round state fix +var Revision = "2" // validation -> commit var Version = Fmt("v%s/%s.%s.%s", Spec, Major, Minor, Revision) diff --git a/test/run_test.sh b/test/run_test.sh index 188c94f51..90e8f5026 100644 --- a/test/run_test.sh +++ b/test/run_test.sh @@ -12,7 +12,7 @@ make test_race # run the app tests bash test/app/test.sh -if [[ "$BRANCH" == "master" || "$BRANCH" == "staging" ]]; then +if [[ "$BRANCH" == "master" || $(echo "$BRANCH" | grep "release-") != "" ]]; then echo "" echo "* branch $BRANCH; testing libs" # checkout every github.com/tendermint dir and run its tests diff --git a/test/test.sh b/test/test.sh index 99e1fcdfe..fe1c6206d 100644 --- a/test/test.sh +++ b/test/test.sh @@ -23,7 +23,7 @@ echo "* running basic peer tests" bash test/p2p/test.sh tester BRANCH=`git rev-parse --abbrev-ref HEAD` -if [[ "$BRANCH" == "master" || "$BRANCH" == "staging" ]]; then +if [[ "$BRANCH" == "master" || $(echo "$BRANCH" | grep "release-") != "" ]]; then echo "" echo "* branch $BRANCH; running mintnet/netmon throughput benchmark" bash tests/net/test.sh diff --git a/version/version.go b/version/version.go index eccabb798..e275ce5b0 100644 --- a/version/version.go +++ b/version/version.go @@ -1,7 +1,7 @@ package version const Maj = "0" -const Min = "6" // tmsp refactor +const Min = "7" // tmsp useability (protobuf, unix); optimizations; broadcast_tx_commit const Fix = "0" const Version = Maj + "." + Min + "." + Fix From 5ab3336c91f134bba9f803a1a37124ab9f689f56 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Fri, 5 Aug 2016 19:18:05 -0400 Subject: [PATCH 30/38] meow circle --- circle.yml | 4 ++-- test/test.sh | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/circle.yml b/circle.yml index 2c5075d45..da71a97ce 100644 --- a/circle.yml +++ b/circle.yml @@ -8,8 +8,8 @@ machine: circlehost: 127.0.0.1 localhost: 127.0.0.1 pre: - - curl -sSL https://s3.amazonaws.com/circle-downloads/install-circleci-docker.sh | bash -s -- $DOCKER_VERSION - - curl -sSL -o /usr/bin/docker-machine https://github.com/docker/machine/releases/download/v$DOCKER_MACHINE_VERSION/docker-machine-linux-x86_64; sudo chmod 0755 /usr/bin/docker-machine + - curl -sSL https://s3.amazonaws.com/circle-downloads/install-circleci-docker.sh | sudo bash -s -- $DOCKER_VERSION + - sudo curl -sSL -o /usr/bin/docker-machine https://github.com/docker/machine/releases/download/v$DOCKER_MACHINE_VERSION/docker-machine-linux-x86_64; sudo chmod 0755 /usr/bin/docker-machine services: - docker diff --git a/test/test.sh b/test/test.sh index fe1c6206d..47a2aa86a 100644 --- a/test/test.sh +++ b/test/test.sh @@ -26,5 +26,5 @@ BRANCH=`git rev-parse --abbrev-ref HEAD` if [[ "$BRANCH" == "master" || $(echo "$BRANCH" | grep "release-") != "" ]]; then echo "" echo "* branch $BRANCH; running mintnet/netmon throughput benchmark" - bash tests/net/test.sh + bash test/net/test.sh fi From dcf368632cb605dd8423f1f19f67022a378a1502 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Fri, 5 Aug 2016 20:01:32 -0400 Subject: [PATCH 31/38] test/net: install mintnet, netmon --- test/net/test.sh | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/test/net/test.sh b/test/net/test.sh index c426e0ffa..02a5bdf0f 100644 --- a/test/net/test.sh +++ b/test/net/test.sh @@ -17,6 +17,14 @@ export TM_IMAGE="tendermint/tmbase" set +e go get github.com/tendermint/network_testing set -e + +# install mintnet, netmon +# TODO: specify branch + +go get github.com/tendermint/mintnet +go get github.com/tendermint/netmon + + cd $GOPATH/src/github.com/tendermint/network_testing bash experiments/exp_throughput.sh $DATACENTER $VALSETSIZE $BLOCKSIZE $TX_SIZE $NTXS $MACH_PREFIX $RESULTSDIR $CLOUD_PROVIDER From 3aebc17bff6544386a459ff4693c6ccf8346e929 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Fri, 5 Aug 2016 20:02:08 -0400 Subject: [PATCH 32/38] net test --- circle.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/circle.yml b/circle.yml index da71a97ce..25494f8af 100644 --- a/circle.yml +++ b/circle.yml @@ -28,6 +28,7 @@ dependencies: test: override: - - "cd $REPO && make test_integrations" + - "cd $REPO && bash test/net/test.sh" + #- "cd $REPO && make test_integrations" post: - bash <(curl -s https://codecov.io/bash) From 0dde260c058aad64cf13d58899fbee877f25cfb8 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Fri, 5 Aug 2016 20:16:35 -0400 Subject: [PATCH 33/38] meow circle --- circle.yml | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/circle.yml b/circle.yml index 25494f8af..41a768db1 100644 --- a/circle.yml +++ b/circle.yml @@ -9,7 +9,6 @@ machine: localhost: 127.0.0.1 pre: - curl -sSL https://s3.amazonaws.com/circle-downloads/install-circleci-docker.sh | sudo bash -s -- $DOCKER_VERSION - - sudo curl -sSL -o /usr/bin/docker-machine https://github.com/docker/machine/releases/download/v$DOCKER_MACHINE_VERSION/docker-machine-linux-x86_64; sudo chmod 0755 /usr/bin/docker-machine services: - docker @@ -23,8 +22,10 @@ checkout: dependencies: override: + - sudo curl -sSL -o /usr/bin/docker-machine https://github.com/docker/machine/releases/download/v$DOCKER_MACHINE_VERSION/docker-machine-linux-x86_64; sudo chmod 0755 /usr/bin/docker-machine - go version - docker version + - docker-machine version test: override: From 45ff05d47d09c91448e543374d51b93ab33f1bbe Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Fri, 5 Aug 2016 20:42:52 -0400 Subject: [PATCH 34/38] test/net: use MACH_PREFIX --- circle.yml | 1 + test/net/test.sh | 7 ++++++- 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/circle.yml b/circle.yml index 41a768db1..23a678333 100644 --- a/circle.yml +++ b/circle.yml @@ -1,5 +1,6 @@ machine: environment: + MACH_PREFIX: tendermint-test-mach GOPATH: /home/ubuntu/.go_workspace REPO: $GOPATH/src/github.com/$CIRCLE_PROJECT_USERNAME/$CIRCLE_PROJECT_REPONAME DOCKER_VERSION: 1.10.0 diff --git a/test/net/test.sh b/test/net/test.sh index 02a5bdf0f..b5809b26b 100644 --- a/test/net/test.sh +++ b/test/net/test.sh @@ -6,10 +6,15 @@ VALSETSIZE=4 BLOCKSIZE=8092 TX_SIZE=200 NTXS=$((BLOCKSIZE*4)) -MACH_PREFIX=mach RESULTSDIR=results CLOUD_PROVIDER=digitalocean +set +u +if [[ "$MACH_PREFIX" == "" ]]; then + MACH_PREFIX=mach +fi +set -u + export TMHEAD=`git rev-parse --abbrev-ref HEAD` export TM_IMAGE="tendermint/tmbase" From 628528da5af6f3c20b1cd361dafcdb1508057a7a Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Fri, 5 Aug 2016 21:20:01 -0400 Subject: [PATCH 35/38] test/net: cleanup --- circle.yml | 3 +-- test/net/test.sh | 8 +++++++- 2 files changed, 8 insertions(+), 3 deletions(-) diff --git a/circle.yml b/circle.yml index 23a678333..2f6141b2d 100644 --- a/circle.yml +++ b/circle.yml @@ -30,7 +30,6 @@ dependencies: test: override: - - "cd $REPO && bash test/net/test.sh" - #- "cd $REPO && make test_integrations" + - "cd $REPO && make test_integrations" post: - bash <(curl -s https://codecov.io/bash) diff --git a/test/net/test.sh b/test/net/test.sh index b5809b26b..9ec03e9a2 100644 --- a/test/net/test.sh +++ b/test/net/test.sh @@ -33,4 +33,10 @@ go get github.com/tendermint/netmon cd $GOPATH/src/github.com/tendermint/network_testing bash experiments/exp_throughput.sh $DATACENTER $VALSETSIZE $BLOCKSIZE $TX_SIZE $NTXS $MACH_PREFIX $RESULTSDIR $CLOUD_PROVIDER -# TODO echo result! +# TODO: publish result! + +# cleanup + +mintnet destroy --machines $MACH_PREFIX[1-$VALSETSIZE] + + From 3ac9ff0dcc639ca45853e2db8d7b030070021c7c Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Sat, 6 Aug 2016 23:30:46 -0400 Subject: [PATCH 36/38] test: dont run cloud test on push to master --- README.md | 45 +++++++++++++-------------------------------- state/execution.go | 2 ++ test/test.sh | 14 +++++--------- 3 files changed, 20 insertions(+), 41 deletions(-) diff --git a/README.md b/README.md index 38e2443d9..4071858fb 100644 --- a/README.md +++ b/README.md @@ -6,6 +6,16 @@ Simple, Secure, Scalable Blockchain Platform _NOTE: This is yet pre-alpha non-production-quality software._ +## App Developers + +Tendermint Core is Byzantine Fault Tolerant (BFT) middleware that takes an application, written in any programming language, +and replicates it (ie. the latest state) on many machines. +See the [application developers guide](guide://github.com/tendermint/tendermint/wiki/Application-Developers) to get started. + +## Contributing + +Yay open source! Please see our [contributing guidelines](https://github.com/tendermint/tendermint/wiki/Contributing). + ## Resources ### Tendermint Core @@ -29,37 +39,8 @@ _NOTE: This is yet pre-alpha non-production-quality software._ * [Go-Merkle](http://github.com/tendermint/go-merkle) * -### Install - -Make sure you have installed Go and [set the GOPATH](https://github.com/tendermint/tendermint/wiki/Setting-GOPATH). - -Install `glide`, used for dependency management: - -``` -go get https://github.com/Masterminds/glide -``` - -Install tendermint: - -``` -mkdir -p $GOPATH/src/github.com/tendermint -git clone https://github.com/tendermint/tendermint $GOPATH/src/github.com/tendermint/tendermint -cd $GOPATH/src/github.com/tendermint/tendermint -glide install -go install ./cmd/tendermint -``` - -Initialize a sample tendermint directory with an example genesis file (in `~/.tendermint`): - -``` -tendermint init -``` - -Now run the tendermint node: +## Install -``` -tendermint node --proxy_app=dummy -``` +`go get -u github.com/tendermint/tendermint/cmd/tendermint` -For tutorials on running other applications with Tendermint, and for launching test networks, -see http://tendermint.com/guide/ +For more details, see the [install guide](https://github.com/tendermint/tendermint/wiki/Installation). diff --git a/state/execution.go b/state/execution.go index 1acd04930..13e776c8c 100644 --- a/state/execution.go +++ b/state/execution.go @@ -80,6 +80,8 @@ func (s *State) execBlockOnProxyApp(eventCache events.Fireable, proxyAppConn pro } proxyAppConn.SetResponseCallback(proxyCb) + // TODO: BeginBlock + // Run txs of block for _, tx := range block.Txs { proxyAppConn.AppendTxAsync(tx) diff --git a/test/test.sh b/test/test.sh index 47a2aa86a..7d69728d5 100644 --- a/test/test.sh +++ b/test/test.sh @@ -1,19 +1,14 @@ #! /bin/bash -# integrations test -# this is the script run by eg CircleCI. -# It creates a docker container, -# installs the dependencies, -# and runs the tests. -# If we pushed to STAGING or MASTER, -# it will also run the tests for all dependencies +# Top Level Testing Script +# See the github.com/tendermint/tendermint/test/README.md echo "" echo "* building docker file" docker build -t tester -f ./test/Dockerfile . echo "" -echo "* running go tests and broadcast tests" +echo "* running go tests and app tests" docker run -t tester bash test/run_test.sh # test basic network connectivity @@ -22,8 +17,9 @@ echo "" echo "* running basic peer tests" bash test/p2p/test.sh tester +# only run the cloud benchmark for releases BRANCH=`git rev-parse --abbrev-ref HEAD` -if [[ "$BRANCH" == "master" || $(echo "$BRANCH" | grep "release-") != "" ]]; then +if [[ $(echo "$BRANCH" | grep "release-") != "" ]]; then echo "" echo "* branch $BRANCH; running mintnet/netmon throughput benchmark" bash test/net/test.sh From 34f607fe491a934798677ba9823d0f03ad7f1d9e Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Sat, 6 Aug 2016 23:31:06 -0400 Subject: [PATCH 37/38] test: README.md --- test/README.md | 28 ++++++++++++++++++++++++++++ 1 file changed, 28 insertions(+) create mode 100644 test/README.md diff --git a/test/README.md b/test/README.md new file mode 100644 index 000000000..8333e27ad --- /dev/null +++ b/test/README.md @@ -0,0 +1,28 @@ +# Tendermint Tests + +The unit tests (ie. the `go test` s) can be run with `make test`. +The integration tests can be run wtih `make test_integrations`. + +Running the integrations test will build a docker container with latest tendermint +and run the following tests in docker containers: + +- go tests, with --race +- app tests + - dummy app over socket + - counter app over socket + - counter app over grpc +- p2p tests + - start a local dummy app testnet on a docker network (requires docker version 1.10+) + - send a tx on each node and ensure the state root is updated on all of them + +If on a `release-x.x.x` branch, we also run + +- `go test` for all our dependency libs (test/test_libs.sh) +- network_testing - benchmark a mintnet based cloud deploy using netmon + +# Coverage + +TODO! + + + From 444077505b43c6c1756cb6a0bed7ab0c1d7c51f0 Mon Sep 17 00:00:00 2001 From: Ethan Buchman Date: Sat, 6 Aug 2016 23:37:21 -0400 Subject: [PATCH 38/38] fix README --- README.md | 9 +++------ 1 file changed, 3 insertions(+), 6 deletions(-) diff --git a/README.md b/README.md index 4071858fb..9819b3626 100644 --- a/README.md +++ b/README.md @@ -6,11 +6,9 @@ Simple, Secure, Scalable Blockchain Platform _NOTE: This is yet pre-alpha non-production-quality software._ -## App Developers - -Tendermint Core is Byzantine Fault Tolerant (BFT) middleware that takes an application, written in any programming language, -and replicates it (ie. the latest state) on many machines. -See the [application developers guide](guide://github.com/tendermint/tendermint/wiki/Application-Developers) to get started. +Tendermint Core is Byzantine Fault Tolerant (BFT) middleware that takes a state transition machine, written in any programming language, +and replicates it on many machines. +See the [application developers guide](https://github.com/tendermint/tendermint/wiki/Application-Developers) to get started. ## Contributing @@ -37,7 +35,6 @@ Yay open source! Please see our [contributing guidelines](https://github.com/ten * [Go-Wire](http://github.com/tendermint/go-wire) * [Go-P2P](http://github.com/tendermint/go-p2p) * [Go-Merkle](http://github.com/tendermint/go-merkle) -* ## Install