mirror of
https://github.com/kaspanet/kaspad.git
synced 2025-06-06 06:06:49 +00:00
[NOD-1130] Integrate RPC with the new architecture (#807)
* [NOD-1130] Delete rpcadapters.go. * [NOD-1130] Delete p2p. Move rpc to top level. * [NOD-1130] Remove DAGParams from rpcserverConfig. * [NOD-1130] Remove rpcserverPeer, rpcserverConnManager, rpcserverSyncManager, and rpcserverConfig. * [NOD-1130] Remove wallet RPC commands. * [NOD-1130] Remove wallet RPC commands. * [NOD-1130] Remove connmgr and peer. * [NOD-1130] Move rpcmodel into rpc. * [NOD-1130] Implement ConnectionCount. * [NOD-1130] Remove ping and node RPC commands. * [NOD-1130] Dummify handleGetNetTotals. * [NOD-1130] Add NetConnection to Peer. * [NOD-1130] Fix merge errors. * [NOD-1130] Implement Peers. * [NOD-1130] Fix HandleGetConnectedPeerInfo. * [NOD-1130] Fix SendRawTransaction. * [NOD-1130] Rename addManualNode to connect and removeManualNode to disconnect. * [NOD-1130] Add a stub for AddBlock. * [NOD-1130] Fix tests. * [NOD-1130] Replace half-baked contents of RemoveConnection with a stub. * [NOD-1130] Fix merge errors. * [NOD-1130] Make golint happy. * [NOD-1130] Get rid of something weird. * [NOD-1130] Rename minerClient back to client. * [NOD-1130] Add a few fields to GetConnectedPeerInfoResult. * [NOD-1130] Rename oneTry to isPermanent. * [NOD-1130] Implement ConnectionCount in NetAdapter. * [NOD-1130] Move RawMempoolVerbose out of mempool. * [NOD-1130] Move isSynced into the mining package. * [NOD-1130] Fix a compilation error. * [NOD-1130] Make golint happy. * [NOD-1130] Fix merge errors.
This commit is contained in:
parent
8e1958c20b
commit
3d45c8de50
@ -16,7 +16,7 @@ import (
|
||||
"strings"
|
||||
|
||||
"github.com/jessevdk/go-flags"
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
)
|
||||
|
||||
@ -24,7 +24,7 @@ const (
|
||||
// unusableFlags are the command usage flags which this utility are not
|
||||
// able to use. In particular it doesn't support websockets and
|
||||
// consequently notifications.
|
||||
unusableFlags = rpcmodel.UFWebsocketOnly | rpcmodel.UFNotification
|
||||
unusableFlags = model.UFWebsocketOnly | model.UFNotification
|
||||
)
|
||||
|
||||
var (
|
||||
@ -45,10 +45,10 @@ func listCommands() {
|
||||
)
|
||||
|
||||
// Get a list of registered commands and categorize and filter them.
|
||||
cmdMethods := rpcmodel.RegisteredCmdMethods()
|
||||
cmdMethods := model.RegisteredCmdMethods()
|
||||
categorized := make([][]string, numCategories)
|
||||
for _, method := range cmdMethods {
|
||||
flags, err := rpcmodel.MethodUsageFlags(method)
|
||||
flags, err := model.MethodUsageFlags(method)
|
||||
if err != nil {
|
||||
// This should never happen since the method was just
|
||||
// returned from the package, but be safe.
|
||||
@ -60,7 +60,7 @@ func listCommands() {
|
||||
continue
|
||||
}
|
||||
|
||||
usage, err := rpcmodel.MethodUsageText(method)
|
||||
usage, err := model.MethodUsageText(method)
|
||||
if err != nil {
|
||||
// This should never happen since the method was just
|
||||
// returned from the package, but be safe.
|
||||
|
@ -11,7 +11,7 @@ import (
|
||||
"net/http"
|
||||
|
||||
"github.com/btcsuite/go-socks/socks"
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
)
|
||||
|
||||
// newHTTPClient returns a new HTTP client that is configured according to the
|
||||
@ -117,7 +117,7 @@ func sendPostRequest(marshalledJSON []byte, cfg *ConfigFlags) ([]byte, error) {
|
||||
}
|
||||
|
||||
// Unmarshal the response.
|
||||
var resp rpcmodel.Response
|
||||
var resp model.Response
|
||||
if err := json.Unmarshal(respBytes, &resp); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -11,7 +11,7 @@ import (
|
||||
"path/filepath"
|
||||
"strings"
|
||||
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
)
|
||||
|
||||
const (
|
||||
@ -21,7 +21,7 @@ const (
|
||||
|
||||
// commandUsage display the usage for a specific command.
|
||||
func commandUsage(method string) {
|
||||
usage, err := rpcmodel.MethodUsageText(method)
|
||||
usage, err := model.MethodUsageText(method)
|
||||
if err != nil {
|
||||
// This should never happen since the method was already checked
|
||||
// before calling this function, but be safe.
|
||||
@ -60,7 +60,7 @@ func main() {
|
||||
// Ensure the specified method identifies a valid registered command and
|
||||
// is one of the usable types.
|
||||
method := args[0]
|
||||
usageFlags, err := rpcmodel.MethodUsageFlags(method)
|
||||
usageFlags, err := model.MethodUsageFlags(method)
|
||||
if err != nil {
|
||||
fmt.Fprintf(os.Stderr, "Unrecognized command '%s'\n", method)
|
||||
fmt.Fprintln(os.Stderr, listCmdMessage)
|
||||
@ -105,13 +105,13 @@ func main() {
|
||||
|
||||
// Attempt to create the appropriate command using the arguments
|
||||
// provided by the user.
|
||||
cmd, err := rpcmodel.NewCommand(method, params...)
|
||||
cmd, err := model.NewCommand(method, params...)
|
||||
if err != nil {
|
||||
// Show the error along with its error code when it's a
|
||||
// rpcmodel.Error as it reallistcally will always be since the
|
||||
// model.Error as it reallistcally will always be since the
|
||||
// NewCommand function is only supposed to return errors of that
|
||||
// type.
|
||||
var rpcModelErr rpcmodel.Error
|
||||
var rpcModelErr model.Error
|
||||
if ok := errors.As(err, &rpcModelErr); ok {
|
||||
fmt.Fprintf(os.Stderr, "%s error: %s (command code: %s)\n",
|
||||
method, err, rpcModelErr.ErrorCode)
|
||||
@ -119,7 +119,7 @@ func main() {
|
||||
os.Exit(1)
|
||||
}
|
||||
|
||||
// The error is not a rpcmodel.Error and this really should not
|
||||
// The error is not a model.Error and this really should not
|
||||
// happen. Nevertheless, fallback to just showing the error
|
||||
// if it should happen due to a bug in the package.
|
||||
fmt.Fprintf(os.Stderr, "%s error: %s\n", method, err)
|
||||
@ -129,7 +129,7 @@ func main() {
|
||||
|
||||
// Marshal the command into a JSON-RPC byte slice in preparation for
|
||||
// sending it to the RPC server.
|
||||
marshalledJSON, err := rpcmodel.MarshalCommand(1, cmd)
|
||||
marshalledJSON, err := model.MarshalCommand(1, cmd)
|
||||
if err != nil {
|
||||
fmt.Fprintln(os.Stderr, err)
|
||||
os.Exit(1)
|
||||
|
@ -1,7 +1,7 @@
|
||||
package main
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/rpcclient"
|
||||
"github.com/kaspanet/kaspad/rpc/client"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/kaspanet/kaspad/wire"
|
||||
"github.com/pkg/errors"
|
||||
@ -10,30 +10,30 @@ import (
|
||||
)
|
||||
|
||||
type minerClient struct {
|
||||
*rpcclient.Client
|
||||
*client.Client
|
||||
onBlockAdded chan struct{}
|
||||
}
|
||||
|
||||
func newMinerClient(connCfg *rpcclient.ConnConfig) (*minerClient, error) {
|
||||
client := &minerClient{
|
||||
func newMinerClient(connCfg *client.ConnConfig) (*minerClient, error) {
|
||||
minerClient := &minerClient{
|
||||
onBlockAdded: make(chan struct{}, 1),
|
||||
}
|
||||
notificationHandlers := &rpcclient.NotificationHandlers{
|
||||
notificationHandlers := &client.NotificationHandlers{
|
||||
OnFilteredBlockAdded: func(_ uint64, header *wire.BlockHeader,
|
||||
txs []*util.Tx) {
|
||||
client.onBlockAdded <- struct{}{}
|
||||
minerClient.onBlockAdded <- struct{}{}
|
||||
},
|
||||
}
|
||||
var err error
|
||||
client.Client, err = rpcclient.New(connCfg, notificationHandlers)
|
||||
minerClient.Client, err = client.New(connCfg, notificationHandlers)
|
||||
if err != nil {
|
||||
return nil, errors.Errorf("Error connecting to address %s: %s", connCfg.Host, err)
|
||||
}
|
||||
|
||||
if err = client.NotifyBlocks(); err != nil {
|
||||
return nil, errors.Errorf("Error while registering client %s for block notifications: %s", client.Host(), err)
|
||||
if err = minerClient.NotifyBlocks(); err != nil {
|
||||
return nil, errors.Errorf("Error while registering minerClient %s for block notifications: %s", minerClient.Host(), err)
|
||||
}
|
||||
return client, nil
|
||||
return minerClient, nil
|
||||
}
|
||||
|
||||
func connectToServer(cfg *configFlags) (*minerClient, error) {
|
||||
@ -47,7 +47,7 @@ func connectToServer(cfg *configFlags) (*minerClient, error) {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
connCfg := &rpcclient.ConnConfig{
|
||||
connCfg := &client.ConnConfig{
|
||||
Host: rpcAddr,
|
||||
Endpoint: "ws",
|
||||
User: cfg.RPCUser,
|
||||
|
@ -3,7 +3,7 @@ package main
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/kaspanet/kaspad/logs"
|
||||
"github.com/kaspanet/kaspad/rpcclient"
|
||||
"github.com/kaspanet/kaspad/rpc/client"
|
||||
"github.com/kaspanet/kaspad/util/panics"
|
||||
"os"
|
||||
)
|
||||
@ -28,5 +28,5 @@ func initLog(logFile, errLogFile string) {
|
||||
}
|
||||
|
||||
func enableRPCLogging() {
|
||||
rpcclient.UseLogger(backendLog, logs.LevelTrace)
|
||||
client.UseLogger(backendLog, logs.LevelTrace)
|
||||
}
|
||||
|
@ -7,8 +7,8 @@ import (
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/kaspanet/kaspad/rpcclient"
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
clientpkg "github.com/kaspanet/kaspad/rpc/client"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
"github.com/pkg/errors"
|
||||
@ -79,7 +79,7 @@ func logHashRate() {
|
||||
func mineNextBlock(client *minerClient, miningAddr util.Address, foundBlock chan *util.Block, mineWhenNotSynced bool,
|
||||
templateStopChan chan struct{}, errChan chan error) {
|
||||
|
||||
newTemplateChan := make(chan *rpcmodel.GetBlockTemplateResult)
|
||||
newTemplateChan := make(chan *model.GetBlockTemplateResult)
|
||||
spawn("templatesLoop", func() {
|
||||
templatesLoop(client, miningAddr, newTemplateChan, errChan, templateStopChan)
|
||||
})
|
||||
@ -91,7 +91,7 @@ func mineNextBlock(client *minerClient, miningAddr util.Address, foundBlock chan
|
||||
func handleFoundBlock(client *minerClient, block *util.Block) error {
|
||||
log.Infof("Found block %s with parents %s. Submitting to %s", block.Hash(), block.MsgBlock().Header.ParentHashes, client.Host())
|
||||
|
||||
err := client.SubmitBlock(block, &rpcmodel.SubmitBlockOptions{})
|
||||
err := client.SubmitBlock(block, &model.SubmitBlockOptions{})
|
||||
if err != nil {
|
||||
return errors.Errorf("Error submitting block %s to %s: %s", block.Hash(), client.Host(), err)
|
||||
}
|
||||
@ -120,7 +120,7 @@ func solveBlock(block *util.Block, stopChan chan struct{}, foundBlock chan *util
|
||||
}
|
||||
|
||||
func templatesLoop(client *minerClient, miningAddr util.Address,
|
||||
newTemplateChan chan *rpcmodel.GetBlockTemplateResult, errChan chan error, stopChan chan struct{}) {
|
||||
newTemplateChan chan *model.GetBlockTemplateResult, errChan chan error, stopChan chan struct{}) {
|
||||
|
||||
longPollID := ""
|
||||
getBlockTemplateLongPoll := func() {
|
||||
@ -130,7 +130,7 @@ func templatesLoop(client *minerClient, miningAddr util.Address,
|
||||
log.Infof("Requesting template without longPollID from %s", client.Host())
|
||||
}
|
||||
template, err := getBlockTemplate(client, miningAddr, longPollID)
|
||||
if nativeerrors.Is(err, rpcclient.ErrResponseTimedOut) {
|
||||
if nativeerrors.Is(err, clientpkg.ErrResponseTimedOut) {
|
||||
log.Infof("Got timeout while requesting template '%s' from %s", longPollID, client.Host())
|
||||
return
|
||||
} else if err != nil {
|
||||
@ -157,11 +157,11 @@ func templatesLoop(client *minerClient, miningAddr util.Address,
|
||||
}
|
||||
}
|
||||
|
||||
func getBlockTemplate(client *minerClient, miningAddr util.Address, longPollID string) (*rpcmodel.GetBlockTemplateResult, error) {
|
||||
func getBlockTemplate(client *minerClient, miningAddr util.Address, longPollID string) (*model.GetBlockTemplateResult, error) {
|
||||
return client.GetBlockTemplate(miningAddr.String(), longPollID)
|
||||
}
|
||||
|
||||
func solveLoop(newTemplateChan chan *rpcmodel.GetBlockTemplateResult, foundBlock chan *util.Block,
|
||||
func solveLoop(newTemplateChan chan *model.GetBlockTemplateResult, foundBlock chan *util.Block,
|
||||
mineWhenNotSynced bool, errChan chan error) {
|
||||
|
||||
var stopOldTemplateSolving chan struct{}
|
||||
@ -179,7 +179,7 @@ func solveLoop(newTemplateChan chan *rpcmodel.GetBlockTemplateResult, foundBlock
|
||||
}
|
||||
|
||||
stopOldTemplateSolving = make(chan struct{})
|
||||
block, err := rpcclient.ConvertGetBlockTemplateResultToBlock(template)
|
||||
block, err := clientpkg.ConvertGetBlockTemplateResultToBlock(template)
|
||||
if err != nil {
|
||||
errChan <- errors.Errorf("Error parsing block: %s", err)
|
||||
return
|
||||
|
@ -102,3 +102,10 @@ func (c *ConnectionManager) AddConnectionRequest(address string, isPermanent boo
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
// RemoveConnection disconnects the connection for the given address
|
||||
// and removes it entirely from the connection manager.
|
||||
func (c *ConnectionManager) RemoveConnection(address string) {
|
||||
// TODO(libp2p): unimplemented
|
||||
panic("unimplemented")
|
||||
}
|
||||
|
@ -107,3 +107,8 @@ func (c *ConnectionManager) connectionsLoop() {
|
||||
<-time.Tick(connectionsLoopInterval)
|
||||
}
|
||||
}
|
||||
|
||||
// ConnectionCount returns the count of the connected connections
|
||||
func (c *ConnectionManager) ConnectionCount() int {
|
||||
return c.netAdapter.ConnectionCount()
|
||||
}
|
||||
|
@ -1,27 +0,0 @@
|
||||
connmgr
|
||||
=======
|
||||
|
||||
[](https://choosealicense.com/licenses/isc/)
|
||||
[](http://godoc.org/github.com/kaspanet/kaspad/connmgr)
|
||||
|
||||
Package connmgr implements a generic Kaspa network connection manager.
|
||||
|
||||
## Overview
|
||||
|
||||
Connection Manager handles all the general connection concerns such as
|
||||
maintaining a set number of outbound connections, sourcing peers, banning,
|
||||
limiting max connections, etc.
|
||||
|
||||
The package provides a generic connection manager which is able to accept
|
||||
connection requests from a source or a set of given addresses, dial them and
|
||||
notify the caller on connections. The main intended use is to initialize a pool
|
||||
of active connections and maintain them to remain connected to the P2P network.
|
||||
|
||||
In addition the connection manager provides the following utilities:
|
||||
|
||||
- Notifications on connections or disconnections
|
||||
- Handle failures and retry new addresses from the source
|
||||
- Connect only to specified addresses
|
||||
- Permanent connections with increasing backoff retry timers
|
||||
- Disconnect or Remove an established connection
|
||||
|
@ -1,782 +0,0 @@
|
||||
// Copyright (c) 2016 The btcsuite developers
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package connmgr
|
||||
|
||||
import (
|
||||
nativeerrors "errors"
|
||||
"fmt"
|
||||
"net"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
|
||||
"github.com/kaspanet/kaspad/addrmgr"
|
||||
"github.com/kaspanet/kaspad/config"
|
||||
"github.com/kaspanet/kaspad/wire"
|
||||
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
// maxFailedAttempts is the maximum number of successive failed connection
|
||||
// attempts after which network failure is assumed and new connections will
|
||||
// be delayed by the configured retry duration.
|
||||
const maxFailedAttempts = 25
|
||||
|
||||
var (
|
||||
// maxRetryDuration is the max duration of time retrying of a persistent
|
||||
// connection is allowed to grow to. This is necessary since the retry
|
||||
// logic uses a backoff mechanism which increases the interval base times
|
||||
// the number of retries that have been done.
|
||||
maxRetryDuration = time.Minute * 5
|
||||
|
||||
// defaultRetryDuration is the default duration of time for retrying
|
||||
// persistent connections.
|
||||
defaultRetryDuration = time.Second * 5
|
||||
)
|
||||
|
||||
var (
|
||||
//ErrDialNil is used to indicate that Dial cannot be nil in the configuration.
|
||||
ErrDialNil = errors.New("Config: Dial cannot be nil")
|
||||
|
||||
// ErrMaxOutboundPeers is an error that is thrown when the max amount of peers had
|
||||
// been reached.
|
||||
ErrMaxOutboundPeers = errors.New("max outbound peers reached")
|
||||
|
||||
// ErrAlreadyConnected is an error that is thrown if the peer is already
|
||||
// connected.
|
||||
ErrAlreadyConnected = errors.New("peer already connected")
|
||||
|
||||
// ErrAlreadyPermanent is an error that is thrown if the peer is already
|
||||
// connected as a permanent peer.
|
||||
ErrAlreadyPermanent = errors.New("peer exists as a permanent peer")
|
||||
|
||||
// ErrPeerNotFound is an error that is thrown if the peer was not found.
|
||||
ErrPeerNotFound = errors.New("peer not found")
|
||||
|
||||
//ErrAddressManagerNil is used to indicate that Address Manager cannot be nil in the configuration.
|
||||
ErrAddressManagerNil = errors.New("Config: Address manager cannot be nil")
|
||||
)
|
||||
|
||||
// ConnState represents the state of the requested connection.
|
||||
type ConnState uint8
|
||||
|
||||
// ConnState can be either pending, established, disconnected or failed. When
|
||||
// a new connection is requested, it is attempted and categorized as
|
||||
// established or failed depending on the connection result. An established
|
||||
// connection which was disconnected is categorized as disconnected.
|
||||
const (
|
||||
ConnPending ConnState = iota
|
||||
ConnFailing
|
||||
ConnCanceled
|
||||
ConnEstablished
|
||||
ConnDisconnected
|
||||
)
|
||||
|
||||
// ConnReq is the connection request to a network address. If permanent, the
|
||||
// connection will be retried on disconnection.
|
||||
type ConnReq struct {
|
||||
// The following variables must only be used atomically.
|
||||
id uint64
|
||||
|
||||
Addr *net.TCPAddr
|
||||
Permanent bool
|
||||
|
||||
conn net.Conn
|
||||
state ConnState
|
||||
stateMtx sync.RWMutex
|
||||
retryCount uint32
|
||||
}
|
||||
|
||||
// updateState updates the state of the connection request.
|
||||
func (c *ConnReq) updateState(state ConnState) {
|
||||
c.stateMtx.Lock()
|
||||
defer c.stateMtx.Unlock()
|
||||
c.state = state
|
||||
}
|
||||
|
||||
// ID returns a unique identifier for the connection request.
|
||||
func (c *ConnReq) ID() uint64 {
|
||||
return atomic.LoadUint64(&c.id)
|
||||
}
|
||||
|
||||
// State is the connection state of the requested connection.
|
||||
func (c *ConnReq) State() ConnState {
|
||||
c.stateMtx.RLock()
|
||||
defer c.stateMtx.RUnlock()
|
||||
state := c.state
|
||||
return state
|
||||
}
|
||||
|
||||
// String returns a human-readable string for the connection request.
|
||||
func (c *ConnReq) String() string {
|
||||
if c.Addr == nil || c.Addr.String() == "" {
|
||||
return fmt.Sprintf("reqid %d", atomic.LoadUint64(&c.id))
|
||||
}
|
||||
return fmt.Sprintf("%s (reqid %d)", c.Addr, atomic.LoadUint64(&c.id))
|
||||
}
|
||||
|
||||
// Config holds the configuration options related to the connection manager.
|
||||
type Config struct {
|
||||
// Listeners defines a slice of listeners for which the connection
|
||||
// manager will take ownership of and accept connections. When a
|
||||
// connection is accepted, the OnAccept handler will be invoked with the
|
||||
// connection. Since the connection manager takes ownership of these
|
||||
// listeners, they will be closed when the connection manager is
|
||||
// stopped.
|
||||
//
|
||||
// This field will not have any effect if the OnAccept field is not
|
||||
// also specified. It may be nil if the caller does not wish to listen
|
||||
// for incoming connections.
|
||||
Listeners []net.Listener
|
||||
|
||||
// OnAccept is a callback that is fired when an inbound connection is
|
||||
// accepted. It is the caller's responsibility to close the connection.
|
||||
// Failure to close the connection will result in the connection manager
|
||||
// believing the connection is still active and thus have undesirable
|
||||
// side effects such as still counting toward maximum connection limits.
|
||||
//
|
||||
// This field will not have any effect if the Listeners field is not
|
||||
// also specified since there couldn't possibly be any accepted
|
||||
// connections in that case.
|
||||
OnAccept func(net.Conn)
|
||||
|
||||
// TargetOutbound is the number of outbound network connections to
|
||||
// maintain. Defaults to 8.
|
||||
TargetOutbound uint32
|
||||
|
||||
// RetryDuration is the duration to wait before retrying connection
|
||||
// requests. Defaults to 5s.
|
||||
RetryDuration time.Duration
|
||||
|
||||
// OnConnection is a callback that is fired when a new outbound
|
||||
// connection is established.
|
||||
OnConnection func(*ConnReq, net.Conn)
|
||||
|
||||
// OnConnectionFailed is a callback that is fired when a new outbound
|
||||
// connection has failed to be established.
|
||||
OnConnectionFailed func(*ConnReq)
|
||||
|
||||
// OnDisconnection is a callback that is fired when an outbound
|
||||
// connection is disconnected.
|
||||
OnDisconnection func(*ConnReq)
|
||||
|
||||
AddrManager *addrmgr.AddrManager
|
||||
|
||||
// Dial connects to the address on the named network. It cannot be nil.
|
||||
Dial func(net.Addr) (net.Conn, error)
|
||||
}
|
||||
|
||||
// registerPending is used to register a pending connection attempt. By
|
||||
// registering pending connection attempts we allow callers to cancel pending
|
||||
// connection attempts before their successful or in the case they're not
|
||||
// longer wanted.
|
||||
type registerPending struct {
|
||||
c *ConnReq
|
||||
done chan struct{}
|
||||
}
|
||||
|
||||
// handleConnected is used to queue a successful connection.
|
||||
type handleConnected struct {
|
||||
c *ConnReq
|
||||
conn net.Conn
|
||||
}
|
||||
|
||||
// handleDisconnected is used to remove a connection.
|
||||
type handleDisconnected struct {
|
||||
id uint64
|
||||
retry bool
|
||||
}
|
||||
|
||||
// handleFailed is used to remove a pending connection.
|
||||
type handleFailed struct {
|
||||
c *ConnReq
|
||||
err error
|
||||
}
|
||||
|
||||
// ConnManager provides a manager to handle network connections.
|
||||
type ConnManager struct {
|
||||
// The following variables must only be used atomically.
|
||||
connReqCount uint64
|
||||
start int32
|
||||
stop int32
|
||||
|
||||
addressMtx sync.Mutex
|
||||
usedOutboundGroups map[string]int64
|
||||
usedAddresses map[string]struct{}
|
||||
|
||||
cfg Config
|
||||
appCfg *config.Config
|
||||
wg sync.WaitGroup
|
||||
failedAttempts uint64
|
||||
requests chan interface{}
|
||||
quit chan struct{}
|
||||
}
|
||||
|
||||
// handleFailedConn handles a connection failed due to a disconnect or any
|
||||
// other failure. If permanent, it retries the connection after the configured
|
||||
// retry duration. Otherwise, if required, it makes a new connection request.
|
||||
// After maxFailedConnectionAttempts new connections will be retried after the
|
||||
// configured retry duration.
|
||||
func (cm *ConnManager) handleFailedConn(c *ConnReq, err error) {
|
||||
if atomic.LoadInt32(&cm.stop) != 0 {
|
||||
return
|
||||
}
|
||||
|
||||
// Don't write throttled logs more than once every throttledConnFailedLogInterval
|
||||
shouldWriteLog := shouldWriteConnFailedLog(err)
|
||||
if shouldWriteLog {
|
||||
// If we are to write a log, set its lastLogTime to now
|
||||
setConnFailedLastLogTime(err, time.Now())
|
||||
}
|
||||
|
||||
if c.Permanent {
|
||||
c.retryCount++
|
||||
d := time.Duration(c.retryCount) * cm.cfg.RetryDuration
|
||||
if d > maxRetryDuration {
|
||||
d = maxRetryDuration
|
||||
}
|
||||
if shouldWriteLog {
|
||||
log.Debugf("Retrying further connections to %s every %s", c, d)
|
||||
}
|
||||
spawnAfter("ConnManager.connect-withDelay", d, func() {
|
||||
cm.connect(c)
|
||||
})
|
||||
} else {
|
||||
if c.Addr != nil {
|
||||
cm.releaseAddress(c.Addr)
|
||||
}
|
||||
cm.failedAttempts++
|
||||
if cm.failedAttempts >= maxFailedAttempts {
|
||||
if shouldWriteLog {
|
||||
log.Debugf("Max failed connection attempts reached: [%d] "+
|
||||
"-- retrying further connections every %s", maxFailedAttempts,
|
||||
cm.cfg.RetryDuration)
|
||||
}
|
||||
spawnAfter("ConnManager.NewConnReq-withDelay", cm.cfg.RetryDuration, cm.NewConnReq)
|
||||
} else {
|
||||
spawn("ConnManager.NewConnReq", cm.NewConnReq)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
func (cm *ConnManager) releaseAddress(addr *net.TCPAddr) {
|
||||
cm.addressMtx.Lock()
|
||||
defer cm.addressMtx.Unlock()
|
||||
|
||||
groupKey := cm.usedOutboundGroupsKey(addr)
|
||||
cm.usedOutboundGroups[groupKey]--
|
||||
if cm.usedOutboundGroups[groupKey] < 0 {
|
||||
panic(fmt.Errorf("cm.usedOutboundGroups[%s] has a negative value of %d. This should never happen", groupKey, cm.usedOutboundGroups[groupKey]))
|
||||
}
|
||||
delete(cm.usedAddresses, usedAddressesKey(addr))
|
||||
}
|
||||
|
||||
func (cm *ConnManager) markAddressAsUsed(addr *net.TCPAddr) {
|
||||
cm.usedOutboundGroups[cm.usedOutboundGroupsKey(addr)]++
|
||||
cm.usedAddresses[usedAddressesKey(addr)] = struct{}{}
|
||||
}
|
||||
|
||||
func (cm *ConnManager) isOutboundGroupUsed(addr *net.TCPAddr) bool {
|
||||
_, ok := cm.usedOutboundGroups[cm.usedOutboundGroupsKey(addr)]
|
||||
return ok
|
||||
}
|
||||
|
||||
func (cm *ConnManager) isAddressUsed(addr *net.TCPAddr) bool {
|
||||
_, ok := cm.usedAddresses[usedAddressesKey(addr)]
|
||||
return ok
|
||||
}
|
||||
|
||||
func (cm *ConnManager) usedOutboundGroupsKey(addr *net.TCPAddr) string {
|
||||
// A fake service flag is used since it doesn't affect the group key.
|
||||
na := wire.NewNetAddress(addr, wire.SFNodeNetwork)
|
||||
return cm.cfg.AddrManager.GroupKey(na)
|
||||
}
|
||||
|
||||
func usedAddressesKey(addr *net.TCPAddr) string {
|
||||
return addr.String()
|
||||
}
|
||||
|
||||
// throttledError defines an error type whose logs get throttled. This is to
|
||||
// prevent flooding the logs with identical errors.
|
||||
type throttledError error
|
||||
|
||||
var (
|
||||
// throttledConnFailedLogInterval is the minimum duration of time between
|
||||
// the logs defined in throttledConnFailedLogs.
|
||||
throttledConnFailedLogInterval = time.Minute * 10
|
||||
|
||||
// throttledConnFailedLogs are logs that get written at most every
|
||||
// throttledConnFailedLogInterval. Each entry in this map defines a type
|
||||
// of error that we want to throttle. The value of each entry is the last
|
||||
// time that type of log had been written.
|
||||
throttledConnFailedLogs = map[throttledError]time.Time{
|
||||
ErrNoAddress: {},
|
||||
}
|
||||
|
||||
// ErrNoAddress is an error that is thrown when there aren't any
|
||||
// valid connection addresses.
|
||||
ErrNoAddress throttledError = errors.New("no valid connect address")
|
||||
)
|
||||
|
||||
// shouldWriteConnFailedLog resolves whether to write logs related to connection
|
||||
// failures. Errors that had not been previously registered in throttledConnFailedLogs
|
||||
// and non-error (nil values) must always be logged.
|
||||
func shouldWriteConnFailedLog(err error) bool {
|
||||
if err == nil {
|
||||
return true
|
||||
}
|
||||
lastLogTime, ok := throttledConnFailedLogs[err]
|
||||
return !ok || lastLogTime.Add(throttledConnFailedLogInterval).Before(time.Now())
|
||||
}
|
||||
|
||||
// setConnFailedLastLogTime sets the last log time of the specified error
|
||||
func setConnFailedLastLogTime(err error, lastLogTime time.Time) {
|
||||
var throttledErr throttledError
|
||||
nativeerrors.As(err, &throttledErr)
|
||||
throttledConnFailedLogs[err] = lastLogTime
|
||||
}
|
||||
|
||||
// connHandler handles all connection related requests. It must be run as a
|
||||
// goroutine.
|
||||
//
|
||||
// The connection handler makes sure that we maintain a pool of active outbound
|
||||
// connections so that we remain connected to the network. Connection requests
|
||||
// are processed and mapped by their assigned ids.
|
||||
func (cm *ConnManager) connHandler() {
|
||||
|
||||
var (
|
||||
// pending holds all registered conn requests that have yet to
|
||||
// succeed.
|
||||
pending = make(map[uint64]*ConnReq)
|
||||
|
||||
// conns represents the set of all actively connected peers.
|
||||
conns = make(map[uint64]*ConnReq, cm.cfg.TargetOutbound)
|
||||
)
|
||||
|
||||
out:
|
||||
for {
|
||||
select {
|
||||
case req := <-cm.requests:
|
||||
switch msg := req.(type) {
|
||||
|
||||
case registerPending:
|
||||
connReq := msg.c
|
||||
connReq.updateState(ConnPending)
|
||||
pending[msg.c.id] = connReq
|
||||
close(msg.done)
|
||||
|
||||
case handleConnected:
|
||||
connReq := msg.c
|
||||
|
||||
if _, ok := pending[connReq.id]; !ok {
|
||||
if msg.conn != nil {
|
||||
msg.conn.Close()
|
||||
}
|
||||
log.Debugf("Ignoring connection for "+
|
||||
"canceled connreq=%s", connReq)
|
||||
continue
|
||||
}
|
||||
|
||||
connReq.updateState(ConnEstablished)
|
||||
connReq.conn = msg.conn
|
||||
conns[connReq.id] = connReq
|
||||
log.Debugf("Connected to %s", connReq)
|
||||
connReq.retryCount = 0
|
||||
|
||||
delete(pending, connReq.id)
|
||||
|
||||
if cm.cfg.OnConnection != nil {
|
||||
cm.cfg.OnConnection(connReq, msg.conn)
|
||||
}
|
||||
|
||||
case handleDisconnected:
|
||||
connReq, ok := conns[msg.id]
|
||||
if !ok {
|
||||
connReq, ok = pending[msg.id]
|
||||
if !ok {
|
||||
log.Errorf("Unknown connid=%d",
|
||||
msg.id)
|
||||
continue
|
||||
}
|
||||
|
||||
// Pending connection was found, remove
|
||||
// it from pending map if we should
|
||||
// ignore a later, successful
|
||||
// connection.
|
||||
connReq.updateState(ConnCanceled)
|
||||
log.Debugf("Canceling: %s", connReq)
|
||||
delete(pending, msg.id)
|
||||
continue
|
||||
|
||||
}
|
||||
|
||||
// An existing connection was located, mark as
|
||||
// disconnected and execute disconnection
|
||||
// callback.
|
||||
log.Debugf("Disconnected from %s", connReq)
|
||||
delete(conns, msg.id)
|
||||
|
||||
if connReq.conn != nil {
|
||||
connReq.conn.Close()
|
||||
}
|
||||
|
||||
if cm.cfg.OnDisconnection != nil {
|
||||
spawn("cm.cfg.OnDisconnection", func() {
|
||||
cm.cfg.OnDisconnection(connReq)
|
||||
})
|
||||
}
|
||||
|
||||
// All internal state has been cleaned up, if
|
||||
// this connection is being removed, we will
|
||||
// make no further attempts with this request.
|
||||
if !msg.retry {
|
||||
connReq.updateState(ConnDisconnected)
|
||||
continue
|
||||
}
|
||||
|
||||
// Otherwise, we will attempt a reconnection.
|
||||
// The connection request is re added to the
|
||||
// pending map, so that subsequent processing
|
||||
// of connections and failures do not ignore
|
||||
// the request.
|
||||
connReq.updateState(ConnPending)
|
||||
log.Debugf("Reconnecting to %s",
|
||||
connReq)
|
||||
pending[msg.id] = connReq
|
||||
cm.handleFailedConn(connReq, nil)
|
||||
|
||||
case handleFailed:
|
||||
connReq := msg.c
|
||||
|
||||
if _, ok := pending[connReq.id]; !ok {
|
||||
log.Debugf("Ignoring connection for "+
|
||||
"canceled conn req: %s", connReq)
|
||||
continue
|
||||
}
|
||||
|
||||
connReq.updateState(ConnFailing)
|
||||
if shouldWriteConnFailedLog(msg.err) {
|
||||
log.Debugf("Failed to connect to %s: %s",
|
||||
connReq, msg.err)
|
||||
}
|
||||
cm.handleFailedConn(connReq, msg.err)
|
||||
|
||||
if cm.cfg.OnConnectionFailed != nil {
|
||||
cm.cfg.OnConnectionFailed(connReq)
|
||||
}
|
||||
}
|
||||
|
||||
case <-cm.quit:
|
||||
break out
|
||||
}
|
||||
}
|
||||
|
||||
cm.wg.Done()
|
||||
log.Trace("Connection handler done")
|
||||
}
|
||||
|
||||
// NotifyConnectionRequestComplete notifies the connection
|
||||
// manager that a peer had been successfully connected and
|
||||
// marked as good.
|
||||
func (cm *ConnManager) NotifyConnectionRequestComplete() {
|
||||
cm.failedAttempts = 0
|
||||
}
|
||||
|
||||
// NewConnReq creates a new connection request and connects to the
|
||||
// corresponding address.
|
||||
func (cm *ConnManager) NewConnReq() {
|
||||
if atomic.LoadInt32(&cm.stop) != 0 {
|
||||
return
|
||||
}
|
||||
|
||||
c := &ConnReq{}
|
||||
atomic.StoreUint64(&c.id, atomic.AddUint64(&cm.connReqCount, 1))
|
||||
|
||||
// Submit a request of a pending connection attempt to the connection
|
||||
// manager. By registering the id before the connection is even
|
||||
// established, we'll be able to later cancel the connection via the
|
||||
// Remove method.
|
||||
done := make(chan struct{})
|
||||
select {
|
||||
case cm.requests <- registerPending{c, done}:
|
||||
case <-cm.quit:
|
||||
return
|
||||
}
|
||||
|
||||
// Wait for the registration to successfully add the pending conn req to
|
||||
// the conn manager's internal state.
|
||||
select {
|
||||
case <-done:
|
||||
case <-cm.quit:
|
||||
return
|
||||
}
|
||||
err := cm.associateAddressToConnReq(c)
|
||||
if err != nil {
|
||||
select {
|
||||
case cm.requests <- handleFailed{c, err}:
|
||||
case <-cm.quit:
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
cm.connect(c)
|
||||
}
|
||||
|
||||
func (cm *ConnManager) associateAddressToConnReq(c *ConnReq) error {
|
||||
cm.addressMtx.Lock()
|
||||
defer cm.addressMtx.Unlock()
|
||||
|
||||
addr, err := cm.getNewAddress()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
cm.markAddressAsUsed(addr)
|
||||
c.Addr = addr
|
||||
return nil
|
||||
}
|
||||
|
||||
// Connect assigns an id and dials a connection to the address of the
|
||||
// connection request.
|
||||
func (cm *ConnManager) Connect(c *ConnReq) error {
|
||||
err := func() error {
|
||||
cm.addressMtx.Lock()
|
||||
defer cm.addressMtx.Unlock()
|
||||
|
||||
if cm.isAddressUsed(c.Addr) {
|
||||
return fmt.Errorf("address %s is already in use", c.Addr)
|
||||
}
|
||||
cm.markAddressAsUsed(c.Addr)
|
||||
return nil
|
||||
}()
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
cm.connect(c)
|
||||
return nil
|
||||
}
|
||||
|
||||
// connect assigns an id and dials a connection to the address of the
|
||||
// connection request. This function assumes that the connection address
|
||||
// has checked and already marked as used.
|
||||
func (cm *ConnManager) connect(c *ConnReq) {
|
||||
if atomic.LoadInt32(&cm.stop) != 0 {
|
||||
return
|
||||
}
|
||||
|
||||
if atomic.LoadUint64(&c.id) == 0 {
|
||||
atomic.StoreUint64(&c.id, atomic.AddUint64(&cm.connReqCount, 1))
|
||||
|
||||
// Submit a request of a pending connection attempt to the
|
||||
// connection manager. By registering the id before the
|
||||
// connection is even established, we'll be able to later
|
||||
// cancel the connection via the Remove method.
|
||||
done := make(chan struct{})
|
||||
select {
|
||||
case cm.requests <- registerPending{c, done}:
|
||||
case <-cm.quit:
|
||||
return
|
||||
}
|
||||
|
||||
// Wait for the registration to successfully add the pending
|
||||
// conn req to the conn manager's internal state.
|
||||
select {
|
||||
case <-done:
|
||||
case <-cm.quit:
|
||||
return
|
||||
}
|
||||
}
|
||||
|
||||
log.Debugf("Attempting to connect to %s", c)
|
||||
|
||||
conn, err := cm.cfg.Dial(c.Addr)
|
||||
if err != nil {
|
||||
select {
|
||||
case cm.requests <- handleFailed{c, err}:
|
||||
case <-cm.quit:
|
||||
}
|
||||
return
|
||||
}
|
||||
|
||||
select {
|
||||
case cm.requests <- handleConnected{c, conn}:
|
||||
case <-cm.quit:
|
||||
}
|
||||
}
|
||||
|
||||
// Disconnect disconnects the connection corresponding to the given connection
|
||||
// id. If permanent, the connection will be retried with an increasing backoff
|
||||
// duration.
|
||||
func (cm *ConnManager) Disconnect(id uint64) {
|
||||
if atomic.LoadInt32(&cm.stop) != 0 {
|
||||
return
|
||||
}
|
||||
|
||||
select {
|
||||
case cm.requests <- handleDisconnected{id, true}:
|
||||
case <-cm.quit:
|
||||
}
|
||||
}
|
||||
|
||||
// Remove removes the connection corresponding to the given connection id from
|
||||
// known connections.
|
||||
//
|
||||
// NOTE: This method can also be used to cancel a lingering connection attempt
|
||||
// that hasn't yet succeeded.
|
||||
func (cm *ConnManager) Remove(id uint64) {
|
||||
if atomic.LoadInt32(&cm.stop) != 0 {
|
||||
return
|
||||
}
|
||||
|
||||
select {
|
||||
case cm.requests <- handleDisconnected{id, false}:
|
||||
case <-cm.quit:
|
||||
}
|
||||
}
|
||||
|
||||
// listenHandler accepts incoming connections on a given listener. It must be
|
||||
// run as a goroutine.
|
||||
func (cm *ConnManager) listenHandler(listener net.Listener) {
|
||||
log.Infof("Server listening on %s", listener.Addr())
|
||||
for atomic.LoadInt32(&cm.stop) == 0 {
|
||||
conn, err := listener.Accept()
|
||||
if err != nil {
|
||||
// Only log the error if not forcibly shutting down.
|
||||
if atomic.LoadInt32(&cm.stop) == 0 {
|
||||
log.Errorf("Can't accept connection: %s", err)
|
||||
}
|
||||
continue
|
||||
}
|
||||
spawn("SPAWN_PLACEHOLDER_NAME", func() {
|
||||
cm.cfg.OnAccept(conn)
|
||||
})
|
||||
}
|
||||
|
||||
cm.wg.Done()
|
||||
log.Tracef("Listener handler done for %s", listener.Addr())
|
||||
}
|
||||
|
||||
// Start launches the connection manager and begins connecting to the network.
|
||||
func (cm *ConnManager) Start() {
|
||||
// Already started?
|
||||
if atomic.AddInt32(&cm.start, 1) != 1 {
|
||||
return
|
||||
}
|
||||
|
||||
log.Trace("Connection manager started")
|
||||
cm.wg.Add(1)
|
||||
spawn("SPAWN_PLACEHOLDER_NAME", cm.connHandler)
|
||||
|
||||
// Start all the listeners so long as the caller requested them and
|
||||
// provided a callback to be invoked when connections are accepted.
|
||||
if cm.cfg.OnAccept != nil {
|
||||
for _, listener := range cm.cfg.Listeners {
|
||||
// Declaring this variable is necessary as it needs be declared in the same
|
||||
// scope of the anonymous function below it.
|
||||
listenerCopy := listener
|
||||
cm.wg.Add(1)
|
||||
spawn("SPAWN_PLACEHOLDER_NAME", func() {
|
||||
cm.listenHandler(listenerCopy)
|
||||
})
|
||||
}
|
||||
}
|
||||
|
||||
for i := atomic.LoadUint64(&cm.connReqCount); i < uint64(cm.cfg.TargetOutbound); i++ {
|
||||
spawn("SPAWN_PLACEHOLDER_NAME", cm.NewConnReq)
|
||||
}
|
||||
}
|
||||
|
||||
// Wait blocks until the connection manager halts gracefully.
|
||||
func (cm *ConnManager) Wait() {
|
||||
cm.wg.Wait()
|
||||
}
|
||||
|
||||
// Stop gracefully shuts down the connection manager.
|
||||
func (cm *ConnManager) Stop() {
|
||||
if atomic.AddInt32(&cm.stop, 1) != 1 {
|
||||
log.Warnf("Connection manager already stopped")
|
||||
return
|
||||
}
|
||||
|
||||
// Stop all the listeners. There will not be any listeners if
|
||||
// listening is disabled.
|
||||
for _, listener := range cm.cfg.Listeners {
|
||||
// Ignore the error since this is shutdown and there is no way
|
||||
// to recover anyways.
|
||||
_ = listener.Close()
|
||||
}
|
||||
|
||||
close(cm.quit)
|
||||
log.Trace("Connection manager stopped")
|
||||
}
|
||||
|
||||
func (cm *ConnManager) getNewAddress() (*net.TCPAddr, error) {
|
||||
for tries := 0; tries < 100; tries++ {
|
||||
addr := cm.cfg.AddrManager.GetAddress()
|
||||
if addr == nil {
|
||||
break
|
||||
}
|
||||
|
||||
// Check if there's already a connection to the same address.
|
||||
netAddr := addr.NetAddress().TCPAddress()
|
||||
if cm.isAddressUsed(netAddr) {
|
||||
continue
|
||||
}
|
||||
|
||||
// Address will not be invalid, local or unroutable
|
||||
// because addrmanager rejects those on addition.
|
||||
// Just check that we don't already have an address
|
||||
// in the same group so that we are not connecting
|
||||
// to the same network segment at the expense of
|
||||
// others.
|
||||
//
|
||||
// Networks that accept unroutable connections are exempt
|
||||
// from this rule, since they're meant to run within a
|
||||
// private subnet, like 10.0.0.0/16.
|
||||
if !cm.appCfg.NetParams().AcceptUnroutable && cm.isOutboundGroupUsed(netAddr) {
|
||||
continue
|
||||
}
|
||||
|
||||
// only allow recent nodes (10mins) after we failed 30
|
||||
// times
|
||||
if tries < 30 && time.Since(addr.LastAttempt().ToNativeTime()) < 10*time.Minute {
|
||||
continue
|
||||
}
|
||||
|
||||
// allow nondefault ports after 50 failed tries.
|
||||
if tries < 50 && fmt.Sprintf("%d", netAddr.Port) !=
|
||||
cm.appCfg.NetParams().DefaultPort {
|
||||
continue
|
||||
}
|
||||
|
||||
return netAddr, nil
|
||||
}
|
||||
return nil, ErrNoAddress
|
||||
}
|
||||
|
||||
// New returns a new connection manager.
|
||||
// Use Start to start connecting to the network.
|
||||
func New(cfg *Config, appCfg *config.Config) (*ConnManager, error) {
|
||||
if cfg.Dial == nil {
|
||||
return nil, errors.WithStack(ErrDialNil)
|
||||
}
|
||||
if cfg.AddrManager == nil {
|
||||
return nil, errors.WithStack(ErrAddressManagerNil)
|
||||
}
|
||||
// Default to sane values
|
||||
if cfg.RetryDuration <= 0 {
|
||||
cfg.RetryDuration = defaultRetryDuration
|
||||
}
|
||||
cm := ConnManager{
|
||||
cfg: *cfg, // Copy so caller can't mutate
|
||||
appCfg: appCfg,
|
||||
requests: make(chan interface{}),
|
||||
quit: make(chan struct{}),
|
||||
usedAddresses: make(map[string]struct{}),
|
||||
usedOutboundGroups: make(map[string]int64),
|
||||
}
|
||||
return &cm, nil
|
||||
}
|
@ -1,955 +0,0 @@
|
||||
// Copyright (c) 2016 The btcsuite developers
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package connmgr
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"io"
|
||||
"io/ioutil"
|
||||
"net"
|
||||
"sync/atomic"
|
||||
"testing"
|
||||
"time"
|
||||
|
||||
"github.com/kaspanet/kaspad/addrmgr"
|
||||
"github.com/kaspanet/kaspad/config"
|
||||
"github.com/kaspanet/kaspad/dagconfig"
|
||||
"github.com/kaspanet/kaspad/dbaccess"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
func init() {
|
||||
// Override the max retry duration when running tests.
|
||||
maxRetryDuration = 2 * time.Millisecond
|
||||
}
|
||||
|
||||
func defaultAppConfig() *config.Config {
|
||||
return &config.Config{
|
||||
Flags: &config.Flags{
|
||||
NetworkFlags: config.NetworkFlags{
|
||||
ActiveNetParams: &dagconfig.SimnetParams},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// mockAddr mocks a network address
|
||||
type mockAddr struct {
|
||||
net, address string
|
||||
}
|
||||
|
||||
func (m mockAddr) Network() string { return m.net }
|
||||
func (m mockAddr) String() string { return m.address }
|
||||
|
||||
// mockConn mocks a network connection by implementing the net.Conn interface.
|
||||
type mockConn struct {
|
||||
io.Reader
|
||||
io.Writer
|
||||
io.Closer
|
||||
|
||||
// local network, address for the connection.
|
||||
lnet, laddr string
|
||||
|
||||
// remote network, address for the connection.
|
||||
rAddr net.Addr
|
||||
}
|
||||
|
||||
// LocalAddr returns the local address for the connection.
|
||||
func (c mockConn) LocalAddr() net.Addr {
|
||||
return &mockAddr{c.lnet, c.laddr}
|
||||
}
|
||||
|
||||
// RemoteAddr returns the remote address for the connection.
|
||||
func (c mockConn) RemoteAddr() net.Addr {
|
||||
return &mockAddr{c.rAddr.Network(), c.rAddr.String()}
|
||||
}
|
||||
|
||||
// Close handles closing the connection.
|
||||
func (c mockConn) Close() error {
|
||||
return nil
|
||||
}
|
||||
|
||||
func (c mockConn) SetDeadline(t time.Time) error { return nil }
|
||||
func (c mockConn) SetReadDeadline(t time.Time) error { return nil }
|
||||
func (c mockConn) SetWriteDeadline(t time.Time) error { return nil }
|
||||
|
||||
// mockDialer mocks the net.Dial interface by returning a mock connection to
|
||||
// the given address.
|
||||
func mockDialer(addr net.Addr) (net.Conn, error) {
|
||||
r, w := io.Pipe()
|
||||
c := &mockConn{rAddr: addr}
|
||||
c.Reader = r
|
||||
c.Writer = w
|
||||
return c, nil
|
||||
}
|
||||
|
||||
// TestNewConfig tests that new ConnManager config is validated as expected.
|
||||
func TestNewConfig(t *testing.T) {
|
||||
appCfg := defaultAppConfig()
|
||||
_, err := New(&Config{}, appCfg)
|
||||
if !errors.Is(err, ErrDialNil) {
|
||||
t.Fatalf("New expected error: %s, got %s", ErrDialNil, err)
|
||||
}
|
||||
|
||||
_, err = New(&Config{
|
||||
Dial: mockDialer,
|
||||
}, appCfg)
|
||||
if !errors.Is(err, ErrAddressManagerNil) {
|
||||
t.Fatalf("New expected error: %s, got %s", ErrAddressManagerNil, err)
|
||||
}
|
||||
|
||||
amgr, teardown := addressManagerForTest(t, "TestNewConfig", defaultAppConfig(), 10)
|
||||
defer teardown()
|
||||
|
||||
_, err = New(&Config{
|
||||
Dial: mockDialer,
|
||||
AddrManager: amgr,
|
||||
}, appCfg)
|
||||
if err != nil {
|
||||
t.Fatalf("New unexpected error: %v", err)
|
||||
}
|
||||
}
|
||||
|
||||
// TestStartStop tests that the connection manager starts and stops as
|
||||
// expected.
|
||||
func TestStartStop(t *testing.T) {
|
||||
connected := make(chan *ConnReq)
|
||||
disconnected := make(chan *ConnReq)
|
||||
|
||||
amgr, teardown := addressManagerForTest(t, "TestStartStop", defaultAppConfig(), 10)
|
||||
defer teardown()
|
||||
|
||||
cmgr, err := New(&Config{
|
||||
TargetOutbound: 1,
|
||||
AddrManager: amgr,
|
||||
Dial: mockDialer,
|
||||
OnConnection: func(c *ConnReq, conn net.Conn) {
|
||||
connected <- c
|
||||
},
|
||||
OnDisconnection: func(c *ConnReq) {
|
||||
disconnected <- c
|
||||
},
|
||||
}, defaultAppConfig())
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error from New: %s", err)
|
||||
}
|
||||
cmgr.Start()
|
||||
gotConnReq := <-connected
|
||||
cmgr.Stop()
|
||||
// already stopped
|
||||
cmgr.Stop()
|
||||
// ignored
|
||||
cr := &ConnReq{
|
||||
Addr: &net.TCPAddr{
|
||||
IP: net.ParseIP("127.0.0.1"),
|
||||
Port: 18555,
|
||||
},
|
||||
Permanent: true,
|
||||
}
|
||||
err = cmgr.Connect(cr)
|
||||
if err != nil {
|
||||
t.Fatalf("Connect error: %s", err)
|
||||
}
|
||||
if cr.ID() != 0 {
|
||||
t.Fatalf("start/stop: got id: %v, want: 0", cr.ID())
|
||||
}
|
||||
cmgr.Disconnect(gotConnReq.ID())
|
||||
cmgr.Remove(gotConnReq.ID())
|
||||
select {
|
||||
case <-disconnected:
|
||||
t.Fatalf("start/stop: unexpected disconnection")
|
||||
case <-time.Tick(10 * time.Millisecond):
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
func addressManagerForTest(t *testing.T, testName string, appConfig *config.Config, numAddresses uint8) (*addrmgr.AddrManager, func()) {
|
||||
amgr, teardown := createEmptyAddressManagerForTest(t, testName, appConfig)
|
||||
|
||||
for i := uint8(0); i < numAddresses; i++ {
|
||||
ip := fmt.Sprintf("173.%d.115.66:16511", i)
|
||||
err := amgr.AddAddressByIP(ip, nil)
|
||||
if err != nil {
|
||||
t.Fatalf("AddAddressByIP unexpectedly failed to add IP %s: %s", ip, err)
|
||||
}
|
||||
}
|
||||
|
||||
return amgr, teardown
|
||||
}
|
||||
|
||||
func createEmptyAddressManagerForTest(t *testing.T, testName string, appConfig *config.Config) (*addrmgr.AddrManager, func()) {
|
||||
path, err := ioutil.TempDir("", fmt.Sprintf("%s-database", testName))
|
||||
if err != nil {
|
||||
t.Fatalf("createEmptyAddressManagerForTest: TempDir unexpectedly "+
|
||||
"failed: %s", err)
|
||||
}
|
||||
|
||||
databaseContext, err := dbaccess.New(path)
|
||||
if err != nil {
|
||||
t.Fatalf("error creating db: %s", err)
|
||||
}
|
||||
|
||||
return addrmgr.New(appConfig, databaseContext), func() {
|
||||
// Wait for the connection manager to finish, so it'll
|
||||
// have access to the address manager as long as it's
|
||||
// alive.
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
|
||||
err := databaseContext.Close()
|
||||
if err != nil {
|
||||
t.Fatalf("error closing the database: %s", err)
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// TestConnectMode tests that the connection manager works in the connect mode.
|
||||
//
|
||||
// In connect mode, automatic connections are disabled, so we test that
|
||||
// requests using Connect are handled and that no other connections are made.
|
||||
func TestConnectMode(t *testing.T) {
|
||||
appConfig := defaultAppConfig()
|
||||
|
||||
connected := make(chan *ConnReq)
|
||||
amgr, teardown := addressManagerForTest(t, "TestConnectMode", appConfig, 10)
|
||||
defer teardown()
|
||||
|
||||
cmgr, err := New(&Config{
|
||||
TargetOutbound: 0,
|
||||
Dial: mockDialer,
|
||||
OnConnection: func(c *ConnReq, conn net.Conn) {
|
||||
connected <- c
|
||||
},
|
||||
AddrManager: amgr,
|
||||
}, appConfig)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error from New: %s", err)
|
||||
}
|
||||
cr := &ConnReq{
|
||||
Addr: &net.TCPAddr{
|
||||
IP: net.ParseIP("127.0.0.1"),
|
||||
Port: 18555,
|
||||
},
|
||||
Permanent: true,
|
||||
}
|
||||
cmgr.Start()
|
||||
cmgr.Connect(cr)
|
||||
gotConnReq := <-connected
|
||||
wantID := cr.ID()
|
||||
gotID := gotConnReq.ID()
|
||||
if gotID != wantID {
|
||||
t.Fatalf("connect mode: %v - want ID %v, got ID %v", cr.Addr, wantID, gotID)
|
||||
}
|
||||
gotState := cr.State()
|
||||
wantState := ConnEstablished
|
||||
if gotState != wantState {
|
||||
t.Fatalf("connect mode: %v - want state %v, got state %v", cr.Addr, wantState, gotState)
|
||||
}
|
||||
select {
|
||||
case c := <-connected:
|
||||
t.Fatalf("connect mode: got unexpected connection - %v", c.Addr)
|
||||
case <-time.After(time.Millisecond):
|
||||
break
|
||||
}
|
||||
cmgr.Stop()
|
||||
cmgr.Wait()
|
||||
}
|
||||
|
||||
// TestTargetOutbound tests the target number of outbound connections.
|
||||
//
|
||||
// We wait until all connections are established, then test they there are the
|
||||
// only connections made.
|
||||
func TestTargetOutbound(t *testing.T) {
|
||||
appConfig := defaultAppConfig()
|
||||
|
||||
const numAddressesInAddressManager = 10
|
||||
targetOutbound := uint32(numAddressesInAddressManager - 2)
|
||||
connected := make(chan *ConnReq)
|
||||
|
||||
amgr, teardown := addressManagerForTest(t, "TestTargetOutbound", appConfig, 10)
|
||||
defer teardown()
|
||||
|
||||
cmgr, err := New(&Config{
|
||||
TargetOutbound: targetOutbound,
|
||||
Dial: mockDialer,
|
||||
AddrManager: amgr,
|
||||
OnConnection: func(c *ConnReq, conn net.Conn) {
|
||||
connected <- c
|
||||
},
|
||||
}, defaultAppConfig())
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error from New: %s", err)
|
||||
}
|
||||
cmgr.Start()
|
||||
for i := uint32(0); i < targetOutbound; i++ {
|
||||
<-connected
|
||||
}
|
||||
|
||||
select {
|
||||
case c := <-connected:
|
||||
t.Fatalf("target outbound: got unexpected connection - %v", c.Addr)
|
||||
case <-time.After(time.Millisecond):
|
||||
break
|
||||
}
|
||||
cmgr.Stop()
|
||||
cmgr.Wait()
|
||||
}
|
||||
|
||||
// TestDuplicateOutboundConnections tests that connection requests cannot use an already used address.
|
||||
// It checks it by creating one connection request for each address in the address manager, so that
|
||||
// the next connection request will have to fail because no unused address will be available.
|
||||
func TestDuplicateOutboundConnections(t *testing.T) {
|
||||
appConfig := defaultAppConfig()
|
||||
|
||||
const numAddressesInAddressManager = 10
|
||||
targetOutbound := uint32(numAddressesInAddressManager - 1)
|
||||
connected := make(chan struct{})
|
||||
failedConnections := make(chan struct{})
|
||||
|
||||
amgr, teardown := addressManagerForTest(t, "TestDuplicateOutboundConnections", appConfig, 10)
|
||||
defer teardown()
|
||||
|
||||
cmgr, err := New(&Config{
|
||||
TargetOutbound: targetOutbound,
|
||||
Dial: mockDialer,
|
||||
AddrManager: amgr,
|
||||
OnConnection: func(c *ConnReq, conn net.Conn) {
|
||||
connected <- struct{}{}
|
||||
},
|
||||
OnConnectionFailed: func(_ *ConnReq) {
|
||||
failedConnections <- struct{}{}
|
||||
},
|
||||
}, defaultAppConfig())
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error from New: %s", err)
|
||||
}
|
||||
cmgr.Start()
|
||||
for i := uint32(0); i < targetOutbound; i++ {
|
||||
<-connected
|
||||
}
|
||||
|
||||
time.Sleep(time.Millisecond)
|
||||
|
||||
// Here we check that making a manual connection request beyond the target outbound connection
|
||||
// doesn't fail, so we can know that the reason such connection request will fail is an address
|
||||
// related issue.
|
||||
cmgr.NewConnReq()
|
||||
select {
|
||||
case <-connected:
|
||||
break
|
||||
case <-time.After(time.Millisecond):
|
||||
t.Fatalf("connection request unexpectedly didn't connect")
|
||||
}
|
||||
|
||||
select {
|
||||
case <-failedConnections:
|
||||
t.Fatalf("a connection request unexpectedly failed")
|
||||
case <-time.After(time.Millisecond):
|
||||
break
|
||||
}
|
||||
|
||||
// After we created numAddressesInAddressManager connection requests, this request should fail
|
||||
// because there aren't any more available addresses.
|
||||
cmgr.NewConnReq()
|
||||
select {
|
||||
case <-connected:
|
||||
t.Fatalf("connection request unexpectedly succeeded")
|
||||
case <-time.After(time.Millisecond):
|
||||
t.Fatalf("connection request didn't fail as expected")
|
||||
case <-failedConnections:
|
||||
break
|
||||
}
|
||||
|
||||
cmgr.Stop()
|
||||
cmgr.Wait()
|
||||
}
|
||||
|
||||
// TestSameOutboundGroupConnections tests that connection requests cannot use an address with an already used
|
||||
// address CIDR group.
|
||||
// It checks it by creating an address manager with only two addresses, that both belong to the same CIDR group
|
||||
// and checks that the second connection request fails.
|
||||
func TestSameOutboundGroupConnections(t *testing.T) {
|
||||
appConfig := defaultAppConfig()
|
||||
|
||||
amgr, teardown := createEmptyAddressManagerForTest(t, "TestSameOutboundGroupConnections", appConfig)
|
||||
defer teardown()
|
||||
|
||||
err := amgr.AddAddressByIP("173.190.115.66:16511", nil)
|
||||
if err != nil {
|
||||
t.Fatalf("AddAddressByIP unexpectedly failed: %s", err)
|
||||
}
|
||||
|
||||
err = amgr.AddAddressByIP("173.190.115.67:16511", nil)
|
||||
if err != nil {
|
||||
t.Fatalf("AddAddressByIP unexpectedly failed: %s", err)
|
||||
}
|
||||
|
||||
connected := make(chan struct{})
|
||||
failedConnections := make(chan struct{})
|
||||
cmgr, err := New(&Config{
|
||||
TargetOutbound: 0,
|
||||
Dial: mockDialer,
|
||||
AddrManager: amgr,
|
||||
OnConnection: func(c *ConnReq, conn net.Conn) {
|
||||
connected <- struct{}{}
|
||||
},
|
||||
OnConnectionFailed: func(_ *ConnReq) {
|
||||
failedConnections <- struct{}{}
|
||||
},
|
||||
}, appConfig)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error from New: %s", err)
|
||||
}
|
||||
|
||||
cmgr.Start()
|
||||
|
||||
cmgr.NewConnReq()
|
||||
select {
|
||||
case <-connected:
|
||||
break
|
||||
case <-time.After(time.Millisecond):
|
||||
t.Fatalf("connection request unexpectedly didn't connect")
|
||||
}
|
||||
|
||||
select {
|
||||
case <-failedConnections:
|
||||
t.Fatalf("a connection request unexpectedly failed")
|
||||
case <-time.After(time.Millisecond):
|
||||
break
|
||||
}
|
||||
|
||||
cmgr.NewConnReq()
|
||||
select {
|
||||
case <-connected:
|
||||
t.Fatalf("connection request unexpectedly succeeded")
|
||||
case <-time.After(time.Millisecond):
|
||||
t.Fatalf("connection request didn't fail as expected")
|
||||
case <-failedConnections:
|
||||
break
|
||||
}
|
||||
|
||||
cmgr.Stop()
|
||||
cmgr.Wait()
|
||||
}
|
||||
|
||||
// TestRetryPermanent tests that permanent connection requests are retried.
|
||||
//
|
||||
// We make a permanent connection request using Connect, disconnect it using
|
||||
// Disconnect and we wait for it to be connected back.
|
||||
func TestRetryPermanent(t *testing.T) {
|
||||
appConfig := defaultAppConfig()
|
||||
|
||||
connected := make(chan *ConnReq)
|
||||
disconnected := make(chan *ConnReq)
|
||||
|
||||
amgr, teardown := addressManagerForTest(t, "TestRetryPermanent", appConfig, 10)
|
||||
defer teardown()
|
||||
|
||||
cmgr, err := New(&Config{
|
||||
RetryDuration: time.Millisecond,
|
||||
TargetOutbound: 0,
|
||||
Dial: mockDialer,
|
||||
OnConnection: func(c *ConnReq, conn net.Conn) {
|
||||
connected <- c
|
||||
},
|
||||
OnDisconnection: func(c *ConnReq) {
|
||||
disconnected <- c
|
||||
},
|
||||
AddrManager: amgr,
|
||||
}, appConfig)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error from New: %s", err)
|
||||
}
|
||||
|
||||
cr := &ConnReq{
|
||||
Addr: &net.TCPAddr{
|
||||
IP: net.ParseIP("127.0.0.1"),
|
||||
Port: 18555,
|
||||
},
|
||||
Permanent: true,
|
||||
}
|
||||
go cmgr.Connect(cr)
|
||||
cmgr.Start()
|
||||
gotConnReq := <-connected
|
||||
wantID := cr.ID()
|
||||
gotID := gotConnReq.ID()
|
||||
if gotID != wantID {
|
||||
t.Fatalf("retry: %v - want ID %v, got ID %v", cr.Addr, wantID, gotID)
|
||||
}
|
||||
gotState := cr.State()
|
||||
wantState := ConnEstablished
|
||||
if gotState != wantState {
|
||||
t.Fatalf("retry: %v - want state %v, got state %v", cr.Addr, wantState, gotState)
|
||||
}
|
||||
|
||||
cmgr.Disconnect(cr.ID())
|
||||
gotConnReq = <-disconnected
|
||||
wantID = cr.ID()
|
||||
gotID = gotConnReq.ID()
|
||||
if gotID != wantID {
|
||||
t.Fatalf("retry: %v - want ID %v, got ID %v", cr.Addr, wantID, gotID)
|
||||
}
|
||||
gotState = cr.State()
|
||||
wantState = ConnPending
|
||||
if gotState != wantState {
|
||||
// There is a small chance that connection has already been established,
|
||||
// so check for that as well
|
||||
if gotState != ConnEstablished {
|
||||
t.Fatalf("retry: %v - want state %v, got state %v", cr.Addr, wantState, gotState)
|
||||
}
|
||||
}
|
||||
|
||||
gotConnReq = <-connected
|
||||
wantID = cr.ID()
|
||||
gotID = gotConnReq.ID()
|
||||
if gotID != wantID {
|
||||
t.Fatalf("retry: %v - want ID %v, got ID %v", cr.Addr, wantID, gotID)
|
||||
}
|
||||
gotState = cr.State()
|
||||
wantState = ConnEstablished
|
||||
if gotState != wantState {
|
||||
t.Fatalf("retry: %v - want state %v, got state %v", cr.Addr, wantState, gotState)
|
||||
}
|
||||
|
||||
cmgr.Remove(cr.ID())
|
||||
gotConnReq = <-disconnected
|
||||
|
||||
// Wait for status to be updated
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
wantID = cr.ID()
|
||||
gotID = gotConnReq.ID()
|
||||
if gotID != wantID {
|
||||
t.Fatalf("retry: %v - want ID %v, got ID %v", cr.Addr, wantID, gotID)
|
||||
}
|
||||
gotState = cr.State()
|
||||
wantState = ConnDisconnected
|
||||
if gotState != wantState {
|
||||
t.Fatalf("retry: %v - want state %v, got state %v", cr.Addr, wantState, gotState)
|
||||
}
|
||||
cmgr.Stop()
|
||||
cmgr.Wait()
|
||||
}
|
||||
|
||||
// TestMaxRetryDuration tests the maximum retry duration.
|
||||
//
|
||||
// We have a timed dialer which initially returns err but after RetryDuration
|
||||
// hits maxRetryDuration returns a mock conn.
|
||||
func TestMaxRetryDuration(t *testing.T) {
|
||||
appConfig := defaultAppConfig()
|
||||
|
||||
networkUp := make(chan struct{})
|
||||
time.AfterFunc(5*time.Millisecond, func() {
|
||||
close(networkUp)
|
||||
})
|
||||
timedDialer := func(addr net.Addr) (net.Conn, error) {
|
||||
select {
|
||||
case <-networkUp:
|
||||
return mockDialer(addr)
|
||||
default:
|
||||
return nil, errors.New("network down")
|
||||
}
|
||||
}
|
||||
|
||||
amgr, teardown := addressManagerForTest(t, "TestMaxRetryDuration", appConfig, 10)
|
||||
defer teardown()
|
||||
|
||||
connected := make(chan *ConnReq)
|
||||
cmgr, err := New(&Config{
|
||||
RetryDuration: time.Millisecond,
|
||||
TargetOutbound: 0,
|
||||
Dial: timedDialer,
|
||||
OnConnection: func(c *ConnReq, conn net.Conn) {
|
||||
connected <- c
|
||||
},
|
||||
AddrManager: amgr,
|
||||
}, appConfig)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error from New: %s", err)
|
||||
}
|
||||
|
||||
cr := &ConnReq{
|
||||
Addr: &net.TCPAddr{
|
||||
IP: net.ParseIP("127.0.0.1"),
|
||||
Port: 18555,
|
||||
},
|
||||
Permanent: true,
|
||||
}
|
||||
go cmgr.Connect(cr)
|
||||
cmgr.Start()
|
||||
// retry in 1ms
|
||||
// retry in 2ms - max retry duration reached
|
||||
// retry in 2ms - timedDialer returns mockDial
|
||||
select {
|
||||
case <-connected:
|
||||
case <-time.Tick(100 * time.Millisecond):
|
||||
t.Fatalf("max retry duration: connection timeout")
|
||||
}
|
||||
cmgr.Stop()
|
||||
cmgr.Wait()
|
||||
}
|
||||
|
||||
// TestNetworkFailure tests that the connection manager handles a network
|
||||
// failure gracefully.
|
||||
func TestNetworkFailure(t *testing.T) {
|
||||
appConfig := defaultAppConfig()
|
||||
|
||||
var dials uint32
|
||||
errDialer := func(net net.Addr) (net.Conn, error) {
|
||||
atomic.AddUint32(&dials, 1)
|
||||
return nil, errors.New("network down")
|
||||
}
|
||||
|
||||
amgr, teardown := addressManagerForTest(t, "TestNetworkFailure", appConfig, 10)
|
||||
defer teardown()
|
||||
|
||||
cmgr, err := New(&Config{
|
||||
TargetOutbound: 5,
|
||||
RetryDuration: 5 * time.Millisecond,
|
||||
Dial: errDialer,
|
||||
AddrManager: amgr,
|
||||
OnConnection: func(c *ConnReq, conn net.Conn) {
|
||||
t.Fatalf("network failure: got unexpected connection - %v", c.Addr)
|
||||
},
|
||||
}, appConfig)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error from New: %s", err)
|
||||
}
|
||||
cmgr.Start()
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
cmgr.Stop()
|
||||
cmgr.Wait()
|
||||
wantMaxDials := uint32(75)
|
||||
if atomic.LoadUint32(&dials) > wantMaxDials {
|
||||
t.Fatalf("network failure: unexpected number of dials - got %v, want < %v",
|
||||
atomic.LoadUint32(&dials), wantMaxDials)
|
||||
}
|
||||
}
|
||||
|
||||
// TestStopFailed tests that failed connections are ignored after connmgr is
|
||||
// stopped.
|
||||
//
|
||||
// We have a dailer which sets the stop flag on the conn manager and returns an
|
||||
// err so that the handler assumes that the conn manager is stopped and ignores
|
||||
// the failure.
|
||||
func TestStopFailed(t *testing.T) {
|
||||
appConfig := defaultAppConfig()
|
||||
|
||||
done := make(chan struct{}, 1)
|
||||
waitDialer := func(addr net.Addr) (net.Conn, error) {
|
||||
done <- struct{}{}
|
||||
time.Sleep(time.Millisecond)
|
||||
return nil, errors.New("network down")
|
||||
}
|
||||
|
||||
amgr, teardown := addressManagerForTest(t, "TestStopFailed", appConfig, 10)
|
||||
defer teardown()
|
||||
|
||||
cmgr, err := New(&Config{
|
||||
Dial: waitDialer,
|
||||
AddrManager: amgr,
|
||||
}, appConfig)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error from New: %s", err)
|
||||
}
|
||||
cmgr.Start()
|
||||
go func() {
|
||||
<-done
|
||||
atomic.StoreInt32(&cmgr.stop, 1)
|
||||
time.Sleep(2 * time.Millisecond)
|
||||
atomic.StoreInt32(&cmgr.stop, 0)
|
||||
cmgr.Stop()
|
||||
}()
|
||||
cr := &ConnReq{
|
||||
Addr: &net.TCPAddr{
|
||||
IP: net.ParseIP("127.0.0.1"),
|
||||
Port: 18555,
|
||||
},
|
||||
Permanent: true,
|
||||
}
|
||||
go cmgr.Connect(cr)
|
||||
cmgr.Wait()
|
||||
}
|
||||
|
||||
// TestRemovePendingConnection tests that it's possible to cancel a pending
|
||||
// connection, removing its internal state from the ConnMgr.
|
||||
func TestRemovePendingConnection(t *testing.T) {
|
||||
appConfig := defaultAppConfig()
|
||||
|
||||
// Create a ConnMgr instance with an instance of a dialer that'll never
|
||||
// succeed.
|
||||
wait := make(chan struct{})
|
||||
indefiniteDialer := func(addr net.Addr) (net.Conn, error) {
|
||||
<-wait
|
||||
return nil, errors.Errorf("error")
|
||||
}
|
||||
|
||||
amgr, teardown := addressManagerForTest(t, "TestRemovePendingConnection", appConfig, 10)
|
||||
defer teardown()
|
||||
|
||||
cmgr, err := New(&Config{
|
||||
Dial: indefiniteDialer,
|
||||
AddrManager: amgr,
|
||||
}, appConfig)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error from New: %s", err)
|
||||
}
|
||||
cmgr.Start()
|
||||
|
||||
// Establish a connection request to a random IP we've chosen.
|
||||
cr := &ConnReq{
|
||||
Addr: &net.TCPAddr{
|
||||
IP: net.ParseIP("127.0.0.1"),
|
||||
Port: 18555,
|
||||
},
|
||||
Permanent: true,
|
||||
}
|
||||
go cmgr.Connect(cr)
|
||||
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
|
||||
if cr.State() != ConnPending {
|
||||
t.Fatalf("pending request hasn't been registered, status: %v",
|
||||
cr.State())
|
||||
}
|
||||
|
||||
// The request launched above will actually never be able to establish
|
||||
// a connection. So we'll cancel it _before_ it's able to be completed.
|
||||
cmgr.Remove(cr.ID())
|
||||
|
||||
time.Sleep(10 * time.Millisecond)
|
||||
|
||||
// Now examine the status of the connection request, it should read a
|
||||
// status of failed.
|
||||
if cr.State() != ConnCanceled {
|
||||
t.Fatalf("request wasn't canceled, status is: %v", cr.State())
|
||||
}
|
||||
|
||||
close(wait)
|
||||
cmgr.Stop()
|
||||
cmgr.Wait()
|
||||
}
|
||||
|
||||
// TestCancelIgnoreDelayedConnection tests that a canceled connection request will
|
||||
// not execute the on connection callback, even if an outstanding retry
|
||||
// succeeds.
|
||||
func TestCancelIgnoreDelayedConnection(t *testing.T) {
|
||||
appConfig := defaultAppConfig()
|
||||
|
||||
retryTimeout := 10 * time.Millisecond
|
||||
|
||||
// Setup a dialer that will continue to return an error until the
|
||||
// connect chan is signaled, the dial attempt immediately after will
|
||||
// succeed in returning a connection.
|
||||
connect := make(chan struct{})
|
||||
failingDialer := func(addr net.Addr) (net.Conn, error) {
|
||||
select {
|
||||
case <-connect:
|
||||
return mockDialer(addr)
|
||||
default:
|
||||
}
|
||||
|
||||
return nil, errors.Errorf("error")
|
||||
}
|
||||
|
||||
connected := make(chan *ConnReq)
|
||||
|
||||
amgr, teardown := addressManagerForTest(t, "TestCancelIgnoreDelayedConnection", appConfig, 10)
|
||||
defer teardown()
|
||||
|
||||
cmgr, err := New(&Config{
|
||||
Dial: failingDialer,
|
||||
RetryDuration: retryTimeout,
|
||||
OnConnection: func(c *ConnReq, conn net.Conn) {
|
||||
connected <- c
|
||||
},
|
||||
AddrManager: amgr,
|
||||
}, appConfig)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error from New: %s", err)
|
||||
}
|
||||
cmgr.Start()
|
||||
|
||||
// Establish a connection request to a random IP we've chosen.
|
||||
cr := &ConnReq{
|
||||
Addr: &net.TCPAddr{
|
||||
IP: net.ParseIP("127.0.0.1"),
|
||||
Port: 18555,
|
||||
},
|
||||
}
|
||||
cmgr.Connect(cr)
|
||||
|
||||
// Allow for the first retry timeout to elapse.
|
||||
time.Sleep(2 * retryTimeout)
|
||||
|
||||
// Connection be marked as failed, even after reattempting to
|
||||
// connect.
|
||||
if cr.State() != ConnFailing {
|
||||
t.Fatalf("failing request should have status failed, status: %v",
|
||||
cr.State())
|
||||
}
|
||||
|
||||
// Remove the connection, and then immediately allow the next connection
|
||||
// to succeed.
|
||||
cmgr.Remove(cr.ID())
|
||||
close(connect)
|
||||
|
||||
// Allow the connection manager to process the removal.
|
||||
time.Sleep(5 * time.Millisecond)
|
||||
|
||||
// Now examine the status of the connection request, it should read a
|
||||
// status of canceled.
|
||||
if cr.State() != ConnCanceled {
|
||||
t.Fatalf("request wasn't canceled, status is: %v", cr.State())
|
||||
}
|
||||
|
||||
// Finally, the connection manager should not signal the on-connection
|
||||
// callback, since we explicitly canceled this request. We give a
|
||||
// generous window to ensure the connection manager's lienar backoff is
|
||||
// allowed to properly elapse.
|
||||
select {
|
||||
case <-connected:
|
||||
t.Fatalf("on-connect should not be called for canceled req")
|
||||
case <-time.After(5 * retryTimeout):
|
||||
}
|
||||
cmgr.Stop()
|
||||
cmgr.Wait()
|
||||
}
|
||||
|
||||
// mockListener implements the net.Listener interface and is used to test
|
||||
// code that deals with net.Listeners without having to actually make any real
|
||||
// connections.
|
||||
type mockListener struct {
|
||||
localAddr string
|
||||
provideConn chan net.Conn
|
||||
}
|
||||
|
||||
// Accept returns a mock connection when it receives a signal via the Connect
|
||||
// function.
|
||||
//
|
||||
// This is part of the net.Listener interface.
|
||||
func (m *mockListener) Accept() (net.Conn, error) {
|
||||
for conn := range m.provideConn {
|
||||
return conn, nil
|
||||
}
|
||||
return nil, errors.New("network connection closed")
|
||||
}
|
||||
|
||||
// Close closes the mock listener which will cause any blocked Accept
|
||||
// operations to be unblocked and return errors.
|
||||
//
|
||||
// This is part of the net.Listener interface.
|
||||
func (m *mockListener) Close() error {
|
||||
close(m.provideConn)
|
||||
return nil
|
||||
}
|
||||
|
||||
// Addr returns the address the mock listener was configured with.
|
||||
//
|
||||
// This is part of the net.Listener interface.
|
||||
func (m *mockListener) Addr() net.Addr {
|
||||
return &mockAddr{"tcp", m.localAddr}
|
||||
}
|
||||
|
||||
// Connect fakes a connection to the mock listener from the provided remote
|
||||
// address. It will cause the Accept function to return a mock connection
|
||||
// configured with the provided remote address and the local address for the
|
||||
// mock listener.
|
||||
func (m *mockListener) Connect(ip string, port int) {
|
||||
m.provideConn <- &mockConn{
|
||||
laddr: m.localAddr,
|
||||
lnet: "tcp",
|
||||
rAddr: &net.TCPAddr{
|
||||
IP: net.ParseIP(ip),
|
||||
Port: port,
|
||||
},
|
||||
}
|
||||
}
|
||||
|
||||
// newMockListener returns a new mock listener for the provided local address
|
||||
// and port. No ports are actually opened.
|
||||
func newMockListener(localAddr string) *mockListener {
|
||||
return &mockListener{
|
||||
localAddr: localAddr,
|
||||
provideConn: make(chan net.Conn),
|
||||
}
|
||||
}
|
||||
|
||||
// TestListeners ensures providing listeners to the connection manager along
|
||||
// with an accept callback works properly.
|
||||
func TestListeners(t *testing.T) {
|
||||
appConfig := defaultAppConfig()
|
||||
|
||||
// Setup a connection manager with a couple of mock listeners that
|
||||
// notify a channel when they receive mock connections.
|
||||
receivedConns := make(chan net.Conn)
|
||||
listener1 := newMockListener("127.0.0.1:16111")
|
||||
listener2 := newMockListener("127.0.0.1:9333")
|
||||
listeners := []net.Listener{listener1, listener2}
|
||||
|
||||
amgr, teardown := addressManagerForTest(t, "TestListeners", appConfig, 10)
|
||||
defer teardown()
|
||||
|
||||
cmgr, err := New(&Config{
|
||||
Listeners: listeners,
|
||||
OnAccept: func(conn net.Conn) {
|
||||
receivedConns <- conn
|
||||
},
|
||||
Dial: mockDialer,
|
||||
AddrManager: amgr,
|
||||
}, appConfig)
|
||||
if err != nil {
|
||||
t.Fatalf("unexpected error from New: %s", err)
|
||||
}
|
||||
cmgr.Start()
|
||||
|
||||
// Fake a couple of mock connections to each of the listeners.
|
||||
go func() {
|
||||
for i, listener := range listeners {
|
||||
l := listener.(*mockListener)
|
||||
l.Connect("127.0.0.1", 10000+i*2)
|
||||
l.Connect("127.0.0.1", 10000+i*2+1)
|
||||
}
|
||||
}()
|
||||
|
||||
// Tally the receive connections to ensure the expected number are
|
||||
// received. Also, fail the test after a timeout so it will not hang
|
||||
// forever should the test not work.
|
||||
expectedNumConns := len(listeners) * 2
|
||||
var numConns int
|
||||
out:
|
||||
for {
|
||||
select {
|
||||
case <-receivedConns:
|
||||
numConns++
|
||||
if numConns == expectedNumConns {
|
||||
break out
|
||||
}
|
||||
|
||||
case <-time.After(time.Millisecond * 50):
|
||||
t.Fatalf("Timeout waiting for %d expected connections",
|
||||
expectedNumConns)
|
||||
}
|
||||
}
|
||||
|
||||
cmgr.Stop()
|
||||
cmgr.Wait()
|
||||
}
|
||||
|
||||
// TestConnReqString ensures that ConnReq.String() does not crash
|
||||
func TestConnReqString(t *testing.T) {
|
||||
defer func() {
|
||||
if r := recover(); r != nil {
|
||||
t.Fatalf("ConnReq.String crashed %v", r)
|
||||
}
|
||||
}()
|
||||
cr1 := &ConnReq{
|
||||
Addr: &net.TCPAddr{
|
||||
IP: net.ParseIP("127.0.0.1"),
|
||||
Port: 18555,
|
||||
},
|
||||
Permanent: true,
|
||||
}
|
||||
_ = cr1.String()
|
||||
cr2 := &ConnReq{}
|
||||
_ = cr2.String()
|
||||
}
|
@ -1,23 +0,0 @@
|
||||
/*
|
||||
Package connmgr implements a generic Kaspa network connection manager.
|
||||
|
||||
Connection Manager Overview
|
||||
|
||||
Connection Manager handles all the general connection concerns such as
|
||||
maintaining a set number of outbound connections, sourcing peers, banning,
|
||||
limiting max connections, etc.
|
||||
|
||||
The package provides a generic connection manager which is able to accept
|
||||
connection requests from a source or a set of given addresses, dial them and
|
||||
notify the caller on connections. The main intended use is to initialize a pool
|
||||
of active connections and maintain them to remain connected to the P2P network.
|
||||
|
||||
In addition the connection manager provides the following utilities:
|
||||
|
||||
- Notifications on connections or disconnections
|
||||
- Handle failures and retry new addresses from the source
|
||||
- Connect only to specified addresses
|
||||
- Permanent connections with increasing backoff retry timers
|
||||
- Disconnect or Remove an established connection
|
||||
*/
|
||||
package connmgr
|
@ -1,144 +0,0 @@
|
||||
// Copyright (c) 2016 The btcsuite developers
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package connmgr
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"math"
|
||||
"sync"
|
||||
"time"
|
||||
)
|
||||
|
||||
const (
|
||||
// Halflife defines the time (in seconds) by which the transient part
|
||||
// of the ban score decays to one half of it's original value.
|
||||
Halflife = 60
|
||||
|
||||
// lambda is the decaying constant.
|
||||
lambda = math.Ln2 / Halflife
|
||||
|
||||
// Lifetime defines the maximum age of the transient part of the ban
|
||||
// score to be considered a non-zero score (in seconds).
|
||||
Lifetime = 1800
|
||||
|
||||
// precomputedLen defines the amount of decay factors (one per second) that
|
||||
// should be precomputed at initialization.
|
||||
precomputedLen = 64
|
||||
)
|
||||
|
||||
// precomputedFactor stores precomputed exponential decay factors for the first
|
||||
// 'precomputedLen' seconds starting from t == 0.
|
||||
var precomputedFactor [precomputedLen]float64
|
||||
|
||||
// init precomputes decay factors.
|
||||
func init() {
|
||||
for i := range precomputedFactor {
|
||||
precomputedFactor[i] = math.Exp(-1.0 * float64(i) * lambda)
|
||||
}
|
||||
}
|
||||
|
||||
// decayFactor returns the decay factor at t seconds, using precalculated values
|
||||
// if available, or calculating the factor if needed.
|
||||
func decayFactor(t int64) float64 {
|
||||
if t < precomputedLen {
|
||||
return precomputedFactor[t]
|
||||
}
|
||||
return math.Exp(-1.0 * float64(t) * lambda)
|
||||
}
|
||||
|
||||
// DynamicBanScore provides dynamic ban scores consisting of a persistent and a
|
||||
// decaying component.
|
||||
//
|
||||
// The decaying score enables the creation of evasive logic which handles
|
||||
// misbehaving peers (especially application layer DoS attacks) gracefully
|
||||
// by disconnecting and banning peers attempting various kinds of flooding.
|
||||
// DynamicBanScore allows these two approaches to be used in tandem.
|
||||
//
|
||||
// Zero value: Values of type DynamicBanScore are immediately ready for use upon
|
||||
// declaration.
|
||||
type DynamicBanScore struct {
|
||||
lastUnix int64
|
||||
transient float64
|
||||
persistent uint32
|
||||
mtx sync.Mutex
|
||||
}
|
||||
|
||||
// String returns the ban score as a human-readable string.
|
||||
func (s *DynamicBanScore) String() string {
|
||||
s.mtx.Lock()
|
||||
defer s.mtx.Unlock()
|
||||
r := fmt.Sprintf("persistent %d + transient %f at %d = %d as of now",
|
||||
s.persistent, s.transient, s.lastUnix, s.Int())
|
||||
return r
|
||||
}
|
||||
|
||||
// Int returns the current ban score, the sum of the persistent and decaying
|
||||
// scores.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (s *DynamicBanScore) Int() uint32 {
|
||||
s.mtx.Lock()
|
||||
defer s.mtx.Unlock()
|
||||
r := s.int(time.Now())
|
||||
return r
|
||||
}
|
||||
|
||||
// Increase increases both the persistent and decaying scores by the values
|
||||
// passed as parameters. The resulting score is returned.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (s *DynamicBanScore) Increase(persistent, transient uint32) uint32 {
|
||||
s.mtx.Lock()
|
||||
defer s.mtx.Unlock()
|
||||
r := s.increase(persistent, transient, time.Now())
|
||||
return r
|
||||
}
|
||||
|
||||
// Reset set both persistent and decaying scores to zero.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (s *DynamicBanScore) Reset() {
|
||||
s.mtx.Lock()
|
||||
defer s.mtx.Unlock()
|
||||
s.persistent = 0
|
||||
s.transient = 0
|
||||
s.lastUnix = 0
|
||||
}
|
||||
|
||||
// int returns the ban score, the sum of the persistent and decaying scores at a
|
||||
// given point in time.
|
||||
//
|
||||
// This function is not safe for concurrent access. It is intended to be used
|
||||
// internally and during testing.
|
||||
func (s *DynamicBanScore) int(t time.Time) uint32 {
|
||||
dt := t.Unix() - s.lastUnix
|
||||
if s.transient < 1 || dt < 0 || Lifetime < dt {
|
||||
return s.persistent
|
||||
}
|
||||
return s.persistent + uint32(s.transient*decayFactor(dt))
|
||||
}
|
||||
|
||||
// increase increases the persistent, the decaying or both scores by the values
|
||||
// passed as parameters. The resulting score is calculated as if the action was
|
||||
// carried out at the point time represented by the third parameter. The
|
||||
// resulting score is returned.
|
||||
//
|
||||
// This function is not safe for concurrent access.
|
||||
func (s *DynamicBanScore) increase(persistent, transient uint32, t time.Time) uint32 {
|
||||
s.persistent += persistent
|
||||
tu := t.Unix()
|
||||
dt := tu - s.lastUnix
|
||||
|
||||
if transient > 0 {
|
||||
if Lifetime < dt {
|
||||
s.transient = 0
|
||||
} else if s.transient > 1 && dt > 0 {
|
||||
s.transient *= decayFactor(dt)
|
||||
}
|
||||
s.transient += float64(transient)
|
||||
s.lastUnix = tu
|
||||
}
|
||||
return s.persistent + uint32(s.transient)
|
||||
}
|
@ -1,68 +0,0 @@
|
||||
// Copyright (c) 2016 The btcsuite developers
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package connmgr
|
||||
|
||||
import (
|
||||
"math"
|
||||
"testing"
|
||||
"time"
|
||||
)
|
||||
|
||||
// TestDynamicBanScoreDecay tests the exponential decay implemented in
|
||||
// DynamicBanScore.
|
||||
func TestDynamicBanScoreDecay(t *testing.T) {
|
||||
var bs DynamicBanScore
|
||||
base := time.Now()
|
||||
|
||||
r := bs.increase(100, 50, base)
|
||||
if r != 150 {
|
||||
t.Errorf("Unexpected result %d after ban score increase.", r)
|
||||
}
|
||||
|
||||
r = bs.int(base.Add(time.Minute))
|
||||
if r != 125 {
|
||||
t.Errorf("Halflife check failed - %d instead of 125", r)
|
||||
}
|
||||
|
||||
r = bs.int(base.Add(7 * time.Minute))
|
||||
if r != 100 {
|
||||
t.Errorf("Decay after 7m - %d instead of 100", r)
|
||||
}
|
||||
}
|
||||
|
||||
// TestDynamicBanScoreLifetime tests that DynamicBanScore properly yields zero
|
||||
// once the maximum age is reached.
|
||||
func TestDynamicBanScoreLifetime(t *testing.T) {
|
||||
var bs DynamicBanScore
|
||||
base := time.Now()
|
||||
|
||||
bs.increase(0, math.MaxUint32, base)
|
||||
r := bs.int(base.Add(Lifetime * time.Second))
|
||||
if r != 3 { // 3, not 4 due to precision loss and truncating 3.999...
|
||||
t.Errorf("Pre max age check with MaxUint32 failed - %d", r)
|
||||
}
|
||||
r = bs.int(base.Add((Lifetime + 1) * time.Second))
|
||||
if r != 0 {
|
||||
t.Errorf("Zero after max age check failed - %d instead of 0", r)
|
||||
}
|
||||
}
|
||||
|
||||
// TestDynamicBanScore tests exported functions of DynamicBanScore. Exponential
|
||||
// decay or other time based behavior is tested by other functions.
|
||||
func TestDynamicBanScoreReset(t *testing.T) {
|
||||
var bs DynamicBanScore
|
||||
if bs.Int() != 0 {
|
||||
t.Errorf("Initial state is not zero.")
|
||||
}
|
||||
bs.Increase(100, 0)
|
||||
r := bs.Int()
|
||||
if r != 100 {
|
||||
t.Errorf("Unexpected result %d after ban score increase.", r)
|
||||
}
|
||||
bs.Reset()
|
||||
if bs.Int() != 0 {
|
||||
t.Errorf("Failed to reset ban score.")
|
||||
}
|
||||
}
|
@ -1,14 +0,0 @@
|
||||
// Copyright (c) 2016 The btcsuite developers
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package connmgr
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/logger"
|
||||
"github.com/kaspanet/kaspad/util/panics"
|
||||
)
|
||||
|
||||
var log, _ = logger.Get(logger.SubsystemTags.CMGR)
|
||||
var spawn = panics.GoroutineWrapperFunc(log)
|
||||
var spawnAfter = panics.AfterFuncWrapperFunc(log)
|
100
connmgr/seed.go
100
connmgr/seed.go
@ -1,100 +0,0 @@
|
||||
// Copyright (c) 2016 The btcsuite developers
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package connmgr
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
mrand "math/rand"
|
||||
"net"
|
||||
"strconv"
|
||||
"time"
|
||||
|
||||
"github.com/kaspanet/kaspad/util/mstime"
|
||||
|
||||
"github.com/kaspanet/kaspad/config"
|
||||
"github.com/kaspanet/kaspad/util/subnetworkid"
|
||||
|
||||
"github.com/kaspanet/kaspad/dagconfig"
|
||||
"github.com/kaspanet/kaspad/wire"
|
||||
)
|
||||
|
||||
const (
|
||||
// These constants are used by the DNS seed code to pick a random last
|
||||
// seen time.
|
||||
secondsIn3Days int32 = 24 * 60 * 60 * 3
|
||||
secondsIn4Days int32 = 24 * 60 * 60 * 4
|
||||
|
||||
// SubnetworkIDPrefixChar is the prefix of subnetworkID, when building a DNS seed request
|
||||
SubnetworkIDPrefixChar byte = 'n'
|
||||
|
||||
// ServiceFlagPrefixChar is the prefix of service flag, when building a DNS seed request
|
||||
ServiceFlagPrefixChar byte = 'x'
|
||||
)
|
||||
|
||||
// OnSeed is the signature of the callback function which is invoked when DNS
|
||||
// seeding is succesfull.
|
||||
type OnSeed func(addrs []*wire.NetAddress)
|
||||
|
||||
// LookupFunc is the signature of the DNS lookup function.
|
||||
type LookupFunc func(string) ([]net.IP, error)
|
||||
|
||||
// SeedFromDNS uses DNS seeding to populate the address manager with peers.
|
||||
func SeedFromDNS(mainConfig *config.Config, dagParams *dagconfig.Params, reqServices wire.ServiceFlag, includeAllSubnetworks bool,
|
||||
subnetworkID *subnetworkid.SubnetworkID, lookupFn LookupFunc, seedFn OnSeed) {
|
||||
|
||||
var dnsSeeds []string
|
||||
if mainConfig != nil && mainConfig.DNSSeed != "" {
|
||||
dnsSeeds = []string{mainConfig.DNSSeed}
|
||||
} else {
|
||||
dnsSeeds = dagParams.DNSSeeds
|
||||
}
|
||||
|
||||
for _, dnsseed := range dnsSeeds {
|
||||
var host string
|
||||
if reqServices == wire.SFNodeNetwork {
|
||||
host = dnsseed
|
||||
} else {
|
||||
host = fmt.Sprintf("%c%x.%s", ServiceFlagPrefixChar, uint64(reqServices), dnsseed)
|
||||
}
|
||||
|
||||
if !includeAllSubnetworks {
|
||||
if subnetworkID != nil {
|
||||
host = fmt.Sprintf("%c%s.%s", SubnetworkIDPrefixChar, subnetworkID, host)
|
||||
} else {
|
||||
host = fmt.Sprintf("%c.%s", SubnetworkIDPrefixChar, host)
|
||||
}
|
||||
}
|
||||
|
||||
spawn("SPAWN_PLACEHOLDER_NAME", func() {
|
||||
randSource := mrand.New(mrand.NewSource(time.Now().UnixNano()))
|
||||
|
||||
seedpeers, err := lookupFn(host)
|
||||
if err != nil {
|
||||
log.Infof("DNS discovery failed on seed %s: %s", host, err)
|
||||
return
|
||||
}
|
||||
numPeers := len(seedpeers)
|
||||
|
||||
log.Infof("%d addresses found from DNS seed %s", numPeers, host)
|
||||
|
||||
if numPeers == 0 {
|
||||
return
|
||||
}
|
||||
addresses := make([]*wire.NetAddress, len(seedpeers))
|
||||
// if this errors then we have *real* problems
|
||||
intPort, _ := strconv.Atoi(dagParams.DefaultPort)
|
||||
for i, peer := range seedpeers {
|
||||
addresses[i] = wire.NewNetAddressTimestamp(
|
||||
// seed with addresses from a time randomly selected
|
||||
// between 3 and 7 days ago.
|
||||
mstime.Now().Add(-1*time.Second*time.Duration(secondsIn3Days+
|
||||
randSource.Int31n(secondsIn4Days))),
|
||||
0, peer, uint16(intPort))
|
||||
}
|
||||
|
||||
seedFn(addresses)
|
||||
})
|
||||
}
|
||||
}
|
11
kaspad.go
11
kaspad.go
@ -23,7 +23,7 @@ import (
|
||||
"github.com/kaspanet/kaspad/mempool"
|
||||
"github.com/kaspanet/kaspad/mining"
|
||||
"github.com/kaspanet/kaspad/protocol"
|
||||
"github.com/kaspanet/kaspad/server/rpc"
|
||||
"github.com/kaspanet/kaspad/rpc"
|
||||
"github.com/kaspanet/kaspad/signal"
|
||||
"github.com/kaspanet/kaspad/txscript"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
@ -135,7 +135,8 @@ func newKaspad(cfg *config.Config, databaseContext *dbaccess.DatabaseContext, in
|
||||
return nil, err
|
||||
}
|
||||
|
||||
rpcServer, err := setupRPC(cfg, dag, txMempool, sigCache, acceptanceIndex)
|
||||
rpcServer, err := setupRPC(cfg, dag, txMempool, sigCache, acceptanceIndex,
|
||||
connectionManager, addressManager, protocolManager)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -201,7 +202,8 @@ func setupMempool(cfg *config.Config, dag *blockdag.BlockDAG, sigCache *txscript
|
||||
}
|
||||
|
||||
func setupRPC(cfg *config.Config, dag *blockdag.BlockDAG, txMempool *mempool.TxPool, sigCache *txscript.SigCache,
|
||||
acceptanceIndex *indexers.AcceptanceIndex) (*rpc.Server, error) {
|
||||
acceptanceIndex *indexers.AcceptanceIndex, connectionManager *connmanager.ConnectionManager,
|
||||
addressManager *addrmgr.AddrManager, protocolManager *protocol.Manager) (*rpc.Server, error) {
|
||||
|
||||
if !cfg.DisableRPC {
|
||||
policy := mining.Policy{
|
||||
@ -209,7 +211,8 @@ func setupRPC(cfg *config.Config, dag *blockdag.BlockDAG, txMempool *mempool.TxP
|
||||
}
|
||||
blockTemplateGenerator := mining.NewBlkTmplGenerator(&policy, txMempool, dag, sigCache)
|
||||
|
||||
rpcServer, err := rpc.NewRPCServer(cfg, dag, txMempool, acceptanceIndex, blockTemplateGenerator)
|
||||
rpcServer, err := rpc.NewRPCServer(cfg, dag, txMempool, acceptanceIndex, blockTemplateGenerator,
|
||||
connectionManager, addressManager, protocolManager)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
@ -17,7 +17,6 @@ import (
|
||||
"github.com/kaspanet/kaspad/blockdag"
|
||||
"github.com/kaspanet/kaspad/logger"
|
||||
"github.com/kaspanet/kaspad/mining"
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/txscript"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
@ -1294,42 +1293,6 @@ func (mp *TxPool) MiningDescs() []*mining.TxDesc {
|
||||
return descs
|
||||
}
|
||||
|
||||
// RawMempoolVerbose returns all of the entries in the mempool as a fully
|
||||
// populated jsonrpc result.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (mp *TxPool) RawMempoolVerbose() map[string]*rpcmodel.GetRawMempoolVerboseResult {
|
||||
mp.mtx.RLock()
|
||||
defer mp.mtx.RUnlock()
|
||||
|
||||
result := make(map[string]*rpcmodel.GetRawMempoolVerboseResult, len(mp.pool))
|
||||
|
||||
for _, desc := range mp.pool {
|
||||
// Calculate the current priority based on the inputs to
|
||||
// the transaction. Use zero if one or more of the
|
||||
// input transactions can't be found for some reason.
|
||||
tx := desc.Tx
|
||||
|
||||
mpd := &rpcmodel.GetRawMempoolVerboseResult{
|
||||
Size: int32(tx.MsgTx().SerializeSize()),
|
||||
Fee: util.Amount(desc.Fee).ToKAS(),
|
||||
Time: desc.Added.UnixMilliseconds(),
|
||||
Depends: make([]string, 0),
|
||||
}
|
||||
for _, txIn := range tx.MsgTx().TxIn {
|
||||
txID := &txIn.PreviousOutpoint.TxID
|
||||
if mp.haveTransaction(txID) {
|
||||
mpd.Depends = append(mpd.Depends,
|
||||
txID.String())
|
||||
}
|
||||
}
|
||||
|
||||
result[tx.ID().String()] = mpd
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
||||
|
||||
// LastUpdated returns the last time a transaction was added to or removed from
|
||||
// the main pool. It does not include the orphan pool.
|
||||
//
|
||||
|
@ -7,6 +7,7 @@ package mining
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/util/mstime"
|
||||
"github.com/pkg/errors"
|
||||
"time"
|
||||
|
||||
"github.com/kaspanet/kaspad/blockdag"
|
||||
"github.com/kaspanet/kaspad/txscript"
|
||||
@ -228,3 +229,21 @@ func (g *BlkTmplGenerator) UpdateBlockTime(msgBlock *wire.MsgBlock) error {
|
||||
func (g *BlkTmplGenerator) TxSource() TxSource {
|
||||
return g.txSource
|
||||
}
|
||||
|
||||
// IsSynced checks if the node is synced enough based upon its worldview.
|
||||
// This is used to determine if the node can support mining and requesting newly-mined blocks.
|
||||
// To do that, first it checks if the selected tip timestamp is not older than maxTipAge. If that's the case, it means
|
||||
// the node is synced since blocks' timestamps are not allowed to deviate too much into the future.
|
||||
// If that's not the case it checks the rate it added new blocks to the DAG recently. If it's faster than
|
||||
// blockRate * maxSyncRateDeviation it means the node is not synced, since when the node is synced it shouldn't add
|
||||
// blocks to the DAG faster than the block rate.
|
||||
func (g *BlkTmplGenerator) IsSynced() bool {
|
||||
const maxTipAge = 5 * time.Minute
|
||||
isCloseToCurrentTime := g.dag.Now().Sub(g.dag.SelectedTipHeader().Timestamp) <= maxTipAge
|
||||
if isCloseToCurrentTime {
|
||||
return true
|
||||
}
|
||||
|
||||
const maxSyncRateDeviation = 1.05
|
||||
return g.dag.IsSyncRateBelowThreshold(maxSyncRateDeviation)
|
||||
}
|
||||
|
@ -17,7 +17,7 @@ import (
|
||||
|
||||
// RouterInitializer is a function that initializes a new
|
||||
// router to be used with a new connection
|
||||
type RouterInitializer func() (*routerpkg.Router, error)
|
||||
type RouterInitializer func(netConnection *NetConnection) (*routerpkg.Router, error)
|
||||
|
||||
// NetAdapter is an abstraction layer over networking.
|
||||
// This type expects a RouteInitializer function. This
|
||||
@ -99,15 +99,19 @@ func (na *NetAdapter) Connections() []*NetConnection {
|
||||
return netConnections
|
||||
}
|
||||
|
||||
// ConnectionCount returns the count of the connected connections
|
||||
func (na *NetAdapter) ConnectionCount() int {
|
||||
return len(na.connectionsToIDs)
|
||||
}
|
||||
|
||||
func (na *NetAdapter) onConnectedHandler(connection server.Connection) error {
|
||||
router, err := na.routerInitializer()
|
||||
netConnection := newNetConnection(connection, nil)
|
||||
router, err := na.routerInitializer(netConnection)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
connection.Start(router)
|
||||
|
||||
netConnection := newNetConnection(connection, nil)
|
||||
|
||||
na.routersToConnections[router] = netConnection
|
||||
|
||||
na.connectionsToIDs[netConnection] = nil
|
||||
|
@ -1,15 +0,0 @@
|
||||
netsync
|
||||
=======
|
||||
|
||||
[](https://choosealicense.com/licenses/isc/)
|
||||
[](http://godoc.org/github.com/kaspanet/kaspad/netsync)
|
||||
|
||||
## Overview
|
||||
|
||||
This package implements a concurrency safe block syncing protocol. The
|
||||
SyncManager communicates with connected peers to perform an initial block
|
||||
download, keep the chain and unconfirmed transaction pool in sync, and announce
|
||||
new blocks connected to the DAG. The sync manager selects a single
|
||||
sync peer that it downloads all blocks from until it is up to date with the
|
||||
the peer's selected tip.
|
||||
|
@ -1,77 +0,0 @@
|
||||
// Copyright (c) 2015-2017 The btcsuite developers
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package netsync
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/util/mstime"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
"github.com/kaspanet/kaspad/logs"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
)
|
||||
|
||||
// blockProgressLogger provides periodic logging for other services in order
|
||||
// to show users progress of certain "actions" involving some or all current
|
||||
// blocks. Ex: syncing, indexing all blocks, etc.
|
||||
type blockProgressLogger struct {
|
||||
receivedLogBlocks int64
|
||||
receivedLogTx int64
|
||||
lastBlockLogTime mstime.Time
|
||||
|
||||
subsystemLogger *logs.Logger
|
||||
progressAction string
|
||||
sync.Mutex
|
||||
}
|
||||
|
||||
// newBlockProgressLogger returns a new block progress logger.
|
||||
// The progress message is templated as follows:
|
||||
// {progressAction} {numProcessed} {blocks|block} in the last {timePeriod}
|
||||
// ({numTxs}, height {lastBlockHeight}, {lastBlockTimeStamp})
|
||||
func newBlockProgressLogger(progressMessage string, logger *logs.Logger) *blockProgressLogger {
|
||||
return &blockProgressLogger{
|
||||
lastBlockLogTime: mstime.Now(),
|
||||
progressAction: progressMessage,
|
||||
subsystemLogger: logger,
|
||||
}
|
||||
}
|
||||
|
||||
// LogBlockBlueScore logs a new block blue score as an information message
|
||||
// to show progress to the user. In order to prevent spam, it limits logging to
|
||||
// one message every 10 seconds with duration and totals included.
|
||||
func (b *blockProgressLogger) LogBlockBlueScore(block *util.Block, blueScore uint64) {
|
||||
b.Lock()
|
||||
defer b.Unlock()
|
||||
|
||||
b.receivedLogBlocks++
|
||||
b.receivedLogTx += int64(len(block.MsgBlock().Transactions))
|
||||
|
||||
now := mstime.Now()
|
||||
duration := now.Sub(b.lastBlockLogTime)
|
||||
if duration < time.Second*10 {
|
||||
return
|
||||
}
|
||||
|
||||
// Truncate the duration to 10s of milliseconds.
|
||||
durationMillis := int64(duration / time.Millisecond)
|
||||
tDuration := 10 * time.Millisecond * time.Duration(durationMillis/10)
|
||||
|
||||
// Log information about new block height.
|
||||
blockStr := "blocks"
|
||||
if b.receivedLogBlocks == 1 {
|
||||
blockStr = "block"
|
||||
}
|
||||
txStr := "transactions"
|
||||
if b.receivedLogTx == 1 {
|
||||
txStr = "transaction"
|
||||
}
|
||||
b.subsystemLogger.Infof("%s %d %s in the last %s (%d %s, blue score %d, %s)",
|
||||
b.progressAction, b.receivedLogBlocks, blockStr, tDuration, b.receivedLogTx,
|
||||
txStr, blueScore, block.MsgBlock().Header.Timestamp)
|
||||
|
||||
b.receivedLogBlocks = 0
|
||||
b.receivedLogTx = 0
|
||||
b.lastBlockLogTime = now
|
||||
}
|
@ -1,9 +0,0 @@
|
||||
/*
|
||||
Package netsync implements a concurrency safe block syncing protocol. The
|
||||
SyncManager communicates with connected peers to perform an initial block
|
||||
download, keep the DAG and unconfirmed transaction pool in sync, and announce
|
||||
new blocks connected to the DAG. Currently the sync manager selects a single
|
||||
sync peer that it downloads all blocks from until it is up to date with the
|
||||
selected tip of the sync peer.
|
||||
*/
|
||||
package netsync
|
@ -1,33 +0,0 @@
|
||||
// Copyright (c) 2017 The btcsuite developers
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package netsync
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/blockdag"
|
||||
"github.com/kaspanet/kaspad/dagconfig"
|
||||
"github.com/kaspanet/kaspad/mempool"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/kaspanet/kaspad/wire"
|
||||
)
|
||||
|
||||
// PeerNotifier exposes methods to notify peers of status changes to
|
||||
// transactions, blocks, etc. Currently server (in the main package) implements
|
||||
// this interface.
|
||||
type PeerNotifier interface {
|
||||
AnnounceNewTransactions(newTxs []*mempool.TxDesc)
|
||||
|
||||
RelayInventory(invVect *wire.InvVect, data interface{})
|
||||
|
||||
TransactionConfirmed(tx *util.Tx)
|
||||
}
|
||||
|
||||
// Config is a configuration struct used to initialize a new SyncManager.
|
||||
type Config struct {
|
||||
PeerNotifier PeerNotifier
|
||||
DAG *blockdag.BlockDAG
|
||||
TxMemPool *mempool.TxPool
|
||||
DAGParams *dagconfig.Params
|
||||
MaxPeers int
|
||||
}
|
@ -1,13 +0,0 @@
|
||||
// Copyright (c) 2017 The btcsuite developers
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package netsync
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/logger"
|
||||
"github.com/kaspanet/kaspad/util/panics"
|
||||
)
|
||||
|
||||
var log, _ = logger.Get(logger.SubsystemTags.SYNC)
|
||||
var spawn = panics.GoroutineWrapperFunc(log)
|
1239
netsync/manager.go
1239
netsync/manager.go
File diff suppressed because it is too large
Load Diff
@ -1,49 +0,0 @@
|
||||
peer
|
||||
====
|
||||
|
||||
[](https://choosealicense.com/licenses/isc/)
|
||||
[](http://godoc.org/github.com/kaspanet/kaspad/peer)
|
||||
|
||||
Package peer provides a common base for creating and managing kaspa network
|
||||
peers.
|
||||
|
||||
## Overview
|
||||
|
||||
This package builds upon the wire package, which provides the fundamental
|
||||
primitives necessary to speak the kaspa wire protocol, in order to simplify
|
||||
the process of creating fully functional peers.
|
||||
|
||||
A quick overview of the major features peer provides are as follows:
|
||||
|
||||
- Provides a basic concurrent safe kaspa peer for handling kaspa
|
||||
communications via the peer-to-peer protocol
|
||||
- Full duplex reading and writing of kaspa protocol messages
|
||||
- Automatic handling of the initial handshake process including protocol
|
||||
version negotiation
|
||||
- Asynchronous message queueing of outbound messages with optional channel for
|
||||
notification when the message is actually sent
|
||||
- Flexible peer configuration
|
||||
- Caller is responsible for creating outgoing connections and listening for
|
||||
incoming connections so they have flexibility to establish connections as
|
||||
they see fit (proxies, etc)
|
||||
- User agent name and version
|
||||
- Maximum supported protocol version
|
||||
- Ability to register callbacks for handling kaspa protocol messages
|
||||
- Inventory message batching and send trickling with known inventory detection
|
||||
and avoidance
|
||||
- Automatic periodic keep-alive pinging and pong responses
|
||||
- Random nonce generation and self connection detection
|
||||
- Proper handling of bloom filter related commands when the caller does not
|
||||
specify the related flag to signal support
|
||||
- Disconnects the peer when the protocol version is high enough
|
||||
- Does not invoke the related callbacks for older protocol versions
|
||||
- Snapshottable peer statistics such as the total number of bytes read and
|
||||
written, the remote address, user agent, and negotiated protocol version
|
||||
- Helper functions pushing addresses, getblockinvs, getheaders, and reject
|
||||
messages
|
||||
- These could all be sent manually via the standard message output function,
|
||||
but the helpers provide additional nice functionality such as duplicate
|
||||
filtering and address randomization
|
||||
- Ability to wait for shutdown/disconnect
|
||||
- Comprehensive test coverage
|
||||
|
@ -1,42 +0,0 @@
|
||||
package peer
|
||||
|
||||
// Ban scores for misbehaving nodes
|
||||
const (
|
||||
BanScoreUnrequestedBlock = 100
|
||||
BanScoreInvalidBlock = 100
|
||||
BanScoreInvalidInvBlock = 100
|
||||
BanScoreOrphanInvAsPartOfNetsync = 100
|
||||
BanScoreMalformedBlueScoreInOrphan = 100
|
||||
|
||||
BanScoreRequestNonExistingBlock = 10
|
||||
|
||||
BanScoreUnrequestedSelectedTip = 20
|
||||
BanScoreUnrequestedTx = 20
|
||||
BanScoreInvalidTx = 100
|
||||
|
||||
BanScoreMalformedMessage = 10
|
||||
|
||||
BanScoreNonVersionFirstMessage = 1
|
||||
BanScoreDuplicateVersion = 1
|
||||
BanScoreDuplicateVerack = 1
|
||||
|
||||
BanScoreSentTooManyAddresses = 20
|
||||
BanScoreMsgAddressesWithInvalidSubnetwork = 10
|
||||
|
||||
BanScoreInvalidFeeFilter = 100
|
||||
BanScoreNoFilterLoaded = 5
|
||||
|
||||
BanScoreInvalidMsgGetBlockInvs = 10
|
||||
|
||||
BanScoreInvalidMsgGetBlockLocator = 100
|
||||
|
||||
BanScoreEmptyBlockLocator = 100
|
||||
|
||||
BanScoreSentTxToBlocksOnly = 20
|
||||
|
||||
BanScoreNodeBloomFlagViolation = 100
|
||||
|
||||
BanScoreStallTimeout = 1
|
||||
|
||||
BanScoreUnrequestedMessage = 100
|
||||
)
|
133
peer/doc.go
133
peer/doc.go
@ -1,133 +0,0 @@
|
||||
/*
|
||||
Package peer provides a common base for creating and managing kaspa network
|
||||
peers.
|
||||
|
||||
Overview
|
||||
|
||||
This package builds upon the wire package, which provides the fundamental
|
||||
primitives necessary to speak the kaspa wire protocol, in order to simplify
|
||||
the process of creating fully functional peers. In essence, it provides a
|
||||
common base for creating concurrent safe fully validating nodes, Simplified
|
||||
Payment Verification (SPV) nodes, proxies, etc.
|
||||
|
||||
A quick overview of the major features peer provides are as follows:
|
||||
|
||||
- Provides a basic concurrent safe kaspa peer for handling kaspa
|
||||
communications via the peer-to-peer protocol
|
||||
- Full duplex reading and writing of kaspa protocol messages
|
||||
- Automatic handling of the initial handshake process including protocol
|
||||
version negotiation
|
||||
- Asynchronous message queuing of outbound messages with optional channel for
|
||||
notification when the message is actually sent
|
||||
- Flexible peer configuration
|
||||
- Caller is responsible for creating outgoing connections and listening for
|
||||
incoming connections so they have flexibility to establish connections as
|
||||
they see fit (proxies, etc)
|
||||
- User agent name and version
|
||||
- Kaspa network
|
||||
- Service support signalling (full nodes, bloom filters, etc)
|
||||
- Maximum supported protocol version
|
||||
- Ability to register callbacks for handling kaspa protocol messages
|
||||
- Inventory message batching and send trickling with known inventory detection
|
||||
and avoidance
|
||||
- Automatic periodic keep-alive pinging and pong responses
|
||||
- Random nonce generation and self connection detection
|
||||
- Proper handling of bloom filter related commands when the caller does not
|
||||
specify the related flag to signal support
|
||||
- Disconnects the peer when the protocol version is high enough
|
||||
- Does not invoke the related callbacks for older protocol versions
|
||||
- Snapshottable peer statistics such as the total number of bytes read and
|
||||
written, the remote address, user agent, and negotiated protocol version
|
||||
- Helper functions pushing addresses, getblockinvs, getheaders, and reject
|
||||
messages
|
||||
- These could all be sent manually via the standard message output function,
|
||||
but the helpers provide additional nice functionality such as duplicate
|
||||
filtering and address randomization
|
||||
- Ability to wait for shutdown/disconnect
|
||||
- Comprehensive test coverage
|
||||
|
||||
Peer Configuration
|
||||
|
||||
All peer configuration is handled with the Config struct. This allows the
|
||||
caller to specify things such as the user agent name and version, the kaspa
|
||||
network to use, which services it supports, and callbacks to invoke when kaspa
|
||||
messages are received. See the documentation for each field of the Config
|
||||
struct for more details.
|
||||
|
||||
Inbound and Outbound Peers
|
||||
|
||||
A peer can either be inbound or outbound. The caller is responsible for
|
||||
establishing the connection to remote peers and listening for incoming peers.
|
||||
This provides high flexibility for things such as connecting via proxies, acting
|
||||
as a proxy, creating bridge peers, choosing whether to listen for inbound peers,
|
||||
etc.
|
||||
|
||||
NewOutboundPeer and NewInboundPeer functions must be followed by calling Connect
|
||||
with a net.Conn instance to the peer. This will start all async I/O goroutines
|
||||
and initiate the protocol negotiation process. Once finished with the peer call
|
||||
Disconnect to disconnect from the peer and clean up all resources.
|
||||
WaitForDisconnect can be used to block until peer disconnection and resource
|
||||
cleanup has completed.
|
||||
|
||||
Callbacks
|
||||
|
||||
In order to do anything useful with a peer, it is necessary to react to kaspa
|
||||
messages. This is accomplished by creating an instance of the MessageListeners
|
||||
struct with the callbacks to be invoke specified and setting the Listeners field
|
||||
of the Config struct specified when creating a peer to it.
|
||||
|
||||
For convenience, a callback hook for all of the currently supported kaspa
|
||||
messages is exposed which receives the peer instance and the concrete message
|
||||
type. In addition, a hook for OnRead is provided so even custom messages types
|
||||
for which this package does not directly provide a hook, as long as they
|
||||
implement the wire.Message interface, can be used. Finally, the OnWrite hook
|
||||
is provided, which in conjunction with OnRead, can be used to track server-wide
|
||||
byte counts.
|
||||
|
||||
It is often useful to use closures which encapsulate state when specifying the
|
||||
callback handlers. This provides a clean method for accessing that state when
|
||||
callbacks are invoked.
|
||||
|
||||
Queuing Messages and Inventory
|
||||
|
||||
The QueueMessage function provides the fundamental means to send messages to the
|
||||
remote peer. As the name implies, this employs a non-blocking queue. A done
|
||||
channel which will be notified when the message is actually sent can optionally
|
||||
be specified. There are certain message types which are better sent using other
|
||||
functions which provide additional functionality.
|
||||
|
||||
Of special interest are inventory messages. Rather than manually sending MsgInv
|
||||
messages via Queuemessage, the inventory vectors should be queued using the
|
||||
QueueInventory function. It employs batching and trickling along with
|
||||
intelligent known remote peer inventory detection and avoidance through the use
|
||||
of a most-recently used algorithm.
|
||||
|
||||
Message Sending Helper Functions
|
||||
|
||||
In addition to the bare QueueMessage function previously described, the
|
||||
PushAddrMsg, PushGetBlockInvsMsg, PushGetHeadersMsg, and PushRejectMsg functions
|
||||
are provided as a convenience. While it is of course possible to create and
|
||||
send these message manually via QueueMessage, these helper functions provided
|
||||
additional useful functionality that is typically desired.
|
||||
|
||||
For example, the PushAddrMsg function automatically limits the addresses to the
|
||||
maximum number allowed by the message and randomizes the chosen addresses when
|
||||
there are too many. This allows the caller to simply provide a slice of known
|
||||
addresses, such as that returned by the addrmgr package, without having to worry
|
||||
about the details.
|
||||
|
||||
Next, the PushGetBlockInvsMsg and PushGetHeadersMsg functions will construct proper
|
||||
messages using a block locator and ignore back to back duplicate requests.
|
||||
|
||||
Finally, the PushRejectMsg function can be used to easily create and send an
|
||||
appropriate reject message based on the provided parameters as well as
|
||||
optionally provides a flag to cause it to block until the message is actually
|
||||
sent.
|
||||
|
||||
Peer Statistics
|
||||
|
||||
A snapshot of the current peer statistics can be obtained with the StatsSnapshot
|
||||
function. This includes statistics such as the total number of bytes read and
|
||||
written, the remote address, user agent, and negotiated protocol version.
|
||||
*/
|
||||
package peer
|
14
peer/log.go
14
peer/log.go
@ -1,14 +0,0 @@
|
||||
// Copyright (c) 2015-2016 The btcsuite developers
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package peer
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/logger"
|
||||
"github.com/kaspanet/kaspad/util/panics"
|
||||
)
|
||||
|
||||
var log, _ = logger.Get(logger.SubsystemTags.PEER)
|
||||
var spawn = panics.GoroutineWrapperFunc(log)
|
||||
var spawnAfter = panics.AfterFuncWrapperFunc(log)
|
@ -1,173 +0,0 @@
|
||||
package peer
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/kaspanet/kaspad/logs"
|
||||
"github.com/kaspanet/kaspad/txscript"
|
||||
"github.com/kaspanet/kaspad/util/mstime"
|
||||
"github.com/kaspanet/kaspad/wire"
|
||||
"strings"
|
||||
)
|
||||
|
||||
const (
|
||||
// maxRejectReasonLen is the maximum length of a sanitized reject reason
|
||||
// that will be logged.
|
||||
maxRejectReasonLen = 250
|
||||
)
|
||||
|
||||
// formatLockTime returns a transaction lock time as a human-readable string.
|
||||
func formatLockTime(lockTime uint64) string {
|
||||
// The lock time field of a transaction is either a block blue score at
|
||||
// which the transaction is finalized or a timestamp depending on if the
|
||||
// value is before the lockTimeThreshold. When it is under the
|
||||
// threshold it is a block blue score.
|
||||
if lockTime < txscript.LockTimeThreshold {
|
||||
return fmt.Sprintf("blue score %d", lockTime)
|
||||
}
|
||||
|
||||
return mstime.UnixMilliseconds(int64(lockTime)).String()
|
||||
}
|
||||
|
||||
// invSummary returns an inventory message as a human-readable string.
|
||||
func invSummary(invList []*wire.InvVect) string {
|
||||
// No inventory.
|
||||
invLen := len(invList)
|
||||
if invLen == 0 {
|
||||
return "empty"
|
||||
}
|
||||
|
||||
// One inventory item.
|
||||
if invLen == 1 {
|
||||
iv := invList[0]
|
||||
switch iv.Type {
|
||||
case wire.InvTypeError:
|
||||
return fmt.Sprintf("error %s", iv.Hash)
|
||||
case wire.InvTypeBlock:
|
||||
return fmt.Sprintf("block %s", iv.Hash)
|
||||
case wire.InvTypeSyncBlock:
|
||||
return fmt.Sprintf("sync block %s", iv.Hash)
|
||||
case wire.InvTypeMissingAncestor:
|
||||
return fmt.Sprintf("missing ancestor %s", iv.Hash)
|
||||
case wire.InvTypeTx:
|
||||
return fmt.Sprintf("tx %s", iv.Hash)
|
||||
}
|
||||
|
||||
return fmt.Sprintf("unknown (%d) %s", uint32(iv.Type), iv.Hash)
|
||||
}
|
||||
|
||||
// More than one inv item.
|
||||
return fmt.Sprintf("size %d", invLen)
|
||||
}
|
||||
|
||||
// sanitizeString strips any characters which are even remotely dangerous, such
|
||||
// as html control characters, from the passed string. It also limits it to
|
||||
// the passed maximum size, which can be 0 for unlimited. When the string is
|
||||
// limited, it will also add "..." to the string to indicate it was truncated.
|
||||
func sanitizeString(str string, maxLength uint) string {
|
||||
const safeChars = "abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXY" +
|
||||
"Z01234567890 .,;_/:?@"
|
||||
|
||||
// Strip any characters not in the safeChars string removed.
|
||||
str = strings.Map(func(r rune) rune {
|
||||
if strings.ContainsRune(safeChars, r) {
|
||||
return r
|
||||
}
|
||||
return -1
|
||||
}, str)
|
||||
|
||||
// Limit the string to the max allowed length.
|
||||
if maxLength > 0 && uint(len(str)) > maxLength {
|
||||
str = str[:maxLength]
|
||||
str = str + "..."
|
||||
}
|
||||
return str
|
||||
}
|
||||
|
||||
// messageSummary returns a human-readable string which summarizes a message.
|
||||
// Not all messages have or need a summary. This is used for debug logging.
|
||||
func messageSummary(msg wire.Message) string {
|
||||
switch msg := msg.(type) {
|
||||
case *wire.MsgVersion:
|
||||
return fmt.Sprintf("agent %s, pver %d, selected tip %s",
|
||||
msg.UserAgent, msg.ProtocolVersion, msg.SelectedTipHash)
|
||||
|
||||
case *wire.MsgVerAck:
|
||||
// No summary.
|
||||
|
||||
case *wire.MsgGetAddresses:
|
||||
if msg.IncludeAllSubnetworks {
|
||||
return "all subnetworks and full nodes"
|
||||
}
|
||||
if msg.SubnetworkID == nil {
|
||||
return "full nodes"
|
||||
}
|
||||
return fmt.Sprintf("subnetwork ID %v", msg.SubnetworkID)
|
||||
|
||||
case *wire.MsgAddresses:
|
||||
return fmt.Sprintf("%d addr", len(msg.AddrList))
|
||||
|
||||
case *wire.MsgPing:
|
||||
// No summary - perhaps add nonce.
|
||||
|
||||
case *wire.MsgPong:
|
||||
// No summary - perhaps add nonce.
|
||||
|
||||
case *wire.MsgTx:
|
||||
return fmt.Sprintf("hash %s, %d inputs, %d outputs, lock %s",
|
||||
msg.TxID(), len(msg.TxIn), len(msg.TxOut),
|
||||
formatLockTime(msg.LockTime))
|
||||
|
||||
case *wire.MsgBlock:
|
||||
header := &msg.Header
|
||||
return fmt.Sprintf("hash %s, ver %d, %d tx, %s", msg.BlockHash(),
|
||||
header.Version, len(msg.Transactions), header.Timestamp)
|
||||
|
||||
case *wire.MsgInv:
|
||||
return invSummary(msg.InvList)
|
||||
|
||||
case *wire.MsgNotFound:
|
||||
return invSummary(msg.InvList)
|
||||
|
||||
case *wire.MsgGetData:
|
||||
return invSummary(msg.InvList)
|
||||
|
||||
case *wire.MsgGetBlocks:
|
||||
return fmt.Sprintf("low hash %s, high hash %s", msg.LowHash,
|
||||
msg.HighHash)
|
||||
|
||||
case *wire.MsgGetBlockLocator:
|
||||
return fmt.Sprintf("high hash %s, low hash %s", msg.HighHash,
|
||||
msg.LowHash)
|
||||
|
||||
case *wire.MsgBlockLocator:
|
||||
if len(msg.BlockLocatorHashes) > 0 {
|
||||
return fmt.Sprintf("locator first hash: %s, last hash: %s", msg.BlockLocatorHashes[0], msg.BlockLocatorHashes[len(msg.BlockLocatorHashes)-1])
|
||||
}
|
||||
return fmt.Sprintf("no locator")
|
||||
|
||||
case *wire.MsgReject:
|
||||
// Ensure the variable length strings don't contain any
|
||||
// characters which are even remotely dangerous such as HTML
|
||||
// control characters, etc. Also limit them to sane length for
|
||||
// logging.
|
||||
rejReason := sanitizeString(msg.Reason, maxRejectReasonLen)
|
||||
summary := fmt.Sprintf("cmd %s, code %s, reason %s", msg.Cmd,
|
||||
msg.Code, rejReason)
|
||||
if msg.Cmd == wire.CmdBlock || msg.Cmd == wire.CmdTx {
|
||||
summary += fmt.Sprintf(", hash %s", msg.Hash)
|
||||
}
|
||||
return summary
|
||||
}
|
||||
|
||||
// No summary for other messages.
|
||||
return ""
|
||||
}
|
||||
|
||||
func messageLogLevel(msg wire.Message) logs.Level {
|
||||
switch msg.(type) {
|
||||
case *wire.MsgReject:
|
||||
return logs.LevelWarn
|
||||
default:
|
||||
return logs.LevelDebug
|
||||
}
|
||||
}
|
@ -1,127 +0,0 @@
|
||||
// Copyright (c) 2013-2015 The btcsuite developers
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package peer
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"container/list"
|
||||
"fmt"
|
||||
"sync"
|
||||
|
||||
"github.com/kaspanet/kaspad/wire"
|
||||
)
|
||||
|
||||
// mruInventoryMap provides a concurrency safe map that is limited to a maximum
|
||||
// number of items with eviction for the oldest entry when the limit is
|
||||
// exceeded.
|
||||
type mruInventoryMap struct {
|
||||
invMtx sync.Mutex
|
||||
invMap map[wire.InvVect]*list.Element // nearly O(1) lookups
|
||||
invList *list.List // O(1) insert, update, delete
|
||||
limit uint
|
||||
}
|
||||
|
||||
// String returns the map as a human-readable string.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (m *mruInventoryMap) String() string {
|
||||
m.invMtx.Lock()
|
||||
defer m.invMtx.Unlock()
|
||||
|
||||
lastEntryNum := len(m.invMap) - 1
|
||||
curEntry := 0
|
||||
buf := bytes.NewBufferString("[")
|
||||
for iv := range m.invMap {
|
||||
buf.WriteString(iv.String())
|
||||
if curEntry < lastEntryNum {
|
||||
buf.WriteString(", ")
|
||||
}
|
||||
curEntry++
|
||||
}
|
||||
buf.WriteString("]")
|
||||
|
||||
return fmt.Sprintf("<%d>%s", m.limit, buf.String())
|
||||
}
|
||||
|
||||
// Exists returns whether or not the passed inventory item is in the map.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (m *mruInventoryMap) Exists(iv *wire.InvVect) bool {
|
||||
m.invMtx.Lock()
|
||||
defer m.invMtx.Unlock()
|
||||
_, exists := m.invMap[*iv]
|
||||
|
||||
return exists
|
||||
}
|
||||
|
||||
// Add adds the passed inventory to the map and handles eviction of the oldest
|
||||
// item if adding the new item would exceed the max limit. Adding an existing
|
||||
// item makes it the most recently used item.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (m *mruInventoryMap) Add(iv *wire.InvVect) {
|
||||
m.invMtx.Lock()
|
||||
defer m.invMtx.Unlock()
|
||||
|
||||
// When the limit is zero, nothing can be added to the map, so just
|
||||
// return.
|
||||
if m.limit == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
// When the entry already exists move it to the front of the list
|
||||
// thereby marking it most recently used.
|
||||
if node, exists := m.invMap[*iv]; exists {
|
||||
m.invList.MoveToFront(node)
|
||||
return
|
||||
}
|
||||
|
||||
// Evict the least recently used entry (back of the list) if the the new
|
||||
// entry would exceed the size limit for the map. Also reuse the list
|
||||
// node so a new one doesn't have to be allocated.
|
||||
if uint(len(m.invMap))+1 > m.limit {
|
||||
node := m.invList.Back()
|
||||
lru := node.Value.(*wire.InvVect)
|
||||
|
||||
// Evict least recently used item.
|
||||
delete(m.invMap, *lru)
|
||||
|
||||
// Reuse the list node of the item that was just evicted for the
|
||||
// new item.
|
||||
node.Value = iv
|
||||
m.invList.MoveToFront(node)
|
||||
m.invMap[*iv] = node
|
||||
return
|
||||
}
|
||||
|
||||
// The limit hasn't been reached yet, so just add the new item.
|
||||
node := m.invList.PushFront(iv)
|
||||
m.invMap[*iv] = node
|
||||
}
|
||||
|
||||
// Delete deletes the passed inventory item from the map (if it exists).
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (m *mruInventoryMap) Delete(iv *wire.InvVect) {
|
||||
m.invMtx.Lock()
|
||||
defer m.invMtx.Unlock()
|
||||
if node, exists := m.invMap[*iv]; exists {
|
||||
m.invList.Remove(node)
|
||||
delete(m.invMap, *iv)
|
||||
}
|
||||
}
|
||||
|
||||
// newMruInventoryMap returns a new inventory map that is limited to the number
|
||||
// of entries specified by limit. When the number of entries exceeds the limit,
|
||||
// the oldest (least recently used) entry will be removed to make room for the
|
||||
// new entry.
|
||||
func newMruInventoryMap(limit uint) *mruInventoryMap {
|
||||
m := mruInventoryMap{
|
||||
invMap: make(map[wire.InvVect]*list.Element),
|
||||
invList: list.New(),
|
||||
limit: limit,
|
||||
}
|
||||
return &m
|
||||
}
|
@ -1,125 +0,0 @@
|
||||
// Copyright (c) 2015 The btcsuite developers
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package peer
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"container/list"
|
||||
"fmt"
|
||||
"sync"
|
||||
)
|
||||
|
||||
// mruNonceMap provides a concurrency safe map that is limited to a maximum
|
||||
// number of items with eviction for the oldest entry when the limit is
|
||||
// exceeded.
|
||||
type mruNonceMap struct {
|
||||
mtx sync.Mutex
|
||||
nonceMap map[uint64]*list.Element // nearly O(1) lookups
|
||||
nonceList *list.List // O(1) insert, update, delete
|
||||
limit uint
|
||||
}
|
||||
|
||||
// String returns the map as a human-readable string.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (m *mruNonceMap) String() string {
|
||||
m.mtx.Lock()
|
||||
defer m.mtx.Unlock()
|
||||
|
||||
lastEntryNum := len(m.nonceMap) - 1
|
||||
curEntry := 0
|
||||
buf := bytes.NewBufferString("[")
|
||||
for nonce := range m.nonceMap {
|
||||
buf.WriteString(fmt.Sprintf("%d", nonce))
|
||||
if curEntry < lastEntryNum {
|
||||
buf.WriteString(", ")
|
||||
}
|
||||
curEntry++
|
||||
}
|
||||
buf.WriteString("]")
|
||||
|
||||
return fmt.Sprintf("<%d>%s", m.limit, buf.String())
|
||||
}
|
||||
|
||||
// Exists returns whether or not the passed nonce is in the map.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (m *mruNonceMap) Exists(nonce uint64) bool {
|
||||
m.mtx.Lock()
|
||||
defer m.mtx.Unlock()
|
||||
_, exists := m.nonceMap[nonce]
|
||||
|
||||
return exists
|
||||
}
|
||||
|
||||
// Add adds the passed nonce to the map and handles eviction of the oldest item
|
||||
// if adding the new item would exceed the max limit. Adding an existing item
|
||||
// makes it the most recently used item.
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (m *mruNonceMap) Add(nonce uint64) {
|
||||
m.mtx.Lock()
|
||||
defer m.mtx.Unlock()
|
||||
|
||||
// When the limit is zero, nothing can be added to the map, so just
|
||||
// return.
|
||||
if m.limit == 0 {
|
||||
return
|
||||
}
|
||||
|
||||
// When the entry already exists move it to the front of the list
|
||||
// thereby marking it most recently used.
|
||||
if node, exists := m.nonceMap[nonce]; exists {
|
||||
m.nonceList.MoveToFront(node)
|
||||
return
|
||||
}
|
||||
|
||||
// Evict the least recently used entry (back of the list) if the the new
|
||||
// entry would exceed the size limit for the map. Also reuse the list
|
||||
// node so a new one doesn't have to be allocated.
|
||||
if uint(len(m.nonceMap))+1 > m.limit {
|
||||
node := m.nonceList.Back()
|
||||
lru := node.Value.(uint64)
|
||||
|
||||
// Evict least recently used item.
|
||||
delete(m.nonceMap, lru)
|
||||
|
||||
// Reuse the list node of the item that was just evicted for the
|
||||
// new item.
|
||||
node.Value = nonce
|
||||
m.nonceList.MoveToFront(node)
|
||||
m.nonceMap[nonce] = node
|
||||
return
|
||||
}
|
||||
|
||||
// The limit hasn't been reached yet, so just add the new item.
|
||||
node := m.nonceList.PushFront(nonce)
|
||||
m.nonceMap[nonce] = node
|
||||
}
|
||||
|
||||
// Delete deletes the passed nonce from the map (if it exists).
|
||||
//
|
||||
// This function is safe for concurrent access.
|
||||
func (m *mruNonceMap) Delete(nonce uint64) {
|
||||
m.mtx.Lock()
|
||||
defer m.mtx.Unlock()
|
||||
if node, exists := m.nonceMap[nonce]; exists {
|
||||
m.nonceList.Remove(node)
|
||||
delete(m.nonceMap, nonce)
|
||||
}
|
||||
}
|
||||
|
||||
// newMruNonceMap returns a new nonce map that is limited to the number of
|
||||
// entries specified by limit. When the number of entries exceeds the limit,
|
||||
// the oldest (least recently used) entry will be removed to make room for the
|
||||
// new entry.
|
||||
func newMruNonceMap(limit uint) *mruNonceMap {
|
||||
m := mruNonceMap{
|
||||
nonceMap: make(map[uint64]*list.Element),
|
||||
nonceList: list.New(),
|
||||
limit: limit,
|
||||
}
|
||||
return &m
|
||||
}
|
1979
peer/peer.go
1979
peer/peer.go
File diff suppressed because it is too large
Load Diff
@ -46,3 +46,9 @@ func (f *FlowContext) OnNewBlock(block *util.Block) error {
|
||||
func (f *FlowContext) SharedRequestedBlocks() *blockrelay.SharedRequestedBlocks {
|
||||
return f.sharedRequestedBlocks
|
||||
}
|
||||
|
||||
// AddBlock adds the given block to the DAG and propagates it.
|
||||
func (f *FlowContext) AddBlock(block *util.Block) error {
|
||||
// TODO(libp2p): unimplemented
|
||||
panic("unimplemented")
|
||||
}
|
||||
|
@ -35,6 +35,7 @@ type FlowContext struct {
|
||||
|
||||
isInIBD uint32
|
||||
startIBDMutex sync.Mutex
|
||||
ibdPeer *peerpkg.Peer
|
||||
|
||||
peers map[*id.ID]*peerpkg.Peer
|
||||
peersMutex sync.RWMutex
|
||||
|
@ -24,6 +24,7 @@ func (f *FlowContext) StartIBDIfRequired() {
|
||||
}
|
||||
|
||||
atomic.StoreUint32(&f.isInIBD, 1)
|
||||
f.ibdPeer = peer
|
||||
peer.StartIBD()
|
||||
}
|
||||
|
||||
@ -64,7 +65,18 @@ func (f *FlowContext) requestSelectedTips() {
|
||||
|
||||
// FinishIBD finishes the current IBD flow and starts a new one if required.
|
||||
func (f *FlowContext) FinishIBD() {
|
||||
f.ibdPeer = nil
|
||||
|
||||
atomic.StoreUint32(&f.isInIBD, 0)
|
||||
|
||||
f.StartIBDIfRequired()
|
||||
}
|
||||
|
||||
// IBDPeer returns the currently active IBD peer.
|
||||
// Returns nil if we aren't currently in IBD
|
||||
func (f *FlowContext) IBDPeer() *peerpkg.Peer {
|
||||
if !f.IsInIBD() {
|
||||
return nil
|
||||
}
|
||||
return f.ibdPeer
|
||||
}
|
||||
|
@ -44,3 +44,17 @@ func (f *FlowContext) readyPeerIDs() []*id.ID {
|
||||
func (f *FlowContext) Broadcast(message wire.Message) error {
|
||||
return f.netAdapter.Broadcast(f.readyPeerIDs(), message)
|
||||
}
|
||||
|
||||
// Peers returns the currently active peers
|
||||
func (f *FlowContext) Peers() []*peerpkg.Peer {
|
||||
f.peersMutex.RLock()
|
||||
defer f.peersMutex.RUnlock()
|
||||
|
||||
peers := make([]*peerpkg.Peer, len(f.peers))
|
||||
i := 0
|
||||
for _, peer := range f.peers {
|
||||
peers[i] = peer
|
||||
i++
|
||||
}
|
||||
return peers
|
||||
}
|
||||
|
@ -28,7 +28,8 @@ type HandleHandshakeContext interface {
|
||||
|
||||
// HandleHandshake sets up the handshake protocol - It sends a version message and waits for an incoming
|
||||
// version message, as well as a verack for the sent version
|
||||
func HandleHandshake(context HandleHandshakeContext, router *routerpkg.Router) (peer *peerpkg.Peer, closed bool, err error) {
|
||||
func HandleHandshake(context HandleHandshakeContext, router *routerpkg.Router,
|
||||
netConnection *netadapter.NetConnection) (peer *peerpkg.Peer, closed bool, err error) {
|
||||
|
||||
receiveVersionRoute, err := router.AddIncomingRoute([]wire.MessageCommand{wire.CmdVersion})
|
||||
if err != nil {
|
||||
@ -49,7 +50,7 @@ func HandleHandshake(context HandleHandshakeContext, router *routerpkg.Router) (
|
||||
errChanUsed := uint32(0)
|
||||
errChan := make(chan error)
|
||||
|
||||
peer = peerpkg.New()
|
||||
peer = peerpkg.New(netConnection)
|
||||
|
||||
var peerAddress *wire.NetAddress
|
||||
spawn("HandleHandshake-ReceiveVersion", func() {
|
||||
|
@ -7,6 +7,8 @@ import (
|
||||
"github.com/kaspanet/kaspad/mempool"
|
||||
"github.com/kaspanet/kaspad/netadapter"
|
||||
"github.com/kaspanet/kaspad/protocol/flowcontext"
|
||||
peerpkg "github.com/kaspanet/kaspad/protocol/peer"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
)
|
||||
|
||||
// Manager manages the p2p protocol
|
||||
@ -39,3 +41,24 @@ func (m *Manager) Start() error {
|
||||
func (m *Manager) Stop() error {
|
||||
return m.context.NetAdapter().Stop()
|
||||
}
|
||||
|
||||
// Peers returns the currently active peers
|
||||
func (m *Manager) Peers() []*peerpkg.Peer {
|
||||
return m.context.Peers()
|
||||
}
|
||||
|
||||
// IBDPeer returns the currently active IBD peer.
|
||||
// Returns nil if we aren't currently in IBD
|
||||
func (m *Manager) IBDPeer() *peerpkg.Peer {
|
||||
return m.context.IBDPeer()
|
||||
}
|
||||
|
||||
// AddTransaction adds transaction to the mempool and propagates it.
|
||||
func (m *Manager) AddTransaction(tx *util.Tx) error {
|
||||
return m.context.AddTransaction(tx)
|
||||
}
|
||||
|
||||
// AddBlock adds the given block to the DAG and propagates it.
|
||||
func (m *Manager) AddBlock(block *util.Block) error {
|
||||
return m.context.AddBlock(block)
|
||||
}
|
||||
|
@ -1,6 +1,7 @@
|
||||
package peer
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/netadapter"
|
||||
"sync"
|
||||
"sync/atomic"
|
||||
"time"
|
||||
@ -15,6 +16,8 @@ import (
|
||||
|
||||
// Peer holds data about a peer.
|
||||
type Peer struct {
|
||||
connection *netadapter.NetConnection
|
||||
|
||||
selectedTipHashMtx sync.RWMutex
|
||||
selectedTipHash *daghash.Hash
|
||||
|
||||
@ -39,8 +42,9 @@ type Peer struct {
|
||||
}
|
||||
|
||||
// New returns a new Peer
|
||||
func New() *Peer {
|
||||
func New(connection *netadapter.NetConnection) *Peer {
|
||||
return &Peer{
|
||||
connection: connection,
|
||||
selectedTipRequestChan: make(chan struct{}),
|
||||
ibdStartChan: make(chan struct{}),
|
||||
}
|
||||
@ -113,8 +117,7 @@ func (p *Peer) SetPingIdle() {
|
||||
}
|
||||
|
||||
func (p *Peer) String() string {
|
||||
//TODO(libp2p)
|
||||
panic("unimplemented")
|
||||
return p.connection.String()
|
||||
}
|
||||
|
||||
// RequestSelectedTipIfRequired notifies the peer that requesting
|
||||
@ -156,3 +159,17 @@ func (p *Peer) StartIBD() {
|
||||
func (p *Peer) WaitForIBDStart() {
|
||||
<-p.ibdStartChan
|
||||
}
|
||||
|
||||
// Address returns the address associated with this connection
|
||||
func (p *Peer) Address() string {
|
||||
return p.connection.Address()
|
||||
}
|
||||
|
||||
// LastPingDuration returns the duration of the last ping to
|
||||
// this peer
|
||||
func (p *Peer) LastPingDuration() time.Duration {
|
||||
p.pingLock.Lock()
|
||||
defer p.pingLock.Unlock()
|
||||
|
||||
return p.lastPingDuration
|
||||
}
|
||||
|
@ -2,6 +2,7 @@ package protocol
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/kaspanet/kaspad/netadapter"
|
||||
"sync/atomic"
|
||||
|
||||
"github.com/kaspanet/kaspad/protocol/flows/handshake"
|
||||
@ -19,11 +20,10 @@ import (
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
func (m *Manager) routerInitializer() (*routerpkg.Router, error) {
|
||||
|
||||
func (m *Manager) routerInitializer(netConnection *netadapter.NetConnection) (*routerpkg.Router, error) {
|
||||
router := routerpkg.NewRouter()
|
||||
spawn("newRouterInitializer-startFlows", func() {
|
||||
err := m.startFlows(router)
|
||||
err := m.startFlows(netConnection, router)
|
||||
if err != nil {
|
||||
if protocolErr := &(protocolerrors.ProtocolError{}); errors.As(err, &protocolErr) {
|
||||
if protocolErr.ShouldBan {
|
||||
@ -50,14 +50,13 @@ func (m *Manager) routerInitializer() (*routerpkg.Router, error) {
|
||||
}
|
||||
})
|
||||
return router, nil
|
||||
|
||||
}
|
||||
|
||||
func (m *Manager) startFlows(router *routerpkg.Router) error {
|
||||
func (m *Manager) startFlows(netConnection *netadapter.NetConnection, router *routerpkg.Router) error {
|
||||
stop := make(chan error)
|
||||
stopped := uint32(0)
|
||||
|
||||
peer, closed, err := handshake.HandleHandshake(m.context, router)
|
||||
peer, closed, err := handshake.HandleHandshake(m.context, router, netConnection)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
@ -3,7 +3,7 @@
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package rpcclient
|
||||
package client
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
@ -13,7 +13,7 @@ import (
|
||||
|
||||
"github.com/pkg/errors"
|
||||
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
"github.com/kaspanet/kaspad/wire"
|
||||
)
|
||||
@ -45,7 +45,7 @@ func (r FutureGetSelectedTipHashResult) Receive() (*daghash.Hash, error) {
|
||||
//
|
||||
// See GetSelectedTipHash for the blocking version and more details.
|
||||
func (c *Client) GetSelectedTipHashAsync() FutureGetSelectedTipHashResult {
|
||||
cmd := rpcmodel.NewGetSelectedTipHashCmd()
|
||||
cmd := model.NewGetSelectedTipHashCmd()
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -100,7 +100,7 @@ func (c *Client) GetBlockAsync(blockHash *daghash.Hash, subnetworkID *string) Fu
|
||||
hash = blockHash.String()
|
||||
}
|
||||
|
||||
cmd := rpcmodel.NewGetBlockCmd(hash, pointers.Bool(false), pointers.Bool(false), subnetworkID)
|
||||
cmd := model.NewGetBlockCmd(hash, pointers.Bool(false), pointers.Bool(false), subnetworkID)
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -118,13 +118,13 @@ type FutureGetBlocksResult chan *response
|
||||
|
||||
// Receive waits for the response promised by the future and returns the blocks
|
||||
// starting from lowHash up to the virtual ordered by blue score.
|
||||
func (r FutureGetBlocksResult) Receive() (*rpcmodel.GetBlocksResult, error) {
|
||||
func (r FutureGetBlocksResult) Receive() (*model.GetBlocksResult, error) {
|
||||
res, err := receiveFuture(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var result rpcmodel.GetBlocksResult
|
||||
var result model.GetBlocksResult
|
||||
if err := json.Unmarshal(res, &result); err != nil {
|
||||
return nil, errors.Wrap(err, string(res))
|
||||
}
|
||||
@ -137,13 +137,13 @@ func (r FutureGetBlocksResult) Receive() (*rpcmodel.GetBlocksResult, error) {
|
||||
//
|
||||
// See GetBlocks for the blocking version and more details.
|
||||
func (c *Client) GetBlocksAsync(includeRawBlockData bool, IncludeVerboseBlockData bool, lowHash *string) FutureGetBlocksResult {
|
||||
cmd := rpcmodel.NewGetBlocksCmd(includeRawBlockData, IncludeVerboseBlockData, lowHash)
|
||||
cmd := model.NewGetBlocksCmd(includeRawBlockData, IncludeVerboseBlockData, lowHash)
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
// GetBlocks returns the blocks starting from lowHash up to the virtual ordered
|
||||
// by blue score.
|
||||
func (c *Client) GetBlocks(includeRawBlockData bool, includeVerboseBlockData bool, lowHash *string) (*rpcmodel.GetBlocksResult, error) {
|
||||
func (c *Client) GetBlocks(includeRawBlockData bool, includeVerboseBlockData bool, lowHash *string) (*model.GetBlocksResult, error) {
|
||||
return c.GetBlocksAsync(includeRawBlockData, includeVerboseBlockData, lowHash).Receive()
|
||||
}
|
||||
|
||||
@ -153,14 +153,14 @@ type FutureGetBlockVerboseResult chan *response
|
||||
|
||||
// Receive waits for the response promised by the future and returns the data
|
||||
// structure from the server with information about the requested block.
|
||||
func (r FutureGetBlockVerboseResult) Receive() (*rpcmodel.GetBlockVerboseResult, error) {
|
||||
func (r FutureGetBlockVerboseResult) Receive() (*model.GetBlockVerboseResult, error) {
|
||||
res, err := receiveFuture(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Unmarshal the raw result into a BlockResult.
|
||||
var blockResult rpcmodel.GetBlockVerboseResult
|
||||
var blockResult model.GetBlockVerboseResult
|
||||
err = json.Unmarshal(res, &blockResult)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "couldn't decode getBlock response")
|
||||
@ -179,7 +179,7 @@ func (c *Client) GetBlockVerboseAsync(blockHash *daghash.Hash, subnetworkID *str
|
||||
hash = blockHash.String()
|
||||
}
|
||||
|
||||
cmd := rpcmodel.NewGetBlockCmd(hash, pointers.Bool(true), pointers.Bool(false), subnetworkID)
|
||||
cmd := model.NewGetBlockCmd(hash, pointers.Bool(true), pointers.Bool(false), subnetworkID)
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -188,7 +188,7 @@ func (c *Client) GetBlockVerboseAsync(blockHash *daghash.Hash, subnetworkID *str
|
||||
//
|
||||
// See GetBlockVerboseTx to retrieve transaction data structures as well.
|
||||
// See GetBlock to retrieve a raw block instead.
|
||||
func (c *Client) GetBlockVerbose(blockHash *daghash.Hash, subnetworkID *string) (*rpcmodel.GetBlockVerboseResult, error) {
|
||||
func (c *Client) GetBlockVerbose(blockHash *daghash.Hash, subnetworkID *string) (*model.GetBlockVerboseResult, error) {
|
||||
return c.GetBlockVerboseAsync(blockHash, subnetworkID).Receive()
|
||||
}
|
||||
|
||||
@ -203,7 +203,7 @@ func (c *Client) GetBlockVerboseTxAsync(blockHash *daghash.Hash, subnetworkID *s
|
||||
hash = blockHash.String()
|
||||
}
|
||||
|
||||
cmd := rpcmodel.NewGetBlockCmd(hash, pointers.Bool(true), pointers.Bool(true), subnetworkID)
|
||||
cmd := model.NewGetBlockCmd(hash, pointers.Bool(true), pointers.Bool(true), subnetworkID)
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -212,7 +212,7 @@ func (c *Client) GetBlockVerboseTxAsync(blockHash *daghash.Hash, subnetworkID *s
|
||||
//
|
||||
// See GetBlockVerbose if only transaction hashes are preferred.
|
||||
// See GetBlock to retrieve a raw block instead.
|
||||
func (c *Client) GetBlockVerboseTx(blockHash *daghash.Hash, subnetworkID *string) (*rpcmodel.GetBlockVerboseResult, error) {
|
||||
func (c *Client) GetBlockVerboseTx(blockHash *daghash.Hash, subnetworkID *string) (*model.GetBlockVerboseResult, error) {
|
||||
return c.GetBlockVerboseTxAsync(blockHash, subnetworkID).Receive()
|
||||
}
|
||||
|
||||
@ -243,7 +243,7 @@ func (r FutureGetBlockCountResult) Receive() (int64, error) {
|
||||
//
|
||||
// See GetBlockCount for the blocking version and more details.
|
||||
func (c *Client) GetBlockCountAsync() FutureGetBlockCountResult {
|
||||
cmd := rpcmodel.NewGetBlockCountCmd()
|
||||
cmd := model.NewGetBlockCountCmd()
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -260,13 +260,13 @@ type FutureGetChainFromBlockResult chan *response
|
||||
// parent chain starting from startHash up to the virtual. If startHash is not in
|
||||
// the selected parent chain, it goes down the DAG until it does reach a hash in
|
||||
// the selected parent chain while collecting hashes into RemovedChainBlockHashes.
|
||||
func (r FutureGetChainFromBlockResult) Receive() (*rpcmodel.GetChainFromBlockResult, error) {
|
||||
func (r FutureGetChainFromBlockResult) Receive() (*model.GetChainFromBlockResult, error) {
|
||||
res, err := receiveFuture(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var result rpcmodel.GetChainFromBlockResult
|
||||
var result model.GetChainFromBlockResult
|
||||
if err := json.Unmarshal(res, &result); err != nil {
|
||||
return nil, errors.Wrap(err, "couldn't decode getChainFromBlock response")
|
||||
}
|
||||
@ -279,7 +279,7 @@ func (r FutureGetChainFromBlockResult) Receive() (*rpcmodel.GetChainFromBlockRes
|
||||
//
|
||||
// See GetChainFromBlock for the blocking version and more details.
|
||||
func (c *Client) GetChainFromBlockAsync(includeBlocks bool, startHash *string) FutureGetChainFromBlockResult {
|
||||
cmd := rpcmodel.NewGetChainFromBlockCmd(includeBlocks, startHash)
|
||||
cmd := model.NewGetChainFromBlockCmd(includeBlocks, startHash)
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -287,7 +287,7 @@ func (c *Client) GetChainFromBlockAsync(includeBlocks bool, startHash *string) F
|
||||
// up to the virtual. If startHash is not in the selected parent chain, it goes
|
||||
// down the DAG until it does reach a hash in the selected parent chain while
|
||||
// collecting hashes into RemovedChainBlockHashes.
|
||||
func (c *Client) GetChainFromBlock(includeBlocks bool, startHash *string) (*rpcmodel.GetChainFromBlockResult, error) {
|
||||
func (c *Client) GetChainFromBlock(includeBlocks bool, startHash *string) (*model.GetChainFromBlockResult, error) {
|
||||
return c.GetChainFromBlockAsync(includeBlocks, startHash).Receive()
|
||||
}
|
||||
|
||||
@ -318,7 +318,7 @@ func (r FutureGetDifficultyResult) Receive() (float64, error) {
|
||||
//
|
||||
// See GetDifficulty for the blocking version and more details.
|
||||
func (c *Client) GetDifficultyAsync() FutureGetDifficultyResult {
|
||||
cmd := rpcmodel.NewGetDifficultyCmd()
|
||||
cmd := model.NewGetDifficultyCmd()
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -334,13 +334,13 @@ type FutureGetBlockDAGInfoResult chan *response
|
||||
|
||||
// Receive waits for the response promised by the future and returns dag info
|
||||
// result provided by the server.
|
||||
func (r FutureGetBlockDAGInfoResult) Receive() (*rpcmodel.GetBlockDAGInfoResult, error) {
|
||||
func (r FutureGetBlockDAGInfoResult) Receive() (*model.GetBlockDAGInfoResult, error) {
|
||||
res, err := receiveFuture(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var dagInfo rpcmodel.GetBlockDAGInfoResult
|
||||
var dagInfo model.GetBlockDAGInfoResult
|
||||
if err := json.Unmarshal(res, &dagInfo); err != nil {
|
||||
return nil, errors.Wrap(err, "couldn't decode getBlockDagInfo response")
|
||||
}
|
||||
@ -353,14 +353,14 @@ func (r FutureGetBlockDAGInfoResult) Receive() (*rpcmodel.GetBlockDAGInfoResult,
|
||||
//
|
||||
// See GetBlockDAGInfo for the blocking version and more details.
|
||||
func (c *Client) GetBlockDAGInfoAsync() FutureGetBlockDAGInfoResult {
|
||||
cmd := rpcmodel.NewGetBlockDAGInfoCmd()
|
||||
cmd := model.NewGetBlockDAGInfoCmd()
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
// GetBlockDAGInfo returns information related to the processing state of
|
||||
// various dag-specific details such as the current difficulty from the tip
|
||||
// of the main dag.
|
||||
func (c *Client) GetBlockDAGInfo() (*rpcmodel.GetBlockDAGInfoResult, error) {
|
||||
func (c *Client) GetBlockDAGInfo() (*model.GetBlockDAGInfoResult, error) {
|
||||
return c.GetBlockDAGInfoAsync().Receive()
|
||||
}
|
||||
|
||||
@ -430,7 +430,7 @@ func (c *Client) GetBlockHeaderAsync(blockHash *daghash.Hash) FutureGetBlockHead
|
||||
hash = blockHash.String()
|
||||
}
|
||||
|
||||
cmd := rpcmodel.NewGetBlockHeaderCmd(hash, pointers.Bool(false))
|
||||
cmd := model.NewGetBlockHeaderCmd(hash, pointers.Bool(false))
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -448,14 +448,14 @@ type FutureGetBlockHeaderVerboseResult chan *response
|
||||
|
||||
// Receive waits for the response promised by the future and returns the
|
||||
// data structure of the blockheader requested from the server given its hash.
|
||||
func (r FutureGetBlockHeaderVerboseResult) Receive() (*rpcmodel.GetBlockHeaderVerboseResult, error) {
|
||||
func (r FutureGetBlockHeaderVerboseResult) Receive() (*model.GetBlockHeaderVerboseResult, error) {
|
||||
res, err := receiveFuture(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Unmarshal result as a string.
|
||||
var bh rpcmodel.GetBlockHeaderVerboseResult
|
||||
var bh model.GetBlockHeaderVerboseResult
|
||||
err = json.Unmarshal(res, &bh)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "couldn't decode getBlockHeader response")
|
||||
@ -475,7 +475,7 @@ func (c *Client) GetBlockHeaderVerboseAsync(blockHash *daghash.Hash) FutureGetBl
|
||||
hash = blockHash.String()
|
||||
}
|
||||
|
||||
cmd := rpcmodel.NewGetBlockHeaderCmd(hash, pointers.Bool(true))
|
||||
cmd := model.NewGetBlockHeaderCmd(hash, pointers.Bool(true))
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -483,7 +483,7 @@ func (c *Client) GetBlockHeaderVerboseAsync(blockHash *daghash.Hash) FutureGetBl
|
||||
// blockheader from the server given its hash.
|
||||
//
|
||||
// See GetBlockHeader to retrieve a blockheader instead.
|
||||
func (c *Client) GetBlockHeaderVerbose(blockHash *daghash.Hash) (*rpcmodel.GetBlockHeaderVerboseResult, error) {
|
||||
func (c *Client) GetBlockHeaderVerbose(blockHash *daghash.Hash) (*model.GetBlockHeaderVerboseResult, error) {
|
||||
return c.GetBlockHeaderVerboseAsync(blockHash).Receive()
|
||||
}
|
||||
|
||||
@ -494,14 +494,14 @@ type FutureGetMempoolEntryResult chan *response
|
||||
// Receive waits for the response promised by the future and returns a data
|
||||
// structure with information about the transaction in the memory pool given
|
||||
// its hash.
|
||||
func (r FutureGetMempoolEntryResult) Receive() (*rpcmodel.GetMempoolEntryResult, error) {
|
||||
func (r FutureGetMempoolEntryResult) Receive() (*model.GetMempoolEntryResult, error) {
|
||||
res, err := receiveFuture(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Unmarshal the result as an array of strings.
|
||||
var mempoolEntryResult rpcmodel.GetMempoolEntryResult
|
||||
var mempoolEntryResult model.GetMempoolEntryResult
|
||||
err = json.Unmarshal(res, &mempoolEntryResult)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "couldn't decode getMempoolEntry response")
|
||||
@ -516,13 +516,13 @@ func (r FutureGetMempoolEntryResult) Receive() (*rpcmodel.GetMempoolEntryResult,
|
||||
//
|
||||
// See GetMempoolEntry for the blocking version and more details.
|
||||
func (c *Client) GetMempoolEntryAsync(txHash string) FutureGetMempoolEntryResult {
|
||||
cmd := rpcmodel.NewGetMempoolEntryCmd(txHash)
|
||||
cmd := model.NewGetMempoolEntryCmd(txHash)
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
// GetMempoolEntry returns a data structure with information about the
|
||||
// transaction in the memory pool given its hash.
|
||||
func (c *Client) GetMempoolEntry(txHash string) (*rpcmodel.GetMempoolEntryResult, error) {
|
||||
func (c *Client) GetMempoolEntry(txHash string) (*model.GetMempoolEntryResult, error) {
|
||||
return c.GetMempoolEntryAsync(txHash).Receive()
|
||||
}
|
||||
|
||||
@ -564,7 +564,7 @@ func (r FutureGetRawMempoolResult) Receive() ([]*daghash.Hash, error) {
|
||||
//
|
||||
// See GetRawMempool for the blocking version and more details.
|
||||
func (c *Client) GetRawMempoolAsync() FutureGetRawMempoolResult {
|
||||
cmd := rpcmodel.NewGetRawMempoolCmd(pointers.Bool(false))
|
||||
cmd := model.NewGetRawMempoolCmd(pointers.Bool(false))
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -583,7 +583,7 @@ type FutureGetRawMempoolVerboseResult chan *response
|
||||
// Receive waits for the response promised by the future and returns a map of
|
||||
// transaction hashes to an associated data structure with information about the
|
||||
// transaction for all transactions in the memory pool.
|
||||
func (r FutureGetRawMempoolVerboseResult) Receive() (map[string]rpcmodel.GetRawMempoolVerboseResult, error) {
|
||||
func (r FutureGetRawMempoolVerboseResult) Receive() (map[string]model.GetRawMempoolVerboseResult, error) {
|
||||
res, err := receiveFuture(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@ -591,7 +591,7 @@ func (r FutureGetRawMempoolVerboseResult) Receive() (map[string]rpcmodel.GetRawM
|
||||
|
||||
// Unmarshal the result as a map of strings (tx shas) to their detailed
|
||||
// results.
|
||||
var mempoolItems map[string]rpcmodel.GetRawMempoolVerboseResult
|
||||
var mempoolItems map[string]model.GetRawMempoolVerboseResult
|
||||
err = json.Unmarshal(res, &mempoolItems)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "couldn't decode getRawMempool response")
|
||||
@ -605,7 +605,7 @@ func (r FutureGetRawMempoolVerboseResult) Receive() (map[string]rpcmodel.GetRawM
|
||||
//
|
||||
// See GetRawMempoolVerbose for the blocking version and more details.
|
||||
func (c *Client) GetRawMempoolVerboseAsync() FutureGetRawMempoolVerboseResult {
|
||||
cmd := rpcmodel.NewGetRawMempoolCmd(pointers.Bool(true))
|
||||
cmd := model.NewGetRawMempoolCmd(pointers.Bool(true))
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -614,7 +614,7 @@ func (c *Client) GetRawMempoolVerboseAsync() FutureGetRawMempoolVerboseResult {
|
||||
// the memory pool.
|
||||
//
|
||||
// See GetRawMempool to retrieve only the transaction hashes instead.
|
||||
func (c *Client) GetRawMempoolVerbose() (map[string]rpcmodel.GetRawMempoolVerboseResult, error) {
|
||||
func (c *Client) GetRawMempoolVerbose() (map[string]model.GetRawMempoolVerboseResult, error) {
|
||||
return c.GetRawMempoolVerboseAsync().Receive()
|
||||
}
|
||||
|
||||
@ -624,14 +624,14 @@ type FutureGetSubnetworkResult chan *response
|
||||
|
||||
// Receive waits for the response promised by the future and returns information
|
||||
// regarding the requested subnetwork
|
||||
func (r FutureGetSubnetworkResult) Receive() (*rpcmodel.GetSubnetworkResult, error) {
|
||||
func (r FutureGetSubnetworkResult) Receive() (*model.GetSubnetworkResult, error) {
|
||||
res, err := receiveFuture(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Unmarshal result as a getSubnetwork result object.
|
||||
var getSubnetworkResult *rpcmodel.GetSubnetworkResult
|
||||
var getSubnetworkResult *model.GetSubnetworkResult
|
||||
err = json.Unmarshal(res, &getSubnetworkResult)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "couldn't decode getSubnetwork response")
|
||||
@ -646,12 +646,12 @@ func (r FutureGetSubnetworkResult) Receive() (*rpcmodel.GetSubnetworkResult, err
|
||||
//
|
||||
// See GetSubnetwork for the blocking version and more details.
|
||||
func (c *Client) GetSubnetworkAsync(subnetworkID string) FutureGetSubnetworkResult {
|
||||
cmd := rpcmodel.NewGetSubnetworkCmd(subnetworkID)
|
||||
cmd := model.NewGetSubnetworkCmd(subnetworkID)
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
// GetSubnetwork provides information about a subnetwork given its ID.
|
||||
func (c *Client) GetSubnetwork(subnetworkID string) (*rpcmodel.GetSubnetworkResult, error) {
|
||||
func (c *Client) GetSubnetwork(subnetworkID string) (*model.GetSubnetworkResult, error) {
|
||||
return c.GetSubnetworkAsync(subnetworkID).Receive()
|
||||
}
|
||||
|
||||
@ -661,7 +661,7 @@ type FutureGetTxOutResult chan *response
|
||||
|
||||
// Receive waits for the response promised by the future and returns a
|
||||
// transaction given its hash.
|
||||
func (r FutureGetTxOutResult) Receive() (*rpcmodel.GetTxOutResult, error) {
|
||||
func (r FutureGetTxOutResult) Receive() (*model.GetTxOutResult, error) {
|
||||
res, err := receiveFuture(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@ -674,7 +674,7 @@ func (r FutureGetTxOutResult) Receive() (*rpcmodel.GetTxOutResult, error) {
|
||||
}
|
||||
|
||||
// Unmarshal result as an gettxout result object.
|
||||
var txOutInfo *rpcmodel.GetTxOutResult
|
||||
var txOutInfo *model.GetTxOutResult
|
||||
err = json.Unmarshal(res, &txOutInfo)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "couldn't decode getTxOut response")
|
||||
@ -694,13 +694,13 @@ func (c *Client) GetTxOutAsync(txHash *daghash.Hash, index uint32, mempool bool)
|
||||
hash = txHash.String()
|
||||
}
|
||||
|
||||
cmd := rpcmodel.NewGetTxOutCmd(hash, index, &mempool)
|
||||
cmd := model.NewGetTxOutCmd(hash, index, &mempool)
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
// GetTxOut returns the transaction output info if it's unspent and
|
||||
// nil, otherwise.
|
||||
func (c *Client) GetTxOut(txHash *daghash.Hash, index uint32, mempool bool) (*rpcmodel.GetTxOutResult, error) {
|
||||
func (c *Client) GetTxOut(txHash *daghash.Hash, index uint32, mempool bool) (*model.GetTxOutResult, error) {
|
||||
return c.GetTxOutAsync(txHash, index, mempool).Receive()
|
||||
}
|
||||
|
||||
@ -710,13 +710,13 @@ type FutureRescanBlocksResult chan *response
|
||||
|
||||
// Receive waits for the response promised by the future and returns the
|
||||
// discovered rescanblocks data.
|
||||
func (r FutureRescanBlocksResult) Receive() ([]rpcmodel.RescannedBlock, error) {
|
||||
func (r FutureRescanBlocksResult) Receive() ([]model.RescannedBlock, error) {
|
||||
res, err := receiveFuture(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var rescanBlocksResult []rpcmodel.RescannedBlock
|
||||
var rescanBlocksResult []model.RescannedBlock
|
||||
err = json.Unmarshal(res, &rescanBlocksResult)
|
||||
if err != nil {
|
||||
return nil, errors.Wrap(err, "couldn't decode rescanBlocks response")
|
||||
@ -736,13 +736,13 @@ func (c *Client) RescanBlocksAsync(blockHashes []*daghash.Hash) FutureRescanBloc
|
||||
strBlockHashes[i] = blockHashes[i].String()
|
||||
}
|
||||
|
||||
cmd := rpcmodel.NewRescanBlocksCmd(strBlockHashes)
|
||||
cmd := model.NewRescanBlocksCmd(strBlockHashes)
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
// RescanBlocks rescans the blocks identified by blockHashes, in order, using
|
||||
// the client's loaded transaction filter. The blocks do not need to be on the
|
||||
// main dag, but they do need to be adjacent to each other.
|
||||
func (c *Client) RescanBlocks(blockHashes []*daghash.Hash) ([]rpcmodel.RescannedBlock, error) {
|
||||
func (c *Client) RescanBlocks(blockHashes []*daghash.Hash) ([]model.RescannedBlock, error) {
|
||||
return c.RescanBlocksAsync(blockHashes).Receive()
|
||||
}
|
@ -1,5 +1,5 @@
|
||||
/*
|
||||
Package rpcclient implements a websocket-enabled kaspa JSON-RPC client.
|
||||
Package client implements a websocket-enabled kaspa JSON-RPC client.
|
||||
|
||||
Overview
|
||||
|
||||
@ -116,15 +116,15 @@ the type can vary, but usually will be best handled by simply showing/logging
|
||||
it.
|
||||
|
||||
The third category of errors, that is errors returned by the server, can be
|
||||
detected by type asserting the error in a *rpcmodel.RPCError. For example, to
|
||||
detected by type asserting the error in a *model.RPCError. For example, to
|
||||
detect if a command is unimplemented by the remote RPC server:
|
||||
|
||||
netTotals, err := client.GetNetTotals()
|
||||
if err != nil {
|
||||
var jErr *rpcmodel.RPCError
|
||||
var jErr *model.RPCError
|
||||
if errors.As(err, jErr) {
|
||||
switch jErr.Code {
|
||||
case rpcmodel.ErrRPCUnimplemented:
|
||||
case model.ErrRPCUnimplemented:
|
||||
// Handle not implemented error
|
||||
|
||||
// Handle other specific errors you care about
|
||||
@ -146,4 +146,4 @@ The following full-blown client examples are in the examples directory:
|
||||
Connects to a kaspad RPC server using TLS-secured websockets, registers for
|
||||
block added notifications, and gets the current block count
|
||||
*/
|
||||
package rpcclient
|
||||
package client
|
@ -7,12 +7,12 @@ package main
|
||||
import (
|
||||
"log"
|
||||
|
||||
"github.com/kaspanet/kaspad/rpcclient"
|
||||
"github.com/kaspanet/kaspad/rpc/client"
|
||||
)
|
||||
|
||||
func main() {
|
||||
// Connect to a local kaspa RPC server using HTTP POST mode.
|
||||
connCfg := &rpcclient.ConnConfig{
|
||||
connCfg := &client.ConnConfig{
|
||||
Host: "localhost:8332",
|
||||
User: "yourrpcuser",
|
||||
Pass: "yourrpcpass",
|
||||
@ -21,7 +21,7 @@ func main() {
|
||||
}
|
||||
// Notice the notification parameter is nil since notifications are
|
||||
// not supported in HTTP POST mode.
|
||||
client, err := rpcclient.New(connCfg, nil)
|
||||
client, err := client.New(connCfg, nil)
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
@ -10,7 +10,7 @@ import (
|
||||
"path/filepath"
|
||||
"time"
|
||||
|
||||
"github.com/kaspanet/kaspad/rpcclient"
|
||||
"github.com/kaspanet/kaspad/rpc/client"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/kaspanet/kaspad/wire"
|
||||
)
|
||||
@ -20,7 +20,7 @@ func main() {
|
||||
// Also note most of these handlers will only be called if you register
|
||||
// for notifications. See the documentation of the rpcclient
|
||||
// NotificationHandlers type for more details about each handler.
|
||||
ntfnHandlers := rpcclient.NotificationHandlers{
|
||||
ntfnHandlers := client.NotificationHandlers{
|
||||
OnFilteredBlockAdded: func(blueScore uint64, header *wire.BlockHeader, txns []*util.Tx) {
|
||||
log.Printf("Block added: %s (%d) %s",
|
||||
header.BlockHash(), blueScore, header.Timestamp)
|
||||
@ -33,14 +33,14 @@ func main() {
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
||||
connCfg := &rpcclient.ConnConfig{
|
||||
connCfg := &client.ConnConfig{
|
||||
Host: "localhost:16110",
|
||||
Endpoint: "ws",
|
||||
User: "yourrpcuser",
|
||||
Pass: "yourrpcpass",
|
||||
Certificates: certs,
|
||||
}
|
||||
client, err := rpcclient.New(connCfg, &ntfnHandlers)
|
||||
client, err := client.New(connCfg, &ntfnHandlers)
|
||||
if err != nil {
|
||||
log.Fatal(err)
|
||||
}
|
@ -2,7 +2,7 @@
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package rpcclient
|
||||
package client
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
@ -26,7 +26,7 @@ import (
|
||||
|
||||
"github.com/btcsuite/go-socks/socks"
|
||||
"github.com/btcsuite/websocket"
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
)
|
||||
|
||||
var (
|
||||
@ -252,13 +252,13 @@ func (c *Client) trackRegisteredNtfns(cmd interface{}) {
|
||||
defer c.ntfnStateLock.Unlock()
|
||||
|
||||
switch bcmd := cmd.(type) {
|
||||
case *rpcmodel.NotifyBlocksCmd:
|
||||
case *model.NotifyBlocksCmd:
|
||||
c.ntfnState.notifyBlocks = true
|
||||
|
||||
case *rpcmodel.NotifyChainChangesCmd:
|
||||
case *model.NotifyChainChangesCmd:
|
||||
c.ntfnState.notifyChainChanges = true
|
||||
|
||||
case *rpcmodel.NotifyNewTransactionsCmd:
|
||||
case *model.NotifyNewTransactionsCmd:
|
||||
if bcmd.Verbose != nil && *bcmd.Verbose {
|
||||
c.ntfnState.notifyNewTxVerbose = true
|
||||
} else {
|
||||
@ -289,8 +289,8 @@ type (
|
||||
// rawResponse is a partially-unmarshaled JSON-RPC response. For this
|
||||
// to be valid (according to JSON-RPC 1.0 spec), ID may not be nil.
|
||||
rawResponse struct {
|
||||
Result json.RawMessage `json:"result"`
|
||||
Error *rpcmodel.RPCError `json:"error"`
|
||||
Result json.RawMessage `json:"result"`
|
||||
Error *model.RPCError `json:"error"`
|
||||
}
|
||||
)
|
||||
|
||||
@ -302,7 +302,7 @@ type response struct {
|
||||
}
|
||||
|
||||
// result checks whether the unmarshaled response contains a non-nil error,
|
||||
// returning an unmarshaled rpcmodel.RPCError (or an unmarshaling error) if so.
|
||||
// returning an unmarshaled model.RPCError (or an unmarshaling error) if so.
|
||||
// If the response is not an error, the raw bytes of the request are
|
||||
// returned for further unmashaling into specific result types.
|
||||
func (r rawResponse) result() (result []byte, err error) {
|
||||
@ -896,14 +896,14 @@ func (c *Client) sendRequest(data *jsonRequestData) chan *response {
|
||||
// configuration of the client.
|
||||
func (c *Client) sendCmd(cmd interface{}) chan *response {
|
||||
// Get the method associated with the command.
|
||||
method, err := rpcmodel.CommandMethod(cmd)
|
||||
method, err := model.CommandMethod(cmd)
|
||||
if err != nil {
|
||||
return newFutureError(err)
|
||||
}
|
||||
|
||||
// Marshal the command.
|
||||
id := c.NextID()
|
||||
marshalledJSON, err := rpcmodel.MarshalCommand(id, cmd)
|
||||
marshalledJSON, err := model.MarshalCommand(id, cmd)
|
||||
if err != nil {
|
||||
return newFutureError(err)
|
||||
}
|
@ -2,7 +2,7 @@
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package rpcclient
|
||||
package client
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/logs"
|
@ -2,7 +2,7 @@
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package rpcclient
|
||||
package client
|
||||
|
||||
import (
|
||||
"encoding/hex"
|
||||
@ -10,7 +10,7 @@ import (
|
||||
"strconv"
|
||||
"strings"
|
||||
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
"github.com/kaspanet/kaspad/wire"
|
||||
@ -47,7 +47,7 @@ func (r FutureSubmitBlockResult) Receive() error {
|
||||
// returned instance.
|
||||
//
|
||||
// See SubmitBlock for the blocking version and more details.
|
||||
func (c *Client) SubmitBlockAsync(block *util.Block, options *rpcmodel.SubmitBlockOptions) FutureSubmitBlockResult {
|
||||
func (c *Client) SubmitBlockAsync(block *util.Block, options *model.SubmitBlockOptions) FutureSubmitBlockResult {
|
||||
blockHex := ""
|
||||
if block != nil {
|
||||
blockBytes, err := block.Bytes()
|
||||
@ -58,12 +58,12 @@ func (c *Client) SubmitBlockAsync(block *util.Block, options *rpcmodel.SubmitBlo
|
||||
blockHex = hex.EncodeToString(blockBytes)
|
||||
}
|
||||
|
||||
cmd := rpcmodel.NewSubmitBlockCmd(blockHex, options)
|
||||
cmd := model.NewSubmitBlockCmd(blockHex, options)
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
// SubmitBlock attempts to submit a new block into the kaspa network.
|
||||
func (c *Client) SubmitBlock(block *util.Block, options *rpcmodel.SubmitBlockOptions) error {
|
||||
func (c *Client) SubmitBlock(block *util.Block, options *model.SubmitBlockOptions) error {
|
||||
return c.SubmitBlockAsync(block, options).Receive()
|
||||
}
|
||||
|
||||
@ -77,24 +77,24 @@ type FutureGetBlockTemplateResult chan *response
|
||||
//
|
||||
// See GetBlockTemplate for the blocking version and more details
|
||||
func (c *Client) GetBlockTemplateAsync(payAddress string, longPollID string) FutureGetBlockTemplateResult {
|
||||
request := &rpcmodel.TemplateRequest{
|
||||
request := &model.TemplateRequest{
|
||||
Mode: "template",
|
||||
LongPollID: longPollID,
|
||||
PayAddress: payAddress,
|
||||
}
|
||||
cmd := rpcmodel.NewGetBlockTemplateCmd(request)
|
||||
cmd := model.NewGetBlockTemplateCmd(request)
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
// Receive waits for the response promised by the future and returns an error if
|
||||
// any occurred when submitting the block.
|
||||
func (r FutureGetBlockTemplateResult) Receive() (*rpcmodel.GetBlockTemplateResult, error) {
|
||||
func (r FutureGetBlockTemplateResult) Receive() (*model.GetBlockTemplateResult, error) {
|
||||
res, err := receiveFuture(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
var result rpcmodel.GetBlockTemplateResult
|
||||
var result model.GetBlockTemplateResult
|
||||
if err := json.Unmarshal(res, &result); err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -102,12 +102,12 @@ func (r FutureGetBlockTemplateResult) Receive() (*rpcmodel.GetBlockTemplateResul
|
||||
}
|
||||
|
||||
// GetBlockTemplate request a block template from the server, to mine upon
|
||||
func (c *Client) GetBlockTemplate(payAddress string, longPollID string) (*rpcmodel.GetBlockTemplateResult, error) {
|
||||
func (c *Client) GetBlockTemplate(payAddress string, longPollID string) (*model.GetBlockTemplateResult, error) {
|
||||
return c.GetBlockTemplateAsync(payAddress, longPollID).Receive()
|
||||
}
|
||||
|
||||
// ConvertGetBlockTemplateResultToBlock Accepts a GetBlockTemplateResult and parses it into a Block
|
||||
func ConvertGetBlockTemplateResultToBlock(template *rpcmodel.GetBlockTemplateResult) (*util.Block, error) {
|
||||
func ConvertGetBlockTemplateResultToBlock(template *model.GetBlockTemplateResult) (*util.Block, error) {
|
||||
// parse parent hashes
|
||||
parentHashes := make([]*daghash.Hash, len(template.ParentHashes))
|
||||
for i, parentHash := range template.ParentHashes {
|
@ -2,7 +2,7 @@
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package rpcclient
|
||||
package client
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
@ -12,7 +12,7 @@ import (
|
||||
"github.com/kaspanet/kaspad/util/pointers"
|
||||
"github.com/kaspanet/kaspad/wire"
|
||||
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
)
|
||||
|
||||
// FutureAddNodeResult is a future promise to deliver the result of an
|
||||
@ -32,7 +32,7 @@ func (r FutureAddNodeResult) Receive() error {
|
||||
//
|
||||
// See AddNode for the blocking version and more details.
|
||||
func (c *Client) AddManualNodeAsync(host string) FutureAddNodeResult {
|
||||
cmd := rpcmodel.NewAddManualNodeCmd(host, pointers.Bool(false))
|
||||
cmd := model.NewConnectCmd(host, pointers.Bool(false))
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -45,87 +45,6 @@ func (c *Client) AddManualNode(host string) error {
|
||||
return c.AddManualNodeAsync(host).Receive()
|
||||
}
|
||||
|
||||
// FutureGetManualNodeInfoResult is a future promise to deliver the result of a
|
||||
// GetManualNodeInfoAsync RPC invocation (or an applicable error).
|
||||
type FutureGetManualNodeInfoResult chan *response
|
||||
|
||||
// Receive waits for the response promised by the future and returns information
|
||||
// about manually added (persistent) peers.
|
||||
func (r FutureGetManualNodeInfoResult) Receive() ([]rpcmodel.GetManualNodeInfoResult, error) {
|
||||
res, err := receiveFuture(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Unmarshal as an array of getmanualnodeinfo result objects.
|
||||
var nodeInfo []rpcmodel.GetManualNodeInfoResult
|
||||
err = json.Unmarshal(res, &nodeInfo)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return nodeInfo, nil
|
||||
}
|
||||
|
||||
// GetManualNodeInfoAsync returns an instance of a type that can be used to get
|
||||
// the result of the RPC at some future time by invoking the Receive function on
|
||||
// the returned instance.
|
||||
//
|
||||
// See GetManualNodeInfo for the blocking version and more details.
|
||||
func (c *Client) GetManualNodeInfoAsync(peer string) FutureGetManualNodeInfoResult {
|
||||
cmd := rpcmodel.NewGetManualNodeInfoCmd(peer, nil)
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
// GetManualNodeInfo returns information about manually added (persistent) peers.
|
||||
//
|
||||
// See GetManualNodeInfoNoDNS to retrieve only a list of the added (persistent)
|
||||
// peers.
|
||||
func (c *Client) GetManualNodeInfo(peer string) ([]rpcmodel.GetManualNodeInfoResult, error) {
|
||||
return c.GetManualNodeInfoAsync(peer).Receive()
|
||||
}
|
||||
|
||||
// FutureGetManualNodeInfoNoDNSResult is a future promise to deliver the result
|
||||
// of a GetManualNodeInfoNoDNSAsync RPC invocation (or an applicable error).
|
||||
type FutureGetManualNodeInfoNoDNSResult chan *response
|
||||
|
||||
// Receive waits for the response promised by the future and returns a list of
|
||||
// manually added (persistent) peers.
|
||||
func (r FutureGetManualNodeInfoNoDNSResult) Receive() ([]string, error) {
|
||||
res, err := receiveFuture(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Unmarshal result as an array of strings.
|
||||
var nodes []string
|
||||
err = json.Unmarshal(res, &nodes)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return nodes, nil
|
||||
}
|
||||
|
||||
// GetManualNodeInfoNoDNSAsync returns an instance of a type that can be used to
|
||||
// get the result of the RPC at some future time by invoking the Receive
|
||||
// function on the returned instance.
|
||||
//
|
||||
// See GetManualNodeInfoNoDNS for the blocking version and more details.
|
||||
func (c *Client) GetManualNodeInfoNoDNSAsync(peer string) FutureGetManualNodeInfoNoDNSResult {
|
||||
cmd := rpcmodel.NewGetManualNodeInfoCmd(peer, pointers.Bool(false))
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
// GetManualNodeInfoNoDNS returns a list of manually added (persistent) peers.
|
||||
// This works by setting the dns flag to false in the underlying RPC.
|
||||
//
|
||||
// See GetManualNodeInfo to obtain more information about each added (persistent)
|
||||
// peer.
|
||||
func (c *Client) GetManualNodeInfoNoDNS(peer string) ([]string, error) {
|
||||
return c.GetManualNodeInfoNoDNSAsync(peer).Receive()
|
||||
}
|
||||
|
||||
// FutureGetConnectionCountResult is a future promise to deliver the result
|
||||
// of a GetConnectionCountAsync RPC invocation (or an applicable error).
|
||||
type FutureGetConnectionCountResult chan *response
|
||||
@ -154,7 +73,7 @@ func (r FutureGetConnectionCountResult) Receive() (int64, error) {
|
||||
//
|
||||
// See GetConnectionCount for the blocking version and more details.
|
||||
func (c *Client) GetConnectionCountAsync() FutureGetConnectionCountResult {
|
||||
cmd := rpcmodel.NewGetConnectionCountCmd()
|
||||
cmd := model.NewGetConnectionCountCmd()
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -180,7 +99,7 @@ func (r FuturePingResult) Receive() error {
|
||||
//
|
||||
// See Ping for the blocking version and more details.
|
||||
func (c *Client) PingAsync() FuturePingResult {
|
||||
cmd := rpcmodel.NewPingCmd()
|
||||
cmd := model.NewPingCmd()
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -198,14 +117,14 @@ type FutureGetConnectedPeerInfo chan *response
|
||||
|
||||
// Receive waits for the response promised by the future and returns data about
|
||||
// each connected network peer.
|
||||
func (r FutureGetConnectedPeerInfo) Receive() ([]rpcmodel.GetConnectedPeerInfoResult, error) {
|
||||
func (r FutureGetConnectedPeerInfo) Receive() ([]model.GetConnectedPeerInfoResult, error) {
|
||||
res, err := receiveFuture(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Unmarshal result as an array of getConnectedPeerInfo result objects.
|
||||
var peerInfo []rpcmodel.GetConnectedPeerInfoResult
|
||||
var peerInfo []model.GetConnectedPeerInfoResult
|
||||
err = json.Unmarshal(res, &peerInfo)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@ -220,12 +139,12 @@ func (r FutureGetConnectedPeerInfo) Receive() ([]rpcmodel.GetConnectedPeerInfoRe
|
||||
//
|
||||
// See GetConnectedPeerInfo for the blocking version and more details.
|
||||
func (c *Client) GetConnectedPeerInfoAsync() FutureGetConnectedPeerInfo {
|
||||
cmd := rpcmodel.NewGetConnectedPeerInfoCmd()
|
||||
cmd := model.NewGetConnectedPeerInfoCmd()
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
// GetConnectedPeerInfo returns data about each connected network peer.
|
||||
func (c *Client) GetConnectedPeerInfo() ([]rpcmodel.GetConnectedPeerInfoResult, error) {
|
||||
func (c *Client) GetConnectedPeerInfo() ([]model.GetConnectedPeerInfoResult, error) {
|
||||
return c.GetConnectedPeerInfoAsync().Receive()
|
||||
}
|
||||
|
||||
@ -235,14 +154,14 @@ type FutureGetNetTotalsResult chan *response
|
||||
|
||||
// Receive waits for the response promised by the future and returns network
|
||||
// traffic statistics.
|
||||
func (r FutureGetNetTotalsResult) Receive() (*rpcmodel.GetNetTotalsResult, error) {
|
||||
func (r FutureGetNetTotalsResult) Receive() (*model.GetNetTotalsResult, error) {
|
||||
res, err := receiveFuture(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Unmarshal result as a getnettotals result object.
|
||||
var totals rpcmodel.GetNetTotalsResult
|
||||
var totals model.GetNetTotalsResult
|
||||
err = json.Unmarshal(res, &totals)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@ -257,12 +176,12 @@ func (r FutureGetNetTotalsResult) Receive() (*rpcmodel.GetNetTotalsResult, error
|
||||
//
|
||||
// See GetNetTotals for the blocking version and more details.
|
||||
func (c *Client) GetNetTotalsAsync() FutureGetNetTotalsResult {
|
||||
cmd := rpcmodel.NewGetNetTotalsCmd()
|
||||
cmd := model.NewGetNetTotalsCmd()
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
// GetNetTotals returns network traffic statistics.
|
||||
func (c *Client) GetNetTotals() (*rpcmodel.GetNetTotalsResult, error) {
|
||||
func (c *Client) GetNetTotals() (*model.GetNetTotalsResult, error) {
|
||||
return c.GetNetTotalsAsync().Receive()
|
||||
}
|
||||
|
||||
@ -294,7 +213,7 @@ func (r FutureDebugLevelResult) Receive() (string, error) {
|
||||
//
|
||||
// See DebugLevel for the blocking version and more details.
|
||||
func (c *Client) DebugLevelAsync(levelSpec string) FutureDebugLevelResult {
|
||||
cmd := rpcmodel.NewDebugLevelCmd(levelSpec)
|
||||
cmd := model.NewDebugLevelCmd(levelSpec)
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -350,12 +269,12 @@ func (r FutureGetSelectedTipResult) Receive() (*wire.MsgBlock, error) {
|
||||
//
|
||||
// See GetSelectedTip for the blocking version and more details.
|
||||
func (c *Client) GetSelectedTipAsync() FutureGetSelectedTipResult {
|
||||
cmd := rpcmodel.NewGetSelectedTipCmd(pointers.Bool(false), pointers.Bool(false))
|
||||
cmd := model.NewGetSelectedTipCmd(pointers.Bool(false), pointers.Bool(false))
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
// GetSelectedTip returns the block of the selected DAG tip
|
||||
func (c *Client) GetSelectedTip() (*rpcmodel.GetBlockVerboseResult, error) {
|
||||
func (c *Client) GetSelectedTip() (*model.GetBlockVerboseResult, error) {
|
||||
return c.GetSelectedTipVerboseAsync().Receive()
|
||||
}
|
||||
|
||||
@ -365,14 +284,14 @@ type FutureGetSelectedTipVerboseResult chan *response
|
||||
|
||||
// Receive waits for the response promised by the future and returns the data
|
||||
// structure from the server with information about the requested block.
|
||||
func (r FutureGetSelectedTipVerboseResult) Receive() (*rpcmodel.GetBlockVerboseResult, error) {
|
||||
func (r FutureGetSelectedTipVerboseResult) Receive() (*model.GetBlockVerboseResult, error) {
|
||||
res, err := receiveFuture(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Unmarshal the raw result into a BlockResult.
|
||||
var blockResult rpcmodel.GetBlockVerboseResult
|
||||
var blockResult model.GetBlockVerboseResult
|
||||
err = json.Unmarshal(res, &blockResult)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@ -386,7 +305,7 @@ func (r FutureGetSelectedTipVerboseResult) Receive() (*rpcmodel.GetBlockVerboseR
|
||||
//
|
||||
// See GeSelectedTipBlockVerbose for the blocking version and more details.
|
||||
func (c *Client) GetSelectedTipVerboseAsync() FutureGetSelectedTipVerboseResult {
|
||||
cmd := rpcmodel.NewGetSelectedTipCmd(pointers.Bool(true), pointers.Bool(false))
|
||||
cmd := model.NewGetSelectedTipCmd(pointers.Bool(true), pointers.Bool(false))
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -418,7 +337,7 @@ func (r FutureGetCurrentNetResult) Receive() (wire.KaspaNet, error) {
|
||||
//
|
||||
// See GetCurrentNet for the blocking version and more details.
|
||||
func (c *Client) GetCurrentNetAsync() FutureGetCurrentNetResult {
|
||||
cmd := rpcmodel.NewGetCurrentNetCmd()
|
||||
cmd := model.NewGetCurrentNetCmd()
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -470,7 +389,7 @@ func (c *Client) GetTopHeadersAsync(highHash *daghash.Hash) FutureGetHeadersResu
|
||||
if highHash != nil {
|
||||
hash = pointers.String(highHash.String())
|
||||
}
|
||||
cmd := rpcmodel.NewGetTopHeadersCmd(hash)
|
||||
cmd := model.NewGetTopHeadersCmd(hash)
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -492,7 +411,7 @@ func (c *Client) GetHeadersAsync(lowHash, highHash *daghash.Hash) FutureGetHeade
|
||||
if highHash != nil {
|
||||
highHashStr = highHash.String()
|
||||
}
|
||||
cmd := rpcmodel.NewGetHeadersCmd(lowHashStr, highHashStr)
|
||||
cmd := model.NewGetHeadersCmd(lowHashStr, highHashStr)
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -509,14 +428,14 @@ type FutureSessionResult chan *response
|
||||
|
||||
// Receive waits for the response promised by the future and returns the
|
||||
// session result.
|
||||
func (r FutureSessionResult) Receive() (*rpcmodel.SessionResult, error) {
|
||||
func (r FutureSessionResult) Receive() (*model.SessionResult, error) {
|
||||
res, err := receiveFuture(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Unmarshal result as a session result object.
|
||||
var session rpcmodel.SessionResult
|
||||
var session model.SessionResult
|
||||
err = json.Unmarshal(res, &session)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@ -536,14 +455,14 @@ func (c *Client) SessionAsync() FutureSessionResult {
|
||||
return newFutureError(ErrWebsocketsRequired)
|
||||
}
|
||||
|
||||
cmd := rpcmodel.NewSessionCmd()
|
||||
cmd := model.NewSessionCmd()
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
// Session returns details regarding a websocket client's current connection.
|
||||
//
|
||||
// This RPC requires the client to be running in websocket mode.
|
||||
func (c *Client) Session() (*rpcmodel.SessionResult, error) {
|
||||
func (c *Client) Session() (*model.SessionResult, error) {
|
||||
return c.SessionAsync().Receive()
|
||||
}
|
||||
|
||||
@ -553,7 +472,7 @@ type FutureVersionResult chan *response
|
||||
|
||||
// Receive waits for the response promised by the future and returns the version
|
||||
// result.
|
||||
func (r FutureVersionResult) Receive() (map[string]rpcmodel.VersionResult,
|
||||
func (r FutureVersionResult) Receive() (map[string]model.VersionResult,
|
||||
error) {
|
||||
res, err := receiveFuture(r)
|
||||
if err != nil {
|
||||
@ -561,7 +480,7 @@ func (r FutureVersionResult) Receive() (map[string]rpcmodel.VersionResult,
|
||||
}
|
||||
|
||||
// Unmarshal result as a version result object.
|
||||
var vr map[string]rpcmodel.VersionResult
|
||||
var vr map[string]model.VersionResult
|
||||
err = json.Unmarshal(res, &vr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@ -576,11 +495,11 @@ func (r FutureVersionResult) Receive() (map[string]rpcmodel.VersionResult,
|
||||
//
|
||||
// See Version for the blocking version and more details.
|
||||
func (c *Client) VersionAsync() FutureVersionResult {
|
||||
cmd := rpcmodel.NewVersionCmd()
|
||||
cmd := model.NewVersionCmd()
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
// Version returns information about the server's JSON-RPC API versions.
|
||||
func (c *Client) Version() (map[string]rpcmodel.VersionResult, error) {
|
||||
func (c *Client) Version() (map[string]model.VersionResult, error) {
|
||||
return c.VersionAsync().Receive()
|
||||
}
|
@ -3,7 +3,7 @@
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package rpcclient
|
||||
package client
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
@ -13,7 +13,7 @@ import (
|
||||
"github.com/kaspanet/kaspad/util/mstime"
|
||||
"github.com/pkg/errors"
|
||||
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
"github.com/kaspanet/kaspad/wire"
|
||||
@ -112,7 +112,7 @@ type NotificationHandlers struct {
|
||||
// memory pool. It will only be invoked if a preceding call to
|
||||
// NotifyNewTransactions with the verbose flag set to true has been
|
||||
// made to register for the notification and the function is non-nil.
|
||||
OnTxAcceptedVerbose func(txDetails *rpcmodel.TxRawResult)
|
||||
OnTxAcceptedVerbose func(txDetails *model.TxRawResult)
|
||||
|
||||
// OnUnknownNotification is invoked when an unrecognized notification
|
||||
// is received. This typically means the notification handling code
|
||||
@ -136,7 +136,7 @@ func (c *Client) handleNotification(ntfn *rawNotification) {
|
||||
switch ntfn.Method {
|
||||
|
||||
// ChainChangedNtfnMethod
|
||||
case rpcmodel.ChainChangedNtfnMethod:
|
||||
case model.ChainChangedNtfnMethod:
|
||||
// Ignore the notification if the client is not interested in
|
||||
// it.
|
||||
if c.ntfnHandlers.OnChainChanged == nil {
|
||||
@ -153,7 +153,7 @@ func (c *Client) handleNotification(ntfn *rawNotification) {
|
||||
c.ntfnHandlers.OnChainChanged(removedChainBlockHashes, addedChainBlocks)
|
||||
|
||||
// OnFilteredBlockAdded
|
||||
case rpcmodel.FilteredBlockAddedNtfnMethod:
|
||||
case model.FilteredBlockAddedNtfnMethod:
|
||||
// Ignore the notification if the client is not interested in
|
||||
// it.
|
||||
if c.ntfnHandlers.OnFilteredBlockAdded == nil {
|
||||
@ -172,7 +172,7 @@ func (c *Client) handleNotification(ntfn *rawNotification) {
|
||||
blockHeader, transactions)
|
||||
|
||||
// OnRelevantTxAccepted
|
||||
case rpcmodel.RelevantTxAcceptedNtfnMethod:
|
||||
case model.RelevantTxAcceptedNtfnMethod:
|
||||
// Ignore the notification if the client is not interested in
|
||||
// it.
|
||||
if c.ntfnHandlers.OnRelevantTxAccepted == nil {
|
||||
@ -189,7 +189,7 @@ func (c *Client) handleNotification(ntfn *rawNotification) {
|
||||
c.ntfnHandlers.OnRelevantTxAccepted(transaction)
|
||||
|
||||
// OnTxAccepted
|
||||
case rpcmodel.TxAcceptedNtfnMethod:
|
||||
case model.TxAcceptedNtfnMethod:
|
||||
// Ignore the notification if the client is not interested in
|
||||
// it.
|
||||
if c.ntfnHandlers.OnTxAccepted == nil {
|
||||
@ -206,7 +206,7 @@ func (c *Client) handleNotification(ntfn *rawNotification) {
|
||||
c.ntfnHandlers.OnTxAccepted(hash, amt)
|
||||
|
||||
// OnTxAcceptedVerbose
|
||||
case rpcmodel.TxAcceptedVerboseNtfnMethod:
|
||||
case model.TxAcceptedVerboseNtfnMethod:
|
||||
// Ignore the notification if the client is not interested in
|
||||
// it.
|
||||
if c.ntfnHandlers.OnTxAcceptedVerbose == nil {
|
||||
@ -264,7 +264,7 @@ func parseChainChangedParams(params []json.RawMessage) (removedChainBlockHashes
|
||||
}
|
||||
|
||||
// Unmarshal first parameter as a raw transaction result object.
|
||||
var rawParam rpcmodel.ChainChangedRawParam
|
||||
var rawParam model.ChainChangedRawParam
|
||||
err = json.Unmarshal(params[0], &rawParam)
|
||||
if err != nil {
|
||||
return nil, nil, err
|
||||
@ -425,7 +425,7 @@ func parseTxAcceptedNtfnParams(params []json.RawMessage) (*daghash.Hash,
|
||||
|
||||
// parseTxAcceptedVerboseNtfnParams parses out details about a raw transaction
|
||||
// from the parameters of a txacceptedverbose notification.
|
||||
func parseTxAcceptedVerboseNtfnParams(params []json.RawMessage) (*rpcmodel.TxRawResult,
|
||||
func parseTxAcceptedVerboseNtfnParams(params []json.RawMessage) (*model.TxRawResult,
|
||||
error) {
|
||||
|
||||
if len(params) != 1 {
|
||||
@ -433,7 +433,7 @@ func parseTxAcceptedVerboseNtfnParams(params []json.RawMessage) (*rpcmodel.TxRaw
|
||||
}
|
||||
|
||||
// Unmarshal first parameter as a raw transaction result object.
|
||||
var rawTx rpcmodel.TxRawResult
|
||||
var rawTx model.TxRawResult
|
||||
err := json.Unmarshal(params[0], &rawTx)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
@ -473,7 +473,7 @@ func (c *Client) NotifyBlocksAsync() FutureNotifyBlocksResult {
|
||||
return newNilFutureResult()
|
||||
}
|
||||
|
||||
cmd := rpcmodel.NewNotifyBlocksCmd()
|
||||
cmd := model.NewNotifyBlocksCmd()
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -516,7 +516,7 @@ func (c *Client) NotifyChainChangesAsync() FutureNotifyBlocksResult {
|
||||
return newNilFutureResult()
|
||||
}
|
||||
|
||||
cmd := rpcmodel.NewNotifyChainChangesCmd()
|
||||
cmd := model.NewNotifyChainChangesCmd()
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -559,7 +559,7 @@ func (c *Client) NotifyNewTransactionsAsync(verbose bool, subnetworkID *string)
|
||||
return newNilFutureResult()
|
||||
}
|
||||
|
||||
cmd := rpcmodel.NewNotifyNewTransactionsCmd(&verbose, subnetworkID)
|
||||
cmd := model.NewNotifyNewTransactionsCmd(&verbose, subnetworkID)
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
@ -599,15 +599,15 @@ func (c *Client) LoadTxFilterAsync(reload bool, addresses []util.Address,
|
||||
for i, a := range addresses {
|
||||
addrStrs[i] = a.EncodeAddress()
|
||||
}
|
||||
outpointObjects := make([]rpcmodel.Outpoint, len(outpoints))
|
||||
outpointObjects := make([]model.Outpoint, len(outpoints))
|
||||
for i := range outpoints {
|
||||
outpointObjects[i] = rpcmodel.Outpoint{
|
||||
outpointObjects[i] = model.Outpoint{
|
||||
TxID: outpoints[i].TxID.String(),
|
||||
Index: outpoints[i].Index,
|
||||
}
|
||||
}
|
||||
|
||||
cmd := rpcmodel.NewLoadTxFilterCmd(reload, addrStrs, outpointObjects)
|
||||
cmd := model.NewLoadTxFilterCmd(reload, addrStrs, outpointObjects)
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
@ -2,11 +2,11 @@
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package rpcclient
|
||||
package client
|
||||
|
||||
import (
|
||||
"encoding/json"
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
@ -42,7 +42,7 @@ func (c *Client) RawRequestAsync(method string, params []json.RawMessage) Future
|
||||
// since that relies on marshalling registered jsonrpc commands rather
|
||||
// than custom commands.
|
||||
id := c.NextID()
|
||||
rawRequest := &rpcmodel.Request{
|
||||
rawRequest := &model.Request{
|
||||
JSONRPC: "1.0",
|
||||
ID: id,
|
||||
Method: method,
|
63
rpc/client/rawtransactions.go
Normal file
63
rpc/client/rawtransactions.go
Normal file
@ -0,0 +1,63 @@
|
||||
// Copyright (c) 2014-2017 The btcsuite developers
|
||||
// Use of this source code is governed by an ISC
|
||||
// license that can be found in the LICENSE file.
|
||||
|
||||
package client
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/hex"
|
||||
"encoding/json"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
"github.com/kaspanet/kaspad/wire"
|
||||
)
|
||||
|
||||
// FutureSendRawTransactionResult is a future promise to deliver the result
|
||||
// of a SendRawTransactionAsync RPC invocation (or an applicable error).
|
||||
type FutureSendRawTransactionResult chan *response
|
||||
|
||||
// Receive waits for the response promised by the future and returns the result
|
||||
// of submitting the encoded transaction to the server which then relays it to
|
||||
// the network.
|
||||
func (r FutureSendRawTransactionResult) Receive() (*daghash.TxID, error) {
|
||||
res, err := receiveFuture(r)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
// Unmarshal result as a string.
|
||||
var txIDStr string
|
||||
err = json.Unmarshal(res, &txIDStr)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return daghash.NewTxIDFromStr(txIDStr)
|
||||
}
|
||||
|
||||
// SendRawTransactionAsync returns an instance of a type that can be used to get
|
||||
// the result of the RPC at some future time by invoking the Receive function on
|
||||
// the returned instance.
|
||||
//
|
||||
// See SendRawTransaction for the blocking version and more details.
|
||||
func (c *Client) SendRawTransactionAsync(tx *wire.MsgTx, allowHighFees bool) FutureSendRawTransactionResult {
|
||||
txHex := ""
|
||||
if tx != nil {
|
||||
// Serialize the transaction and convert to hex string.
|
||||
buf := bytes.NewBuffer(make([]byte, 0, tx.SerializeSize()))
|
||||
if err := tx.Serialize(buf); err != nil {
|
||||
return newFutureError(err)
|
||||
}
|
||||
txHex = hex.EncodeToString(buf.Bytes())
|
||||
}
|
||||
|
||||
cmd := model.NewSendRawTransactionCmd(txHex, &allowHighFees)
|
||||
return c.sendCmd(cmd)
|
||||
}
|
||||
|
||||
// SendRawTransaction submits the encoded transaction to the server which will
|
||||
// then relay it to the network.
|
||||
func (c *Client) SendRawTransaction(tx *wire.MsgTx, allowHighFees bool) (*daghash.TxID, error) {
|
||||
return c.SendRawTransactionAsync(tx, allowHighFees).Receive()
|
||||
}
|
@ -5,7 +5,7 @@ import (
|
||||
"encoding/hex"
|
||||
"fmt"
|
||||
"github.com/kaspanet/kaspad/dagconfig"
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/txscript"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
@ -18,8 +18,8 @@ import (
|
||||
var (
|
||||
// ErrRPCUnimplemented is an error returned to RPC clients when the
|
||||
// provided command is recognized, but not implemented.
|
||||
ErrRPCUnimplemented = &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCUnimplemented,
|
||||
ErrRPCUnimplemented = &model.RPCError{
|
||||
Code: model.ErrRPCUnimplemented,
|
||||
Message: "Command unimplemented",
|
||||
}
|
||||
)
|
||||
@ -29,19 +29,19 @@ var (
|
||||
// RPC server subsystem since internal errors really should not occur. The
|
||||
// context parameter is only used in the log message and may be empty if it's
|
||||
// not needed.
|
||||
func internalRPCError(errStr, context string) *rpcmodel.RPCError {
|
||||
func internalRPCError(errStr, context string) *model.RPCError {
|
||||
logStr := errStr
|
||||
if context != "" {
|
||||
logStr = context + ": " + errStr
|
||||
}
|
||||
log.Error(logStr)
|
||||
return rpcmodel.NewRPCError(rpcmodel.ErrRPCInternal.Code, errStr)
|
||||
return model.NewRPCError(model.ErrRPCInternal.Code, errStr)
|
||||
}
|
||||
|
||||
// rpcDecodeHexError is a convenience function for returning a nicely formatted
|
||||
// RPC error which indicates the provided hex string failed to decode.
|
||||
func rpcDecodeHexError(gotHex string) *rpcmodel.RPCError {
|
||||
return rpcmodel.NewRPCError(rpcmodel.ErrRPCDecodeHexString,
|
||||
func rpcDecodeHexError(gotHex string) *model.RPCError {
|
||||
return model.NewRPCError(model.ErrRPCDecodeHexString,
|
||||
fmt.Sprintf("Argument must be hexadecimal string (not %q)",
|
||||
gotHex))
|
||||
}
|
||||
@ -49,8 +49,8 @@ func rpcDecodeHexError(gotHex string) *rpcmodel.RPCError {
|
||||
// rpcNoTxInfoError is a convenience function for returning a nicely formatted
|
||||
// RPC error which indicates there is no information available for the provided
|
||||
// transaction hash.
|
||||
func rpcNoTxInfoError(txID *daghash.TxID) *rpcmodel.RPCError {
|
||||
return rpcmodel.NewRPCError(rpcmodel.ErrRPCNoTxInfo,
|
||||
func rpcNoTxInfoError(txID *daghash.TxID) *model.RPCError {
|
||||
return model.NewRPCError(model.ErrRPCNoTxInfo,
|
||||
fmt.Sprintf("No information available about transaction %s",
|
||||
txID))
|
||||
}
|
||||
@ -69,8 +69,8 @@ func messageToHex(msg wire.Message) (string, error) {
|
||||
|
||||
// createVinList returns a slice of JSON objects for the inputs of the passed
|
||||
// transaction.
|
||||
func createVinList(mtx *wire.MsgTx) []rpcmodel.Vin {
|
||||
vinList := make([]rpcmodel.Vin, len(mtx.TxIn))
|
||||
func createVinList(mtx *wire.MsgTx) []model.Vin {
|
||||
vinList := make([]model.Vin, len(mtx.TxIn))
|
||||
for i, txIn := range mtx.TxIn {
|
||||
// The disassembled string will contain [error] inline
|
||||
// if the script doesn't fully parse, so ignore the
|
||||
@ -81,7 +81,7 @@ func createVinList(mtx *wire.MsgTx) []rpcmodel.Vin {
|
||||
vinEntry.TxID = txIn.PreviousOutpoint.TxID.String()
|
||||
vinEntry.Vout = txIn.PreviousOutpoint.Index
|
||||
vinEntry.Sequence = txIn.Sequence
|
||||
vinEntry.ScriptSig = &rpcmodel.ScriptSig{
|
||||
vinEntry.ScriptSig = &model.ScriptSig{
|
||||
Asm: disbuf,
|
||||
Hex: hex.EncodeToString(txIn.SignatureScript),
|
||||
}
|
||||
@ -92,8 +92,8 @@ func createVinList(mtx *wire.MsgTx) []rpcmodel.Vin {
|
||||
|
||||
// createVoutList returns a slice of JSON objects for the outputs of the passed
|
||||
// transaction.
|
||||
func createVoutList(mtx *wire.MsgTx, dagParams *dagconfig.Params, filterAddrMap map[string]struct{}) []rpcmodel.Vout {
|
||||
voutList := make([]rpcmodel.Vout, 0, len(mtx.TxOut))
|
||||
func createVoutList(mtx *wire.MsgTx, dagParams *dagconfig.Params, filterAddrMap map[string]struct{}) []model.Vout {
|
||||
voutList := make([]model.Vout, 0, len(mtx.TxOut))
|
||||
for i, v := range mtx.TxOut {
|
||||
// The disassembled string will contain [error] inline if the
|
||||
// script doesn't fully parse, so ignore the error here.
|
||||
@ -123,7 +123,7 @@ func createVoutList(mtx *wire.MsgTx, dagParams *dagconfig.Params, filterAddrMap
|
||||
continue
|
||||
}
|
||||
|
||||
var vout rpcmodel.Vout
|
||||
var vout model.Vout
|
||||
vout.N = uint32(i)
|
||||
vout.Value = v.Value
|
||||
vout.ScriptPubKey.Address = encodedAddr
|
||||
@ -141,7 +141,7 @@ func createVoutList(mtx *wire.MsgTx, dagParams *dagconfig.Params, filterAddrMap
|
||||
// to a raw transaction JSON object.
|
||||
func createTxRawResult(dagParams *dagconfig.Params, mtx *wire.MsgTx,
|
||||
txID string, blkHeader *wire.BlockHeader, blkHash string,
|
||||
acceptingBlock *daghash.Hash, isInMempool bool) (*rpcmodel.TxRawResult, error) {
|
||||
acceptingBlock *daghash.Hash, isInMempool bool) (*model.TxRawResult, error) {
|
||||
|
||||
mtxHex, err := messageToHex(mtx)
|
||||
if err != nil {
|
||||
@ -153,7 +153,7 @@ func createTxRawResult(dagParams *dagconfig.Params, mtx *wire.MsgTx,
|
||||
payloadHash = mtx.PayloadHash.String()
|
||||
}
|
||||
|
||||
txReply := &rpcmodel.TxRawResult{
|
||||
txReply := &model.TxRawResult{
|
||||
Hex: mtxHex,
|
||||
TxID: txID,
|
||||
Hash: mtx.TxHash().String(),
|
||||
@ -201,34 +201,34 @@ func getDifficultyRatio(bits uint32, params *dagconfig.Params) float64 {
|
||||
return diff
|
||||
}
|
||||
|
||||
// buildGetBlockVerboseResult takes a block and convert it to rpcmodel.GetBlockVerboseResult
|
||||
// buildGetBlockVerboseResult takes a block and convert it to model.GetBlockVerboseResult
|
||||
//
|
||||
// This function MUST be called with the DAG state lock held (for reads).
|
||||
func buildGetBlockVerboseResult(s *Server, block *util.Block, isVerboseTx bool) (*rpcmodel.GetBlockVerboseResult, error) {
|
||||
func buildGetBlockVerboseResult(s *Server, block *util.Block, isVerboseTx bool) (*model.GetBlockVerboseResult, error) {
|
||||
hash := block.Hash()
|
||||
params := s.cfg.DAGParams
|
||||
params := s.dag.Params
|
||||
blockHeader := block.MsgBlock().Header
|
||||
|
||||
blockBlueScore, err := s.cfg.DAG.BlueScoreByBlockHash(hash)
|
||||
blockBlueScore, err := s.dag.BlueScoreByBlockHash(hash)
|
||||
if err != nil {
|
||||
context := "Could not get block blue score"
|
||||
return nil, internalRPCError(err.Error(), context)
|
||||
}
|
||||
|
||||
// Get the hashes for the next blocks unless there are none.
|
||||
childHashes, err := s.cfg.DAG.ChildHashesByHash(hash)
|
||||
childHashes, err := s.dag.ChildHashesByHash(hash)
|
||||
if err != nil {
|
||||
context := "No next block"
|
||||
return nil, internalRPCError(err.Error(), context)
|
||||
}
|
||||
|
||||
blockConfirmations, err := s.cfg.DAG.BlockConfirmationsByHashNoLock(hash)
|
||||
blockConfirmations, err := s.dag.BlockConfirmationsByHashNoLock(hash)
|
||||
if err != nil {
|
||||
context := "Could not get block confirmations"
|
||||
return nil, internalRPCError(err.Error(), context)
|
||||
}
|
||||
|
||||
selectedParentHash, err := s.cfg.DAG.SelectedParentHash(hash)
|
||||
selectedParentHash, err := s.dag.SelectedParentHash(hash)
|
||||
if err != nil {
|
||||
context := "Could not get block selected parent"
|
||||
return nil, internalRPCError(err.Error(), context)
|
||||
@ -238,19 +238,19 @@ func buildGetBlockVerboseResult(s *Server, block *util.Block, isVerboseTx bool)
|
||||
selectedParentHashStr = selectedParentHash.String()
|
||||
}
|
||||
|
||||
isChainBlock, err := s.cfg.DAG.IsInSelectedParentChain(hash)
|
||||
isChainBlock, err := s.dag.IsInSelectedParentChain(hash)
|
||||
if err != nil {
|
||||
context := "Could not get whether block is in the selected parent chain"
|
||||
return nil, internalRPCError(err.Error(), context)
|
||||
}
|
||||
|
||||
acceptedBlockHashes, err := s.cfg.DAG.BluesByBlockHash(hash)
|
||||
acceptedBlockHashes, err := s.dag.BluesByBlockHash(hash)
|
||||
if err != nil {
|
||||
context := fmt.Sprintf("Could not get block accepted blocks for block %s", hash)
|
||||
return nil, internalRPCError(err.Error(), context)
|
||||
}
|
||||
|
||||
result := &rpcmodel.GetBlockVerboseResult{
|
||||
result := &model.GetBlockVerboseResult{
|
||||
Hash: hash.String(),
|
||||
Version: blockHeader.Version,
|
||||
VersionHex: fmt.Sprintf("%08x", blockHeader.Version),
|
||||
@ -281,7 +281,7 @@ func buildGetBlockVerboseResult(s *Server, block *util.Block, isVerboseTx bool)
|
||||
result.Tx = txNames
|
||||
} else {
|
||||
txns := block.Transactions()
|
||||
rawTxns := make([]rpcmodel.TxRawResult, len(txns))
|
||||
rawTxns := make([]model.TxRawResult, len(txns))
|
||||
for i, tx := range txns {
|
||||
rawTxn, err := createTxRawResult(params, tx.MsgTx(), tx.ID().String(),
|
||||
&blockHeader, hash.String(), nil, false)
|
||||
@ -296,18 +296,18 @@ func buildGetBlockVerboseResult(s *Server, block *util.Block, isVerboseTx bool)
|
||||
return result, nil
|
||||
}
|
||||
|
||||
func collectChainBlocks(s *Server, hashes []*daghash.Hash) ([]rpcmodel.ChainBlock, error) {
|
||||
chainBlocks := make([]rpcmodel.ChainBlock, 0, len(hashes))
|
||||
func collectChainBlocks(s *Server, hashes []*daghash.Hash) ([]model.ChainBlock, error) {
|
||||
chainBlocks := make([]model.ChainBlock, 0, len(hashes))
|
||||
for _, hash := range hashes {
|
||||
acceptanceData, err := s.cfg.AcceptanceIndex.TxsAcceptanceData(hash)
|
||||
acceptanceData, err := s.acceptanceIndex.TxsAcceptanceData(hash)
|
||||
if err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCInternal.Code,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInternal.Code,
|
||||
Message: fmt.Sprintf("could not retrieve acceptance data for block %s", hash),
|
||||
}
|
||||
}
|
||||
|
||||
acceptedBlocks := make([]rpcmodel.AcceptedBlock, 0, len(acceptanceData))
|
||||
acceptedBlocks := make([]model.AcceptedBlock, 0, len(acceptanceData))
|
||||
for _, blockAcceptanceData := range acceptanceData {
|
||||
acceptedTxIds := make([]string, 0, len(blockAcceptanceData.TxAcceptanceData))
|
||||
for _, txAcceptanceData := range blockAcceptanceData.TxAcceptanceData {
|
||||
@ -315,14 +315,14 @@ func collectChainBlocks(s *Server, hashes []*daghash.Hash) ([]rpcmodel.ChainBloc
|
||||
acceptedTxIds = append(acceptedTxIds, txAcceptanceData.Tx.ID().String())
|
||||
}
|
||||
}
|
||||
acceptedBlock := rpcmodel.AcceptedBlock{
|
||||
acceptedBlock := model.AcceptedBlock{
|
||||
Hash: blockAcceptanceData.BlockHash.String(),
|
||||
AcceptedTxIDs: acceptedTxIds,
|
||||
}
|
||||
acceptedBlocks = append(acceptedBlocks, acceptedBlock)
|
||||
}
|
||||
|
||||
chainBlock := rpcmodel.ChainBlock{
|
||||
chainBlock := model.ChainBlock{
|
||||
Hash: hash.String(),
|
||||
AcceptedBlocks: acceptedBlocks,
|
||||
}
|
||||
@ -335,20 +335,20 @@ func collectChainBlocks(s *Server, hashes []*daghash.Hash) ([]rpcmodel.ChainBloc
|
||||
// correspondent block verbose.
|
||||
//
|
||||
// This function MUST be called with the DAG state lock held (for reads).
|
||||
func hashesToGetBlockVerboseResults(s *Server, hashes []*daghash.Hash) ([]rpcmodel.GetBlockVerboseResult, error) {
|
||||
getBlockVerboseResults := make([]rpcmodel.GetBlockVerboseResult, 0, len(hashes))
|
||||
func hashesToGetBlockVerboseResults(s *Server, hashes []*daghash.Hash) ([]model.GetBlockVerboseResult, error) {
|
||||
getBlockVerboseResults := make([]model.GetBlockVerboseResult, 0, len(hashes))
|
||||
for _, blockHash := range hashes {
|
||||
block, err := s.cfg.DAG.BlockByHash(blockHash)
|
||||
block, err := s.dag.BlockByHash(blockHash)
|
||||
if err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCInternal.Code,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInternal.Code,
|
||||
Message: fmt.Sprintf("could not retrieve block %s.", blockHash),
|
||||
}
|
||||
}
|
||||
getBlockVerboseResult, err := buildGetBlockVerboseResult(s, block, false)
|
||||
if err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCInternal.Code,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInternal.Code,
|
||||
Message: fmt.Sprintf("could not build getBlockVerboseResult for block %s: %s", blockHash, err),
|
||||
}
|
||||
}
|
24
rpc/handle_connect.go
Normal file
24
rpc/handle_connect.go
Normal file
@ -0,0 +1,24 @@
|
||||
package rpc
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util/network"
|
||||
)
|
||||
|
||||
// handleConnect handles connect commands.
|
||||
func handleConnect(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
c := cmd.(*model.ConnectCmd)
|
||||
|
||||
isPermanent := c.IsPermanent != nil && *c.IsPermanent
|
||||
|
||||
address, err := network.NormalizeAddress(c.Address, s.dag.Params.DefaultPort)
|
||||
if err != nil {
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInvalidParameter,
|
||||
Message: err.Error(),
|
||||
}
|
||||
}
|
||||
|
||||
s.connectionManager.AddConnectionRequest(address, isPermanent)
|
||||
return nil, nil
|
||||
}
|
@ -3,12 +3,12 @@ package rpc
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/kaspanet/kaspad/logger"
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
)
|
||||
|
||||
// handleDebugLevel handles debugLevel commands.
|
||||
func handleDebugLevel(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
c := cmd.(*rpcmodel.DebugLevelCmd)
|
||||
c := cmd.(*model.DebugLevelCmd)
|
||||
|
||||
// Special show command to list supported subsystems.
|
||||
if c.LevelSpec == "show" {
|
||||
@ -18,8 +18,8 @@ func handleDebugLevel(s *Server, cmd interface{}, closeChan <-chan struct{}) (in
|
||||
|
||||
err := logger.ParseAndSetDebugLevels(c.LevelSpec)
|
||||
if err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCInvalidParams.Code,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInvalidParams.Code,
|
||||
Message: err.Error(),
|
||||
}
|
||||
}
|
22
rpc/handle_disconnect.go
Normal file
22
rpc/handle_disconnect.go
Normal file
@ -0,0 +1,22 @@
|
||||
package rpc
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util/network"
|
||||
)
|
||||
|
||||
// handleDisconnect handles disconnect commands.
|
||||
func handleDisconnect(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
c := cmd.(*model.DisconnectCmd)
|
||||
|
||||
address, err := network.NormalizeAddress(c.Address, s.dag.Params.DefaultPort)
|
||||
if err != nil {
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInvalidParameter,
|
||||
Message: err.Error(),
|
||||
}
|
||||
}
|
||||
|
||||
s.connectionManager.RemoveConnection(address)
|
||||
return nil, nil
|
||||
}
|
@ -5,7 +5,7 @@ import (
|
||||
"bytes"
|
||||
"encoding/hex"
|
||||
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
"github.com/kaspanet/kaspad/util/subnetworkid"
|
||||
@ -13,7 +13,7 @@ import (
|
||||
|
||||
// handleGetBlock implements the getBlock command.
|
||||
func handleGetBlock(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
c := cmd.(*rpcmodel.GetBlockCmd)
|
||||
c := cmd.(*model.GetBlockCmd)
|
||||
|
||||
// Load the raw block bytes from the database.
|
||||
hash, err := daghash.NewHashFromStr(c.Hash)
|
||||
@ -22,32 +22,32 @@ func handleGetBlock(s *Server, cmd interface{}, closeChan <-chan struct{}) (inte
|
||||
}
|
||||
|
||||
// Return an appropriate error if the block is known to be invalid
|
||||
if s.cfg.DAG.IsKnownInvalid(hash) {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCBlockInvalid,
|
||||
if s.dag.IsKnownInvalid(hash) {
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCBlockInvalid,
|
||||
Message: "Block is known to be invalid",
|
||||
}
|
||||
}
|
||||
|
||||
// Return an appropriate error if the block is an orphan
|
||||
if s.cfg.DAG.IsKnownOrphan(hash) {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCOrphanBlock,
|
||||
if s.dag.IsKnownOrphan(hash) {
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCOrphanBlock,
|
||||
Message: "Block is an orphan",
|
||||
}
|
||||
}
|
||||
|
||||
block, err := s.cfg.DAG.BlockByHash(hash)
|
||||
block, err := s.dag.BlockByHash(hash)
|
||||
if err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCBlockNotFound,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCBlockNotFound,
|
||||
Message: "Block not found",
|
||||
}
|
||||
}
|
||||
blockBytes, err := block.Bytes()
|
||||
if err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCBlockInvalid,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCBlockInvalid,
|
||||
Message: "Cannot serialize block",
|
||||
}
|
||||
}
|
||||
@ -56,18 +56,18 @@ func handleGetBlock(s *Server, cmd interface{}, closeChan <-chan struct{}) (inte
|
||||
if c.Subnetwork != nil {
|
||||
requestSubnetworkID, err := subnetworkid.NewFromStr(*c.Subnetwork)
|
||||
if err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCInvalidRequest.Code,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInvalidRequest.Code,
|
||||
Message: "invalid subnetwork string",
|
||||
}
|
||||
}
|
||||
nodeSubnetworkID := s.appCfg.SubnetworkID
|
||||
nodeSubnetworkID := s.cfg.SubnetworkID
|
||||
|
||||
if requestSubnetworkID != nil {
|
||||
if nodeSubnetworkID != nil {
|
||||
if !nodeSubnetworkID.IsEqual(requestSubnetworkID) {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCInvalidRequest.Code,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInvalidRequest.Code,
|
||||
Message: "subnetwork does not match this partial node",
|
||||
}
|
||||
}
|
||||
@ -98,8 +98,8 @@ func handleGetBlock(s *Server, cmd interface{}, closeChan <-chan struct{}) (inte
|
||||
return nil, internalRPCError(err.Error(), context)
|
||||
}
|
||||
|
||||
s.cfg.DAG.RLock()
|
||||
defer s.cfg.DAG.RUnlock()
|
||||
s.dag.RLock()
|
||||
defer s.dag.RUnlock()
|
||||
blockReply, err := buildGetBlockVerboseResult(s, block, c.VerboseTx == nil || !*c.VerboseTx)
|
||||
if err != nil {
|
||||
return nil, err
|
@ -2,5 +2,5 @@ package rpc
|
||||
|
||||
// handleGetBlockCount implements the getBlockCount command.
|
||||
func handleGetBlockCount(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
return s.cfg.DAG.BlockCount(), nil
|
||||
return s.dag.BlockCount(), nil
|
||||
}
|
@ -4,7 +4,7 @@ import (
|
||||
"fmt"
|
||||
"github.com/kaspanet/kaspad/blockdag"
|
||||
"github.com/kaspanet/kaspad/dagconfig"
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
"github.com/pkg/errors"
|
||||
"strings"
|
||||
@ -14,10 +14,10 @@ import (
|
||||
func handleGetBlockDAGInfo(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
// Obtain a snapshot of the current best known DAG state. We'll
|
||||
// populate the response to this call primarily from this snapshot.
|
||||
params := s.cfg.DAGParams
|
||||
dag := s.cfg.DAG
|
||||
params := s.dag.Params
|
||||
dag := s.dag
|
||||
|
||||
dagInfo := &rpcmodel.GetBlockDAGInfoResult{
|
||||
dagInfo := &model.GetBlockDAGInfoResult{
|
||||
DAG: params.Name,
|
||||
Blocks: dag.BlockCount(),
|
||||
Headers: dag.BlockCount(),
|
||||
@ -25,7 +25,7 @@ func handleGetBlockDAGInfo(s *Server, cmd interface{}, closeChan <-chan struct{}
|
||||
Difficulty: getDifficultyRatio(dag.CurrentBits(), params),
|
||||
MedianTime: dag.CalcPastMedianTime().UnixMilliseconds(),
|
||||
Pruned: false,
|
||||
Bip9SoftForks: make(map[string]*rpcmodel.Bip9SoftForkDescription),
|
||||
Bip9SoftForks: make(map[string]*model.Bip9SoftForkDescription),
|
||||
}
|
||||
|
||||
// Finally, query the BIP0009 version bits state for all currently
|
||||
@ -39,8 +39,8 @@ func handleGetBlockDAGInfo(s *Server, cmd interface{}, closeChan <-chan struct{}
|
||||
forkName = "dummy"
|
||||
|
||||
default:
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCInternal.Code,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInternal.Code,
|
||||
Message: fmt.Sprintf("Unknown deployment %d "+
|
||||
"detected", deployment),
|
||||
}
|
||||
@ -59,8 +59,8 @@ func handleGetBlockDAGInfo(s *Server, cmd interface{}, closeChan <-chan struct{}
|
||||
// non-nil error is returned.
|
||||
statusString, err := softForkStatus(deploymentStatus)
|
||||
if err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCInternal.Code,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInternal.Code,
|
||||
Message: fmt.Sprintf("unknown deployment status: %d",
|
||||
deploymentStatus),
|
||||
}
|
||||
@ -68,7 +68,7 @@ func handleGetBlockDAGInfo(s *Server, cmd interface{}, closeChan <-chan struct{}
|
||||
|
||||
// Finally, populate the soft-fork description with all the
|
||||
// information gathered above.
|
||||
dagInfo.Bip9SoftForks[forkName] = &rpcmodel.Bip9SoftForkDescription{
|
||||
dagInfo.Bip9SoftForks[forkName] = &model.Bip9SoftForkDescription{
|
||||
Status: strings.ToLower(statusString),
|
||||
Bit: deploymentDetails.BitNumber,
|
||||
StartTime: int64(deploymentDetails.StartTime),
|
@ -4,24 +4,24 @@ import (
|
||||
"bytes"
|
||||
"encoding/hex"
|
||||
"fmt"
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
"strconv"
|
||||
)
|
||||
|
||||
// handleGetBlockHeader implements the getBlockHeader command.
|
||||
func handleGetBlockHeader(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
c := cmd.(*rpcmodel.GetBlockHeaderCmd)
|
||||
c := cmd.(*model.GetBlockHeaderCmd)
|
||||
|
||||
// Fetch the header from DAG.
|
||||
hash, err := daghash.NewHashFromStr(c.Hash)
|
||||
if err != nil {
|
||||
return nil, rpcDecodeHexError(c.Hash)
|
||||
}
|
||||
blockHeader, err := s.cfg.DAG.HeaderByHash(hash)
|
||||
blockHeader, err := s.dag.HeaderByHash(hash)
|
||||
if err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCBlockNotFound,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCBlockNotFound,
|
||||
Message: "Block not found",
|
||||
}
|
||||
}
|
||||
@ -41,27 +41,27 @@ func handleGetBlockHeader(s *Server, cmd interface{}, closeChan <-chan struct{})
|
||||
// The verbose flag is set, so generate the JSON object and return it.
|
||||
|
||||
// Get the hashes for the next blocks unless there are none.
|
||||
childHashes, err := s.cfg.DAG.ChildHashesByHash(hash)
|
||||
childHashes, err := s.dag.ChildHashesByHash(hash)
|
||||
if err != nil {
|
||||
context := "No next block"
|
||||
return nil, internalRPCError(err.Error(), context)
|
||||
}
|
||||
childHashStrings := daghash.Strings(childHashes)
|
||||
|
||||
blockConfirmations, err := s.cfg.DAG.BlockConfirmationsByHash(hash)
|
||||
blockConfirmations, err := s.dag.BlockConfirmationsByHash(hash)
|
||||
if err != nil {
|
||||
context := "Could not get block confirmations"
|
||||
return nil, internalRPCError(err.Error(), context)
|
||||
}
|
||||
|
||||
selectedParentHash, err := s.cfg.DAG.SelectedParentHash(hash)
|
||||
selectedParentHash, err := s.dag.SelectedParentHash(hash)
|
||||
if err != nil {
|
||||
context := "Could not get block selected parent"
|
||||
return nil, internalRPCError(err.Error(), context)
|
||||
}
|
||||
|
||||
params := s.cfg.DAGParams
|
||||
blockHeaderReply := rpcmodel.GetBlockHeaderVerboseResult{
|
||||
params := s.dag.Params
|
||||
blockHeaderReply := model.GetBlockHeaderVerboseResult{
|
||||
Hash: c.Hash,
|
||||
Confirmations: blockConfirmations,
|
||||
Version: blockHeader.Version,
|
@ -13,7 +13,7 @@ import (
|
||||
|
||||
"github.com/kaspanet/kaspad/blockdag"
|
||||
"github.com/kaspanet/kaspad/mining"
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/txscript"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
@ -86,7 +86,7 @@ func builderScript(builder *txscript.ScriptBuilder) []byte {
|
||||
|
||||
// handleGetBlockTemplate implements the getBlockTemplate command.
|
||||
func handleGetBlockTemplate(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
c := cmd.(*rpcmodel.GetBlockTemplateCmd)
|
||||
c := cmd.(*model.GetBlockTemplateCmd)
|
||||
request := c.Request
|
||||
|
||||
// Set the default mode and override it if supplied.
|
||||
@ -102,8 +102,8 @@ func handleGetBlockTemplate(s *Server, cmd interface{}, closeChan <-chan struct{
|
||||
return handleGetBlockTemplateProposal(s, request)
|
||||
}
|
||||
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCInvalidParameter,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInvalidParameter,
|
||||
Message: "Invalid mode",
|
||||
}
|
||||
}
|
||||
@ -112,21 +112,21 @@ func handleGetBlockTemplate(s *Server, cmd interface{}, closeChan <-chan struct{
|
||||
// deals with generating and returning block templates to the caller. It
|
||||
// handles both long poll requests as specified by BIP 0022 as well as regular
|
||||
// requests.
|
||||
func handleGetBlockTemplateRequest(s *Server, request *rpcmodel.TemplateRequest, closeChan <-chan struct{}) (interface{}, error) {
|
||||
func handleGetBlockTemplateRequest(s *Server, request *model.TemplateRequest, closeChan <-chan struct{}) (interface{}, error) {
|
||||
// Return an error if there are no peers connected since there is no
|
||||
// way to relay a found block or receive transactions to work on.
|
||||
// However, allow this state when running in the regression test or
|
||||
// simulation test mode.
|
||||
if !(s.appCfg.RegressionTest || s.appCfg.Simnet) &&
|
||||
s.cfg.ConnMgr.ConnectedCount() == 0 {
|
||||
if !(s.cfg.RegressionTest || s.cfg.Simnet) &&
|
||||
s.connectionManager.ConnectionCount() == 0 {
|
||||
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCClientNotConnected,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCClientNotConnected,
|
||||
Message: "Kaspa is not connected",
|
||||
}
|
||||
}
|
||||
|
||||
payAddr, err := util.DecodeAddress(request.PayAddress, s.cfg.DAGParams.Prefix)
|
||||
payAddr, err := util.DecodeAddress(request.PayAddress, s.dag.Params.Prefix)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -209,7 +209,7 @@ func handleGetBlockTemplateLongPoll(s *Server, longPollID string, payAddr util.A
|
||||
// template identified by the provided long poll ID is stale or
|
||||
// invalid. Otherwise, it returns a channel that will notify
|
||||
// when there's a more current template.
|
||||
func blockTemplateOrLongPollChan(s *Server, longPollID string, payAddr util.Address) (*rpcmodel.GetBlockTemplateResult, chan struct{}, error) {
|
||||
func blockTemplateOrLongPollChan(s *Server, longPollID string, payAddr util.Address) (*model.GetBlockTemplateResult, chan struct{}, error) {
|
||||
state := s.gbtWorkState
|
||||
|
||||
state.Lock()
|
||||
@ -262,11 +262,11 @@ func blockTemplateOrLongPollChan(s *Server, longPollID string, payAddr util.Addr
|
||||
|
||||
// handleGetBlockTemplateProposal is a helper for handleGetBlockTemplate which
|
||||
// deals with block proposals.
|
||||
func handleGetBlockTemplateProposal(s *Server, request *rpcmodel.TemplateRequest) (interface{}, error) {
|
||||
func handleGetBlockTemplateProposal(s *Server, request *model.TemplateRequest) (interface{}, error) {
|
||||
hexData := request.Data
|
||||
if hexData == "" {
|
||||
return false, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCType,
|
||||
return false, &model.RPCError{
|
||||
Code: model.ErrRPCType,
|
||||
Message: fmt.Sprintf("Data must contain the " +
|
||||
"hex-encoded serialized block that is being " +
|
||||
"proposed"),
|
||||
@ -279,34 +279,34 @@ func handleGetBlockTemplateProposal(s *Server, request *rpcmodel.TemplateRequest
|
||||
}
|
||||
dataBytes, err := hex.DecodeString(hexData)
|
||||
if err != nil {
|
||||
return false, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCDeserialization,
|
||||
return false, &model.RPCError{
|
||||
Code: model.ErrRPCDeserialization,
|
||||
Message: fmt.Sprintf("Data must be "+
|
||||
"hexadecimal string (not %q)", hexData),
|
||||
}
|
||||
}
|
||||
var msgBlock wire.MsgBlock
|
||||
if err := msgBlock.Deserialize(bytes.NewReader(dataBytes)); err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCDeserialization,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCDeserialization,
|
||||
Message: "Block decode failed: " + err.Error(),
|
||||
}
|
||||
}
|
||||
block := util.NewBlock(&msgBlock)
|
||||
|
||||
// Ensure the block is building from the expected parent blocks.
|
||||
expectedParentHashes := s.cfg.DAG.TipHashes()
|
||||
expectedParentHashes := s.dag.TipHashes()
|
||||
parentHashes := block.MsgBlock().Header.ParentHashes
|
||||
if !daghash.AreEqual(expectedParentHashes, parentHashes) {
|
||||
return "bad-parentblk", nil
|
||||
}
|
||||
|
||||
if err := s.cfg.DAG.CheckConnectBlockTemplate(block); err != nil {
|
||||
if err := s.dag.CheckConnectBlockTemplate(block); err != nil {
|
||||
if !errors.As(err, &blockdag.RuleError{}) {
|
||||
errStr := fmt.Sprintf("Failed to process block proposal: %s", err)
|
||||
log.Error(errStr)
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCVerify,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCVerify,
|
||||
Message: errStr,
|
||||
}
|
||||
}
|
||||
@ -524,7 +524,7 @@ func (state *gbtWorkState) templateUpdateChan(tipHashes []*daghash.Hash, lastGen
|
||||
//
|
||||
// This function MUST be called with the state locked.
|
||||
func (state *gbtWorkState) updateBlockTemplate(s *Server, payAddr util.Address) error {
|
||||
generator := s.cfg.Generator
|
||||
generator := s.blockTemplateGenerator
|
||||
lastTxUpdate := generator.TxSource().LastUpdated()
|
||||
if lastTxUpdate.IsZero() {
|
||||
lastTxUpdate = mstime.Now()
|
||||
@ -536,7 +536,7 @@ func (state *gbtWorkState) updateBlockTemplate(s *Server, payAddr util.Address)
|
||||
// generated.
|
||||
var msgBlock *wire.MsgBlock
|
||||
var targetDifficulty string
|
||||
tipHashes := s.cfg.DAG.TipHashes()
|
||||
tipHashes := s.dag.TipHashes()
|
||||
template := state.template
|
||||
if template == nil || state.tipHashes == nil ||
|
||||
!daghash.AreEqual(state.tipHashes, tipHashes) ||
|
||||
@ -575,7 +575,7 @@ func (state *gbtWorkState) updateBlockTemplate(s *Server, payAddr util.Address)
|
||||
// Get the minimum allowed timestamp for the block based on the
|
||||
// median timestamp of the last several blocks per the DAG
|
||||
// consensus rules.
|
||||
minTimestamp := s.cfg.DAG.NextBlockMinimumTime()
|
||||
minTimestamp := s.dag.NextBlockMinimumTime()
|
||||
|
||||
// Update work state to ensure another block template isn't
|
||||
// generated until needed.
|
||||
@ -621,12 +621,12 @@ func (state *gbtWorkState) updateBlockTemplate(s *Server, payAddr util.Address)
|
||||
}
|
||||
|
||||
// blockTemplateResult returns the current block template associated with the
|
||||
// state as a rpcmodel.GetBlockTemplateResult that is ready to be encoded to JSON
|
||||
// state as a model.GetBlockTemplateResult that is ready to be encoded to JSON
|
||||
// and returned to the caller.
|
||||
//
|
||||
// This function MUST be called with the state locked.
|
||||
func (state *gbtWorkState) blockTemplateResult(s *Server) (*rpcmodel.GetBlockTemplateResult, error) {
|
||||
dag := s.cfg.DAG
|
||||
func (state *gbtWorkState) blockTemplateResult(s *Server) (*model.GetBlockTemplateResult, error) {
|
||||
dag := s.dag
|
||||
// Ensure the timestamps are still in valid range for the template.
|
||||
// This should really only ever happen if the local clock is changed
|
||||
// after the template is generated, but it's important to avoid serving
|
||||
@ -637,8 +637,8 @@ func (state *gbtWorkState) blockTemplateResult(s *Server) (*rpcmodel.GetBlockTem
|
||||
adjustedTime := dag.Now()
|
||||
maxTime := adjustedTime.Add(time.Millisecond * time.Duration(dag.TimestampDeviationTolerance))
|
||||
if header.Timestamp.After(maxTime) {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCOutOfRange,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCOutOfRange,
|
||||
Message: fmt.Sprintf("The template time is after the "+
|
||||
"maximum allowed time for a block - template "+
|
||||
"time %s, maximum time %s", adjustedTime,
|
||||
@ -650,7 +650,7 @@ func (state *gbtWorkState) blockTemplateResult(s *Server) (*rpcmodel.GetBlockTem
|
||||
// transaction. The result does not include the coinbase, so notice
|
||||
// the adjustments to the various lengths and indices.
|
||||
numTx := len(msgBlock.Transactions)
|
||||
transactions := make([]rpcmodel.GetBlockTemplateResultTx, 0, numTx-1)
|
||||
transactions := make([]model.GetBlockTemplateResultTx, 0, numTx-1)
|
||||
txIndex := make(map[daghash.TxID]int64, numTx)
|
||||
for i, tx := range msgBlock.Transactions {
|
||||
txID := tx.TxID()
|
||||
@ -680,7 +680,7 @@ func (state *gbtWorkState) blockTemplateResult(s *Server) (*rpcmodel.GetBlockTem
|
||||
return nil, internalRPCError(err.Error(), context)
|
||||
}
|
||||
|
||||
resultTx := rpcmodel.GetBlockTemplateResultTx{
|
||||
resultTx := model.GetBlockTemplateResultTx{
|
||||
Data: hex.EncodeToString(txBuf.Bytes()),
|
||||
ID: txID.String(),
|
||||
Depends: depends,
|
||||
@ -704,9 +704,9 @@ func (state *gbtWorkState) blockTemplateResult(s *Server) (*rpcmodel.GetBlockTem
|
||||
// This is not a straight-up error because the choice of whether
|
||||
// to mine or not is the responsibility of the miner rather
|
||||
// than the node's.
|
||||
isSynced := s.cfg.SyncMgr.IsSynced()
|
||||
isSynced := s.blockTemplateGenerator.IsSynced()
|
||||
|
||||
reply := rpcmodel.GetBlockTemplateResult{
|
||||
reply := model.GetBlockTemplateResult{
|
||||
Bits: strconv.FormatInt(int64(header.Bits), 16),
|
||||
CurTime: header.Timestamp.UnixMilliseconds(),
|
||||
Height: template.Height,
|
@ -2,7 +2,7 @@ package rpc
|
||||
|
||||
import (
|
||||
"encoding/hex"
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
)
|
||||
@ -14,7 +14,7 @@ const (
|
||||
)
|
||||
|
||||
func handleGetBlocks(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
c := cmd.(*rpcmodel.GetBlocksCmd)
|
||||
c := cmd.(*model.GetBlocksCmd)
|
||||
var lowHash *daghash.Hash
|
||||
if c.LowHash != nil {
|
||||
lowHash = &daghash.Hash{}
|
||||
@ -24,19 +24,19 @@ func handleGetBlocks(s *Server, cmd interface{}, closeChan <-chan struct{}) (int
|
||||
}
|
||||
}
|
||||
|
||||
s.cfg.DAG.RLock()
|
||||
defer s.cfg.DAG.RUnlock()
|
||||
s.dag.RLock()
|
||||
defer s.dag.RUnlock()
|
||||
|
||||
// If lowHash is not in the DAG, there's nothing to do; return an error.
|
||||
if lowHash != nil && !s.cfg.DAG.IsKnownBlock(lowHash) {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCBlockNotFound,
|
||||
if lowHash != nil && !s.dag.IsKnownBlock(lowHash) {
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCBlockNotFound,
|
||||
Message: "Block not found",
|
||||
}
|
||||
}
|
||||
|
||||
// Retrieve the block hashes.
|
||||
blockHashes, err := s.cfg.DAG.BlockHashesFrom(lowHash, maxBlocksInGetBlocksResult)
|
||||
blockHashes, err := s.dag.BlockHashesFrom(lowHash, maxBlocksInGetBlocksResult)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -47,7 +47,7 @@ func handleGetBlocks(s *Server, cmd interface{}, closeChan <-chan struct{}) (int
|
||||
hashes[i] = blockHash.String()
|
||||
}
|
||||
|
||||
result := &rpcmodel.GetBlocksResult{
|
||||
result := &model.GetBlocksResult{
|
||||
Hashes: hashes,
|
||||
RawBlocks: nil,
|
||||
VerboseBlocks: nil,
|
||||
@ -77,7 +77,7 @@ func handleGetBlocks(s *Server, cmd interface{}, closeChan <-chan struct{}) (int
|
||||
func hashesToBlockBytes(s *Server, hashes []*daghash.Hash) ([][]byte, error) {
|
||||
blocks := make([][]byte, len(hashes))
|
||||
for i, hash := range hashes {
|
||||
block, err := s.cfg.DAG.BlockByHash(hash)
|
||||
block, err := s.dag.BlockByHash(hash)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -98,8 +98,8 @@ func blockBytesToStrings(blockBytesSlice [][]byte) []string {
|
||||
return rawBlocks
|
||||
}
|
||||
|
||||
func blockBytesToBlockVerboseResults(s *Server, blockBytesSlice [][]byte) ([]rpcmodel.GetBlockVerboseResult, error) {
|
||||
verboseBlocks := make([]rpcmodel.GetBlockVerboseResult, len(blockBytesSlice))
|
||||
func blockBytesToBlockVerboseResults(s *Server, blockBytesSlice [][]byte) ([]model.GetBlockVerboseResult, error) {
|
||||
verboseBlocks := make([]model.GetBlockVerboseResult, len(blockBytesSlice))
|
||||
for i, blockBytes := range blockBytesSlice {
|
||||
block, err := util.NewBlockFromBytes(blockBytes)
|
||||
if err != nil {
|
@ -2,7 +2,7 @@ package rpc
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
)
|
||||
|
||||
@ -14,16 +14,16 @@ const (
|
||||
|
||||
// handleGetChainFromBlock implements the getChainFromBlock command.
|
||||
func handleGetChainFromBlock(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
if s.cfg.AcceptanceIndex == nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCNoAcceptanceIndex,
|
||||
if s.acceptanceIndex == nil {
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCNoAcceptanceIndex,
|
||||
Message: "The acceptance index must be " +
|
||||
"enabled to get the selected parent chain " +
|
||||
"(specify --acceptanceindex)",
|
||||
}
|
||||
}
|
||||
|
||||
c := cmd.(*rpcmodel.GetChainFromBlockCmd)
|
||||
c := cmd.(*model.GetChainFromBlockCmd)
|
||||
var startHash *daghash.Hash
|
||||
if c.StartHash != nil {
|
||||
startHash = &daghash.Hash{}
|
||||
@ -33,20 +33,20 @@ func handleGetChainFromBlock(s *Server, cmd interface{}, closeChan <-chan struct
|
||||
}
|
||||
}
|
||||
|
||||
s.cfg.DAG.RLock()
|
||||
defer s.cfg.DAG.RUnlock()
|
||||
s.dag.RLock()
|
||||
defer s.dag.RUnlock()
|
||||
|
||||
// If startHash is not in the selected parent chain, there's nothing
|
||||
// to do; return an error.
|
||||
if startHash != nil && !s.cfg.DAG.IsInDAG(startHash) {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCBlockNotFound,
|
||||
if startHash != nil && !s.dag.IsInDAG(startHash) {
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCBlockNotFound,
|
||||
Message: "Block not found in the DAG",
|
||||
}
|
||||
}
|
||||
|
||||
// Retrieve the selected parent chain.
|
||||
removedChainHashes, addedChainHashes, err := s.cfg.DAG.SelectedParentChain(startHash)
|
||||
removedChainHashes, addedChainHashes, err := s.dag.SelectedParentChain(startHash)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
@ -59,8 +59,8 @@ func handleGetChainFromBlock(s *Server, cmd interface{}, closeChan <-chan struct
|
||||
// Collect addedChainBlocks.
|
||||
addedChainBlocks, err := collectChainBlocks(s, addedChainHashes)
|
||||
if err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCInternal.Code,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInternal.Code,
|
||||
Message: fmt.Sprintf("could not collect chain blocks: %s", err),
|
||||
}
|
||||
}
|
||||
@ -71,7 +71,7 @@ func handleGetChainFromBlock(s *Server, cmd interface{}, closeChan <-chan struct
|
||||
removedHashes[i] = hash.String()
|
||||
}
|
||||
|
||||
result := &rpcmodel.GetChainFromBlockResult{
|
||||
result := &model.GetChainFromBlockResult{
|
||||
RemovedChainBlockHashes: removedHashes,
|
||||
AddedChainBlocks: addedChainBlocks,
|
||||
Blocks: nil,
|
30
rpc/handle_get_connected_peer_info.go
Normal file
30
rpc/handle_get_connected_peer_info.go
Normal file
@ -0,0 +1,30 @@
|
||||
package rpc
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
)
|
||||
|
||||
// handleGetConnectedPeerInfo implements the getConnectedPeerInfo command.
|
||||
func handleGetConnectedPeerInfo(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
peers := s.protocolManager.Peers()
|
||||
infos := make([]*model.GetConnectedPeerInfoResult, 0, len(peers))
|
||||
for _, peer := range peers {
|
||||
info := &model.GetConnectedPeerInfoResult{
|
||||
ID: peer.ID().String(),
|
||||
Address: peer.Address(),
|
||||
LastPingDuration: peer.LastPingDuration().Milliseconds(),
|
||||
SelectedTipHash: peer.SelectedTipHash().String(),
|
||||
IsSyncNode: peer == s.protocolManager.IBDPeer(),
|
||||
|
||||
// TODO(libp2p): populate the following with real values
|
||||
IsInbound: false,
|
||||
BanScore: 0,
|
||||
TimeOffset: 0,
|
||||
UserAgent: "",
|
||||
ProtocolVersion: 0,
|
||||
TimeConnected: 0,
|
||||
}
|
||||
infos = append(infos, info)
|
||||
}
|
||||
return infos, nil
|
||||
}
|
@ -2,5 +2,5 @@ package rpc
|
||||
|
||||
// handleGetConnectionCount implements the getConnectionCount command.
|
||||
func handleGetConnectionCount(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
return s.cfg.ConnMgr.ConnectedCount(), nil
|
||||
return s.connectionManager.ConnectionCount(), nil
|
||||
}
|
@ -2,5 +2,5 @@ package rpc
|
||||
|
||||
// handleGetCurrentNet implements the getCurrentNet command.
|
||||
func handleGetCurrentNet(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
return s.cfg.DAGParams.Net, nil
|
||||
return s.dag.Params.Net, nil
|
||||
}
|
@ -2,5 +2,5 @@ package rpc
|
||||
|
||||
// handleGetDifficulty implements the getDifficulty command.
|
||||
func handleGetDifficulty(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
return getDifficultyRatio(s.cfg.DAG.SelectedTipHeader().Bits, s.cfg.DAGParams), nil
|
||||
return getDifficultyRatio(s.dag.SelectedTipHeader().Bits, s.dag.Params), nil
|
||||
}
|
@ -3,7 +3,7 @@ package rpc
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/hex"
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
)
|
||||
|
||||
@ -11,7 +11,7 @@ const getHeadersMaxHeaders = 2000
|
||||
|
||||
// handleGetHeaders implements the getHeaders command.
|
||||
func handleGetHeaders(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
c := cmd.(*rpcmodel.GetHeadersCmd)
|
||||
c := cmd.(*model.GetHeadersCmd)
|
||||
|
||||
lowHash := &daghash.ZeroHash
|
||||
if c.LowHash != "" {
|
||||
@ -27,10 +27,10 @@ func handleGetHeaders(s *Server, cmd interface{}, closeChan <-chan struct{}) (in
|
||||
return nil, rpcDecodeHexError(c.HighHash)
|
||||
}
|
||||
}
|
||||
headers, err := s.cfg.SyncMgr.AntiPastHeadersBetween(lowHash, highHash, getHeadersMaxHeaders)
|
||||
headers, err := s.dag.AntiPastHeadersBetween(lowHash, highHash, getHeadersMaxHeaders)
|
||||
if err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCMisc,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCMisc,
|
||||
Message: err.Error(),
|
||||
}
|
||||
}
|
24
rpc/handle_get_info.go
Normal file
24
rpc/handle_get_info.go
Normal file
@ -0,0 +1,24 @@
|
||||
package rpc
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/version"
|
||||
)
|
||||
|
||||
// handleGetInfo implements the getInfo command. We only return the fields
|
||||
// that are not related to wallet functionality.
|
||||
func handleGetInfo(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
ret := &model.InfoDAGResult{
|
||||
Version: version.Version(),
|
||||
ProtocolVersion: int32(maxProtocolVersion),
|
||||
Blocks: s.dag.BlockCount(),
|
||||
Connections: int32(s.connectionManager.ConnectionCount()),
|
||||
Proxy: s.cfg.Proxy,
|
||||
Difficulty: getDifficultyRatio(s.dag.CurrentBits(), s.dag.Params),
|
||||
Testnet: s.cfg.Testnet,
|
||||
Devnet: s.cfg.Devnet,
|
||||
RelayFee: s.cfg.MinRelayTxFee.ToKAS(),
|
||||
}
|
||||
|
||||
return ret, nil
|
||||
}
|
@ -1,30 +1,30 @@
|
||||
package rpc
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
)
|
||||
|
||||
func handleGetMempoolEntry(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
c := cmd.(*rpcmodel.GetMempoolEntryCmd)
|
||||
c := cmd.(*model.GetMempoolEntryCmd)
|
||||
txID, err := daghash.NewTxIDFromStr(c.TxID)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
txDesc, err := s.cfg.TxMemPool.FetchTxDesc(txID)
|
||||
txDesc, err := s.txMempool.FetchTxDesc(txID)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
tx := txDesc.Tx
|
||||
rawTx, err := createTxRawResult(s.cfg.DAGParams, tx.MsgTx(), tx.ID().String(),
|
||||
rawTx, err := createTxRawResult(s.dag.Params, tx.MsgTx(), tx.ID().String(),
|
||||
nil, "", nil, true)
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
return &rpcmodel.GetMempoolEntryResult{
|
||||
return &model.GetMempoolEntryResult{
|
||||
Fee: txDesc.Fee,
|
||||
Time: txDesc.Added.UnixMilliseconds(),
|
||||
RawTx: *rawTx,
|
@ -1,17 +1,17 @@
|
||||
package rpc
|
||||
|
||||
import "github.com/kaspanet/kaspad/rpcmodel"
|
||||
import "github.com/kaspanet/kaspad/rpc/model"
|
||||
|
||||
// handleGetMempoolInfo implements the getMempoolInfo command.
|
||||
func handleGetMempoolInfo(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
mempoolTxns := s.cfg.TxMemPool.TxDescs()
|
||||
mempoolTxns := s.txMempool.TxDescs()
|
||||
|
||||
var numBytes int64
|
||||
for _, txD := range mempoolTxns {
|
||||
numBytes += int64(txD.Tx.MsgTx().SerializeSize())
|
||||
}
|
||||
|
||||
ret := &rpcmodel.GetMempoolInfoResult{
|
||||
ret := &model.GetMempoolInfoResult{
|
||||
Size: int64(len(mempoolTxns)),
|
||||
Bytes: numBytes,
|
||||
}
|
@ -1,16 +1,16 @@
|
||||
package rpc
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"time"
|
||||
)
|
||||
|
||||
// handleGetNetTotals implements the getNetTotals command.
|
||||
func handleGetNetTotals(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
totalBytesRecv, totalBytesSent := s.cfg.ConnMgr.NetTotals()
|
||||
reply := &rpcmodel.GetNetTotalsResult{
|
||||
TotalBytesRecv: totalBytesRecv,
|
||||
TotalBytesSent: totalBytesSent,
|
||||
// TODO(libp2p): fill this up with real values
|
||||
reply := &model.GetNetTotalsResult{
|
||||
TotalBytesRecv: 0,
|
||||
TotalBytesSent: 0,
|
||||
TimeMillis: time.Now().UTC().UnixNano() / int64(time.Millisecond),
|
||||
}
|
||||
return reply, nil
|
@ -1,26 +1,26 @@
|
||||
package rpc
|
||||
|
||||
import "github.com/kaspanet/kaspad/rpcmodel"
|
||||
import "github.com/kaspanet/kaspad/rpc/model"
|
||||
|
||||
// handleGetPeerAddresses handles getPeerAddresses commands.
|
||||
func handleGetPeerAddresses(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
peersState, err := s.cfg.addressManager.PeersStateForSerialization()
|
||||
peersState, err := s.addressManager.PeersStateForSerialization()
|
||||
if err != nil {
|
||||
return nil, err
|
||||
}
|
||||
|
||||
rpcPeersState := rpcmodel.GetPeerAddressesResult{
|
||||
rpcPeersState := model.GetPeerAddressesResult{
|
||||
Version: peersState.Version,
|
||||
Key: peersState.Key,
|
||||
Addresses: make([]*rpcmodel.GetPeerAddressesKnownAddressResult, len(peersState.Addresses)),
|
||||
NewBuckets: make(map[string]*rpcmodel.GetPeerAddressesNewBucketResult),
|
||||
NewBucketFullNodes: rpcmodel.GetPeerAddressesNewBucketResult{},
|
||||
TriedBuckets: make(map[string]*rpcmodel.GetPeerAddressesTriedBucketResult),
|
||||
TriedBucketFullNodes: rpcmodel.GetPeerAddressesTriedBucketResult{},
|
||||
Addresses: make([]*model.GetPeerAddressesKnownAddressResult, len(peersState.Addresses)),
|
||||
NewBuckets: make(map[string]*model.GetPeerAddressesNewBucketResult),
|
||||
NewBucketFullNodes: model.GetPeerAddressesNewBucketResult{},
|
||||
TriedBuckets: make(map[string]*model.GetPeerAddressesTriedBucketResult),
|
||||
TriedBucketFullNodes: model.GetPeerAddressesTriedBucketResult{},
|
||||
}
|
||||
|
||||
for i, addr := range peersState.Addresses {
|
||||
rpcPeersState.Addresses[i] = &rpcmodel.GetPeerAddressesKnownAddressResult{
|
||||
rpcPeersState.Addresses[i] = &model.GetPeerAddressesKnownAddressResult{
|
||||
Addr: addr.Addr,
|
||||
Src: addr.Src,
|
||||
SubnetworkID: addr.SubnetworkID,
|
||||
@ -32,7 +32,7 @@ func handleGetPeerAddresses(s *Server, cmd interface{}, closeChan <-chan struct{
|
||||
}
|
||||
|
||||
for subnetworkID, bucket := range peersState.NewBuckets {
|
||||
rpcPeersState.NewBuckets[subnetworkID] = &rpcmodel.GetPeerAddressesNewBucketResult{}
|
||||
rpcPeersState.NewBuckets[subnetworkID] = &model.GetPeerAddressesNewBucketResult{}
|
||||
for i, addr := range bucket {
|
||||
rpcPeersState.NewBuckets[subnetworkID][i] = addr
|
||||
}
|
||||
@ -43,7 +43,7 @@ func handleGetPeerAddresses(s *Server, cmd interface{}, closeChan <-chan struct{
|
||||
}
|
||||
|
||||
for subnetworkID, bucket := range peersState.TriedBuckets {
|
||||
rpcPeersState.TriedBuckets[subnetworkID] = &rpcmodel.GetPeerAddressesTriedBucketResult{}
|
||||
rpcPeersState.TriedBuckets[subnetworkID] = &model.GetPeerAddressesTriedBucketResult{}
|
||||
for i, addr := range bucket {
|
||||
rpcPeersState.TriedBuckets[subnetworkID][i] = addr
|
||||
}
|
58
rpc/handle_get_raw_mempool.go
Normal file
58
rpc/handle_get_raw_mempool.go
Normal file
@ -0,0 +1,58 @@
|
||||
package rpc
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
)
|
||||
|
||||
// handleGetRawMempool implements the getRawMempool command.
|
||||
func handleGetRawMempool(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
c := cmd.(*model.GetRawMempoolCmd)
|
||||
mp := s.txMempool
|
||||
|
||||
if c.Verbose != nil && *c.Verbose {
|
||||
return rawMempoolVerbose(s), nil
|
||||
}
|
||||
|
||||
// The response is simply an array of the transaction hashes if the
|
||||
// verbose flag is not set.
|
||||
descs := mp.TxDescs()
|
||||
hashStrings := make([]string, len(descs))
|
||||
for i := range hashStrings {
|
||||
hashStrings[i] = descs[i].Tx.ID().String()
|
||||
}
|
||||
|
||||
return hashStrings, nil
|
||||
}
|
||||
|
||||
// rawMempoolVerbose returns all of the entries in the mempool as a fully
|
||||
// populated jsonrpc result.
|
||||
func rawMempoolVerbose(s *Server) map[string]*model.GetRawMempoolVerboseResult {
|
||||
descs := s.txMempool.TxDescs()
|
||||
result := make(map[string]*model.GetRawMempoolVerboseResult, len(descs))
|
||||
|
||||
for _, desc := range descs {
|
||||
// Calculate the current priority based on the inputs to
|
||||
// the transaction. Use zero if one or more of the
|
||||
// input transactions can't be found for some reason.
|
||||
tx := desc.Tx
|
||||
|
||||
mpd := &model.GetRawMempoolVerboseResult{
|
||||
Size: int32(tx.MsgTx().SerializeSize()),
|
||||
Fee: util.Amount(desc.Fee).ToKAS(),
|
||||
Time: desc.Added.UnixMilliseconds(),
|
||||
Depends: make([]string, 0),
|
||||
}
|
||||
for _, txIn := range tx.MsgTx().TxIn {
|
||||
txID := &txIn.PreviousOutpoint.TxID
|
||||
if s.txMempool.HaveTransaction(txID) {
|
||||
mpd.Depends = append(mpd.Depends,
|
||||
txID.String())
|
||||
}
|
||||
}
|
||||
|
||||
result[tx.ID().String()] = mpd
|
||||
}
|
||||
|
||||
return result
|
||||
}
|
@ -2,25 +2,25 @@ package rpc
|
||||
|
||||
import (
|
||||
"encoding/hex"
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
)
|
||||
|
||||
// handleGetSelectedTip implements the getSelectedTip command.
|
||||
func handleGetSelectedTip(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
getSelectedTipCmd := cmd.(*rpcmodel.GetSelectedTipCmd)
|
||||
selectedTipHash := s.cfg.DAG.SelectedTipHash()
|
||||
getSelectedTipCmd := cmd.(*model.GetSelectedTipCmd)
|
||||
selectedTipHash := s.dag.SelectedTipHash()
|
||||
|
||||
block, err := s.cfg.DAG.BlockByHash(selectedTipHash)
|
||||
block, err := s.dag.BlockByHash(selectedTipHash)
|
||||
if err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCBlockNotFound,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCBlockNotFound,
|
||||
Message: "Block not found",
|
||||
}
|
||||
}
|
||||
blockBytes, err := block.Bytes()
|
||||
if err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCBlockInvalid,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCBlockInvalid,
|
||||
Message: "Cannot serialize block",
|
||||
}
|
||||
}
|
@ -2,5 +2,5 @@ package rpc
|
||||
|
||||
// handleGetSelectedTipHash implements the getSelectedTipHash command.
|
||||
func handleGetSelectedTipHash(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
return s.cfg.DAG.SelectedTipHash().String(), nil
|
||||
return s.dag.SelectedTipHash().String(), nil
|
||||
}
|
@ -1,13 +1,13 @@
|
||||
package rpc
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util/subnetworkid"
|
||||
)
|
||||
|
||||
// handleGetSubnetwork handles the getSubnetwork command.
|
||||
func handleGetSubnetwork(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
c := cmd.(*rpcmodel.GetSubnetworkCmd)
|
||||
c := cmd.(*model.GetSubnetworkCmd)
|
||||
|
||||
subnetworkID, err := subnetworkid.NewFromStr(c.SubnetworkID)
|
||||
if err != nil {
|
||||
@ -17,17 +17,17 @@ func handleGetSubnetwork(s *Server, cmd interface{}, closeChan <-chan struct{})
|
||||
var gasLimit *uint64
|
||||
if !subnetworkID.IsEqual(subnetworkid.SubnetworkIDNative) &&
|
||||
!subnetworkID.IsBuiltIn() {
|
||||
limit, err := s.cfg.DAG.GasLimit(subnetworkID)
|
||||
limit, err := s.dag.GasLimit(subnetworkID)
|
||||
if err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCSubnetworkNotFound,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCSubnetworkNotFound,
|
||||
Message: "Subnetwork not found.",
|
||||
}
|
||||
}
|
||||
gasLimit = &limit
|
||||
}
|
||||
|
||||
subnetworkReply := &rpcmodel.GetSubnetworkResult{
|
||||
subnetworkReply := &model.GetSubnetworkResult{
|
||||
GasLimit: gasLimit,
|
||||
}
|
||||
return subnetworkReply, nil
|
@ -3,7 +3,7 @@ package rpc
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/hex"
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
)
|
||||
|
||||
@ -11,7 +11,7 @@ const getTopHeadersMaxHeaders = getHeadersMaxHeaders
|
||||
|
||||
// handleGetTopHeaders implements the getTopHeaders command.
|
||||
func handleGetTopHeaders(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
c := cmd.(*rpcmodel.GetTopHeadersCmd)
|
||||
c := cmd.(*model.GetTopHeadersCmd)
|
||||
|
||||
var highHash *daghash.Hash
|
||||
if c.HighHash != nil {
|
||||
@ -21,7 +21,7 @@ func handleGetTopHeaders(s *Server, cmd interface{}, closeChan <-chan struct{})
|
||||
return nil, rpcDecodeHexError(*c.HighHash)
|
||||
}
|
||||
}
|
||||
headers, err := s.cfg.DAG.GetTopHeaders(highHash, getTopHeadersMaxHeaders)
|
||||
headers, err := s.dag.GetTopHeaders(highHash, getTopHeadersMaxHeaders)
|
||||
if err != nil {
|
||||
return nil, internalRPCError(err.Error(),
|
||||
"Failed to get top headers")
|
@ -3,7 +3,7 @@ package rpc
|
||||
import (
|
||||
"encoding/hex"
|
||||
"fmt"
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/txscript"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
@ -13,7 +13,7 @@ import (
|
||||
|
||||
// handleGetTxOut handles getTxOut commands.
|
||||
func handleGetTxOut(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
c := cmd.(*rpcmodel.GetTxOutCmd)
|
||||
c := cmd.(*model.GetTxOutCmd)
|
||||
|
||||
// Convert the provided transaction hash hex to a Hash.
|
||||
txID, err := daghash.NewTxIDFromStr(c.TxID)
|
||||
@ -35,16 +35,16 @@ func handleGetTxOut(s *Server, cmd interface{}, closeChan <-chan struct{}) (inte
|
||||
}
|
||||
// TODO: This is racy. It should attempt to fetch it directly and check
|
||||
// the error.
|
||||
if includeMempool && s.cfg.TxMemPool.HaveTransaction(txID) {
|
||||
tx, err := s.cfg.TxMemPool.FetchTransaction(txID)
|
||||
if includeMempool && s.txMempool.HaveTransaction(txID) {
|
||||
tx, err := s.txMempool.FetchTransaction(txID)
|
||||
if err != nil {
|
||||
return nil, rpcNoTxInfoError(txID)
|
||||
}
|
||||
|
||||
mtx := tx.MsgTx()
|
||||
if c.Vout > uint32(len(mtx.TxOut)-1) {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCInvalidTxVout,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInvalidTxVout,
|
||||
Message: "Output index number (vout) does not " +
|
||||
"exist for transaction.",
|
||||
}
|
||||
@ -57,14 +57,14 @@ func handleGetTxOut(s *Server, cmd interface{}, closeChan <-chan struct{}) (inte
|
||||
return nil, internalRPCError(errStr, "")
|
||||
}
|
||||
|
||||
selectedTipHash = s.cfg.DAG.SelectedTipHash().String()
|
||||
selectedTipHash = s.dag.SelectedTipHash().String()
|
||||
value = txOut.Value
|
||||
scriptPubKey = txOut.ScriptPubKey
|
||||
isCoinbase = mtx.IsCoinBase()
|
||||
isInMempool = true
|
||||
} else {
|
||||
out := wire.Outpoint{TxID: *txID, Index: c.Vout}
|
||||
entry, ok := s.cfg.DAG.GetUTXOEntry(out)
|
||||
entry, ok := s.dag.GetUTXOEntry(out)
|
||||
if !ok {
|
||||
return nil, rpcNoTxInfoError(txID)
|
||||
}
|
||||
@ -78,7 +78,7 @@ func handleGetTxOut(s *Server, cmd interface{}, closeChan <-chan struct{}) (inte
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
utxoConfirmations, ok := s.cfg.DAG.UTXOConfirmations(&out)
|
||||
utxoConfirmations, ok := s.dag.UTXOConfirmations(&out)
|
||||
if !ok {
|
||||
errStr := fmt.Sprintf("Cannot get confirmations for tx id %s, index %d",
|
||||
out.TxID, out.Index)
|
||||
@ -86,7 +86,7 @@ func handleGetTxOut(s *Server, cmd interface{}, closeChan <-chan struct{}) (inte
|
||||
}
|
||||
confirmations = &utxoConfirmations
|
||||
|
||||
selectedTipHash = s.cfg.DAG.SelectedTipHash().String()
|
||||
selectedTipHash = s.dag.SelectedTipHash().String()
|
||||
value = entry.Amount()
|
||||
scriptPubKey = entry.ScriptPubKey()
|
||||
isCoinbase = entry.IsCoinbase()
|
||||
@ -101,18 +101,18 @@ func handleGetTxOut(s *Server, cmd interface{}, closeChan <-chan struct{}) (inte
|
||||
// Ignore the error here since an error means the script couldn't parse
|
||||
// and there is no additional information about it anyways.
|
||||
scriptClass, addr, _ := txscript.ExtractScriptPubKeyAddress(scriptPubKey,
|
||||
s.cfg.DAGParams)
|
||||
s.dag.Params)
|
||||
var address *string
|
||||
if addr != nil {
|
||||
address = pointers.String(addr.EncodeAddress())
|
||||
}
|
||||
|
||||
txOutReply := &rpcmodel.GetTxOutResult{
|
||||
txOutReply := &model.GetTxOutResult{
|
||||
SelectedTip: selectedTipHash,
|
||||
Confirmations: confirmations,
|
||||
IsInMempool: isInMempool,
|
||||
Value: util.Amount(value).ToKAS(),
|
||||
ScriptPubKey: rpcmodel.ScriptPubKeyResult{
|
||||
ScriptPubKey: model.ScriptPubKeyResult{
|
||||
Asm: disbuf,
|
||||
Hex: hex.EncodeToString(scriptPubKey),
|
||||
Type: scriptClass.String(),
|
@ -1,10 +1,10 @@
|
||||
package rpc
|
||||
|
||||
import "github.com/kaspanet/kaspad/rpcmodel"
|
||||
import "github.com/kaspanet/kaspad/rpc/model"
|
||||
|
||||
// handleHelp implements the help command.
|
||||
func handleHelp(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
c := cmd.(*rpcmodel.HelpCmd)
|
||||
c := cmd.(*model.HelpCmd)
|
||||
|
||||
// Provide a usage overview of all commands when no specific command
|
||||
// was specified.
|
||||
@ -26,8 +26,8 @@ func handleHelp(s *Server, cmd interface{}, closeChan <-chan struct{}) (interfac
|
||||
// for commands that are unimplemented or related to wallet
|
||||
// functionality.
|
||||
if _, ok := rpcHandlers[command]; !ok {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCInvalidParameter,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInvalidParameter,
|
||||
Message: "Unknown command: " + command,
|
||||
}
|
||||
}
|
@ -1,7 +1,7 @@
|
||||
package rpc
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
"github.com/kaspanet/kaspad/wire"
|
||||
)
|
||||
@ -11,14 +11,14 @@ import (
|
||||
//
|
||||
// NOTE: This extension is ported from github.com/decred/dcrd
|
||||
func handleLoadTxFilter(wsc *wsClient, icmd interface{}) (interface{}, error) {
|
||||
cmd := icmd.(*rpcmodel.LoadTxFilterCmd)
|
||||
cmd := icmd.(*model.LoadTxFilterCmd)
|
||||
|
||||
outpoints := make([]wire.Outpoint, len(cmd.Outpoints))
|
||||
for i := range cmd.Outpoints {
|
||||
txID, err := daghash.NewTxIDFromStr(cmd.Outpoints[i].TxID)
|
||||
if err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCInvalidParameter,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInvalidParameter,
|
||||
Message: err.Error(),
|
||||
}
|
||||
}
|
||||
@ -28,7 +28,7 @@ func handleLoadTxFilter(wsc *wsClient, icmd interface{}) (interface{}, error) {
|
||||
}
|
||||
}
|
||||
|
||||
params := wsc.server.cfg.DAGParams
|
||||
params := wsc.server.dag.Params
|
||||
|
||||
reloadedFilterData := func() bool {
|
||||
wsc.Lock()
|
@ -1,13 +1,13 @@
|
||||
package rpc
|
||||
|
||||
import "github.com/kaspanet/kaspad/rpcmodel"
|
||||
import "github.com/kaspanet/kaspad/rpc/model"
|
||||
|
||||
// handleNotifyChainChanges implements the notifyChainChanges command extension for
|
||||
// websocket connections.
|
||||
func handleNotifyChainChanges(wsc *wsClient, icmd interface{}) (interface{}, error) {
|
||||
if wsc.server.cfg.AcceptanceIndex == nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCNoAcceptanceIndex,
|
||||
if wsc.server.acceptanceIndex == nil {
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCNoAcceptanceIndex,
|
||||
Message: "The acceptance index must be " +
|
||||
"enabled to receive chain changes " +
|
||||
"(specify --acceptanceindex)",
|
@ -1,22 +1,22 @@
|
||||
package rpc
|
||||
|
||||
import (
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util/subnetworkid"
|
||||
)
|
||||
|
||||
// handleNotifyNewTransations implements the notifyNewTransactions command
|
||||
// extension for websocket connections.
|
||||
func handleNotifyNewTransactions(wsc *wsClient, icmd interface{}) (interface{}, error) {
|
||||
cmd, ok := icmd.(*rpcmodel.NotifyNewTransactionsCmd)
|
||||
cmd, ok := icmd.(*model.NotifyNewTransactionsCmd)
|
||||
if !ok {
|
||||
return nil, rpcmodel.ErrRPCInternal
|
||||
return nil, model.ErrRPCInternal
|
||||
}
|
||||
|
||||
isVerbose := cmd.Verbose != nil && *cmd.Verbose
|
||||
if !isVerbose && cmd.Subnetwork != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCInvalidParameter,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInvalidParameter,
|
||||
Message: "Subnetwork switch is only allowed if verbose=true",
|
||||
}
|
||||
}
|
||||
@ -26,30 +26,30 @@ func handleNotifyNewTransactions(wsc *wsClient, icmd interface{}) (interface{},
|
||||
var err error
|
||||
subnetworkID, err = subnetworkid.NewFromStr(*cmd.Subnetwork)
|
||||
if err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCInvalidParameter,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInvalidParameter,
|
||||
Message: "Subnetwork is malformed",
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if isVerbose {
|
||||
nodeSubnetworkID := wsc.server.cfg.DAG.SubnetworkID()
|
||||
nodeSubnetworkID := wsc.server.dag.SubnetworkID()
|
||||
if nodeSubnetworkID.IsEqual(subnetworkid.SubnetworkIDNative) && subnetworkID != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCInvalidParameter,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInvalidParameter,
|
||||
Message: "Subnetwork switch is disabled when node is in Native subnetwork",
|
||||
}
|
||||
} else if nodeSubnetworkID != nil {
|
||||
if subnetworkID == nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCInvalidParameter,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInvalidParameter,
|
||||
Message: "Subnetwork switch is required when node is partial",
|
||||
}
|
||||
}
|
||||
if !nodeSubnetworkID.IsEqual(subnetworkID) {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCInvalidParameter,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInvalidParameter,
|
||||
Message: "Subnetwork must equal the node's subnetwork when the node is partial",
|
||||
}
|
||||
}
|
@ -2,7 +2,7 @@ package rpc
|
||||
|
||||
import (
|
||||
"fmt"
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util/daghash"
|
||||
)
|
||||
|
||||
@ -11,16 +11,16 @@ import (
|
||||
//
|
||||
// NOTE: This extension is ported from github.com/decred/dcrd
|
||||
func handleRescanBlocks(wsc *wsClient, icmd interface{}) (interface{}, error) {
|
||||
cmd, ok := icmd.(*rpcmodel.RescanBlocksCmd)
|
||||
cmd, ok := icmd.(*model.RescanBlocksCmd)
|
||||
if !ok {
|
||||
return nil, rpcmodel.ErrRPCInternal
|
||||
return nil, model.ErrRPCInternal
|
||||
}
|
||||
|
||||
// Load client's transaction filter. Must exist in order to continue.
|
||||
filter := wsc.FilterData()
|
||||
if filter == nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCMisc,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCMisc,
|
||||
Message: "Transaction filter must be loaded before rescanning",
|
||||
}
|
||||
}
|
||||
@ -35,24 +35,24 @@ func handleRescanBlocks(wsc *wsClient, icmd interface{}) (interface{}, error) {
|
||||
blockHashes[i] = hash
|
||||
}
|
||||
|
||||
discoveredData := make([]rpcmodel.RescannedBlock, 0, len(blockHashes))
|
||||
discoveredData := make([]model.RescannedBlock, 0, len(blockHashes))
|
||||
|
||||
// Iterate over each block in the request and rescan. When a block
|
||||
// contains relevant transactions, add it to the response.
|
||||
bc := wsc.server.cfg.DAG
|
||||
params := wsc.server.cfg.DAGParams
|
||||
bc := wsc.server.dag
|
||||
params := wsc.server.dag.Params
|
||||
var lastBlockHash *daghash.Hash
|
||||
for i := range blockHashes {
|
||||
block, err := bc.BlockByHash(blockHashes[i])
|
||||
if err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCBlockNotFound,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCBlockNotFound,
|
||||
Message: "Failed to fetch block: " + err.Error(),
|
||||
}
|
||||
}
|
||||
if lastBlockHash != nil && !block.MsgBlock().Header.ParentHashes[0].IsEqual(lastBlockHash) { // TODO: (Stas) This is likely wrong. Modified to satisfy compilation.
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCInvalidParameter,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCInvalidParameter,
|
||||
Message: fmt.Sprintf("Block %s is not a child of %s",
|
||||
blockHashes[i], lastBlockHash),
|
||||
}
|
||||
@ -61,7 +61,7 @@ func handleRescanBlocks(wsc *wsClient, icmd interface{}) (interface{}, error) {
|
||||
|
||||
transactions := rescanBlockFilter(filter, block, params)
|
||||
if len(transactions) != 0 {
|
||||
discoveredData = append(discoveredData, rpcmodel.RescannedBlock{
|
||||
discoveredData = append(discoveredData, model.RescannedBlock{
|
||||
Hash: cmd.BlockHashes[i],
|
||||
Transactions: transactions,
|
||||
})
|
46
rpc/handle_send_raw_transaction.go
Normal file
46
rpc/handle_send_raw_transaction.go
Normal file
@ -0,0 +1,46 @@
|
||||
package rpc
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"encoding/hex"
|
||||
"github.com/kaspanet/kaspad/mempool"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
"github.com/kaspanet/kaspad/wire"
|
||||
"github.com/pkg/errors"
|
||||
)
|
||||
|
||||
// handleSendRawTransaction implements the sendRawTransaction command.
|
||||
func handleSendRawTransaction(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
c := cmd.(*model.SendRawTransactionCmd)
|
||||
// Deserialize and send off to tx relay
|
||||
hexStr := c.HexTx
|
||||
serializedTx, err := hex.DecodeString(hexStr)
|
||||
if err != nil {
|
||||
return nil, rpcDecodeHexError(hexStr)
|
||||
}
|
||||
var msgTx wire.MsgTx
|
||||
err = msgTx.Deserialize(bytes.NewReader(serializedTx))
|
||||
if err != nil {
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCDeserialization,
|
||||
Message: "TX decode failed: " + err.Error(),
|
||||
}
|
||||
}
|
||||
|
||||
tx := util.NewTx(&msgTx)
|
||||
err = s.protocolManager.AddTransaction(tx)
|
||||
if err != nil {
|
||||
if !errors.As(err, &mempool.RuleError{}) {
|
||||
panic(err)
|
||||
}
|
||||
|
||||
log.Debugf("Rejected transaction %s: %s", tx.ID(), err)
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCVerify,
|
||||
Message: "TX rejected: " + err.Error(),
|
||||
}
|
||||
}
|
||||
|
||||
return tx.ID().String(), nil
|
||||
}
|
@ -1,9 +1,9 @@
|
||||
package rpc
|
||||
|
||||
import "github.com/kaspanet/kaspad/rpcmodel"
|
||||
import "github.com/kaspanet/kaspad/rpc/model"
|
||||
|
||||
// handleSession implements the session command extension for websocket
|
||||
// connections.
|
||||
func handleSession(wsc *wsClient, icmd interface{}) (interface{}, error) {
|
||||
return &rpcmodel.SessionResult{SessionID: wsc.sessionID}, nil
|
||||
return &model.SessionResult{SessionID: wsc.sessionID}, nil
|
||||
}
|
@ -3,14 +3,13 @@ package rpc
|
||||
import (
|
||||
"encoding/hex"
|
||||
"fmt"
|
||||
"github.com/kaspanet/kaspad/blockdag"
|
||||
"github.com/kaspanet/kaspad/rpcmodel"
|
||||
"github.com/kaspanet/kaspad/rpc/model"
|
||||
"github.com/kaspanet/kaspad/util"
|
||||
)
|
||||
|
||||
// handleSubmitBlock implements the submitBlock command.
|
||||
func handleSubmitBlock(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
c := cmd.(*rpcmodel.SubmitBlockCmd)
|
||||
c := cmd.(*model.SubmitBlockCmd)
|
||||
|
||||
// Deserialize the submitted block.
|
||||
hexStr := c.HexBlock
|
||||
@ -24,18 +23,18 @@ func handleSubmitBlock(s *Server, cmd interface{}, closeChan <-chan struct{}) (i
|
||||
|
||||
block, err := util.NewBlockFromBytes(serializedBlock)
|
||||
if err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCDeserialization,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCDeserialization,
|
||||
Message: "Block decode failed: " + err.Error(),
|
||||
}
|
||||
}
|
||||
|
||||
// Process this block using the same rules as blocks coming from other
|
||||
// nodes. This will in turn relay it to the network like normal.
|
||||
_, err = s.cfg.SyncMgr.SubmitBlock(block, blockdag.BFNone)
|
||||
err = s.protocolManager.AddBlock(block)
|
||||
if err != nil {
|
||||
return nil, &rpcmodel.RPCError{
|
||||
Code: rpcmodel.ErrRPCVerify,
|
||||
return nil, &model.RPCError{
|
||||
Code: model.ErrRPCVerify,
|
||||
Message: fmt.Sprintf("Block rejected. Reason: %s", err),
|
||||
}
|
||||
}
|
@ -6,5 +6,5 @@ import (
|
||||
|
||||
// handleUptime implements the uptime command.
|
||||
func handleUptime(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
return mstime.Now().UnixMilliseconds() - s.cfg.StartupTime, nil
|
||||
return mstime.Now().UnixMilliseconds() - s.startupTime.UnixMilliseconds(), nil
|
||||
}
|
@ -1,6 +1,6 @@
|
||||
package rpc
|
||||
|
||||
import "github.com/kaspanet/kaspad/rpcmodel"
|
||||
import "github.com/kaspanet/kaspad/rpc/model"
|
||||
|
||||
// API version constants
|
||||
const (
|
||||
@ -12,7 +12,7 @@ const (
|
||||
|
||||
// handleVersion implements the version command.
|
||||
func handleVersion(s *Server, cmd interface{}, closeChan <-chan struct{}) (interface{}, error) {
|
||||
result := map[string]rpcmodel.VersionResult{
|
||||
result := map[string]model.VersionResult{
|
||||
"kaspadjsonrpcapi": {
|
||||
VersionString: jsonrpcSemverString,
|
||||
Major: jsonrpcSemverMajor,
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
x
Reference in New Issue
Block a user