logging: implement lazy sprinting (backport #8898) (#9109)

This commit is contained in:
mergify[bot]
2022-07-29 09:20:40 -04:00
committed by GitHub
parent b4eaccd242
commit 4b109a147d
18 changed files with 473 additions and 138 deletions

View File

@@ -101,7 +101,7 @@ RETRY_LOOP:
if cli.mustConnect {
return err
}
cli.Logger.Error(fmt.Sprintf("abci.grpcClient failed to connect to %v. Retrying...\n", cli.addr), "err", err)
cli.Logger.Error("abci.grpcClient failed to connect, Retrying...", "addr", cli.addr, "err", err)
time.Sleep(time.Second * dialRetryIntervalSeconds)
continue RETRY_LOOP
}

View File

@@ -77,8 +77,10 @@ func (cli *socketClient) OnStart() error {
if cli.mustConnect {
return err
}
cli.Logger.Error(fmt.Sprintf("abci.socketClient failed to connect to %v. Retrying after %vs...",
cli.addr, dialRetryIntervalSeconds), "err", err)
cli.Logger.Error("abci.socketClient failed to connect, retrying after",
"retry_after", dialRetryIntervalSeconds,
"target", cli.addr,
"err", err)
time.Sleep(time.Second * dialRetryIntervalSeconds)
continue
}

View File

@@ -7,6 +7,7 @@ import (
"time"
cstypes "github.com/tendermint/tendermint/internal/consensus/types"
tmstrings "github.com/tendermint/tendermint/internal/libs/strings"
"github.com/tendermint/tendermint/internal/p2p"
sm "github.com/tendermint/tendermint/internal/state"
"github.com/tendermint/tendermint/libs/bits"
@@ -1143,7 +1144,7 @@ func (r *Reactor) handleDataMessage(envelope p2p.Envelope, msgI Message) error {
}
if r.WaitSync() {
logger.Info("ignoring message received during sync", "msg", fmt.Sprintf("%T", msgI))
logger.Info("ignoring message received during sync", "msg", tmstrings.LazySprintf("%T", msgI))
return nil
}

View File

@@ -17,6 +17,7 @@ import (
"github.com/tendermint/tendermint/crypto"
cstypes "github.com/tendermint/tendermint/internal/consensus/types"
"github.com/tendermint/tendermint/internal/libs/fail"
tmstrings "github.com/tendermint/tendermint/internal/libs/strings"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
sm "github.com/tendermint/tendermint/internal/state"
tmevents "github.com/tendermint/tendermint/libs/events"
@@ -666,11 +667,9 @@ func (cs *State) updateToState(state sm.State) {
// signal the new round step, because other services (eg. txNotifier)
// depend on having an up-to-date peer state!
if state.LastBlockHeight <= cs.state.LastBlockHeight {
cs.Logger.Debug(
"ignoring updateToState()",
cs.Logger.Debug("ignoring updateToState()",
"new_height", state.LastBlockHeight+1,
"old_height", cs.state.LastBlockHeight+1,
)
"old_height", cs.state.LastBlockHeight+1)
cs.newStep()
return
}
@@ -903,12 +902,10 @@ func (cs *State) handleMsg(mi msgInfo) {
}
if err != nil && msg.Round != cs.Round {
cs.Logger.Debug(
"received block part from wrong round",
cs.Logger.Debug("received block part from wrong round",
"height", cs.Height,
"cs_round", cs.Round,
"block_round", msg.Round,
)
"block_round", msg.Round)
err = nil
}
@@ -936,7 +933,7 @@ func (cs *State) handleMsg(mi msgInfo) {
// We could make note of this and help filter in broadcastHasVoteMessage().
default:
cs.Logger.Error("unknown msg type", "type", fmt.Sprintf("%T", msg))
cs.Logger.Error("unknown msg type", "type", tmstrings.LazySprintf("%T", msg))
return
}
@@ -1041,10 +1038,10 @@ func (cs *State) enterNewRound(height int64, round int32) {
logger := cs.Logger.With("height", height, "round", round)
if cs.Height != height || round < cs.Round || (cs.Round == round && cs.Step != cstypes.RoundStepNewHeight) {
logger.Debug(
"entering new round with invalid args",
"current", fmt.Sprintf("%v/%v/%v", cs.Height, cs.Round, cs.Step),
)
logger.Debug("entering new round with invalid args",
"height", cs.Height,
"round", cs.Round,
"step", cs.Step)
return
}
@@ -1052,7 +1049,10 @@ func (cs *State) enterNewRound(height int64, round int32) {
logger.Debug("need to set a buffer and log message here for sanity", "start_time", cs.StartTime, "now", now)
}
logger.Debug("entering new round", "current", fmt.Sprintf("%v/%v/%v", cs.Height, cs.Round, cs.Step))
logger.Debug("entering new round",
"height", cs.Height,
"round", cs.Round,
"step", cs.Step)
// increment validators if necessary
validators := cs.Validators
@@ -1120,14 +1120,17 @@ func (cs *State) enterPropose(height int64, round int32) {
logger := cs.Logger.With("height", height, "round", round)
if cs.Height != height || round < cs.Round || (cs.Round == round && cstypes.RoundStepPropose <= cs.Step) {
logger.Debug(
"entering propose step with invalid args",
"current", fmt.Sprintf("%v/%v/%v", cs.Height, cs.Round, cs.Step),
)
logger.Debug("entering propose step with invalid args",
"height", cs.Height,
"round", cs.Round,
"step", cs.Step)
return
}
logger.Debug("entering propose step", "current", fmt.Sprintf("%v/%v/%v", cs.Height, cs.Round, cs.Step))
logger.Debug("entering propose step",
"height", cs.Height,
"round", cs.Round,
"step", cs.Step)
defer func() {
// Done enterPropose:
@@ -1169,17 +1172,13 @@ func (cs *State) enterPropose(height int64, round int32) {
}
if cs.isProposer(address) {
logger.Debug(
"propose step; our turn to propose",
"proposer", address,
)
logger.Debug("propose step; our turn to propose",
"proposer", address)
cs.decideProposal(height, round)
} else {
logger.Debug(
"propose step; not our turn to propose",
"proposer", cs.Validators.GetProposer().Address,
)
logger.Debug("propose step; not our turn to propose",
"proposer", cs.Validators.GetProposer().Address)
}
}
@@ -1298,10 +1297,10 @@ func (cs *State) enterPrevote(height int64, round int32) {
logger := cs.Logger.With("height", height, "round", round)
if cs.Height != height || round < cs.Round || (cs.Round == round && cstypes.RoundStepPrevote <= cs.Step) {
logger.Debug(
"entering prevote step with invalid args",
"current", fmt.Sprintf("%v/%v/%v", cs.Height, cs.Round, cs.Step),
)
logger.Debug("entering prevote step with invalid args",
"height", cs.Height,
"round", cs.Round,
"step", cs.Step)
return
}
@@ -1311,7 +1310,10 @@ func (cs *State) enterPrevote(height int64, round int32) {
cs.newStep()
}()
logger.Debug("entering prevote step", "current", fmt.Sprintf("%v/%v/%v", cs.Height, cs.Round, cs.Step))
logger.Debug("entering prevote step",
"height", cs.Height,
"round", cs.Round,
"step", cs.Step)
// Sign and broadcast vote as necessary
cs.doPrevote(height, round)
@@ -1358,10 +1360,10 @@ func (cs *State) enterPrevoteWait(height int64, round int32) {
logger := cs.Logger.With("height", height, "round", round)
if cs.Height != height || round < cs.Round || (cs.Round == round && cstypes.RoundStepPrevoteWait <= cs.Step) {
logger.Debug(
"entering prevote wait step with invalid args",
"current", fmt.Sprintf("%v/%v/%v", cs.Height, cs.Round, cs.Step),
)
logger.Debug("entering prevote wait step with invalid args",
"height", cs.Height,
"round", cs.Round,
"step", cs.Step)
return
}
@@ -1372,7 +1374,10 @@ func (cs *State) enterPrevoteWait(height int64, round int32) {
))
}
logger.Debug("entering prevote wait step", "current", fmt.Sprintf("%v/%v/%v", cs.Height, cs.Round, cs.Step))
logger.Debug("entering prevote wait step",
"height", cs.Height,
"round", cs.Round,
"step", cs.Step)
defer func() {
// Done enterPrevoteWait:
@@ -1394,14 +1399,18 @@ func (cs *State) enterPrecommit(height int64, round int32) {
logger := cs.Logger.With("height", height, "round", round)
if cs.Height != height || round < cs.Round || (cs.Round == round && cstypes.RoundStepPrecommit <= cs.Step) {
logger.Debug(
"entering precommit step with invalid args",
"current", fmt.Sprintf("%v/%v/%v", cs.Height, cs.Round, cs.Step),
)
logger.Debug("entering precommit step with invalid args",
"height", cs.Height,
"round", cs.Round,
"step", cs.Step)
return
}
logger.Debug("entering precommit step", "current", fmt.Sprintf("%v/%v/%v", cs.Height, cs.Round, cs.Step))
logger.Debug("entering precommit step",
"height", cs.Height,
"round", cs.Round,
"step", cs.Step)
defer func() {
// Done enterPrecommit:
@@ -1517,11 +1526,10 @@ func (cs *State) enterPrecommitWait(height int64, round int32) {
logger := cs.Logger.With("height", height, "round", round)
if cs.Height != height || round < cs.Round || (cs.Round == round && cs.TriggeredTimeoutPrecommit) {
logger.Debug(
"entering precommit wait step with invalid args",
logger.Debug("entering precommit wait step with invalid args",
"triggered_timeout", cs.TriggeredTimeoutPrecommit,
"current", fmt.Sprintf("%v/%v", cs.Height, cs.Round),
)
"height", cs.Height,
"round", cs.Round)
return
}
@@ -1532,7 +1540,10 @@ func (cs *State) enterPrecommitWait(height int64, round int32) {
))
}
logger.Debug("entering precommit wait step", "current", fmt.Sprintf("%v/%v/%v", cs.Height, cs.Round, cs.Step))
logger.Debug("entering precommit wait step",
"height", cs.Height,
"round", cs.Round,
"step", cs.Step)
defer func() {
// Done enterPrecommitWait:
@@ -1549,14 +1560,17 @@ func (cs *State) enterCommit(height int64, commitRound int32) {
logger := cs.Logger.With("height", height, "commit_round", commitRound)
if cs.Height != height || cstypes.RoundStepCommit <= cs.Step {
logger.Debug(
"entering commit step with invalid args",
"current", fmt.Sprintf("%v/%v/%v", cs.Height, cs.Round, cs.Step),
)
logger.Debug("entering commit step with invalid args",
"height", cs.Height,
"round", cs.Round,
"step", cs.Step)
return
}
logger.Debug("entering commit step", "current", fmt.Sprintf("%v/%v/%v", cs.Height, cs.Round, cs.Step))
logger.Debug("entering commit step",
"height", cs.Height,
"round", cs.Round,
"step", cs.Step)
defer func() {
// Done enterCommit:
@@ -1625,9 +1639,8 @@ func (cs *State) tryFinalizeCommit(height int64) {
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?
logger.Debug(
"failed attempt to finalize commit; we do not have the commit block",
"proposal_block", cs.ProposalBlock.Hash(),
logger.Debug("failed attempt to finalize commit; we do not have the commit block",
"proposal_block", tmstrings.LazyBlockHash(cs.ProposalBlock),
"commit_block", blockID.Hash,
)
return
@@ -1669,11 +1682,10 @@ func (cs *State) finalizeCommit(height int64) {
logger.Info(
"finalizing commit of block",
"hash", block.Hash(),
"hash", tmstrings.LazyBlockHash(block),
"root", block.AppHash,
"num_txs", len(block.Txs),
)
logger.Debug(fmt.Sprintf("%v", block))
fail.Fail() // XXX
@@ -1777,8 +1789,11 @@ func (cs *State) RecordMetrics(height int64, block *types.Block) {
address types.Address
)
if commitSize != valSetLen {
cs.Logger.Error(fmt.Sprintf("commit size (%d) doesn't match valset length (%d) at height %d\n\n%v\n\n%v",
commitSize, valSetLen, block.Height, block.LastCommit.Signatures, cs.LastValidators.Validators))
cs.Logger.Error("commit size doesn't match valset",
"size", commitSize,
"valset_len", valSetLen,
"height", block.Height,
"extra", tmstrings.LazySprintf("\n%v\n\n%v", block.LastCommit.Signatures, cs.LastValidators.Validators))
return
}
@@ -1907,8 +1922,7 @@ func (cs *State) addProposalBlockPart(msg *BlockPartMessage, peerID types.NodeID
cs.metrics.BlockGossipPartsReceived.With("matches_current", "false").Add(1)
// NOTE: this can happen when we've gone to a higher round and
// then receive parts from the previous round - not necessarily a bad peer.
cs.Logger.Debug(
"received a block part when we are not expecting any",
cs.Logger.Debug("received a block part when we are not expecting any",
"height", height,
"round", round,
"index", part.Index,
@@ -1968,11 +1982,9 @@ func (cs *State) handleCompleteProposal(blockHeight int64) {
blockID, hasTwoThirds := prevotes.TwoThirdsMajority()
if hasTwoThirds && !blockID.IsZero() && (cs.ValidRound < cs.Round) {
if cs.ProposalBlock.HashesTo(blockID.Hash) {
cs.Logger.Debug(
"updating valid block to new proposal block",
cs.Logger.Debug("updating valid block to new proposal block",
"valid_round", cs.Round,
"valid_block_hash", cs.ProposalBlock.Hash(),
)
"valid_block_hash", tmstrings.LazyBlockHash(cs.ProposalBlock))
cs.ValidRound = cs.Round
cs.ValidBlock = cs.ProposalBlock
@@ -2010,23 +2022,19 @@ func (cs *State) tryAddVote(vote *types.Vote, peerID types.NodeID) (bool, error)
}
if bytes.Equal(vote.ValidatorAddress, cs.privValidatorPubKey.Address()) {
cs.Logger.Error(
"found conflicting vote from ourselves; did you unsafe_reset a validator?",
cs.Logger.Error("found conflicting vote from ourselves; did you unsafe_reset a validator?",
"height", vote.Height,
"round", vote.Round,
"type", vote.Type,
)
"type", vote.Type)
return added, err
}
// report conflicting votes to the evidence pool
cs.evpool.ReportConflictingVotes(voteErr.VoteA, voteErr.VoteB)
cs.Logger.Debug(
"found and sent conflicting votes to the evidence pool",
cs.Logger.Debug("found and sent conflicting votes to the evidence pool",
"vote_a", voteErr.VoteA,
"vote_b", voteErr.VoteB,
)
"vote_b", voteErr.VoteB)
return added, err
} else if errors.Is(err, types.ErrVoteNonDeterministicSignature) {
@@ -2046,13 +2054,11 @@ func (cs *State) tryAddVote(vote *types.Vote, peerID types.NodeID) (bool, error)
}
func (cs *State) addVote(vote *types.Vote, peerID types.NodeID) (added bool, err error) {
cs.Logger.Debug(
"adding vote",
cs.Logger.Debug("adding vote",
"vote_height", vote.Height,
"vote_type", vote.Type,
"val_index", vote.ValidatorIndex,
"cs_height", cs.Height,
)
"cs_height", cs.Height)
// A precommit for the previous height?
// These come in while we wait timeoutCommit
@@ -2142,11 +2148,9 @@ func (cs *State) addVote(vote *types.Vote, peerID types.NodeID) (added bool, err
cs.ValidBlock = cs.ProposalBlock
cs.ValidBlockParts = cs.ProposalBlockParts
} else {
cs.Logger.Debug(
"valid block we do not know about; set ProposalBlock=nil",
"proposal", cs.ProposalBlock.Hash(),
"block_id", blockID.Hash,
)
cs.Logger.Debug("valid block we do not know about; set ProposalBlock=nil",
"proposal", tmstrings.LazyBlockHash(cs.ProposalBlock),
"block_id", blockID.Hash)
// we're getting the wrong block
cs.ProposalBlock = nil

View File

@@ -0,0 +1,148 @@
package strings
import (
"fmt"
"strings"
tmbytes "github.com/tendermint/tendermint/libs/bytes"
)
type lazyStringf struct {
tmpl string
args []interface{}
out string
}
func (s *lazyStringf) String() string {
if s.out == "" && s.tmpl != "" {
s.out = fmt.Sprintf(s.tmpl, s.args)
s.args = nil
s.tmpl = ""
}
return s.out
}
// LazySprintf creates a fmt.Stringer implementation with similar
// semantics as fmt.Sprintf, *except* that the string is built when
// String() is called on the object. This means that format arguments
// are resolved/captured into string format when String() is called,
// and not, as in fmt.Sprintf when that function returns.
//
// As a result, if you use this type in go routines or defer
// statements it's possible to pass an argument to LazySprintf which
// has one value at the call site and a different value when the
// String() is evaluated, which may lead to unexpected outcomes. In
// these situations, either be *extremely* careful about the arguments
// passed to this function or use fmt.Sprintf.
//
// The implementation also caches the output of the underlying
// fmt.Sprintf statement when String() is called, so subsequent calls
// will produce the same result.
func LazySprintf(t string, args ...interface{}) fmt.Stringer {
return &lazyStringf{tmpl: t, args: args}
}
type lazyStringer struct {
val fmt.Stringer
out string
}
func (l *lazyStringer) String() string {
if l.out == "" && l.val != nil {
l.out = l.val.String()
l.val = nil
}
return l.out
}
// LazyStringer captures a fmt.Stringer implementation resolving the
// underlying string *only* when the String() method is called and
// caching the result for future use.
func LazyStringer(v fmt.Stringer) fmt.Stringer { return &lazyStringer{val: v} }
type lazyBlockHash struct {
block interface{ Hash() tmbytes.HexBytes }
out string
}
// LazyBlockHash defers block Hash until the Stringer interface is invoked.
// This is particularly useful for avoiding calling Sprintf when debugging is not
// active.
//
// As a result, if you use this type in go routines or defer
// statements it's possible to pass an argument to LazyBlockHash that
// has one value at the call site and a different value when the
// String() is evaluated, which may lead to unexpected outcomes. In
// these situations, either be *extremely* careful about the arguments
// passed to this function or use fmt.Sprintf.
//
// The implementation also caches the output of the string form of the
// block hash when String() is called, so subsequent calls will
// produce the same result.
func LazyBlockHash(block interface{ Hash() tmbytes.HexBytes }) fmt.Stringer {
return &lazyBlockHash{block: block}
}
func (l *lazyBlockHash) String() string {
if l.out == "" && l.block != nil {
l.out = l.block.Hash().String()
l.block = nil
}
return l.out
}
// SplitAndTrimEmpty slices s into all subslices separated by sep and returns a
// slice of the string s with all leading and trailing Unicode code points
// contained in cutset removed. If sep is empty, SplitAndTrim splits after each
// UTF-8 sequence. First part is equivalent to strings.SplitN with a count of
// -1. also filter out empty strings, only return non-empty strings.
func SplitAndTrimEmpty(s, sep, cutset string) []string {
if s == "" {
return []string{}
}
spl := strings.Split(s, sep)
nonEmptyStrings := make([]string, 0, len(spl))
for i := 0; i < len(spl); i++ {
element := strings.Trim(spl[i], cutset)
if element != "" {
nonEmptyStrings = append(nonEmptyStrings, element)
}
}
return nonEmptyStrings
}
// ASCIITrim removes spaces from an a ASCII string, erroring if the
// sequence is not an ASCII string.
func ASCIITrim(s string) (string, error) {
if len(s) == 0 {
return "", nil
}
r := make([]byte, 0, len(s))
for _, b := range []byte(s) {
switch {
case b == 32:
continue // skip space
case 32 < b && b <= 126:
r = append(r, b)
default:
return "", fmt.Errorf("non-ASCII (non-tab) char 0x%X", b)
}
}
return string(r), nil
}
// StringSliceEqual checks if string slices a and b are equal
func StringSliceEqual(a, b []string) bool {
if len(a) != len(b) {
return false
}
for i := 0; i < len(a); i++ {
if a[i] != b[i] {
return false
}
}
return true
}

View File

@@ -0,0 +1,89 @@
package strings
import (
"testing"
"github.com/stretchr/testify/require"
)
func TestSplitAndTrimEmpty(t *testing.T) {
testCases := []struct {
s string
sep string
cutset string
expected []string
}{
{"a,b,c", ",", " ", []string{"a", "b", "c"}},
{" a , b , c ", ",", " ", []string{"a", "b", "c"}},
{" a, b, c ", ",", " ", []string{"a", "b", "c"}},
{" a, ", ",", " ", []string{"a"}},
{" ", ",", " ", []string{}},
}
for _, tc := range testCases {
require.Equal(t, tc.expected, SplitAndTrimEmpty(tc.s, tc.sep, tc.cutset), "%s", tc.s)
}
}
func assertCorrectTrim(t *testing.T, input, expected string) {
t.Helper()
output, err := ASCIITrim(input)
require.NoError(t, err)
require.Equal(t, expected, output)
}
func TestASCIITrim(t *testing.T) {
t.Run("Validation", func(t *testing.T) {
t.Run("NonASCII", func(t *testing.T) {
notASCIIText := []string{
"\xC2", "\xC2\xA2", "\xFF", "\x80", "\xF0", "\n", "\t",
}
for _, v := range notASCIIText {
_, err := ASCIITrim(v)
require.Error(t, err, "%q is not ascii-text", v)
}
})
t.Run("EmptyString", func(t *testing.T) {
out, err := ASCIITrim("")
require.NoError(t, err)
require.Zero(t, out)
})
t.Run("ASCIIText", func(t *testing.T) {
asciiText := []string{
" ", ".", "x", "$", "_", "abcdefg;", "-", "0x00", "0", "123",
}
for _, v := range asciiText {
_, err := ASCIITrim(v)
require.NoError(t, err, "%q is ascii-text", v)
}
})
_, err := ASCIITrim("\xC2\xA2")
require.Error(t, err)
})
t.Run("Trimming", func(t *testing.T) {
assertCorrectTrim(t, " ", "")
assertCorrectTrim(t, " a", "a")
assertCorrectTrim(t, "a ", "a")
assertCorrectTrim(t, " a ", "a")
})
}
func TestStringSliceEqual(t *testing.T) {
tests := []struct {
a []string
b []string
want bool
}{
{[]string{"hello", "world"}, []string{"hello", "world"}, true},
{[]string{"test"}, []string{"test"}, true},
{[]string{"test1"}, []string{"test2"}, false},
{[]string{"hello", "world."}, []string{"hello", "world!"}, false},
{[]string{"only 1 word"}, []string{"two", "words!"}, false},
{[]string{"two", "words!"}, []string{"only 1 word"}, false},
}
for i, tt := range tests {
require.Equal(t, tt.want, StringSliceEqual(tt.a, tt.b),
"StringSliceEqual failed on test %d", i)
}
}

View File

@@ -9,6 +9,7 @@ import (
"github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/internal/libs/clist"
tmstrings "github.com/tendermint/tendermint/internal/libs/strings"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/internal/mempool"
"github.com/tendermint/tendermint/internal/p2p"
@@ -370,9 +371,8 @@ func (r *Reactor) broadcastTxRoutine(peerID types.NodeID, closer *tmsync.Closer)
Txs: [][]byte{memTx.tx},
},
}
r.Logger.Debug(
"gossiped tx to peer",
"tx", fmt.Sprintf("%X", memTx.tx.Hash()),
r.Logger.Debug("gossiped tx to peer",
"tx", tmstrings.LazySprintf("%X", memTx.tx.Hash()),
"peer", peerID,
)
}

View File

@@ -369,7 +369,7 @@ func (c *MConnection) Send(chID byte, msgBytes []byte) bool {
// Send message to channel.
channel, ok := c.channelsIdx[chID]
if !ok {
c.Logger.Error(fmt.Sprintf("Cannot send bytes, unknown channel %X", chID))
c.Logger.Error("Cannot send bytes to unknown channel", "channel", chID)
return false
}

View File

@@ -466,7 +466,8 @@ func (r *ReactorV2) calculateNextRequestTime(added int) {
// If the peer store is nearly full, wait the maximum interval.
if ratio := r.peerManager.PeerRatio(); ratio >= 0.95 {
r.Logger.Debug("Peer manager is nearly full",
"sleep_period", fullCapacityInterval, "ratio", ratio)
"sleep_period", fullCapacityInterval,
"ratio", ratio)
r.nextRequestInterval = fullCapacityInterval
return
}

View File

@@ -213,13 +213,11 @@ func (s *pqScheduler) process() {
} else {
pqEnvTmpChIDStr := strconv.Itoa(int(pqEnvTmp.envelope.channelID))
s.metrics.PeerQueueDroppedMsgs.With("ch_id", pqEnvTmpChIDStr).Add(1)
s.logger.Debug(
"dropped envelope",
s.logger.Debug("dropped envelope",
"ch_id", pqEnvTmpChIDStr,
"priority", pqEnvTmp.priority,
"msg_size", pqEnvTmp.size,
"capacity", s.capacity,
)
"capacity", s.capacity)
// dequeue/drop from the priority queue
heap.Remove(s.pq, pqEnvTmp.index)
@@ -241,13 +239,11 @@ func (s *pqScheduler) process() {
// There is not sufficient capacity to drop lower priority Envelopes,
// so we drop the incoming Envelope.
s.metrics.PeerQueueDroppedMsgs.With("ch_id", chIDStr).Add(1)
s.logger.Debug(
"dropped envelope",
s.logger.Debug("dropped envelope",
"ch_id", chIDStr,
"priority", pqEnv.priority,
"msg_size", pqEnv.size,
"capacity", s.capacity,
)
"capacity", s.capacity)
}
}

View File

@@ -13,6 +13,7 @@ import (
"github.com/gogo/protobuf/proto"
"github.com/tendermint/tendermint/crypto"
tmstrings "github.com/tendermint/tendermint/internal/libs/strings"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/libs/service"
"github.com/tendermint/tendermint/types"
@@ -585,7 +586,7 @@ func (r *Router) acceptPeers(transport Transport) {
closeErr := conn.Close()
r.logger.Debug("rate limiting incoming peer",
"err", err,
"ip", incomingIP.String(),
"ip", tmstrings.LazyStringer(incomingIP),
"close_err", closeErr,
)

View File

@@ -615,8 +615,7 @@ func (r *Reactor) handleSnapshotMessage(envelope p2p.Envelope) error {
func (r *Reactor) handleChunkMessage(envelope p2p.Envelope) error {
switch msg := envelope.Message.(type) {
case *ssproto.ChunkRequest:
r.Logger.Debug(
"received chunk request",
r.Logger.Debug("received chunk request",
"height", msg.Height,
"format", msg.Format,
"chunk", msg.Index,
@@ -628,19 +627,16 @@ func (r *Reactor) handleChunkMessage(envelope p2p.Envelope) error {
Chunk: msg.Index,
})
if err != nil {
r.Logger.Error(
"failed to load chunk",
r.Logger.Error("failed to load chunk",
"height", msg.Height,
"format", msg.Format,
"chunk", msg.Index,
"err", err,
"peer", envelope.From,
)
"peer", envelope.From)
return nil
}
r.Logger.Debug(
"sending chunk",
r.Logger.Debug("sending chunk",
"height", msg.Height,
"format", msg.Format,
"chunk", msg.Index,
@@ -666,13 +662,11 @@ func (r *Reactor) handleChunkMessage(envelope p2p.Envelope) error {
return nil
}
r.Logger.Debug(
"received chunk; adding to sync",
r.Logger.Debug("received chunk; adding to sync",
"height", msg.Height,
"format", msg.Format,
"chunk", msg.Index,
"peer", envelope.From,
)
"peer", envelope.From)
_, err := r.syncer.AddChunk(&chunk{
Height: msg.Height,
Format: msg.Format,
@@ -681,14 +675,12 @@ func (r *Reactor) handleChunkMessage(envelope p2p.Envelope) error {
Sender: envelope.From,
})
if err != nil {
r.Logger.Error(
"failed to add chunk",
r.Logger.Error("failed to add chunk",
"height", msg.Height,
"format", msg.Format,
"chunk", msg.Index,
"err", err,
"peer", envelope.From,
)
"peer", envelope.From)
return nil
}

View File

@@ -115,11 +115,9 @@ func (s *syncer) AddChunk(chunk *chunk) (bool, error) {
return false, err
}
if added {
s.logger.Debug("Added chunk to queue", "height", chunk.Height, "format", chunk.Format,
"chunk", chunk.Index)
s.logger.Debug("Added chunk to queue", "height", chunk.Height, "format", chunk.Format, "chunk", chunk.Index)
} else {
s.logger.Debug("Ignoring duplicate chunk in queue", "height", chunk.Height, "format", chunk.Format,
"chunk", chunk.Index)
s.logger.Debug("Ignoring duplicate chunk in queue", "height", chunk.Height, "format", chunk.Format, "chunk", chunk.Index)
}
return added, nil
}
@@ -184,9 +182,13 @@ func (s *syncer) SyncAny(
discoveryTime = minimumDiscoveryTime
}
timer := time.NewTimer(discoveryTime)
defer timer.Stop()
if discoveryTime > 0 {
requestSnapshots()
s.logger.Info(fmt.Sprintf("Discovering snapshots for %v", discoveryTime))
s.logger.Info("discovering snapshots",
"interval", discoveryTime)
time.Sleep(discoveryTime)
}
@@ -196,8 +198,11 @@ func (s *syncer) SyncAny(
snapshot *snapshot
chunks *chunkQueue
err error
iters int
)
for {
iters++
// If not nil, we're going to retry restoration of the same snapshot.
if snapshot == nil {
snapshot = s.snapshots.Best()
@@ -207,9 +212,16 @@ func (s *syncer) SyncAny(
if discoveryTime == 0 {
return sm.State{}, nil, errNoSnapshots
}
s.logger.Info(fmt.Sprintf("Discovering snapshots for %v", discoveryTime))
time.Sleep(discoveryTime)
continue
s.logger.Info("discovering snapshots",
"iterations", iters,
"interval", discoveryTime)
timer.Reset(discoveryTime)
select {
case <-ctx.Done():
return sm.State{}, nil, ctx.Err()
case <-timer.C:
continue
}
}
if chunks == nil {
chunks, err = newChunkQueue(snapshot, s.tempDir)
@@ -537,13 +549,11 @@ func (s *syncer) requestChunk(snapshot *snapshot, chunk uint32) {
return
}
s.logger.Debug(
"Requesting snapshot chunk",
s.logger.Debug("Requesting snapshot chunk",
"height", snapshot.Height,
"format", snapshot.Format,
"chunk", chunk,
"peer", peer,
)
"peer", peer)
msg := p2p.Envelope{
To: peer,

View File

@@ -3,8 +3,94 @@ package strings
import (
"fmt"
"strings"
tmbytes "github.com/tendermint/tendermint/libs/bytes"
)
type lazyStringf struct {
tmpl string
args []interface{}
out string
}
func (s *lazyStringf) String() string {
if s.out == "" && s.tmpl != "" {
s.out = fmt.Sprintf(s.tmpl, s.args)
s.args = nil
s.tmpl = ""
}
return s.out
}
// LazySprintf creates a fmt.Stringer implementation with similar
// semantics as fmt.Sprintf, *except* that the string is built when
// String() is called on the object. This means that format arguments
// are resolved/captured into string format when String() is called,
// and not, as in fmt.Sprintf when that function returns.
//
// As a result, if you use this type in go routines or defer
// statements it's possible to pass an argument to LazySprintf which
// has one value at the call site and a different value when the
// String() is evaluated, which may lead to unexpected outcomes. In
// these situations, either be *extremely* careful about the arguments
// passed to this function or use fmt.Sprintf.
//
// The implementation also caches the output of the underlying
// fmt.Sprintf statement when String() is called, so subsequent calls
// will produce the same result.
func LazySprintf(t string, args ...interface{}) fmt.Stringer {
return &lazyStringf{tmpl: t, args: args}
}
type lazyStringer struct {
val fmt.Stringer
out string
}
func (l *lazyStringer) String() string {
if l.out == "" && l.val != nil {
l.out = l.val.String()
l.val = nil
}
return l.out
}
// LazyStringer captures a fmt.Stringer implementation resolving the
// underlying string *only* when the String() method is called and
// caching the result for future use.
func LazyStringer(v fmt.Stringer) fmt.Stringer { return &lazyStringer{val: v} }
type lazyBlockHash struct {
block interface{ Hash() tmbytes.HexBytes }
out string
}
// LazyBlockHash defers block Hash until the Stringer interface is invoked.
// This is particularly useful for avoiding calling Sprintf when debugging is not
// active.
//
// As a result, if you use this type in go routines or defer
// statements it's possible to pass an argument to LazyBlockHash that
// has one value at the call site and a different value when the
// String() is evaluated, which may lead to unexpected outcomes. In
// these situations, either be *extremely* careful about the arguments
// passed to this function or use fmt.Sprintf.
//
// The implementation also caches the output of the string form of the
// block hash when String() is called, so subsequent calls will
// produce the same result.
func LazyBlockHash(block interface{ Hash() tmbytes.HexBytes }) fmt.Stringer {
return &lazyBlockHash{block: block}
}
func (l *lazyBlockHash) String() string {
if l.out == "" && l.block != nil {
l.out = l.block.Hash().String()
l.block = nil
}
return l.out
}
// SplitAndTrimEmpty slices s into all subslices separated by sep and returns a
// slice of the string s with all leading and trailing Unicode code points
// contained in cutset removed. If sep is empty, SplitAndTrim splits after each

View File

@@ -9,6 +9,7 @@ import (
"sync"
"time"
tmstrings "github.com/tendermint/tendermint/internal/libs/strings"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/libs/log"
tmmath "github.com/tendermint/tendermint/libs/math"
@@ -459,7 +460,8 @@ func (c *Client) VerifyHeader(ctx context.Context, newHeader *types.Header, now
return fmt.Errorf("existing trusted header %X does not match newHeader %X", l.Hash(), newHeader.Hash())
}
c.logger.Debug("header has already been verified",
"height", newHeader.Height, "hash", newHeader.Hash())
"height", newHeader.Height,
"hash", tmstrings.LazyBlockHash(newHeader))
return nil
}
@@ -560,7 +562,7 @@ func (c *Client) verifySequential(
// 2) Verify them
c.logger.Debug("verify adjacent newLightBlock against verifiedBlock",
"trustedHeight", verifiedBlock.Height,
"trustedHash", verifiedBlock.Hash(),
"trustedHash", tmstrings.LazyBlockHash(verifiedBlock),
"newHeight", interimBlock.Height,
"newHash", interimBlock.Hash())
@@ -647,9 +649,9 @@ func (c *Client) verifySkipping(
for {
c.logger.Debug("verify non-adjacent newHeader against verifiedBlock",
"trustedHeight", verifiedBlock.Height,
"trustedHash", verifiedBlock.Hash(),
"trustedHash", tmstrings.LazyBlockHash(verifiedBlock),
"newHeight", blockCache[depth].Height,
"newHash", blockCache[depth].Hash())
"newHash", tmstrings.LazyBlockHash(blockCache[depth]))
// Verify the untrusted header. This function is equivalent to
// ValidAndVerified in the spec
@@ -881,9 +883,9 @@ func (c *Client) backwards(
interimHeader = interimBlock.Header
c.logger.Debug("verify newHeader against verifiedHeader",
"trustedHeight", verifiedHeader.Height,
"trustedHash", verifiedHeader.Hash(),
"trustedHash", tmstrings.LazyBlockHash(verifiedHeader),
"newHeight", interimHeader.Height,
"newHash", interimHeader.Hash())
"newHash", tmstrings.LazyBlockHash(interimHeader))
if err := VerifyBackwards(interimHeader, verifiedHeader); err != nil {
// verification has failed
c.logger.Info("backwards verification failed, replacing primary...", "err", err, "primary", c.primary)

View File

@@ -34,8 +34,11 @@ func (c *Client) detectDivergence(ctx context.Context, primaryTrace []*types.Lig
lastVerifiedHeader = primaryTrace[len(primaryTrace)-1].SignedHeader
witnessesToRemove = make([]int, 0)
)
c.logger.Debug("running detector against trace", "endBlockHeight", lastVerifiedHeader.Height,
"endBlockHash", lastVerifiedHeader.Hash, "length", len(primaryTrace))
c.logger.Debug("running detector against trace",
"finalizeBlockHeight", lastVerifiedHeader.Height,
"finalizeBlockHash", lastVerifiedHeader.Hash,
"length", len(primaryTrace))
c.providerMutex.Lock()
defer c.providerMutex.Unlock()

View File

@@ -51,7 +51,7 @@ func DefaultConfig() *Config {
//
// NOTE: This function blocks - you may want to call it in a go-routine.
func Serve(listener net.Listener, handler http.Handler, logger log.Logger, config *Config) error {
logger.Info(fmt.Sprintf("Starting RPC HTTP server on %s", listener.Addr()))
logger.Info("Starting RPC HTTP server on", "addr", listener.Addr())
s := &http.Server{
Handler: RecoverAndLogHandler(maxBytesHandler{h: handler, n: config.MaxBodyBytes}, logger),
ReadTimeout: config.ReadTimeout,

View File

@@ -50,7 +50,7 @@ func cleanupDir(dir string) error {
return err
}
logger.Info(fmt.Sprintf("Removing testnet directory %q", dir))
logger.Info("Removing testnet", "directory", dir)
// On Linux, some local files in the volume will be owned by root since Tendermint
// runs as root inside the container, so we need to clean them up from within a