blockbook/blockbook.go

517 lines
15 KiB
Go
Raw Normal View History

2017-08-28 09:50:57 -06:00
package main
import (
"blockbook/bchain"
"blockbook/bchain/coins"
"blockbook/common"
"blockbook/db"
"blockbook/server"
"context"
2017-08-28 09:50:57 -06:00
"flag"
2018-03-09 09:05:27 -07:00
"log"
"math/rand"
"net/http"
_ "net/http/pprof"
"os"
"os/signal"
"strings"
"sync/atomic"
"syscall"
2017-08-28 09:50:57 -06:00
"time"
2017-09-12 18:50:34 -06:00
"github.com/erikdubbelboer/gspt"
"github.com/golang/glog"
"github.com/juju/errors"
2017-08-28 09:50:57 -06:00
)
2018-02-06 01:12:50 -07:00
// debounce too close requests for resync
const debounceResyncIndexMs = 1009
// debounce too close requests for resync mempool (ZeroMQ sends message for each tx, when new block there are many transactions)
const debounceResyncMempoolMs = 1009
2018-05-23 00:54:02 -06:00
// store internal state about once every minute
const storeInternalStatePeriodMs = 59699
2017-08-28 09:50:57 -06:00
var (
blockchain = flag.String("blockchaincfg", "", "path to blockchain RPC service configuration json file")
2017-08-28 09:50:57 -06:00
dbPath = flag.String("datadir", "./data", "path to database directory")
dbCache = flag.Int("dbcache", 1<<29, "size of the rocksdb cache")
2017-08-28 09:50:57 -06:00
2018-03-01 10:37:01 -07:00
blockFrom = flag.Int("blockheight", -1, "height of the starting block")
2018-01-29 09:27:42 -07:00
blockUntil = flag.Int("blockuntil", -1, "height of the final block")
rollbackHeight = flag.Int("rollback", -1, "rollback to the given height and quit")
2017-10-05 06:35:07 -06:00
2017-08-28 09:50:57 -06:00
queryAddress = flag.String("address", "", "query contents of this address")
2018-01-31 07:03:06 -07:00
synchronize = flag.Bool("sync", false, "synchronizes until tip, if together with zeromq, keeps index synchronized")
repair = flag.Bool("repair", false, "repair the database")
2018-03-09 09:05:27 -07:00
prof = flag.String("prof", "", "http server binding [address]:port of the interface to profiling data /debug/pprof/ (default no profiling)")
2017-10-06 04:57:51 -06:00
2018-02-26 08:44:25 -07:00
syncChunk = flag.Int("chunk", 100, "block chunk size for processing")
2018-05-03 06:49:19 -06:00
syncWorkers = flag.Int("workers", 8, "number of workers to process blocks")
2018-02-26 08:44:25 -07:00
dryRun = flag.Bool("dryrun", false, "do not index blocks, only download")
2018-01-18 08:44:31 -07:00
debugMode = flag.Bool("debug", false, "debug mode, return more verbose errors, reload templates on each request")
internalBinding = flag.String("internal", "", "internal http server binding [address]:port, (default no internal server)")
2018-01-19 07:58:46 -07:00
publicBinding = flag.String("public", "", "public http server binding [address]:port[/path], (default no public server)")
2018-02-07 14:56:17 -07:00
certFiles = flag.String("certfile", "", "to enable SSL specify path to certificate files without extension, expecting <certfile>.crt and <certfile>.key, (default no SSL)")
explorerURL = flag.String("explorer", "", "address of blockchain explorer")
2018-05-14 07:49:08 -06:00
noTxCache = flag.Bool("notxcache", false, "disable tx cache")
computeColumnStats = flag.Bool("computedbstats", false, "compute column stats and exit")
// resync index at least each resyncIndexPeriodMs (could be more often if invoked by message from ZeroMQ)
resyncIndexPeriodMs = flag.Int("resyncindexperiod", 935093, "resync index period in milliseconds")
// resync mempool at least each resyncMempoolPeriodMs (could be more often if invoked by message from ZeroMQ)
resyncMempoolPeriodMs = flag.Int("resyncmempoolperiod", 60017, "resync mempool period in milliseconds")
2017-08-28 09:50:57 -06:00
)
2018-01-31 07:03:06 -07:00
var (
2018-05-23 00:54:02 -06:00
chanSyncIndex = make(chan struct{})
chanSyncMempool = make(chan struct{})
chanStoreInternalState = make(chan struct{})
chanSyncIndexDone = make(chan struct{})
chanSyncMempoolDone = make(chan struct{})
chanStoreInternalStateDone = make(chan struct{})
chain bchain.BlockChain
index *db.RocksDB
txCache *db.TxCache
syncWorker *db.SyncWorker
internalState *common.InternalState
callbacksOnNewBlock []bchain.OnNewBlockFunc
callbacksOnNewTxAddr []bchain.OnNewTxAddrFunc
2018-05-23 00:54:02 -06:00
chanOsSignal chan os.Signal
inShutdown int32
2018-01-31 07:03:06 -07:00
)
2018-04-12 07:23:16 -06:00
func init() {
2018-05-17 04:31:45 -06:00
glog.MaxSize = 1024 * 1024 * 8
2018-04-12 07:23:16 -06:00
glog.CopyStandardLogTo("INFO")
}
func getBlockChainWithRetry(coin string, configfile string, pushHandler func(bchain.NotificationType), metrics *common.Metrics, seconds int) (bchain.BlockChain, error) {
var chain bchain.BlockChain
var err error
timer := time.NewTimer(time.Second)
for i := 0; ; i++ {
if chain, err = coins.NewBlockChain(coin, configfile, pushHandler, metrics); err != nil {
if i < seconds {
glog.Error("rpc: ", err, " Retrying...")
select {
case <-chanOsSignal:
return nil, errors.New("Interrupted")
case <-timer.C:
timer.Reset(time.Second)
continue
}
} else {
return nil, err
}
}
return chain, nil
}
}
2017-08-28 09:50:57 -06:00
func main() {
flag.Parse()
2018-01-31 07:03:06 -07:00
defer glog.Flush()
rand.Seed(time.Now().UTC().UnixNano())
2018-02-28 16:59:25 -07:00
chanOsSignal = make(chan os.Signal, 1)
signal.Notify(chanOsSignal, syscall.SIGHUP, syscall.SIGINT, syscall.SIGQUIT, syscall.SIGTERM)
glog.Infof("Blockbook: %+v, debug mode %v", common.GetVersionInfo(), *debugMode)
2018-05-22 09:22:22 -06:00
2018-03-09 09:05:27 -07:00
if *prof != "" {
go func() {
log.Println(http.ListenAndServe(*prof, nil))
}()
2018-01-19 07:58:46 -07:00
}
2017-09-12 08:53:40 -06:00
if *repair {
if err := db.RepairRocksDB(*dbPath); err != nil {
glog.Fatalf("RepairRocksDB %s: %v", *dbPath, err)
2017-09-12 08:53:40 -06:00
}
return
}
2018-06-05 08:14:46 -06:00
if *blockchain == "" {
glog.Fatal("Missing blockchaincfg configuration parameter")
}
2018-06-27 16:36:56 -06:00
coin, coinShortcut, err := coins.GetCoinNameFromConfig(*blockchain)
2018-03-13 04:34:49 -06:00
if err != nil {
2018-06-05 08:14:46 -06:00
glog.Fatal("config: ", err)
2018-03-13 04:34:49 -06:00
}
gspt.SetProcTitle("blockbook-" + normalizeName(coin))
2018-06-05 08:14:46 -06:00
metrics, err := common.GetMetrics(coin)
if err != nil {
glog.Fatal("metrics: ", err)
}
2018-06-05 08:14:46 -06:00
if chain, err = getBlockChainWithRetry(coin, *blockchain, pushSynchronizationHandler, metrics, 60); err != nil {
glog.Fatal("rpc: ", err)
2017-10-07 03:05:35 -06:00
}
index, err = db.NewRocksDB(*dbPath, *dbCache, chain.GetChainParser(), metrics)
2017-08-28 09:50:57 -06:00
if err != nil {
glog.Fatal("rocksDB: ", err)
2017-08-28 09:50:57 -06:00
}
2018-01-31 07:03:06 -07:00
defer index.Close()
2017-08-28 09:50:57 -06:00
2018-06-27 16:36:56 -06:00
internalState, err = newInternalState(coin, coinShortcut, index)
2018-05-22 04:56:51 -06:00
if err != nil {
glog.Error("internalState: ", err)
return
2018-05-22 04:56:51 -06:00
}
index.SetInternalState(internalState)
if internalState.DbState != common.DbStateClosed {
2018-08-18 16:23:26 -06:00
if internalState.DbState == common.DbStateInconsistent {
glog.Error("internalState: database is in inconsistent state and cannot be used")
return
}
glog.Warning("internalState: database was left in open state, possibly previous ungraceful shutdown")
2018-05-22 04:56:51 -06:00
}
if *computeColumnStats {
internalState.DbState = common.DbStateOpen
2018-06-08 05:19:57 -06:00
err = index.ComputeInternalStateColumnStats(chanOsSignal)
if err != nil {
glog.Error("internalState: ", err)
}
2018-06-04 05:24:40 -06:00
glog.Info("DB size on disk: ", index.DatabaseSizeOnDisk(), ", DB size as computed: ", internalState.DBSizeTotal())
return
}
syncWorker, err = db.NewSyncWorker(index, chain, *syncWorkers, *syncChunk, *blockFrom, *dryRun, chanOsSignal, metrics, internalState)
if err != nil {
glog.Fatalf("NewSyncWorker %v", err)
}
2018-05-22 04:56:51 -06:00
// set the DbState to open at this moment, after all important workers are initialized
internalState.DbState = common.DbStateOpen
err = index.StoreInternalState(internalState)
2018-05-22 04:56:51 -06:00
if err != nil {
glog.Fatal("internalState: ", err)
}
2018-01-29 09:27:42 -07:00
if *rollbackHeight >= 0 {
bestHeight, bestHash, err := index.GetBestBlock()
2018-01-29 09:27:42 -07:00
if err != nil {
glog.Error("rollbackHeight: ", err)
return
2018-01-29 09:27:42 -07:00
}
if uint32(*rollbackHeight) > bestHeight {
glog.Infof("nothing to rollback, rollbackHeight %d, bestHeight: %d", *rollbackHeight, bestHeight)
2018-01-29 09:27:42 -07:00
} else {
hashes := []string{bestHash}
for height := bestHeight - 1; height >= uint32(*rollbackHeight); height-- {
hash, err := index.GetBlockHash(height)
if err != nil {
glog.Error("rollbackHeight: ", err)
return
}
hashes = append(hashes, hash)
}
err = syncWorker.DisconnectBlocks(uint32(*rollbackHeight), bestHeight, hashes)
2018-01-29 09:27:42 -07:00
if err != nil {
glog.Error("rollbackHeight: ", err)
return
2018-01-29 09:27:42 -07:00
}
}
return
}
2018-05-14 07:49:08 -06:00
if txCache, err = db.NewTxCache(index, chain, metrics, !*noTxCache); err != nil {
2018-03-06 04:36:24 -07:00
glog.Error("txCache ", err)
return
}
var internalServer *server.InternalServer
if *internalBinding != "" {
internalServer, err = server.NewInternalServer(*internalBinding, *certFiles, index, chain, txCache, internalState)
2018-01-18 08:44:31 -07:00
if err != nil {
glog.Error("https: ", err)
return
2018-01-18 08:44:31 -07:00
}
go func() {
err = internalServer.Run()
if err != nil {
2018-01-31 07:34:20 -07:00
if err.Error() == "http: Server closed" {
glog.Info("internal server: closed")
2018-01-31 07:34:20 -07:00
} else {
glog.Error(err)
return
2018-01-31 07:34:20 -07:00
}
}
}()
2018-01-18 08:44:31 -07:00
}
if *synchronize {
if err := syncWorker.ResyncIndex(nil); err != nil {
glog.Error("resyncIndex ", err)
return
}
2018-06-01 05:22:56 -06:00
if _, err = chain.ResyncMempool(nil); err != nil {
glog.Error("resyncMempool ", err)
return
}
}
var publicServer *server.PublicServer
if *publicBinding != "" {
publicServer, err = server.NewPublicServer(*publicBinding, *certFiles, index, chain, txCache, *explorerURL, metrics, internalState, *debugMode)
if err != nil {
glog.Error("socketio: ", err)
return
}
go func() {
err = publicServer.Run()
if err != nil {
if err.Error() == "http: Server closed" {
glog.Info("public server: closed")
} else {
glog.Error(err)
return
}
}
}()
callbacksOnNewBlock = append(callbacksOnNewBlock, publicServer.OnNewBlock)
callbacksOnNewTxAddr = append(callbacksOnNewTxAddr, publicServer.OnNewTxAddr)
}
if *synchronize {
// start the synchronization loops after the server interfaces are started
go syncIndexLoop()
go syncMempoolLoop()
2018-05-23 00:54:02 -06:00
go storeInternalStateLoop()
}
2018-03-01 10:37:01 -07:00
if *blockFrom >= 0 {
2017-08-28 09:50:57 -06:00
if *blockUntil < 0 {
2018-03-01 10:37:01 -07:00
*blockUntil = *blockFrom
2017-08-28 09:50:57 -06:00
}
2018-03-01 10:37:01 -07:00
height := uint32(*blockFrom)
2017-08-28 09:50:57 -06:00
until := uint32(*blockUntil)
address := *queryAddress
if address != "" {
if err = index.GetTransactions(address, height, until, printResult); err != nil {
glog.Error("GetTransactions ", err)
return
2017-08-28 09:50:57 -06:00
}
2018-01-31 07:03:06 -07:00
} else if !*synchronize {
2018-04-28 16:17:30 -06:00
if err = syncWorker.ConnectBlocksParallel(height, until); err != nil {
glog.Error("connectBlocksParallel ", err)
return
2017-08-28 09:50:57 -06:00
}
}
}
if internalServer != nil || publicServer != nil || chain != nil {
waitForSignalAndShutdown(internalServer, publicServer, chain, 10*time.Second)
}
2018-01-31 07:03:06 -07:00
2018-02-02 08:17:33 -07:00
if *synchronize {
close(chanSyncIndex)
close(chanSyncMempool)
2018-05-23 00:54:02 -06:00
close(chanStoreInternalState)
2018-02-02 08:17:33 -07:00
<-chanSyncIndexDone
<-chanSyncMempoolDone
2018-06-08 05:19:57 -06:00
<-chanStoreInternalStateDone
2018-02-02 08:17:33 -07:00
}
2018-01-31 07:03:06 -07:00
}
2018-06-27 16:36:56 -06:00
func newInternalState(coin string, coinShortcut string, d *db.RocksDB) (*common.InternalState, error) {
2018-05-30 06:44:40 -06:00
is, err := d.LoadInternalState(coin)
if err != nil {
return nil, err
}
2018-06-27 16:36:56 -06:00
is.CoinShortcut = coinShortcut
2018-05-30 06:44:40 -06:00
name, err := os.Hostname()
if err != nil {
glog.Error("get hostname ", err)
} else {
if i := strings.IndexByte(name, '.'); i > 0 {
name = name[:i]
}
2018-05-30 06:44:40 -06:00
is.Host = name
}
return is, nil
}
func tickAndDebounce(tickTime time.Duration, debounceTime time.Duration, input chan struct{}, f func()) {
timer := time.NewTimer(tickTime)
var firstDebounce time.Time
Loop:
for {
select {
case _, ok := <-input:
if !timer.Stop() {
<-timer.C
}
// exit loop on closed input channel
if !ok {
break Loop
}
if firstDebounce.IsZero() {
firstDebounce = time.Now()
}
// debounce for up to debounceTime period
// afterwards execute immediately
if firstDebounce.Add(debounceTime).After(time.Now()) {
timer.Reset(debounceTime)
} else {
timer.Reset(0)
}
case <-timer.C:
// do the action, if not in shutdown, then start the loop again
if atomic.LoadInt32(&inShutdown) == 0 {
f()
}
timer.Reset(tickTime)
firstDebounce = time.Time{}
}
}
}
2018-01-31 09:51:48 -07:00
func syncIndexLoop() {
defer close(chanSyncIndexDone)
glog.Info("syncIndexLoop starting")
// resync index about every 15 minutes if there are no chanSyncIndex requests, with debounce 1 second
tickAndDebounce(time.Duration(*resyncIndexPeriodMs)*time.Millisecond, debounceResyncIndexMs*time.Millisecond, chanSyncIndex, func() {
2018-03-01 10:37:01 -07:00
if err := syncWorker.ResyncIndex(onNewBlockHash); err != nil {
glog.Error("syncIndexLoop ", errors.ErrorStack(err))
2018-01-31 09:51:48 -07:00
}
})
2018-01-31 09:51:48 -07:00
glog.Info("syncIndexLoop stopped")
}
func onNewBlockHash(hash string, height uint32) {
for _, c := range callbacksOnNewBlock {
c(hash, height)
}
}
2018-01-31 09:51:48 -07:00
func syncMempoolLoop() {
defer close(chanSyncMempoolDone)
glog.Info("syncMempoolLoop starting")
// resync mempool about every minute if there are no chanSyncMempool requests, with debounce 1 second
tickAndDebounce(time.Duration(*resyncMempoolPeriodMs)*time.Millisecond, debounceResyncMempoolMs*time.Millisecond, chanSyncMempool, func() {
internalState.StartedMempoolSync()
2018-06-01 05:22:56 -06:00
if count, err := chain.ResyncMempool(onNewTxAddr); err != nil {
glog.Error("syncMempoolLoop ", errors.ErrorStack(err))
} else {
2018-06-01 05:22:56 -06:00
internalState.FinishedMempoolSync(count)
2018-01-31 09:51:48 -07:00
}
})
2018-01-31 09:51:48 -07:00
glog.Info("syncMempoolLoop stopped")
}
2018-05-23 00:54:02 -06:00
func storeInternalStateLoop() {
2018-06-08 05:19:57 -06:00
stopCompute := make(chan os.Signal)
defer func() {
close(stopCompute)
close(chanStoreInternalStateDone)
}()
var computeRunning bool
lastCompute := time.Now()
2018-06-12 13:24:18 -06:00
// randomize the duration between ComputeInternalStateColumnStats to avoid peaks after reboot of machine with multiple blockbooks
computePeriod := 9*time.Hour + time.Duration(rand.Float64()*float64((2*time.Hour).Nanoseconds()))
2018-08-26 13:29:10 -06:00
lastLogMemory := time.Now()
logMemoryPeriod := 15 * time.Minute
glog.Info("storeInternalStateLoop starting with db stats recompute period ", computePeriod)
2018-05-23 00:54:02 -06:00
tickAndDebounce(storeInternalStatePeriodMs*time.Millisecond, (storeInternalStatePeriodMs-1)*time.Millisecond, chanStoreInternalState, func() {
if !computeRunning && lastCompute.Add(computePeriod).Before(time.Now()) {
2018-06-08 05:19:57 -06:00
computeRunning = true
go func() {
err := index.ComputeInternalStateColumnStats(stopCompute)
if err != nil {
glog.Error("computeInternalStateColumnStats error: ", err)
}
lastCompute = time.Now()
computeRunning = false
}()
}
if err := index.StoreInternalState(internalState); err != nil {
2018-05-23 00:54:02 -06:00
glog.Error("storeInternalStateLoop ", errors.ErrorStack(err))
}
2018-08-26 13:29:10 -06:00
if lastLogMemory.Add(logMemoryPeriod).Before(time.Now()) {
glog.Info(index.GetMemoryStats())
lastLogMemory = time.Now()
}
2018-05-23 00:54:02 -06:00
})
glog.Info("storeInternalStateLoop stopped")
}
func onNewTxAddr(txid string, addr string, isOutput bool) {
for _, c := range callbacksOnNewTxAddr {
c(txid, addr, isOutput)
}
}
func pushSynchronizationHandler(nt bchain.NotificationType) {
if atomic.LoadInt32(&inShutdown) != 0 {
return
}
glog.V(1).Infof("MQ: notification ", nt)
if nt == bchain.NotificationNewBlock {
2018-01-31 09:51:48 -07:00
chanSyncIndex <- struct{}{}
} else if nt == bchain.NotificationNewTx {
2018-01-31 09:51:48 -07:00
chanSyncMempool <- struct{}{}
2018-01-31 07:03:06 -07:00
} else {
glog.Error("MQ: unknown notification sent")
2018-01-31 07:03:06 -07:00
}
}
func waitForSignalAndShutdown(internal *server.InternalServer, public *server.PublicServer, chain bchain.BlockChain, timeout time.Duration) {
2018-02-28 16:59:25 -07:00
sig := <-chanOsSignal
atomic.StoreInt32(&inShutdown, 1)
glog.Infof("shutdown: %v", sig)
ctx, cancel := context.WithTimeout(context.Background(), timeout)
defer cancel()
if internal != nil {
if err := internal.Shutdown(ctx); err != nil {
glog.Error("internal server: shutdown error: ", err)
2018-01-18 12:32:10 -07:00
}
}
if public != nil {
if err := public.Shutdown(ctx); err != nil {
glog.Error("public server: shutdown error: ", err)
}
}
if chain != nil {
if err := chain.Shutdown(ctx); err != nil {
glog.Error("rpc: shutdown error: ", err)
}
}
2017-08-28 09:50:57 -06:00
}
2018-02-03 11:14:27 -07:00
func printResult(txid string, vout uint32, isOutput bool) error {
glog.Info(txid, vout, isOutput)
2017-08-28 09:50:57 -06:00
return nil
}
func normalizeName(s string) string {
s = strings.ToLower(s)
s = strings.Replace(s, " ", "-", -1)
return s
}