Browse Source

sync: remove special mutexes (#7438)

pull/7439/head
Sam Kleinman 2 years ago
committed by GitHub
parent
commit
d0e03f01fc
No known key found for this signature in database GPG Key ID: 4AEE18F83AFDEB23
51 changed files with 98 additions and 149 deletions
  1. +1
    -2
      abci/client/client.go
  2. +2
    -2
      abci/client/creators.go
  3. +1
    -2
      abci/client/grpc_client.go
  4. +4
    -4
      abci/client/local_client.go
  5. +2
    -2
      abci/client/socket_client.go
  6. +3
    -3
      abci/server/socket_server.go
  7. +3
    -3
      internal/blocksync/pool.go
  8. +1
    -2
      internal/consensus/byzantine_test.go
  9. +1
    -2
      internal/consensus/common_test.go
  10. +1
    -1
      internal/consensus/peer_state.go
  11. +2
    -2
      internal/consensus/reactor.go
  12. +1
    -2
      internal/consensus/reactor_test.go
  13. +2
    -2
      internal/consensus/state.go
  14. +1
    -1
      internal/evidence/reactor.go
  15. +2
    -4
      internal/libs/clist/clist.go
  16. +2
    -3
      internal/libs/flowrate/flowrate.go
  17. +0
    -18
      internal/libs/sync/deadlock.go
  18. +0
    -16
      internal/libs/sync/sync.go
  19. +2
    -3
      internal/libs/tempfile/tempfile.go
  20. +2
    -3
      internal/libs/timer/throttle_timer.go
  21. +2
    -3
      internal/libs/timer/throttle_timer_test.go
  22. +2
    -2
      internal/mempool/cache.go
  23. +2
    -2
      internal/mempool/ids.go
  24. +2
    -2
      internal/mempool/mempool.go
  25. +2
    -3
      internal/mempool/priority_queue.go
  26. +1
    -1
      internal/mempool/reactor.go
  27. +3
    -3
      internal/mempool/tx.go
  28. +2
    -2
      internal/p2p/conn/connection.go
  29. +3
    -3
      internal/p2p/conn/secret_connection.go
  30. +2
    -2
      internal/statesync/chunks.go
  31. +2
    -2
      internal/statesync/reactor.go
  32. +2
    -2
      internal/statesync/snapshots.go
  33. +3
    -3
      internal/statesync/stateprovider.go
  34. +2
    -2
      internal/statesync/syncer.go
  35. +1
    -2
      internal/statesync/syncer_test.go
  36. +4
    -4
      libs/events/events.go
  37. +2
    -3
      libs/json/structs.go
  38. +2
    -3
      libs/json/types.go
  39. +1
    -2
      light/client.go
  40. +2
    -2
      light/store/db/db.go
  41. +3
    -3
      privval/secret_connection.go
  42. +2
    -2
      privval/signer_endpoint.go
  43. +2
    -2
      privval/signer_listener_endpoint.go
  44. +2
    -2
      privval/signer_server.go
  45. +2
    -2
      rpc/client/http/ws.go
  46. +3
    -3
      rpc/jsonrpc/client/http_json_client.go
  47. +1
    -2
      rpc/jsonrpc/client/ws_client.go
  48. +2
    -2
      rpc/jsonrpc/client/ws_client_test.go
  49. +2
    -2
      types/block.go
  50. +2
    -2
      types/part_set.go
  51. +2
    -2
      types/vote_set.go

+ 1
- 2
abci/client/client.go View File

@ -6,7 +6,6 @@ import (
"sync"
"github.com/tendermint/tendermint/abci/types"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/libs/service"
)
@ -88,7 +87,7 @@ type ReqRes struct {
*sync.WaitGroup
*types.Response // Not set atomically, so be sure to use WaitGroup.
mtx tmsync.Mutex
mtx sync.Mutex
done bool // Gets set to true once *after* WaitGroup.Done().
cb func(*types.Response) // A single callback that may be set.
}


+ 2
- 2
abci/client/creators.go View File

@ -2,9 +2,9 @@ package abciclient
import (
"fmt"
"sync"
"github.com/tendermint/tendermint/abci/types"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/libs/log"
)
@ -14,7 +14,7 @@ type Creator func(log.Logger) (Client, error)
// NewLocalCreator returns a Creator for the given app,
// which will be running locally.
func NewLocalCreator(app types.Application) Creator {
mtx := new(tmsync.Mutex)
mtx := new(sync.Mutex)
return func(_ log.Logger) (Client, error) {
return NewLocalClient(mtx, app), nil


+ 1
- 2
abci/client/grpc_client.go View File

@ -10,7 +10,6 @@ import (
"google.golang.org/grpc"
"github.com/tendermint/tendermint/abci/types"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/libs/log"
tmnet "github.com/tendermint/tendermint/libs/net"
"github.com/tendermint/tendermint/libs/service"
@ -27,7 +26,7 @@ type grpcClient struct {
conn *grpc.ClientConn
chReqRes chan *ReqRes // dispatches "async" responses to callbacks *in order*, needed by mempool
mtx tmsync.Mutex
mtx sync.Mutex
addr string
err error
resCb func(*types.Request, *types.Response) // listens to all callbacks


+ 4
- 4
abci/client/local_client.go View File

@ -2,9 +2,9 @@ package abciclient
import (
"context"
"sync"
types "github.com/tendermint/tendermint/abci/types"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/libs/service"
)
@ -15,7 +15,7 @@ import (
type localClient struct {
service.BaseService
mtx *tmsync.Mutex
mtx *sync.Mutex
types.Application
Callback
}
@ -26,9 +26,9 @@ var _ Client = (*localClient)(nil)
// methods of the given app.
//
// Both Async and Sync methods ignore the given context.Context parameter.
func NewLocalClient(mtx *tmsync.Mutex, app types.Application) Client {
func NewLocalClient(mtx *sync.Mutex, app types.Application) Client {
if mtx == nil {
mtx = new(tmsync.Mutex)
mtx = new(sync.Mutex)
}
cli := &localClient{
mtx: mtx,


+ 2
- 2
abci/client/socket_client.go View File

@ -9,10 +9,10 @@ import (
"io"
"net"
"reflect"
"sync"
"time"
"github.com/tendermint/tendermint/abci/types"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/libs/log"
tmnet "github.com/tendermint/tendermint/libs/net"
"github.com/tendermint/tendermint/libs/service"
@ -41,7 +41,7 @@ type socketClient struct {
reqQueue chan *reqResWithContext
mtx tmsync.Mutex
mtx sync.Mutex
err error
reqSent *list.List // list of requests sent, waiting for response
resCb func(*types.Request, *types.Response) // called on all requests, if set.


+ 3
- 3
abci/server/socket_server.go View File

@ -7,9 +7,9 @@ import (
"io"
"net"
"runtime"
"sync"
"github.com/tendermint/tendermint/abci/types"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/libs/log"
tmnet "github.com/tendermint/tendermint/libs/net"
"github.com/tendermint/tendermint/libs/service"
@ -25,11 +25,11 @@ type SocketServer struct {
addr string
listener net.Listener
connsMtx tmsync.Mutex
connsMtx sync.Mutex
conns map[int]net.Conn
nextConnID int
appMtx tmsync.Mutex
appMtx sync.Mutex
app types.Application
}


+ 3
- 3
internal/blocksync/pool.go View File

@ -5,11 +5,11 @@ import (
"errors"
"fmt"
"math"
"sync"
"sync/atomic"
"time"
"github.com/tendermint/tendermint/internal/libs/flowrate"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/libs/service"
"github.com/tendermint/tendermint/types"
@ -73,7 +73,7 @@ type BlockPool struct {
lastAdvance time.Time
mtx tmsync.RWMutex
mtx sync.RWMutex
// block requests
requesters map[int64]*bpRequester
height int64 // the lowest key in requesters.
@ -560,7 +560,7 @@ type bpRequester struct {
gotBlockCh chan struct{}
redoCh chan types.NodeID // redo may send multitime, add peerId to identify repeat
mtx tmsync.Mutex
mtx sync.Mutex
peerID types.NodeID
block *types.Block
}


+ 1
- 2
internal/consensus/byzantine_test.go View File

@ -16,7 +16,6 @@ import (
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/internal/eventbus"
"github.com/tendermint/tendermint/internal/evidence"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/internal/mempool"
"github.com/tendermint/tendermint/internal/p2p"
sm "github.com/tendermint/tendermint/internal/state"
@ -68,7 +67,7 @@ func TestByzantinePrevoteEquivocation(t *testing.T) {
blockStore := store.NewBlockStore(blockDB)
// one for mempool, one for consensus
mtx := new(tmsync.Mutex)
mtx := new(sync.Mutex)
proxyAppConnMem := abciclient.NewLocalClient(mtx, app)
proxyAppConnCon := abciclient.NewLocalClient(mtx, app)


+ 1
- 2
internal/consensus/common_test.go View File

@ -21,7 +21,6 @@ import (
"github.com/tendermint/tendermint/config"
cstypes "github.com/tendermint/tendermint/internal/consensus/types"
"github.com/tendermint/tendermint/internal/eventbus"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/internal/mempool"
sm "github.com/tendermint/tendermint/internal/state"
"github.com/tendermint/tendermint/internal/store"
@ -440,7 +439,7 @@ func newStateWithConfigAndBlockStore(
blockStore *store.BlockStore,
) *State {
// one for mempool, one for consensus
mtx := new(tmsync.Mutex)
mtx := new(sync.Mutex)
proxyAppConnMem := abciclient.NewLocalClient(mtx, app)
proxyAppConnCon := abciclient.NewLocalClient(mtx, app)


+ 1
- 1
internal/consensus/peer_state.go View File

@ -40,7 +40,7 @@ type PeerState struct {
logger log.Logger
// NOTE: Modify below using setters, never directly.
mtx tmsync.RWMutex
mtx sync.RWMutex
running bool
PRS cstypes.PeerRoundState `json:"round_state"`
Stats *peerStateStats `json:"stats"`


+ 2
- 2
internal/consensus/reactor.go View File

@ -5,11 +5,11 @@ import (
"errors"
"fmt"
"runtime/debug"
"sync"
"time"
cstypes "github.com/tendermint/tendermint/internal/consensus/types"
"github.com/tendermint/tendermint/internal/eventbus"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/internal/p2p"
sm "github.com/tendermint/tendermint/internal/state"
"github.com/tendermint/tendermint/libs/bits"
@ -117,7 +117,7 @@ type Reactor struct {
eventBus *eventbus.EventBus
Metrics *Metrics
mtx tmsync.RWMutex
mtx sync.RWMutex
peers map[types.NodeID]*PeerState
waitSync bool


+ 1
- 2
internal/consensus/reactor_test.go View File

@ -21,7 +21,6 @@ import (
"github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/crypto/encoding"
"github.com/tendermint/tendermint/internal/eventbus"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/internal/mempool"
"github.com/tendermint/tendermint/internal/p2p"
"github.com/tendermint/tendermint/internal/p2p/p2ptest"
@ -392,7 +391,7 @@ func TestReactorWithEvidence(t *testing.T) {
blockStore := store.NewBlockStore(blockDB)
// one for mempool, one for consensus
mtx := new(tmsync.Mutex)
mtx := new(sync.Mutex)
proxyAppConnMem := abciclient.NewLocalClient(mtx, app)
proxyAppConnCon := abciclient.NewLocalClient(mtx, app)


+ 2
- 2
internal/consensus/state.go View File

@ -8,6 +8,7 @@ import (
"io"
"os"
"runtime/debug"
"sync"
"time"
"github.com/gogo/protobuf/proto"
@ -17,7 +18,6 @@ import (
cstypes "github.com/tendermint/tendermint/internal/consensus/types"
"github.com/tendermint/tendermint/internal/eventbus"
"github.com/tendermint/tendermint/internal/libs/fail"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
sm "github.com/tendermint/tendermint/internal/state"
tmevents "github.com/tendermint/tendermint/libs/events"
tmjson "github.com/tendermint/tendermint/libs/json"
@ -100,7 +100,7 @@ type State struct {
evpool evidencePool
// internal state
mtx tmsync.RWMutex
mtx sync.RWMutex
cstypes.RoundState
state sm.State // State until height-1.
// privValidator pubkey, memoized for the duration of one block


+ 1
- 1
internal/evidence/reactor.go View File

@ -53,7 +53,7 @@ type Reactor struct {
peerWG sync.WaitGroup
mtx tmsync.Mutex
mtx sync.Mutex
peerRoutines map[types.NodeID]*tmsync.Closer
}


+ 2
- 4
internal/libs/clist/clist.go View File

@ -14,8 +14,6 @@ to ensure garbage collection of removed elements.
import (
"fmt"
"sync"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
)
// MaxLength is the max allowed number of elements a linked list is
@ -44,7 +42,7 @@ waiting on NextWait() (since it's just a read operation).
*/
type CElement struct {
mtx tmsync.RWMutex
mtx sync.RWMutex
prev *CElement
prevWg *sync.WaitGroup
prevWaitCh chan struct{}
@ -220,7 +218,7 @@ func (e *CElement) SetRemoved() {
// Operations are goroutine-safe.
// Panics if length grows beyond the max.
type CList struct {
mtx tmsync.RWMutex
mtx sync.RWMutex
wg *sync.WaitGroup
waitCh chan struct{}
head *CElement // first element


+ 2
- 3
internal/libs/flowrate/flowrate.go View File

@ -8,14 +8,13 @@ package flowrate
import (
"math"
"sync"
"time"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
)
// Monitor monitors and limits the transfer rate of a data stream.
type Monitor struct {
mu tmsync.Mutex // Mutex guarding access to all internal fields
mu sync.Mutex // Mutex guarding access to all internal fields
active bool // Flag indicating an active transfer
start time.Duration // Transfer start time (clock() value)
bytes int64 // Total number of bytes transferred


+ 0
- 18
internal/libs/sync/deadlock.go View File

@ -1,18 +0,0 @@
//go:build deadlock
// +build deadlock
package sync
import (
deadlock "github.com/sasha-s/go-deadlock"
)
// A Mutex is a mutual exclusion lock.
type Mutex struct {
deadlock.Mutex
}
// An RWMutex is a reader/writer mutual exclusion lock.
type RWMutex struct {
deadlock.RWMutex
}

+ 0
- 16
internal/libs/sync/sync.go View File

@ -1,16 +0,0 @@
//go:build !deadlock
// +build !deadlock
package sync
import "sync"
// A Mutex is a mutual exclusion lock.
type Mutex struct {
sync.Mutex
}
// An RWMutex is a reader/writer mutual exclusion lock.
type RWMutex struct {
sync.RWMutex
}

+ 2
- 3
internal/libs/tempfile/tempfile.go View File

@ -7,9 +7,8 @@ import (
"path/filepath"
"strconv"
"strings"
"sync"
"time"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
)
const (
@ -32,7 +31,7 @@ const (
var (
atomicWriteFileRand uint64
atomicWriteFileRandMu tmsync.Mutex
atomicWriteFileRandMu sync.Mutex
)
func writeFileRandReseed() uint64 {


+ 2
- 3
internal/libs/timer/throttle_timer.go View File

@ -1,9 +1,8 @@
package timer
import (
"sync"
"time"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
)
/*
@ -18,7 +17,7 @@ type ThrottleTimer struct {
quit chan struct{}
dur time.Duration
mtx tmsync.Mutex
mtx sync.Mutex
timer *time.Timer
isSet bool
}


+ 2
- 3
internal/libs/timer/throttle_timer_test.go View File

@ -1,19 +1,18 @@
package timer
import (
"sync"
"testing"
"time"
// make govet noshadow happy...
asrt "github.com/stretchr/testify/assert"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
)
type thCounter struct {
input chan struct{}
mtx tmsync.Mutex
mtx sync.Mutex
count int
}


+ 2
- 2
internal/mempool/cache.go View File

@ -2,8 +2,8 @@ package mempool
import (
"container/list"
"sync"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/types"
)
@ -29,7 +29,7 @@ var _ TxCache = (*LRUTxCache)(nil)
// LRUTxCache maintains a thread-safe LRU cache of raw transactions. The cache
// only stores the hash of the raw transaction.
type LRUTxCache struct {
mtx tmsync.Mutex
mtx sync.Mutex
size int
cacheMap map[types.TxKey]*list.Element
list *list.List


+ 2
- 2
internal/mempool/ids.go View File

@ -2,13 +2,13 @@ package mempool
import (
"fmt"
"sync"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/types"
)
type IDs struct {
mtx tmsync.RWMutex
mtx sync.RWMutex
peerMap map[types.NodeID]uint16
nextID uint16 // assumes that a node will never have over 65536 active peers
activeIDs map[uint16]struct{} // used to check if a given peerID key is used


+ 2
- 2
internal/mempool/mempool.go View File

@ -6,13 +6,13 @@ import (
"errors"
"fmt"
"reflect"
"sync"
"sync/atomic"
"time"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/config"
"github.com/tendermint/tendermint/internal/libs/clist"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/internal/proxy"
"github.com/tendermint/tendermint/libs/log"
tmmath "github.com/tendermint/tendermint/libs/math"
@ -86,7 +86,7 @@ type TxMempool struct {
// from the mempool. A read-lock is implicitly acquired when executing CheckTx,
// however, a caller must explicitly grab a write-lock via Lock when updating
// the mempool via Update().
mtx tmsync.RWMutex
mtx sync.RWMutex
preCheck PreCheckFunc
postCheck PostCheckFunc
}


+ 2
- 3
internal/mempool/priority_queue.go View File

@ -3,15 +3,14 @@ package mempool
import (
"container/heap"
"sort"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"sync"
)
var _ heap.Interface = (*TxPriorityQueue)(nil)
// TxPriorityQueue defines a thread-safe priority queue for valid transactions.
type TxPriorityQueue struct {
mtx tmsync.RWMutex
mtx sync.RWMutex
txs []*WrappedTx
}


+ 1
- 1
internal/mempool/reactor.go View File

@ -57,7 +57,7 @@ type Reactor struct {
// Reactor. observePanic is called with the recovered value.
observePanic func(interface{})
mtx tmsync.Mutex
mtx sync.Mutex
peerRoutines map[types.NodeID]*tmsync.Closer
}


+ 3
- 3
internal/mempool/tx.go View File

@ -2,10 +2,10 @@ package mempool
import (
"sort"
"sync"
"time"
"github.com/tendermint/tendermint/internal/libs/clist"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/types"
)
@ -76,7 +76,7 @@ func (wtx *WrappedTx) Size() int {
// access is not allowed. Regardless, it is not expected for the mempool to
// need mutative access.
type TxStore struct {
mtx tmsync.RWMutex
mtx sync.RWMutex
hashTxs map[types.TxKey]*WrappedTx // primary index
senderTxs map[string]*WrappedTx // sender is defined by the ABCI application
}
@ -217,7 +217,7 @@ func (txs *TxStore) GetOrSetPeerByTxHash(hash types.TxKey, peerID uint16) (*Wrap
// references which is used during Insert in order to determine sorted order. If
// less returns true, a <= b.
type WrappedTxList struct {
mtx tmsync.RWMutex
mtx sync.RWMutex
txs []*WrappedTx
less func(*WrappedTx, *WrappedTx) bool
}


+ 2
- 2
internal/p2p/conn/connection.go View File

@ -10,6 +10,7 @@ import (
"net"
"reflect"
"runtime/debug"
"sync"
"sync/atomic"
"time"
@ -17,7 +18,6 @@ import (
"github.com/tendermint/tendermint/internal/libs/flowrate"
"github.com/tendermint/tendermint/internal/libs/protoio"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/internal/libs/timer"
"github.com/tendermint/tendermint/libs/log"
tmmath "github.com/tendermint/tendermint/libs/math"
@ -100,7 +100,7 @@ type MConnection struct {
// used to ensure FlushStop and OnStop
// are safe to call concurrently.
stopMtx tmsync.Mutex
stopMtx sync.Mutex
cancel context.CancelFunc


+ 3
- 3
internal/p2p/conn/secret_connection.go View File

@ -11,6 +11,7 @@ import (
"io"
"math"
"net"
"sync"
"time"
gogotypes "github.com/gogo/protobuf/types"
@ -25,7 +26,6 @@ import (
"github.com/tendermint/tendermint/crypto/ed25519"
"github.com/tendermint/tendermint/crypto/encoding"
"github.com/tendermint/tendermint/internal/libs/protoio"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/libs/async"
tmp2p "github.com/tendermint/tendermint/proto/tendermint/p2p"
)
@ -76,11 +76,11 @@ type SecretConnection struct {
// are independent, so we can use two mtxs.
// All .Read are covered by recvMtx,
// all .Write are covered by sendMtx.
recvMtx tmsync.Mutex
recvMtx sync.Mutex
recvBuffer []byte
recvNonce *[aeadNonceSize]byte
sendMtx tmsync.Mutex
sendMtx sync.Mutex
sendNonce *[aeadNonceSize]byte
}


+ 2
- 2
internal/statesync/chunks.go View File

@ -6,9 +6,9 @@ import (
"os"
"path/filepath"
"strconv"
"sync"
"time"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/types"
)
@ -28,7 +28,7 @@ type chunk struct {
// iterator over all chunks, but callers can request chunks to be retried, optionally after
// refetching.
type chunkQueue struct {
tmsync.Mutex
sync.Mutex
snapshot *snapshot // if this is nil, the queue has been closed
dir string // temp dir for on-disk chunk storage
chunkFiles map[uint32]string // path to temporary chunk file


+ 2
- 2
internal/statesync/reactor.go View File

@ -8,11 +8,11 @@ import (
"reflect"
"runtime/debug"
"sort"
"sync"
"time"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/config"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/internal/p2p"
"github.com/tendermint/tendermint/internal/proxy"
sm "github.com/tendermint/tendermint/internal/state"
@ -151,7 +151,7 @@ type Reactor struct {
// These will only be set when a state sync is in progress. It is used to feed
// received snapshots and chunks into the syncer and manage incoming and outgoing
// providers.
mtx tmsync.RWMutex
mtx sync.RWMutex
syncer *syncer
providers map[types.NodeID]*BlockProvider
stateProvider StateProvider


+ 2
- 2
internal/statesync/snapshots.go View File

@ -6,8 +6,8 @@ import (
"math/rand"
"sort"
"strings"
"sync"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/types"
)
@ -41,7 +41,7 @@ func (s *snapshot) Key() snapshotKey {
// snapshotPool discovers and aggregates snapshots across peers.
type snapshotPool struct {
tmsync.Mutex
sync.Mutex
snapshots map[snapshotKey]*snapshot
snapshotPeers map[snapshotKey]map[types.NodeID]types.NodeID


+ 3
- 3
internal/statesync/stateprovider.go View File

@ -6,11 +6,11 @@ import (
"errors"
"fmt"
"strings"
"sync"
"time"
dbm "github.com/tendermint/tm-db"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/internal/p2p"
sm "github.com/tendermint/tendermint/internal/state"
"github.com/tendermint/tendermint/libs/log"
@ -40,7 +40,7 @@ type StateProvider interface {
}
type stateProviderRPC struct {
tmsync.Mutex // light.Client is not concurrency-safe
sync.Mutex // light.Client is not concurrency-safe
lc *light.Client
initialHeight int64
providers map[lightprovider.Provider]string
@ -197,7 +197,7 @@ func rpcClient(server string) (*rpchttp.HTTP, error) {
}
type stateProviderP2P struct {
tmsync.Mutex // light.Client is not concurrency-safe
sync.Mutex // light.Client is not concurrency-safe
lc *light.Client
initialHeight int64
paramsSendCh *p2p.Channel


+ 2
- 2
internal/statesync/syncer.go View File

@ -5,11 +5,11 @@ import (
"context"
"errors"
"fmt"
"sync"
"time"
abci "github.com/tendermint/tendermint/abci/types"
"github.com/tendermint/tendermint/config"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/internal/p2p"
"github.com/tendermint/tendermint/internal/proxy"
sm "github.com/tendermint/tendermint/internal/state"
@ -63,7 +63,7 @@ type syncer struct {
fetchers int32
retryTimeout time.Duration
mtx tmsync.RWMutex
mtx sync.RWMutex
chunks *chunkQueue
metrics *Metrics


+ 1
- 2
internal/statesync/syncer_test.go View File

@ -12,7 +12,6 @@ import (
"github.com/stretchr/testify/require"
abci "github.com/tendermint/tendermint/abci/types"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/internal/proxy"
proxymocks "github.com/tendermint/tendermint/internal/proxy/mocks"
sm "github.com/tendermint/tendermint/internal/state"
@ -132,7 +131,7 @@ func TestSyncer_SyncAny(t *testing.T) {
}).Times(2).Return(&abci.ResponseOfferSnapshot{Result: abci.ResponseOfferSnapshot_ACCEPT}, nil)
chunkRequests := make(map[uint32]int)
chunkRequestsMtx := tmsync.Mutex{}
chunkRequestsMtx := sync.Mutex{}
var wg sync.WaitGroup
wg.Add(4)


+ 4
- 4
libs/events/events.go View File

@ -4,8 +4,8 @@ package events
import (
"context"
"fmt"
"sync"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/libs/service"
)
@ -56,7 +56,7 @@ type EventSwitch interface {
type eventSwitch struct {
service.BaseService
mtx tmsync.RWMutex
mtx sync.RWMutex
eventCells map[string]*eventCell
listeners map[string]*eventListener
}
@ -166,7 +166,7 @@ func (evsw *eventSwitch) FireEvent(ctx context.Context, event string, data Event
// eventCell handles keeping track of listener callbacks for a given event.
type eventCell struct {
mtx tmsync.RWMutex
mtx sync.RWMutex
listeners map[string]EventCallback
}
@ -213,7 +213,7 @@ type EventCallback func(ctx context.Context, data EventData) error
type eventListener struct {
id string
mtx tmsync.RWMutex
mtx sync.RWMutex
removed bool
events []string
}


+ 2
- 3
libs/json/structs.go View File

@ -4,9 +4,8 @@ import (
"fmt"
"reflect"
"strings"
"sync"
"unicode"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
)
var (
@ -16,7 +15,7 @@ var (
// structCache is a cache of struct info.
type structInfoCache struct {
tmsync.RWMutex
sync.RWMutex
structInfos map[reflect.Type]*structInfo
}


+ 2
- 3
libs/json/types.go View File

@ -4,8 +4,7 @@ import (
"errors"
"fmt"
"reflect"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"sync"
)
var (
@ -39,7 +38,7 @@ type typeInfo struct {
// types is a type registry. It is safe for concurrent use.
type types struct {
tmsync.RWMutex
sync.RWMutex
byType map[reflect.Type]*typeInfo
byName map[string]*typeInfo
}


+ 1
- 2
light/client.go View File

@ -9,7 +9,6 @@ import (
"sync"
"time"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/libs/log"
tmmath "github.com/tendermint/tendermint/libs/math"
"github.com/tendermint/tendermint/light/provider"
@ -134,7 +133,7 @@ type Client struct {
providerTimeout time.Duration
// Mutex for locking during changes of the light clients providers
providerMutex tmsync.Mutex
providerMutex sync.Mutex
// Primary provider of new headers.
primary provider.Provider
// Providers used to "witness" new headers.


+ 2
- 2
light/store/db/db.go View File

@ -3,11 +3,11 @@ package db
import (
"encoding/binary"
"fmt"
"sync"
"github.com/google/orderedcode"
dbm "github.com/tendermint/tm-db"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/light/store"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
"github.com/tendermint/tendermint/types"
@ -21,7 +21,7 @@ const (
type dbs struct {
db dbm.DB
mtx tmsync.RWMutex
mtx sync.RWMutex
size uint16
}


+ 3
- 3
privval/secret_connection.go View File

@ -11,6 +11,7 @@ import (
"io"
"math"
"net"
"sync"
"time"
gogotypes "github.com/gogo/protobuf/types"
@ -25,7 +26,6 @@ import (
"github.com/tendermint/tendermint/crypto/ed25519"
"github.com/tendermint/tendermint/crypto/encoding"
"github.com/tendermint/tendermint/internal/libs/protoio"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/libs/async"
tmprivval "github.com/tendermint/tendermint/proto/tendermint/privval"
)
@ -80,11 +80,11 @@ type SecretConnection struct {
// are independent, so we can use two mtxs.
// All .Read are covered by recvMtx,
// all .Write are covered by sendMtx.
recvMtx tmsync.Mutex
recvMtx sync.Mutex
recvBuffer []byte
recvNonce *[aeadNonceSize]byte
sendMtx tmsync.Mutex
sendMtx sync.Mutex
sendNonce *[aeadNonceSize]byte
}


+ 2
- 2
privval/signer_endpoint.go View File

@ -3,10 +3,10 @@ package privval
import (
"fmt"
"net"
"sync"
"time"
"github.com/tendermint/tendermint/internal/libs/protoio"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/libs/service"
privvalproto "github.com/tendermint/tendermint/proto/tendermint/privval"
@ -20,7 +20,7 @@ type signerEndpoint struct {
service.BaseService
logger log.Logger
connMtx tmsync.Mutex
connMtx sync.Mutex
conn net.Conn
timeoutReadWrite time.Duration


+ 2
- 2
privval/signer_listener_endpoint.go View File

@ -4,9 +4,9 @@ import (
"context"
"fmt"
"net"
"sync"
"time"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/libs/log"
"github.com/tendermint/tendermint/libs/service"
privvalproto "github.com/tendermint/tendermint/proto/tendermint/privval"
@ -39,7 +39,7 @@ type SignerListenerEndpoint struct {
pingTimer *time.Ticker
pingInterval time.Duration
instanceMtx tmsync.Mutex // Ensures instance public methods access, i.e. SendRequest
instanceMtx sync.Mutex // Ensures instance public methods access, i.e. SendRequest
}
// NewSignerListenerEndpoint returns an instance of SignerListenerEndpoint.


+ 2
- 2
privval/signer_server.go View File

@ -3,8 +3,8 @@ package privval
import (
"context"
"io"
"sync"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/libs/service"
privvalproto "github.com/tendermint/tendermint/proto/tendermint/privval"
"github.com/tendermint/tendermint/types"
@ -24,7 +24,7 @@ type SignerServer struct {
chainID string
privVal types.PrivValidator
handlerMtx tmsync.Mutex
handlerMtx sync.Mutex
validationRequestHandler ValidationRequestHandlerFunc
}


+ 2
- 2
rpc/client/http/ws.go View File

@ -5,9 +5,9 @@ import (
"errors"
"fmt"
"strings"
"sync"
"time"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
tmjson "github.com/tendermint/tendermint/libs/json"
"github.com/tendermint/tendermint/libs/pubsub"
rpcclient "github.com/tendermint/tendermint/rpc/client"
@ -48,7 +48,7 @@ type wsEvents struct {
*rpcclient.RunState
ws *jsonrpcclient.WSClient
mtx tmsync.RWMutex
mtx sync.RWMutex
subscriptions map[string]*wsSubscription
}


+ 3
- 3
rpc/jsonrpc/client/http_json_client.go View File

@ -11,9 +11,9 @@ import (
"net/http"
"net/url"
"strings"
"sync"
"time"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
rpctypes "github.com/tendermint/tendermint/rpc/jsonrpc/types"
)
@ -130,7 +130,7 @@ type Client struct {
client *http.Client
mtx tmsync.Mutex
mtx sync.Mutex
nextReqID int
}
@ -304,7 +304,7 @@ type jsonRPCBufferedRequest struct {
type RequestBatch struct {
client *Client
mtx tmsync.Mutex
mtx sync.Mutex
requests []*jsonRPCBufferedRequest
}


+ 1
- 2
rpc/jsonrpc/client/ws_client.go View File

@ -13,7 +13,6 @@ import (
"github.com/gorilla/websocket"
metrics "github.com/rcrowley/go-metrics"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
tmclient "github.com/tendermint/tendermint/rpc/client"
rpctypes "github.com/tendermint/tendermint/rpc/jsonrpc/types"
)
@ -70,7 +69,7 @@ type WSClient struct { // nolint: maligned
wg sync.WaitGroup
mtx tmsync.RWMutex
mtx sync.RWMutex
sentLastPingAt time.Time
reconnecting bool
nextReqID int


+ 2
- 2
rpc/jsonrpc/client/ws_client_test.go View File

@ -6,6 +6,7 @@ import (
"net/http"
"net/http/httptest"
"runtime"
"sync"
"testing"
"time"
@ -13,7 +14,6 @@ import (
"github.com/gorilla/websocket"
"github.com/stretchr/testify/require"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/libs/log"
rpctypes "github.com/tendermint/tendermint/rpc/jsonrpc/types"
)
@ -22,7 +22,7 @@ var wsCallTimeout = 5 * time.Second
type myHandler struct {
closeConnAfterRead bool
mtx tmsync.RWMutex
mtx sync.RWMutex
}
var upgrader = websocket.Upgrader{


+ 2
- 2
types/block.go View File

@ -5,6 +5,7 @@ import (
"errors"
"fmt"
"strings"
"sync"
"time"
"github.com/gogo/protobuf/proto"
@ -13,7 +14,6 @@ import (
"github.com/tendermint/tendermint/crypto"
"github.com/tendermint/tendermint/crypto/merkle"
"github.com/tendermint/tendermint/crypto/tmhash"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/libs/bits"
tmbytes "github.com/tendermint/tendermint/libs/bytes"
tmmath "github.com/tendermint/tendermint/libs/math"
@ -40,7 +40,7 @@ const (
// Block defines the atomic unit of a Tendermint blockchain.
type Block struct {
mtx tmsync.Mutex
mtx sync.Mutex
Header `json:"header"`
Data `json:"data"`


+ 2
- 2
types/part_set.go View File

@ -5,9 +5,9 @@ import (
"errors"
"fmt"
"io"
"sync"
"github.com/tendermint/tendermint/crypto/merkle"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/libs/bits"
tmbytes "github.com/tendermint/tendermint/libs/bytes"
tmjson "github.com/tendermint/tendermint/libs/json"
@ -151,7 +151,7 @@ type PartSet struct {
total uint32
hash []byte
mtx tmsync.Mutex
mtx sync.Mutex
parts []*Part
partsBitArray *bits.BitArray
count uint32


+ 2
- 2
types/vote_set.go View File

@ -4,8 +4,8 @@ import (
"bytes"
"fmt"
"strings"
"sync"
tmsync "github.com/tendermint/tendermint/internal/libs/sync"
"github.com/tendermint/tendermint/libs/bits"
tmjson "github.com/tendermint/tendermint/libs/json"
tmproto "github.com/tendermint/tendermint/proto/tendermint/types"
@ -65,7 +65,7 @@ type VoteSet struct {
signedMsgType tmproto.SignedMsgType
valSet *ValidatorSet
mtx tmsync.Mutex
mtx sync.Mutex
votesBitArray *bits.BitArray
votes []*Vote // Primary votes to share
sum int64 // Sum of voting power for seen votes, discounting conflicts


Loading…
Cancel
Save