diff --git a/CHANGELOG_PENDING.md b/CHANGELOG_PENDING.md index 84254d818..acc10379a 100644 --- a/CHANGELOG_PENDING.md +++ b/CHANGELOG_PENDING.md @@ -77,6 +77,13 @@ program](https://hackerone.com/tendermint). - [libs/common] \#4237 Move byte functions from `libs/common` into pkg `bytes` - [libs/common] \#4237 Move throttletimer functions from `libs/common` into pkg `timer` - [libs/common] \#4237 Move tempfile functions from `libs/common` into pkg `tempfile` + - [libs/common] \#4240 Move os functions from `libs/common` into pkg `os` + - [libs/common] \#4240 Move net functions from `libs/common` into pkg `net` + - [libs/common] \#4240 Move mathematical functions and types out of `libs/common` to `math` pkg + - [libs/common] \#4240 Move string functions out of `libs/common` to `strings` pkg + - [libs/common] \#4240 Move async functions out of `libs/common` to `async` pkg + - [libs/common] \#4240 Move bit functions out of `libs/common` to `bits` pkg + - [libs/common] \#4240 Move cmap functions out of `libs/common` to `cmap` pkg - Blockchain Protocol diff --git a/abci/client/grpc_client.go b/abci/client/grpc_client.go index 514e27533..f853a4254 100644 --- a/abci/client/grpc_client.go +++ b/abci/client/grpc_client.go @@ -10,7 +10,7 @@ import ( "google.golang.org/grpc" "github.com/tendermint/tendermint/abci/types" - cmn "github.com/tendermint/tendermint/libs/common" + tmnet "github.com/tendermint/tendermint/libs/net" "github.com/tendermint/tendermint/libs/service" ) @@ -41,7 +41,7 @@ func NewGRPCClient(addr string, mustConnect bool) *grpcClient { } func dialerFunc(ctx context.Context, addr string) (net.Conn, error) { - return cmn.Connect(addr) + return tmnet.Connect(addr) } func (cli *grpcClient) OnStart() error { diff --git a/abci/client/socket_client.go b/abci/client/socket_client.go index a3d07ff0a..2e765119b 100644 --- a/abci/client/socket_client.go +++ b/abci/client/socket_client.go @@ -12,7 +12,7 @@ import ( "time" "github.com/tendermint/tendermint/abci/types" - cmn "github.com/tendermint/tendermint/libs/common" + tmnet "github.com/tendermint/tendermint/libs/net" "github.com/tendermint/tendermint/libs/service" "github.com/tendermint/tendermint/libs/timer" ) @@ -62,7 +62,7 @@ func (cli *socketClient) OnStart() error { var conn net.Conn RETRY_LOOP: for { - conn, err = cmn.Connect(cli.addr) + conn, err = tmnet.Connect(cli.addr) if err != nil { if cli.mustConnect { return err diff --git a/abci/cmd/abci-cli/abci-cli.go b/abci/cmd/abci-cli/abci-cli.go index 3f283e9bd..d5a9aca27 100644 --- a/abci/cmd/abci-cli/abci-cli.go +++ b/abci/cmd/abci-cli/abci-cli.go @@ -11,8 +11,8 @@ import ( "github.com/spf13/cobra" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + tmos "github.com/tendermint/tendermint/libs/os" abcicli "github.com/tendermint/tendermint/abci/client" "github.com/tendermint/tendermint/abci/example/code" @@ -640,7 +640,7 @@ func cmdCounter(cmd *cobra.Command, args []string) error { } // Stop upon receiving SIGTERM or CTRL-C. - cmn.TrapSignal(logger, func() { + tmos.TrapSignal(logger, func() { // Cleanup srv.Stop() }) @@ -672,7 +672,7 @@ func cmdKVStore(cmd *cobra.Command, args []string) error { } // Stop upon receiving SIGTERM or CTRL-C. - cmn.TrapSignal(logger, func() { + tmos.TrapSignal(logger, func() { // Cleanup srv.Stop() }) diff --git a/abci/example/example_test.go b/abci/example/example_test.go index c96dd2778..d40976015 100644 --- a/abci/example/example_test.go +++ b/abci/example/example_test.go @@ -13,8 +13,8 @@ import ( "golang.org/x/net/context" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + tmnet "github.com/tendermint/tendermint/libs/net" abcicli "github.com/tendermint/tendermint/abci/client" "github.com/tendermint/tendermint/abci/example/code" @@ -108,7 +108,7 @@ func testStream(t *testing.T, app types.Application) { // test grpc func dialerFunc(ctx context.Context, addr string) (net.Conn, error) { - return cmn.Connect(addr) + return tmnet.Connect(addr) } func testGRPCSync(t *testing.T, app types.ABCIApplicationServer) { diff --git a/abci/server/grpc_server.go b/abci/server/grpc_server.go index 0f8e42d7f..b953c404d 100644 --- a/abci/server/grpc_server.go +++ b/abci/server/grpc_server.go @@ -6,7 +6,7 @@ import ( "google.golang.org/grpc" "github.com/tendermint/tendermint/abci/types" - cmn "github.com/tendermint/tendermint/libs/common" + tmnet "github.com/tendermint/tendermint/libs/net" "github.com/tendermint/tendermint/libs/service" ) @@ -23,7 +23,7 @@ type GRPCServer struct { // NewGRPCServer returns a new gRPC ABCI server func NewGRPCServer(protoAddr string, app types.ABCIApplicationServer) service.Service { - proto, addr := cmn.ProtocolAndAddress(protoAddr) + proto, addr := tmnet.ProtocolAndAddress(protoAddr) s := &GRPCServer{ proto: proto, addr: addr, diff --git a/abci/server/socket_server.go b/abci/server/socket_server.go index c69be3813..efb4d94e0 100644 --- a/abci/server/socket_server.go +++ b/abci/server/socket_server.go @@ -8,7 +8,7 @@ import ( "sync" "github.com/tendermint/tendermint/abci/types" - cmn "github.com/tendermint/tendermint/libs/common" + tmnet "github.com/tendermint/tendermint/libs/net" "github.com/tendermint/tendermint/libs/service" ) @@ -30,7 +30,7 @@ type SocketServer struct { } func NewSocketServer(protoAddr string, app types.Application) service.Service { - proto, addr := cmn.ProtocolAndAddress(protoAddr) + proto, addr := tmnet.ProtocolAndAddress(protoAddr) s := &SocketServer{ proto: proto, addr: addr, diff --git a/abci/tests/benchmarks/parallel/parallel.go b/abci/tests/benchmarks/parallel/parallel.go index 78b69ed12..fe213313d 100644 --- a/abci/tests/benchmarks/parallel/parallel.go +++ b/abci/tests/benchmarks/parallel/parallel.go @@ -6,12 +6,12 @@ import ( "log" "github.com/tendermint/tendermint/abci/types" - cmn "github.com/tendermint/tendermint/libs/common" + tmnet "github.com/tendermint/tendermint/libs/net" ) func main() { - conn, err := cmn.Connect("unix://test.sock") + conn, err := tmnet.Connect("unix://test.sock") if err != nil { log.Fatal(err.Error()) } diff --git a/abci/tests/benchmarks/simple/simple.go b/abci/tests/benchmarks/simple/simple.go index f3ac8a170..b18eaa580 100644 --- a/abci/tests/benchmarks/simple/simple.go +++ b/abci/tests/benchmarks/simple/simple.go @@ -8,12 +8,12 @@ import ( "reflect" "github.com/tendermint/tendermint/abci/types" - cmn "github.com/tendermint/tendermint/libs/common" + tmnet "github.com/tendermint/tendermint/libs/net" ) func main() { - conn, err := cmn.Connect("unix://test.sock") + conn, err := tmnet.Connect("unix://test.sock") if err != nil { log.Fatal(err.Error()) } diff --git a/blockchain/v1/reactor_fsm_test.go b/blockchain/v1/reactor_fsm_test.go index 511664121..1d4b55418 100644 --- a/blockchain/v1/reactor_fsm_test.go +++ b/blockchain/v1/reactor_fsm_test.go @@ -6,8 +6,8 @@ import ( "time" "github.com/stretchr/testify/assert" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + tmmath "github.com/tendermint/tendermint/libs/math" "github.com/tendermint/tendermint/libs/rand" "github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/types" @@ -736,7 +736,7 @@ func makeCorrectTransitionSequence(startingHeight int64, numBlocks int64, numPee continue } if randomPeerHeights { - peerHeights[i] = int64(cmn.MaxInt(rand.RandIntn(int(numBlocks)), int(startingHeight)+1)) + peerHeights[i] = int64(tmmath.MaxInt(rand.RandIntn(int(numBlocks)), int(startingHeight)+1)) } else { peerHeights[i] = numBlocks } diff --git a/cmd/priv_val_server/main.go b/cmd/priv_val_server/main.go index 22af6418f..813ce6b14 100644 --- a/cmd/priv_val_server/main.go +++ b/cmd/priv_val_server/main.go @@ -6,8 +6,9 @@ import ( "time" "github.com/tendermint/tendermint/crypto/ed25519" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + tmnet "github.com/tendermint/tendermint/libs/net" + tmos "github.com/tendermint/tendermint/libs/os" "github.com/tendermint/tendermint/privval" ) @@ -36,7 +37,7 @@ func main() { pv := privval.LoadFilePV(*privValKeyPath, *privValStatePath) var dialer privval.SocketDialer - protocol, address := cmn.ProtocolAndAddress(*addr) + protocol, address := tmnet.ProtocolAndAddress(*addr) switch protocol { case "unix": dialer = privval.DialUnixFn(address) @@ -57,7 +58,7 @@ func main() { } // Stop upon receiving SIGTERM or CTRL-C. - cmn.TrapSignal(logger, func() { + tmos.TrapSignal(logger, func() { err := ss.Stop() if err != nil { panic(err) diff --git a/cmd/tendermint/commands/gen_node_key.go b/cmd/tendermint/commands/gen_node_key.go index 38dc5c66d..e9f1f8103 100644 --- a/cmd/tendermint/commands/gen_node_key.go +++ b/cmd/tendermint/commands/gen_node_key.go @@ -5,7 +5,7 @@ import ( "github.com/spf13/cobra" - cmn "github.com/tendermint/tendermint/libs/common" + tmos "github.com/tendermint/tendermint/libs/os" "github.com/tendermint/tendermint/p2p" ) @@ -19,7 +19,7 @@ var GenNodeKeyCmd = &cobra.Command{ func genNodeKey(cmd *cobra.Command, args []string) error { nodeKeyFile := config.NodeKeyFile() - if cmn.FileExists(nodeKeyFile) { + if tmos.FileExists(nodeKeyFile) { return fmt.Errorf("node key at %s already exists", nodeKeyFile) } diff --git a/cmd/tendermint/commands/init.go b/cmd/tendermint/commands/init.go index 3d93ca3d1..c6903c388 100644 --- a/cmd/tendermint/commands/init.go +++ b/cmd/tendermint/commands/init.go @@ -5,7 +5,7 @@ import ( "github.com/spf13/cobra" cfg "github.com/tendermint/tendermint/config" - cmn "github.com/tendermint/tendermint/libs/common" + tmos "github.com/tendermint/tendermint/libs/os" "github.com/tendermint/tendermint/libs/rand" "github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/privval" @@ -29,7 +29,7 @@ func initFilesWithConfig(config *cfg.Config) error { privValKeyFile := config.PrivValidatorKeyFile() privValStateFile := config.PrivValidatorStateFile() var pv *privval.FilePV - if cmn.FileExists(privValKeyFile) { + if tmos.FileExists(privValKeyFile) { pv = privval.LoadFilePV(privValKeyFile, privValStateFile) logger.Info("Found private validator", "keyFile", privValKeyFile, "stateFile", privValStateFile) @@ -41,7 +41,7 @@ func initFilesWithConfig(config *cfg.Config) error { } nodeKeyFile := config.NodeKeyFile() - if cmn.FileExists(nodeKeyFile) { + if tmos.FileExists(nodeKeyFile) { logger.Info("Found node key", "path", nodeKeyFile) } else { if _, err := p2p.LoadOrGenNodeKey(nodeKeyFile); err != nil { @@ -52,7 +52,7 @@ func initFilesWithConfig(config *cfg.Config) error { // genesis file genFile := config.GenesisFile() - if cmn.FileExists(genFile) { + if tmos.FileExists(genFile) { logger.Info("Found genesis file", "path", genFile) } else { genDoc := types.GenesisDoc{ diff --git a/cmd/tendermint/commands/lite.go b/cmd/tendermint/commands/lite.go index 1cc82ab2e..a63daf20f 100644 --- a/cmd/tendermint/commands/lite.go +++ b/cmd/tendermint/commands/lite.go @@ -7,7 +7,7 @@ import ( "github.com/pkg/errors" "github.com/spf13/cobra" - cmn "github.com/tendermint/tendermint/libs/common" + tmos "github.com/tendermint/tendermint/libs/os" "github.com/tendermint/tendermint/lite/proxy" rpcclient "github.com/tendermint/tendermint/rpc/client" ) @@ -65,7 +65,7 @@ func EnsureAddrHasSchemeOrDefaultToTCP(addr string) (string, error) { func runProxy(cmd *cobra.Command, args []string) error { // Stop upon receiving SIGTERM or CTRL-C. - cmn.TrapSignal(logger, func() { + tmos.TrapSignal(logger, func() { // TODO: close up shop }) diff --git a/cmd/tendermint/commands/reset_priv_validator.go b/cmd/tendermint/commands/reset_priv_validator.go index 4e8bde8b4..eef837c60 100644 --- a/cmd/tendermint/commands/reset_priv_validator.go +++ b/cmd/tendermint/commands/reset_priv_validator.go @@ -5,8 +5,8 @@ import ( "github.com/spf13/cobra" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + tmos "github.com/tendermint/tendermint/libs/os" "github.com/tendermint/tendermint/privval" ) @@ -58,7 +58,7 @@ func ResetAll(dbDir, addrBookFile, privValKeyFile, privValStateFile string, logg logger.Error("Error removing all blockchain history", "dir", dbDir, "err", err) } // recreate the dbDir since the privVal state needs to live there - cmn.EnsureDir(dbDir, 0700) + tmos.EnsureDir(dbDir, 0700) resetFilePV(privValKeyFile, privValStateFile, logger) } diff --git a/cmd/tendermint/commands/root_test.go b/cmd/tendermint/commands/root_test.go index 229385af9..a39f92d38 100644 --- a/cmd/tendermint/commands/root_test.go +++ b/cmd/tendermint/commands/root_test.go @@ -15,7 +15,7 @@ import ( cfg "github.com/tendermint/tendermint/config" "github.com/tendermint/tendermint/libs/cli" - cmn "github.com/tendermint/tendermint/libs/common" + tmos "github.com/tendermint/tendermint/libs/os" ) var ( @@ -140,7 +140,7 @@ func TestRootConfig(t *testing.T) { // XXX: path must match cfg.defaultConfigPath configFilePath := filepath.Join(defaultRoot, "config") - err := cmn.EnsureDir(configFilePath, 0700) + err := tmos.EnsureDir(configFilePath, 0700) require.Nil(t, err) // write the non-defaults to a different path diff --git a/cmd/tendermint/commands/run_node.go b/cmd/tendermint/commands/run_node.go index 577706461..6d9f575d0 100644 --- a/cmd/tendermint/commands/run_node.go +++ b/cmd/tendermint/commands/run_node.go @@ -11,7 +11,7 @@ import ( "github.com/spf13/cobra" cfg "github.com/tendermint/tendermint/config" - cmn "github.com/tendermint/tendermint/libs/common" + tmos "github.com/tendermint/tendermint/libs/os" nm "github.com/tendermint/tendermint/node" ) @@ -105,7 +105,7 @@ func NewRunNodeCmd(nodeProvider nm.Provider) *cobra.Command { } // Stop upon receiving SIGTERM or CTRL-C. - cmn.TrapSignal(logger, func() { + tmos.TrapSignal(logger, func() { if n.IsRunning() { n.Stop() } diff --git a/cmd/tendermint/commands/show_validator.go b/cmd/tendermint/commands/show_validator.go index d76d21970..b0c673373 100644 --- a/cmd/tendermint/commands/show_validator.go +++ b/cmd/tendermint/commands/show_validator.go @@ -6,7 +6,7 @@ import ( "github.com/pkg/errors" "github.com/spf13/cobra" - cmn "github.com/tendermint/tendermint/libs/common" + tmos "github.com/tendermint/tendermint/libs/os" "github.com/tendermint/tendermint/privval" ) @@ -19,7 +19,7 @@ var ShowValidatorCmd = &cobra.Command{ func showValidator(cmd *cobra.Command, args []string) error { keyFilePath := config.PrivValidatorKeyFile() - if !cmn.FileExists(keyFilePath) { + if !tmos.FileExists(keyFilePath) { return fmt.Errorf("private validator file %s does not exist", keyFilePath) } diff --git a/config/toml.go b/config/toml.go index c561ecc69..609f1487a 100644 --- a/config/toml.go +++ b/config/toml.go @@ -7,7 +7,7 @@ import ( "path/filepath" "text/template" - cmn "github.com/tendermint/tendermint/libs/common" + tmos "github.com/tendermint/tendermint/libs/os" ) // DefaultDirPerm is the default permissions used when creating directories. @@ -27,20 +27,20 @@ func init() { // EnsureRoot creates the root, config, and data directories if they don't exist, // and panics if it fails. func EnsureRoot(rootDir string) { - if err := cmn.EnsureDir(rootDir, DefaultDirPerm); err != nil { + if err := tmos.EnsureDir(rootDir, DefaultDirPerm); err != nil { panic(err.Error()) } - if err := cmn.EnsureDir(filepath.Join(rootDir, defaultConfigDir), DefaultDirPerm); err != nil { + if err := tmos.EnsureDir(filepath.Join(rootDir, defaultConfigDir), DefaultDirPerm); err != nil { panic(err.Error()) } - if err := cmn.EnsureDir(filepath.Join(rootDir, defaultDataDir), DefaultDirPerm); err != nil { + if err := tmos.EnsureDir(filepath.Join(rootDir, defaultDataDir), DefaultDirPerm); err != nil { panic(err.Error()) } configFilePath := filepath.Join(rootDir, defaultConfigFilePath) // Write default config file if missing. - if !cmn.FileExists(configFilePath) { + if !tmos.FileExists(configFilePath) { writeDefaultConfigFile(configFilePath) } } @@ -59,7 +59,7 @@ func WriteConfigFile(configFilePath string, config *Config) { panic(err) } - cmn.MustWriteFile(configFilePath, buffer.Bytes(), 0644) + tmos.MustWriteFile(configFilePath, buffer.Bytes(), 0644) } // Note: any changes to the comments/variables/mapstructure @@ -413,10 +413,10 @@ func ResetTestRootWithChainID(testName string, chainID string) *Config { panic(err) } // ensure config and data subdirs are created - if err := cmn.EnsureDir(filepath.Join(rootDir, defaultConfigDir), DefaultDirPerm); err != nil { + if err := tmos.EnsureDir(filepath.Join(rootDir, defaultConfigDir), DefaultDirPerm); err != nil { panic(err) } - if err := cmn.EnsureDir(filepath.Join(rootDir, defaultDataDir), DefaultDirPerm); err != nil { + if err := tmos.EnsureDir(filepath.Join(rootDir, defaultDataDir), DefaultDirPerm); err != nil { panic(err) } @@ -427,19 +427,19 @@ func ResetTestRootWithChainID(testName string, chainID string) *Config { privStateFilePath := filepath.Join(rootDir, baseConfig.PrivValidatorState) // Write default config file if missing. - if !cmn.FileExists(configFilePath) { + if !tmos.FileExists(configFilePath) { writeDefaultConfigFile(configFilePath) } - if !cmn.FileExists(genesisFilePath) { + if !tmos.FileExists(genesisFilePath) { if chainID == "" { chainID = "tendermint_test" } testGenesis := fmt.Sprintf(testGenesisFmt, chainID) - cmn.MustWriteFile(genesisFilePath, []byte(testGenesis), 0644) + tmos.MustWriteFile(genesisFilePath, []byte(testGenesis), 0644) } // we always overwrite the priv val - cmn.MustWriteFile(privKeyFilePath, []byte(testPrivValidatorKey), 0644) - cmn.MustWriteFile(privStateFilePath, []byte(testPrivValidatorState), 0644) + tmos.MustWriteFile(privKeyFilePath, []byte(testPrivValidatorKey), 0644) + tmos.MustWriteFile(privStateFilePath, []byte(testPrivValidatorState), 0644) config := TestConfig().SetRoot(rootDir) return config diff --git a/consensus/common_test.go b/consensus/common_test.go index 9f5cac40b..c6cd18193 100644 --- a/consensus/common_test.go +++ b/consensus/common_test.go @@ -23,8 +23,8 @@ import ( cfg "github.com/tendermint/tendermint/config" cstypes "github.com/tendermint/tendermint/consensus/types" tmbytes "github.com/tendermint/tendermint/libs/bytes" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + tmos "github.com/tendermint/tendermint/libs/os" tmpubsub "github.com/tendermint/tendermint/libs/pubsub" mempl "github.com/tendermint/tendermint/mempool" "github.com/tendermint/tendermint/p2p" @@ -50,7 +50,7 @@ var consensusReplayConfig *cfg.Config var ensureTimeout = time.Millisecond * 100 func ensureDir(dir string, mode os.FileMode) { - if err := cmn.EnsureDir(dir, mode); err != nil { + if err := tmos.EnsureDir(dir, mode); err != nil { panic(err) } } diff --git a/consensus/reactor.go b/consensus/reactor.go index 8af2abaff..40868ff9d 100644 --- a/consensus/reactor.go +++ b/consensus/reactor.go @@ -10,7 +10,7 @@ import ( amino "github.com/tendermint/go-amino" cstypes "github.com/tendermint/tendermint/consensus/types" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/bits" tmevents "github.com/tendermint/tendermint/libs/events" "github.com/tendermint/tendermint/libs/log" "github.com/tendermint/tendermint/p2p" @@ -262,7 +262,7 @@ func (conR *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) { } // Respond with a VoteSetBitsMessage showing which votes we have. // (and consequently shows which we don't have) - var ourVotes *cmn.BitArray + var ourVotes *bits.BitArray switch msg.Type { case types.PrevoteType: ourVotes = votes.Prevotes(msg.Round).BitArrayByBlockID(msg.BlockID) @@ -336,7 +336,7 @@ func (conR *Reactor) Receive(chID byte, src p2p.Peer, msgBytes []byte) { cs.mtx.Unlock() if height == msg.Height { - var ourVotes *cmn.BitArray + var ourVotes *bits.BitArray switch msg.Type { case types.PrevoteType: ourVotes = votes.Prevotes(msg.Round).BitArrayByBlockID(msg.BlockID) @@ -998,7 +998,7 @@ func (ps *PeerState) SetHasProposal(proposal *types.Proposal) { } ps.PRS.ProposalBlockPartsHeader = proposal.BlockID.PartsHeader - ps.PRS.ProposalBlockParts = cmn.NewBitArray(proposal.BlockID.PartsHeader.Total) + ps.PRS.ProposalBlockParts = bits.NewBitArray(proposal.BlockID.PartsHeader.Total) ps.PRS.ProposalPOLRound = proposal.POLRound ps.PRS.ProposalPOL = nil // Nil until ProposalPOLMessage received. } @@ -1013,7 +1013,7 @@ func (ps *PeerState) InitProposalBlockParts(partsHeader types.PartSetHeader) { } ps.PRS.ProposalBlockPartsHeader = partsHeader - ps.PRS.ProposalBlockParts = cmn.NewBitArray(partsHeader.Total) + ps.PRS.ProposalBlockParts = bits.NewBitArray(partsHeader.Total) } // SetHasProposalBlockPart sets the given block part index as known for the peer. @@ -1072,7 +1072,7 @@ func (ps *PeerState) PickVoteToSend(votes types.VoteSetReader) (vote *types.Vote return nil, false } -func (ps *PeerState) getVoteBitArray(height int64, round int, votesType types.SignedMsgType) *cmn.BitArray { +func (ps *PeerState) getVoteBitArray(height int64, round int, votesType types.SignedMsgType) *bits.BitArray { if !types.IsVoteTypeValid(votesType) { return nil } @@ -1143,7 +1143,7 @@ func (ps *PeerState) ensureCatchupCommitRound(height int64, round int, numValida if round == ps.PRS.Round { ps.PRS.CatchupCommit = ps.PRS.Precommits } else { - ps.PRS.CatchupCommit = cmn.NewBitArray(numValidators) + ps.PRS.CatchupCommit = bits.NewBitArray(numValidators) } } @@ -1160,20 +1160,20 @@ func (ps *PeerState) EnsureVoteBitArrays(height int64, numValidators int) { func (ps *PeerState) ensureVoteBitArrays(height int64, numValidators int) { if ps.PRS.Height == height { if ps.PRS.Prevotes == nil { - ps.PRS.Prevotes = cmn.NewBitArray(numValidators) + ps.PRS.Prevotes = bits.NewBitArray(numValidators) } if ps.PRS.Precommits == nil { - ps.PRS.Precommits = cmn.NewBitArray(numValidators) + ps.PRS.Precommits = bits.NewBitArray(numValidators) } if ps.PRS.CatchupCommit == nil { - ps.PRS.CatchupCommit = cmn.NewBitArray(numValidators) + ps.PRS.CatchupCommit = bits.NewBitArray(numValidators) } if ps.PRS.ProposalPOL == nil { - ps.PRS.ProposalPOL = cmn.NewBitArray(numValidators) + ps.PRS.ProposalPOL = bits.NewBitArray(numValidators) } } else if ps.PRS.Height == height+1 { if ps.PRS.LastCommit == nil { - ps.PRS.LastCommit = cmn.NewBitArray(numValidators) + ps.PRS.LastCommit = bits.NewBitArray(numValidators) } } } @@ -1343,7 +1343,7 @@ func (ps *PeerState) ApplyHasVoteMessage(msg *HasVoteMessage) { // `ourVotes` is a BitArray of votes we have for msg.BlockID // NOTE: if ourVotes is nil (e.g. msg.Height < rs.Height), // we conservatively overwrite ps's votes w/ msg.Votes. -func (ps *PeerState) ApplyVoteSetBitsMessage(msg *VoteSetBitsMessage, ourVotes *cmn.BitArray) { +func (ps *PeerState) ApplyVoteSetBitsMessage(msg *VoteSetBitsMessage, ourVotes *bits.BitArray) { ps.mtx.Lock() defer ps.mtx.Unlock() @@ -1456,7 +1456,7 @@ type NewValidBlockMessage struct { Height int64 Round int BlockPartsHeader types.PartSetHeader - BlockParts *cmn.BitArray + BlockParts *bits.BitArray IsCommit bool } @@ -1514,7 +1514,7 @@ func (m *ProposalMessage) String() string { type ProposalPOLMessage struct { Height int64 ProposalPOLRound int - ProposalPOL *cmn.BitArray + ProposalPOL *bits.BitArray } // ValidateBasic performs basic validation. @@ -1656,7 +1656,7 @@ type VoteSetBitsMessage struct { Round int Type types.SignedMsgType BlockID types.BlockID - Votes *cmn.BitArray + Votes *bits.BitArray } // ValidateBasic performs basic validation. diff --git a/consensus/reactor_test.go b/consensus/reactor_test.go index c0e05ee5f..fb4927754 100644 --- a/consensus/reactor_test.go +++ b/consensus/reactor_test.go @@ -20,8 +20,8 @@ import ( cfg "github.com/tendermint/tendermint/config" cstypes "github.com/tendermint/tendermint/consensus/types" "github.com/tendermint/tendermint/crypto/tmhash" + "github.com/tendermint/tendermint/libs/bits" "github.com/tendermint/tendermint/libs/bytes" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" mempl "github.com/tendermint/tendermint/mempool" "github.com/tendermint/tendermint/p2p" @@ -712,11 +712,11 @@ func TestNewValidBlockMessageValidateBasic(t *testing.T) { "blockParts bit array size 1 not equal to BlockPartsHeader.Total 2", }, { - func(msg *NewValidBlockMessage) { msg.BlockPartsHeader.Total = 0; msg.BlockParts = cmn.NewBitArray(0) }, + func(msg *NewValidBlockMessage) { msg.BlockPartsHeader.Total = 0; msg.BlockParts = bits.NewBitArray(0) }, "empty blockParts", }, { - func(msg *NewValidBlockMessage) { msg.BlockParts = cmn.NewBitArray(types.MaxBlockPartsCount + 1) }, + func(msg *NewValidBlockMessage) { msg.BlockParts = bits.NewBitArray(types.MaxBlockPartsCount + 1) }, "blockParts bit array size 1602 not equal to BlockPartsHeader.Total 1", }, } @@ -730,7 +730,7 @@ func TestNewValidBlockMessageValidateBasic(t *testing.T) { BlockPartsHeader: types.PartSetHeader{ Total: 1, }, - BlockParts: cmn.NewBitArray(1), + BlockParts: bits.NewBitArray(1), } tc.malleateFn(msg) @@ -750,8 +750,8 @@ func TestProposalPOLMessageValidateBasic(t *testing.T) { {func(msg *ProposalPOLMessage) {}, ""}, {func(msg *ProposalPOLMessage) { msg.Height = -1 }, "negative Height"}, {func(msg *ProposalPOLMessage) { msg.ProposalPOLRound = -1 }, "negative ProposalPOLRound"}, - {func(msg *ProposalPOLMessage) { msg.ProposalPOL = cmn.NewBitArray(0) }, "empty ProposalPOL bit array"}, - {func(msg *ProposalPOLMessage) { msg.ProposalPOL = cmn.NewBitArray(types.MaxVotesCount + 1) }, + {func(msg *ProposalPOLMessage) { msg.ProposalPOL = bits.NewBitArray(0) }, "empty ProposalPOL bit array"}, + {func(msg *ProposalPOLMessage) { msg.ProposalPOL = bits.NewBitArray(types.MaxVotesCount + 1) }, "ProposalPOL bit array is too big: 10001, max: 10000"}, } @@ -761,7 +761,7 @@ func TestProposalPOLMessageValidateBasic(t *testing.T) { msg := &ProposalPOLMessage{ Height: 1, ProposalPOLRound: 1, - ProposalPOL: cmn.NewBitArray(1), + ProposalPOL: bits.NewBitArray(1), } tc.malleateFn(msg) @@ -906,7 +906,7 @@ func TestVoteSetBitsMessageValidateBasic(t *testing.T) { }, } }, "wrong BlockID: wrong PartsHeader: negative Total"}, - {func(msg *VoteSetBitsMessage) { msg.Votes = cmn.NewBitArray(types.MaxVotesCount + 1) }, + {func(msg *VoteSetBitsMessage) { msg.Votes = bits.NewBitArray(types.MaxVotesCount + 1) }, "votes bit array is too big: 10001, max: 10000"}, } @@ -917,7 +917,7 @@ func TestVoteSetBitsMessageValidateBasic(t *testing.T) { Height: 1, Round: 0, Type: 0x01, - Votes: cmn.NewBitArray(1), + Votes: bits.NewBitArray(1), BlockID: types.BlockID{}, } diff --git a/consensus/replay_file.go b/consensus/replay_file.go index 9477f9c4c..b8b8c51da 100644 --- a/consensus/replay_file.go +++ b/consensus/replay_file.go @@ -13,8 +13,8 @@ import ( dbm "github.com/tendermint/tm-db" cfg "github.com/tendermint/tendermint/config" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + tmos "github.com/tendermint/tendermint/libs/os" "github.com/tendermint/tendermint/mock" "github.com/tendermint/tendermint/proxy" sm "github.com/tendermint/tendermint/state" @@ -35,7 +35,7 @@ func RunReplayFile(config cfg.BaseConfig, csConfig *cfg.ConsensusConfig, console consensusState := newConsensusStateForReplay(config, csConfig) if err := consensusState.ReplayFile(csConfig.WalFile(), console); err != nil { - cmn.Exit(fmt.Sprintf("Error during consensus replay: %v", err)) + tmos.Exit(fmt.Sprintf("Error during consensus replay: %v", err)) } } @@ -180,9 +180,9 @@ func (pb *playback) replayConsoleLoop() int { bufReader := bufio.NewReader(os.Stdin) line, more, err := bufReader.ReadLine() if more { - cmn.Exit("input is too long") + tmos.Exit("input is too long") } else if err != nil { - cmn.Exit(err.Error()) + tmos.Exit(err.Error()) } tokens := strings.Split(string(line), " ") @@ -217,7 +217,7 @@ func (pb *playback) replayConsoleLoop() int { newStepSub, err := pb.cs.eventBus.Subscribe(ctx, subscriber, types.EventQueryNewRoundStep) if err != nil { - cmn.Exit(fmt.Sprintf("failed to subscribe %s to %v", subscriber, types.EventQueryNewRoundStep)) + tmos.Exit(fmt.Sprintf("failed to subscribe %s to %v", subscriber, types.EventQueryNewRoundStep)) } defer pb.cs.eventBus.Unsubscribe(ctx, subscriber, types.EventQueryNewRoundStep) @@ -284,11 +284,11 @@ func newConsensusStateForReplay(config cfg.BaseConfig, csConfig *cfg.ConsensusCo stateDB := dbm.NewDB("state", dbType, config.DBDir()) gdoc, err := sm.MakeGenesisDocFromFile(config.GenesisFile()) if err != nil { - cmn.Exit(err.Error()) + tmos.Exit(err.Error()) } state, err := sm.MakeGenesisState(gdoc) if err != nil { - cmn.Exit(err.Error()) + tmos.Exit(err.Error()) } // Create proxyAppConn connection (consensus, mempool, query) @@ -296,19 +296,19 @@ func newConsensusStateForReplay(config cfg.BaseConfig, csConfig *cfg.ConsensusCo proxyApp := proxy.NewAppConns(clientCreator) err = proxyApp.Start() if err != nil { - cmn.Exit(fmt.Sprintf("Error starting proxy app conns: %v", err)) + tmos.Exit(fmt.Sprintf("Error starting proxy app conns: %v", err)) } eventBus := types.NewEventBus() if err := eventBus.Start(); err != nil { - cmn.Exit(fmt.Sprintf("Failed to start event bus: %v", err)) + tmos.Exit(fmt.Sprintf("Failed to start event bus: %v", err)) } handshaker := NewHandshaker(stateDB, state, blockStore, gdoc) handshaker.SetEventBus(eventBus) err = handshaker.Handshake(proxyApp) if err != nil { - cmn.Exit(fmt.Sprintf("Error on handshake: %v", err)) + tmos.Exit(fmt.Sprintf("Error on handshake: %v", err)) } mempool, evpool := mock.Mempool{}, sm.MockEvidencePool{} diff --git a/consensus/state.go b/consensus/state.go index f7395cba7..6cf9ab3da 100644 --- a/consensus/state.go +++ b/consensus/state.go @@ -10,9 +10,9 @@ import ( "github.com/pkg/errors" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/fail" "github.com/tendermint/tendermint/libs/log" + tmos "github.com/tendermint/tendermint/libs/os" "github.com/tendermint/tendermint/libs/service" tmtime "github.com/tendermint/tendermint/types/time" @@ -1434,7 +1434,7 @@ func (cs *State) finalizeCommit(height int64) { block) if err != nil { cs.Logger.Error("Error on ApplyBlock. Did the application crash? Please restart tendermint", "err", err) - err := cmn.Kill() + err := tmos.Kill() if err != nil { cs.Logger.Error("Failed to kill this process - please do so manually", "err", err) } diff --git a/consensus/types/peer_round_state.go b/consensus/types/peer_round_state.go index 41b5bd81b..937f597fe 100644 --- a/consensus/types/peer_round_state.go +++ b/consensus/types/peer_round_state.go @@ -4,7 +4,7 @@ import ( "fmt" "time" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/bits" "github.com/tendermint/tendermint/types" ) @@ -23,21 +23,21 @@ type PeerRoundState struct { // True if peer has proposal for this round Proposal bool `json:"proposal"` ProposalBlockPartsHeader types.PartSetHeader `json:"proposal_block_parts_header"` // - ProposalBlockParts *cmn.BitArray `json:"proposal_block_parts"` // + ProposalBlockParts *bits.BitArray `json:"proposal_block_parts"` // ProposalPOLRound int `json:"proposal_pol_round"` // Proposal's POL round. -1 if none. // nil until ProposalPOLMessage received. - ProposalPOL *cmn.BitArray `json:"proposal_pol"` - Prevotes *cmn.BitArray `json:"prevotes"` // All votes peer has for this round - Precommits *cmn.BitArray `json:"precommits"` // All precommits peer has for this round - LastCommitRound int `json:"last_commit_round"` // Round of commit for last height. -1 if none. - LastCommit *cmn.BitArray `json:"last_commit"` // All commit precommits of commit for last height. + ProposalPOL *bits.BitArray `json:"proposal_pol"` + Prevotes *bits.BitArray `json:"prevotes"` // All votes peer has for this round + Precommits *bits.BitArray `json:"precommits"` // All precommits peer has for this round + LastCommitRound int `json:"last_commit_round"` // Round of commit for last height. -1 if none. + LastCommit *bits.BitArray `json:"last_commit"` // All commit precommits of commit for last height. // Round that we have commit for. Not necessarily unique. -1 if none. CatchupCommitRound int `json:"catchup_commit_round"` // All commit precommits peer has for this height & CatchupCommitRound - CatchupCommit *cmn.BitArray `json:"catchup_commit"` + CatchupCommit *bits.BitArray `json:"catchup_commit"` } // String returns a string representation of the PeerRoundState diff --git a/consensus/wal.go b/consensus/wal.go index c00df13e7..64f04fee0 100644 --- a/consensus/wal.go +++ b/consensus/wal.go @@ -12,8 +12,8 @@ import ( amino "github.com/tendermint/go-amino" auto "github.com/tendermint/tendermint/libs/autofile" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + tmos "github.com/tendermint/tendermint/libs/os" "github.com/tendermint/tendermint/libs/service" "github.com/tendermint/tendermint/types" tmtime "github.com/tendermint/tendermint/types/time" @@ -94,7 +94,7 @@ var _ WAL = &baseWAL{} // NewWAL returns a new write-ahead logger based on `baseWAL`, which implements // WAL. It's flushed and synced to disk every 2s and once when stopped. func NewWAL(walFile string, groupOptions ...func(*auto.Group)) (*baseWAL, error) { - err := cmn.EnsureDir(filepath.Dir(walFile), 0700) + err := tmos.EnsureDir(filepath.Dir(walFile), 0700) if err != nil { return nil, errors.Wrap(err, "failed to ensure WAL directory is in place") } diff --git a/docs/app-dev/abci-cli.md b/docs/app-dev/abci-cli.md index 20f1b70c2..a72dc220f 100644 --- a/docs/app-dev/abci-cli.md +++ b/docs/app-dev/abci-cli.md @@ -93,7 +93,7 @@ func cmdKVStore(cmd *cobra.Command, args []string) error { } // Stop upon receiving SIGTERM or CTRL-C. - cmn.TrapSignal(logger, func() { + tmos.TrapSignal(logger, func() { // Cleanup srv.Stop() }) @@ -244,7 +244,7 @@ func cmdCounter(cmd *cobra.Command, args []string) error { } // Stop upon receiving SIGTERM or CTRL-C. - cmn.TrapSignal(logger, func() { + tmos.TrapSignal(logger, func() { // Cleanup srv.Stop() }) diff --git a/libs/common/async.go b/libs/async/async.go similarity index 99% rename from libs/common/async.go rename to libs/async/async.go index 7f9c4e872..e2487661a 100644 --- a/libs/common/async.go +++ b/libs/async/async.go @@ -1,4 +1,4 @@ -package common +package async import ( "sync/atomic" diff --git a/libs/common/async_test.go b/libs/async/async_test.go similarity index 99% rename from libs/common/async_test.go rename to libs/async/async_test.go index 276fe886e..ed85827b8 100644 --- a/libs/common/async_test.go +++ b/libs/async/async_test.go @@ -1,4 +1,4 @@ -package common +package async import ( "fmt" diff --git a/libs/autofile/autofile_test.go b/libs/autofile/autofile_test.go index 921a317e9..664264cdc 100644 --- a/libs/autofile/autofile_test.go +++ b/libs/autofile/autofile_test.go @@ -9,7 +9,7 @@ import ( "github.com/stretchr/testify/require" - cmn "github.com/tendermint/tendermint/libs/common" + tmos "github.com/tendermint/tendermint/libs/os" ) func TestSIGHUP(t *testing.T) { @@ -49,10 +49,10 @@ func TestSIGHUP(t *testing.T) { require.NoError(t, err) // Both files should exist - if body := cmn.MustReadFile(name + "_old"); string(body) != "Line 1\nLine 2\n" { + if body := tmos.MustReadFile(name + "_old"); string(body) != "Line 1\nLine 2\n" { t.Errorf("unexpected body %s", body) } - if body := cmn.MustReadFile(name); string(body) != "Line 3\nLine 4\n" { + if body := tmos.MustReadFile(name); string(body) != "Line 3\nLine 4\n" { t.Errorf("unexpected body %s", body) } } diff --git a/libs/autofile/cmd/logjack.go b/libs/autofile/cmd/logjack.go index 20002eff3..ca970ffa2 100644 --- a/libs/autofile/cmd/logjack.go +++ b/libs/autofile/cmd/logjack.go @@ -9,7 +9,7 @@ import ( "strings" auto "github.com/tendermint/tendermint/libs/autofile" - cmn "github.com/tendermint/tendermint/libs/common" + tmos "github.com/tendermint/tendermint/libs/os" ) const Version = "0.0.1" @@ -41,7 +41,7 @@ func (fmtLogger) Info(msg string, keyvals ...interface{}) { func main() { // Stop upon receiving SIGTERM or CTRL-C. - cmn.TrapSignal(fmtLogger{}, func() { + tmos.TrapSignal(fmtLogger{}, func() { fmt.Println("logjack shutting down") }) diff --git a/libs/autofile/group_test.go b/libs/autofile/group_test.go index 2029e56dc..381740be6 100644 --- a/libs/autofile/group_test.go +++ b/libs/autofile/group_test.go @@ -9,14 +9,14 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - cmn "github.com/tendermint/tendermint/libs/common" + tmos "github.com/tendermint/tendermint/libs/os" "github.com/tendermint/tendermint/libs/rand" ) func createTestGroupWithHeadSizeLimit(t *testing.T, headSizeLimit int64) *Group { testID := rand.RandStr(12) testDir := "_test_" + testID - err := cmn.EnsureDir(testDir, 0700) + err := tmos.EnsureDir(testDir, 0700) require.NoError(t, err, "Error creating dir") headPath := testDir + "/myfile" diff --git a/libs/common/bit_array.go b/libs/bits/bit_array.go similarity index 97% rename from libs/common/bit_array.go rename to libs/bits/bit_array.go index 08bc0ee85..0894802d3 100644 --- a/libs/common/bit_array.go +++ b/libs/bits/bit_array.go @@ -1,4 +1,4 @@ -package common +package bits import ( "encoding/binary" @@ -7,6 +7,7 @@ import ( "strings" "sync" + tmmath "github.com/tendermint/tendermint/libs/math" "github.com/tendermint/tendermint/libs/rand" ) @@ -121,8 +122,8 @@ func (bA *BitArray) Or(o *BitArray) *BitArray { } bA.mtx.Lock() o.mtx.Lock() - c := bA.copyBits(MaxInt(bA.Bits, o.Bits)) - smaller := MinInt(len(bA.Elems), len(o.Elems)) + c := bA.copyBits(tmmath.MaxInt(bA.Bits, o.Bits)) + smaller := tmmath.MinInt(len(bA.Elems), len(o.Elems)) for i := 0; i < smaller; i++ { c.Elems[i] |= o.Elems[i] } @@ -148,7 +149,7 @@ func (bA *BitArray) And(o *BitArray) *BitArray { } func (bA *BitArray) and(o *BitArray) *BitArray { - c := bA.copyBits(MinInt(bA.Bits, o.Bits)) + c := bA.copyBits(tmmath.MinInt(bA.Bits, o.Bits)) for i := 0; i < len(c.Elems); i++ { c.Elems[i] &= o.Elems[i] } @@ -190,7 +191,7 @@ func (bA *BitArray) Sub(o *BitArray) *BitArray { // If o is longer, those bits are ignored. // If bA is longer, then skipping those iterations is equivalent // to right padding with 0's - smaller := MinInt(len(bA.Elems), len(o.Elems)) + smaller := tmmath.MinInt(len(bA.Elems), len(o.Elems)) for i := 0; i < smaller; i++ { // &^ is and not in golang c.Elems[i] &^= o.Elems[i] diff --git a/libs/common/bit_array_test.go b/libs/bits/bit_array_test.go similarity index 99% rename from libs/common/bit_array_test.go rename to libs/bits/bit_array_test.go index ee88bb6ca..3f26efd0c 100644 --- a/libs/common/bit_array_test.go +++ b/libs/bits/bit_array_test.go @@ -1,4 +1,4 @@ -package common +package bits import ( "bytes" diff --git a/libs/common/cmap.go b/libs/cmap/cmap.go similarity index 98% rename from libs/common/cmap.go rename to libs/cmap/cmap.go index d87adb762..da2275d7d 100644 --- a/libs/common/cmap.go +++ b/libs/cmap/cmap.go @@ -1,4 +1,4 @@ -package common +package cmap import "sync" diff --git a/libs/common/cmap_test.go b/libs/cmap/cmap_test.go similarity index 98% rename from libs/common/cmap_test.go rename to libs/cmap/cmap_test.go index 373057cb9..b6ea0117d 100644 --- a/libs/common/cmap_test.go +++ b/libs/cmap/cmap_test.go @@ -1,4 +1,4 @@ -package common +package cmap import ( "fmt" diff --git a/libs/common/LICENSE b/libs/common/LICENSE deleted file mode 100644 index 8a142a71b..000000000 --- a/libs/common/LICENSE +++ /dev/null @@ -1,193 +0,0 @@ -Tendermint Go-Common -Copyright (C) 2015 Tendermint - - - - Apache License - Version 2.0, January 2004 - https://www.apache.org/licenses/ - - TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION - - 1. Definitions. - - "License" shall mean the terms and conditions for use, reproduction, - and distribution as defined by Sections 1 through 9 of this document. - - "Licensor" shall mean the copyright owner or entity authorized by - the copyright owner that is granting the License. - - "Legal Entity" shall mean the union of the acting entity and all - other entities that control, are controlled by, or are under common - control with that entity. For the purposes of this definition, - "control" means (i) the power, direct or indirect, to cause the - direction or management of such entity, whether by contract or - otherwise, or (ii) ownership of fifty percent (50%) or more of the - outstanding shares, or (iii) beneficial ownership of such entity. - - "You" (or "Your") shall mean an individual or Legal Entity - exercising permissions granted by this License. - - "Source" form shall mean the preferred form for making modifications, - including but not limited to software source code, documentation - source, and configuration files. - - "Object" form shall mean any form resulting from mechanical - transformation or translation of a Source form, including but - not limited to compiled object code, generated documentation, - and conversions to other media types. - - "Work" shall mean the work of authorship, whether in Source or - Object form, made available under the License, as indicated by a - copyright notice that is included in or attached to the work - (an example is provided in the Appendix below). - - "Derivative Works" shall mean any work, whether in Source or Object - form, that is based on (or derived from) the Work and for which the - editorial revisions, annotations, elaborations, or other modifications - represent, as a whole, an original work of authorship. For the purposes - of this License, Derivative Works shall not include works that remain - separable from, or merely link (or bind by name) to the interfaces of, - the Work and Derivative Works thereof. - - "Contribution" shall mean any work of authorship, including - the original version of the Work and any modifications or additions - to that Work or Derivative Works thereof, that is intentionally - submitted to Licensor for inclusion in the Work by the copyright owner - or by an individual or Legal Entity authorized to submit on behalf of - the copyright owner. For the purposes of this definition, "submitted" - means any form of electronic, verbal, or written communication sent - to the Licensor or its representatives, including but not limited to - communication on electronic mailing lists, source code control systems, - and issue tracking systems that are managed by, or on behalf of, the - Licensor for the purpose of discussing and improving the Work, but - excluding communication that is conspicuously marked or otherwise - designated in writing by the copyright owner as "Not a Contribution." - - "Contributor" shall mean Licensor and any individual or Legal Entity - on behalf of whom a Contribution has been received by Licensor and - subsequently incorporated within the Work. - - 2. Grant of Copyright License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - copyright license to reproduce, prepare Derivative Works of, - publicly display, publicly perform, sublicense, and distribute the - Work and such Derivative Works in Source or Object form. - - 3. Grant of Patent License. Subject to the terms and conditions of - this License, each Contributor hereby grants to You a perpetual, - worldwide, non-exclusive, no-charge, royalty-free, irrevocable - (except as stated in this section) patent license to make, have made, - use, offer to sell, sell, import, and otherwise transfer the Work, - where such license applies only to those patent claims licensable - by such Contributor that are necessarily infringed by their - Contribution(s) alone or by combination of their Contribution(s) - with the Work to which such Contribution(s) was submitted. If You - institute patent litigation against any entity (including a - cross-claim or counterclaim in a lawsuit) alleging that the Work - or a Contribution incorporated within the Work constitutes direct - or contributory patent infringement, then any patent licenses - granted to You under this License for that Work shall terminate - as of the date such litigation is filed. - - 4. Redistribution. You may reproduce and distribute copies of the - Work or Derivative Works thereof in any medium, with or without - modifications, and in Source or Object form, provided that You - meet the following conditions: - - (a) You must give any other recipients of the Work or - Derivative Works a copy of this License; and - - (b) You must cause any modified files to carry prominent notices - stating that You changed the files; and - - (c) You must retain, in the Source form of any Derivative Works - that You distribute, all copyright, patent, trademark, and - attribution notices from the Source form of the Work, - excluding those notices that do not pertain to any part of - the Derivative Works; and - - (d) If the Work includes a "NOTICE" text file as part of its - distribution, then any Derivative Works that You distribute must - include a readable copy of the attribution notices contained - within such NOTICE file, excluding those notices that do not - pertain to any part of the Derivative Works, in at least one - of the following places: within a NOTICE text file distributed - as part of the Derivative Works; within the Source form or - documentation, if provided along with the Derivative Works; or, - within a display generated by the Derivative Works, if and - wherever such third-party notices normally appear. The contents - of the NOTICE file are for informational purposes only and - do not modify the License. You may add Your own attribution - notices within Derivative Works that You distribute, alongside - or as an addendum to the NOTICE text from the Work, provided - that such additional attribution notices cannot be construed - as modifying the License. - - You may add Your own copyright statement to Your modifications and - may provide additional or different license terms and conditions - for use, reproduction, or distribution of Your modifications, or - for any such Derivative Works as a whole, provided Your use, - reproduction, and distribution of the Work otherwise complies with - the conditions stated in this License. - - 5. Submission of Contributions. Unless You explicitly state otherwise, - any Contribution intentionally submitted for inclusion in the Work - by You to the Licensor shall be under the terms and conditions of - this License, without any additional terms or conditions. - Notwithstanding the above, nothing herein shall supersede or modify - the terms of any separate license agreement you may have executed - with Licensor regarding such Contributions. - - 6. Trademarks. This License does not grant permission to use the trade - names, trademarks, service marks, or product names of the Licensor, - except as required for reasonable and customary use in describing the - origin of the Work and reproducing the content of the NOTICE file. - - 7. Disclaimer of Warranty. Unless required by applicable law or - agreed to in writing, Licensor provides the Work (and each - Contributor provides its Contributions) on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or - implied, including, without limitation, any warranties or conditions - of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A - PARTICULAR PURPOSE. You are solely responsible for determining the - appropriateness of using or redistributing the Work and assume any - risks associated with Your exercise of permissions under this License. - - 8. Limitation of Liability. In no event and under no legal theory, - whether in tort (including negligence), contract, or otherwise, - unless required by applicable law (such as deliberate and grossly - negligent acts) or agreed to in writing, shall any Contributor be - liable to You for damages, including any direct, indirect, special, - incidental, or consequential damages of any character arising as a - result of this License or out of the use or inability to use the - Work (including but not limited to damages for loss of goodwill, - work stoppage, computer failure or malfunction, or any and all - other commercial damages or losses), even if such Contributor - has been advised of the possibility of such damages. - - 9. Accepting Warranty or Additional Liability. While redistributing - the Work or Derivative Works thereof, You may choose to offer, - and charge a fee for, acceptance of support, warranty, indemnity, - or other liability obligations and/or rights consistent with this - License. However, in accepting such obligations, You may act only - on Your own behalf and on Your sole responsibility, not on behalf - of any other Contributor, and only if You agree to indemnify, - defend, and hold each Contributor harmless for any liability - incurred by, or claims asserted against, such Contributor by reason - of your accepting any such warranty or additional liability. - - END OF TERMS AND CONDITIONS - - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - - https://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. diff --git a/libs/common/fraction.go b/libs/math/fraction.go similarity index 96% rename from libs/common/fraction.go rename to libs/math/fraction.go index b5b46e791..dca8a8e79 100644 --- a/libs/common/fraction.go +++ b/libs/math/fraction.go @@ -1,4 +1,4 @@ -package common +package math import "fmt" diff --git a/libs/common/math.go b/libs/math/math.go similarity index 96% rename from libs/common/math.go rename to libs/math/math.go index ae91f2058..cf567a97a 100644 --- a/libs/common/math.go +++ b/libs/math/math.go @@ -1,4 +1,4 @@ -package common +package math func MaxInt64(a, b int64) int64 { if a > b { diff --git a/libs/common/net.go b/libs/net/net.go similarity index 98% rename from libs/common/net.go rename to libs/net/net.go index c7fff4cc3..fa85256fa 100644 --- a/libs/common/net.go +++ b/libs/net/net.go @@ -1,4 +1,4 @@ -package common +package net import ( "net" diff --git a/libs/common/net_test.go b/libs/net/net_test.go similarity index 97% rename from libs/common/net_test.go rename to libs/net/net_test.go index 38d2ae82d..38cd58f6a 100644 --- a/libs/common/net_test.go +++ b/libs/net/net_test.go @@ -1,4 +1,4 @@ -package common +package net import ( "testing" diff --git a/libs/common/os.go b/libs/os/os.go similarity index 99% rename from libs/common/os.go rename to libs/os/os.go index 4b25653ae..b56726c94 100644 --- a/libs/common/os.go +++ b/libs/os/os.go @@ -1,4 +1,4 @@ -package common +package os import ( "fmt" diff --git a/libs/common/string.go b/libs/strings/string.go similarity index 99% rename from libs/common/string.go rename to libs/strings/string.go index 4f8a8f20d..466759233 100644 --- a/libs/common/string.go +++ b/libs/strings/string.go @@ -1,4 +1,4 @@ -package common +package strings import ( "fmt" diff --git a/libs/common/string_test.go b/libs/strings/string_test.go similarity index 99% rename from libs/common/string_test.go rename to libs/strings/string_test.go index 35b6fafc7..1ec7b0d56 100644 --- a/libs/common/string_test.go +++ b/libs/strings/string_test.go @@ -1,4 +1,4 @@ -package common +package strings import ( "testing" diff --git a/lite2/client.go b/lite2/client.go index db1e494a1..64c5b9452 100644 --- a/lite2/client.go +++ b/lite2/client.go @@ -7,8 +7,8 @@ import ( "github.com/pkg/errors" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + tmmath "github.com/tendermint/tendermint/libs/math" "github.com/tendermint/tendermint/libs/rand" "github.com/tendermint/tendermint/lite2/provider" "github.com/tendermint/tendermint/lite2/store" @@ -70,7 +70,7 @@ func SequentialVerification() Option { // which must sign the new header in order for us to trust it. NOTE this only // applies to non-adjusted headers. For adjusted headers, sequential // verification is used. -func SkippingVerification(trustLevel cmn.Fraction) Option { +func SkippingVerification(trustLevel tmmath.Fraction) Option { if err := ValidateTrustLevel(trustLevel); err != nil { panic(err) } @@ -97,7 +97,7 @@ type Client struct { chainID string trustingPeriod time.Duration // see TrustOptions.Period verificationMode mode - trustLevel cmn.Fraction + trustLevel tmmath.Fraction // Primary provider of new headers. primary provider.Provider diff --git a/lite2/verifier.go b/lite2/verifier.go index 1ab1ef144..3737dbbf6 100644 --- a/lite2/verifier.go +++ b/lite2/verifier.go @@ -6,14 +6,14 @@ import ( "github.com/pkg/errors" - cmn "github.com/tendermint/tendermint/libs/common" + tmmath "github.com/tendermint/tendermint/libs/math" "github.com/tendermint/tendermint/types" ) var ( // DefaultTrustLevel - new header can be trusted if at least one correct old // validator signed it. - DefaultTrustLevel = cmn.Fraction{Numerator: 1, Denominator: 3} + DefaultTrustLevel = tmmath.Fraction{Numerator: 1, Denominator: 3} ) func Verify( @@ -24,7 +24,7 @@ func Verify( h2Vals *types.ValidatorSet, trustingPeriod time.Duration, now time.Time, - trustLevel cmn.Fraction) error { + trustLevel tmmath.Fraction) error { if err := ValidateTrustLevel(trustLevel); err != nil { return err @@ -106,7 +106,7 @@ func verifyNewHeaderAndVals( // ValidateTrustLevel checks that trustLevel is within the allowed range [1/3, // 1]. If not, it returns an error. 1/3 is the minimum amount of trust needed // which does not break the security model. -func ValidateTrustLevel(lvl cmn.Fraction) error { +func ValidateTrustLevel(lvl tmmath.Fraction) error { if lvl.Numerator*3 < lvl.Denominator || // < 1/3 lvl.Numerator > lvl.Denominator || // > 1 lvl.Denominator == 0 { diff --git a/lite2/verifier_test.go b/lite2/verifier_test.go index 570253400..321e9bdc3 100644 --- a/lite2/verifier_test.go +++ b/lite2/verifier_test.go @@ -7,7 +7,7 @@ import ( "github.com/stretchr/testify/assert" - cmn "github.com/tendermint/tendermint/libs/common" + tmmath "github.com/tendermint/tendermint/libs/math" "github.com/tendermint/tendermint/types" ) @@ -249,23 +249,23 @@ func TestVerifyNonAdjustedHeaders(t *testing.T) { func TestValidateTrustLevel(t *testing.T) { testCases := []struct { - lvl cmn.Fraction + lvl tmmath.Fraction valid bool }{ // valid - 0: {cmn.Fraction{Numerator: 1, Denominator: 1}, true}, - 1: {cmn.Fraction{Numerator: 1, Denominator: 3}, true}, - 2: {cmn.Fraction{Numerator: 2, Denominator: 3}, true}, - 3: {cmn.Fraction{Numerator: 3, Denominator: 3}, true}, - 4: {cmn.Fraction{Numerator: 4, Denominator: 5}, true}, + 0: {tmmath.Fraction{Numerator: 1, Denominator: 1}, true}, + 1: {tmmath.Fraction{Numerator: 1, Denominator: 3}, true}, + 2: {tmmath.Fraction{Numerator: 2, Denominator: 3}, true}, + 3: {tmmath.Fraction{Numerator: 3, Denominator: 3}, true}, + 4: {tmmath.Fraction{Numerator: 4, Denominator: 5}, true}, // invalid - 5: {cmn.Fraction{Numerator: 6, Denominator: 5}, false}, - 6: {cmn.Fraction{Numerator: -1, Denominator: 3}, false}, - 7: {cmn.Fraction{Numerator: 0, Denominator: 1}, false}, - 8: {cmn.Fraction{Numerator: -1, Denominator: -3}, false}, - 9: {cmn.Fraction{Numerator: 0, Denominator: 0}, false}, - 10: {cmn.Fraction{Numerator: 1, Denominator: 0}, false}, + 5: {tmmath.Fraction{Numerator: 6, Denominator: 5}, false}, + 6: {tmmath.Fraction{Numerator: -1, Denominator: 3}, false}, + 7: {tmmath.Fraction{Numerator: 0, Denominator: 1}, false}, + 8: {tmmath.Fraction{Numerator: -1, Denominator: -3}, false}, + 9: {tmmath.Fraction{Numerator: 0, Denominator: 0}, false}, + 10: {tmmath.Fraction{Numerator: 1, Denominator: 0}, false}, } for _, tc := range testCases { diff --git a/mempool/clist_mempool.go b/mempool/clist_mempool.go index e2f6ba814..7732032a4 100644 --- a/mempool/clist_mempool.go +++ b/mempool/clist_mempool.go @@ -15,8 +15,9 @@ import ( cfg "github.com/tendermint/tendermint/config" auto "github.com/tendermint/tendermint/libs/autofile" "github.com/tendermint/tendermint/libs/clist" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + tmmath "github.com/tendermint/tendermint/libs/math" + tmos "github.com/tendermint/tendermint/libs/os" "github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/proxy" "github.com/tendermint/tendermint/types" @@ -135,7 +136,7 @@ func WithMetrics(metrics *Metrics) CListMempoolOption { // *not thread safe* func (mem *CListMempool) InitWAL() { walDir := mem.config.WalDir() - err := cmn.EnsureDir(walDir, 0700) + err := tmos.EnsureDir(walDir, 0700) if err != nil { panic(errors.Wrap(err, "Error ensuring WAL dir")) } @@ -480,7 +481,7 @@ func (mem *CListMempool) ReapMaxBytesMaxGas(maxBytes, maxGas int64) types.Txs { var totalGas int64 // TODO: we will get a performance boost if we have a good estimate of avg // size per tx, and set the initial capacity based off of that. - // txs := make([]types.Tx, 0, cmn.MinInt(mem.txs.Len(), max/mem.avgTxSize)) + // txs := make([]types.Tx, 0, tmmath.MinInt(mem.txs.Len(), max/mem.avgTxSize)) txs := make([]types.Tx, 0, mem.txs.Len()) for e := mem.txs.Front(); e != nil; e = e.Next() { memTx := e.Value.(*mempoolTx) @@ -517,7 +518,7 @@ func (mem *CListMempool) ReapMaxTxs(max int) types.Txs { time.Sleep(time.Millisecond * 10) } - txs := make([]types.Tx, 0, cmn.MinInt(mem.txs.Len(), max)) + txs := make([]types.Tx, 0, tmmath.MinInt(mem.txs.Len(), max)) for e := mem.txs.Front(); e != nil && len(txs) <= max; e = e.Next() { memTx := e.Value.(*mempoolTx) txs = append(txs, memTx.tx) diff --git a/p2p/conn/connection.go b/p2p/conn/connection.go index f9c4ee1d6..1c2088636 100644 --- a/p2p/conn/connection.go +++ b/p2p/conn/connection.go @@ -16,9 +16,9 @@ import ( "github.com/pkg/errors" amino "github.com/tendermint/go-amino" - cmn "github.com/tendermint/tendermint/libs/common" flow "github.com/tendermint/tendermint/libs/flowrate" "github.com/tendermint/tendermint/libs/log" + tmmath "github.com/tendermint/tendermint/libs/math" "github.com/tendermint/tendermint/libs/service" "github.com/tendermint/tendermint/libs/timer" ) @@ -560,7 +560,7 @@ FOR_LOOP: // Peek into bufConnReader for debugging /* if numBytes := c.bufConnReader.Buffered(); numBytes > 0 { - bz, err := c.bufConnReader.Peek(cmn.MinInt(numBytes, 100)) + bz, err := c.bufConnReader.Peek(tmmath.MinInt(numBytes, 100)) if err == nil { // return } else { @@ -818,14 +818,14 @@ func (ch *Channel) nextPacketMsg() PacketMsg { packet := PacketMsg{} packet.ChannelID = ch.desc.ID maxSize := ch.maxPacketMsgPayloadSize - packet.Bytes = ch.sending[:cmn.MinInt(maxSize, len(ch.sending))] + packet.Bytes = ch.sending[:tmmath.MinInt(maxSize, len(ch.sending))] if len(ch.sending) <= maxSize { packet.EOF = byte(0x01) ch.sending = nil atomic.AddInt32(&ch.sendQueueSize, -1) // decrement sendQueueSize } else { packet.EOF = byte(0x00) - ch.sending = ch.sending[cmn.MinInt(maxSize, len(ch.sending)):] + ch.sending = ch.sending[tmmath.MinInt(maxSize, len(ch.sending)):] } return packet } diff --git a/p2p/conn/secret_connection.go b/p2p/conn/secret_connection.go index e26fd6943..d6b20186f 100644 --- a/p2p/conn/secret_connection.go +++ b/p2p/conn/secret_connection.go @@ -23,7 +23,7 @@ import ( "github.com/tendermint/tendermint/crypto" "github.com/tendermint/tendermint/crypto/ed25519" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/async" ) // 4 + 1024 == 1028 total frame size @@ -296,7 +296,7 @@ func genEphKeys() (ephPub, ephPriv *[32]byte) { func shareEphPubKey(conn io.ReadWriter, locEphPub *[32]byte) (remEphPub *[32]byte, err error) { // Send our pubkey and receive theirs in tandem. - var trs, _ = cmn.Parallel( + var trs, _ = async.Parallel( func(_ int) (val interface{}, abort bool, err error) { var _, err1 = cdc.MarshalBinaryLengthPrefixedWriter(conn, locEphPub) if err1 != nil { @@ -401,7 +401,7 @@ type authSigMessage struct { func shareAuthSignature(sc io.ReadWriter, pubKey crypto.PubKey, signature []byte) (recvMsg authSigMessage, err error) { // Send our info and receive theirs in tandem. - var trs, _ = cmn.Parallel( + var trs, _ = async.Parallel( func(_ int) (val interface{}, abort bool, err error) { var _, err1 = cdc.MarshalBinaryLengthPrefixedWriter(sc, authSigMessage{pubKey, signature}) if err1 != nil { diff --git a/p2p/conn/secret_connection_test.go b/p2p/conn/secret_connection_test.go index d6d85fd80..e4fa01af9 100644 --- a/p2p/conn/secret_connection_test.go +++ b/p2p/conn/secret_connection_test.go @@ -19,7 +19,8 @@ import ( "github.com/tendermint/tendermint/crypto" "github.com/tendermint/tendermint/crypto/ed25519" "github.com/tendermint/tendermint/crypto/secp256k1" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/async" + tmos "github.com/tendermint/tendermint/libs/os" "github.com/tendermint/tendermint/libs/rand" ) @@ -53,7 +54,7 @@ func makeSecretConnPair(tb testing.TB) (fooSecConn, barSecConn *SecretConnection var barPubKey = barPrvKey.PubKey() // Make connections from both sides in parallel. - var trs, ok = cmn.Parallel( + var trs, ok = async.Parallel( func(_ int) (val interface{}, abort bool, err error) { fooSecConn, err = MakeSecretConnection(fooConn, fooPrvKey) if err != nil { @@ -178,7 +179,7 @@ func TestSecretConnectionReadWrite(t *testing.T) { } // A helper that will run with (fooConn, fooWrites, fooReads) and vice versa - genNodeRunner := func(id string, nodeConn kvstoreConn, nodeWrites []string, nodeReads *[]string) cmn.Task { + genNodeRunner := func(id string, nodeConn kvstoreConn, nodeWrites []string, nodeReads *[]string) async.Task { return func(_ int) (interface{}, bool, error) { // Initiate cryptographic private key and secret connection trhough nodeConn. nodePrvKey := ed25519.GenPrivKey() @@ -188,7 +189,7 @@ func TestSecretConnectionReadWrite(t *testing.T) { return nil, true, err } // In parallel, handle some reads and writes. - var trs, ok = cmn.Parallel( + var trs, ok = async.Parallel( func(_ int) (interface{}, bool, error) { // Node writes: for _, nodeWrite := range nodeWrites { @@ -241,7 +242,7 @@ func TestSecretConnectionReadWrite(t *testing.T) { } // Run foo & bar in parallel - var trs, ok = cmn.Parallel( + var trs, ok = async.Parallel( genNodeRunner("foo", fooConn, fooWrites, &fooReads), genNodeRunner("bar", barConn, barWrites, &barReads), ) @@ -292,7 +293,7 @@ func TestDeriveSecretsAndChallengeGolden(t *testing.T) { if *update { t.Logf("Updating golden test vector file %s", goldenFilepath) data := createGoldenTestVectors(t) - cmn.WriteFile(goldenFilepath, []byte(data), 0644) + tmos.WriteFile(goldenFilepath, []byte(data), 0644) } f, err := os.Open(goldenFilepath) if err != nil { diff --git a/p2p/key.go b/p2p/key.go index 504cda75c..f4c9413ab 100644 --- a/p2p/key.go +++ b/p2p/key.go @@ -8,7 +8,7 @@ import ( "github.com/tendermint/tendermint/crypto" "github.com/tendermint/tendermint/crypto/ed25519" - cmn "github.com/tendermint/tendermint/libs/common" + tmos "github.com/tendermint/tendermint/libs/os" ) // ID is a hex-encoded crypto.Address @@ -47,7 +47,7 @@ func PubKeyToID(pubKey crypto.PubKey) ID { // LoadOrGenNodeKey attempts to load the NodeKey from the given filePath. // If the file does not exist, it generates and saves a new NodeKey. func LoadOrGenNodeKey(filePath string) (*NodeKey, error) { - if cmn.FileExists(filePath) { + if tmos.FileExists(filePath) { nodeKey, err := LoadNodeKey(filePath) if err != nil { return nil, err diff --git a/p2p/node_info.go b/p2p/node_info.go index 5b72446fd..24d5e11e1 100644 --- a/p2p/node_info.go +++ b/p2p/node_info.go @@ -5,7 +5,7 @@ import ( "reflect" "github.com/tendermint/tendermint/libs/bytes" - cmn "github.com/tendermint/tendermint/libs/common" + tmstrings "github.com/tendermint/tendermint/libs/strings" "github.com/tendermint/tendermint/version" ) @@ -130,7 +130,7 @@ func (info DefaultNodeInfo) Validate() error { // Validate Version if len(info.Version) > 0 && - (!cmn.IsASCIIText(info.Version) || cmn.ASCIITrim(info.Version) == "") { + (!tmstrings.IsASCIIText(info.Version) || tmstrings.ASCIITrim(info.Version) == "") { return fmt.Errorf("info.Version must be valid ASCII text without tabs, but got %v", info.Version) } @@ -149,7 +149,7 @@ func (info DefaultNodeInfo) Validate() error { } // Validate Moniker. - if !cmn.IsASCIIText(info.Moniker) || cmn.ASCIITrim(info.Moniker) == "" { + if !tmstrings.IsASCIIText(info.Moniker) || tmstrings.ASCIITrim(info.Moniker) == "" { return fmt.Errorf("info.Moniker must be valid non-empty ASCII text without tabs, but got %v", info.Moniker) } @@ -163,7 +163,7 @@ func (info DefaultNodeInfo) Validate() error { } // XXX: Should we be more strict about address formats? rpcAddr := other.RPCAddress - if len(rpcAddr) > 0 && (!cmn.IsASCIIText(rpcAddr) || cmn.ASCIITrim(rpcAddr) == "") { + if len(rpcAddr) > 0 && (!tmstrings.IsASCIIText(rpcAddr) || tmstrings.ASCIITrim(rpcAddr) == "") { return fmt.Errorf("info.Other.RPCAddress=%v must be valid ASCII text without tabs", rpcAddr) } diff --git a/p2p/peer.go b/p2p/peer.go index 1454b1cc1..7a6d6f868 100644 --- a/p2p/peer.go +++ b/p2p/peer.go @@ -5,7 +5,7 @@ import ( "net" "time" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/cmap" "github.com/tendermint/tendermint/libs/log" "github.com/tendermint/tendermint/libs/service" @@ -111,7 +111,7 @@ type peer struct { channels []byte // User data - Data *cmn.CMap + Data *cmap.CMap metrics *Metrics metricsTicker *time.Ticker @@ -132,7 +132,7 @@ func newPeer( peerConn: pc, nodeInfo: nodeInfo, channels: nodeInfo.(DefaultNodeInfo).Channels, // TODO - Data: cmn.NewCMap(), + Data: cmap.NewCMap(), metricsTicker: time.NewTicker(metricsTickerDuration), metrics: NopMetrics(), } diff --git a/p2p/pex/addrbook.go b/p2p/pex/addrbook.go index 94576084c..b9dde58c2 100644 --- a/p2p/pex/addrbook.go +++ b/p2p/pex/addrbook.go @@ -15,7 +15,7 @@ import ( "time" "github.com/tendermint/tendermint/crypto" - cmn "github.com/tendermint/tendermint/libs/common" + tmmath "github.com/tendermint/tendermint/libs/math" tmrand "github.com/tendermint/tendermint/libs/rand" "github.com/tendermint/tendermint/libs/service" "github.com/tendermint/tendermint/p2p" @@ -345,10 +345,10 @@ func (a *addrBook) GetSelection() []*p2p.NetAddress { return nil } - numAddresses := cmn.MaxInt( - cmn.MinInt(minGetSelection, bookSize), + numAddresses := tmmath.MaxInt( + tmmath.MinInt(minGetSelection, bookSize), bookSize*getSelectionPercent/100) - numAddresses = cmn.MinInt(maxGetSelection, numAddresses) + numAddresses = tmmath.MinInt(maxGetSelection, numAddresses) // XXX: instead of making a list of all addresses, shuffling, and slicing a random chunk, // could we just select a random numAddresses of indexes? @@ -402,14 +402,14 @@ func (a *addrBook) GetSelectionWithBias(biasTowardsNewAddrs int) []*p2p.NetAddre biasTowardsNewAddrs = 0 } - numAddresses := cmn.MaxInt( - cmn.MinInt(minGetSelection, bookSize), + numAddresses := tmmath.MaxInt( + tmmath.MinInt(minGetSelection, bookSize), bookSize*getSelectionPercent/100) - numAddresses = cmn.MinInt(maxGetSelection, numAddresses) + numAddresses = tmmath.MinInt(maxGetSelection, numAddresses) // number of new addresses that, if possible, should be in the beginning of the selection // if there are no enough old addrs, will choose new addr instead. - numRequiredNewAdd := cmn.MaxInt(percentageOfNum(biasTowardsNewAddrs, numAddresses), numAddresses-a.nOld) + numRequiredNewAdd := tmmath.MaxInt(percentageOfNum(biasTowardsNewAddrs, numAddresses), numAddresses-a.nOld) selection := a.randomPickAddresses(bucketTypeNew, numRequiredNewAdd) selection = append(selection, a.randomPickAddresses(bucketTypeOld, numAddresses-len(selection))...) return selection diff --git a/p2p/pex/addrbook_test.go b/p2p/pex/addrbook_test.go index 8d424224f..de9d09182 100644 --- a/p2p/pex/addrbook_test.go +++ b/p2p/pex/addrbook_test.go @@ -11,8 +11,8 @@ import ( "github.com/stretchr/testify/assert" "github.com/stretchr/testify/require" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + tmmath "github.com/tendermint/tendermint/libs/math" "github.com/tendermint/tendermint/libs/rand" "github.com/tendermint/tendermint/p2p" ) @@ -483,8 +483,8 @@ func testAddrBookAddressSelection(t *testing.T, bookSize int) { // There is at least one partition and at most three. var ( k = percentageOfNum(biasToSelectNewPeers, nAddrs) - expNew = cmn.MinInt(nNew, cmn.MaxInt(k, nAddrs-nBookOld)) - expOld = cmn.MinInt(nOld, nAddrs-expNew) + expNew = tmmath.MinInt(nNew, tmmath.MaxInt(k, nAddrs-nBookOld)) + expOld = tmmath.MinInt(nOld, nAddrs-expNew) ) // Verify that the number of old and new addresses are as expected diff --git a/p2p/pex/pex_reactor.go b/p2p/pex/pex_reactor.go index 77235741a..e86ad2a5e 100644 --- a/p2p/pex/pex_reactor.go +++ b/p2p/pex/pex_reactor.go @@ -9,7 +9,8 @@ import ( "github.com/pkg/errors" amino "github.com/tendermint/go-amino" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/cmap" + tmmath "github.com/tendermint/tendermint/libs/math" "github.com/tendermint/tendermint/libs/rand" "github.com/tendermint/tendermint/libs/service" "github.com/tendermint/tendermint/p2p" @@ -85,8 +86,8 @@ type Reactor struct { ensurePeersPeriod time.Duration // TODO: should go in the config // maps to prevent abuse - requestsSent *cmn.CMap // ID->struct{}: unanswered send requests - lastReceivedRequests *cmn.CMap // ID->time.Time: last time peer requested from us + requestsSent *cmap.CMap // ID->struct{}: unanswered send requests + lastReceivedRequests *cmap.CMap // ID->time.Time: last time peer requested from us seedAddrs []*p2p.NetAddress @@ -131,8 +132,8 @@ func NewReactor(b AddrBook, config *ReactorConfig) *Reactor { book: b, config: config, ensurePeersPeriod: defaultEnsurePeersPeriod, - requestsSent: cmn.NewCMap(), - lastReceivedRequests: cmn.NewCMap(), + requestsSent: cmap.NewCMap(), + lastReceivedRequests: cmap.NewCMap(), crawlPeerInfos: make(map[p2p.ID]crawlPeerInfo), } r.BaseReactor = *p2p.NewBaseReactor("Reactor", r) @@ -452,7 +453,7 @@ func (r *Reactor) ensurePeers() { // bias to prefer more vetted peers when we have fewer connections. // not perfect, but somewhate ensures that we prioritize connecting to more-vetted // NOTE: range here is [10, 90]. Too high ? - newBias := cmn.MinInt(out, 8)*10 + 10 + newBias := tmmath.MinInt(out, 8)*10 + 10 toDial := make(map[p2p.ID]*p2p.NetAddress) // Try maxAttempts times to pick numToDial addresses to dial diff --git a/p2p/switch.go b/p2p/switch.go index 587de39aa..4c14006d0 100644 --- a/p2p/switch.go +++ b/p2p/switch.go @@ -9,7 +9,7 @@ import ( "github.com/pkg/errors" "github.com/tendermint/tendermint/config" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/cmap" "github.com/tendermint/tendermint/libs/rand" "github.com/tendermint/tendermint/libs/service" "github.com/tendermint/tendermint/p2p/conn" @@ -74,8 +74,8 @@ type Switch struct { chDescs []*conn.ChannelDescriptor reactorsByCh map[byte]Reactor peers *PeerSet - dialing *cmn.CMap - reconnecting *cmn.CMap + dialing *cmap.CMap + reconnecting *cmap.CMap nodeInfo NodeInfo // our node info nodeKey *NodeKey // our node privkey addrBook AddrBook @@ -114,8 +114,8 @@ func NewSwitch( chDescs: make([]*conn.ChannelDescriptor, 0), reactorsByCh: make(map[byte]Reactor), peers: NewPeerSet(), - dialing: cmn.NewCMap(), - reconnecting: cmn.NewCMap(), + dialing: cmap.NewCMap(), + reconnecting: cmap.NewCMap(), metrics: NopMetrics(), transport: transport, filterTimeout: defaultFilterTimeout, diff --git a/p2p/test_util.go b/p2p/test_util.go index 7ef50235c..89c8015a5 100644 --- a/p2p/test_util.go +++ b/p2p/test_util.go @@ -9,8 +9,8 @@ import ( "github.com/tendermint/tendermint/crypto" "github.com/tendermint/tendermint/crypto/ed25519" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + tmnet "github.com/tendermint/tendermint/libs/net" "github.com/tendermint/tendermint/libs/rand" "github.com/tendermint/tendermint/config" @@ -276,7 +276,7 @@ func testNodeInfoWithNetwork(id ID, name, network string) NodeInfo { } func getFreePort() int { - port, err := cmn.GetFreePort() + port, err := tmnet.GetFreePort() if err != nil { panic(err) } diff --git a/privval/file.go b/privval/file.go index d77ec269d..c31e95a6a 100644 --- a/privval/file.go +++ b/privval/file.go @@ -10,7 +10,7 @@ import ( "github.com/tendermint/tendermint/crypto" "github.com/tendermint/tendermint/crypto/ed25519" tmbytes "github.com/tendermint/tendermint/libs/bytes" - cmn "github.com/tendermint/tendermint/libs/common" + tmos "github.com/tendermint/tendermint/libs/os" "github.com/tendermint/tendermint/libs/tempfile" "github.com/tendermint/tendermint/types" tmtime "github.com/tendermint/tendermint/types/time" @@ -183,12 +183,12 @@ func LoadFilePVEmptyState(keyFilePath, stateFilePath string) *FilePV { func loadFilePV(keyFilePath, stateFilePath string, loadState bool) *FilePV { keyJSONBytes, err := ioutil.ReadFile(keyFilePath) if err != nil { - cmn.Exit(err.Error()) + tmos.Exit(err.Error()) } pvKey := FilePVKey{} err = cdc.UnmarshalJSON(keyJSONBytes, &pvKey) if err != nil { - cmn.Exit(fmt.Sprintf("Error reading PrivValidator key from %v: %v\n", keyFilePath, err)) + tmos.Exit(fmt.Sprintf("Error reading PrivValidator key from %v: %v\n", keyFilePath, err)) } // overwrite pubkey and address for convenience @@ -200,11 +200,11 @@ func loadFilePV(keyFilePath, stateFilePath string, loadState bool) *FilePV { if loadState { stateJSONBytes, err := ioutil.ReadFile(stateFilePath) if err != nil { - cmn.Exit(err.Error()) + tmos.Exit(err.Error()) } err = cdc.UnmarshalJSON(stateJSONBytes, &pvState) if err != nil { - cmn.Exit(fmt.Sprintf("Error reading PrivValidator state from %v: %v\n", stateFilePath, err)) + tmos.Exit(fmt.Sprintf("Error reading PrivValidator state from %v: %v\n", stateFilePath, err)) } } @@ -220,7 +220,7 @@ func loadFilePV(keyFilePath, stateFilePath string, loadState bool) *FilePV { // or else generates a new one and saves it to the filePaths. func LoadOrGenFilePV(keyFilePath, stateFilePath string) *FilePV { var pv *FilePV - if cmn.FileExists(keyFilePath) { + if tmos.FileExists(keyFilePath) { pv = LoadFilePV(keyFilePath, stateFilePath) } else { pv = GenFilePV(keyFilePath, stateFilePath) diff --git a/privval/signer_listener_endpoint_test.go b/privval/signer_listener_endpoint_test.go index 25cfd0877..3c5f6d599 100644 --- a/privval/signer_listener_endpoint_test.go +++ b/privval/signer_listener_endpoint_test.go @@ -9,8 +9,8 @@ import ( "github.com/stretchr/testify/require" "github.com/tendermint/tendermint/crypto/ed25519" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + tmnet "github.com/tendermint/tendermint/libs/net" "github.com/tendermint/tendermint/libs/rand" "github.com/tendermint/tendermint/types" ) @@ -134,7 +134,7 @@ func TestRetryConnToRemoteSigner(t *testing.T) { /////////////////////////////////// func newSignerListenerEndpoint(logger log.Logger, addr string, timeoutReadWrite time.Duration) *SignerListenerEndpoint { - proto, address := cmn.ProtocolAndAddress(addr) + proto, address := tmnet.ProtocolAndAddress(addr) ln, err := net.Listen(proto, address) logger.Info("SignerListener: Listening", "proto", proto, "address", address) diff --git a/privval/socket_dialers.go b/privval/socket_dialers.go index fb1f9477b..1945e7728 100644 --- a/privval/socket_dialers.go +++ b/privval/socket_dialers.go @@ -6,7 +6,7 @@ import ( "github.com/pkg/errors" "github.com/tendermint/tendermint/crypto" - cmn "github.com/tendermint/tendermint/libs/common" + tmnet "github.com/tendermint/tendermint/libs/net" p2pconn "github.com/tendermint/tendermint/p2p/conn" ) @@ -22,7 +22,7 @@ type SocketDialer func() (net.Conn, error) // privKey for the authenticated encryption handshake. func DialTCPFn(addr string, timeoutReadWrite time.Duration, privKey crypto.PrivKey) SocketDialer { return func() (net.Conn, error) { - conn, err := cmn.Connect(addr) + conn, err := tmnet.Connect(addr) if err == nil { deadline := time.Now().Add(timeoutReadWrite) err = conn.SetDeadline(deadline) diff --git a/privval/utils.go b/privval/utils.go index 65368eb28..64f4f8c2f 100644 --- a/privval/utils.go +++ b/privval/utils.go @@ -7,8 +7,8 @@ import ( "github.com/pkg/errors" "github.com/tendermint/tendermint/crypto/ed25519" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + tmnet "github.com/tendermint/tendermint/libs/net" ) // IsConnTimeout returns a boolean indicating whether the error is known to @@ -29,7 +29,7 @@ func IsConnTimeout(err error) bool { func NewSignerListener(listenAddr string, logger log.Logger) (*SignerListenerEndpoint, error) { var listener net.Listener - protocol, address := cmn.ProtocolAndAddress(listenAddr) + protocol, address := tmnet.ProtocolAndAddress(listenAddr) ln, err := net.Listen(protocol, address) if err != nil { return nil, err @@ -54,7 +54,7 @@ func NewSignerListener(listenAddr string, logger log.Logger) (*SignerListenerEnd // GetFreeLocalhostAddrPort returns a free localhost:port address func GetFreeLocalhostAddrPort() string { - port, err := cmn.GetFreePort() + port, err := tmnet.GetFreePort() if err != nil { panic(err) } diff --git a/rpc/client/rpc_test.go b/rpc/client/rpc_test.go index e9b3a30c4..e0dba3f56 100644 --- a/rpc/client/rpc_test.go +++ b/rpc/client/rpc_test.go @@ -16,8 +16,8 @@ import ( "github.com/tendermint/tendermint/crypto/ed25519" "github.com/tendermint/tendermint/crypto/tmhash" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + tmmath "github.com/tendermint/tendermint/libs/math" mempl "github.com/tendermint/tendermint/mempool" "github.com/tendermint/tendermint/privval" "github.com/tendermint/tendermint/rpc/client" @@ -644,7 +644,7 @@ func testBatchedJSONRPCCalls(t *testing.T, c *client.HTTP) { bresult2, ok := bresults[1].(*ctypes.ResultBroadcastTxCommit) require.True(t, ok) require.Equal(t, *bresult2, *r2) - apph := cmn.MaxInt64(bresult1.Height, bresult2.Height) + 1 + apph := tmmath.MaxInt64(bresult1.Height, bresult2.Height) + 1 client.WaitForHeight(c, apph, nil) diff --git a/rpc/core/blocks.go b/rpc/core/blocks.go index 1a7ccd53e..5bcb3a05b 100644 --- a/rpc/core/blocks.go +++ b/rpc/core/blocks.go @@ -3,7 +3,7 @@ package core import ( "fmt" - cmn "github.com/tendermint/tendermint/libs/common" + tmmath "github.com/tendermint/tendermint/libs/math" ctypes "github.com/tendermint/tendermint/rpc/core/types" rpctypes "github.com/tendermint/tendermint/rpc/lib/types" sm "github.com/tendermint/tendermint/state" @@ -54,11 +54,11 @@ func filterMinMax(height, min, max, limit int64) (int64, int64, error) { } // limit max to the height - max = cmn.MinInt64(height, max) + max = tmmath.MinInt64(height, max) // limit min to within `limit` of max // so the total number of blocks returned will be `limit` - min = cmn.MaxInt64(min, max-limit+1) + min = tmmath.MaxInt64(min, max-limit+1) if min > max { return min, max, fmt.Errorf("min height %d can't be greater than max height %d", min, max) diff --git a/rpc/core/consensus.go b/rpc/core/consensus.go index 794fb113b..6a5a49e03 100644 --- a/rpc/core/consensus.go +++ b/rpc/core/consensus.go @@ -2,7 +2,7 @@ package core import ( cm "github.com/tendermint/tendermint/consensus" - cmn "github.com/tendermint/tendermint/libs/common" + tmmath "github.com/tendermint/tendermint/libs/math" ctypes "github.com/tendermint/tendermint/rpc/core/types" rpctypes "github.com/tendermint/tendermint/rpc/lib/types" sm "github.com/tendermint/tendermint/state" @@ -37,7 +37,7 @@ func Validators(ctx *rpctypes.Context, heightPtr *int64, page, perPage int) (*ct skipCount := validateSkipCount(page, perPage) - v := validators.Validators[skipCount : skipCount+cmn.MinInt(perPage, totalCount-skipCount)] + v := validators.Validators[skipCount : skipCount+tmmath.MinInt(perPage, totalCount-skipCount)] return &ctypes.ResultValidators{ BlockHeight: height, diff --git a/rpc/core/tx.go b/rpc/core/tx.go index cdc37f017..26d5364f4 100644 --- a/rpc/core/tx.go +++ b/rpc/core/tx.go @@ -3,7 +3,7 @@ package core import ( "fmt" - cmn "github.com/tendermint/tendermint/libs/common" + tmmath "github.com/tendermint/tendermint/libs/math" tmquery "github.com/tendermint/tendermint/libs/pubsub/query" ctypes "github.com/tendermint/tendermint/rpc/core/types" @@ -78,7 +78,7 @@ func TxSearch(ctx *rpctypes.Context, query string, prove bool, page, perPage int } skipCount := validateSkipCount(page, perPage) - apiResults := make([]*ctypes.ResultTx, cmn.MinInt(perPage, totalCount-skipCount)) + apiResults := make([]*ctypes.ResultTx, tmmath.MinInt(perPage, totalCount-skipCount)) var proof types.TxProof // if there's no tx in the results array, we don't need to loop through the apiResults array for i := 0; i < len(apiResults); i++ { diff --git a/rpc/grpc/client_server.go b/rpc/grpc/client_server.go index 5967b8ac5..133eb53ff 100644 --- a/rpc/grpc/client_server.go +++ b/rpc/grpc/client_server.go @@ -6,7 +6,7 @@ import ( "golang.org/x/net/context" "google.golang.org/grpc" - cmn "github.com/tendermint/tendermint/libs/common" + tmnet "github.com/tendermint/tendermint/libs/net" ) // Config is an gRPC server configuration. @@ -34,5 +34,5 @@ func StartGRPCClient(protoAddr string) BroadcastAPIClient { } func dialerFunc(ctx context.Context, addr string) (net.Conn, error) { - return cmn.Connect(addr) + return tmnet.Connect(addr) } diff --git a/rpc/lib/test/main.go b/rpc/lib/test/main.go index 2e433b901..891f9388d 100644 --- a/rpc/lib/test/main.go +++ b/rpc/lib/test/main.go @@ -7,8 +7,8 @@ import ( amino "github.com/tendermint/go-amino" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + tmos "github.com/tendermint/tendermint/libs/os" rpcserver "github.com/tendermint/tendermint/rpc/lib/server" rpctypes "github.com/tendermint/tendermint/rpc/lib/types" ) @@ -33,13 +33,13 @@ func main() { ) // Stop upon receiving SIGTERM or CTRL-C. - cmn.TrapSignal(logger, func() {}) + tmos.TrapSignal(logger, func() {}) rpcserver.RegisterRPCFuncs(mux, routes, cdc, logger) config := rpcserver.DefaultConfig() listener, err := rpcserver.Listen("0.0.0.0:8008", config) if err != nil { - cmn.Exit(err.Error()) + tmos.Exit(err.Error()) } rpcserver.StartHTTPServer(listener, mux, logger, config) } diff --git a/rpc/test/helpers.go b/rpc/test/helpers.go index 83ba2445f..32c465c2e 100644 --- a/rpc/test/helpers.go +++ b/rpc/test/helpers.go @@ -12,7 +12,7 @@ import ( "github.com/tendermint/tendermint/libs/log" cfg "github.com/tendermint/tendermint/config" - cmn "github.com/tendermint/tendermint/libs/common" + tmnet "github.com/tendermint/tendermint/libs/net" nm "github.com/tendermint/tendermint/node" "github.com/tendermint/tendermint/p2p" "github.com/tendermint/tendermint/privval" @@ -74,7 +74,7 @@ func makePathname() string { } func randPort() int { - port, err := cmn.GetFreePort() + port, err := tmnet.GetFreePort() if err != nil { panic(err) } diff --git a/state/store.go b/state/store.go index b1a9c7f81..545718688 100644 --- a/state/store.go +++ b/state/store.go @@ -4,7 +4,8 @@ import ( "fmt" abci "github.com/tendermint/tendermint/abci/types" - cmn "github.com/tendermint/tendermint/libs/common" + tmmath "github.com/tendermint/tendermint/libs/math" + tmos "github.com/tendermint/tendermint/libs/os" "github.com/tendermint/tendermint/types" dbm "github.com/tendermint/tm-db" ) @@ -79,7 +80,7 @@ func loadState(db dbm.DB, key []byte) (state State) { err := cdc.UnmarshalBinaryBare(buf, &state) if err != nil { // DATA HAS BEEN CORRUPTED OR THE SPEC HAS CHANGED - cmn.Exit(fmt.Sprintf(`LoadState: Data has been corrupted or its spec has changed: + tmos.Exit(fmt.Sprintf(`LoadState: Data has been corrupted or its spec has changed: %v\n`, err)) } // TODO: ensure that buf is completely read. @@ -155,7 +156,7 @@ func LoadABCIResponses(db dbm.DB, height int64) (*ABCIResponses, error) { err := cdc.UnmarshalBinaryBare(buf, abciResponses) if err != nil { // DATA HAS BEEN CORRUPTED OR THE SPEC HAS CHANGED - cmn.Exit(fmt.Sprintf(`LoadABCIResponses: Data has been corrupted or its spec has + tmos.Exit(fmt.Sprintf(`LoadABCIResponses: Data has been corrupted or its spec has changed: %v\n`, err)) } // TODO: ensure that buf is completely read. @@ -213,7 +214,7 @@ func LoadValidators(db dbm.DB, height int64) (*types.ValidatorSet, error) { func lastStoredHeightFor(height, lastHeightChanged int64) int64 { checkpointHeight := height - height%valSetCheckpointInterval - return cmn.MaxInt64(checkpointHeight, lastHeightChanged) + return tmmath.MaxInt64(checkpointHeight, lastHeightChanged) } // CONTRACT: Returned ValidatorsInfo can be mutated. @@ -227,7 +228,7 @@ func loadValidatorsInfo(db dbm.DB, height int64) *ValidatorsInfo { err := cdc.UnmarshalBinaryBare(buf, v) if err != nil { // DATA HAS BEEN CORRUPTED OR THE SPEC HAS CHANGED - cmn.Exit(fmt.Sprintf(`LoadValidators: Data has been corrupted or its spec has changed: + tmos.Exit(fmt.Sprintf(`LoadValidators: Data has been corrupted or its spec has changed: %v\n`, err)) } // TODO: ensure that buf is completely read. @@ -304,7 +305,7 @@ func loadConsensusParamsInfo(db dbm.DB, height int64) *ConsensusParamsInfo { err := cdc.UnmarshalBinaryBare(buf, paramsInfo) if err != nil { // DATA HAS BEEN CORRUPTED OR THE SPEC HAS CHANGED - cmn.Exit(fmt.Sprintf(`LoadConsensusParams: Data has been corrupted or its spec has changed: + tmos.Exit(fmt.Sprintf(`LoadConsensusParams: Data has been corrupted or its spec has changed: %v\n`, err)) } // TODO: ensure that buf is completely read. diff --git a/state/txindex/kv/kv.go b/state/txindex/kv/kv.go index 46365dafb..956af153e 100644 --- a/state/txindex/kv/kv.go +++ b/state/txindex/kv/kv.go @@ -13,8 +13,8 @@ import ( dbm "github.com/tendermint/tm-db" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/pubsub/query" + tmstring "github.com/tendermint/tendermint/libs/strings" "github.com/tendermint/tendermint/state/txindex" "github.com/tendermint/tendermint/types" ) @@ -91,7 +91,7 @@ func (txi *TxIndex) AddBatch(b *txindex.Batch) error { txi.indexEvents(result, hash, storeBatch) // index tx by height - if txi.indexAllEvents || cmn.StringInSlice(types.TxHeightKey, txi.compositeKeysToIndex) { + if txi.indexAllEvents || tmstring.StringInSlice(types.TxHeightKey, txi.compositeKeysToIndex) { storeBatch.Set(keyForHeight(result), hash) } @@ -121,7 +121,7 @@ func (txi *TxIndex) Index(result *types.TxResult) error { txi.indexEvents(result, hash, b) // index tx by height - if txi.indexAllEvents || cmn.StringInSlice(types.TxHeightKey, txi.compositeKeysToIndex) { + if txi.indexAllEvents || tmstring.StringInSlice(types.TxHeightKey, txi.compositeKeysToIndex) { b.Set(keyForHeight(result), hash) } @@ -150,7 +150,7 @@ func (txi *TxIndex) indexEvents(result *types.TxResult, hash []byte, store dbm.S } compositeTag := fmt.Sprintf("%s.%s", event.Type, string(attr.Key)) - if txi.indexAllEvents || cmn.StringInSlice(compositeTag, txi.compositeKeysToIndex) { + if txi.indexAllEvents || tmstring.StringInSlice(compositeTag, txi.compositeKeysToIndex) { store.Set(keyForEvent(compositeTag, attr.Value, result), hash) } } diff --git a/tools/tm-monitor/main.go b/tools/tm-monitor/main.go index bcabcf73e..7e4bffca2 100644 --- a/tools/tm-monitor/main.go +++ b/tools/tm-monitor/main.go @@ -6,8 +6,8 @@ import ( "os" "strings" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + tmos "github.com/tendermint/tendermint/libs/os" monitor "github.com/tendermint/tendermint/tools/tm-monitor/monitor" ) @@ -59,7 +59,7 @@ Examples: } // Stop upon receiving SIGTERM or CTRL-C. - cmn.TrapSignal(logger, func() { + tmos.TrapSignal(logger, func() { if !noton { ton.Stop() } diff --git a/tools/tm-signer-harness/internal/test_harness.go b/tools/tm-signer-harness/internal/test_harness.go index 216cf6851..c489a2fd4 100644 --- a/tools/tm-signer-harness/internal/test_harness.go +++ b/tools/tm-signer-harness/internal/test_harness.go @@ -13,8 +13,9 @@ import ( "github.com/tendermint/tendermint/privval" "github.com/tendermint/tendermint/state" - cmn "github.com/tendermint/tendermint/libs/common" "github.com/tendermint/tendermint/libs/log" + tmnet "github.com/tendermint/tendermint/libs/net" + tmos "github.com/tendermint/tendermint/libs/os" "github.com/tendermint/tendermint/types" ) @@ -321,10 +322,10 @@ func (th *TestHarness) Shutdown(err error) { // newTestHarnessListener creates our client instance which we will use for testing. func newTestHarnessListener(logger log.Logger, cfg TestHarnessConfig) (*privval.SignerListenerEndpoint, error) { - proto, addr := cmn.ProtocolAndAddress(cfg.BindAddr) + proto, addr := tmnet.ProtocolAndAddress(cfg.BindAddr) if proto == "unix" { // make sure the socket doesn't exist - if so, try to delete it - if cmn.FileExists(addr) { + if tmos.FileExists(addr) { if err := os.Remove(addr); err != nil { logger.Error("Failed to remove existing Unix domain socket", "addr", addr) return nil, err diff --git a/types/block.go b/types/block.go index c67b9ade9..45cb8d827 100644 --- a/types/block.go +++ b/types/block.go @@ -12,8 +12,9 @@ import ( "github.com/tendermint/tendermint/crypto" "github.com/tendermint/tendermint/crypto/merkle" "github.com/tendermint/tendermint/crypto/tmhash" + "github.com/tendermint/tendermint/libs/bits" tmbytes "github.com/tendermint/tendermint/libs/bytes" - cmn "github.com/tendermint/tendermint/libs/common" + tmmath "github.com/tendermint/tendermint/libs/math" "github.com/tendermint/tendermint/version" ) @@ -561,7 +562,7 @@ type Commit struct { // NOTE: can't memoize in constructor because constructor isn't used for // unmarshaling. hash tmbytes.HexBytes - bitArray *cmn.BitArray + bitArray *bits.BitArray } // NewCommit returns a new Commit. @@ -645,9 +646,9 @@ func (commit *Commit) Size() int { // BitArray returns a BitArray of which validators voted for BlockID or nil in this commit. // Implements VoteSetReader. -func (commit *Commit) BitArray() *cmn.BitArray { +func (commit *Commit) BitArray() *bits.BitArray { if commit.bitArray == nil { - commit.bitArray = cmn.NewBitArray(len(commit.Signatures)) + commit.bitArray = bits.NewBitArray(len(commit.Signatures)) for i, commitSig := range commit.Signatures { // TODO: need to check the BlockID otherwise we could be counting conflicts, // not just the one with +2/3 ! @@ -829,7 +830,7 @@ func (data *Data) StringIndented(indent string) string { if data == nil { return "nil-Data" } - txStrings := make([]string, cmn.MinInt(len(data.Txs), 21)) + txStrings := make([]string, tmmath.MinInt(len(data.Txs), 21)) for i, tx := range data.Txs { if i == 20 { txStrings[i] = fmt.Sprintf("... (%v total)", len(data.Txs)) @@ -867,7 +868,7 @@ func (data *EvidenceData) StringIndented(indent string) string { if data == nil { return "nil-Evidence" } - evStrings := make([]string, cmn.MinInt(len(data.Evidence), 21)) + evStrings := make([]string, tmmath.MinInt(len(data.Evidence), 21)) for i, ev := range data.Evidence { if i == 20 { evStrings[i] = fmt.Sprintf("... (%v total)", len(data.Evidence)) diff --git a/types/block_test.go b/types/block_test.go index d3454f54b..ba920f63f 100644 --- a/types/block_test.go +++ b/types/block_test.go @@ -17,8 +17,8 @@ import ( "github.com/tendermint/tendermint/crypto" "github.com/tendermint/tendermint/crypto/merkle" "github.com/tendermint/tendermint/crypto/tmhash" + "github.com/tendermint/tendermint/libs/bits" "github.com/tendermint/tendermint/libs/bytes" - cmn "github.com/tendermint/tendermint/libs/common" tmrand "github.com/tendermint/tendermint/libs/rand" tmtime "github.com/tendermint/tendermint/types/time" "github.com/tendermint/tendermint/version" @@ -221,7 +221,7 @@ func TestCommit(t *testing.T) { } require.NotNil(t, commit.BitArray()) - assert.Equal(t, cmn.NewBitArray(10).Size(), commit.BitArray().Size()) + assert.Equal(t, bits.NewBitArray(10).Size(), commit.BitArray().Size()) assert.Equal(t, voteSet.GetByIndex(0), commit.GetByIndex(0)) assert.True(t, commit.IsCommit()) diff --git a/types/genesis.go b/types/genesis.go index a28bafdff..73a7847d8 100644 --- a/types/genesis.go +++ b/types/genesis.go @@ -11,7 +11,7 @@ import ( "github.com/tendermint/tendermint/crypto" tmbytes "github.com/tendermint/tendermint/libs/bytes" - cmn "github.com/tendermint/tendermint/libs/common" + tmos "github.com/tendermint/tendermint/libs/os" tmtime "github.com/tendermint/tendermint/types/time" ) @@ -50,7 +50,7 @@ func (genDoc *GenesisDoc) SaveAs(file string) error { if err != nil { return err } - return cmn.WriteFile(file, genDocBytes, 0644) + return tmos.WriteFile(file, genDocBytes, 0644) } // ValidatorHash returns the hash of the validator set contained in the GenesisDoc diff --git a/types/params.go b/types/params.go index 773ec7731..157c0447c 100644 --- a/types/params.go +++ b/types/params.go @@ -5,7 +5,7 @@ import ( abci "github.com/tendermint/tendermint/abci/types" "github.com/tendermint/tendermint/crypto/tmhash" - cmn "github.com/tendermint/tendermint/libs/common" + tmstrings "github.com/tendermint/tendermint/libs/strings" ) const ( @@ -159,7 +159,7 @@ func (params *ConsensusParams) Hash() []byte { func (params *ConsensusParams) Equals(params2 *ConsensusParams) bool { return params.Block == params2.Block && params.Evidence == params2.Evidence && - cmn.StringSliceEqual(params.Validator.PubKeyTypes, params2.Validator.PubKeyTypes) + tmstrings.StringSliceEqual(params.Validator.PubKeyTypes, params2.Validator.PubKeyTypes) } // Update returns a copy of the params with updates from the non-zero fields of p2. diff --git a/types/part_set.go b/types/part_set.go index cdacc0083..51af767b8 100644 --- a/types/part_set.go +++ b/types/part_set.go @@ -9,8 +9,9 @@ import ( "github.com/pkg/errors" "github.com/tendermint/tendermint/crypto/merkle" + "github.com/tendermint/tendermint/libs/bits" tmbytes "github.com/tendermint/tendermint/libs/bytes" - cmn "github.com/tendermint/tendermint/libs/common" + tmmath "github.com/tendermint/tendermint/libs/math" ) var ( @@ -92,7 +93,7 @@ type PartSet struct { mtx sync.Mutex parts []*Part - partsBitArray *cmn.BitArray + partsBitArray *bits.BitArray count int } @@ -103,11 +104,11 @@ func NewPartSetFromData(data []byte, partSize int) *PartSet { total := (len(data) + partSize - 1) / partSize parts := make([]*Part, total) partsBytes := make([][]byte, total) - partsBitArray := cmn.NewBitArray(total) + partsBitArray := bits.NewBitArray(total) for i := 0; i < total; i++ { part := &Part{ Index: i, - Bytes: data[i*partSize : cmn.MinInt(len(data), (i+1)*partSize)], + Bytes: data[i*partSize : tmmath.MinInt(len(data), (i+1)*partSize)], } parts[i] = part partsBytes[i] = part.Bytes @@ -133,7 +134,7 @@ func NewPartSetFromHeader(header PartSetHeader) *PartSet { total: header.Total, hash: header.Hash, parts: make([]*Part, header.Total), - partsBitArray: cmn.NewBitArray(header.Total), + partsBitArray: bits.NewBitArray(header.Total), count: 0, } } @@ -155,7 +156,7 @@ func (ps *PartSet) HasHeader(header PartSetHeader) bool { return ps.Header().Equals(header) } -func (ps *PartSet) BitArray() *cmn.BitArray { +func (ps *PartSet) BitArray() *bits.BitArray { ps.mtx.Lock() defer ps.mtx.Unlock() return ps.partsBitArray.Copy() @@ -288,8 +289,8 @@ func (ps *PartSet) MarshalJSON() ([]byte, error) { defer ps.mtx.Unlock() return cdc.MarshalJSON(struct { - CountTotal string `json:"count/total"` - PartsBitArray *cmn.BitArray `json:"parts_bit_array"` + CountTotal string `json:"count/total"` + PartsBitArray *bits.BitArray `json:"parts_bit_array"` }{ fmt.Sprintf("%d/%d", ps.Count(), ps.Total()), ps.partsBitArray, diff --git a/types/signable.go b/types/signable.go index 72d2ea9ac..074654cc5 100644 --- a/types/signable.go +++ b/types/signable.go @@ -2,14 +2,14 @@ package types import ( "github.com/tendermint/tendermint/crypto/ed25519" - cmn "github.com/tendermint/tendermint/libs/common" + tmmath "github.com/tendermint/tendermint/libs/math" ) var ( // MaxSignatureSize is a maximum allowed signature size for the Proposal // and Vote. // XXX: secp256k1 does not have Size nor MaxSize defined. - MaxSignatureSize = cmn.MaxInt(ed25519.SignatureSize, 64) + MaxSignatureSize = tmmath.MaxInt(ed25519.SignatureSize, 64) ) // Signable is an interface for all signable things. diff --git a/types/validator_set.go b/types/validator_set.go index 9796fe453..29859a48a 100644 --- a/types/validator_set.go +++ b/types/validator_set.go @@ -9,9 +9,8 @@ import ( "strings" "github.com/pkg/errors" - "github.com/tendermint/tendermint/crypto/merkle" - cmn "github.com/tendermint/tendermint/libs/common" + tmmath "github.com/tendermint/tendermint/libs/math" ) const ( @@ -749,7 +748,7 @@ func (vals *ValidatorSet) VerifyFutureCommit(newSet *ValidatorSet, chainID strin // NOTE the given validators do not necessarily correspond to the validator set // for this commit, but there may be some intersection. func (vals *ValidatorSet) VerifyCommitTrusting(chainID string, blockID BlockID, - height int64, commit *Commit, trustLevel cmn.Fraction) error { + height int64, commit *Commit, trustLevel tmmath.Fraction) error { if trustLevel.Numerator*3 < trustLevel.Denominator || // < 1/3 trustLevel.Numerator > trustLevel.Denominator { // > 1 diff --git a/types/validator_set_test.go b/types/validator_set_test.go index 9cb6d6536..ce7556918 100644 --- a/types/validator_set_test.go +++ b/types/validator_set_test.go @@ -13,7 +13,7 @@ import ( "github.com/tendermint/tendermint/crypto" "github.com/tendermint/tendermint/crypto/ed25519" - cmn "github.com/tendermint/tendermint/libs/common" + tmmath "github.com/tendermint/tendermint/libs/math" "github.com/tendermint/tendermint/libs/rand" tmtime "github.com/tendermint/tendermint/types/time" ) @@ -1013,7 +1013,7 @@ func TestValSetUpdatesOrderIndependenceTestsExecute(t *testing.T) { // perform at most 20 permutations on the updates and call UpdateWithChangeSet() n := len(tt.updateVals) - maxNumPerms := cmn.MinInt(20, n*n) + maxNumPerms := tmmath.MinInt(20, n*n) for j := 0; j < maxNumPerms; j++ { // create a copy of original set and apply a random permutation of updates valSetCopy := valSet.Copy() diff --git a/types/vote_set.go b/types/vote_set.go index d1e388dd2..82698fe51 100644 --- a/types/vote_set.go +++ b/types/vote_set.go @@ -8,7 +8,7 @@ import ( "github.com/pkg/errors" - cmn "github.com/tendermint/tendermint/libs/common" + "github.com/tendermint/tendermint/libs/bits" ) const ( @@ -66,7 +66,7 @@ type VoteSet struct { valSet *ValidatorSet mtx sync.Mutex - votesBitArray *cmn.BitArray + votesBitArray *bits.BitArray votes []*Vote // Primary votes to share sum int64 // Sum of voting power for seen votes, discounting conflicts maj23 *BlockID // First 2/3 majority seen @@ -85,7 +85,7 @@ func NewVoteSet(chainID string, height int64, round int, signedMsgType SignedMsg round: round, signedMsgType: signedMsgType, valSet: valSet, - votesBitArray: cmn.NewBitArray(valSet.Size()), + votesBitArray: bits.NewBitArray(valSet.Size()), votes: make([]*Vote, valSet.Size()), sum: 0, maj23: nil, @@ -340,7 +340,7 @@ func (voteSet *VoteSet) SetPeerMaj23(peerID P2PID, blockID BlockID) error { } // Implements VoteSetReader. -func (voteSet *VoteSet) BitArray() *cmn.BitArray { +func (voteSet *VoteSet) BitArray() *bits.BitArray { if voteSet == nil { return nil } @@ -349,7 +349,7 @@ func (voteSet *VoteSet) BitArray() *cmn.BitArray { return voteSet.votesBitArray.Copy() } -func (voteSet *VoteSet) BitArrayByBlockID(blockID BlockID) *cmn.BitArray { +func (voteSet *VoteSet) BitArrayByBlockID(blockID BlockID) *bits.BitArray { if voteSet == nil { return nil } @@ -580,16 +580,16 @@ func (voteSet *VoteSet) MakeCommit() *Commit { 2. A peer claims to have a 2/3 majority w/ blockKey (peerMaj23=true) */ type blockVotes struct { - peerMaj23 bool // peer claims to have maj23 - bitArray *cmn.BitArray // valIndex -> hasVote? - votes []*Vote // valIndex -> *Vote - sum int64 // vote sum + peerMaj23 bool // peer claims to have maj23 + bitArray *bits.BitArray // valIndex -> hasVote? + votes []*Vote // valIndex -> *Vote + sum int64 // vote sum } func newBlockVotes(peerMaj23 bool, numValidators int) *blockVotes { return &blockVotes{ peerMaj23: peerMaj23, - bitArray: cmn.NewBitArray(numValidators), + bitArray: bits.NewBitArray(numValidators), votes: make([]*Vote, numValidators), sum: 0, } @@ -619,7 +619,7 @@ type VoteSetReader interface { GetRound() int Type() byte Size() int - BitArray() *cmn.BitArray + BitArray() *bits.BitArray GetByIndex(int) *Vote IsCommit() bool }