Browse Source

new logging

pull/491/head
Anton Kaliaev 7 years ago
parent
commit
f803544195
No known key found for this signature in database GPG Key ID: 7B6881D965918214
71 changed files with 742 additions and 537 deletions
  1. +0
    -7
      blockchain/log.go
  2. +15
    -6
      blockchain/pool.go
  3. +9
    -6
      blockchain/pool_test.go
  4. +9
    -9
      blockchain/reactor.go
  5. +2
    -2
      cmd/tendermint/commands/init.go
  6. +9
    -10
      cmd/tendermint/commands/reset_priv_validator.go
  7. +17
    -3
      cmd/tendermint/commands/root.go
  8. +3
    -3
      cmd/tendermint/commands/run_node.go
  9. +2
    -2
      cmd/tendermint/commands/show_validator.go
  10. +1
    -1
      config/toml.go
  11. +9
    -4
      consensus/byzantine_test.go
  12. +15
    -1
      consensus/common_test.go
  13. +3
    -3
      consensus/height_vote_set.go
  14. +0
    -18
      consensus/log.go
  15. +42
    -42
      consensus/reactor.go
  16. +10
    -7
      consensus/reactor_test.go
  17. +27
    -21
      consensus/replay.go
  18. +3
    -3
      consensus/replay_file.go
  19. +14
    -10
      consensus/replay_test.go
  20. +62
    -62
      consensus/state.go
  21. +14
    -14
      consensus/state_test.go
  22. +6
    -6
      consensus/ticker.go
  23. +3
    -3
      consensus/wal.go
  24. +19
    -8
      glide.lock
  25. +3
    -3
      glide.yaml
  26. +0
    -18
      mempool/log.go
  27. +17
    -9
      mempool/mempool.go
  28. +2
    -0
      mempool/mempool_test.go
  29. +7
    -7
      mempool/reactor.go
  30. +0
    -7
      node/log.go
  31. +19
    -14
      node/node.go
  32. +3
    -2
      node/node_test.go
  33. +25
    -23
      p2p/addrbook.go
  34. +8
    -0
      p2p/addrbook_test.go
  35. +21
    -21
      p2p/connection.go
  36. +7
    -2
      p2p/connection_test.go
  37. +153
    -0
      p2p/glide.lock
  38. +11
    -10
      p2p/listener.go
  39. +3
    -1
      p2p/listener_test.go
  40. +0
    -7
      p2p/log.go
  41. +4
    -4
      p2p/peer.go
  42. +11
    -11
      p2p/pex_reactor.go
  43. +12
    -1
      p2p/pex_reactor_test.go
  44. +19
    -20
      p2p/switch.go
  45. +4
    -5
      p2p/switch_test.go
  46. +0
    -7
      proxy/log.go
  47. +1
    -1
      proxy/multi_app_conn.go
  48. +1
    -1
      rpc/core/abci.go
  49. +3
    -2
      rpc/core/blocks.go
  50. +2
    -2
      rpc/core/events.go
  51. +0
    -7
      rpc/core/log.go
  52. +3
    -3
      rpc/core/mempool.go
  53. +1
    -1
      rpc/core/net.go
  54. +7
    -0
      rpc/core/pipe.go
  55. +2
    -1
      rpc/lib/README.md
  56. +1
    -1
      rpc/lib/client/http_client.go
  57. +0
    -7
      rpc/lib/client/log.go
  58. +3
    -3
      rpc/lib/client/ws_client.go
  59. +7
    -4
      rpc/lib/rpc_test.go
  60. +24
    -17
      rpc/lib/server/handlers.go
  61. +9
    -8
      rpc/lib/server/http_server.go
  62. +0
    -7
      rpc/lib/server/log.go
  63. +6
    -3
      rpc/lib/test/main.go
  64. +5
    -8
      rpc/test/helpers.go
  65. +16
    -15
      state/execution.go
  66. +2
    -0
      state/execution_test.go
  67. +0
    -7
      state/log.go
  68. +14
    -2
      state/state.go
  69. +5
    -1
      state/state_test.go
  70. +0
    -7
      types/log.go
  71. +7
    -6
      types/priv_validator.go

+ 0
- 7
blockchain/log.go View File

@ -1,7 +0,0 @@
package blockchain
import (
"github.com/tendermint/tmlibs/logger"
)
var log = logger.New("module", "blockchain")

+ 15
- 6
blockchain/pool.go View File

@ -5,9 +5,10 @@ import (
"sync"
"time"
"github.com/tendermint/tendermint/types"
. "github.com/tendermint/tmlibs/common"
flow "github.com/tendermint/tmlibs/flowrate"
"github.com/tendermint/tendermint/types"
"github.com/tendermint/tmlibs/log"
)
const (
@ -58,7 +59,7 @@ func NewBlockPool(start int, requestsCh chan<- BlockRequest, timeoutsCh chan<- s
requestsCh: requestsCh,
timeoutsCh: timeoutsCh,
}
bp.BaseService = *NewBaseService(log, "BlockPool", bp)
bp.BaseService = *NewBaseService(nil, "BlockPool", bp)
return bp
}
@ -106,7 +107,7 @@ func (pool *BlockPool) removeTimedoutPeers() {
// XXX remove curRate != 0
if curRate != 0 && curRate < minRecvRate {
pool.sendTimeout(peer.id)
log.Warn("SendTimeout", "peer", peer.id, "reason", "curRate too low")
pool.Logger.Error("SendTimeout", "peer", peer.id, "reason", "curRate too low")
peer.didTimeout = true
}
}
@ -132,7 +133,7 @@ func (pool *BlockPool) IsCaughtUp() bool {
// Need at least 1 peer to be considered caught up.
if len(pool.peers) == 0 {
log.Debug("Blockpool has no peers")
pool.Logger.Debug("Blockpool has no peers")
return false
}
@ -142,7 +143,7 @@ func (pool *BlockPool) IsCaughtUp() bool {
}
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)
pool.Logger.Info(Fmt("IsCaughtUp: %v", isCaughtUp), "height", height, "maxPeerHeight", maxPeerHeight)
return isCaughtUp
}
@ -226,6 +227,7 @@ func (pool *BlockPool) SetPeerHeight(peerID string, height int) {
peer.height = height
} else {
peer = newBPPeer(pool, peerID, height)
peer.setLogger(pool.Logger.With("peer", peerID))
pool.peers[peerID] = peer
}
}
@ -328,6 +330,8 @@ type bpPeer struct {
numPending int32
timeout *time.Timer
didTimeout bool
logger log.Logger
}
func newBPPeer(pool *BlockPool, peerID string, height int) *bpPeer {
@ -336,10 +340,15 @@ func newBPPeer(pool *BlockPool, peerID string, height int) *bpPeer {
id: peerID,
height: height,
numPending: 0,
logger: log.NewNopLogger(),
}
return peer
}
func (peer *bpPeer) setLogger(l log.Logger) {
peer.logger = l
}
func (peer *bpPeer) resetMonitor() {
peer.recvMonitor = flow.New(time.Second, time.Second*40)
var initialValue = float64(minRecvRate) * math.E
@ -377,7 +386,7 @@ func (peer *bpPeer) onTimeout() {
defer peer.pool.mtx.Unlock()
peer.pool.sendTimeout(peer.id)
log.Warn("SendTimeout", "peer", peer.id, "reason", "onTimeout")
peer.logger.Error("SendTimeout", "reason", "onTimeout")
peer.didTimeout = true
}


+ 9
- 6
blockchain/pool_test.go View File

@ -5,8 +5,9 @@ import (
"testing"
"time"
. "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tendermint/types"
. "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tmlibs/log"
)
func init() {
@ -34,6 +35,7 @@ func TestBasic(t *testing.T) {
timeoutsCh := make(chan string, 100)
requestsCh := make(chan BlockRequest, 100)
pool := NewBlockPool(start, requestsCh, timeoutsCh)
pool.SetLogger(log.TestingLogger())
pool.Start()
defer pool.Stop()
@ -65,7 +67,7 @@ func TestBasic(t *testing.T) {
case peerID := <-timeoutsCh:
t.Errorf("timeout: %v", peerID)
case request := <-requestsCh:
log.Info("TEST: Pulled new BlockRequest", "request", request)
t.Logf("Pulled new BlockRequest %v", request)
if request.Height == 300 {
return // Done!
}
@ -73,7 +75,7 @@ func TestBasic(t *testing.T) {
go func() {
block := &types.Block{Header: &types.Header{Height: request.Height}}
pool.AddBlock(request.PeerID, block, 123)
log.Info("TEST: Added block", "block", request.Height, "peer", request.PeerID)
t.Logf("Added block from peer %v (height: %v)", request.PeerID, request.Height)
}()
}
}
@ -85,11 +87,12 @@ func TestTimeout(t *testing.T) {
timeoutsCh := make(chan string, 100)
requestsCh := make(chan BlockRequest, 100)
pool := NewBlockPool(start, requestsCh, timeoutsCh)
pool.SetLogger(log.TestingLogger())
pool.Start()
defer pool.Stop()
for _, peer := range peers {
log.Info("Peer", "id", peer.id)
t.Logf("Peer #%v", peer.id)
}
// Introduce each peer.
@ -120,7 +123,7 @@ func TestTimeout(t *testing.T) {
for {
select {
case peerID := <-timeoutsCh:
log.Info("Timeout", "peerID", peerID)
t.Logf("Peer #%v timeouted", peerID)
if _, ok := timedOut[peerID]; !ok {
counter++
if counter == len(peers) {
@ -128,7 +131,7 @@ func TestTimeout(t *testing.T) {
}
}
case request := <-requestsCh:
log.Info("TEST: Pulled new BlockRequest", "request", request)
t.Logf("Pulled new BlockRequest %v", request)
}
}
}

+ 9
- 9
blockchain/reactor.go View File

@ -6,7 +6,7 @@ import (
"reflect"
"time"
"github.com/tendermint/go-wire"
wire "github.com/tendermint/go-wire"
"github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/proxy"
sm "github.com/tendermint/tendermint/state"
@ -78,7 +78,7 @@ func NewBlockchainReactor(state *sm.State, proxyAppConn proxy.AppConnConsensus,
requestsCh: requestsCh,
timeoutsCh: timeoutsCh,
}
bcR.BaseReactor = *p2p.NewBaseReactor(log, "BlockchainReactor", bcR)
bcR.BaseReactor = *p2p.NewBaseReactor("BlockchainReactor", bcR)
return bcR
}
@ -128,11 +128,11 @@ func (bcR *BlockchainReactor) RemovePeer(peer *p2p.Peer, reason interface{}) {
func (bcR *BlockchainReactor) Receive(chID byte, src *p2p.Peer, msgBytes []byte) {
_, msg, err := DecodeMessage(msgBytes)
if err != nil {
log.Warn("Error decoding message", "error", err)
bcR.Logger.Error("Error decoding message", "error", err)
return
}
log.Debug("Receive", "src", src, "chID", chID, "msg", msg)
bcR.Logger.Debug("Receive", "src", src, "chID", chID, "msg", msg)
switch msg := msg.(type) {
case *bcBlockRequestMessage:
@ -160,7 +160,7 @@ func (bcR *BlockchainReactor) Receive(chID byte, src *p2p.Peer, msgBytes []byte)
// Got a peer status. Unverified.
bcR.pool.SetPeerHeight(src.Key, msg.Height)
default:
log.Warn(cmn.Fmt("Unknown message type %v", reflect.TypeOf(msg)))
bcR.Logger.Error(cmn.Fmt("Unknown message type %v", reflect.TypeOf(msg)))
}
}
@ -200,10 +200,10 @@ FOR_LOOP:
case _ = <-switchToConsensusTicker.C:
height, numPending, _ := bcR.pool.GetStatus()
outbound, inbound, _ := bcR.Switch.NumPeers()
log.Info("Consensus ticker", "numPending", numPending, "total", len(bcR.pool.requesters),
bcR.Logger.Info("Consensus ticker", "numPending", numPending, "total", len(bcR.pool.requesters),
"outbound", outbound, "inbound", inbound)
if bcR.pool.IsCaughtUp() {
log.Notice("Time to switch to consensus reactor!", "height", height)
bcR.Logger.Info("Time to switch to consensus reactor!", "height", height)
bcR.pool.Stop()
conR := bcR.Switch.Reactor("CONSENSUS").(consensusReactor)
@ -217,7 +217,7 @@ FOR_LOOP:
for i := 0; i < 10; i++ {
// See if there are any blocks to sync.
first, second := bcR.pool.PeekTwoBlocks()
//log.Info("TrySync peeked", "first", first, "second", second)
//bcR.Logger.Info("TrySync peeked", "first", first, "second", second)
if first == nil || second == nil {
// We need both to sync the first block.
break SYNC_LOOP
@ -231,7 +231,7 @@ FOR_LOOP:
err := bcR.state.Validators.VerifyCommit(
bcR.state.ChainID, types.BlockID{first.Hash(), firstPartsHeader}, first.Height, second.LastCommit)
if err != nil {
log.Info("error in validation", "error", err)
bcR.Logger.Info("error in validation", "error", err)
bcR.pool.RedoRequest(first.Height)
break SYNC_LOOP
} else {


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

@ -40,8 +40,8 @@ func initFiles(cmd *cobra.Command, args []string) {
genDoc.SaveAs(genFile)
}
log.Notice("Initialized tendermint", "genesis", config.GenesisFile(), "priv_validator", config.PrivValidatorFile())
logger.Info("Initialized tendermint", "genesis", config.GenesisFile(), "priv_validator", config.PrivValidatorFile())
} else {
log.Notice("Already initialized", "priv_validator", config.PrivValidatorFile())
logger.Info("Already initialized", "priv_validator", config.PrivValidatorFile())
}
}

+ 9
- 10
cmd/tendermint/commands/reset_priv_validator.go View File

@ -5,8 +5,8 @@ import (
"github.com/spf13/cobra"
"github.com/tendermint/log15"
"github.com/tendermint/tendermint/types"
"github.com/tendermint/tmlibs/log"
)
var resetAllCmd = &cobra.Command{
@ -29,34 +29,33 @@ func init() {
// XXX: this is totally unsafe.
// it's only suitable for testnets.
func resetAll(cmd *cobra.Command, args []string) {
ResetAll(config.DBDir(), config.PrivValidatorFile(), log)
ResetAll(config.DBDir(), config.PrivValidatorFile(), logger)
}
// XXX: this is totally unsafe.
// it's only suitable for testnets.
func resetPrivValidator(cmd *cobra.Command, args []string) {
resetPrivValidatorLocal(config.PrivValidatorFile(), log)
resetPrivValidatorLocal(config.PrivValidatorFile(), logger)
}
// Exported so other CLI tools can use it
func ResetAll(dbDir, privValFile string, l log15.Logger) {
resetPrivValidatorLocal(privValFile, l)
func ResetAll(dbDir, privValFile string, logger log.Logger) {
resetPrivValidatorLocal(privValFile, logger)
os.RemoveAll(dbDir)
l.Notice("Removed all data", "dir", dbDir)
logger.Info("Removed all data", "dir", dbDir)
}
func resetPrivValidatorLocal(privValFile string, l log15.Logger) {
func resetPrivValidatorLocal(privValFile string, logger log.Logger) {
// Get PrivValidator
var privValidator *types.PrivValidator
if _, err := os.Stat(privValFile); err == nil {
privValidator = types.LoadPrivValidator(privValFile)
privValidator.Reset()
l.Notice("Reset PrivValidator", "file", privValFile)
logger.Info("Reset PrivValidator", "file", privValFile)
} else {
privValidator = types.GenPrivValidator()
privValidator.SetFile(privValFile)
privValidator.Save()
l.Notice("Generated PrivValidator", "file", privValFile)
logger.Info("Generated PrivValidator", "file", privValFile)
}
}

+ 17
- 3
cmd/tendermint/commands/root.go View File

@ -1,16 +1,19 @@
package commands
import (
"fmt"
"os"
"github.com/spf13/cobra"
"github.com/spf13/viper"
cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tmlibs/logger"
"github.com/tendermint/tmlibs/log"
)
var (
config = cfg.DefaultConfig()
log = logger.New("module", "main")
logger = log.NewTMLogger(log.NewSyncWriter(os.Stdout)).With("module", "main")
)
func init() {
@ -24,7 +27,18 @@ var RootCmd = &cobra.Command{
err := viper.Unmarshal(config)
config.SetRoot(config.RootDir)
cfg.EnsureRoot(config.RootDir)
logger.SetLogLevel(config.LogLevel)
var option log.Option
switch config.LogLevel {
case "info":
option = log.AllowInfo()
case "debug":
option = log.AllowDebug()
case "error":
option = log.AllowError()
default:
return fmt.Errorf("Expected either \"info\", \"debug\" or \"error\" log level, given %v", config.LogLevel)
}
logger = log.NewFilter(logger, option)
return err
},
}

+ 3
- 3
cmd/tendermint/commands/run_node.go View File

@ -58,7 +58,7 @@ func runNode(cmd *cobra.Command, args []string) error {
// always available, remove.
genDocFile := config.GenesisFile()
if !cmn.FileExists(genDocFile) {
log.Notice(cmn.Fmt("Waiting for genesis file %v...", genDocFile))
logger.Info(cmn.Fmt("Waiting for genesis file %v...", genDocFile))
for {
time.Sleep(time.Second)
if !cmn.FileExists(genDocFile) {
@ -80,11 +80,11 @@ func runNode(cmd *cobra.Command, args []string) error {
}
// Create & start node
n := node.NewNodeDefault(config)
n := node.NewNodeDefault(config, logger.With("module", "node"))
if _, err := n.Start(); err != nil {
return fmt.Errorf("Failed to start node: %v", err)
} else {
log.Notice("Started node", "nodeInfo", n.Switch().NodeInfo())
logger.Info("Started node", "nodeInfo", n.Switch().NodeInfo())
}
// Trap signal, run forever.


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

@ -5,7 +5,7 @@ import (
"github.com/spf13/cobra"
data "github.com/tendermint/go-wire/data"
"github.com/tendermint/go-wire/data"
"github.com/tendermint/tendermint/types"
)
@ -20,7 +20,7 @@ func init() {
}
func showValidator(cmd *cobra.Command, args []string) {
privValidator := types.LoadOrGenPrivValidator(config.PrivValidatorFile())
privValidator := types.LoadOrGenPrivValidator(config.PrivValidatorFile(), logger)
pubKeyJSONBytes, _ := data.ToJSON(privValidator.PubKey)
fmt.Println(string(pubKeyJSONBytes))
}

+ 1
- 1
config/toml.go View File

@ -35,7 +35,7 @@ node_laddr = "tcp://0.0.0.0:46656"
seeds = ""
fast_sync = true
db_backend = "leveldb"
log_level = "notice"
log_level = "info"
rpc_laddr = "tcp://0.0.0.0:46657"
`


+ 9
- 4
consensus/byzantine_test.go View File

@ -9,6 +9,7 @@ import (
"github.com/tendermint/tendermint/types"
. "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tmlibs/events"
"github.com/tendermint/tmlibs/log"
)
func init() {
@ -32,8 +33,10 @@ func TestByzantine(t *testing.T) {
css[0].SetTimeoutTicker(NewTimeoutTicker())
switches := make([]*p2p.Switch, N)
p2pLogger := log.TestingLogger().With("module", "p2p")
for i := 0; i < N; i++ {
switches[i] = p2p.NewSwitch(config.P2P)
switches[i].SetLogger(p2pLogger)
}
reactors := make([]p2p.Reactor, N)
@ -53,13 +56,14 @@ func TestByzantine(t *testing.T) {
// make byzantine
css[i].decideProposal = func(j int) func(int, int) {
return func(height, round int) {
byzantineDecideProposalFunc(height, round, css[j], switches[j])
byzantineDecideProposalFunc(t, height, round, css[j], switches[j])
}
}(i)
css[i].doPrevote = func(height, round int) {}
}
eventSwitch := events.NewEventSwitch()
eventSwitch.SetLogger(log.TestingLogger().With("module", "events"))
_, err := eventSwitch.Start()
if err != nil {
t.Fatalf("Failed to start switch: %v", err)
@ -67,6 +71,7 @@ func TestByzantine(t *testing.T) {
eventChans[i] = subscribeToEvent(eventSwitch, "tester", types.EventStringNewBlock(), 1)
conR := NewConsensusReactor(css[i], true) // so we dont start the consensus states
conR.SetLogger(log.TestingLogger())
conR.SetEventSwitch(eventSwitch)
var conRI p2p.Reactor
@ -115,7 +120,7 @@ func TestByzantine(t *testing.T) {
case <-eventChans[ind2]:
}
log.Notice("A block has been committed. Healing partition")
t.Log("A block has been committed. Healing partition")
// connect the partitions
p2p.Connect2Switches(switches, ind0, ind1)
@ -153,7 +158,7 @@ func TestByzantine(t *testing.T) {
//-------------------------------
// byzantine consensus functions
func byzantineDecideProposalFunc(height, round int, cs *ConsensusState, sw *p2p.Switch) {
func byzantineDecideProposalFunc(t *testing.T, height, round int, cs *ConsensusState, sw *p2p.Switch) {
// byzantine user should create two proposals and try to split the vote.
// Avoid sending on internalMsgQueue and running consensus state.
@ -174,7 +179,7 @@ func byzantineDecideProposalFunc(height, round int, cs *ConsensusState, sw *p2p.
// broadcast conflicting proposals/block parts to peers
peers := sw.Peers().List()
log.Notice("Byzantine: broadcasting conflicting proposals", "peers", len(peers))
t.Logf("Byzantine: broadcasting conflicting proposals to %d peers", len(peers))
for i, peer := range peers {
if i < len(peers)/2 {
go sendProposalAndParts(height, round, cs, peer, proposal1, block1Hash, blockParts1)


+ 15
- 1
consensus/common_test.go View File

@ -21,6 +21,7 @@ import (
"github.com/tendermint/tendermint/types"
. "github.com/tendermint/tmlibs/common"
dbm "github.com/tendermint/tmlibs/db"
"github.com/tendermint/tmlibs/log"
"github.com/tendermint/abci/example/counter"
"github.com/tendermint/abci/example/dummy"
@ -249,12 +250,15 @@ func newConsensusStateWithConfig(thisConfig *cfg.Config, state *sm.State, pv *ty
// Make Mempool
mempool := mempl.NewMempool(thisConfig.Mempool, proxyAppConnMem)
mempool.SetLogger(log.TestingLogger().With("module", "mempool"))
// Make ConsensusReactor
cs := NewConsensusState(thisConfig.Consensus, state, proxyAppConnCon, blockStore, mempool)
cs.SetLogger(log.TestingLogger())
cs.SetPrivValidator(pv)
evsw := types.NewEventSwitch()
evsw.SetLogger(log.TestingLogger().With("module", "events"))
cs.SetEventSwitch(evsw)
evsw.Start()
return cs
@ -263,7 +267,7 @@ func newConsensusStateWithConfig(thisConfig *cfg.Config, state *sm.State, pv *ty
func loadPrivValidator(config *cfg.Config) *types.PrivValidator {
privValidatorFile := config.PrivValidatorFile()
ensureDir(path.Dir(privValidatorFile), 0700)
privValidator := types.LoadOrGenPrivValidator(privValidatorFile)
privValidator := types.LoadOrGenPrivValidator(privValidatorFile, log.TestingLogger())
privValidator.Reset()
return privValidator
}
@ -271,16 +275,20 @@ func loadPrivValidator(config *cfg.Config) *types.PrivValidator {
func fixedConsensusState() *ConsensusState {
stateDB := dbm.NewMemDB()
state := sm.MakeGenesisStateFromFile(stateDB, config.GenesisFile())
state.SetLogger(log.TestingLogger().With("module", "state"))
privValidator := loadPrivValidator(config)
cs := newConsensusState(state, privValidator, counter.NewCounterApplication(true))
cs.SetLogger(log.TestingLogger())
return cs
}
func fixedConsensusStateDummy() *ConsensusState {
stateDB := dbm.NewMemDB()
state := sm.MakeGenesisStateFromFile(stateDB, config.GenesisFile())
state.SetLogger(log.TestingLogger().With("module", "state"))
privValidator := loadPrivValidator(config)
cs := newConsensusState(state, privValidator, dummy.NewDummyApplication())
cs.SetLogger(log.TestingLogger())
return cs
}
@ -291,6 +299,7 @@ func randConsensusState(nValidators int) (*ConsensusState, []*validatorStub) {
vss := make([]*validatorStub, nValidators)
cs := newConsensusState(state, privVals[0], counter.NewCounterApplication(true))
cs.SetLogger(log.TestingLogger())
for i := 0; i < nValidators; i++ {
vss[i] = NewValidatorStub(privVals[i], i)
@ -322,10 +331,12 @@ func randConsensusNet(nValidators int, testName string, tickerFunc func() Timeou
for i := 0; i < nValidators; i++ {
db := dbm.NewMemDB() // each state needs its own db
state := sm.MakeGenesisState(db, genDoc)
state.SetLogger(log.TestingLogger().With("module", "state"))
state.Save()
thisConfig := ResetConfig(Fmt("%s_%d", testName, i))
ensureDir(path.Dir(thisConfig.Consensus.WalFile()), 0700) // dir for wal
css[i] = newConsensusStateWithConfig(thisConfig, state, privVals[i], appFunc())
css[i].SetLogger(log.TestingLogger())
css[i].SetTimeoutTicker(tickerFunc())
}
return css
@ -338,6 +349,7 @@ func randConsensusNetWithPeers(nValidators, nPeers int, testName string, tickerF
for i := 0; i < nPeers; i++ {
db := dbm.NewMemDB() // each state needs its own db
state := sm.MakeGenesisState(db, genDoc)
state.SetLogger(log.TestingLogger().With("module", "state"))
state.Save()
thisConfig := ResetConfig(Fmt("%s_%d", testName, i))
ensureDir(path.Dir(thisConfig.Consensus.WalFile()), 0700) // dir for wal
@ -351,6 +363,7 @@ func randConsensusNetWithPeers(nValidators, nPeers int, testName string, tickerF
}
css[i] = newConsensusStateWithConfig(thisConfig, state, privVal, appFunc())
css[i].SetLogger(log.TestingLogger())
css[i].SetTimeoutTicker(tickerFunc())
}
return css
@ -392,6 +405,7 @@ func randGenesisState(numValidators int, randPower bool, minPower int64) (*sm.St
genDoc, privValidators := randGenesisDoc(numValidators, randPower, minPower)
db := dbm.NewMemDB()
s0 := sm.MakeGenesisState(db, genDoc)
s0.SetLogger(log.TestingLogger().With("module", "state"))
s0.Save()
return s0, privValidators
}


+ 3
- 3
consensus/height_vote_set.go View File

@ -4,8 +4,8 @@ import (
"strings"
"sync"
. "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tendermint/types"
. "github.com/tendermint/tmlibs/common"
)
type RoundVoteSet struct {
@ -91,7 +91,7 @@ func (hvs *HeightVoteSet) addRound(round int) {
if _, ok := hvs.roundVoteSets[round]; ok {
PanicSanity("addRound() for an existing round")
}
log.Debug("addRound(round)", "round", round)
// log.Debug("addRound(round)", "round", round)
prevotes := types.NewVoteSet(hvs.chainID, hvs.height, round, types.VoteTypePrevote, hvs.valSet)
precommits := types.NewVoteSet(hvs.chainID, hvs.height, round, types.VoteTypePrecommit, hvs.valSet)
hvs.roundVoteSets[round] = RoundVoteSet{
@ -118,7 +118,7 @@ func (hvs *HeightVoteSet) AddVote(vote *types.Vote, peerKey string) (added bool,
// Peer has sent a vote that does not match our round,
// for more than one round. Bad peer!
// TODO punish peer.
log.Warn("Deal with peer giving votes from unwanted rounds")
// log.Warn("Deal with peer giving votes from unwanted rounds")
return
}
}


+ 0
- 18
consensus/log.go View File

@ -1,18 +0,0 @@
package consensus
import (
"github.com/tendermint/tmlibs/logger"
)
var log = logger.New("module", "consensus")
/*
func init() {
log.SetHandler(
logger.LvlFilterHandler(
logger.LvlDebug,
logger.BypassHandler(),
),
)
}
*/

+ 42
- 42
consensus/reactor.go View File

@ -8,7 +8,7 @@ import (
"sync"
"time"
"github.com/tendermint/go-wire"
wire "github.com/tendermint/go-wire"
"github.com/tendermint/tendermint/p2p"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
@ -41,12 +41,12 @@ func NewConsensusReactor(consensusState *ConsensusState, fastSync bool) *Consens
conS: consensusState,
fastSync: fastSync,
}
conR.BaseReactor = *p2p.NewBaseReactor(log, "ConsensusReactor", conR)
conR.BaseReactor = *p2p.NewBaseReactor("ConsensusReactor", conR)
return conR
}
func (conR *ConsensusReactor) OnStart() error {
log.Notice("ConsensusReactor ", "fastSync", conR.fastSync)
conR.Logger.Info("ConsensusReactor ", "fastSync", conR.fastSync)
conR.BaseReactor.OnStart()
// callbacks for broadcasting new steps and votes to peers
@ -70,7 +70,7 @@ func (conR *ConsensusReactor) OnStop() {
// Switch from the fast_sync to the consensus:
// reset the state, turn off fast_sync, start the consensus-state-machine
func (conR *ConsensusReactor) SwitchToConsensus(state *sm.State) {
log.Notice("SwitchToConsensus")
conR.Logger.Info("SwitchToConsensus")
conR.conS.reconstructLastCommit(state)
// NOTE: The line below causes broadcastNewRoundStepRoutine() to
// broadcast a NewRoundStepMessage.
@ -148,17 +148,17 @@ func (conR *ConsensusReactor) RemovePeer(peer *p2p.Peer, reason interface{}) {
// NOTE: blocks on consensus state for proposals, block parts, and votes
func (conR *ConsensusReactor) Receive(chID byte, src *p2p.Peer, msgBytes []byte) {
if !conR.IsRunning() {
log.Debug("Receive", "src", src, "chId", chID, "bytes", msgBytes)
conR.Logger.Debug("Receive", "src", src, "chId", chID, "bytes", msgBytes)
return
}
_, msg, err := DecodeMessage(msgBytes)
if err != nil {
log.Warn("Error decoding message", "src", src, "chId", chID, "msg", msg, "error", err, "bytes", msgBytes)
conR.Logger.Error("Error decoding message", "src", src, "chId", chID, "msg", msg, "error", err, "bytes", msgBytes)
// TODO punish peer?
return
}
log.Debug("Receive", "src", src, "chId", chID, "msg", msg)
conR.Logger.Debug("Receive", "src", src, "chId", chID, "msg", msg)
// Get peer states
ps := src.Data.Get(types.PeerStateKey).(*PeerState)
@ -191,7 +191,7 @@ func (conR *ConsensusReactor) Receive(chID byte, src *p2p.Peer, msgBytes []byte)
case types.VoteTypePrecommit:
ourVotes = votes.Precommits(msg.Round).BitArrayByBlockID(msg.BlockID)
default:
log.Warn("Bad VoteSetBitsMessage field Type")
conR.Logger.Error("Bad VoteSetBitsMessage field Type")
return
}
src.TrySend(VoteSetBitsChannel, struct{ ConsensusMessage }{&VoteSetBitsMessage{
@ -202,12 +202,12 @@ func (conR *ConsensusReactor) Receive(chID byte, src *p2p.Peer, msgBytes []byte)
Votes: ourVotes,
}})
default:
log.Warn(Fmt("Unknown message type %v", reflect.TypeOf(msg)))
conR.Logger.Error(Fmt("Unknown message type %v", reflect.TypeOf(msg)))
}
case DataChannel:
if conR.fastSync {
log.Warn("Ignoring message received during fastSync", "msg", msg)
conR.Logger.Error("Ignoring message received during fastSync", "msg", msg)
return
}
switch msg := msg.(type) {
@ -220,12 +220,12 @@ func (conR *ConsensusReactor) Receive(chID byte, src *p2p.Peer, msgBytes []byte)
ps.SetHasProposalBlockPart(msg.Height, msg.Round, msg.Part.Index)
conR.conS.peerMsgQueue <- msgInfo{msg, src.Key}
default:
log.Warn(Fmt("Unknown message type %v", reflect.TypeOf(msg)))
conR.Logger.Error(Fmt("Unknown message type %v", reflect.TypeOf(msg)))
}
case VoteChannel:
if conR.fastSync {
log.Warn("Ignoring message received during fastSync", "msg", msg)
conR.Logger.Info("Ignoring message received during fastSync", "msg", msg)
return
}
switch msg := msg.(type) {
@ -242,12 +242,12 @@ func (conR *ConsensusReactor) Receive(chID byte, src *p2p.Peer, msgBytes []byte)
default:
// don't punish (leave room for soft upgrades)
log.Warn(Fmt("Unknown message type %v", reflect.TypeOf(msg)))
conR.Logger.Error(Fmt("Unknown message type %v", reflect.TypeOf(msg)))
}
case VoteSetBitsChannel:
if conR.fastSync {
log.Warn("Ignoring message received during fastSync", "msg", msg)
conR.Logger.Info("Ignoring message received during fastSync", "msg", msg)
return
}
switch msg := msg.(type) {
@ -265,7 +265,7 @@ func (conR *ConsensusReactor) Receive(chID byte, src *p2p.Peer, msgBytes []byte)
case types.VoteTypePrecommit:
ourVotes = votes.Precommits(msg.Round).BitArrayByBlockID(msg.BlockID)
default:
log.Warn("Bad VoteSetBitsMessage field Type")
conR.Logger.Error("Bad VoteSetBitsMessage field Type")
return
}
ps.ApplyVoteSetBitsMessage(msg, ourVotes)
@ -274,15 +274,15 @@ func (conR *ConsensusReactor) Receive(chID byte, src *p2p.Peer, msgBytes []byte)
}
default:
// don't punish (leave room for soft upgrades)
log.Warn(Fmt("Unknown message type %v", reflect.TypeOf(msg)))
conR.Logger.Error(Fmt("Unknown message type %v", reflect.TypeOf(msg)))
}
default:
log.Warn(Fmt("Unknown chId %X", chID))
conR.Logger.Error(Fmt("Unknown chId %X", chID))
}
if err != nil {
log.Warn("Error in Receive()", "error", err)
conR.Logger.Error("Error in Receive()", "error", err)
}
}
@ -376,13 +376,13 @@ func (conR *ConsensusReactor) sendNewRoundStepMessages(peer *p2p.Peer) {
}
func (conR *ConsensusReactor) gossipDataRoutine(peer *p2p.Peer, ps *PeerState) {
log := log.New("peer", peer)
logger := conR.Logger.With("peer", peer)
OUTER_LOOP:
for {
// Manage disconnects from self or peer.
if !peer.IsRunning() || !conR.IsRunning() {
log.Notice(Fmt("Stopping gossipDataRoutine for %v.", peer))
logger.Info("Stopping gossipDataRoutine for peer")
return
}
rs := conR.conS.GetRoundState()
@ -390,7 +390,7 @@ OUTER_LOOP:
// Send proposal Block parts?
if rs.ProposalBlockParts.HasHeader(prs.ProposalBlockPartsHeader) {
//log.Info("ProposalBlockParts matched", "blockParts", prs.ProposalBlockParts)
//logger.Info("ProposalBlockParts matched", "blockParts", prs.ProposalBlockParts)
if index, ok := rs.ProposalBlockParts.BitArray().Sub(prs.ProposalBlockParts.Copy()).PickRandom(); ok {
part := rs.ProposalBlockParts.GetPart(index)
msg := &BlockPartMessage{
@ -407,16 +407,16 @@ OUTER_LOOP:
// If the peer is on a previous height, help catch up.
if (0 < prs.Height) && (prs.Height < rs.Height) {
//log.Info("Data catchup", "height", rs.Height, "peerHeight", prs.Height, "peerProposalBlockParts", prs.ProposalBlockParts)
//logger.Info("Data catchup", "height", rs.Height, "peerHeight", prs.Height, "peerProposalBlockParts", prs.ProposalBlockParts)
if index, ok := prs.ProposalBlockParts.Not().PickRandom(); ok {
// Ensure that the peer's PartSetHeader is correct
blockMeta := conR.conS.blockStore.LoadBlockMeta(prs.Height)
if blockMeta == nil {
log.Warn("Failed to load block meta", "peer height", prs.Height, "our height", rs.Height, "blockstore height", conR.conS.blockStore.Height(), "pv", conR.conS.privValidator)
logger.Error("Failed to load block meta", "peer height", prs.Height, "our height", rs.Height, "blockstore height", conR.conS.blockStore.Height(), "pv", conR.conS.privValidator)
time.Sleep(peerGossipSleepDuration)
continue OUTER_LOOP
} else if !blockMeta.BlockID.PartsHeader.Equals(prs.ProposalBlockPartsHeader) {
log.Info("Peer ProposalBlockPartsHeader mismatch, sleeping",
logger.Info("Peer ProposalBlockPartsHeader mismatch, sleeping",
"peerHeight", prs.Height, "blockPartsHeader", blockMeta.BlockID.PartsHeader, "peerBlockPartsHeader", prs.ProposalBlockPartsHeader)
time.Sleep(peerGossipSleepDuration)
continue OUTER_LOOP
@ -424,7 +424,7 @@ OUTER_LOOP:
// Load the part
part := conR.conS.blockStore.LoadBlockPart(prs.Height, index)
if part == nil {
log.Warn("Could not load part", "index", index,
logger.Error("Could not load part", "index", index,
"peerHeight", prs.Height, "blockPartsHeader", blockMeta.BlockID.PartsHeader, "peerBlockPartsHeader", prs.ProposalBlockPartsHeader)
time.Sleep(peerGossipSleepDuration)
continue OUTER_LOOP
@ -440,7 +440,7 @@ OUTER_LOOP:
}
continue OUTER_LOOP
} else {
//log.Info("No parts to send in catch-up, sleeping")
//logger.Info("No parts to send in catch-up, sleeping")
time.Sleep(peerGossipSleepDuration)
continue OUTER_LOOP
}
@ -448,7 +448,7 @@ OUTER_LOOP:
// If height and round don't match, sleep.
if (rs.Height != prs.Height) || (rs.Round != prs.Round) {
//log.Info("Peer Height|Round mismatch, sleeping", "peerHeight", prs.Height, "peerRound", prs.Round, "peer", peer)
//logger.Info("Peer Height|Round mismatch, sleeping", "peerHeight", prs.Height, "peerRound", prs.Round, "peer", peer)
time.Sleep(peerGossipSleepDuration)
continue OUTER_LOOP
}
@ -489,7 +489,7 @@ OUTER_LOOP:
}
func (conR *ConsensusReactor) gossipVotesRoutine(peer *p2p.Peer, ps *PeerState) {
log := log.New("peer", peer)
logger := conR.Logger.With("peer", peer)
// Simple hack to throttle logs upon sleep.
var sleeping = 0
@ -498,7 +498,7 @@ OUTER_LOOP:
for {
// Manage disconnects from self or peer.
if !peer.IsRunning() || !conR.IsRunning() {
log.Notice(Fmt("Stopping gossipVotesRoutine for %v.", peer))
logger.Info("Stopping gossipVotesRoutine for peer")
return
}
rs := conR.conS.GetRoundState()
@ -511,7 +511,7 @@ OUTER_LOOP:
sleeping = 0
}
//log.Debug("gossipVotesRoutine", "rsHeight", rs.Height, "rsRound", rs.Round,
//logger.Debug("gossipVotesRoutine", "rsHeight", rs.Height, "rsRound", rs.Round,
// "prsHeight", prs.Height, "prsRound", prs.Round, "prsStep", prs.Step)
// If height matches, then send LastCommit, Prevotes, Precommits.
@ -519,21 +519,21 @@ OUTER_LOOP:
// If there are lastCommits to send...
if prs.Step == RoundStepNewHeight {
if ps.PickSendVote(rs.LastCommit) {
log.Debug("Picked rs.LastCommit to send")
logger.Debug("Picked rs.LastCommit to send")
continue OUTER_LOOP
}
}
// If there are prevotes to send...
if prs.Step <= RoundStepPrevote && prs.Round != -1 && prs.Round <= rs.Round {
if ps.PickSendVote(rs.Votes.Prevotes(prs.Round)) {
log.Debug("Picked rs.Prevotes(prs.Round) to send")
logger.Debug("Picked rs.Prevotes(prs.Round) to send")
continue OUTER_LOOP
}
}
// If there are precommits to send...
if prs.Step <= RoundStepPrecommit && prs.Round != -1 && prs.Round <= rs.Round {
if ps.PickSendVote(rs.Votes.Precommits(prs.Round)) {
log.Debug("Picked rs.Precommits(prs.Round) to send")
logger.Debug("Picked rs.Precommits(prs.Round) to send")
continue OUTER_LOOP
}
}
@ -541,7 +541,7 @@ OUTER_LOOP:
if prs.ProposalPOLRound != -1 {
if polPrevotes := rs.Votes.Prevotes(prs.ProposalPOLRound); polPrevotes != nil {
if ps.PickSendVote(polPrevotes) {
log.Debug("Picked rs.Prevotes(prs.ProposalPOLRound) to send")
logger.Debug("Picked rs.Prevotes(prs.ProposalPOLRound) to send")
continue OUTER_LOOP
}
}
@ -552,7 +552,7 @@ OUTER_LOOP:
// If peer is lagging by height 1, send LastCommit.
if prs.Height != 0 && rs.Height == prs.Height+1 {
if ps.PickSendVote(rs.LastCommit) {
log.Debug("Picked rs.LastCommit to send")
logger.Debug("Picked rs.LastCommit to send")
continue OUTER_LOOP
}
}
@ -563,9 +563,9 @@ OUTER_LOOP:
// Load the block commit for prs.Height,
// which contains precommit signatures for prs.Height.
commit := conR.conS.blockStore.LoadBlockCommit(prs.Height)
log.Info("Loaded BlockCommit for catch-up", "height", prs.Height, "commit", commit)
logger.Info("Loaded BlockCommit for catch-up", "height", prs.Height, "commit", commit)
if ps.PickSendVote(commit) {
log.Debug("Picked Catchup commit to send")
logger.Debug("Picked Catchup commit to send")
continue OUTER_LOOP
}
}
@ -573,7 +573,7 @@ OUTER_LOOP:
if sleeping == 0 {
// We sent nothing. Sleep...
sleeping = 1
log.Debug("No votes to send, sleeping", "peer", peer,
logger.Debug("No votes to send, sleeping", "peer", peer,
"localPV", rs.Votes.Prevotes(rs.Round).BitArray(), "peerPV", prs.Prevotes,
"localPC", rs.Votes.Precommits(rs.Round).BitArray(), "peerPC", prs.Precommits)
} else if sleeping == 2 {
@ -589,13 +589,13 @@ OUTER_LOOP:
// NOTE: `queryMaj23Routine` has a simple crude design since it only comes
// into play for liveness when there's a signature DDoS attack happening.
func (conR *ConsensusReactor) queryMaj23Routine(peer *p2p.Peer, ps *PeerState) {
log := log.New("peer", peer)
logger := conR.Logger.With("peer", peer)
OUTER_LOOP:
for {
// Manage disconnects from self or peer.
if !peer.IsRunning() || !conR.IsRunning() {
log.Notice(Fmt("Stopping queryMaj23Routine for %v.", peer))
logger.Info("Stopping queryMaj23Routine for peer")
return
}
@ -952,8 +952,8 @@ func (ps *PeerState) SetHasVote(vote *types.Vote) {
}
func (ps *PeerState) setHasVote(height int, round int, type_ byte, index int) {
log := log.New("peer", ps.Peer, "peerRound", ps.Round, "height", height, "round", round)
log.Debug("setHasVote(LastCommit)", "lastCommit", ps.LastCommit, "index", index)
logger := ps.Peer.Logger.With("peerRound", ps.Round, "height", height, "round", round)
logger.Debug("setHasVote(LastCommit)", "lastCommit", ps.LastCommit, "index", index)
// NOTE: some may be nil BitArrays -> no side effects.
ps.getVoteBitArray(height, round, type_).SetIndex(index, true)


+ 10
- 7
consensus/reactor_test.go View File

@ -10,6 +10,7 @@ import (
"github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tendermint/types"
"github.com/tendermint/tmlibs/events"
"github.com/tendermint/tmlibs/log"
)
func init() {
@ -24,8 +25,10 @@ func startConsensusNet(t *testing.T, css []*ConsensusState, N int, subscribeEven
eventChans := make([]chan interface{}, N)
for i := 0; i < N; i++ {
reactors[i] = NewConsensusReactor(css[i], true) // so we dont start the consensus states
reactors[i].SetLogger(log.TestingLogger())
eventSwitch := events.NewEventSwitch()
eventSwitch.SetLogger(log.TestingLogger().With("module", "events"))
_, err := eventSwitch.Start()
if err != nil {
t.Fatalf("Failed to start switch: %v", err)
@ -96,7 +99,7 @@ func TestVotingPowerChange(t *testing.T) {
}, css)
//---------------------------------------------------------------------------
log.Info("---------------------------- Testing changing the voting power of one validator a few times")
t.Log("---------------------------- Testing changing the voting power of one validator a few times")
val1PubKey := css[0].privValidator.(*types.PrivValidator).PubKey
updateValidatorTx := dummy.MakeValSetChangeTx(val1PubKey.Bytes(), 25)
@ -157,7 +160,7 @@ func TestValidatorSetChanges(t *testing.T) {
}, css)
//---------------------------------------------------------------------------
log.Info("---------------------------- Testing adding one validator")
t.Log("---------------------------- Testing adding one validator")
newValidatorPubKey1 := css[nVals].privValidator.(*types.PrivValidator).PubKey
newValidatorTx1 := dummy.MakeValSetChangeTx(newValidatorPubKey1.Bytes(), uint64(testMinPower))
@ -183,7 +186,7 @@ func TestValidatorSetChanges(t *testing.T) {
waitForAndValidateBlock(t, nPeers, activeVals, eventChans, css)
//---------------------------------------------------------------------------
log.Info("---------------------------- Testing changing the voting power of one validator")
t.Log("---------------------------- Testing changing the voting power of one validator")
updateValidatorPubKey1 := css[nVals].privValidator.(*types.PrivValidator).PubKey
updateValidatorTx1 := dummy.MakeValSetChangeTx(updateValidatorPubKey1.Bytes(), 25)
@ -199,7 +202,7 @@ func TestValidatorSetChanges(t *testing.T) {
}
//---------------------------------------------------------------------------
log.Info("---------------------------- Testing adding two validators at once")
t.Log("---------------------------- Testing adding two validators at once")
newValidatorPubKey2 := css[nVals+1].privValidator.(*types.PrivValidator).PubKey
newValidatorTx2 := dummy.MakeValSetChangeTx(newValidatorPubKey2.Bytes(), uint64(testMinPower))
@ -215,7 +218,7 @@ func TestValidatorSetChanges(t *testing.T) {
waitForAndValidateBlock(t, nPeers, activeVals, eventChans, css)
//---------------------------------------------------------------------------
log.Info("---------------------------- Testing removing two validators at once")
t.Log("---------------------------- Testing removing two validators at once")
removeValidatorTx2 := dummy.MakeValSetChangeTx(newValidatorPubKey2.Bytes(), 0)
removeValidatorTx3 := dummy.MakeValSetChangeTx(newValidatorPubKey3.Bytes(), 0)
@ -251,7 +254,7 @@ func waitForAndValidateBlock(t *testing.T, n int, activeVals map[string]struct{}
timeoutWaitGroup(t, n, func(wg *sync.WaitGroup, j int) {
newBlockI := <-eventChans[j]
newBlock := newBlockI.(types.TMEventData).Unwrap().(types.EventDataNewBlock).Block
log.Warn("Got block", "height", newBlock.Height, "validator", j)
t.Logf("[WARN] Got block height=%v validator=%v", newBlock.Height, j)
err := validateBlock(newBlock, activeVals)
if err != nil {
t.Fatal(err)
@ -262,7 +265,7 @@ func waitForAndValidateBlock(t *testing.T, n int, activeVals map[string]struct{}
eventChans[j] <- struct{}{}
wg.Done()
log.Warn("Done wait group", "height", newBlock.Height, "validator", j)
t.Logf("[WARN] Done wait group height=%v validator=%v", newBlock.Height, j)
}, css)
}


+ 27
- 21
consensus/replay.go View File

@ -11,9 +11,10 @@ import (
"time"
abci "github.com/tendermint/abci/types"
"github.com/tendermint/go-wire"
wire "github.com/tendermint/go-wire"
auto "github.com/tendermint/tmlibs/autofile"
cmn "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tmlibs/log"
"github.com/tendermint/tendermint/proxy"
sm "github.com/tendermint/tendermint/state"
@ -49,7 +50,7 @@ func (cs *ConsensusState) readReplayMessage(msgBytes []byte, newStepCh chan inte
// for logging
switch m := msg.Msg.(type) {
case types.EventDataRoundState:
log.Notice("Replay: New Step", "height", m.Height, "round", m.Round, "step", m.Step)
cs.Logger.Info("Replay: New Step", "height", m.Height, "round", m.Round, "step", m.Step)
// these are playback checks
ticker := time.After(time.Second * 2)
if newStepCh != nil {
@ -71,19 +72,19 @@ func (cs *ConsensusState) readReplayMessage(msgBytes []byte, newStepCh chan inte
switch msg := m.Msg.(type) {
case *ProposalMessage:
p := msg.Proposal
log.Notice("Replay: Proposal", "height", p.Height, "round", p.Round, "header",
cs.Logger.Info("Replay: Proposal", "height", p.Height, "round", p.Round, "header",
p.BlockPartsHeader, "pol", p.POLRound, "peer", peerKey)
case *BlockPartMessage:
log.Notice("Replay: BlockPart", "height", msg.Height, "round", msg.Round, "peer", peerKey)
cs.Logger.Info("Replay: BlockPart", "height", msg.Height, "round", msg.Round, "peer", peerKey)
case *VoteMessage:
v := msg.Vote
log.Notice("Replay: Vote", "height", v.Height, "round", v.Round, "type", v.Type,
cs.Logger.Info("Replay: Vote", "height", v.Height, "round", v.Round, "type", v.Type,
"blockID", v.BlockID, "peer", peerKey)
}
cs.handleMsg(m, cs.RoundState)
case timeoutInfo:
log.Notice("Replay: Timeout", "height", m.Height, "round", m.Round, "step", m.Step, "dur", m.Duration)
cs.Logger.Info("Replay: Timeout", "height", m.Height, "round", m.Round, "step", m.Step, "dur", m.Duration)
cs.handleTimeout(m, cs.RoundState)
default:
return fmt.Errorf("Replay: Unknown TimedWALMessage type: %v", reflect.TypeOf(msg.Msg))
@ -113,12 +114,12 @@ func (cs *ConsensusState) catchupReplay(csHeight int) error {
// Search for last height marker
gr, found, err = cs.wal.group.Search("#ENDHEIGHT: ", makeHeightSearchFunc(csHeight-1))
if err == io.EOF {
log.Warn("Replay: wal.group.Search returned EOF", "#ENDHEIGHT", csHeight-1)
cs.Logger.Error("Replay: wal.group.Search returned EOF", "#ENDHEIGHT", csHeight-1)
// if we upgraded from 0.9 to 0.9.1, we may have #HEIGHT instead
// TODO (0.10.0): remove this
gr, found, err = cs.wal.group.Search("#HEIGHT: ", makeHeightSearchFunc(csHeight))
if err == io.EOF {
log.Warn("Replay: wal.group.Search returned EOF", "#HEIGHT", csHeight)
cs.Logger.Error("Replay: wal.group.Search returned EOF", "#HEIGHT", csHeight)
return nil
} else if err != nil {
return err
@ -133,7 +134,7 @@ func (cs *ConsensusState) catchupReplay(csHeight int) error {
// TODO (0.10.0): remove this
gr, found, err = cs.wal.group.Search("#HEIGHT: ", makeHeightSearchFunc(csHeight))
if err == io.EOF {
log.Warn("Replay: wal.group.Search returned EOF", "#HEIGHT", csHeight)
cs.Logger.Error("Replay: wal.group.Search returned EOF", "#HEIGHT", csHeight)
return nil
} else if err != nil {
return err
@ -145,7 +146,7 @@ func (cs *ConsensusState) catchupReplay(csHeight int) error {
// return errors.New(cmn.Fmt("Cannot replay height %d. WAL does not contain #ENDHEIGHT for %d.", csHeight, csHeight-1))
}
log.Notice("Catchup by replaying consensus messages", "height", csHeight)
cs.Logger.Info("Catchup by replaying consensus messages", "height", csHeight)
for {
line, err := gr.ReadLine()
@ -163,7 +164,7 @@ func (cs *ConsensusState) catchupReplay(csHeight int) error {
return err
}
}
log.Notice("Replay: Done")
cs.Logger.Info("Replay: Done")
return nil
}
@ -198,14 +199,19 @@ func makeHeightSearchFunc(height int) auto.SearchFunc {
// we were last and using the WAL to recover there
type Handshaker struct {
state *sm.State
store types.BlockStore
state *sm.State
store types.BlockStore
logger log.Logger
nBlocks int // number of blocks applied to the state
}
func NewHandshaker(state *sm.State, store types.BlockStore) *Handshaker {
return &Handshaker{state, store, 0}
return &Handshaker{state, store, log.NewNopLogger(), 0}
}
func (h *Handshaker) SetLogger(l log.Logger) {
h.logger = l
}
func (h *Handshaker) NBlocks() int {
@ -223,7 +229,7 @@ func (h *Handshaker) Handshake(proxyApp proxy.AppConns) error {
blockHeight := int(res.LastBlockHeight) // XXX: beware overflow
appHash := res.LastBlockAppHash
log.Notice("ABCI Handshake", "appHeight", blockHeight, "appHash", appHash)
h.logger.Info("ABCI Handshake", "appHeight", blockHeight, "appHash", appHash)
// TODO: check version
@ -233,7 +239,7 @@ func (h *Handshaker) Handshake(proxyApp proxy.AppConns) error {
return errors.New(cmn.Fmt("Error on replay: %v", err))
}
log.Notice("Completed ABCI Handshake - Tendermint and App are synced", "appHeight", blockHeight, "appHash", appHash)
h.logger.Info("Completed ABCI Handshake - Tendermint and App are synced", "appHeight", blockHeight, "appHash", appHash)
// TODO: (on restart) replay mempool
@ -246,7 +252,7 @@ func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp p
storeBlockHeight := h.store.Height()
stateBlockHeight := h.state.LastBlockHeight
log.Notice("ABCI Replay Blocks", "appHeight", appBlockHeight, "storeHeight", storeBlockHeight, "stateHeight", stateBlockHeight)
h.logger.Info("ABCI Replay Blocks", "appHeight", appBlockHeight, "storeHeight", storeBlockHeight, "stateHeight", stateBlockHeight)
// If appBlockHeight == 0 it means that we are at genesis and hence should send InitChain
if appBlockHeight == 0 {
@ -298,14 +304,14 @@ func (h *Handshaker) ReplayBlocks(appHash []byte, appBlockHeight int, proxyApp p
// so replayBlock with the real app.
// NOTE: We could instead use the cs.WAL on cs.Start,
// but we'd have to allow the WAL to replay a block that wrote it's ENDHEIGHT
log.Info("Replay last block using real app")
h.logger.Info("Replay last block using real app")
return h.replayBlock(storeBlockHeight, proxyApp.Consensus())
} else if appBlockHeight == storeBlockHeight {
// We ran Commit, but didn't save the state, so replayBlock with mock app
abciResponses := h.state.LoadABCIResponses()
mockApp := newMockProxyApp(appHash, abciResponses)
log.Info("Replay last block using mock app")
h.logger.Info("Replay last block using mock app")
return h.replayBlock(storeBlockHeight, mockApp)
}
@ -329,9 +335,9 @@ func (h *Handshaker) replayBlocks(proxyApp proxy.AppConns, appBlockHeight, store
finalBlock -= 1
}
for i := appBlockHeight + 1; i <= finalBlock; i++ {
log.Info("Applying block", "height", i)
h.logger.Info("Applying block", "height", i)
block := h.store.LoadBlock(i)
appHash, err = sm.ExecCommitBlock(proxyApp.Consensus(), block)
appHash, err = sm.ExecCommitBlock(proxyApp.Consensus(), block, h.logger)
if err != nil {
return nil, err
}


+ 3
- 3
consensus/replay_file.go View File

@ -113,7 +113,7 @@ func (pb *playback) replayReset(count int, newStepCh chan interface{}) error {
pb.fp = fp
pb.scanner = bufio.NewScanner(fp)
count = pb.count - count
log.Notice(cmn.Fmt("Reseting from %d to %d", pb.count, count))
fmt.Printf("Reseting from %d to %d\n", pb.count, count)
pb.count = 0
pb.cs = newCS
for i := 0; pb.scanner.Scan() && i < count; i++ {
@ -126,8 +126,7 @@ func (pb *playback) replayReset(count int, newStepCh chan interface{}) error {
}
func (cs *ConsensusState) startForReplay() {
log.Warn("Replay commands are disabled until someone updates them and writes tests")
cs.Logger.Error("Replay commands are disabled until someone updates them and writes tests")
/* TODO:!
// since we replay tocks we just ignore ticks
go func() {
@ -259,6 +258,7 @@ func newConsensusStateForReplay(config cfg.BaseConfig, csConfig *cfg.ConsensusCo
}
consensusState := NewConsensusState(csConfig, state.Copy(), proxyApp.Consensus(), blockStore, types.MockMempool{})
consensusState.SetEventSwitch(eventSwitch)
return consensusState
}

+ 14
- 10
consensus/replay_test.go View File

@ -13,8 +13,8 @@ import (
"time"
"github.com/tendermint/abci/example/dummy"
"github.com/tendermint/go-crypto"
"github.com/tendermint/go-wire"
crypto "github.com/tendermint/go-crypto"
wire "github.com/tendermint/go-wire"
cmn "github.com/tendermint/tmlibs/common"
dbm "github.com/tendermint/tmlibs/db"
@ -22,6 +22,7 @@ import (
"github.com/tendermint/tendermint/proxy"
sm "github.com/tendermint/tendermint/state"
"github.com/tendermint/tendermint/types"
"github.com/tendermint/tmlibs/log"
)
func init() {
@ -165,9 +166,9 @@ func toPV(pv PrivValidator) *types.PrivValidator {
return pv.(*types.PrivValidator)
}
func setupReplayTest(thisCase *testCase, nLines int, crashAfter bool) (*ConsensusState, chan interface{}, string, string) {
fmt.Println("-------------------------------------")
log.Notice(cmn.Fmt("Starting replay test %v (of %d lines of WAL). Crash after = %v", thisCase.name, nLines, crashAfter))
func setupReplayTest(t *testing.T, thisCase *testCase, nLines int, crashAfter bool) (*ConsensusState, chan interface{}, string, string) {
t.Log("-------------------------------------")
t.Logf("Starting replay test %v (of %d lines of WAL). Crash after = %v", thisCase.name, nLines, crashAfter)
lineStep := nLines
if crashAfter {
@ -186,7 +187,7 @@ func setupReplayTest(thisCase *testCase, nLines int, crashAfter bool) (*Consensu
toPV(cs.privValidator).LastHeight = 1 // first block
toPV(cs.privValidator).LastStep = thisCase.stepMap[lineStep]
log.Warn("setupReplayTest", "LastStep", toPV(cs.privValidator).LastStep)
t.Logf("[WARN] setupReplayTest LastStep=%v", toPV(cs.privValidator).LastStep)
newBlockCh := subscribeToEvent(cs.evsw, "tester", types.EventStringNewBlock(), 1)
@ -211,7 +212,7 @@ func TestWALCrashAfterWrite(t *testing.T) {
for _, thisCase := range testCases {
split := strings.Split(thisCase.log, "\n")
for i := 0; i < len(split)-1; i++ {
cs, newBlockCh, _, walFile := setupReplayTest(thisCase, i+1, true)
cs, newBlockCh, _, walFile := setupReplayTest(t, thisCase, i+1, true)
runReplayTest(t, cs, walFile, newBlockCh, thisCase, i+1)
}
}
@ -225,7 +226,7 @@ func TestWALCrashBeforeWritePropose(t *testing.T) {
for _, thisCase := range testCases {
lineNum := thisCase.proposeLine
// setup replay test where last message is a proposal
cs, newBlockCh, proposalMsg, walFile := setupReplayTest(thisCase, lineNum, false)
cs, newBlockCh, proposalMsg, walFile := setupReplayTest(t, thisCase, lineNum, false)
msg := readTimedWALMessage(t, proposalMsg)
proposal := msg.Msg.(msgInfo).Msg.(*ProposalMessage)
// Set LastSig
@ -249,7 +250,7 @@ func TestWALCrashBeforeWritePrecommit(t *testing.T) {
func testReplayCrashBeforeWriteVote(t *testing.T, thisCase *testCase, lineNum int, eventString string) {
// setup replay test where last message is a vote
cs, newBlockCh, voteMsg, walFile := setupReplayTest(thisCase, lineNum, false)
cs, newBlockCh, voteMsg, walFile := setupReplayTest(t, thisCase, lineNum, false)
types.AddListenerForEvent(cs.evsw, "tester", eventString, func(data types.TMEventData) {
msg := readTimedWALMessage(t, voteMsg)
vote := msg.Msg.(msgInfo).Msg.(*VoteMessage)
@ -322,6 +323,7 @@ func testHandshakeReplay(t *testing.T, nBlocks int, mode uint) {
testPartSize = config.Consensus.BlockPartSize
wal, err := NewWAL(walFile, false)
wal.SetLogger(log.TestingLogger())
if err != nil {
t.Fatal(err)
}
@ -470,7 +472,7 @@ func makeBlockchainFromWAL(wal *WAL) ([]*types.Block, []*types.Commit, error) {
}
defer gr.Close()
log.Notice("Build a blockchain by reading from the WAL")
// log.Notice("Build a blockchain by reading from the WAL")
var blockParts *types.PartSet
var blocks []*types.Block
@ -617,6 +619,8 @@ func makeBlockchain(t *testing.T, chainID string, nBlocks int, privVal *types.Pr
func stateAndStore(config *cfg.Config, pubKey crypto.PubKey) (*sm.State, *mockBlockStore) {
stateDB := dbm.NewMemDB()
state := sm.MakeGenesisStateFromFile(stateDB, config.GenesisFile())
state.SetLogger(log.TestingLogger().With("module", "state"))
store := NewMockBlockStore(config)
return state, store
}


+ 62
- 62
consensus/state.go View File

@ -10,7 +10,7 @@ import (
"time"
fail "github.com/ebuchman/fail-test"
"github.com/tendermint/go-wire"
wire "github.com/tendermint/go-wire"
cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/proxy"
sm "github.com/tendermint/tendermint/state"
@ -237,7 +237,7 @@ func NewConsensusState(config *cfg.ConsensusConfig, state *sm.State, proxyAppCon
// Don't call scheduleRound0 yet.
// We do that upon Start().
cs.reconstructLastCommit(state)
cs.BaseService = *cmn.NewBaseService(log, "ConsensusState", cs)
cs.BaseService = *cmn.NewBaseService(nil, "ConsensusState", cs)
return cs
}
@ -304,7 +304,7 @@ func (cs *ConsensusState) OnStart() error {
walFile := cs.config.WalFile()
if err := cs.OpenWAL(walFile); err != nil {
log.Error("Error loading ConsensusState wal", "error", err.Error())
cs.Logger.Error("Error loading ConsensusState wal", "error", err.Error())
return err
}
@ -318,7 +318,7 @@ func (cs *ConsensusState) OnStart() error {
// we may have lost some votes if the process crashed
// reload from consensus log to catchup
if err := cs.catchupReplay(cs.Height); err != nil {
log.Error("Error on catchup replay. Proceeding to start ConsensusState anyway", "error", err.Error())
cs.Logger.Error("Error on catchup replay. Proceeding to start ConsensusState anyway", "error", err.Error())
// NOTE: if we ever do return an error here,
// make sure to stop the timeoutTicker
}
@ -361,7 +361,7 @@ func (cs *ConsensusState) Wait() {
func (cs *ConsensusState) OpenWAL(walFile string) (err error) {
err = cmn.EnsureDir(path.Dir(walFile), 0700)
if err != nil {
log.Error("Error ensuring ConsensusState wal dir", "error", err.Error())
cs.Logger.Error("Error ensuring ConsensusState wal dir", "error", err.Error())
return err
}
@ -442,7 +442,7 @@ func (cs *ConsensusState) updateRoundStep(round int, step RoundStepType) {
// enterNewRound(height, 0) at cs.StartTime.
func (cs *ConsensusState) scheduleRound0(rs *RoundState) {
//log.Info("scheduleRound0", "now", time.Now(), "startTime", cs.StartTime)
//cs.Logger.Info("scheduleRound0", "now", time.Now(), "startTime", cs.StartTime)
sleepDuration := rs.StartTime.Sub(time.Now())
cs.scheduleTimeout(sleepDuration, rs.Height, 0, RoundStepNewHeight)
}
@ -461,7 +461,7 @@ func (cs *ConsensusState) sendInternalMessage(mi msgInfo) {
// be processed out of order.
// TODO: use CList here for strict determinism and
// attempt push to internalMsgQueue in receiveRoutine
log.Warn("Internal msg queue is full. Using a go-routine")
cs.Logger.Info("Internal msg queue is full. Using a go-routine")
go func() { cs.internalMsgQueue <- mi }()
}
}
@ -507,7 +507,7 @@ func (cs *ConsensusState) updateToState(state *sm.State) {
// This happens when SwitchToConsensus() is called in the reactor.
// We don't want to reset e.g. the Votes.
if cs.state != nil && (state.LastBlockHeight <= cs.state.LastBlockHeight) {
log.Notice("Ignoring updateToState()", "newHeight", state.LastBlockHeight+1, "oldHeight", cs.state.LastBlockHeight+1)
cs.Logger.Info("Ignoring updateToState()", "newHeight", state.LastBlockHeight+1, "oldHeight", cs.state.LastBlockHeight+1)
return
}
@ -576,7 +576,7 @@ func (cs *ConsensusState) receiveRoutine(maxSteps int) {
for {
if maxSteps > 0 {
if cs.nSteps >= maxSteps {
log.Warn("reached max steps. exiting receive routine")
cs.Logger.Error("reached max steps. exiting receive routine")
cs.nSteps = 0
return
}
@ -649,19 +649,19 @@ func (cs *ConsensusState) handleMsg(mi msgInfo, rs RoundState) {
// the peer is sending us CatchupCommit precommits.
// We could make note of this and help filter in broadcastHasVoteMessage().
default:
log.Warn("Unknown msg type", reflect.TypeOf(msg))
cs.Logger.Error("Unknown msg type", reflect.TypeOf(msg))
}
if err != nil {
log.Error("Error with msg", "type", reflect.TypeOf(msg), "peer", peerKey, "error", err, "msg", msg)
cs.Logger.Error("Error with msg", "type", reflect.TypeOf(msg), "peer", peerKey, "error", err, "msg", msg)
}
}
func (cs *ConsensusState) handleTimeout(ti timeoutInfo, rs RoundState) {
log.Debug("Received tock", "timeout", ti.Duration, "height", ti.Height, "round", ti.Round, "step", ti.Step)
cs.Logger.Debug("Received tock", "timeout", ti.Duration, "height", ti.Height, "round", ti.Round, "step", ti.Step)
// timeouts must be for current height, round, step
if ti.Height != rs.Height || ti.Round < rs.Round || (ti.Round == rs.Round && ti.Step < rs.Step) {
log.Debug("Ignoring tock because we're ahead", "height", rs.Height, "round", rs.Round, "step", rs.Step)
cs.Logger.Debug("Ignoring tock because we're ahead", "height", rs.Height, "round", rs.Round, "step", rs.Step)
return
}
@ -699,15 +699,15 @@ func (cs *ConsensusState) handleTimeout(ti timeoutInfo, rs RoundState) {
// NOTE: cs.StartTime was already set for height.
func (cs *ConsensusState) enterNewRound(height int, round int) {
if cs.Height != height || round < cs.Round || (cs.Round == round && cs.Step != RoundStepNewHeight) {
log.Debug(cmn.Fmt("enterNewRound(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
cs.Logger.Debug(cmn.Fmt("enterNewRound(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
}
if now := time.Now(); cs.StartTime.After(now) {
log.Warn("Need to set a buffer and log.Warn() here for sanity.", "startTime", cs.StartTime, "now", now)
cs.Logger.Info("Need to set a buffer and log message here for sanity.", "startTime", cs.StartTime, "now", now)
}
log.Notice(cmn.Fmt("enterNewRound(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
cs.Logger.Info(cmn.Fmt("enterNewRound(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
// Increment validators if necessary
validators := cs.Validators
@ -741,10 +741,10 @@ func (cs *ConsensusState) enterNewRound(height int, round int) {
// Enter: from NewRound(height,round).
func (cs *ConsensusState) enterPropose(height int, round int) {
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPropose <= cs.Step) {
log.Debug(cmn.Fmt("enterPropose(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
cs.Logger.Debug(cmn.Fmt("enterPropose(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
}
log.Info(cmn.Fmt("enterPropose(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
cs.Logger.Info(cmn.Fmt("enterPropose(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
defer func() {
// Done enterPropose:
@ -768,9 +768,9 @@ func (cs *ConsensusState) enterPropose(height int, round int) {
}
if !bytes.Equal(cs.Validators.GetProposer().Address, cs.privValidator.GetAddress()) {
log.Info("enterPropose: Not our turn to propose", "proposer", cs.Validators.GetProposer().Address, "privValidator", cs.privValidator)
cs.Logger.Info("enterPropose: Not our turn to propose", "proposer", cs.Validators.GetProposer().Address, "privValidator", cs.privValidator)
} else {
log.Info("enterPropose: Our turn to propose", "proposer", cs.Validators.GetProposer().Address, "privValidator", cs.privValidator)
cs.Logger.Info("enterPropose: Our turn to propose", "proposer", cs.Validators.GetProposer().Address, "privValidator", cs.privValidator)
cs.decideProposal(height, round)
}
@ -810,11 +810,11 @@ func (cs *ConsensusState) defaultDecideProposal(height, round int) {
part := blockParts.GetPart(i)
cs.sendInternalMessage(msgInfo{&BlockPartMessage{cs.Height, cs.Round, part}, ""})
}
log.Info("Signed proposal", "height", height, "round", round, "proposal", proposal)
log.Debug(cmn.Fmt("Signed proposal block: %v", block))
cs.Logger.Info("Signed proposal", "height", height, "round", round, "proposal", proposal)
cs.Logger.Debug(cmn.Fmt("Signed proposal block: %v", block))
} else {
if !cs.replayMode {
log.Warn("enterPropose: Error signing proposal", "height", height, "round", round, "error", err)
cs.Logger.Error("enterPropose: Error signing proposal", "height", height, "round", round, "error", err)
}
}
}
@ -849,7 +849,7 @@ func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts
commit = cs.LastCommit.MakeCommit()
} else {
// This shouldn't happen.
log.Error("enterPropose: Cannot propose anything: No commit for the previous block.")
cs.Logger.Error("enterPropose: Cannot propose anything: No commit for the previous block.")
return
}
@ -867,7 +867,7 @@ func (cs *ConsensusState) createProposalBlock() (block *types.Block, blockParts
// Otherwise vote nil.
func (cs *ConsensusState) enterPrevote(height int, round int) {
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrevote <= cs.Step) {
log.Debug(cmn.Fmt("enterPrevote(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
cs.Logger.Debug(cmn.Fmt("enterPrevote(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
}
@ -885,7 +885,7 @@ func (cs *ConsensusState) enterPrevote(height int, round int) {
// TODO: catchup event?
}
log.Info(cmn.Fmt("enterPrevote(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
cs.Logger.Info(cmn.Fmt("enterPrevote(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
// Sign and broadcast vote as necessary
cs.doPrevote(height, round)
@ -897,14 +897,14 @@ func (cs *ConsensusState) enterPrevote(height int, round int) {
func (cs *ConsensusState) defaultDoPrevote(height int, round int) {
// If a block is locked, prevote that.
if cs.LockedBlock != nil {
log.Notice("enterPrevote: Block was locked")
cs.Logger.Info("enterPrevote: Block was locked")
cs.signAddVote(types.VoteTypePrevote, cs.LockedBlock.Hash(), cs.LockedBlockParts.Header())
return
}
// If ProposalBlock is nil, prevote nil.
if cs.ProposalBlock == nil {
log.Warn("enterPrevote: ProposalBlock is nil")
cs.Logger.Error("enterPrevote: ProposalBlock is nil")
cs.signAddVote(types.VoteTypePrevote, nil, types.PartSetHeader{})
return
}
@ -913,7 +913,7 @@ func (cs *ConsensusState) defaultDoPrevote(height int, round int) {
err := cs.state.ValidateBlock(cs.ProposalBlock)
if err != nil {
// ProposalBlock is invalid, prevote nil.
log.Warn("enterPrevote: ProposalBlock is invalid", "error", err)
cs.Logger.Error("enterPrevote: ProposalBlock is invalid", "error", err)
cs.signAddVote(types.VoteTypePrevote, nil, types.PartSetHeader{})
return
}
@ -928,13 +928,13 @@ func (cs *ConsensusState) defaultDoPrevote(height int, round int) {
// Enter: any +2/3 prevotes at next round.
func (cs *ConsensusState) enterPrevoteWait(height int, round int) {
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrevoteWait <= cs.Step) {
log.Debug(cmn.Fmt("enterPrevoteWait(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
cs.Logger.Debug(cmn.Fmt("enterPrevoteWait(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
}
if !cs.Votes.Prevotes(round).HasTwoThirdsAny() {
cmn.PanicSanity(cmn.Fmt("enterPrevoteWait(%v/%v), but Prevotes does not have any +2/3 votes", height, round))
}
log.Info(cmn.Fmt("enterPrevoteWait(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
cs.Logger.Info(cmn.Fmt("enterPrevoteWait(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
defer func() {
// Done enterPrevoteWait:
@ -954,11 +954,11 @@ func (cs *ConsensusState) enterPrevoteWait(height int, round int) {
// else, precommit nil otherwise.
func (cs *ConsensusState) enterPrecommit(height int, round int) {
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrecommit <= cs.Step) {
log.Debug(cmn.Fmt("enterPrecommit(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
cs.Logger.Debug(cmn.Fmt("enterPrecommit(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
}
log.Info(cmn.Fmt("enterPrecommit(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
cs.Logger.Info(cmn.Fmt("enterPrecommit(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
defer func() {
// Done enterPrecommit:
@ -971,9 +971,9 @@ func (cs *ConsensusState) enterPrecommit(height int, round int) {
// If we don't have a polka, we must precommit nil
if !ok {
if cs.LockedBlock != nil {
log.Notice("enterPrecommit: No +2/3 prevotes during enterPrecommit while we're locked. Precommitting nil")
cs.Logger.Info("enterPrecommit: No +2/3 prevotes during enterPrecommit while we're locked. Precommitting nil")
} else {
log.Notice("enterPrecommit: No +2/3 prevotes during enterPrecommit. Precommitting nil.")
cs.Logger.Info("enterPrecommit: No +2/3 prevotes during enterPrecommit. Precommitting nil.")
}
cs.signAddVote(types.VoteTypePrecommit, nil, types.PartSetHeader{})
return
@ -991,9 +991,9 @@ func (cs *ConsensusState) enterPrecommit(height int, round int) {
// +2/3 prevoted nil. Unlock and precommit nil.
if len(blockID.Hash) == 0 {
if cs.LockedBlock == nil {
log.Notice("enterPrecommit: +2/3 prevoted for nil.")
cs.Logger.Info("enterPrecommit: +2/3 prevoted for nil.")
} else {
log.Notice("enterPrecommit: +2/3 prevoted for nil. Unlocking")
cs.Logger.Info("enterPrecommit: +2/3 prevoted for nil. Unlocking")
cs.LockedRound = 0
cs.LockedBlock = nil
cs.LockedBlockParts = nil
@ -1007,7 +1007,7 @@ func (cs *ConsensusState) enterPrecommit(height int, round int) {
// If we're already locked on that block, precommit it, and update the LockedRound
if cs.LockedBlock.HashesTo(blockID.Hash) {
log.Notice("enterPrecommit: +2/3 prevoted locked block. Relocking")
cs.Logger.Info("enterPrecommit: +2/3 prevoted locked block. Relocking")
cs.LockedRound = round
types.FireEventRelock(cs.evsw, cs.RoundStateEvent())
cs.signAddVote(types.VoteTypePrecommit, blockID.Hash, blockID.PartsHeader)
@ -1016,7 +1016,7 @@ func (cs *ConsensusState) enterPrecommit(height int, round int) {
// If +2/3 prevoted for proposal block, stage and precommit it
if cs.ProposalBlock.HashesTo(blockID.Hash) {
log.Notice("enterPrecommit: +2/3 prevoted proposal block. Locking", "hash", blockID.Hash)
cs.Logger.Info("enterPrecommit: +2/3 prevoted proposal block. Locking", "hash", blockID.Hash)
// Validate the block.
if err := cs.state.ValidateBlock(cs.ProposalBlock); err != nil {
cmn.PanicConsensus(cmn.Fmt("enterPrecommit: +2/3 prevoted for an invalid block: %v", err))
@ -1048,13 +1048,13 @@ func (cs *ConsensusState) enterPrecommit(height int, round int) {
// Enter: any +2/3 precommits for next round.
func (cs *ConsensusState) enterPrecommitWait(height int, round int) {
if cs.Height != height || round < cs.Round || (cs.Round == round && RoundStepPrecommitWait <= cs.Step) {
log.Debug(cmn.Fmt("enterPrecommitWait(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
cs.Logger.Debug(cmn.Fmt("enterPrecommitWait(%v/%v): Invalid args. Current step: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
return
}
if !cs.Votes.Precommits(round).HasTwoThirdsAny() {
cmn.PanicSanity(cmn.Fmt("enterPrecommitWait(%v/%v), but Precommits does not have any +2/3 votes", height, round))
}
log.Info(cmn.Fmt("enterPrecommitWait(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
cs.Logger.Info(cmn.Fmt("enterPrecommitWait(%v/%v). Current: %v/%v/%v", height, round, cs.Height, cs.Round, cs.Step))
defer func() {
// Done enterPrecommitWait:
@ -1070,10 +1070,10 @@ func (cs *ConsensusState) enterPrecommitWait(height int, round int) {
// Enter: +2/3 precommits for block
func (cs *ConsensusState) enterCommit(height int, commitRound int) {
if cs.Height != height || RoundStepCommit <= cs.Step {
log.Debug(cmn.Fmt("enterCommit(%v/%v): Invalid args. Current step: %v/%v/%v", height, commitRound, cs.Height, cs.Round, cs.Step))
cs.Logger.Debug(cmn.Fmt("enterCommit(%v/%v): Invalid args. Current step: %v/%v/%v", height, commitRound, cs.Height, cs.Round, cs.Step))
return
}
log.Info(cmn.Fmt("enterCommit(%v/%v). Current: %v/%v/%v", height, commitRound, cs.Height, cs.Round, cs.Step))
cs.Logger.Info(cmn.Fmt("enterCommit(%v/%v). Current: %v/%v/%v", height, commitRound, cs.Height, cs.Round, cs.Step))
defer func() {
// Done enterCommit:
@ -1121,13 +1121,13 @@ func (cs *ConsensusState) tryFinalizeCommit(height int) {
blockID, ok := cs.Votes.Precommits(cs.CommitRound).TwoThirdsMajority()
if !ok || len(blockID.Hash) == 0 {
log.Warn("Attempt to finalize failed. There was no +2/3 majority, or +2/3 was for <nil>.", "height", height)
cs.Logger.Error("Attempt to finalize failed. There was no +2/3 majority, or +2/3 was for <nil>.", "height", height)
return
}
if !cs.ProposalBlock.HashesTo(blockID.Hash) {
// TODO: this happens every time if we're not a validator (ugly logs)
// TODO: ^^ wait, why does it matter that we're a validator?
log.Warn("Attempt to finalize failed. We don't have the commit block.", "height", height, "proposal-block", cs.ProposalBlock.Hash(), "commit-block", blockID.Hash)
cs.Logger.Error("Attempt to finalize failed. We don't have the commit block.", "height", height, "proposal-block", cs.ProposalBlock.Hash(), "commit-block", blockID.Hash)
return
}
// go
@ -1137,7 +1137,7 @@ func (cs *ConsensusState) tryFinalizeCommit(height int) {
// Increment height and goto RoundStepNewHeight
func (cs *ConsensusState) finalizeCommit(height int) {
if cs.Height != height || cs.Step != RoundStepCommit {
log.Debug(cmn.Fmt("finalizeCommit(%v): Invalid args. Current step: %v/%v/%v", height, cs.Height, cs.Round, cs.Step))
cs.Logger.Debug(cmn.Fmt("finalizeCommit(%v): Invalid args. Current step: %v/%v/%v", height, cs.Height, cs.Round, cs.Step))
return
}
@ -1157,9 +1157,9 @@ func (cs *ConsensusState) finalizeCommit(height int) {
cmn.PanicConsensus(cmn.Fmt("+2/3 committed an invalid block: %v", err))
}
log.Notice(cmn.Fmt("Finalizing commit of block with %d txs", block.NumTxs),
cs.Logger.Info(cmn.Fmt("Finalizing commit of block with %d txs", block.NumTxs),
"height", block.Height, "hash", block.Hash(), "root", block.AppHash)
log.Info(cmn.Fmt("%v", block))
cs.Logger.Info(cmn.Fmt("%v", block))
fail.Fail() // XXX
@ -1172,7 +1172,7 @@ func (cs *ConsensusState) finalizeCommit(height int) {
cs.blockStore.SaveBlock(block, blockParts, seenCommit)
} else {
// Happens during replay if we already saved the block but didn't commit
log.Info("Calling finalizeCommit on already stored block", "height", block.Height)
cs.Logger.Info("Calling finalizeCommit on already stored block", "height", block.Height)
}
fail.Fail() // XXX
@ -1200,7 +1200,7 @@ func (cs *ConsensusState) finalizeCommit(height int) {
// NOTE: the block.AppHash wont reflect these txs until the next block
err := stateCopy.ApplyBlock(eventCache, cs.proxyAppConn, block, blockParts.Header(), cs.mempool)
if err != nil {
log.Error("Error on ApplyBlock. Did the application crash? Please restart tendermint", "error", err)
cs.Logger.Error("Error on ApplyBlock. Did the application crash? Please restart tendermint", "error", err)
return
}
@ -1293,7 +1293,7 @@ func (cs *ConsensusState) addProposalBlockPart(height int, part *types.Part, ver
var err error
cs.ProposalBlock = wire.ReadBinary(&types.Block{}, cs.ProposalBlockParts.GetReader(), types.MaxBlockSize, &n, &err).(*types.Block)
// NOTE: it's possible to receive complete proposal blocks for future rounds without having the proposal
log.Info("Received complete proposal block", "height", cs.ProposalBlock.Height, "hash", cs.ProposalBlock.Hash())
cs.Logger.Info("Received complete proposal block", "height", cs.ProposalBlock.Height, "hash", cs.ProposalBlock.Hash())
if cs.Step == RoundStepPropose && cs.isProposalComplete() {
// Move onto the next step
cs.enterPrevote(height, cs.Round)
@ -1317,10 +1317,10 @@ func (cs *ConsensusState) tryAddVote(vote *types.Vote, peerKey string) error {
return err
} else if _, ok := err.(*types.ErrVoteConflictingVotes); ok {
if peerKey == "" {
log.Warn("Found conflicting vote from ourselves. Did you unsafe_reset a validator?", "height", vote.Height, "round", vote.Round, "type", vote.Type)
cs.Logger.Error("Found conflicting vote from ourselves. Did you unsafe_reset a validator?", "height", vote.Height, "round", vote.Round, "type", vote.Type)
return err
}
log.Warn("Found conflicting vote. Publish evidence (TODO)")
cs.Logger.Error("Found conflicting vote. Publish evidence (TODO)")
/* TODO
evidenceTx := &types.DupeoutTx{
Address: address,
@ -1332,7 +1332,7 @@ func (cs *ConsensusState) tryAddVote(vote *types.Vote, peerKey string) error {
return err
} else {
// Probably an invalid signature. Bad peer.
log.Warn("Error attempting to add vote", "error", err)
cs.Logger.Error("Error attempting to add vote", "error", err)
return ErrAddingVote
}
}
@ -1342,7 +1342,7 @@ func (cs *ConsensusState) tryAddVote(vote *types.Vote, peerKey string) error {
//-----------------------------------------------------------------------------
func (cs *ConsensusState) addVote(vote *types.Vote, peerKey string) (added bool, err error) {
log.Debug("addVote", "voteHeight", vote.Height, "voteType", vote.Type, "csHeight", cs.Height)
cs.Logger.Debug("addVote", "voteHeight", vote.Height, "voteType", vote.Type, "csHeight", cs.Height)
// A precommit for the previous height?
// These come in while we wait timeoutCommit
@ -1354,7 +1354,7 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerKey string) (added bool,
}
added, err = cs.LastCommit.AddVote(vote)
if added {
log.Info(cmn.Fmt("Added to lastPrecommits: %v", cs.LastCommit.StringShort()))
cs.Logger.Info(cmn.Fmt("Added to lastPrecommits: %v", cs.LastCommit.StringShort()))
types.FireEventVote(cs.evsw, types.EventDataVote{vote})
// if we can skip timeoutCommit and have all the votes now,
@ -1378,7 +1378,7 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerKey string) (added bool,
switch vote.Type {
case types.VoteTypePrevote:
prevotes := cs.Votes.Prevotes(vote.Round)
log.Info("Added to prevote", "vote", vote, "prevotes", prevotes.StringShort())
cs.Logger.Info("Added to prevote", "vote", vote, "prevotes", prevotes.StringShort())
// First, unlock if prevotes is a valid POL.
// >> lockRound < POLRound <= unlockOrChangeLockRound (see spec)
// NOTE: If (lockRound < POLRound) but !(POLRound <= unlockOrChangeLockRound),
@ -1387,7 +1387,7 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerKey string) (added bool,
if (cs.LockedBlock != nil) && (cs.LockedRound < vote.Round) && (vote.Round <= cs.Round) {
blockID, ok := prevotes.TwoThirdsMajority()
if ok && !cs.LockedBlock.HashesTo(blockID.Hash) {
log.Notice("Unlocking because of POL.", "lockedRound", cs.LockedRound, "POLRound", vote.Round)
cs.Logger.Info("Unlocking because of POL.", "lockedRound", cs.LockedRound, "POLRound", vote.Round)
cs.LockedRound = 0
cs.LockedBlock = nil
cs.LockedBlockParts = nil
@ -1411,7 +1411,7 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerKey string) (added bool,
}
case types.VoteTypePrecommit:
precommits := cs.Votes.Precommits(vote.Round)
log.Info("Added to precommit", "vote", vote, "precommits", precommits.StringShort())
cs.Logger.Info("Added to precommit", "vote", vote, "precommits", precommits.StringShort())
blockID, ok := precommits.TwoThirdsMajority()
if ok {
if len(blockID.Hash) == 0 {
@ -1445,7 +1445,7 @@ func (cs *ConsensusState) addVote(vote *types.Vote, peerKey string) (added bool,
}
// Height mismatch, bad peer?
log.Info("Vote ignored and not added", "voteHeight", vote.Height, "csHeight", cs.Height, "err", err)
cs.Logger.Info("Vote ignored and not added", "voteHeight", vote.Height, "csHeight", cs.Height, "err", err)
return
}
@ -1473,11 +1473,11 @@ func (cs *ConsensusState) signAddVote(type_ byte, hash []byte, header types.Part
vote, err := cs.signVote(type_, hash, header)
if err == nil {
cs.sendInternalMessage(msgInfo{&VoteMessage{vote}, ""})
log.Info("Signed and pushed vote", "height", cs.Height, "round", cs.Round, "vote", vote, "error", err)
cs.Logger.Info("Signed and pushed vote", "height", cs.Height, "round", cs.Round, "vote", vote, "error", err)
return vote
} else {
//if !cs.replayMode {
log.Warn("Error signing vote", "height", cs.Height, "round", cs.Round, "vote", vote, "error", err)
cs.Logger.Error("Error signing vote", "height", cs.Height, "round", cs.Round, "vote", vote, "error", err)
//}
return nil
}


+ 14
- 14
consensus/state_test.go View File

@ -375,7 +375,7 @@ func TestLockNoPOL(t *testing.T) {
///
<-newRoundCh
log.Notice("#### ONTO ROUND 1")
t.Log("#### ONTO ROUND 1")
/*
Round2 (cs1, B) // B B2
*/
@ -420,7 +420,7 @@ func TestLockNoPOL(t *testing.T) {
<-timeoutWaitCh
<-newRoundCh
log.Notice("#### ONTO ROUND 2")
t.Log("#### ONTO ROUND 2")
/*
Round3 (vs2, _) // B, B2
*/
@ -461,7 +461,7 @@ func TestLockNoPOL(t *testing.T) {
incrementRound(vs2)
<-newRoundCh
log.Notice("#### ONTO ROUND 3")
t.Log("#### ONTO ROUND 3")
/*
Round4 (vs2, C) // B C // B C
*/
@ -502,7 +502,7 @@ func TestLockPOLRelock(t *testing.T) {
newRoundCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewRound(), 1)
newBlockCh := subscribeToEvent(cs1.evsw, "tester", types.EventStringNewBlockHeader(), 1)
log.Debug("vs2 last round", "lr", vs2.PrivValidator.LastRound)
t.Logf("vs2 last round %v", vs2.PrivValidator.LastRound)
// everything done from perspective of cs1
@ -548,7 +548,7 @@ func TestLockPOLRelock(t *testing.T) {
cs1.SetProposalAndBlock(prop, propBlock, propBlockParts, "some peer")
<-newRoundCh
log.Notice("### ONTO ROUND 1")
t.Log("### ONTO ROUND 1")
/*
Round2 (vs2, C) // B C C C // C C C _)
@ -659,7 +659,7 @@ func TestLockPOLUnlock(t *testing.T) {
cs1.SetProposalAndBlock(prop, propBlock, propBlockParts, "some peer")
<-newRoundCh
log.Notice("#### ONTO ROUND 1")
t.Log("#### ONTO ROUND 1")
/*
Round2 (vs2, C) // B nil nil nil // nil nil nil _
@ -731,7 +731,7 @@ func TestLockPOLSafety1(t *testing.T) {
panic("failed to update validator")
}*/
log.Warn("old prop", "hash", fmt.Sprintf("%X", propBlock.Hash()))
t.Logf("old prop hash %v", fmt.Sprintf("%X", propBlock.Hash()))
// we do see them precommit nil
signAddVotes(cs1, types.VoteTypePrecommit, nil, types.PartSetHeader{}, vs2, vs3, vs4)
@ -746,7 +746,7 @@ func TestLockPOLSafety1(t *testing.T) {
cs1.SetProposalAndBlock(prop, propBlock, propBlockParts, "some peer")
<-newRoundCh
log.Notice("### ONTO ROUND 1")
t.Log("### ONTO ROUND 1")
/*Round2
// we timeout and prevote our lock
// a polka happened but we didn't see it!
@ -765,7 +765,7 @@ func TestLockPOLSafety1(t *testing.T) {
if rs.LockedBlock != nil {
panic("we should not be locked!")
}
log.Warn("new prop", "hash", fmt.Sprintf("%X", propBlockHash))
t.Logf("new prop hash %v", fmt.Sprintf("%X", propBlockHash))
// go to prevote, prevote for proposal block
<-voteCh
validatePrevote(t, cs1, 1, vss[0], propBlockHash)
@ -786,7 +786,7 @@ func TestLockPOLSafety1(t *testing.T) {
<-newRoundCh
log.Notice("### ONTO ROUND 2")
t.Log("### ONTO ROUND 2")
/*Round3
we see the polka from round 1 but we shouldn't unlock!
*/
@ -805,7 +805,7 @@ func TestLockPOLSafety1(t *testing.T) {
// add prevotes from the earlier round
addVotes(cs1, prevotes...)
log.Warn("Done adding prevotes!")
t.Log("Done adding prevotes!")
ensureNoNewStep(newStepCh)
}
@ -849,7 +849,7 @@ func TestLockPOLSafety2(t *testing.T) {
cs1.updateRoundStep(0, RoundStepPrecommitWait)
log.Notice("### ONTO Round 1")
t.Log("### ONTO Round 1")
// jump in at round 1
height := cs1.Height
startTestRound(cs1, height, 1)
@ -886,7 +886,7 @@ func TestLockPOLSafety2(t *testing.T) {
addVotes(cs1, prevotes...)
<-newRoundCh
log.Notice("### ONTO Round 2")
t.Log("### ONTO Round 2")
/*Round2
// now we see the polka from round 1, but we shouldnt unlock
*/
@ -1033,7 +1033,7 @@ func TestHalt1(t *testing.T) {
re = <-newRoundCh
rs = re.(types.TMEventData).Unwrap().(types.EventDataRoundState).RoundState.(*RoundState)
log.Notice("### ONTO ROUND 1")
t.Log("### ONTO ROUND 1")
/*Round2
// we timeout and prevote our lock
// a polka happened but we didn't see it!


+ 6
- 6
consensus/ticker.go View File

@ -39,8 +39,8 @@ func NewTimeoutTicker() TimeoutTicker {
tickChan: make(chan timeoutInfo, tickTockBufferSize),
tockChan: make(chan timeoutInfo, tickTockBufferSize),
}
tt.BaseService = *NewBaseService(nil, "TimeoutTicker", tt)
tt.stopTimer() // don't want to fire until the first scheduled timeout
tt.BaseService = *NewBaseService(log, "TimeoutTicker", tt)
return tt
}
@ -75,7 +75,7 @@ func (t *timeoutTicker) stopTimer() {
select {
case <-t.timer.C:
default:
log.Debug("Timer already stopped")
t.Logger.Debug("Timer already stopped")
}
}
}
@ -84,12 +84,12 @@ func (t *timeoutTicker) stopTimer() {
// timers are interupted and replaced by new ticks from later steps
// timeouts of 0 on the tickChan will be immediately relayed to the tockChan
func (t *timeoutTicker) timeoutRoutine() {
log.Debug("Starting timeout routine")
t.Logger.Debug("Starting timeout routine")
var ti timeoutInfo
for {
select {
case newti := <-t.tickChan:
log.Debug("Received tick", "old_ti", ti, "new_ti", newti)
t.Logger.Debug("Received tick", "old_ti", ti, "new_ti", newti)
// ignore tickers for old height/round/step
if newti.Height < ti.Height {
@ -111,9 +111,9 @@ func (t *timeoutTicker) timeoutRoutine() {
// NOTE time.Timer allows duration to be non-positive
ti = newti
t.timer.Reset(ti.Duration)
log.Debug("Scheduled timeout", "dur", ti.Duration, "height", ti.Height, "round", ti.Round, "step", ti.Step)
t.Logger.Debug("Scheduled timeout", "dur", ti.Duration, "height", ti.Height, "round", ti.Round, "step", ti.Step)
case <-t.timer.C:
log.Info("Timed out", "dur", ti.Duration, "height", ti.Height, "round", ti.Round, "step", ti.Step)
t.Logger.Info("Timed out", "dur", ti.Duration, "height", ti.Height, "round", ti.Round, "step", ti.Step)
// go routine here gaurantees timeoutRoutine doesn't block.
// Determinism comes from playback in the receiveRoutine.
// We can eliminate it by merging the timeoutRoutine into receiveRoutine


+ 3
- 3
consensus/wal.go View File

@ -3,10 +3,10 @@ package consensus
import (
"time"
wire "github.com/tendermint/go-wire"
"github.com/tendermint/tendermint/types"
auto "github.com/tendermint/tmlibs/autofile"
. "github.com/tendermint/tmlibs/common"
"github.com/tendermint/go-wire"
"github.com/tendermint/tendermint/types"
)
//--------------------------------------------------------
@ -49,7 +49,7 @@ func NewWAL(walFile string, light bool) (*WAL, error) {
group: group,
light: light,
}
wal.BaseService = *NewBaseService(log, "WAL", wal)
wal.BaseService = *NewBaseService(nil, "WAL", wal)
_, err = wal.Start()
return wal, err
}


+ 19
- 8
glide.lock View File

@ -1,5 +1,5 @@
hash: c715f30f5021a73c8d35229ad5bb7473bd5b18a24707b3211f264658c44ef63f
updated: 2017-05-05T00:50:11.28834974-04:00
hash: 83b5ca94602b8296526e421b9bc0a99584cff3909684fbda49c31ffcde1eea98
updated: 2017-05-05T18:00:37.128906219Z
imports:
- name: github.com/btcsuite/btcd
version: 4b348c1d33373d672edd83fc576892d0e46686d2
@ -13,6 +13,14 @@ imports:
version: 95f809107225be108efcf10a3509e4ea6ceef3c4
- name: github.com/fsnotify/fsnotify
version: 4da3e2cfbabc9f751898f250b49f2439785783a1
- name: github.com/go-kit/kit
version: 0873e56b0faeae3a1d661b10d629135508ea5504
subpackages:
- log
- log/level
- log/term
- name: github.com/go-logfmt/logfmt
version: 390ab7935ee28ec6b286364bba9b4dd6410cb3d5
- name: github.com/go-stack/stack
version: 7a2f19628aabfe68f0766b59e74d6315f8347d22
- name: github.com/gogo/protobuf
@ -43,6 +51,8 @@ imports:
version: 76626ae9c91c4f2a10f34cad8ce83ea42c93bb75
- name: github.com/jmhodges/levigo
version: c42d9e0ca023e2198120196f842701bb4c55d7b9
- name: github.com/kr/logfmt
version: b84e30acd515aadc4b783ad4ff83aff3299bdfe0
- name: github.com/magiconair/properties
version: 51463bfca2576e06c62a8504b5c0f06d61312647
- name: github.com/mattn/go-colorable
@ -68,11 +78,11 @@ imports:
- name: github.com/spf13/cast
version: acbeb36b902d72a7a4c18e8f3241075e7ab763e4
- name: github.com/spf13/cobra
version: 8f0203be891287870100e4af46262cdf4a4261d1
version: db6b9a8b3f3f400c8ecb4a4d7d02245b8facad66
- name: github.com/spf13/jwalterweatherman
version: fa7ca7e836cf3a8bb4ebf799f472c12d7e903d66
- name: github.com/spf13/pflag
version: c990990ab4981d84da820b7b00c85139ca150b5f
version: 80fe0fb4eba54167e2ccae1c6c950e72abf61b73
- name: github.com/spf13/viper
version: 0967fc9aceab2ce9da34061253ac10fb99bba5b2
- name: github.com/stretchr/testify
@ -96,7 +106,7 @@ imports:
- leveldb/table
- leveldb/util
- name: github.com/tendermint/abci
version: 8d8e35ae537538c9cf6808be3ca9dd7dab81b7f6
version: 50a9967c79d89742151c7f1a981678df8c21f9dc
subpackages:
- client
- example/counter
@ -109,7 +119,7 @@ imports:
- edwards25519
- extra25519
- name: github.com/tendermint/go-crypto
version: 524ba917a3a1636f21ab2c0bf76b6526903ab879
version: e71bbb2509b586f0b24f120b6ba57f32aefa1579
- name: github.com/tendermint/go-wire
version: b53add0b622662731985485f3a19be7f684660b8
subpackages:
@ -120,14 +130,14 @@ imports:
subpackages:
- term
- name: github.com/tendermint/merkleeyes
version: d0aa363fd4e015e509038c3a0ec493bc62ee0b8a
version: c722818b460381bc5b82e38c73ff6e22a9df624d
subpackages:
- app
- client
- iavl
- testutil
- name: github.com/tendermint/tmlibs
version: 2f02ed18e9b706467c9474d024a25a0b7a9c0e97
version: 240215f2aa26d53223c618a5a2127a08d37868f0
subpackages:
- autofile
- cli
@ -136,6 +146,7 @@ imports:
- db
- events
- flowrate
- log
- logger
- merkle
- test


+ 3
- 3
glide.yaml View File

@ -12,7 +12,7 @@ import:
- package: github.com/spf13/cobra
- package: github.com/spf13/viper
- package: github.com/tendermint/abci
version: develop
version: feature/new-logging
subpackages:
- client
- example/dummy
@ -25,7 +25,7 @@ import:
- data
- package: github.com/tendermint/log15
- package: github.com/tendermint/tmlibs
version: cli
version: log
subpackages:
- autofile
- cli
@ -34,7 +34,7 @@ import:
- db
- events
- flowrate
- logger
- log
- merkle
- package: golang.org/x/crypto
subpackages:


+ 0
- 18
mempool/log.go View File

@ -1,18 +0,0 @@
package mempool
import (
"github.com/tendermint/tmlibs/logger"
)
var log = logger.New("module", "mempool")
/*
func init() {
log.SetHandler(
logger.LvlFilterHandler(
logger.LvlDebug,
logger.BypassHandler(),
),
)
}
*/

+ 17
- 9
mempool/mempool.go View File

@ -7,6 +7,8 @@ import (
"sync/atomic"
"time"
"github.com/pkg/errors"
abci "github.com/tendermint/abci/types"
auto "github.com/tendermint/tmlibs/autofile"
"github.com/tendermint/tmlibs/clist"
@ -15,6 +17,7 @@ import (
cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/proxy"
"github.com/tendermint/tendermint/types"
"github.com/tendermint/tmlibs/log"
)
/*
@ -65,6 +68,8 @@ type Mempool struct {
// A log of mempool txs
wal *auto.AutoFile
logger log.Logger
}
func NewMempool(config *cfg.MempoolConfig, proxyAppConn proxy.AppConnMempool) *Mempool {
@ -77,26 +82,29 @@ func NewMempool(config *cfg.MempoolConfig, proxyAppConn proxy.AppConnMempool) *M
rechecking: 0,
recheckCursor: nil,
recheckEnd: nil,
cache: newTxCache(cacheSize),
logger: log.NewNopLogger(),
cache: newTxCache(cacheSize),
}
mempool.initWAL()
proxyAppConn.SetResponseCallback(mempool.resCb)
return mempool
}
// SetLogger allows you to set your own Logger.
func (mem *Mempool) SetLogger(l log.Logger) {
mem.logger = l
}
func (mem *Mempool) initWAL() {
walDir := mem.config.WalDir()
if walDir != "" {
err := cmn.EnsureDir(walDir, 0700)
if err != nil {
log.Error("Error ensuring Mempool wal dir", "error", err)
cmn.PanicSanity(err)
cmn.PanicSanity(errors.Wrap(err, "Error ensuring Mempool wal dir"))
}
af, err := auto.OpenAutoFile(walDir + "/wal")
if err != nil {
log.Error("Error opening Mempool wal file", "error", err)
cmn.PanicSanity(err)
cmn.PanicSanity(errors.Wrap(err, "Error opening Mempool wal file"))
}
mem.wal = af
}
@ -203,7 +211,7 @@ func (mem *Mempool) resCbNormal(req *abci.Request, res *abci.Response) {
mem.txs.PushBack(memTx)
} else {
// ignore bad transaction
log.Info("Bad Transaction", "res", r)
mem.logger.Info("Bad Transaction", "res", r)
// remove from cache (it might be good later)
mem.cache.Remove(req.GetCheckTx().Tx)
@ -241,7 +249,7 @@ func (mem *Mempool) resCbRecheck(req *abci.Request, res *abci.Response) {
if mem.recheckCursor == nil {
// Done!
atomic.StoreInt32(&mem.rechecking, 0)
log.Info("Done rechecking txs")
mem.logger.Info("Done rechecking txs")
}
default:
// ignore other messages
@ -300,7 +308,7 @@ func (mem *Mempool) Update(height int, txs types.Txs) {
// NOTE/XXX: in some apps a tx could be invalidated due to EndBlock,
// so we really still do need to recheck, but this is for debugging
if mem.config.Recheck && (mem.config.RecheckEmpty || len(txs) > 0) {
log.Info("Recheck txs", "numtxs", len(goodTxs))
mem.logger.Info("Recheck txs", "numtxs", len(goodTxs))
mem.recheckTxs(goodTxs)
// At this point, mem.txs are being rechecked.
// mem.recheckCursor re-scans mem.txs and possibly removes some txs.


+ 2
- 0
mempool/mempool_test.go View File

@ -8,6 +8,7 @@ import (
cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/proxy"
"github.com/tendermint/tendermint/types"
"github.com/tendermint/tmlibs/log"
)
func TestSerialReap(t *testing.T) {
@ -19,6 +20,7 @@ func TestSerialReap(t *testing.T) {
appConnMem, _ := cc.NewABCIClient()
appConnCon, _ := cc.NewABCIClient()
mempool := NewMempool(config.Mempool, appConnMem)
mempool.SetLogger(log.TestingLogger())
deliverTxsRange := func(start, end int) {
// Deliver some txs.


+ 7
- 7
mempool/reactor.go View File

@ -7,7 +7,7 @@ import (
"time"
abci "github.com/tendermint/abci/types"
"github.com/tendermint/go-wire"
wire "github.com/tendermint/go-wire"
"github.com/tendermint/tmlibs/clist"
cfg "github.com/tendermint/tendermint/config"
@ -35,7 +35,7 @@ func NewMempoolReactor(config *cfg.MempoolConfig, mempool *Mempool) *MempoolReac
config: config,
Mempool: mempool,
}
memR.BaseReactor = *p2p.NewBaseReactor(log, "MempoolReactor", memR)
memR.BaseReactor = *p2p.NewBaseReactor("MempoolReactor", memR)
return memR
}
@ -63,24 +63,24 @@ func (memR *MempoolReactor) RemovePeer(peer *p2p.Peer, reason interface{}) {
func (memR *MempoolReactor) Receive(chID byte, src *p2p.Peer, msgBytes []byte) {
_, msg, err := DecodeMessage(msgBytes)
if err != nil {
log.Warn("Error decoding message", "error", err)
memR.Logger.Error("Error decoding message", "error", err)
return
}
log.Debug("Receive", "src", src, "chId", chID, "msg", msg)
memR.Logger.Debug("Receive", "src", src, "chId", chID, "msg", msg)
switch msg := msg.(type) {
case *TxMessage:
err := memR.Mempool.CheckTx(msg.Tx, nil)
if err != nil {
// Bad, seen, or conflicting tx.
log.Info("Could not add tx", "tx", msg.Tx)
memR.Logger.Info("Could not add tx", "tx", msg.Tx)
return
} else {
log.Info("Added valid tx", "tx", msg.Tx)
memR.Logger.Info("Added valid tx", "tx", msg.Tx)
}
// broadcasting happens from go routines per peer
default:
log.Warn(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
memR.Logger.Error(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
}
}


+ 0
- 7
node/log.go View File

@ -1,7 +0,0 @@
package node
import (
"github.com/tendermint/tmlibs/logger"
)
var log = logger.New("module", "node")

+ 19
- 14
node/node.go View File

@ -28,6 +28,7 @@ import (
"github.com/tendermint/tendermint/version"
cmn "github.com/tendermint/tmlibs/common"
dbm "github.com/tendermint/tmlibs/db"
"github.com/tendermint/tmlibs/log"
_ "net/http/pprof"
)
@ -57,15 +58,14 @@ type Node struct {
txIndexer txindex.TxIndexer
}
func NewNodeDefault(config *cfg.Config) *Node {
func NewNodeDefault(config *cfg.Config, logger log.Logger) *Node {
// Get PrivValidator
privValidator := types.LoadOrGenPrivValidator(config.PrivValidatorFile())
privValidator := types.LoadOrGenPrivValidator(config.PrivValidatorFile(), logger)
return NewNode(config, privValidator,
proxy.DefaultClientCreator(config.ProxyApp, config.ABCI, config.DBDir()))
proxy.DefaultClientCreator(config.ProxyApp, config.ABCI, config.DBDir()), logger)
}
func NewNode(config *cfg.Config, privValidator *types.PrivValidator, clientCreator proxy.ClientCreator) *Node {
func NewNode(config *cfg.Config, privValidator *types.PrivValidator, clientCreator proxy.ClientCreator, logger log.Logger) *Node {
// Get BlockStore
blockStoreDB := dbm.NewDB("blockstore", config.DBBackend, config.DBDir())
blockStore := bc.NewBlockStore(blockStoreDB)
@ -73,6 +73,7 @@ func NewNode(config *cfg.Config, privValidator *types.PrivValidator, clientCreat
// Get State
stateDB := dbm.NewDB("state", config.DBBackend, config.DBDir())
state := sm.GetState(stateDB, config.GenesisFile())
state.SetLogger(logger.With("module", "state"))
// Create the proxyApp, which manages connections (consensus, mempool, query)
// and sync tendermint and the app by replaying any necessary blocks
@ -83,6 +84,7 @@ func NewNode(config *cfg.Config, privValidator *types.PrivValidator, clientCreat
// reload the state (it may have been updated by the handshake)
state = sm.LoadState(stateDB)
state.SetLogger(logger.With("module", "state"))
// Transaction indexing
var txIndexer txindex.TxIndexer
@ -120,6 +122,7 @@ func NewNode(config *cfg.Config, privValidator *types.PrivValidator, clientCreat
// Make MempoolReactor
mempool := mempl.NewMempool(config.Mempool, proxyApp.Mempool())
mempool.SetLogger(logger.With("module", "mempool"))
mempoolReactor := mempl.NewMempoolReactor(config.Mempool, mempool)
// Make ConsensusReactor
@ -178,7 +181,7 @@ func NewNode(config *cfg.Config, privValidator *types.PrivValidator, clientCreat
if profileHost != "" {
go func() {
log.Warn("Profile server", "error", http.ListenAndServe(profileHost, nil))
logger.Error("Profile server", "error", http.ListenAndServe(profileHost, nil))
}()
}
@ -200,15 +203,14 @@ func NewNode(config *cfg.Config, privValidator *types.PrivValidator, clientCreat
proxyApp: proxyApp,
txIndexer: txIndexer,
}
node.BaseService = *cmn.NewBaseService(log, "Node", node)
node.BaseService = *cmn.NewBaseService(logger, "Node", node)
return node
}
func (n *Node) OnStart() error {
// Create & add listener
protocol, address := ProtocolAndAddress(n.config.P2P.ListenAddress)
l := p2p.NewDefaultListener(protocol, address, n.config.P2P.SkipUPNP)
l := p2p.NewDefaultListener(protocol, address, n.config.P2P.SkipUPNP, n.Logger.With("module", "p2p"))
n.sw.AddListener(l)
// Start the switch
@ -243,14 +245,14 @@ func (n *Node) OnStart() error {
func (n *Node) OnStop() {
n.BaseService.OnStop()
log.Notice("Stopping Node")
n.Logger.Info("Stopping Node")
// TODO: gracefully disconnect from peers.
n.sw.Stop()
for _, l := range n.rpcListeners {
log.Info("Closing rpc listener", "listener", l)
n.Logger.Info("Closing rpc listener", "listener", l)
if err := l.Close(); err != nil {
log.Error("Error closing listener", "listener", l, "error", err)
n.Logger.Error("Error closing listener", "listener", l, "error", err)
}
}
}
@ -289,6 +291,7 @@ func (n *Node) ConfigureRPC() {
rpccore.SetAddrBook(n.addrBook)
rpccore.SetProxyAppQuery(n.proxyApp.Query())
rpccore.SetTxIndexer(n.txIndexer)
rpccore.SetLogger(n.Logger.With("module", "rpc"))
}
func (n *Node) startRPC() ([]net.Listener, error) {
@ -299,10 +302,12 @@ func (n *Node) startRPC() ([]net.Listener, error) {
listeners := make([]net.Listener, len(listenAddrs))
for i, listenAddr := range listenAddrs {
mux := http.NewServeMux()
rpcLogger := n.Logger.With("module", "rpcserver")
wm := rpcserver.NewWebsocketManager(rpccore.Routes, n.evsw)
wm.SetLogger(rpcLogger)
mux.HandleFunc("/websocket", wm.WebsocketHandler)
rpcserver.RegisterRPCFuncs(mux, rpccore.Routes)
listener, err := rpcserver.StartHTTPServer(listenAddr, mux)
rpcserver.RegisterRPCFuncs(mux, rpccore.Routes, rpcLogger)
listener, err := rpcserver.StartHTTPServer(listenAddr, mux, rpcLogger)
if err != nil {
return nil, err
}


+ 3
- 2
node/node_test.go View File

@ -5,15 +5,16 @@ import (
"time"
cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tmlibs/log"
)
func TestNodeStartStop(t *testing.T) {
config := cfg.ResetTestRoot("node_node_test")
// Create & start node
n := NewNodeDefault(config)
n := NewNodeDefault(config, log.TestingLogger())
n.Start()
log.Notice("Started node", "nodeInfo", n.sw.NodeInfo())
t.Logf("Started node %v", n.sw.NodeInfo())
// Wait a bit to initialize
// TODO remove time.Sleep(), make asynchronous.


+ 25
- 23
p2p/addrbook.go View File

@ -107,7 +107,7 @@ func NewAddrBook(filePath string, routabilityStrict bool) *AddrBook {
routabilityStrict: routabilityStrict,
}
am.init()
am.BaseService = *cmn.NewBaseService(log, "AddrBook", am)
am.BaseService = *cmn.NewBaseService(nil, "AddrBook", am)
return am
}
@ -147,7 +147,7 @@ func (a *AddrBook) Wait() {
func (a *AddrBook) AddOurAddress(addr *NetAddress) {
a.mtx.Lock()
defer a.mtx.Unlock()
log.Info("Add our address to book", "addr", addr)
a.Logger.Info("Add our address to book", "addr", addr)
a.ourAddrs[addr.String()] = addr
}
@ -163,7 +163,7 @@ func (a *AddrBook) OurAddresses() []*NetAddress {
func (a *AddrBook) AddAddress(addr *NetAddress, src *NetAddress) {
a.mtx.Lock()
defer a.mtx.Unlock()
log.Info("Add address to book", "addr", addr, "src", src)
a.Logger.Info("Add address to book", "addr", addr, "src", src)
a.addAddress(addr, src)
}
@ -271,7 +271,7 @@ func (a *AddrBook) RemoveAddress(addr *NetAddress) {
if ka == nil {
return
}
log.Info("Remove address from book", "addr", addr)
a.Logger.Info("Remove address from book", "addr", addr)
a.removeFromAllBuckets(ka)
}
@ -318,7 +318,7 @@ type addrBookJSON struct {
}
func (a *AddrBook) saveToFile(filePath string) {
log.Info("Saving AddrBook to file", "size", a.Size())
a.Logger.Info("Saving AddrBook to file", "size", a.Size())
a.mtx.Lock()
defer a.mtx.Unlock()
@ -335,12 +335,12 @@ func (a *AddrBook) saveToFile(filePath string) {
jsonBytes, err := json.MarshalIndent(aJSON, "", "\t")
if err != nil {
log.Error("Failed to save AddrBook to file", "err", err)
a.Logger.Error("Failed to save AddrBook to file", "err", err)
return
}
err = cmn.WriteFileAtomic(filePath, jsonBytes, 0644)
if err != nil {
log.Error("Failed to save AddrBook to file", "file", filePath, "error", err)
a.Logger.Error("Failed to save AddrBook to file", "file", filePath, "error", err)
}
}
@ -387,7 +387,7 @@ func (a *AddrBook) loadFromFile(filePath string) bool {
// Save saves the book.
func (a *AddrBook) Save() {
log.Info("Saving AddrBook to file", "size", a.Size())
a.Logger.Info("Saving AddrBook to file", "size", a.Size())
a.saveToFile(a.filePath)
}
@ -407,7 +407,7 @@ out:
dumpAddressTicker.Stop()
a.saveToFile(a.filePath)
a.wg.Done()
log.Notice("Address handler done")
a.Logger.Info("Address handler done")
}
func (a *AddrBook) getBucket(bucketType byte, bucketIdx int) map[string]*knownAddress {
@ -427,7 +427,7 @@ func (a *AddrBook) getBucket(bucketType byte, bucketIdx int) map[string]*knownAd
func (a *AddrBook) addToNewBucket(ka *knownAddress, bucketIdx int) bool {
// Sanity check
if ka.isOld() {
log.Warn(cmn.Fmt("Cannot add address already in old bucket to a new bucket: %v", ka))
a.Logger.Error(cmn.Fmt("Cannot add address already in old bucket to a new bucket: %v", ka))
return false
}
@ -441,7 +441,7 @@ func (a *AddrBook) addToNewBucket(ka *knownAddress, bucketIdx int) bool {
// Enforce max addresses.
if len(bucket) > newBucketSize {
log.Notice("new bucket is full, expiring old ")
a.Logger.Info("new bucket is full, expiring old ")
a.expireNew(bucketIdx)
}
@ -461,11 +461,11 @@ func (a *AddrBook) addToNewBucket(ka *knownAddress, bucketIdx int) bool {
func (a *AddrBook) addToOldBucket(ka *knownAddress, bucketIdx int) bool {
// Sanity check
if ka.isNew() {
log.Warn(cmn.Fmt("Cannot add new address to old bucket: %v", ka))
a.Logger.Error(cmn.Fmt("Cannot add new address to old bucket: %v", ka))
return false
}
if len(ka.Buckets) != 0 {
log.Warn(cmn.Fmt("Cannot add already old address to another old bucket: %v", ka))
a.Logger.Error(cmn.Fmt("Cannot add already old address to another old bucket: %v", ka))
return false
}
@ -496,7 +496,7 @@ func (a *AddrBook) addToOldBucket(ka *knownAddress, bucketIdx int) bool {
func (a *AddrBook) removeFromBucket(ka *knownAddress, bucketType byte, bucketIdx int) {
if ka.BucketType != bucketType {
log.Warn(cmn.Fmt("Bucket type mismatch: %v", ka))
a.Logger.Error(cmn.Fmt("Bucket type mismatch: %v", ka))
return
}
bucket := a.getBucket(bucketType, bucketIdx)
@ -538,7 +538,7 @@ func (a *AddrBook) pickOldest(bucketType byte, bucketIdx int) *knownAddress {
func (a *AddrBook) addAddress(addr, src *NetAddress) {
if a.routabilityStrict && !addr.Routable() {
log.Warn(cmn.Fmt("Cannot add non-routable address %v", addr))
a.Logger.Error(cmn.Fmt("Cannot add non-routable address %v", addr))
return
}
if _, ok := a.ourAddrs[addr.String()]; ok {
@ -569,7 +569,7 @@ func (a *AddrBook) addAddress(addr, src *NetAddress) {
bucket := a.calcNewBucket(addr, src)
a.addToNewBucket(ka, bucket)
log.Notice("Added new address", "address", addr, "total", a.size())
a.Logger.Info("Added new address", "address", addr, "total", a.size())
}
// Make space in the new buckets by expiring the really bad entries.
@ -578,7 +578,7 @@ func (a *AddrBook) expireNew(bucketIdx int) {
for addrStr, ka := range a.addrNew[bucketIdx] {
// If an entry is bad, throw it away
if ka.isBad() {
log.Notice(cmn.Fmt("expiring bad address %v", addrStr))
a.Logger.Info(cmn.Fmt("expiring bad address %v", addrStr))
a.removeFromBucket(ka, bucketTypeNew, bucketIdx)
return
}
@ -595,11 +595,11 @@ func (a *AddrBook) expireNew(bucketIdx int) {
func (a *AddrBook) moveToOld(ka *knownAddress) {
// Sanity check
if ka.isOld() {
log.Warn(cmn.Fmt("Cannot promote address that is already old %v", ka))
a.Logger.Error(cmn.Fmt("Cannot promote address that is already old %v", ka))
return
}
if len(ka.Buckets) == 0 {
log.Warn(cmn.Fmt("Cannot promote address that isn't in any new buckets %v", ka))
a.Logger.Error(cmn.Fmt("Cannot promote address that isn't in any new buckets %v", ka))
return
}
@ -624,13 +624,13 @@ func (a *AddrBook) moveToOld(ka *knownAddress) {
if !added {
added := a.addToNewBucket(oldest, freedBucket)
if !added {
log.Warn(cmn.Fmt("Could not migrate oldest %v to freedBucket %v", oldest, freedBucket))
a.Logger.Error(cmn.Fmt("Could not migrate oldest %v to freedBucket %v", oldest, freedBucket))
}
}
// Finally, add to bucket again.
added = a.addToOldBucket(ka, oldBucketIdx)
if !added {
log.Warn(cmn.Fmt("Could not re-add ka %v to oldBucketIdx %v", ka, oldBucketIdx))
a.Logger.Error(cmn.Fmt("Could not re-add ka %v to oldBucketIdx %v", ka, oldBucketIdx))
}
}
}
@ -778,7 +778,8 @@ func (ka *knownAddress) markGood() {
func (ka *knownAddress) addBucketRef(bucketIdx int) int {
for _, bucket := range ka.Buckets {
if bucket == bucketIdx {
log.Warn(cmn.Fmt("Bucket already exists in ka.Buckets: %v", ka))
// TODO refactor to return error?
// log.Warn(Fmt("Bucket already exists in ka.Buckets: %v", ka))
return -1
}
}
@ -794,7 +795,8 @@ func (ka *knownAddress) removeBucketRef(bucketIdx int) int {
}
}
if len(buckets) != len(ka.Buckets)-1 {
log.Warn(cmn.Fmt("bucketIdx not found in ka.Buckets: %v", ka))
// TODO refactor to return error?
// log.Warn(Fmt("bucketIdx not found in ka.Buckets: %v", ka))
return -1
}
ka.Buckets = buckets


+ 8
- 0
p2p/addrbook_test.go View File

@ -7,6 +7,7 @@ import (
"testing"
"github.com/stretchr/testify/assert"
"github.com/tendermint/tmlibs/log"
)
func createTempFileName(prefix string) string {
@ -27,9 +28,11 @@ func TestAddrBookSaveLoad(t *testing.T) {
// 0 addresses
book := NewAddrBook(fname, true)
book.SetLogger(log.TestingLogger())
book.saveToFile(fname)
book = NewAddrBook(fname, true)
book.SetLogger(log.TestingLogger())
book.loadFromFile(fname)
assert.Zero(t, book.Size())
@ -45,6 +48,7 @@ func TestAddrBookSaveLoad(t *testing.T) {
book.saveToFile(fname)
book = NewAddrBook(fname, true)
book.SetLogger(log.TestingLogger())
book.loadFromFile(fname)
assert.Equal(t, 100, book.Size())
@ -56,6 +60,7 @@ func TestAddrBookLookup(t *testing.T) {
randAddrs := randNetAddressPairs(t, 100)
book := NewAddrBook(fname, true)
book.SetLogger(log.TestingLogger())
for _, addrSrc := range randAddrs {
addr := addrSrc.addr
src := addrSrc.src
@ -76,6 +81,7 @@ func TestAddrBookPromoteToOld(t *testing.T) {
randAddrs := randNetAddressPairs(t, 100)
book := NewAddrBook(fname, true)
book.SetLogger(log.TestingLogger())
for _, addrSrc := range randAddrs {
book.AddAddress(addrSrc.addr, addrSrc.src)
}
@ -106,6 +112,7 @@ func TestAddrBookHandlesDuplicates(t *testing.T) {
fname := createTempFileName("addrbook_test")
book := NewAddrBook(fname, true)
book.SetLogger(log.TestingLogger())
randAddrs := randNetAddressPairs(t, 100)
@ -152,6 +159,7 @@ func randIPv4Address(t *testing.T) *NetAddress {
func TestAddrBookRemoveAddress(t *testing.T) {
fname := createTempFileName("addrbook_test")
book := NewAddrBook(fname, true)
book.SetLogger(log.TestingLogger())
addr := randIPv4Address(t)
book.AddAddress(addr, addr)


+ 21
- 21
p2p/connection.go View File

@ -140,7 +140,7 @@ func NewMConnectionWithConfig(conn net.Conn, chDescs []*ChannelDescriptor, onRec
mconn.channels = channels
mconn.channelsIdx = channelsIdx
mconn.BaseService = *cmn.NewBaseService(log, "MConnection", mconn)
mconn.BaseService = *cmn.NewBaseService(nil, "MConnection", mconn)
return mconn
}
@ -177,10 +177,10 @@ func (c *MConnection) String() string {
}
func (c *MConnection) flush() {
log.Debug("Flush", "conn", c)
c.Logger.Debug("Flush", "conn", c)
err := c.bufWriter.Flush()
if err != nil {
log.Warn("MConnection flush failed", "error", err)
c.Logger.Error("MConnection flush failed", "error", err)
}
}
@ -208,12 +208,12 @@ func (c *MConnection) Send(chID byte, msg interface{}) bool {
return false
}
log.Debug("Send", "channel", chID, "conn", c, "msg", msg) //, "bytes", wire.BinaryBytes(msg))
c.Logger.Debug("Send", "channel", chID, "conn", c, "msg", msg) //, "bytes", wire.BinaryBytes(msg))
// Send message to channel.
channel, ok := c.channelsIdx[chID]
if !ok {
log.Error(cmn.Fmt("Cannot send bytes, unknown channel %X", chID))
c.Logger.Error(cmn.Fmt("Cannot send bytes, unknown channel %X", chID))
return false
}
@ -225,7 +225,7 @@ func (c *MConnection) Send(chID byte, msg interface{}) bool {
default:
}
} else {
log.Warn("Send failed", "channel", chID, "conn", c, "msg", msg)
c.Logger.Error("Send failed", "channel", chID, "conn", c, "msg", msg)
}
return success
}
@ -237,12 +237,12 @@ func (c *MConnection) TrySend(chID byte, msg interface{}) bool {
return false
}
log.Debug("TrySend", "channel", chID, "conn", c, "msg", msg)
c.Logger.Debug("TrySend", "channel", chID, "conn", c, "msg", msg)
// Send message to channel.
channel, ok := c.channelsIdx[chID]
if !ok {
log.Error(cmn.Fmt("Cannot send bytes, unknown channel %X", chID))
c.Logger.Error(cmn.Fmt("Cannot send bytes, unknown channel %X", chID))
return false
}
@ -267,7 +267,7 @@ func (c *MConnection) CanSend(chID byte) bool {
channel, ok := c.channelsIdx[chID]
if !ok {
log.Error(cmn.Fmt("Unknown channel %X", chID))
c.Logger.Error(cmn.Fmt("Unknown channel %X", chID))
return false
}
return channel.canSend()
@ -291,12 +291,12 @@ FOR_LOOP:
channel.updateStats()
}
case <-c.pingTimer.Ch:
log.Debug("Send Ping")
c.Logger.Debug("Send Ping")
wire.WriteByte(packetTypePing, c.bufWriter, &n, &err)
c.sendMonitor.Update(int(n))
c.flush()
case <-c.pong:
log.Debug("Send Pong")
c.Logger.Debug("Send Pong")
wire.WriteByte(packetTypePong, c.bufWriter, &n, &err)
c.sendMonitor.Update(int(n))
c.flush()
@ -318,7 +318,7 @@ FOR_LOOP:
break FOR_LOOP
}
if err != nil {
log.Warn("Connection failed @ sendRoutine", "conn", c, "error", err)
c.Logger.Error("Connection failed @ sendRoutine", "conn", c, "error", err)
c.stopForError(err)
break FOR_LOOP
}
@ -367,13 +367,13 @@ func (c *MConnection) sendMsgPacket() bool {
if leastChannel == nil {
return true
} else {
// log.Info("Found a msgPacket to send")
// c.Logger.Info("Found a msgPacket to send")
}
// Make & send a msgPacket from this channel
n, err := leastChannel.writeMsgPacketTo(c.bufWriter)
if err != nil {
log.Warn("Failed to write msgPacket", "error", err)
c.Logger.Error("Failed to write msgPacket", "error", err)
c.stopForError(err)
return true
}
@ -415,7 +415,7 @@ FOR_LOOP:
c.recvMonitor.Update(int(n))
if err != nil {
if c.IsRunning() {
log.Warn("Connection failed @ recvRoutine (reading byte)", "conn", c, "error", err)
c.Logger.Error("Connection failed @ recvRoutine (reading byte)", "conn", c, "error", err)
c.stopForError(err)
}
break FOR_LOOP
@ -425,18 +425,18 @@ FOR_LOOP:
switch pktType {
case packetTypePing:
// TODO: prevent abuse, as they cause flush()'s.
log.Debug("Receive Ping")
c.Logger.Debug("Receive Ping")
c.pong <- struct{}{}
case packetTypePong:
// do nothing
log.Debug("Receive Pong")
c.Logger.Debug("Receive Pong")
case packetTypeMsg:
pkt, n, err := msgPacket{}, int(0), error(nil)
wire.ReadBinaryPtr(&pkt, c.bufReader, maxMsgPacketTotalSize, &n, &err)
c.recvMonitor.Update(int(n))
if err != nil {
if c.IsRunning() {
log.Warn("Connection failed @ recvRoutine", "conn", c, "error", err)
c.Logger.Error("Connection failed @ recvRoutine", "conn", c, "error", err)
c.stopForError(err)
}
break FOR_LOOP
@ -448,13 +448,13 @@ FOR_LOOP:
msgBytes, err := channel.recvMsgPacket(pkt)
if err != nil {
if c.IsRunning() {
log.Warn("Connection failed @ recvRoutine", "conn", c, "error", err)
c.Logger.Error("Connection failed @ recvRoutine", "conn", c, "error", err)
c.stopForError(err)
}
break FOR_LOOP
}
if msgBytes != nil {
log.Debug("Received bytes", "chID", pkt.ChannelID, "msgBytes", msgBytes)
c.Logger.Debug("Received bytes", "chID", pkt.ChannelID, "msgBytes", msgBytes)
c.onReceive(pkt.ChannelID, msgBytes)
}
default:
@ -626,7 +626,7 @@ func (ch *Channel) nextMsgPacket() msgPacket {
// Not goroutine-safe
func (ch *Channel) writeMsgPacketTo(w io.Writer) (n int, err error) {
packet := ch.nextMsgPacket()
log.Debug("Write Msg Packet", "conn", ch.conn, "packet", packet)
// log.Debug("Write Msg Packet", "conn", ch.conn, "packet", packet)
wire.WriteByte(packetTypeMsg, w, &n, &err)
wire.WriteBinary(packet, w, &n, &err)
if err == nil {


+ 7
- 2
p2p/connection_test.go View File

@ -8,6 +8,7 @@ import (
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
p2p "github.com/tendermint/tendermint/p2p"
"github.com/tendermint/tmlibs/log"
)
func createMConnection(conn net.Conn) *p2p.MConnection {
@ -15,12 +16,16 @@ func createMConnection(conn net.Conn) *p2p.MConnection {
}
onError := func(r interface{}) {
}
return createMConnectionWithCallbacks(conn, onReceive, onError)
c := createMConnectionWithCallbacks(conn, onReceive, onError)
c.SetLogger(log.TestingLogger())
return c
}
func createMConnectionWithCallbacks(conn net.Conn, onReceive func(chID byte, msgBytes []byte), onError func(r interface{})) *p2p.MConnection {
chDescs := []*p2p.ChannelDescriptor{&p2p.ChannelDescriptor{ID: 0x01, Priority: 1, SendQueueCapacity: 1}}
return p2p.NewMConnection(conn, chDescs, onReceive, onError)
c := p2p.NewMConnection(conn, chDescs, onReceive, onError)
c.SetLogger(log.TestingLogger())
return c
}
func TestMConnectionSend(t *testing.T) {


+ 153
- 0
p2p/glide.lock View File

@ -0,0 +1,153 @@
hash: 56a5590512e71b5bae4af7bd6c5e352008ad5725a8595c3e128cd18042f95b4c
updated: 2017-05-02T13:43:02.638937348Z
imports:
- name: github.com/btcsuite/btcd
version: 4b348c1d33373d672edd83fc576892d0e46686d2
subpackages:
- btcec
- name: github.com/ebuchman/fail-test
version: 95f809107225be108efcf10a3509e4ea6ceef3c4
- name: github.com/fsnotify/fsnotify
version: 4da3e2cfbabc9f751898f250b49f2439785783a1
- name: github.com/go-kit/kit
version: 8a2988aa81f699fc1e647c3c9dddce0113ef1bfb
subpackages:
- log
- log/level
- log/term
- name: github.com/go-logfmt/logfmt
version: 390ab7935ee28ec6b286364bba9b4dd6410cb3d5
- name: github.com/go-stack/stack
version: 100eb0c0a9c5b306ca2fb4f165df21d80ada4b82
- name: github.com/gogo/protobuf
version: 100ba4e885062801d56799d78530b73b178a78f3
subpackages:
- proto
- name: github.com/golang/protobuf
version: 2bba0603135d7d7f5cb73b2125beeda19c09f4ef
subpackages:
- proto
- name: github.com/gorilla/websocket
version: 3ab3a8b8831546bd18fd182c20687ca853b2bb13
- name: github.com/hashicorp/hcl
version: 7fa7fff964d035e8a162cce3a164b3ad02ad651b
subpackages:
- hcl/ast
- hcl/parser
- hcl/scanner
- hcl/strconv
- hcl/token
- json/parser
- json/scanner
- json/token
- name: github.com/kr/logfmt
version: b84e30acd515aadc4b783ad4ff83aff3299bdfe0
- name: github.com/magiconair/properties
version: 51463bfca2576e06c62a8504b5c0f06d61312647
- name: github.com/mattn/go-colorable
version: ded68f7a9561c023e790de24279db7ebf473ea80
- name: github.com/mattn/go-isatty
version: fc9e8d8ef48496124e79ae0df75490096eccf6fe
- name: github.com/mitchellh/mapstructure
version: cc8532a8e9a55ea36402aa21efdf403a60d34096
- name: github.com/pelletier/go-buffruneio
version: c37440a7cf42ac63b919c752ca73a85067e05992
- name: github.com/pelletier/go-toml
version: fe206efb84b2bc8e8cfafe6b4c1826622be969e3
- name: github.com/pkg/errors
version: 645ef00459ed84a119197bfb8d8205042c6df63d
- name: github.com/spf13/afero
version: 9be650865eab0c12963d8753212f4f9c66cdcf12
subpackages:
- mem
- name: github.com/spf13/cast
version: acbeb36b902d72a7a4c18e8f3241075e7ab763e4
- name: github.com/spf13/cobra
version: 10f6b9d7e1631a54ad07c5c0fb71c28a1abfd3c2
- name: github.com/spf13/jwalterweatherman
version: fa7ca7e836cf3a8bb4ebf799f472c12d7e903d66
- name: github.com/spf13/pflag
version: 2300d0f8576fe575f71aaa5b9bbe4e1b0dc2eb51
- name: github.com/spf13/viper
version: 0967fc9aceab2ce9da34061253ac10fb99bba5b2
- name: github.com/stretchr/testify
version: 69483b4bd14f5845b5a1e55bca19e954e827f1d0
subpackages:
- assert
- require
- name: github.com/tendermint/abci
version: 8d8e35ae537538c9cf6808be3ca9dd7dab81b7f6
subpackages:
- client
- example/dummy
- types
- name: github.com/tendermint/ed25519
version: 1f52c6f8b8a5c7908aff4497c186af344b428925
subpackages:
- edwards25519
- extra25519
- name: github.com/tendermint/go-crypto
version: 197a2b270fd94ee03824b158e738fce62862d0b8
- name: github.com/tendermint/go-wire
version: b53add0b622662731985485f3a19be7f684660b8
subpackages:
- data
- name: github.com/tendermint/log15
version: ae0f3d6450da9eac7074b439c8e1c3cabf0d5ce6
subpackages:
- term
- name: github.com/tendermint/tmlibs
version: 9687e4202662c11add147906faf5503af519b4ef
subpackages:
- autofile
- clist
- common
- db
- events
- flowrate
- log
- logger
- merkle
- name: golang.org/x/crypto
version: 96846453c37f0876340a66a47f3f75b1f3a6cd2d
subpackages:
- curve25519
- nacl/box
- nacl/secretbox
- openpgp/armor
- openpgp/errors
- poly1305
- ripemd160
- salsa20/salsa
- name: golang.org/x/net
version: c8c74377599bd978aee1cf3b9b63a8634051cec2
subpackages:
- context
- name: golang.org/x/sys
version: ea9bcade75cb975a0b9738936568ab388b845617
subpackages:
- unix
- name: golang.org/x/text
version: 19e3104b43db45fca0303f489a9536087b184802
subpackages:
- transform
- unicode/norm
- name: google.golang.org/grpc
version: 6914ab1e338c92da4218a23d27fcd03d0ad78d46
- name: gopkg.in/yaml.v2
version: cd8b52f8269e0feb286dfeef29f8fe4d5b397e0b
testImports:
- name: github.com/davecgh/go-spew
version: 6d212800a42e8ab5c146b8ace3490ee17e5225f9
subpackages:
- spew
- name: github.com/pmezard/go-difflib
version: d8ed2627bdf02c080bf22230dbb337003b7aba2d
subpackages:
- difflib
- name: github.com/tendermint/merkleeyes
version: d0aa363fd4e015e509038c3a0ec493bc62ee0b8a
subpackages:
- app
- iavl
- testutil

+ 11
- 10
p2p/listener.go View File

@ -8,6 +8,7 @@ import (
"github.com/tendermint/tendermint/p2p/upnp"
cmn "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tmlibs/log"
)
type Listener interface {
@ -47,7 +48,7 @@ func splitHostPort(addr string) (host string, port int) {
}
// skipUPNP: If true, does not try getUPNPExternalAddress()
func NewDefaultListener(protocol string, lAddr string, skipUPNP bool) Listener {
func NewDefaultListener(protocol string, lAddr string, skipUPNP bool, logger log.Logger) Listener {
// Local listen IP & port
lAddrIP, lAddrPort := splitHostPort(lAddr)
@ -67,7 +68,7 @@ func NewDefaultListener(protocol string, lAddr string, skipUPNP bool) Listener {
}
// Actual listener local IP & port
listenerIP, listenerPort := splitHostPort(listener.Addr().String())
log.Info("Local listener", "ip", listenerIP, "port", listenerPort)
logger.Info("Local listener", "ip", listenerIP, "port", listenerPort)
// Determine internal address...
var intAddr *NetAddress
@ -81,7 +82,7 @@ func NewDefaultListener(protocol string, lAddr string, skipUPNP bool) Listener {
if !skipUPNP {
// If the lAddrIP is INADDR_ANY, try UPnP
if lAddrIP == "" || lAddrIP == "0.0.0.0" {
extAddr = getUPNPExternalAddress(lAddrPort, listenerPort)
extAddr = getUPNPExternalAddress(lAddrPort, listenerPort, logger)
}
}
// Otherwise just use the local address...
@ -98,7 +99,7 @@ func NewDefaultListener(protocol string, lAddr string, skipUPNP bool) Listener {
extAddr: extAddr,
connections: make(chan net.Conn, numBufferedConnections),
}
dl.BaseService = *cmn.NewBaseService(log, "DefaultListener", dl)
dl.BaseService = *cmn.NewBaseService(logger, "DefaultListener", dl)
dl.Start() // Started upon construction
return dl
}
@ -166,17 +167,17 @@ func (l *DefaultListener) String() string {
/* external address helpers */
// UPNP external address discovery & port mapping
func getUPNPExternalAddress(externalPort, internalPort int) *NetAddress {
log.Info("Getting UPNP external address")
func getUPNPExternalAddress(externalPort, internalPort int, logger log.Logger) *NetAddress {
logger.Info("Getting UPNP external address")
nat, err := upnp.Discover()
if err != nil {
log.Info("Could not perform UPNP discover", "error", err)
logger.Info("Could not perform UPNP discover", "error", err)
return nil
}
ext, err := nat.GetExternalAddress()
if err != nil {
log.Info("Could not get UPNP external address", "error", err)
logger.Info("Could not get UPNP external address", "error", err)
return nil
}
@ -187,11 +188,11 @@ func getUPNPExternalAddress(externalPort, internalPort int) *NetAddress {
externalPort, err = nat.AddPortMapping("tcp", externalPort, internalPort, "tendermint", 0)
if err != nil {
log.Info("Could not add UPNP port mapping", "error", err)
logger.Info("Could not add UPNP port mapping", "error", err)
return nil
}
log.Info("Got UPNP external address", "address", ext)
logger.Info("Got UPNP external address", "address", ext)
return NewNetAddressIPPort(ext, uint16(externalPort))
}


+ 3
- 1
p2p/listener_test.go View File

@ -3,11 +3,13 @@ package p2p
import (
"bytes"
"testing"
"github.com/tendermint/tmlibs/log"
)
func TestListener(t *testing.T) {
// Create a listener
l := NewDefaultListener("tcp", ":8001", true)
l := NewDefaultListener("tcp", ":8001", true, log.TestingLogger())
// Dial the listener
lAddr := l.ExternalAddress()


+ 0
- 7
p2p/log.go View File

@ -1,7 +0,0 @@
package p2p
import (
"github.com/tendermint/tmlibs/logger"
)
var log = logger.New("module", "p2p")

+ 4
- 4
p2p/peer.go View File

@ -115,7 +115,7 @@ func newPeerFromConnAndConfig(rawConn net.Conn, outbound bool, reactorsByCh map[
p.mconn = createMConnection(conn, p, reactorsByCh, chDescs, onPeerError, config.MConfig)
p.BaseService = *cmn.NewBaseService(log, "Peer", p)
p.BaseService = *cmn.NewBaseService(nil, "Peer", p)
return p, nil
}
@ -156,7 +156,7 @@ func (p *Peer) HandshakeTimeout(ourNodeInfo *NodeInfo, timeout time.Duration) er
func() {
var n int
wire.ReadBinary(peerNodeInfo, p.conn, maxNodeInfoSize, &n, &err2)
log.Notice("Peer handshake", "peerNodeInfo", peerNodeInfo)
p.Logger.Info("Peer handshake", "peerNodeInfo", peerNodeInfo)
})
if err1 != nil {
return errors.Wrap(err1, "Error during handshake/write")
@ -279,10 +279,10 @@ func (p *Peer) Get(key string) interface{} {
}
func dial(addr *NetAddress, config *PeerConfig) (net.Conn, error) {
log.Info("Dialing address", "address", addr)
// log.Info("Dialing address", "address", addr)
conn, err := addr.DialTimeout(config.DialTimeout * time.Second)
if err != nil {
log.Info("Failed dialing address", "address", addr, "error", err)
// log.Info("Failed dialing address", "address", addr, "error", err)
return nil, err
}
return conn, nil


+ 11
- 11
p2p/pex_reactor.go View File

@ -7,8 +7,8 @@ import (
"reflect"
"time"
cmn "github.com/tendermint/tmlibs/common"
wire "github.com/tendermint/go-wire"
cmn "github.com/tendermint/tmlibs/common"
)
const (
@ -61,7 +61,7 @@ func NewPEXReactor(b *AddrBook) *PEXReactor {
msgCountByPeer: cmn.NewCMap(),
maxMsgCountByPeer: defaultMaxMsgCountByPeer,
}
r.BaseReactor = *NewBaseReactor(log, "PEXReactor", r)
r.BaseReactor = *NewBaseReactor("PEXReactor", r)
return r
}
@ -105,7 +105,7 @@ func (r *PEXReactor) AddPeer(p *Peer) {
addr, err := NewNetAddressString(p.ListenAddr)
if err != nil {
// this should never happen
log.Error("Error in AddPeer: invalid peer address", "addr", p.ListenAddr, "error", err)
r.Logger.Error("Error in AddPeer: invalid peer address", "addr", p.ListenAddr, "error", err)
return
}
r.book.AddAddress(addr, addr)
@ -125,17 +125,17 @@ func (r *PEXReactor) Receive(chID byte, src *Peer, msgBytes []byte) {
r.IncrementMsgCountForPeer(srcAddrStr)
if r.ReachedMaxMsgCountForPeer(srcAddrStr) {
log.Warn("Maximum number of messages reached for peer", "peer", srcAddrStr)
r.Logger.Error("Maximum number of messages reached for peer", "peer", srcAddrStr)
// TODO remove src from peers?
return
}
_, msg, err := DecodeMessage(msgBytes)
if err != nil {
log.Warn("Error decoding message", "error", err)
r.Logger.Error("Error decoding message", "error", err)
return
}
log.Notice("Received message", "msg", msg)
r.Logger.Info("Received message", "msg", msg)
switch msg := msg.(type) {
case *pexRequestMessage:
@ -150,7 +150,7 @@ func (r *PEXReactor) Receive(chID byte, src *Peer, msgBytes []byte) {
}
}
default:
log.Warn(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
r.Logger.Error(fmt.Sprintf("Unknown message type %v", reflect.TypeOf(msg)))
}
}
@ -230,7 +230,7 @@ func (r *PEXReactor) ensurePeersRoutine() {
func (r *PEXReactor) ensurePeers() {
numOutPeers, _, numDialing := r.Switch.NumPeers()
numToDial := minNumOutboundPeers - (numOutPeers + numDialing)
log.Info("Ensure peers", "numOutPeers", numOutPeers, "numDialing", numDialing, "numToDial", numToDial)
r.Logger.Info("Ensure peers", "numOutPeers", numOutPeers, "numDialing", numDialing, "numToDial", numToDial)
if numToDial <= 0 {
return
}
@ -257,13 +257,13 @@ func (r *PEXReactor) ensurePeers() {
alreadyDialing := r.Switch.IsDialing(try)
alreadyConnected := r.Switch.Peers().Has(try.IP.String())
if alreadySelected || alreadyDialing || alreadyConnected {
// log.Info("Cannot dial address", "addr", try,
// r.Logger.Info("Cannot dial address", "addr", try,
// "alreadySelected", alreadySelected,
// "alreadyDialing", alreadyDialing,
// "alreadyConnected", alreadyConnected)
continue
} else {
log.Info("Will dial address", "addr", try)
r.Logger.Info("Will dial address", "addr", try)
picked = try
break
}
@ -289,7 +289,7 @@ func (r *PEXReactor) ensurePeers() {
if peers := r.Switch.Peers().List(); len(peers) > 0 {
i := rand.Int() % len(peers)
peer := peers[i]
log.Info("No addresses to dial. Sending pexRequest to random peer", "peer", peer)
r.Logger.Info("No addresses to dial. Sending pexRequest to random peer", "peer", peer)
r.RequestPEX(peer)
}
}


+ 12
- 1
p2p/pex_reactor_test.go View File

@ -11,6 +11,7 @@ import (
"github.com/stretchr/testify/require"
wire "github.com/tendermint/go-wire"
cmn "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tmlibs/log"
)
func TestPEXReactorBasic(t *testing.T) {
@ -20,8 +21,10 @@ func TestPEXReactorBasic(t *testing.T) {
require.Nil(err)
defer os.RemoveAll(dir)
book := NewAddrBook(dir+"addrbook.json", true)
book.SetLogger(log.TestingLogger())
r := NewPEXReactor(book)
r.SetLogger(log.TestingLogger())
assert.NotNil(r)
assert.NotEmpty(r.GetChannels())
@ -34,8 +37,10 @@ func TestPEXReactorAddRemovePeer(t *testing.T) {
require.Nil(err)
defer os.RemoveAll(dir)
book := NewAddrBook(dir+"addrbook.json", true)
book.SetLogger(log.TestingLogger())
r := NewPEXReactor(book)
r.SetLogger(log.TestingLogger())
size := book.Size()
peer := createRandomPeer(false)
@ -65,11 +70,13 @@ func TestPEXReactorRunning(t *testing.T) {
require.Nil(err)
defer os.RemoveAll(dir)
book := NewAddrBook(dir+"addrbook.json", false)
book.SetLogger(log.TestingLogger())
// create switches
for i := 0; i < N; i++ {
switches[i] = makeSwitch(config, i, "127.0.0.1", "123.123.123", func(i int, sw *Switch) *Switch {
r := NewPEXReactor(book)
r.SetLogger(log.TestingLogger())
r.SetEnsurePeersPeriod(250 * time.Millisecond)
sw.AddReactor("pex", r)
return sw
@ -80,7 +87,7 @@ func TestPEXReactorRunning(t *testing.T) {
for _, s := range switches {
addr, _ := NewNetAddressString(s.NodeInfo().ListenAddr)
book.AddAddress(addr, addr)
s.AddListener(NewDefaultListener("tcp", s.NodeInfo().ListenAddr, true))
s.AddListener(NewDefaultListener("tcp", s.NodeInfo().ListenAddr, true, log.TestingLogger()))
}
// start switches
@ -112,8 +119,10 @@ func TestPEXReactorReceive(t *testing.T) {
require.Nil(err)
defer os.RemoveAll(dir)
book := NewAddrBook(dir+"addrbook.json", true)
book.SetLogger(log.TestingLogger())
r := NewPEXReactor(book)
r.SetLogger(log.TestingLogger())
peer := createRandomPeer(false)
@ -135,8 +144,10 @@ func TestPEXReactorAbuseFromPeer(t *testing.T) {
require.Nil(err)
defer os.RemoveAll(dir)
book := NewAddrBook(dir+"addrbook.json", true)
book.SetLogger(log.TestingLogger())
r := NewPEXReactor(book)
r.SetLogger(log.TestingLogger())
r.SetMaxMsgCountByPeer(5)
peer := createRandomPeer(false)


+ 19
- 20
p2p/switch.go View File

@ -8,7 +8,6 @@ import (
"time"
crypto "github.com/tendermint/go-crypto"
"github.com/tendermint/log15"
cfg "github.com/tendermint/tendermint/config"
cmn "github.com/tendermint/tmlibs/common"
)
@ -35,9 +34,9 @@ type BaseReactor struct {
Switch *Switch
}
func NewBaseReactor(log log15.Logger, name string, impl Reactor) *BaseReactor {
func NewBaseReactor(name string, impl Reactor) *BaseReactor {
return &BaseReactor{
BaseService: *cmn.NewBaseService(log, name, impl),
BaseService: *cmn.NewBaseService(nil, name, impl),
Switch: nil,
}
}
@ -92,7 +91,7 @@ func NewSwitch(config *cfg.P2PConfig) *Switch {
dialing: cmn.NewCMap(),
nodeInfo: nil,
}
sw.BaseService = *cmn.NewBaseService(log, "P2P Switch", sw)
sw.BaseService = *cmn.NewBaseService(nil, "P2P Switch", sw)
return sw
}
@ -225,7 +224,7 @@ func (sw *Switch) AddPeer(peer *Peer) error {
// Add the peer to .peers
// ignore if duplicate or if we already have too many for that IP range
if err := sw.peers.Add(peer); err != nil {
log.Notice("Ignoring peer", "error", err, "peer", peer)
sw.Logger.Info("Ignoring peer", "error", err, "peer", peer)
peer.Stop()
return err
}
@ -235,7 +234,7 @@ func (sw *Switch) AddPeer(peer *Peer) error {
sw.startInitPeer(peer)
}
log.Notice("Added peer", "peer", peer)
sw.Logger.Info("Added peer", "peer", peer)
return nil
}
@ -306,10 +305,10 @@ func (sw *Switch) DialSeeds(addrBook *AddrBook, seeds []string) error {
func (sw *Switch) dialSeed(addr *NetAddress) {
peer, err := sw.DialPeerWithAddress(addr, true)
if err != nil {
log.Error("Error dialing seed", "error", err)
sw.Logger.Error("Error dialing seed", "error", err)
return
} else {
log.Notice("Connected to seed", "peer", peer)
sw.Logger.Info("Connected to seed", "peer", peer)
}
}
@ -319,7 +318,7 @@ func (sw *Switch) DialPeerWithAddress(addr *NetAddress, persistent bool) (*Peer,
peer, err := newOutboundPeerWithConfig(addr, sw.reactorsByCh, sw.chDescs, sw.StopPeerForError, sw.nodePrivKey, sw.peerConfig)
if err != nil {
log.Info("Failed dialing peer", "address", addr, "error", err)
sw.Logger.Info("Failed dialing peer", "address", addr, "error", err)
return nil, err
}
if persistent {
@ -327,11 +326,11 @@ func (sw *Switch) DialPeerWithAddress(addr *NetAddress, persistent bool) (*Peer,
}
err = sw.AddPeer(peer)
if err != nil {
log.Info("Failed adding peer", "address", addr, "error", err)
sw.Logger.Info("Failed adding peer", "address", addr, "error", err)
peer.CloseConn()
return nil, err
}
log.Notice("Dialed and added peer", "address", addr, "peer", peer)
sw.Logger.Info("Dialed and added peer", "address", addr, "peer", peer)
return peer, nil
}
@ -345,7 +344,7 @@ func (sw *Switch) IsDialing(addr *NetAddress) bool {
// NOTE: Broadcast uses goroutines, so order of broadcast may not be preserved.
func (sw *Switch) Broadcast(chID byte, msg interface{}) chan bool {
successChan := make(chan bool, len(sw.peers.List()))
log.Debug("Broadcast", "channel", chID, "msg", msg)
sw.Logger.Debug("Broadcast", "channel", chID, "msg", msg)
for _, peer := range sw.peers.List() {
go func(peer *Peer) {
success := peer.Send(chID, msg)
@ -377,12 +376,12 @@ func (sw *Switch) Peers() IPeerSet {
// TODO: make record depending on reason.
func (sw *Switch) StopPeerForError(peer *Peer, reason interface{}) {
addr := NewNetAddress(peer.Addr())
log.Notice("Stopping peer for error", "peer", peer, "error", reason)
sw.Logger.Info("Stopping peer for error", "peer", peer, "error", reason)
sw.stopAndRemovePeer(peer, reason)
if peer.IsPersistent() {
go func() {
log.Notice("Reconnecting to peer", "peer", peer)
sw.Logger.Info("Reconnecting to peer", "peer", peer)
for i := 1; i < reconnectAttempts; i++ {
if !sw.IsRunning() {
return
@ -391,15 +390,15 @@ func (sw *Switch) StopPeerForError(peer *Peer, reason interface{}) {
peer, err := sw.DialPeerWithAddress(addr, true)
if err != nil {
if i == reconnectAttempts {
log.Notice("Error reconnecting to peer. Giving up", "tries", i, "error", err)
sw.Logger.Info("Error reconnecting to peer. Giving up", "tries", i, "error", err)
return
}
log.Notice("Error reconnecting to peer. Trying again", "tries", i, "error", err)
sw.Logger.Info("Error reconnecting to peer. Trying again", "tries", i, "error", err)
time.Sleep(reconnectInterval)
continue
}
log.Notice("Reconnected to peer", "peer", peer)
sw.Logger.Info("Reconnected to peer", "peer", peer)
return
}
}()
@ -409,7 +408,7 @@ func (sw *Switch) StopPeerForError(peer *Peer, reason interface{}) {
// Disconnect from a peer gracefully.
// TODO: handle graceful disconnects.
func (sw *Switch) StopPeerGracefully(peer *Peer) {
log.Notice("Stopping peer gracefully")
sw.Logger.Info("Stopping peer gracefully")
sw.stopAndRemovePeer(peer, nil)
}
@ -431,14 +430,14 @@ func (sw *Switch) listenerRoutine(l Listener) {
// ignore connection if we already have enough
maxPeers := sw.config.MaxNumPeers
if maxPeers <= sw.peers.Size() {
log.Info("Ignoring inbound connection: already have enough peers", "address", inConn.RemoteAddr().String(), "numPeers", sw.peers.Size(), "max", maxPeers)
sw.Logger.Info("Ignoring inbound connection: already have enough peers", "address", inConn.RemoteAddr().String(), "numPeers", sw.peers.Size(), "max", maxPeers)
continue
}
// New inbound connection!
err := sw.addPeerWithConnectionAndConfig(inConn, sw.peerConfig)
if err != nil {
log.Notice("Ignoring inbound connection: error while adding peer", "address", inConn.RemoteAddr().String(), "error", err)
sw.Logger.Info("Ignoring inbound connection: error while adding peer", "address", inConn.RemoteAddr().String(), "error", err)
continue
}


+ 4
- 5
p2p/switch_test.go View File

@ -12,9 +12,9 @@ import (
"github.com/stretchr/testify/require"
crypto "github.com/tendermint/go-crypto"
wire "github.com/tendermint/go-wire"
cmn "github.com/tendermint/tmlibs/common"
cfg "github.com/tendermint/tendermint/config"
"github.com/tendermint/tmlibs/log"
)
var (
@ -50,7 +50,8 @@ func NewTestReactor(channels []*ChannelDescriptor, logMessages bool) *TestReacto
logMessages: logMessages,
msgsReceived: make(map[byte][]PeerMessage),
}
tr.BaseReactor = *NewBaseReactor(log, "TestReactor", tr)
tr.BaseReactor = *NewBaseReactor("TestReactor", tr)
tr.SetLogger(log.TestingLogger())
return tr
}
@ -285,7 +286,6 @@ func TestSwitchReconnectsToPersistentPeer(t *testing.T) {
}
func BenchmarkSwitches(b *testing.B) {
b.StopTimer()
s1, s2 := makeSwitchPair(b, func(i int, sw *Switch) *Switch {
@ -322,10 +322,9 @@ func BenchmarkSwitches(b *testing.B) {
}
}
log.Warn(cmn.Fmt("success: %v, failure: %v", numSuccess, numFailure))
b.Logf("success: %v, failure: %v", numSuccess, numFailure)
// Allow everything to flush before stopping switches & closing connections.
b.StopTimer()
time.Sleep(1000 * time.Millisecond)
}

+ 0
- 7
proxy/log.go View File

@ -1,7 +0,0 @@
package proxy
import (
"github.com/tendermint/tmlibs/logger"
)
var log = logger.New("module", "proxy")

+ 1
- 1
proxy/multi_app_conn.go View File

@ -48,7 +48,7 @@ func NewMultiAppConn(clientCreator ClientCreator, handshaker Handshaker) *multiA
handshaker: handshaker,
clientCreator: clientCreator,
}
multiAppConn.BaseService = *cmn.NewBaseService(log, "multiAppConn", multiAppConn)
multiAppConn.BaseService = *cmn.NewBaseService(nil, "multiAppConn", multiAppConn)
return multiAppConn
}


+ 1
- 1
rpc/core/abci.go View File

@ -17,7 +17,7 @@ func ABCIQuery(path string, data data.Bytes, prove bool) (*ctypes.ResultABCIQuer
if err != nil {
return nil, err
}
log.Info("ABCIQuery", "path", path, "data", data, "result", resQuery)
logger.Info("ABCIQuery", "path", path, "data", data, "result", resQuery)
return &ctypes.ResultABCIQuery{
resQuery.Result(),
}, nil


+ 3
- 2
rpc/core/blocks.go View File

@ -2,9 +2,10 @@ package core
import (
"fmt"
. "github.com/tendermint/tmlibs/common"
ctypes "github.com/tendermint/tendermint/rpc/core/types"
"github.com/tendermint/tendermint/types"
. "github.com/tendermint/tmlibs/common"
)
//-----------------------------------------------------------------------------
@ -19,7 +20,7 @@ func BlockchainInfo(minHeight, maxHeight int) (*ctypes.ResultBlockchainInfo, err
if minHeight == 0 {
minHeight = MaxInt(1, maxHeight-20)
}
log.Debug("BlockchainInfoHandler", "maxHeight", maxHeight, "minHeight", minHeight)
logger.Debug("BlockchainInfoHandler", "maxHeight", maxHeight, "minHeight", minHeight)
blockMetas := []*types.BlockMeta{}
for height := maxHeight; height >= minHeight; height-- {


+ 2
- 2
rpc/core/events.go View File

@ -7,7 +7,7 @@ import (
)
func Subscribe(wsCtx rpctypes.WSRPCContext, event string) (*ctypes.ResultSubscribe, error) {
log.Notice("Subscribe to event", "remote", wsCtx.GetRemoteAddr(), "event", event)
logger.Info("Subscribe to event", "remote", wsCtx.GetRemoteAddr(), "event", event)
types.AddListenerForEvent(wsCtx.GetEventSwitch(), wsCtx.GetRemoteAddr(), event, func(msg types.TMEventData) {
// NOTE: EventSwitch callbacks must be nonblocking
// NOTE: RPCResponses of subscribed events have id suffix "#event"
@ -18,7 +18,7 @@ func Subscribe(wsCtx rpctypes.WSRPCContext, event string) (*ctypes.ResultSubscri
}
func Unsubscribe(wsCtx rpctypes.WSRPCContext, event string) (*ctypes.ResultUnsubscribe, error) {
log.Notice("Unsubscribe to event", "remote", wsCtx.GetRemoteAddr(), "event", event)
logger.Info("Unsubscribe to event", "remote", wsCtx.GetRemoteAddr(), "event", event)
wsCtx.GetEventSwitch().RemoveListenerForEvent(event, wsCtx.GetRemoteAddr())
return &ctypes.ResultUnsubscribe{}, nil
}

+ 0
- 7
rpc/core/log.go View File

@ -1,7 +0,0 @@
package core
import (
"github.com/tendermint/log15"
)
var log = log15.New("module", "rpc")

+ 3
- 3
rpc/core/mempool.go View File

@ -59,7 +59,7 @@ func BroadcastTxCommit(tx types.Tx) (*ctypes.ResultBroadcastTxCommit, error) {
checkTxResCh <- res
})
if err != nil {
log.Error("err", "err", err)
logger.Error("err", "err", err)
return nil, fmt.Errorf("Error broadcasting transaction: %v", err)
}
checkTxRes := <-checkTxResCh
@ -85,7 +85,7 @@ func BroadcastTxCommit(tx types.Tx) (*ctypes.ResultBroadcastTxCommit, error) {
Data: deliverTxRes.Data,
Log: deliverTxRes.Log,
}
log.Notice("DeliverTx passed ", "tx", data.Bytes(tx), "response", deliverTxR)
logger.Info("DeliverTx passed ", "tx", data.Bytes(tx), "response", deliverTxR)
return &ctypes.ResultBroadcastTxCommit{
CheckTx: checkTxR.Result(),
DeliverTx: deliverTxR.Result(),
@ -93,7 +93,7 @@ func BroadcastTxCommit(tx types.Tx) (*ctypes.ResultBroadcastTxCommit, error) {
Height: deliverTxRes.Height,
}, nil
case <-timer.C:
log.Error("failed to include tx")
logger.Error("failed to include tx")
return &ctypes.ResultBroadcastTxCommit{
CheckTx: checkTxR.Result(),
DeliverTx: abci.Result{},


+ 1
- 1
rpc/core/net.go View File

@ -38,7 +38,7 @@ func UnsafeDialSeeds(seeds []string) (*ctypes.ResultDialSeeds, error) {
return &ctypes.ResultDialSeeds{}, fmt.Errorf("No seeds provided")
}
// starts go routines to dial each seed after random delays
log.Info("DialSeeds", "addrBook", addrBook, "seeds", seeds)
logger.Info("DialSeeds", "addrBook", addrBook, "seeds", seeds)
err := p2pSwitch.DialSeeds(addrBook, seeds)
if err != nil {
return &ctypes.ResultDialSeeds{}, err


+ 7
- 0
rpc/core/pipe.go View File

@ -7,6 +7,7 @@ import (
"github.com/tendermint/tendermint/proxy"
"github.com/tendermint/tendermint/state/txindex"
"github.com/tendermint/tendermint/types"
"github.com/tendermint/tmlibs/log"
)
//----------------------------------------------
@ -44,6 +45,8 @@ var (
genDoc *types.GenesisDoc // cache the genesis structure
addrBook *p2p.AddrBook
txIndexer txindex.TxIndexer
logger log.Logger
)
func SetEventSwitch(evsw types.EventSwitch) {
@ -85,3 +88,7 @@ func SetProxyAppQuery(appConn proxy.AppConnQuery) {
func SetTxIndexer(indexer txindex.TxIndexer) {
txIndexer = indexer
}
func SetLogger(l log.Logger) {
logger = l
}

+ 2
- 1
rpc/lib/README.md View File

@ -82,8 +82,9 @@ mux := http.NewServeMux()
rpcserver.RegisterRPCFuncs(mux, Routes)
wm := rpcserver.NewWebsocketManager(Routes, nil)
mux.HandleFunc("/websocket", wm.WebsocketHandler)
logger := log.NewTMLogger(log.NewSyncWriter(os.Stdout))
go func() {
_, err := rpcserver.StartHTTPServer("0.0.0.0:8008", mux)
_, err := rpcserver.StartHTTPServer("0.0.0.0:8008", mux, logger)
if err != nil {
panic(err)
}


+ 1
- 1
rpc/lib/client/http_client.go View File

@ -26,7 +26,7 @@ func makeHTTPDialer(remoteAddr string) (string, func(string, string) (net.Conn,
parts := strings.SplitN(remoteAddr, "://", 2)
var protocol, address string
if len(parts) != 2 {
log.Warn("WARNING (tendermint/rpc/lib): Please use fully formed listening addresses, including the tcp:// or unix:// prefix")
// log.Warn("WARNING (tendermint/rpc/lib): Please use fully formed listening addresses, including the tcp:// or unix:// prefix")
protocol = types.SocketType(remoteAddr)
address = remoteAddr
} else {


+ 0
- 7
rpc/lib/client/log.go View File

@ -1,7 +0,0 @@
package rpcclient
import (
"github.com/tendermint/log15"
)
var log = log15.New("module", "rpcclient")

+ 3
- 3
rpc/lib/client/ws_client.go View File

@ -37,7 +37,7 @@ func NewWSClient(remoteAddr, endpoint string) *WSClient {
Endpoint: endpoint,
Conn: nil,
}
wsClient.BaseService = *cmn.NewBaseService(log, "WSClient", wsClient)
wsClient.BaseService = *cmn.NewBaseService(nil, "WSClient", wsClient)
return wsClient
}
@ -100,14 +100,14 @@ func (wsc *WSClient) receiveEventsRoutine() {
for {
_, data, err := wsc.ReadMessage()
if err != nil {
log.Info("WSClient failed to read message", "error", err, "data", string(data))
wsc.Logger.Info("WSClient failed to read message", "error", err, "data", string(data))
wsc.Stop()
break
} else {
var response types.RPCResponse
err := json.Unmarshal(data, &response)
if err != nil {
log.Info("WSClient failed to parse message", "error", err, "data", string(data))
wsc.Logger.Info("WSClient failed to parse message", "error", err, "data", string(data))
wsc.ErrorsCh <- err
continue
}


+ 7
- 4
rpc/lib/rpc_test.go View File

@ -17,6 +17,7 @@ import (
client "github.com/tendermint/tendermint/rpc/lib/client"
server "github.com/tendermint/tendermint/rpc/lib/server"
types "github.com/tendermint/tendermint/rpc/lib/types"
"github.com/tendermint/tmlibs/log"
)
// Client and Server should work over tcp or unix sockets
@ -86,22 +87,24 @@ func init() {
}
mux := http.NewServeMux()
server.RegisterRPCFuncs(mux, Routes)
server.RegisterRPCFuncs(mux, Routes, log.TestingLogger())
wm := server.NewWebsocketManager(Routes, nil)
wm.SetLogger(log.TestingLogger())
mux.HandleFunc(websocketEndpoint, wm.WebsocketHandler)
go func() {
_, err := server.StartHTTPServer(tcpAddr, mux)
_, err := server.StartHTTPServer(tcpAddr, mux, log.TestingLogger())
if err != nil {
panic(err)
}
}()
mux2 := http.NewServeMux()
server.RegisterRPCFuncs(mux2, Routes)
server.RegisterRPCFuncs(mux2, Routes, log.TestingLogger())
wm = server.NewWebsocketManager(Routes, nil)
wm.SetLogger(log.TestingLogger())
mux2.HandleFunc(websocketEndpoint, wm.WebsocketHandler)
go func() {
_, err := server.StartHTTPServer(unixAddr, mux2)
_, err := server.StartHTTPServer(unixAddr, mux2, log.TestingLogger())
if err != nil {
panic(err)
}


+ 24
- 17
rpc/lib/server/handlers.go View File

@ -18,18 +18,19 @@ import (
types "github.com/tendermint/tendermint/rpc/lib/types"
cmn "github.com/tendermint/tmlibs/common"
events "github.com/tendermint/tmlibs/events"
"github.com/tendermint/tmlibs/log"
)
// Adds a route for each function in the funcMap, as well as general jsonrpc and websocket handlers for all functions.
// "result" is the interface on which the result objects are registered, and is popualted with every RPCResponse
func RegisterRPCFuncs(mux *http.ServeMux, funcMap map[string]*RPCFunc) {
func RegisterRPCFuncs(mux *http.ServeMux, funcMap map[string]*RPCFunc, logger log.Logger) {
// HTTP endpoints
for funcName, rpcFunc := range funcMap {
mux.HandleFunc("/"+funcName, makeHTTPHandler(rpcFunc))
mux.HandleFunc("/"+funcName, makeHTTPHandler(rpcFunc, logger))
}
// JSONRPC endpoints
mux.HandleFunc("/", makeJSONRPCHandler(funcMap))
mux.HandleFunc("/", makeJSONRPCHandler(funcMap, logger))
}
//-------------------------------------
@ -95,7 +96,7 @@ func funcReturnTypes(f interface{}) []reflect.Type {
// rpc.json
// jsonrpc calls grab the given method's function info and runs reflect.Call
func makeJSONRPCHandler(funcMap map[string]*RPCFunc) http.HandlerFunc {
func makeJSONRPCHandler(funcMap map[string]*RPCFunc, logger log.Logger) http.HandlerFunc {
return func(w http.ResponseWriter, r *http.Request) {
b, _ := ioutil.ReadAll(r.Body)
// if its an empty request (like from a browser),
@ -130,7 +131,7 @@ func makeJSONRPCHandler(funcMap map[string]*RPCFunc) http.HandlerFunc {
return
}
returns := rpcFunc.f.Call(args)
log.Info("HTTPJSONRPC", "method", request.Method, "args", args, "returns", returns)
logger.Info("HTTPJSONRPC", "method", request.Method, "args", args, "returns", returns)
result, err := unreflectResult(returns)
if err != nil {
WriteRPCResponseHTTPError(w, http.StatusInternalServerError, types.NewRPCResponse(request.ID, result, err.Error()))
@ -223,7 +224,7 @@ func jsonParamsToArgsWS(rpcFunc *RPCFunc, params *json.RawMessage, wsCtx types.W
// rpc.http
// convert from a function name to the http handler
func makeHTTPHandler(rpcFunc *RPCFunc) func(http.ResponseWriter, *http.Request) {
func makeHTTPHandler(rpcFunc *RPCFunc, logger log.Logger) func(http.ResponseWriter, *http.Request) {
// Exception for websocket endpoints
if rpcFunc.ws {
return func(w http.ResponseWriter, r *http.Request) {
@ -232,14 +233,14 @@ func makeHTTPHandler(rpcFunc *RPCFunc) func(http.ResponseWriter, *http.Request)
}
// All other endpoints
return func(w http.ResponseWriter, r *http.Request) {
log.Debug("HTTP HANDLER", "req", r)
logger.Debug("HTTP HANDLER", "req", r)
args, err := httpParamsToArgs(rpcFunc, r)
if err != nil {
WriteRPCResponseHTTPError(w, http.StatusBadRequest, types.NewRPCResponse("", nil, fmt.Sprintf("Error converting http params to args: %v", err.Error())))
return
}
returns := rpcFunc.f.Call(args)
log.Info("HTTPRestRPC", "method", r.URL.Path, "args", args, "returns", returns)
logger.Info("HTTPRestRPC", "method", r.URL.Path, "args", args, "returns", returns)
result, err := unreflectResult(returns)
if err != nil {
WriteRPCResponseHTTPError(w, http.StatusInternalServerError, types.NewRPCResponse("", nil, err.Error()))
@ -367,7 +368,7 @@ func NewWSConnection(baseConn *websocket.Conn, funcMap map[string]*RPCFunc, evsw
funcMap: funcMap,
evsw: evsw,
}
wsc.BaseService = *cmn.NewBaseService(log, "wsConnection", wsc)
wsc.BaseService = *cmn.NewBaseService(nil, "wsConnection", wsc)
return wsc
}
@ -417,7 +418,7 @@ func (wsc *wsConnection) OnStop() {
func (wsc *wsConnection) readTimeoutRoutine() {
select {
case <-wsc.readTimeout.C:
log.Notice("Stopping connection due to read timeout")
wsc.Logger.Info("Stopping connection due to read timeout")
wsc.Stop()
case <-wsc.Quit:
return
@ -476,7 +477,7 @@ func (wsc *wsConnection) readRoutine() {
// We use `readTimeout` to handle read timeouts.
_, in, err := wsc.baseConn.ReadMessage()
if err != nil {
log.Notice("Failed to read from connection", "remote", wsc.remoteAddr, "err", err.Error())
wsc.Logger.Info("Failed to read from connection", "remote", wsc.remoteAddr, "err", err.Error())
// an error reading the connection,
// kill the connection
wsc.Stop()
@ -509,7 +510,7 @@ func (wsc *wsConnection) readRoutine() {
continue
}
returns := rpcFunc.f.Call(args)
log.Info("WSJSONRPC", "method", request.Method, "args", args, "returns", returns)
wsc.Logger.Info("WSJSONRPC", "method", request.Method, "args", args, "returns", returns)
result, err := unreflectResult(returns)
if err != nil {
wsc.WriteRPCResponse(types.NewRPCResponse(request.ID, nil, err.Error()))
@ -533,18 +534,18 @@ func (wsc *wsConnection) writeRoutine() {
case <-wsc.pingTicker.C:
err := wsc.baseConn.WriteMessage(websocket.PingMessage, []byte{})
if err != nil {
log.Error("Failed to write ping message on websocket", "error", err)
wsc.Logger.Error("Failed to write ping message on websocket", "error", err)
wsc.Stop()
return
}
case msg := <-wsc.writeChan:
jsonBytes, err := json.Marshal(msg)
if err != nil {
log.Error("Failed to marshal RPCResponse to JSON", "error", err)
wsc.Logger.Error("Failed to marshal RPCResponse to JSON", "error", err)
} else {
wsc.baseConn.SetWriteDeadline(time.Now().Add(time.Second * wsWriteTimeoutSeconds))
if err = wsc.baseConn.WriteMessage(websocket.TextMessage, jsonBytes); err != nil {
log.Warn("Failed to write response on websocket", "error", err)
wsc.Logger.Error("Failed to write response on websocket", "error", err)
wsc.Stop()
return
}
@ -562,6 +563,7 @@ type WebsocketManager struct {
websocket.Upgrader
funcMap map[string]*RPCFunc
evsw events.EventSwitch
logger log.Logger
}
func NewWebsocketManager(funcMap map[string]*RPCFunc, evsw events.EventSwitch) *WebsocketManager {
@ -576,21 +578,26 @@ func NewWebsocketManager(funcMap map[string]*RPCFunc, evsw events.EventSwitch) *
return true
},
},
logger: log.NewNopLogger(),
}
}
func (wm *WebsocketManager) SetLogger(l log.Logger) {
wm.logger = l
}
// Upgrade the request/response (via http.Hijack) and starts the wsConnection.
func (wm *WebsocketManager) WebsocketHandler(w http.ResponseWriter, r *http.Request) {
wsConn, err := wm.Upgrade(w, r, nil)
if err != nil {
// TODO - return http error
log.Error("Failed to upgrade to websocket connection", "error", err)
wm.logger.Error("Failed to upgrade to websocket connection", "error", err)
return
}
// register connection
con := NewWSConnection(wsConn, wm.funcMap, wm.evsw)
log.Notice("New websocket connection", "remote", con.remoteAddr)
wm.logger.Info("New websocket connection", "remote", con.remoteAddr)
con.Start() // Blocking
}


+ 9
- 8
rpc/lib/server/http_server.go View File

@ -13,14 +13,15 @@ import (
"github.com/pkg/errors"
types "github.com/tendermint/tendermint/rpc/lib/types"
"github.com/tendermint/tmlibs/log"
)
func StartHTTPServer(listenAddr string, handler http.Handler) (listener net.Listener, err error) {
func StartHTTPServer(listenAddr string, handler http.Handler, logger log.Logger) (listener net.Listener, err error) {
// listenAddr should be fully formed including tcp:// or unix:// prefix
var proto, addr string
parts := strings.SplitN(listenAddr, "://", 2)
if len(parts) != 2 {
log.Warn("WARNING (tendermint/rpc/lib): Please use fully formed listening addresses, including the tcp:// or unix:// prefix")
logger.Error("WARNING (tendermint/rpc/lib): Please use fully formed listening addresses, including the tcp:// or unix:// prefix")
// we used to allow addrs without tcp/unix prefix by checking for a colon
// TODO: Deprecate
proto = types.SocketType(listenAddr)
@ -30,7 +31,7 @@ func StartHTTPServer(listenAddr string, handler http.Handler) (listener net.List
proto, addr = parts[0], parts[1]
}
log.Notice(fmt.Sprintf("Starting RPC HTTP server on %s socket %v", proto, addr))
logger.Info(fmt.Sprintf("Starting RPC HTTP server on %s socket %v", proto, addr))
listener, err = net.Listen(proto, addr)
if err != nil {
return nil, errors.Errorf("Failed to listen to %v: %v", listenAddr, err)
@ -39,9 +40,9 @@ func StartHTTPServer(listenAddr string, handler http.Handler) (listener net.List
go func() {
res := http.Serve(
listener,
RecoverAndLogHandler(handler),
RecoverAndLogHandler(handler, logger),
)
log.Crit("RPC HTTP server stopped", "result", res)
logger.Error("RPC HTTP server stopped", "result", res)
}()
return listener, nil
}
@ -72,7 +73,7 @@ func WriteRPCResponseHTTP(w http.ResponseWriter, res types.RPCResponse) {
// Wraps an HTTP handler, adding error logging.
// If the inner function panics, the outer function recovers, logs, sends an
// HTTP 500 error response.
func RecoverAndLogHandler(handler http.Handler) http.Handler {
func RecoverAndLogHandler(handler http.Handler, logger log.Logger) http.Handler {
return http.HandlerFunc(func(w http.ResponseWriter, r *http.Request) {
// Wrap the ResponseWriter to remember the status
rww := &ResponseWriterWrapper{-1, w}
@ -96,7 +97,7 @@ func RecoverAndLogHandler(handler http.Handler) http.Handler {
WriteRPCResponseHTTP(rww, res)
} else {
// For the rest,
log.Error("Panic in RPC HTTP handler", "error", e, "stack", string(debug.Stack()))
logger.Error("Panic in RPC HTTP handler", "error", e, "stack", string(debug.Stack()))
rww.WriteHeader(http.StatusInternalServerError)
WriteRPCResponseHTTP(rww, types.NewRPCResponse("", nil, fmt.Sprintf("Internal Server Error: %v", e)))
}
@ -107,7 +108,7 @@ func RecoverAndLogHandler(handler http.Handler) http.Handler {
if rww.Status == -1 {
rww.Status = 200
}
log.Info("Served RPC HTTP response",
logger.Info("Served RPC HTTP response",
"method", r.Method, "url", r.URL,
"status", rww.Status, "duration", durationMS,
"remoteAddr", r.RemoteAddr,


+ 0
- 7
rpc/lib/server/log.go View File

@ -1,7 +0,0 @@
package rpcserver
import (
"github.com/tendermint/log15"
)
var log = log15.New("module", "rpcserver")

+ 6
- 3
rpc/lib/test/main.go View File

@ -3,9 +3,11 @@ package main
import (
"fmt"
"net/http"
"os"
cmn "github.com/tendermint/tmlibs/common"
rpcserver "github.com/tendermint/tendermint/rpc/lib/server"
cmn "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tmlibs/log"
)
var routes = map[string]*rpcserver.RPCFunc{
@ -22,8 +24,9 @@ type Result struct {
func main() {
mux := http.NewServeMux()
rpcserver.RegisterRPCFuncs(mux, routes)
_, err := rpcserver.StartHTTPServer("0.0.0.0:8008", mux)
logger := log.NewTMLogger(log.NewSyncWriter(os.Stdout))
rpcserver.RegisterRPCFuncs(mux, routes, logger)
_, err := rpcserver.StartHTTPServer("0.0.0.0:8008", mux, logger)
if err != nil {
cmn.Exit(err.Error())
}


+ 5
- 8
rpc/test/helpers.go View File

@ -11,7 +11,7 @@ import (
"time"
"github.com/stretchr/testify/require"
logger "github.com/tendermint/tmlibs/logger"
"github.com/tendermint/tmlibs/log"
abci "github.com/tendermint/abci/types"
cfg "github.com/tendermint/tendermint/config"
@ -25,8 +25,6 @@ import (
var config *cfg.Config
const tmLogLevel = "error"
// f**ing long, but unique for each test
func makePathname() string {
// get path
@ -63,9 +61,6 @@ func GetConfig() *cfg.Config {
config.P2P.ListenAddress = tm
config.RPCListenAddress = rpc
config.GRPCListenAddress = grpc
// Shut up the logging
logger.SetLogLevel(tmLogLevel)
}
return config
}
@ -108,10 +103,12 @@ func StartTendermint(app abci.Application) *nm.Node {
func NewTendermint(app abci.Application) *nm.Node {
// Create & start node
config := GetConfig()
logger := log.NewTMLogger(log.NewSyncWriter(os.Stdout))
logger = log.NewFilter(logger, log.AllowError())
privValidatorFile := config.PrivValidatorFile()
privValidator := types.LoadOrGenPrivValidator(privValidatorFile)
privValidator := types.LoadOrGenPrivValidator(privValidatorFile, logger)
papp := proxy.NewLocalClientCreator(app)
node := nm.NewNode(config, privValidator, papp)
node := nm.NewNode(config, privValidator, papp, logger)
return node
}


+ 16
- 15
state/execution.go View File

@ -11,6 +11,7 @@ import (
"github.com/tendermint/tendermint/state/txindex"
"github.com/tendermint/tendermint/types"
cmn "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tmlibs/log"
)
//--------------------------------------------------
@ -26,7 +27,7 @@ func (s *State) ValExecBlock(eventCache types.Fireable, proxyAppConn proxy.AppCo
}
// Execute the block txs
abciResponses, err := execBlockOnProxyApp(eventCache, proxyAppConn, block)
abciResponses, err := execBlockOnProxyApp(eventCache, proxyAppConn, block, s.logger)
if err != nil {
// There was some error in proxyApp
// TODO Report error and wait for proxyApp to be available.
@ -39,7 +40,7 @@ func (s *State) ValExecBlock(eventCache types.Fireable, proxyAppConn proxy.AppCo
// Executes block's transactions on proxyAppConn.
// Returns a list of transaction results and updates to the validator set
// TODO: Generate a bitmap or otherwise store tx validity in state.
func execBlockOnProxyApp(eventCache types.Fireable, proxyAppConn proxy.AppConnConsensus, block *types.Block) (*ABCIResponses, error) {
func execBlockOnProxyApp(eventCache types.Fireable, proxyAppConn proxy.AppConnConsensus, block *types.Block, logger log.Logger) (*ABCIResponses, error) {
var validTxs, invalidTxs = 0, 0
txIndex := 0
@ -58,7 +59,7 @@ func execBlockOnProxyApp(eventCache types.Fireable, proxyAppConn proxy.AppConnCo
if txResult.Code == abci.CodeType_OK {
validTxs++
} else {
log.Debug("Invalid tx", "code", txResult.Code, "log", txResult.Log)
logger.Debug("Invalid tx", "code", txResult.Code, "log", txResult.Log)
invalidTxs++
txError = txResult.Code.String()
}
@ -84,7 +85,7 @@ func execBlockOnProxyApp(eventCache types.Fireable, proxyAppConn proxy.AppConnCo
// Begin block
err := proxyAppConn.BeginBlockSync(block.Hash(), types.TM2PB.Header(block.Header))
if err != nil {
log.Warn("Error in proxyAppConn.BeginBlock", "error", err)
logger.Error("Error in proxyAppConn.BeginBlock", "error", err)
return nil, err
}
@ -99,15 +100,15 @@ func execBlockOnProxyApp(eventCache types.Fireable, proxyAppConn proxy.AppConnCo
// End block
abciResponses.EndBlock, err = proxyAppConn.EndBlockSync(uint64(block.Height))
if err != nil {
log.Warn("Error in proxyAppConn.EndBlock", "error", err)
logger.Error("Error in proxyAppConn.EndBlock", "error", err)
return nil, err
}
valDiff := abciResponses.EndBlock.Diffs
log.Info("Executed block", "height", block.Height, "valid txs", validTxs, "invalid txs", invalidTxs)
logger.Info("Executed block", "height", block.Height, "valid txs", validTxs, "invalid txs", invalidTxs)
if len(valDiff) > 0 {
log.Info("Update to validator set", "updates", abci.ValidatorsString(valDiff))
logger.Info("Update to validator set", "updates", abci.ValidatorsString(valDiff))
}
return abciResponses, nil
@ -251,14 +252,14 @@ func (s *State) CommitStateUpdateMempool(proxyAppConn proxy.AppConnConsensus, bl
// Commit block, get hash back
res := proxyAppConn.CommitSync()
if res.IsErr() {
log.Warn("Error in proxyAppConn.CommitSync", "error", res)
s.logger.Error("Error in proxyAppConn.CommitSync", "error", res)
return res
}
if res.Log != "" {
log.Debug("Commit.Log: " + res.Log)
s.logger.Debug("Commit.Log: " + res.Log)
}
log.Info("Committed state", "hash", res.Data)
s.logger.Info("Committed state", "hash", fmt.Sprintf("%X", res.Data))
// Set the state's new AppHash
s.AppHash = res.Data
@ -286,21 +287,21 @@ func (s *State) indexTxs(abciResponses *ABCIResponses) {
// Exec and commit a block on the proxyApp without validating or mutating the state
// Returns the application root hash (result of abci.Commit)
func ExecCommitBlock(appConnConsensus proxy.AppConnConsensus, block *types.Block) ([]byte, error) {
func ExecCommitBlock(appConnConsensus proxy.AppConnConsensus, block *types.Block, logger log.Logger) ([]byte, error) {
var eventCache types.Fireable // nil
_, err := execBlockOnProxyApp(eventCache, appConnConsensus, block)
_, err := execBlockOnProxyApp(eventCache, appConnConsensus, block, logger)
if err != nil {
log.Warn("Error executing block on proxy app", "height", block.Height, "err", err)
logger.Error("Error executing block on proxy app", "height", block.Height, "err", err)
return nil, err
}
// Commit block, get hash back
res := appConnConsensus.CommitSync()
if res.IsErr() {
log.Warn("Error in proxyAppConn.CommitSync", "error", res)
logger.Error("Error in proxyAppConn.CommitSync", "error", res)
return nil, res
}
if res.Log != "" {
log.Info("Commit.Log: " + res.Log)
logger.Info("Commit.Log: " + res.Log)
}
return res.Data, nil
}

+ 2
- 0
state/execution_test.go View File

@ -11,6 +11,7 @@ import (
"github.com/tendermint/tendermint/state/txindex"
"github.com/tendermint/tendermint/types"
dbm "github.com/tendermint/tmlibs/db"
"github.com/tendermint/tmlibs/log"
)
var (
@ -28,6 +29,7 @@ func TestApplyBlock(t *testing.T) {
defer proxyApp.Stop()
state := state()
state.SetLogger(log.TestingLogger())
indexer := &dummyIndexer{0}
state.TxIndexer = indexer


+ 0
- 7
state/log.go View File

@ -1,7 +0,0 @@
package state
import (
"github.com/tendermint/tmlibs/logger"
)
var log = logger.New("module", "state")

+ 14
- 2
state/state.go View File

@ -9,8 +9,9 @@ import (
abci "github.com/tendermint/abci/types"
cmn "github.com/tendermint/tmlibs/common"
dbm "github.com/tendermint/tmlibs/db"
"github.com/tendermint/tmlibs/log"
"github.com/tendermint/go-wire"
wire "github.com/tendermint/go-wire"
"github.com/tendermint/tendermint/state/txindex"
"github.com/tendermint/tendermint/state/txindex/null"
"github.com/tendermint/tendermint/types"
@ -48,6 +49,8 @@ type State struct {
// Intermediate results from processing
// Persisted separately from the state
abciResponses *ABCIResponses
logger log.Logger
}
func LoadState(db dbm.DB) *State {
@ -71,6 +74,14 @@ func loadState(db dbm.DB, key []byte) *State {
return s
}
func (s *State) SetLogger(l log.Logger) {
s.logger = l
}
func (s *State) GetLogger() log.Logger {
return s.logger
}
func (s *State) Copy() *State {
return &State{
db: s.db,
@ -83,6 +94,7 @@ func (s *State) Copy() *State {
LastValidators: s.LastValidators.Copy(),
AppHash: s.AppHash,
TxIndexer: s.TxIndexer, // pointer here, not value
logger: s.logger,
}
}
@ -140,7 +152,7 @@ func (s *State) SetBlockAndValidators(header *types.Header, blockPartsHeader typ
// update the validator set with the latest abciResponses
err := updateValidators(nextValSet, abciResponses.EndBlock.Diffs)
if err != nil {
log.Warn("Error changing validator set", "error", err)
s.logger.Error("Error changing validator set", "error", err)
// TODO: err or carry on?
}
// Update validator accums and set state variables


+ 5
- 1
state/state_test.go View File

@ -6,9 +6,10 @@ import (
"github.com/stretchr/testify/assert"
abci "github.com/tendermint/abci/types"
"github.com/tendermint/go-crypto"
crypto "github.com/tendermint/go-crypto"
cfg "github.com/tendermint/tendermint/config"
dbm "github.com/tendermint/tmlibs/db"
"github.com/tendermint/tmlibs/log"
)
func TestStateCopyEquals(t *testing.T) {
@ -17,6 +18,7 @@ func TestStateCopyEquals(t *testing.T) {
// Get State db
stateDB := dbm.NewDB("state", config.DBBackend, config.DBDir())
state := GetState(stateDB, config.GenesisFile())
state.SetLogger(log.TestingLogger())
stateCopy := state.Copy()
@ -36,6 +38,7 @@ func TestStateSaveLoad(t *testing.T) {
// Get State db
stateDB := dbm.NewDB("state", config.DBBackend, config.DBDir())
state := GetState(stateDB, config.GenesisFile())
state.SetLogger(log.TestingLogger())
state.LastBlockHeight += 1
state.Save()
@ -52,6 +55,7 @@ func TestABCIResponsesSaveLoad(t *testing.T) {
config := cfg.ResetTestRoot("state_")
stateDB := dbm.NewDB("state", config.DBBackend, config.DBDir())
state := GetState(stateDB, config.GenesisFile())
state.SetLogger(log.TestingLogger())
state.LastBlockHeight += 1


+ 0
- 7
types/log.go View File

@ -1,7 +0,0 @@
package types
import (
"github.com/tendermint/tmlibs/logger"
)
var log = logger.New("module", "types")

+ 7
- 6
types/priv_validator.go View File

@ -9,9 +9,10 @@ import (
"os"
"sync"
. "github.com/tendermint/tmlibs/common"
"github.com/tendermint/go-crypto"
crypto "github.com/tendermint/go-crypto"
data "github.com/tendermint/go-wire/data"
. "github.com/tendermint/tmlibs/common"
"github.com/tendermint/tmlibs/log"
)
const (
@ -107,17 +108,17 @@ func LoadPrivValidator(filePath string) *PrivValidator {
return &privVal
}
func LoadOrGenPrivValidator(filePath string) *PrivValidator {
func LoadOrGenPrivValidator(filePath string, logger log.Logger) *PrivValidator {
var privValidator *PrivValidator
if _, err := os.Stat(filePath); err == nil {
privValidator = LoadPrivValidator(filePath)
log.Notice("Loaded PrivValidator",
logger.Info("Loaded PrivValidator",
"file", filePath, "privValidator", privValidator)
} else {
privValidator = GenPrivValidator()
privValidator.SetFile(filePath)
privValidator.Save()
log.Notice("Generated PrivValidator", "file", filePath)
logger.Info("Generated PrivValidator", "file", filePath)
}
return privValidator
}
@ -212,7 +213,7 @@ func (privVal *PrivValidator) signBytesHRS(height, round int, step int8, signByt
// NOTE: proposals are non-deterministic (include time),
// so we can actually lose them, but will still never sign conflicting ones
if bytes.Equal(privVal.LastSignBytes, signBytes) {
log.Notice("Using privVal.LastSignature", "sig", privVal.LastSignature)
// log.Notice("Using privVal.LastSignature", "sig", privVal.LastSignature)
return privVal.LastSignature, nil
}
}


Loading…
Cancel
Save