Implement getwork RPC.

This commit adds full support for the getwork RPC which can be used to
support external mining through applications such as cgminer.

Closes #125.
This commit is contained in:
Dave Collins 2014-03-20 02:06:10 -05:00
parent a38f93bf06
commit 92ca0e92c0
6 changed files with 633 additions and 33 deletions

View file

@ -98,6 +98,30 @@ type calcNextReqDifficultyMsg struct {
reply chan calcNextReqDifficultyResponse
}
// processBlockResponse is a response sent to the reply channel of a
// processBlockMsg.
type processBlockResponse struct {
isOrphan bool
err error
}
// processBlockMsg is a message type to be sent across the message channel
// for requested a block is processed. Note this call differs from blockMsg
// above in that blockMsg is intended for blocks that can from peers and have
// extra handling whereas this message essentially is just a concurrent safe
// way to call ProcessBlock on the internal block chain instance.
type processBlockMsg struct {
block *btcutil.Block
reply chan processBlockResponse
}
// isCurrentMsg is a message type to be sent across the message channel for
// requesting whether or not the block manager believes it is synced with
// the currently connected peers.
type isCurrentMsg struct {
reply chan bool
}
// headerNode is used as a node in a list of headers that are linked together
// between checkpoints.
type headerNode struct {
@ -119,6 +143,17 @@ type chainState struct {
pastMedianTimeErr error
}
// Best returns the block hash and height known for the tip of the best known
// chain.
//
// This function is safe for concurrent access.
func (c *chainState) Best() (*btcwire.ShaHash, int64) {
c.Lock()
defer c.Unlock()
return c.newestHash, c.newestHeight
}
// blockManager provides a concurrency safe block manager for handling all
// incoming blocks.
type blockManager struct {
@ -983,6 +1018,26 @@ out:
difficulty: difficulty,
err: err,
}
case processBlockMsg:
err := b.blockChain.ProcessBlock(msg.block, false)
if err != nil {
msg.reply <- processBlockResponse{
isOrphan: false,
err: err,
}
}
blockSha, _ := msg.block.Sha()
msg.reply <- processBlockResponse{
isOrphan: b.blockChain.IsKnownOrphan(
blockSha),
err: nil,
}
case isCurrentMsg:
msg.reply <- b.current()
default:
bmgrLog.Warnf("Invalid message type in block "+
"handler: %T", msg)
@ -1224,6 +1279,24 @@ func (b *blockManager) CalcNextRequiredDifficulty(timestamp time.Time) (uint32,
return response.difficulty, response.err
}
// ProcessBlock makes use of ProcessBlock on an internal instance of a block
// chain. It is funneled through the block manager since btcchain is not safe
// for concurrent access.
func (b *blockManager) ProcessBlock(block *btcutil.Block) (bool, error) {
reply := make(chan processBlockResponse)
b.msgChan <- processBlockMsg{block: block, reply: reply}
response := <-reply
return response.isOrphan, response.err
}
// IsCurrent returns whether or not the block manager believes it is synced with
// the connected peers.
func (b *blockManager) IsCurrent() bool {
reply := make(chan bool)
b.msgChan <- isCurrentMsg{reply: reply}
return <-reply
}
// newBlockManager returns a new bitcoin block manager.
// Use Start to begin processing asynchronous block and inv updates.
func newBlockManager(s *server) (*blockManager, error) {

View file

@ -103,10 +103,12 @@ type config struct {
BlockMinSize uint32 `long:"blockminsize" description:"Mininum block size in bytes to be used when creating a block"`
BlockMaxSize uint32 `long:"blockmaxsize" description:"Maximum block size in bytes to be used when creating a block"`
BlockPrioritySize uint32 `long:"blockprioritysize" description:"Size in bytes for high-priority/low-fee transactions when creating a block"`
GetWorkKeys []string `long:"getworkkey" description:"Use the specified payment address for blocks generated by getwork."`
onionlookup func(string) ([]net.IP, error)
lookup func(string) ([]net.IP, error)
oniondial func(string, string) (net.Conn, error)
dial func(string, string) (net.Conn, error)
miningKeys []btcutil.Address
}
// serviceOptions defines the configuration options for btcd as a service on
@ -521,6 +523,27 @@ func loadConfig() (*config, []string, error) {
cfg.BlockPrioritySize = minUint32(cfg.BlockPrioritySize, cfg.BlockMaxSize)
cfg.BlockMinSize = minUint32(cfg.BlockMinSize, cfg.BlockMaxSize)
// Check keys are valid and saved parsed versions.
cfg.miningKeys = make([]btcutil.Address, 0, len(cfg.GetWorkKeys))
for _, strAddr := range cfg.GetWorkKeys {
addr, err := btcutil.DecodeAddress(strAddr, activeNetParams.btcnet)
if err != nil {
str := "%s: the specified getworkkey '%s' failed to decode: %v"
err := fmt.Errorf(str, "loadConfig", strAddr, err)
fmt.Fprintln(os.Stderr, err)
parser.WriteHelp(os.Stderr)
return nil, nil, err
}
if !addr.IsForNet(activeNetParams.btcnet) {
str := "%s: the specified getworkkey '%s' is on the wrong network"
err := fmt.Errorf(str, "loadConfig", strAddr)
fmt.Fprintln(os.Stderr, err)
parser.WriteHelp(os.Stderr)
return nil, nil, err
}
cfg.miningKeys = append(cfg.miningKeys, addr)
}
// Add default port to all listener addresses if needed and remove
// duplicate addresses.
cfg.Listeners = normalizeAddresses(cfg.Listeners,

9
doc.go
View file

@ -77,6 +77,15 @@ Application Options:
--limitfreerelay= Limit relay of transactions with no transaction fee
to the given amount in thousands of bytes per minute
(15)
--blockminsize= Mininum block size in bytes to be used when creating
a block
--blockmaxsize= Maximum block size in bytes to be used when creating
a block (750000)
--blockprioritysize= Size in bytes for high-priority/low-fee transactions
when creating a block (50000)
--getworkkey= Use the specified hex-encoded serialized public keys
as the payment address for blocks generated by
getwork.
Help Options:
-h, --help Show this help message

View file

@ -92,8 +92,9 @@ type txMemPool struct {
orphans map[btcwire.ShaHash]*btcutil.Tx
orphansByPrev map[btcwire.ShaHash]*list.List
outpoints map[btcwire.OutPoint]*btcutil.Tx
pennyTotal float64 // exponentially decaying total for penny spends.
lastPennyUnix int64 // unix time of last ``penny spend''
lastUpdated time.Time // last time pool was updated
pennyTotal float64 // exponentially decaying total for penny spends.
lastPennyUnix int64 // unix time of last ``penny spend''
}
// isDust returns whether or not the passed transaction output amount is
@ -602,6 +603,7 @@ func (mp *txMemPool) removeTransaction(tx *btcutil.Tx) {
delete(mp.outpoints, txIn.PreviousOutpoint)
}
delete(mp.pool, *txHash)
mp.lastUpdated = time.Now()
}
}
@ -655,6 +657,7 @@ func (mp *txMemPool) addTransaction(tx *btcutil.Tx, height, fee int64) {
for _, txIn := range tx.MsgTx().TxIn {
mp.outpoints[txIn.PreviousOutpoint] = tx
}
mp.lastUpdated = time.Now()
}
// checkPoolDoubleSpend checks whether or not the passed transaction is
@ -1073,6 +1076,17 @@ func (mp *txMemPool) TxDescs() []*TxDesc {
return descs
}
// LastUpdated returns the last time a transaction was added to or removed from
// the main pool. It does not include the orphan pool.
//
// This function is safe for concurrent access.
func (mp *txMemPool) LastUpdated() time.Time {
mp.RLock()
defer mp.RUnlock()
return mp.lastUpdated
}
// newTxMemPool returns a new memory pool for validating and storing standalone
// transactions until they are mined into a block.
func newTxMemPool(server *server) *txMemPool {

View file

@ -7,6 +7,7 @@ package main
import (
"container/heap"
"container/list"
"fmt"
"github.com/conformal/btcchain"
"github.com/conformal/btcdb"
"github.com/conformal/btcscript"
@ -769,3 +770,59 @@ mempoolLoop:
sigOpCounts: txSigOpCounts,
}, nil
}
// UpdateBlockTime updates the timestamp in the header of the passed block to
// the current time while taking into account the median time of the last
// several blocks to ensure the new time is after that time per the chain
// consensus rules. Finally, it will update the target difficulty if needed
// based on the new time for the test networks since their target difficulty can
// change based upon time.
func UpdateBlockTime(msgBlock *btcwire.MsgBlock, bManager *blockManager) error {
// The new timestamp is potentially adjusted to ensure it comes after
// the median time of the last several blocks per the chain consensus
// rules.
newTimestamp, err := medianAdjustedTime(&bManager.chainState)
if err != nil {
return err
}
msgBlock.Header.Timestamp = newTimestamp
// Recalculate the required difficulty for the test networks since it
// can change based on time.
if activeNetParams.btcnet == btcwire.TestNet ||
activeNetParams.btcnet == btcwire.TestNet3 {
difficulty, err := bManager.CalcNextRequiredDifficulty(newTimestamp)
if err != nil {
return err
}
msgBlock.Header.Bits = difficulty
}
return nil
}
// UpdateExtraNonce updates the extra nonce in the coinbase script of the passed
// block by regenerating the coinbase script with the passed value and block
// height. It also recalculates and updates the new merkle root the results
// from changing the coinbase script.
func UpdateExtraNonce(msgBlock *btcwire.MsgBlock, blockHeight int64, extraNonce uint64) error {
coinbaseScript := standardCoinbaseScript(blockHeight, extraNonce)
if len(coinbaseScript) > btcchain.MaxCoinbaseScriptLen {
return fmt.Errorf("coinbase transaction script length "+
"of %d is out of range (min: %d, max: %d)",
len(coinbaseScript), btcchain.MinCoinbaseScriptLen,
btcchain.MaxCoinbaseScriptLen)
}
msgBlock.Transactions[0].TxIn[0].SignatureScript = coinbaseScript
// TODO(davec): A btcutil.Block should use saved in the state to avoid
// recalculating all of the other transaction hashes.
// block.Transactions[0].InvalidateCache()
// Recalculate the merkle root with the updated extra nonce.
block := btcutil.NewBlock(msgBlock)
merkles := btcchain.BuildMerkleTreeStore(block.Transactions())
msgBlock.Header.MerkleRoot = *merkles[len(merkles)-1]
return nil
}

View file

@ -10,6 +10,7 @@ import (
"crypto/subtle"
"crypto/tls"
"encoding/base64"
"encoding/binary"
"encoding/hex"
"errors"
"fmt"
@ -22,6 +23,7 @@ import (
"github.com/conformal/fastsha256"
"io/ioutil"
"math/big"
"math/rand"
"net"
"net/http"
"os"
@ -31,9 +33,33 @@ import (
"time"
)
// rpcAuthTimeoutSeconds is the number of seconds a connection to the RPC server
// is allowed to stay open without authenticating before it is closed.
const rpcAuthTimeoutSeconds = 10
const (
// rpcAuthTimeoutSeconds is the number of seconds a connection to the
// RPC server is allowed to stay open without authenticating before it
// is closed.
rpcAuthTimeoutSeconds = 10
// uint256Size is the number of bytes needed to represent an unsigned
// 256-bit integer.
uint256Size = 32
// getworkDataLen is the length of the data field of the getwork RPC.
// It consists of the serialized block header plus the internal sha256
// padding. The internal sha256 padding consists of a single 1 bit
// followed by enough zeros to pad the message out to 56 bytes followed
// by length of the message in bits encoded as a big-endian uint64
// (8 bytes). Thus, the resulting length is a multiple of the sha256
// block size (64 bytes).
getworkDataLen = (1 + ((btcwire.MaxBlockHeaderPayload + 8) /
fastsha256.BlockSize)) * fastsha256.BlockSize
// hash1Len is the length of the hash1 field of the getwork RPC. It
// consists of a zero hash plus the internal sha256 padding. See
// the getworkDataLen comment for details about the internal sha256
// padding format.
hash1Len = (1 + ((btcwire.HashSize + 8) / fastsha256.BlockSize)) *
fastsha256.BlockSize
)
// Errors
var (
@ -69,6 +95,7 @@ var rpcHandlersBeforeInit = map[string]commandHandler{
"getpeerinfo": handleGetPeerInfo,
"getrawmempool": handleGetRawMempool,
"getrawtransaction": handleGetRawTransaction,
"getwork": handleGetWork,
"help": handleHelp,
"ping": handlePing,
"sendrawtransaction": handleSendRawTransaction,
@ -105,7 +132,6 @@ var rpcAskWallet = map[string]bool{
"gettransaction": true,
"gettxout": true,
"gettxoutsetinfo": true,
"getwork": true,
"importprivkey": true,
"importwallet": true,
"keypoolrefill": true,
@ -138,6 +164,33 @@ var rpcUnimplemented = map[string]bool{
"getmininginfo": true,
}
// workStateBlockInfo houses information about how to reconstruct a block given
// its template and signature script.
type workStateBlockInfo struct {
msgBlock *btcwire.MsgBlock
signatureScript []byte
}
// workState houses state that is used in between multiple RPC invocations to
// getwork.
type workState struct {
sync.Mutex
lastTxUpdate time.Time
lastGenerated time.Time
prevHash *btcwire.ShaHash
msgBlock *btcwire.MsgBlock
extraNonce uint64
blockInfo map[btcwire.ShaHash]*workStateBlockInfo
}
// newWorkState returns a new instance of a workState with all internal fields
// initialized and ready to use.
func newWorkState() *workState {
return &workState{
blockInfo: make(map[btcwire.ShaHash]*workStateBlockInfo),
}
}
// rpcServer holds the items the rpc server may need to access (config,
// shutdown, main server, etc.)
type rpcServer struct {
@ -150,6 +203,7 @@ type rpcServer struct {
numClientsMutex sync.Mutex
wg sync.WaitGroup
listeners []net.Listener
workState *workState
quit chan int
}
@ -337,9 +391,10 @@ func newRPCServer(listenAddrs []string, s *server) (*rpcServer, error) {
login := cfg.RPCUser + ":" + cfg.RPCPass
auth := "Basic " + base64.StdEncoding.EncodeToString([]byte(login))
rpc := rpcServer{
authsha: fastsha256.Sum256([]byte(auth)),
server: s,
quit: make(chan int),
authsha: fastsha256.Sum256([]byte(auth)),
server: s,
workState: newWorkState(),
quit: make(chan int),
}
rpc.ntfnMgr = newWsNotificationManager(&rpc)
@ -667,6 +722,46 @@ func createVoutList(mtx *btcwire.MsgTx, net btcwire.BitcoinNet) ([]btcjson.Vout,
return voutList, nil
}
// createTxRawResult converts the passed transaction and associated parameters
// to a raw transaction JSON object.
func createTxRawResult(net btcwire.BitcoinNet, txSha string, mtx *btcwire.MsgTx, blk *btcutil.Block, maxidx int64, blksha *btcwire.ShaHash) (*btcjson.TxRawResult, error) {
mtxHex, err := messageToHex(mtx)
if err != nil {
return nil, err
}
vin, err := createVinList(mtx)
if err != nil {
return nil, err
}
vout, err := createVoutList(mtx, net)
if err != nil {
return nil, err
}
txReply := &btcjson.TxRawResult{
Hex: mtxHex,
Txid: txSha,
Vout: vout,
Vin: vin,
Version: mtx.Version,
LockTime: mtx.LockTime,
}
if blk != nil {
blockHeader := &blk.MsgBlock().Header
idx := blk.Height()
// This is not a typo, they are identical in bitcoind as well.
txReply.Time = blockHeader.Timestamp.Unix()
txReply.Blocktime = blockHeader.Timestamp.Unix()
txReply.BlockHash = blksha.String()
txReply.Confirmations = uint64(1 + maxidx - idx)
}
return txReply, nil
}
// handleDecodeRawTransaction handles decoderawtransaction commands.
func handleDecodeRawTransaction(s *rpcServer, cmd btcjson.Cmd) (interface{}, error) {
c := cmd.(*btcjson.DecodeRawTransactionCmd)
@ -1250,44 +1345,373 @@ func handleGetRawTransaction(s *rpcServer, cmd btcjson.Cmd) (interface{}, error)
return *rawTxn, nil
}
// createTxRawResult converts the passed transaction and associated parameters
// to a raw transaction JSON object.
func createTxRawResult(net btcwire.BitcoinNet, txSha string, mtx *btcwire.MsgTx, blk *btcutil.Block, maxidx int64, blksha *btcwire.ShaHash) (*btcjson.TxRawResult, error) {
mtxHex, err := messageToHex(mtx)
// bigToLEUint256 returns the passed big integer as an unsigned 256-bit integer
// encoded as little-endian bytes. Numbers which are larger than the max
// unsigned 256-bit integer are truncated.
func bigToLEUint256(n *big.Int) [uint256Size]byte {
// Pad or truncate the big-endian big int to correct number of bytes.
nBytes := n.Bytes()
nlen := len(nBytes)
pad := 0
start := 0
if nlen <= uint256Size {
pad = uint256Size - nlen
} else {
start = nlen - uint256Size
}
var buf [uint256Size]byte
copy(buf[pad:], nBytes[start:])
// Reverse the bytes to little endian and return them.
for i := 0; i < uint256Size/2; i++ {
buf[i], buf[uint256Size-1-i] = buf[uint256Size-1-i], buf[i]
}
return buf
}
// reverseUint32Array treats the passed bytes as a series of uint32s and
// reverses the byte order of each uint32. The passed byte slice must be a
// multiple of 4 for a correct result. The passed bytes slice is modified.
func reverseUint32Array(b []byte) {
blen := len(b)
for i := 0; i < blen; i += 4 {
b[i], b[i+3] = b[i+3], b[i]
b[i+1], b[i+2] = b[i+2], b[i+1]
}
}
// handleGetWorkRequest is a helper for handleGetWork which deals with
// generating and returning work to the caller.
//
// This function MUST be called with the RPC workstate locked.
func handleGetWorkRequest(s *rpcServer) (interface{}, error) {
state := s.workState
// Generate a new block template when the current best block has
// changed or the transactions in the memory pool have been updated
// and it has been at least one minute since the last template was
// generated.
lastTxUpdate := s.server.txMemPool.LastUpdated()
latestHash, latestHeight := s.server.blockManager.chainState.Best()
msgBlock := state.msgBlock
if msgBlock == nil || state.prevHash == nil ||
!state.prevHash.IsEqual(latestHash) ||
(state.lastTxUpdate != lastTxUpdate &&
time.Now().After(state.lastGenerated.Add(time.Minute))) {
// Reset the extra nonce and clear all cached template
// variations if the best block changed.
if state.prevHash != nil && !state.prevHash.IsEqual(latestHash) {
state.extraNonce = 0
state.blockInfo = make(map[btcwire.ShaHash]*workStateBlockInfo)
}
// Reset the previous best hash the block template was generated
// against so any errors below cause the next invocation to try
// again.
state.prevHash = nil
// Choose a payment address at random.
rand.Seed(time.Now().UnixNano())
payToAddr := cfg.miningKeys[rand.Intn(len(cfg.miningKeys))]
template, err := NewBlockTemplate(payToAddr, s.server.txMemPool)
if err != nil {
errStr := fmt.Sprintf("Failed to create new block "+
"template: %v", err)
rpcsLog.Errorf(errStr)
return nil, btcjson.Error{
Code: btcjson.ErrInternal.Code,
Message: errStr,
}
}
msgBlock = template.block
// Update work state to ensure another block template isn't
// generated until needed.
state.msgBlock = msgBlock
state.lastGenerated = time.Now()
state.lastTxUpdate = lastTxUpdate
state.prevHash = latestHash
rpcsLog.Debugf("Generated block template (timestamp %v, extra "+
"nonce %d, target %064x, merkle root %s, signature "+
"script %x)", msgBlock.Header.Timestamp,
state.extraNonce,
btcchain.CompactToBig(msgBlock.Header.Bits),
msgBlock.Header.MerkleRoot,
msgBlock.Transactions[0].TxIn[0].SignatureScript)
} else {
// At this point, there is a saved block template and a new
// request for work was made, but either the available
// transactions haven't change or it hasn't been long enough to
// trigger a new block template to be generated. So, update the
// existing block template and track the variations so each
// variation can be regenerated if a caller finds an answer and
// makes a submission against it.
// Update the time of the block template to the current time
// while accounting for the median time of the past several
// blocks per the chain consensus rules.
UpdateBlockTime(msgBlock, s.server.blockManager)
// Increment the extra nonce and update the block template
// with the new value by regenerating the coinbase script and
// setting the merkle root to the new value.
state.extraNonce++
err := UpdateExtraNonce(msgBlock, latestHeight+1, state.extraNonce)
if err != nil {
errStr := fmt.Sprintf("Failed to update extra nonce: "+
"%v", err)
rpcsLog.Warnf(errStr)
return nil, btcjson.Error{
Code: btcjson.ErrInternal.Code,
Message: errStr,
}
}
rpcsLog.Debugf("Updated block template (timestamp %v, extra "+
"nonce %d, target %064x, merkle root %s, signature "+
"script %x)", msgBlock.Header.Timestamp,
state.extraNonce,
btcchain.CompactToBig(msgBlock.Header.Bits),
msgBlock.Header.MerkleRoot,
msgBlock.Transactions[0].TxIn[0].SignatureScript)
}
// In order to efficiently store the variations of block templates that
// have been provided to callers, save a pointer to the block as well as
// the modified signature script keyed by the merkle root. This
// information, along with the data that is included in a work
// submission, is used to rebuild the block before checking the
// submitted solution.
coinbaseTx := msgBlock.Transactions[0]
state.blockInfo[msgBlock.Header.MerkleRoot] = &workStateBlockInfo{
msgBlock: msgBlock,
signatureScript: coinbaseTx.TxIn[0].SignatureScript,
}
// Serialize the block header into a buffer large enough to hold the
// the block header and the internal sha256 padding that is added and
// retuned as part of the data below.
data := make([]byte, 0, getworkDataLen)
buf := bytes.NewBuffer(data)
err := msgBlock.Header.Serialize(buf)
if err != nil {
errStr := fmt.Sprintf("Failed to serialize data: %v", err)
rpcsLog.Warnf(errStr)
return nil, btcjson.Error{
Code: btcjson.ErrInternal.Code,
Message: errStr,
}
return nil, err
}
vin, err := createVinList(mtx)
// Calculate the midstate for the block header. The midstate here is
// the internal state of the sha256 algorithm for the first chunk of the
// block header (sha256 operates on 64-byte chunks) which is before the
// nonce. This allows sophisticated callers to avoid hashing the first
// chunk over and over while iterating the nonce range.
data = data[:buf.Len()]
midstate := fastsha256.MidState256(data)
// Expand the data slice to include the full data buffer and apply the
// internal sha256 padding which consists of a single 1 bit followed
// by enough zeros to pad the message out to 56 bytes followed by the
// length of the message in bits encoded as a big-endian uint64
// (8 bytes). Thus, the resulting length is a multiple of the sha256
// block size (64 bytes). This makes the data ready for sophisticated
// caller to make use of only the second chunk along with the midstate
// for the first chunk.
data = data[:getworkDataLen]
data[btcwire.MaxBlockHeaderPayload] = 0x80
binary.BigEndian.PutUint64(data[len(data)-8:],
btcwire.MaxBlockHeaderPayload*8)
// Create the hash1 field which is a zero hash along with the internal
// sha256 padding as described above. This field is really quite
// useless, but it is required for compatibility with the reference
// implementation.
var hash1 [hash1Len]byte
hash1[btcwire.HashSize] = 0x80
binary.BigEndian.PutUint64(hash1[len(hash1)-8:], btcwire.HashSize*8)
// The final result reverses the each of the fields to little endian.
// In particular, the data, hash1, and midstate fields are treated as
// arrays of uint32s (per the internal sha256 hashing state) which are
// in big endian, and thus each 4 bytes is byte swapped. The target is
// also in big endian, but it is treated as a uint256 and byte swapped
// to little endian accordingly.
//
// The fact the fields are reversed in this way is rather odd and likey
// an artifact of some legacy internal state in the reference
// implementation, but it is required for compatibility.
reverseUint32Array(data)
reverseUint32Array(hash1[:])
reverseUint32Array(midstate[:])
target := bigToLEUint256(btcchain.CompactToBig(msgBlock.Header.Bits))
reply := &btcjson.GetWorkResult{
Data: hex.EncodeToString(data),
Hash1: hex.EncodeToString(hash1[:]),
Midstate: hex.EncodeToString(midstate[:]),
Target: hex.EncodeToString(target[:]),
}
return reply, nil
}
// handleGetWorkSubmission is a helper for handleGetWork which deals with
// the calling submitting work to be verified and processed.
//
// This function MUST be called with the RPC workstate locked.
func handleGetWorkSubmission(s *rpcServer, hexData string) (interface{}, error) {
// Ensure the provided data is sane.
if len(hexData)%2 != 0 {
hexData = "0" + hexData
}
data, err := hex.DecodeString(hexData)
if err != nil {
return nil, err
return false, btcjson.Error{
Code: btcjson.ErrInvalidParameter.Code,
Message: fmt.Sprintf("argument must be "+
"hexadecimal string (not %q)", hexData),
}
}
vout, err := createVoutList(mtx, net)
if len(data) != getworkDataLen {
return false, btcjson.Error{
Code: btcjson.ErrInvalidParameter.Code,
Message: fmt.Sprintf("argument must be "+
"%d bytes (not %d)", getworkDataLen,
len(data)),
}
}
// Reverse the data as if it were an array of 32-bit unsigned integers.
// The fact the getwork request and submission data is reversed in this
// way is rather odd and likey an artifact of some legacy internal state
// in the reference implementation, but it is required for
// compatibility.
reverseUint32Array(data)
// Deserialize the block header from the data.
var submittedHeader btcwire.BlockHeader
bhBuf := bytes.NewBuffer(data[0:btcwire.MaxBlockHeaderPayload])
err = submittedHeader.Deserialize(bhBuf)
if err != nil {
return nil, err
return false, btcjson.Error{
Code: btcjson.ErrInvalidParameter.Code,
Message: fmt.Sprintf("argument does not "+
"contain a valid block header: %v", err),
}
}
txReply := &btcjson.TxRawResult{
Hex: mtxHex,
Txid: txSha,
Vout: vout,
Vin: vin,
Version: mtx.Version,
LockTime: mtx.LockTime,
// Look up the full block for the provided data based on the
// merkle root. Return false to indicate the solve failed if
// it's not available.
state := s.workState
blockInfo, ok := state.blockInfo[submittedHeader.MerkleRoot]
if !ok {
rpcsLog.Debugf("Block submitted via getwork has no matching "+
"template for merkle root %s",
submittedHeader.MerkleRoot)
return false, nil
}
if blk != nil {
blockHeader := &blk.MsgBlock().Header
idx := blk.Height()
// Reconstruct the block using the submitted header stored block info.
msgBlock := blockInfo.msgBlock
block := btcutil.NewBlock(msgBlock)
msgBlock.Header.Timestamp = submittedHeader.Timestamp
msgBlock.Header.Nonce = submittedHeader.Nonce
msgBlock.Transactions[0].TxIn[0].SignatureScript = blockInfo.signatureScript
merkles := btcchain.BuildMerkleTreeStore(block.Transactions())
msgBlock.Header.MerkleRoot = *merkles[len(merkles)-1]
// This is not a typo, they are identical in bitcoind as well.
txReply.Time = blockHeader.Timestamp.Unix()
txReply.Blocktime = blockHeader.Timestamp.Unix()
txReply.BlockHash = blksha.String()
txReply.Confirmations = uint64(1 + maxidx - idx)
// Ensure the submitted block hash is less than the target difficulty.
err = btcchain.CheckProofOfWork(block, activeNetParams.powLimit)
if err != nil {
// Anything other than a rule violation is an unexpected error,
// so return that error as an internal error.
if _, ok := err.(btcchain.RuleError); !ok {
return false, btcjson.Error{
Code: btcjson.ErrInternal.Code,
Message: fmt.Sprintf("Unexpected error while "+
"checking proof of work: %v", err),
}
}
rpcsLog.Debugf("Block submitted via getwork does not meet "+
"the required proof of work: %v", err)
return false, nil
}
return txReply, nil
latestHash, _ := s.server.blockManager.chainState.Best()
if !msgBlock.Header.PrevBlock.IsEqual(latestHash) {
rpcsLog.Debugf("Block submitted via getwork with previous "+
"block %s is stale", msgBlock.Header.PrevBlock)
return false, nil
}
// Process this block using the same rules as blocks coming from other
// nodes. This will in turn relay it to the network like normal.
isOrphan, err := s.server.blockManager.ProcessBlock(block)
if err != nil || isOrphan {
// Anything other than a rule violation is an unexpected error,
// so return that error as an internal error.
if _, ok := err.(btcchain.RuleError); !ok {
return false, btcjson.Error{
Code: btcjson.ErrInternal.Code,
Message: fmt.Sprintf("Unexpected error while",
"processing block: %v", err),
}
}
rpcsLog.Infof("Block submitted via getwork rejected: %v", err)
return false, nil
}
// The block was accepted.
blockSha, _ := block.Sha()
rpcsLog.Infof("Block submitted via getwork accepted: %s", blockSha)
return true, nil
}
// handleGetWork implements the getwork command.
func handleGetWork(s *rpcServer, cmd btcjson.Cmd) (interface{}, error) {
c := cmd.(*btcjson.GetWorkCmd)
// Respond with an error if there are no public keys to pay the created
// blocks to.
if len(cfg.miningKeys) == 0 {
return nil, btcjson.Error{
Code: btcjson.ErrInternal.Code,
Message: "No payment addresses specified via --getworkkey",
}
}
// 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.
if s.server.ConnectedCount() == 0 {
return nil, btcjson.ErrClientNotConnected
}
// No point in generating or accepting work before the chain is synced.
if !s.server.blockManager.IsCurrent() {
return nil, btcjson.ErrClientInInitialDownload
}
// Protect concurrent access from multiple RPC invocations for work
// requests and submission.
s.workState.Lock()
defer s.workState.Unlock()
// When the caller provides data, it is a submission of a supposedly
// solved block that needs to be checked and submitted to the network
// if valid.
if c.Data != "" {
return handleGetWorkSubmission(s, c.Data)
}
// No data was provided, so the caller is requesting work.
return handleGetWorkRequest(s)
}
var helpAddenda = map[string]string{