Skip to content

Commit

Permalink
Merge pull request #375 from lazyledger/hlib/zero-bootstrapping
Browse files Browse the repository at this point in the history
  • Loading branch information
Wondertan committed Jun 11, 2021
2 parents 95af60b + 0b87d05 commit b7fe560
Show file tree
Hide file tree
Showing 19 changed files with 400 additions and 113 deletions.
1 change: 1 addition & 0 deletions cmd/tendermint/commands/light.go
Original file line number Diff line number Diff line change
Expand Up @@ -184,6 +184,7 @@ func runProxy(cmd *cobra.Command, args []string) error {
case daSampling:
cfg := ipfs.DefaultConfig()
cfg.RootDir = dir
cfg.ServeAPI = true
// TODO(ismail): share badger instance
apiProvider := ipfs.Embedded(true, cfg, logger)
var dag coreiface.APIDagService
Expand Down
4 changes: 3 additions & 1 deletion consensus/byzantine_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ import (
abcicli "github.com/lazyledger/lazyledger-core/abci/client"
abci "github.com/lazyledger/lazyledger-core/abci/types"
"github.com/lazyledger/lazyledger-core/evidence"
"github.com/lazyledger/lazyledger-core/ipfs"
"github.com/lazyledger/lazyledger-core/libs/db/memdb"
"github.com/lazyledger/lazyledger-core/libs/log"
"github.com/lazyledger/lazyledger-core/libs/service"
Expand Down Expand Up @@ -79,7 +80,8 @@ func TestByzantinePrevoteEquivocation(t *testing.T) {

// Make State
blockExec := sm.NewBlockExecutor(stateStore, log.TestingLogger(), proxyAppConnCon, mempool, evpool)
cs := NewState(thisConfig.Consensus, state, blockExec, blockStore, mempool, dag, evpool)
cs := NewState(thisConfig.Consensus, state, blockExec, blockStore,
mempool, dag, ipfs.MockRouting(), evpool)
cs.SetLogger(cs.Logger)
// set private validator
pv := privVals[i]
Expand Down
3 changes: 2 additions & 1 deletion consensus/common_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (
abci "github.com/lazyledger/lazyledger-core/abci/types"
cfg "github.com/lazyledger/lazyledger-core/config"
cstypes "github.com/lazyledger/lazyledger-core/consensus/types"
"github.com/lazyledger/lazyledger-core/ipfs"
tmbytes "github.com/lazyledger/lazyledger-core/libs/bytes"
dbm "github.com/lazyledger/lazyledger-core/libs/db"
"github.com/lazyledger/lazyledger-core/libs/db/memdb"
Expand Down Expand Up @@ -402,7 +403,7 @@ func newStateWithConfigAndBlockStore(
}

blockExec := sm.NewBlockExecutor(stateStore, log.TestingLogger(), proxyAppConnCon, mempool, evpool)
cs := NewState(thisConfig.Consensus, state, blockExec, blockStore, mempool, dag, evpool)
cs := NewState(thisConfig.Consensus, state, blockExec, blockStore, mempool, dag, ipfs.MockRouting(), evpool)
cs.SetLogger(log.TestingLogger().With("module", "consensus"))
cs.SetPrivValidator(pv)

Expand Down
4 changes: 3 additions & 1 deletion consensus/reactor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
cstypes "github.com/lazyledger/lazyledger-core/consensus/types"
cryptoenc "github.com/lazyledger/lazyledger-core/crypto/encoding"
"github.com/lazyledger/lazyledger-core/crypto/tmhash"
"github.com/lazyledger/lazyledger-core/ipfs"
"github.com/lazyledger/lazyledger-core/libs/bits"
"github.com/lazyledger/lazyledger-core/libs/bytes"
"github.com/lazyledger/lazyledger-core/libs/db/memdb"
Expand Down Expand Up @@ -184,7 +185,8 @@ func TestReactorWithEvidence(t *testing.T) {

// Make State
blockExec := sm.NewBlockExecutor(stateStore, log.TestingLogger(), proxyAppConnCon, mempool, evpool)
cs := NewState(thisConfig.Consensus, state, blockExec, blockStore, mempool, dag, evpool2)
cs := NewState(thisConfig.Consensus, state, blockExec, blockStore,
mempool, dag, ipfs.MockRouting(), evpool2)
cs.SetLogger(log.TestingLogger().With("module", "consensus"))
cs.SetPrivValidator(pv)

Expand Down
6 changes: 3 additions & 3 deletions consensus/replay_file.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ import (

mdutils "github.com/ipfs/go-merkledag/test"
cfg "github.com/lazyledger/lazyledger-core/config"
"github.com/lazyledger/lazyledger-core/ipfs"
"github.com/lazyledger/lazyledger-core/libs/db/badgerdb"
"github.com/lazyledger/lazyledger-core/libs/log"
tmos "github.com/lazyledger/lazyledger-core/libs/os"
Expand Down Expand Up @@ -130,7 +131,7 @@ func (pb *playback) replayReset(count int, newStepSub types.Subscription) error
pb.cs.Wait()

newCS := NewState(pb.cs.config, pb.genesisState.Copy(), pb.cs.blockExec,
pb.cs.blockStore, pb.cs.txNotifier, mdutils.Mock(), pb.cs.evpool)
pb.cs.blockStore, pb.cs.txNotifier, mdutils.Mock(), ipfs.MockRouting(), pb.cs.evpool)
newCS.SetEventBus(pb.cs.eventBus)
newCS.startForReplay()

Expand Down Expand Up @@ -331,8 +332,7 @@ func newConsensusStateForReplay(config cfg.BaseConfig, csConfig *cfg.ConsensusCo
blockExec := sm.NewBlockExecutor(stateStore, log.TestingLogger(), proxyApp.Consensus(), mempool, evpool)

consensusState := NewState(csConfig, state.Copy(), blockExec,
blockStore, mempool, dag, evpool)

blockStore, mempool, dag, ipfs.MockRouting(), evpool)
consensusState.SetEventBus(eventBus)
return consensusState
}
44 changes: 27 additions & 17 deletions consensus/state.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,8 @@ import (

"github.com/gogo/protobuf/proto"
format "github.com/ipfs/go-ipld-format"
"github.com/libp2p/go-libp2p-core/routing"

cfg "github.com/lazyledger/lazyledger-core/config"
cstypes "github.com/lazyledger/lazyledger-core/consensus/types"
"github.com/lazyledger/lazyledger-core/crypto"
Expand Down Expand Up @@ -94,7 +96,8 @@ type State struct {
// store blocks and commits
blockStore sm.BlockStore

dag format.DAGService
dag format.DAGService
croute routing.ContentRouting

// create and execute blocks
blockExec *sm.BlockExecutor
Expand Down Expand Up @@ -151,6 +154,10 @@ type State struct {

// for reporting metrics
metrics *Metrics

// context of the recent proposed block
proposalCtx context.Context
proposalCancel context.CancelFunc
}

// StateOption sets an optional parameter on the State.
Expand All @@ -164,6 +171,7 @@ func NewState(
blockStore sm.BlockStore,
txNotifier txNotifier,
dag format.DAGService,
croute routing.ContentRouting,
evpool evidencePool,
options ...StateOption,
) *State {
Expand All @@ -172,6 +180,7 @@ func NewState(
blockExec: blockExec,
blockStore: blockStore,
dag: dag,
croute: croute,
txNotifier: txNotifier,
peerMsgQueue: make(chan msgInfo, msgQueueSize),
internalMsgQueue: make(chan msgInfo, msgQueueSize),
Expand Down Expand Up @@ -1112,23 +1121,24 @@ func (cs *State) defaultDecideProposal(height int64, round int32) {
cs.Logger.Error("enterPropose: Error signing proposal", "height", height, "round", round, "err", err)
}

// TODO(ismail): capture this in the Consensus ADR
// post data to ipfs
// TODO(evan): don't hard code context and timeout
//
// longer timeouts result in block proposers failing to propose blocks in time.
ctx, cancel := context.WithTimeout(context.Background(), time.Millisecond*1500)
defer cancel()
cs.Logger.Info("Putting Block to ipfs", "height", block.Height)
// TODO: post data to IPFS in a goroutine
err = ipld.PutBlock(ctx, cs.dag, block)
if err != nil {
// If PutBlock fails we will be the only node that has the data
// this means something is seriously wrong and we can not recover
// from that automatically.
panic(fmt.Sprintf("failure to post block data to IPFS: %s", err.Error()))
// cancel ctx for previous proposal block to ensure block putting/providing does not queues up
if cs.proposalCancel != nil {
cs.proposalCancel()
}
cs.Logger.Info("Finished putting block to ipfs", "height", block.Height)
cs.proposalCtx, cs.proposalCancel = context.WithCancel(context.TODO())
go func(ctx context.Context) {
cs.Logger.Info("Putting Block to IPFS", "height", block.Height)
err = ipld.PutBlock(ctx, cs.dag, block, cs.croute, cs.Logger)
if err != nil {
if errors.Is(err, context.Canceled) {
cs.Logger.Error("Putting Block didn't finish in time and was terminated", "height", block.Height)
return
}
cs.Logger.Error("Failed to put Block to IPFS", "err", err, "height", block.Height)
return
}
cs.Logger.Info("Finished putting block to IPFS", "height", block.Height)
}(cs.proposalCtx)
}

// Returns true if the proposal block is complete &&
Expand Down
20 changes: 10 additions & 10 deletions consensus/wal_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -8,26 +8,25 @@ import (
"io"
mrand "math/rand"
"path/filepath"

// "sync"
"testing"
"time"

mdutils "github.com/ipfs/go-merkledag/test"
"github.com/lazyledger/lazyledger-core/abci/example/kvstore"
cfg "github.com/lazyledger/lazyledger-core/config"
"github.com/lazyledger/lazyledger-core/libs/db/memdb"
"github.com/lazyledger/lazyledger-core/privval"
"github.com/lazyledger/lazyledger-core/proxy"
sm "github.com/lazyledger/lazyledger-core/state"
"github.com/lazyledger/lazyledger-core/store"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"

"github.com/lazyledger/lazyledger-core/abci/example/kvstore"
cfg "github.com/lazyledger/lazyledger-core/config"
"github.com/lazyledger/lazyledger-core/consensus/types"
"github.com/lazyledger/lazyledger-core/crypto/merkle"
"github.com/lazyledger/lazyledger-core/ipfs"
"github.com/lazyledger/lazyledger-core/libs/autofile"
"github.com/lazyledger/lazyledger-core/libs/db/memdb"
"github.com/lazyledger/lazyledger-core/libs/log"
"github.com/lazyledger/lazyledger-core/privval"
"github.com/lazyledger/lazyledger-core/proxy"
sm "github.com/lazyledger/lazyledger-core/state"
"github.com/lazyledger/lazyledger-core/store"
tmtypes "github.com/lazyledger/lazyledger-core/types"
tmtime "github.com/lazyledger/lazyledger-core/types/time"
)
Expand Down Expand Up @@ -339,7 +338,8 @@ func walGenerateNBlocks(t *testing.T, wr io.Writer, numBlocks int) (err error) {
evpool := sm.EmptyEvidencePool{}
blockExec := sm.NewBlockExecutor(stateStore, log.TestingLogger(), proxyApp.Consensus(), mempool, evpool)
require.NoError(t, err)
consensusState := NewState(config.Consensus, state.Copy(), blockExec, blockStore, mempool, dag, evpool)
consensusState := NewState(config.Consensus, state.Copy(), blockExec, blockStore,
mempool, dag, ipfs.MockRouting(), evpool)
consensusState.SetLogger(logger)
consensusState.SetEventBus(eventBus)
if privValidator != nil && privValidator != (*privval.FilePV)(nil) {
Expand Down
9 changes: 9 additions & 0 deletions go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -17,11 +17,16 @@ require (
github.com/gorilla/websocket v1.4.2
github.com/gtank/merlin v0.1.1
github.com/hdevalence/ed25519consensus v0.0.0-20201207055737-7fde80a9d5ff
github.com/ipfs/go-bitswap v0.3.3 // indirect
github.com/ipfs/go-block-format v0.0.2
github.com/ipfs/go-blockservice v0.1.4 // indirect
github.com/ipfs/go-cid v0.0.7
github.com/ipfs/go-datastore v0.4.5 // indirect
github.com/ipfs/go-ipfs v0.8.0
github.com/ipfs/go-ipfs-api v0.2.0
github.com/ipfs/go-ipfs-blockstore v0.1.4 // indirect
github.com/ipfs/go-ipfs-config v0.11.0
github.com/ipfs/go-ipfs-routing v0.1.0 // indirect
github.com/ipfs/go-ipld-format v0.2.0
github.com/ipfs/go-merkledag v0.3.2
github.com/ipfs/go-path v0.0.9 // indirect
Expand All @@ -30,6 +35,10 @@ require (
github.com/lazyledger/nmt v0.5.0
github.com/lazyledger/rsmt2d v0.2.0
github.com/libp2p/go-buffer-pool v0.0.2
github.com/libp2p/go-libp2p v0.12.0 // indirect
github.com/libp2p/go-libp2p-core v0.7.0 // indirect
github.com/libp2p/go-libp2p-kad-dht v0.11.1 // indirect
github.com/libp2p/go-libp2p-kbucket v0.4.7 // indirect
github.com/minio/highwayhash v1.0.1
github.com/multiformats/go-multiaddr v0.3.1
github.com/multiformats/go-multihash v0.0.14
Expand Down
9 changes: 6 additions & 3 deletions ipfs/defaults.go
Original file line number Diff line number Diff line change
Expand Up @@ -125,10 +125,16 @@ func DefaultFullNodeConfig() (*ipfscfg.Config, error) {
// In kDHT all records have TTL, thus we have to regularly(Interval) reprovide/reannounce stored CID to the
// network. Otherwise information that the node stores something will be lost. Should be in tact with kDHT
// record cleaning configuration
// TODO(Wondertan) In case StrategicProviding is true, we have to implement reproviding manually.
Reprovider: ipfscfg.Reprovider{
Interval: "12h",
Strategy: "all",
},
// List of all experimental IPFS features
Experimental: ipfscfg.Experiments{
// Disables BitSwap providing and reproviding in favour of manual providing.
StrategicProviding: true,
},
Routing: ipfscfg.Routing{
// Full node must be available from the WAN thus 'dhtserver'
// Depending on the node type/use-case different modes should be used.
Expand All @@ -152,9 +158,6 @@ func DefaultFullNodeConfig() (*ipfscfg.Config, error) {
// Unused fields:
// we currently don't use PubSub
Pubsub: ipfscfg.PubsubConfig{},
// List of all experimental IPFS features
// We currently don't use any of them
Experimental: ipfscfg.Experiments{},
// bi-directional agreement between peers to hold connections with
Peering: ipfscfg.Peering{},
// local network discovery is useful, but there is no practical reason to have two FullNode in one LAN
Expand Down
3 changes: 1 addition & 2 deletions ipfs/embedded.go
Original file line number Diff line number Diff line change
Expand Up @@ -4,7 +4,6 @@ import (
"context"
"errors"
"fmt"
"io"
"os"
"sync"

Expand All @@ -26,7 +25,7 @@ import (
// Embedded is the provider that embeds IPFS node within the same process.
// It also returns closable for graceful node shutdown.
func Embedded(init bool, cfg *Config, logger log.Logger) APIProvider {
return func() (coreiface.APIDagService, io.Closer, error) {
return func() (coreiface.APIDagService, *core.IpfsNode, error) {
path := cfg.Path()
defer os.Setenv(ipfscfg.EnvDir, path)

Expand Down
40 changes: 34 additions & 6 deletions ipfs/mock.go
Original file line number Diff line number Diff line change
@@ -1,26 +1,54 @@
package ipfs

import (
"io"
"context"

nilrouting "github.com/ipfs/go-ipfs-routing/none"
"github.com/ipfs/go-ipfs/core"
coremock "github.com/ipfs/go-ipfs/core/mock"
ipld "github.com/ipfs/go-ipld-format"
mdutils "github.com/ipfs/go-merkledag/test"
coreiface "github.com/ipfs/interface-go-ipfs-core"
"github.com/libp2p/go-libp2p-core/routing"
mocknet "github.com/libp2p/go-libp2p/p2p/net/mock"

"github.com/lazyledger/lazyledger-core/ipfs/plugin"
)

// Mock provides simple mock IPFS API useful for testing
func Mock() APIProvider {
return func() (coreiface.APIDagService, io.Closer, error) {
dom := dagOnlyMock{mdutils.Mock()}
return func() (coreiface.APIDagService, *core.IpfsNode, error) {
plugin.EnableNMT()

nd, err := MockNode()
if err != nil {
return nil, nil, err
}

return dom, dom, nil
return dagOnlyMock{nd.DAG}, nd, nil
}
}

func MockNode() (*core.IpfsNode, error) {
ctx := context.TODO()
nd, err := core.NewNode(ctx, &core.BuildCfg{
Online: true,
Host: coremock.MockHostOption(mocknet.New(ctx)),
})
if err != nil {
return nil, err
}
nd.Routing = MockRouting()
return nd, err
}

func MockRouting() routing.Routing {
croute, _ := nilrouting.ConstructNilRouting(context.TODO(), nil, nil, nil)
return croute
}

type dagOnlyMock struct {
ipld.DAGService
}

func (dom dagOnlyMock) Dag() coreiface.APIDagService { return dom }
func (dagOnlyMock) Close() error { return nil }
func (dom dagOnlyMock) Pinning() ipld.NodeAdder { return dom }
5 changes: 2 additions & 3 deletions ipfs/provider.go
Original file line number Diff line number Diff line change
@@ -1,10 +1,9 @@
package ipfs

import (
"io"

"github.com/ipfs/go-ipfs/core"
coreiface "github.com/ipfs/interface-go-ipfs-core"
)

// APIProvider allows customizable IPFS core APIs.
type APIProvider func() (coreiface.APIDagService, io.Closer, error)
type APIProvider func() (coreiface.APIDagService, *core.IpfsNode, error)
Loading

0 comments on commit b7fe560

Please sign in to comment.