From acbd7fda60c696dd51beb130c00b633f6c9073ea Mon Sep 17 00:00:00 2001 From: Hector Sanjuan Date: Wed, 20 Feb 2019 14:24:25 +0000 Subject: [PATCH] Consensus: add new "crdt" consensus component This adds a new "crdt" consensus component using go-ds-crdt. This implies several refactors to fully make cluster consensus-component independent: * Delete mapstate and fully adopt dsstate (after people have migrated). * Return errors from state methods rather than ignoring them. * Add a new "datastore" modules so that we can configure datastores in the main configuration like other components. * Let the consensus components fully define the "state.State". Thus, they do not receive the state, they receive the storage where we put the state (a go-datastore). * Allow to customize how the monitor component obtains Peers() (the current peerset), including avoiding using the current peerset. At the moment the crdt consensus uses the monitoring component to define the current peerset. Therefore the monitor component cannot rely on the consensus component to produce a peerset. * Re-factor/re-implementation of "ipfs-cluster-service state" operations. Includes the dissapearance of the "migrate" one. The CRDT consensus component defines creates a crdt-datastore (with ipfs-lite) and uses it to intitialize a dssate. Thus the crdt-store is elegantly wrapped. Any modifications to the state get automatically replicated to other peers. We store all the CRDT DAG blocks in the local datastore. The consensus components only expose a ReadOnly state, as any modifications to the shared state should happen through them. DHT and PubSub facilities must now be created outside of Cluster and passed in so they can be re-used by different components. --- .travis.yml | 3 + api/ipfsproxy/ipfsproxy_test.go | 2 +- api/rest/client/client_test.go | 2 +- api/rest/restapi.go | 10 +- api/types.go | 19 +- cluster.go | 144 +++--- cluster_test.go | 84 ++-- clusterhost.go | 62 ++- cmd/ipfs-cluster-ctl/graph.go | 2 +- cmd/ipfs-cluster-service/configs.go | 27 +- cmd/ipfs-cluster-service/daemon.go | 162 +++++-- cmd/ipfs-cluster-service/lock.go | 46 +- cmd/ipfs-cluster-service/main.go | 174 +++---- cmd/ipfs-cluster-service/state.go | 290 ++++++------ config/config.go | 4 + config/config_test.go | 5 + config_test.go | 35 +- consensus/crdt/config.go | 152 +++++++ consensus/crdt/config_test.go | 71 +++ consensus/crdt/consensus.go | 425 ++++++++++++++++++ consensus/crdt/consensus_test.go | 293 ++++++++++++ consensus/crdt/dagsyncer.go | 73 +++ consensus/raft/config.go | 16 +- consensus/raft/consensus.go | 82 +++- consensus/raft/consensus_test.go | 28 +- consensus/raft/log_op_test.go | 42 +- consensus/raft/logging.go | 2 +- consensus/raft/raft.go | 9 +- datastore/badger/badger.go | 33 ++ datastore/badger/config.go | 113 +++++ datastore/inmem/inmem.go | 14 + go.mod | 70 ++- go.sum | 204 +++++++-- ipfscluster.go | 2 +- ipfscluster_test.go | 340 ++++++++++---- ipfsconn/ipfshttp/ipfshttp.go | 21 + ipfsconn/ipfshttp/ipfshttp_test.go | 2 +- logging.go | 21 +- monitor/metrics/checker.go | 51 ++- monitor/metrics/store.go | 28 +- monitor/metrics/store_test.go | 4 +- monitor/pubsubmon/pubsubmon.go | 64 ++- monitor/pubsubmon/pubsubmon_test.go | 49 +- package.json | 6 + peer_manager_test.go | 291 +++++++----- pintracker/maptracker/maptracker.go | 2 +- pnet_test.go | 3 +- pstoremgr/pstoremgr.go | 10 +- rpc_api.go | 5 +- ...0050-service-state-upgrade-from-current.sh | 32 -- .../t0051-service-state-upgrade-from-old.sh | 35 -- sharness/t0052-service-state-export.sh | 12 +- sharness/test_data/importState | 17 +- sharness/test_data/v1State | Bin 79 -> 0 bytes state/dsstate/datastore.go | 111 +++-- .../datastore_test.go} | 114 ++--- state/empty.go | 28 ++ state/interface.go | 61 ++- state/mapstate/map_state.go | 178 -------- state/mapstate/migrate.go | 280 ------------ test/ipfs_mock.go | 28 +- test/test_test.go | 2 +- 62 files changed, 2990 insertions(+), 1505 deletions(-) create mode 100644 consensus/crdt/config.go create mode 100644 consensus/crdt/config_test.go create mode 100644 consensus/crdt/consensus.go create mode 100644 consensus/crdt/consensus_test.go create mode 100644 consensus/crdt/dagsyncer.go create mode 100644 datastore/badger/badger.go create mode 100644 datastore/badger/config.go create mode 100644 datastore/inmem/inmem.go delete mode 100755 sharness/t0050-service-state-upgrade-from-current.sh delete mode 100755 sharness/t0051-service-state-upgrade-from-old.sh delete mode 100755 sharness/test_data/v1State rename state/{mapstate/map_state_test.go => dsstate/datastore_test.go} (56%) create mode 100644 state/empty.go delete mode 100644 state/mapstate/map_state.go delete mode 100644 state/mapstate/migrate.go diff --git a/.travis.yml b/.travis.yml index bef539e7..43efa34a 100644 --- a/.travis.yml +++ b/.travis.yml @@ -28,6 +28,9 @@ jobs: - go test -v -coverprofile=coverage.txt -covermode=atomic ./... after_success: - bash <(curl -s https://codecov.io/bash) + - name: "Main Tests with crdt consensus" + script: + - go test -v . -consensus crdt - name: "Main Tests with stateless tracker" script: - go test -v . -tracker stateless diff --git a/api/ipfsproxy/ipfsproxy_test.go b/api/ipfsproxy/ipfsproxy_test.go index 8c8c79ad..3fb3cf0d 100644 --- a/api/ipfsproxy/ipfsproxy_test.go +++ b/api/ipfsproxy/ipfsproxy_test.go @@ -25,7 +25,7 @@ func init() { } func testIPFSProxyWithConfig(t *testing.T, cfg *Config) (*Server, *test.IpfsMock) { - mock := test.NewIpfsMock() + mock := test.NewIpfsMock(t) nodeMAddr, _ := ma.NewMultiaddr(fmt.Sprintf("/ip4/%s/tcp/%d", mock.Addr, mock.Port)) proxyMAddr, _ := ma.NewMultiaddr("/ip4/127.0.0.1/tcp/0") diff --git a/api/rest/client/client_test.go b/api/rest/client/client_test.go index 9db22077..24121a08 100644 --- a/api/rest/client/client_test.go +++ b/api/rest/client/client_test.go @@ -267,7 +267,7 @@ func TestProxyAddress(t *testing.T) { func TestIPFS(t *testing.T) { ctx := context.Background() - ipfsMock := test.NewIpfsMock() + ipfsMock := test.NewIpfsMock(t) defer ipfsMock.Close() proxyAddr, err := ma.NewMultiaddr( diff --git a/api/rest/restapi.go b/api/rest/restapi.go index 5ec4e7cb..eaf1e532 100644 --- a/api/rest/restapi.go +++ b/api/rest/restapi.go @@ -20,12 +20,6 @@ import ( "sync" "time" - "github.com/rs/cors" - - "go.opencensus.io/plugin/ochttp" - "go.opencensus.io/plugin/ochttp/propagation/tracecontext" - "go.opencensus.io/trace" - "github.com/ipfs/ipfs-cluster/adder/adderutils" types "github.com/ipfs/ipfs-cluster/api" @@ -41,6 +35,10 @@ import ( peer "github.com/libp2p/go-libp2p-peer" ma "github.com/multiformats/go-multiaddr" manet "github.com/multiformats/go-multiaddr-net" + "github.com/rs/cors" + "go.opencensus.io/plugin/ochttp" + "go.opencensus.io/plugin/ochttp/propagation/tracecontext" + "go.opencensus.io/trace" ) func init() { diff --git a/api/types.go b/api/types.go index 0e2a5856..4d3ccd44 100644 --- a/api/types.go +++ b/api/types.go @@ -452,8 +452,17 @@ type PinOptions struct { Metadata map[string]string `json:"metadata" codec:"m,omitempty"` } -// Equals returns true if two PinOption objects are equivalent. +// Equals returns true if two PinOption objects are equivalent. po and po2 may +// be nil. func (po *PinOptions) Equals(po2 *PinOptions) bool { + if po == nil && po2 != nil || po2 == nil && po != nil { + return false + } + + if po == po2 { // same as pin.Equals() + return false + } + if po.ReplicationFactorMax != po2.ReplicationFactorMax { return false } @@ -692,11 +701,19 @@ func (pin *Pin) ProtoUnmarshal(data []byte) error { // Equals checks if two pins are the same (with the same allocations). // If allocations are the same but in different order, they are still // considered equivalent. +// pin or pin2 may be nil. If both are nil, Equals returns false. func (pin *Pin) Equals(pin2 *Pin) bool { if pin == nil && pin2 != nil || pin2 == nil && pin != nil { return false } + if pin == pin2 { + // ask @lanzafame why this is not true + // in any case, this is anomalous and we should + // not be using this with two nils. + return false + } + if !pin.Cid.Equals(pin2.Cid) { return false } diff --git a/cluster.go b/cluster.go index 82110b5c..d917e603 100644 --- a/cluster.go +++ b/cluster.go @@ -17,17 +17,16 @@ import ( "github.com/ipfs/ipfs-cluster/state" "github.com/ipfs/ipfs-cluster/version" - ocgorpc "github.com/lanzafame/go-libp2p-ocgorpc" - - "go.opencensus.io/trace" - cid "github.com/ipfs/go-cid" + ds "github.com/ipfs/go-datastore" rpc "github.com/libp2p/go-libp2p-gorpc" host "github.com/libp2p/go-libp2p-host" dht "github.com/libp2p/go-libp2p-kad-dht" peer "github.com/libp2p/go-libp2p-peer" - routedhost "github.com/libp2p/go-libp2p/p2p/host/routed" ma "github.com/multiformats/go-multiaddr" + + ocgorpc "github.com/lanzafame/go-libp2p-ocgorpc" + trace "go.opencensus.io/trace" ) // ReadyTimeout specifies the time before giving up @@ -44,10 +43,12 @@ type Cluster struct { ctx context.Context cancel func() - id peer.ID - config *Config - host host.Host - dht *dht.IpfsDHT + id peer.ID + config *Config + host host.Host + dht *dht.IpfsDHT + datastore ds.Datastore + rpcServer *rpc.Server rpcClient *rpc.Client peerManager *pstoremgr.Manager @@ -55,7 +56,6 @@ type Cluster struct { consensus Consensus apis []API ipfs IPFSConnector - state state.State tracker PinTracker monitor PeerMonitor allocator PinAllocator @@ -84,11 +84,12 @@ type Cluster struct { // if you need to wait until the peer is fully up. func NewCluster( host host.Host, + dht *dht.IpfsDHT, cfg *Config, + datastore ds.Datastore, consensus Consensus, apis []API, ipfs IPFSConnector, - st state.State, tracker PinTracker, monitor PeerMonitor, allocator PinAllocator, @@ -117,32 +118,17 @@ func NewCluster( // in daemon.go. peerManager := pstoremgr.New(host, cfg.GetPeerstorePath()) - idht, err := dht.New(ctx, host) - if err != nil { - cancel() - return nil, err - } - - // Let the DHT be maintained regularly - err = idht.Bootstrap(ctx) - if err != nil { - cancel() - return nil, err - } - - rHost := routedhost.Wrap(host, idht) - c := &Cluster{ ctx: ctx, cancel: cancel, id: host.ID(), config: cfg, - host: rHost, - dht: idht, + host: host, + dht: dht, + datastore: datastore, consensus: consensus, apis: apis, ipfs: ipfs, - state: st, tracker: tracker, monitor: monitor, allocator: allocator, @@ -378,7 +364,11 @@ func (c *Cluster) repinFromPeer(ctx context.Context, p peer.ID) { logger.Warning(err) return } - list := cState.List(ctx) + list, err := cState.List(ctx) + if err != nil { + logger.Warning(err) + return + } for _, pin := range list { if containsPeer(pin.Allocations, p) { _, ok, err := c.pin(ctx, pin, []peer.ID{p}, []peer.ID{}) // pin blacklisting this peer @@ -435,6 +425,10 @@ This might be due to one or several causes: } // Cluster is ready. + + // Bootstrap the DHT now that we possibly have some connections + c.dht.Bootstrap(c.ctx) + peers, err := c.consensus.Peers(ctx) if err != nil { logger.Error(err) @@ -512,7 +506,8 @@ func (c *Cluster) Shutdown(ctx context.Context) error { } } - // We left the cluster or were removed. Destroy the Raft state. + // We left the cluster or were removed. Remove any consensus-specific + // state. if c.removed && c.readyB { err := c.consensus.Clean(ctx) if err != nil { @@ -550,6 +545,13 @@ func (c *Cluster) Shutdown(ctx context.Context) error { c.cancel() c.host.Close() // Shutdown all network services c.wg.Wait() + + // Cleanly close the datastore + if err := c.datastore.Close(); err != nil { + logger.Errorf("error closing Datastore: %s", err) + return err + } + c.shutdownB = true close(c.doneCh) return nil @@ -627,7 +629,7 @@ func (c *Cluster) PeerAdd(ctx context.Context, pid peer.ID) (*api.ID, error) { defer c.paMux.Unlock() logger.Debugf("peerAdd called with %s", pid.Pretty()) - // Log the new peer in the log so everyone gets it. + // Let the consensus layer be aware of this peer err := c.consensus.AddPeer(ctx, pid) if err != nil { logger.Error(err) @@ -635,6 +637,26 @@ func (c *Cluster) PeerAdd(ctx context.Context, pid peer.ID) (*api.ID, error) { return id, err } + // Send a ping metric to the new node directly so + // it knows about this one at least + m := &api.Metric{ + Name: pingMetricName, + Peer: c.id, + Valid: true, + } + m.SetTTL(c.config.MonitorPingInterval * 2) + err = c.rpcClient.CallContext( + ctx, + pid, + "Cluster", + "PeerMonitorLogMetric", + m, + &struct{}{}, + ) + if err != nil { + logger.Warning(err) + } + // Ask the new peer to connect its IPFS daemon to the rest err = c.rpcClient.CallContext( ctx, @@ -645,7 +667,7 @@ func (c *Cluster) PeerAdd(ctx context.Context, pid peer.ID) (*api.ID, error) { &struct{}{}, ) if err != nil { - logger.Error(err) + logger.Warning(err) } id := &api.ID{} @@ -775,7 +797,10 @@ func (c *Cluster) StateSync(ctx context.Context) error { } logger.Debug("syncing state to tracker") - clusterPins := cState.List(ctx) + clusterPins, err := cState.List(ctx) + if err != nil { + return err + } trackedPins := c.tracker.StatusAll(ctx) trackedPinsMap := make(map[string]int) @@ -797,13 +822,20 @@ func (c *Cluster) StateSync(ctx context.Context) error { // c. Track items which should not be local as remote for _, p := range trackedPins { pCid := p.Cid - currentPin, has := cState.Get(ctx, pCid) + currentPin, err := cState.Get(ctx, pCid) + if err != nil && err != state.ErrNotFound { + return err + } + + if err == state.ErrNotFound { + logger.Debugf("StateSync: untracking %s: not part of shared state", pCid) + c.tracker.Untrack(ctx, pCid) + continue + } + allocatedHere := containsPeer(currentPin.Allocations, c.id) || currentPin.ReplicationFactorMin == -1 switch { - case !has: - logger.Debugf("StateSync: Untracking %s, is not part of shared state", pCid) - c.tracker.Untrack(ctx, pCid) case p.Status == api.TrackerStatusRemote && allocatedHere: logger.Debugf("StateSync: Tracking %s locally (currently remote)", pCid) c.tracker.Track(ctx, currentPin) @@ -970,7 +1002,7 @@ func (c *Cluster) RecoverLocal(ctx context.Context, h cid.Cid) (pInfo *api.PinIn // of the current global state. This is the source of truth as to which // pins are managed and their allocation, but does not indicate if // the item is successfully pinned. For that, use StatusAll(). -func (c *Cluster) Pins(ctx context.Context) []*api.Pin { +func (c *Cluster) Pins(ctx context.Context) ([]*api.Pin, error) { _, span := trace.StartSpan(ctx, "cluster/Pins") defer span.End() ctx = trace.NewContext(c.ctx, span) @@ -978,7 +1010,7 @@ func (c *Cluster) Pins(ctx context.Context) []*api.Pin { cState, err := c.consensus.State(ctx) if err != nil { logger.Error(err) - return []*api.Pin{} + return nil, err } return cState.List(ctx) } @@ -998,9 +1030,9 @@ func (c *Cluster) PinGet(ctx context.Context, h cid.Cid) (*api.Pin, error) { if err != nil { return nil, err } - pin, ok := st.Get(ctx, h) - if !ok { - return pin, errors.New("cid is not part of the global state") + pin, err := st.Get(ctx, h) + if err != nil { + return nil, err } return pin, nil } @@ -1097,9 +1129,17 @@ func (c *Cluster) setupPin(ctx context.Context, pin *api.Pin) error { } existing, err := c.PinGet(ctx, pin.Cid) - if err == nil && existing.Type != pin.Type { // it exists - return fmt.Errorf("cannot repin CID with different tracking method, clear state with pin rm to proceed. New: %s. Was: %s", pin.Type, existing.Type) + if err != nil && err != state.ErrNotFound { + return err } + + if existing != nil && existing.Type != pin.Type { + msg := "cannot repin CID with different tracking method, " + msg += "clear state with pin rm to proceed. " + msg += "New: %s. Was: %s" + return fmt.Errorf(msg, pin.Type, existing.Type) + } + return checkPinType(pin) } @@ -1137,6 +1177,7 @@ func (c *Cluster) pin(ctx context.Context, pin *api.Pin, blacklist []peer.ID, pr } pin.Allocations = allocs + // Equals can handle nil objects. if curr, _ := c.PinGet(ctx, pin.Cid); curr.Equals(pin) { // skip pinning logger.Debugf("pinning %s skipped: already correctly allocated", pin.Cid) @@ -1144,9 +1185,9 @@ func (c *Cluster) pin(ctx context.Context, pin *api.Pin, blacklist []peer.ID, pr } if len(pin.Allocations) == 0 { - logger.Infof("IPFS cluster pinning %s everywhere:", pin.Cid) + logger.Infof("pinning %s everywhere:", pin.Cid) } else { - logger.Infof("IPFS cluster pinning %s on %s:", pin.Cid, pin.Allocations) + logger.Infof("pinning %s on %s:", pin.Cid, pin.Allocations) } return pin, true, c.consensus.LogPin(ctx, pin) @@ -1160,7 +1201,7 @@ func (c *Cluster) unpin(ctx context.Context, h cid.Cid) (*api.Pin, error) { logger.Info("IPFS cluster unpinning:", h) pin, err := c.PinGet(ctx, h) if err != nil { - return pin, fmt.Errorf("cannot unpin pin uncommitted to state: %s", err) + return nil, err } switch pin.Type { @@ -1482,8 +1523,11 @@ func (c *Cluster) cidsFromMetaPin(ctx context.Context, h cid.Cid) ([]cid.Cid, er list := []cid.Cid{h} - pin, ok := cState.Get(ctx, h) - if !ok { + pin, err := cState.Get(ctx, h) + if err != nil { + return nil, err + } + if pin == nil { return list, nil } diff --git a/cluster_test.go b/cluster_test.go index 1a76a692..97255bcd 100644 --- a/cluster_test.go +++ b/cluster_test.go @@ -14,10 +14,10 @@ import ( "github.com/ipfs/ipfs-cluster/allocator/ascendalloc" "github.com/ipfs/ipfs-cluster/api" "github.com/ipfs/ipfs-cluster/consensus/raft" + "github.com/ipfs/ipfs-cluster/datastore/inmem" "github.com/ipfs/ipfs-cluster/informer/numpin" "github.com/ipfs/ipfs-cluster/monitor/pubsubmon" "github.com/ipfs/ipfs-cluster/state" - "github.com/ipfs/ipfs-cluster/state/mapstate" "github.com/ipfs/ipfs-cluster/test" "github.com/ipfs/ipfs-cluster/version" @@ -138,10 +138,10 @@ type mockTracer struct { mockComponent } -func testingCluster(t *testing.T) (*Cluster, *mockAPI, *mockConnector, state.State, PinTracker) { - clusterCfg, _, _, _, consensusCfg, maptrackerCfg, statelesstrackerCfg, psmonCfg, _, _ := testingConfigs() +func testingCluster(t *testing.T) (*Cluster, *mockAPI, *mockConnector, PinTracker) { + clusterCfg, _, _, _, _, raftCfg, _, maptrackerCfg, statelesstrackerCfg, psmonCfg, _, _ := testingConfigs() - host, err := NewClusterHost(context.Background(), clusterCfg) + host, pubsub, dht, err := NewClusterHost(context.Background(), clusterCfg) if err != nil { t.Fatal(err) } @@ -149,14 +149,14 @@ func testingCluster(t *testing.T) (*Cluster, *mockAPI, *mockConnector, state.Sta api := &mockAPI{} proxy := &mockProxy{} ipfs := &mockConnector{} - st := mapstate.NewMapState() tracker := makePinTracker(t, clusterCfg.ID, maptrackerCfg, statelesstrackerCfg, clusterCfg.Peername) tracer := &mockTracer{} - raftcon, _ := raft.NewConsensus(host, consensusCfg, st, false) + store := inmem.New() + raftcon, _ := raft.NewConsensus(host, raftCfg, store, false) psmonCfg.CheckInterval = 2 * time.Second - mon, err := pubsubmon.New(host, psmonCfg) + mon, err := pubsubmon.New(psmonCfg, pubsub, raftcon.Peers) if err != nil { t.Fatal(err) } @@ -166,15 +166,16 @@ func testingCluster(t *testing.T) (*Cluster, *mockAPI, *mockConnector, state.Sta numpinCfg.Default() inf, _ := numpin.NewInformer(numpinCfg) - ReadyTimeout = consensusCfg.WaitForLeaderTimeout + 1*time.Second + ReadyTimeout = raftCfg.WaitForLeaderTimeout + 1*time.Second cl, err := NewCluster( host, + dht, clusterCfg, + store, raftcon, []API{api, proxy}, ipfs, - st, tracker, mon, alloc, @@ -185,7 +186,7 @@ func testingCluster(t *testing.T) (*Cluster, *mockAPI, *mockConnector, state.Sta t.Fatal("cannot create cluster:", err) } <-cl.Ready() - return cl, api, ipfs, st, tracker + return cl, api, ipfs, tracker } func cleanRaft() { @@ -197,13 +198,13 @@ func cleanRaft() { func testClusterShutdown(t *testing.T) { ctx := context.Background() - cl, _, _, _, _ := testingCluster(t) + cl, _, _, _ := testingCluster(t) err := cl.Shutdown(ctx) if err != nil { t.Error("cluster shutdown failed:", err) } cl.Shutdown(ctx) - cl, _, _, _, _ = testingCluster(t) + cl, _, _, _ = testingCluster(t) err = cl.Shutdown(ctx) if err != nil { t.Error("cluster shutdown failed:", err) @@ -213,16 +214,12 @@ func testClusterShutdown(t *testing.T) { func TestClusterStateSync(t *testing.T) { ctx := context.Background() cleanRaft() - cl, _, _, st, _ := testingCluster(t) + cl, _, _, _ := testingCluster(t) defer cleanRaft() defer cl.Shutdown(ctx) - err := cl.StateSync(ctx) - if err == nil { - t.Fatal("expected an error as there is no state to sync") - } c := test.Cid1 - err = cl.Pin(ctx, api.PinCid(c)) + err := cl.Pin(ctx, api.PinCid(c)) if err != nil { t.Fatal("pin should have worked:", err) } @@ -234,7 +231,11 @@ func TestClusterStateSync(t *testing.T) { // Modify state on the side so the sync does not // happen on an empty slide - st.Rm(ctx, c) + st, err := cl.consensus.State(ctx) + if err != nil { + t.Fatal(err) + } + st.(state.State).Rm(ctx, c) err = cl.StateSync(ctx) if err != nil { t.Fatal("sync with recover should have worked:", err) @@ -243,7 +244,7 @@ func TestClusterStateSync(t *testing.T) { func TestClusterID(t *testing.T) { ctx := context.Background() - cl, _, _, _, _ := testingCluster(t) + cl, _, _, _ := testingCluster(t) defer cleanRaft() defer cl.Shutdown(ctx) id := cl.ID(ctx) @@ -263,7 +264,7 @@ func TestClusterID(t *testing.T) { func TestClusterPin(t *testing.T) { ctx := context.Background() - cl, _, _, _, _ := testingCluster(t) + cl, _, _, _ := testingCluster(t) defer cleanRaft() defer cl.Shutdown(ctx) @@ -286,7 +287,7 @@ func TestClusterPin(t *testing.T) { func TestClusterPinPath(t *testing.T) { ctx := context.Background() - cl, _, _, _, _ := testingCluster(t) + cl, _, _, _ := testingCluster(t) defer cleanRaft() defer cl.Shutdown(ctx) @@ -307,7 +308,7 @@ func TestClusterPinPath(t *testing.T) { func TestAddFile(t *testing.T) { ctx := context.Background() - cl, _, _, _, _ := testingCluster(t) + cl, _, _, _ := testingCluster(t) defer cleanRaft() defer cl.Shutdown(ctx) sth := test.NewShardingTestHelper() @@ -367,7 +368,7 @@ func TestAddFile(t *testing.T) { func TestUnpinShard(t *testing.T) { ctx := context.Background() - cl, _, _, _, _ := testingCluster(t) + cl, _, _, _ := testingCluster(t) defer cleanRaft() defer cl.Shutdown(ctx) sth := test.NewShardingTestHelper() @@ -494,7 +495,7 @@ func TestUnpinShard(t *testing.T) { // } // func TestClusterPinMeta(t *testing.T) { -// cl, _, _, _, _ := testingCluster(t) +// cl, _, _, _ := testingCluster(t) // defer cleanRaft() // defer cl.Shutdown() @@ -502,7 +503,7 @@ func TestUnpinShard(t *testing.T) { // } // func TestClusterUnpinShardFail(t *testing.T) { -// cl, _, _, _, _ := testingCluster(t) +// cl, _, _, _ := testingCluster(t) // defer cleanRaft() // defer cl.Shutdown() @@ -526,7 +527,7 @@ func TestUnpinShard(t *testing.T) { // } // func TestClusterUnpinMeta(t *testing.T) { -// cl, _, _, _, _ := testingCluster(t) +// cl, _, _, _ := testingCluster(t) // defer cleanRaft() // defer cl.Shutdown() @@ -571,7 +572,7 @@ func TestUnpinShard(t *testing.T) { // } // func TestClusterPinShardTwoParents(t *testing.T) { -// cl, _, _, _, _ := testingCluster(t) +// cl, _, _, _ := testingCluster(t) // defer cleanRaft() // defer cl.Shutdown() @@ -588,7 +589,7 @@ func TestUnpinShard(t *testing.T) { // } // func TestClusterUnpinShardSecondParent(t *testing.T) { -// cl, _, _, _, _ := testingCluster(t) +// cl, _, _, _ := testingCluster(t) // defer cleanRaft() // defer cl.Shutdown() @@ -621,7 +622,7 @@ func TestUnpinShard(t *testing.T) { // } // func TestClusterUnpinShardFirstParent(t *testing.T) { -// cl, _, _, _, _ := testingCluster(t) +// cl, _, _, _ := testingCluster(t) // defer cleanRaft() // defer cl.Shutdown() @@ -657,7 +658,7 @@ func TestUnpinShard(t *testing.T) { // } // func TestClusterPinTwoMethodsFail(t *testing.T) { -// cl, _, _, _, _ := testingCluster(t) +// cl, _, _, _ := testingCluster(t) // defer cleanRaft() // defer cl.Shutdown() @@ -693,7 +694,7 @@ func TestUnpinShard(t *testing.T) { // } // func TestClusterRePinShard(t *testing.T) { -// cl, _, _, _, _ := testingCluster(t) +// cl, _, _, _ := testingCluster(t) // defer cleanRaft() // defer cl.Shutdown() @@ -729,7 +730,7 @@ func TestUnpinShard(t *testing.T) { func TestClusterPins(t *testing.T) { ctx := context.Background() - cl, _, _, _, _ := testingCluster(t) + cl, _, _, _ := testingCluster(t) defer cleanRaft() defer cl.Shutdown(ctx) @@ -739,7 +740,10 @@ func TestClusterPins(t *testing.T) { t.Fatal("pin should have worked:", err) } - pins := cl.Pins(ctx) + pins, err := cl.Pins(ctx) + if err != nil { + t.Fatal(err) + } if len(pins) != 1 { t.Fatal("pin should be part of the state") } @@ -750,7 +754,7 @@ func TestClusterPins(t *testing.T) { func TestClusterPinGet(t *testing.T) { ctx := context.Background() - cl, _, _, _, _ := testingCluster(t) + cl, _, _, _ := testingCluster(t) defer cleanRaft() defer cl.Shutdown(ctx) @@ -776,7 +780,7 @@ func TestClusterPinGet(t *testing.T) { func TestClusterUnpin(t *testing.T) { ctx := context.Background() - cl, _, _, _, _ := testingCluster(t) + cl, _, _, _ := testingCluster(t) defer cleanRaft() defer cl.Shutdown(ctx) @@ -807,7 +811,7 @@ func TestClusterUnpin(t *testing.T) { func TestClusterUnpinPath(t *testing.T) { ctx := context.Background() - cl, _, _, _, _ := testingCluster(t) + cl, _, _, _ := testingCluster(t) defer cleanRaft() defer cl.Shutdown(ctx) @@ -837,7 +841,7 @@ func TestClusterUnpinPath(t *testing.T) { func TestClusterPeers(t *testing.T) { ctx := context.Background() - cl, _, _, _, _ := testingCluster(t) + cl, _, _, _ := testingCluster(t) defer cleanRaft() defer cl.Shutdown(ctx) peers := cl.Peers(ctx) @@ -854,7 +858,7 @@ func TestClusterPeers(t *testing.T) { func TestVersion(t *testing.T) { ctx := context.Background() - cl, _, _, _, _ := testingCluster(t) + cl, _, _, _ := testingCluster(t) defer cleanRaft() defer cl.Shutdown(ctx) if cl.Version() != version.Version.String() { @@ -864,7 +868,7 @@ func TestVersion(t *testing.T) { func TestClusterRecoverAllLocal(t *testing.T) { ctx := context.Background() - cl, _, _, _, _ := testingCluster(t) + cl, _, _, _ := testingCluster(t) defer cleanRaft() defer cl.Shutdown(ctx) diff --git a/clusterhost.go b/clusterhost.go index 0ab080f3..950f9ae0 100644 --- a/clusterhost.go +++ b/clusterhost.go @@ -5,22 +5,53 @@ import ( "encoding/hex" libp2p "github.com/libp2p/go-libp2p" + crypto "github.com/libp2p/go-libp2p-crypto" host "github.com/libp2p/go-libp2p-host" ipnet "github.com/libp2p/go-libp2p-interface-pnet" + dht "github.com/libp2p/go-libp2p-kad-dht" pnet "github.com/libp2p/go-libp2p-pnet" + pubsub "github.com/libp2p/go-libp2p-pubsub" + routedhost "github.com/libp2p/go-libp2p/p2p/host/routed" ma "github.com/multiformats/go-multiaddr" ) -// NewClusterHost creates a libp2p Host with the options from the -// provided cluster configuration. -func NewClusterHost(ctx context.Context, cfg *Config) (host.Host, error) { +// NewClusterHost creates a libp2p Host with the options from the provided +// cluster configuration. Using that host, it creates pubsub and a DHT +// instances, for shared use by all cluster components. The returned host uses +// the DHT for routing. The resulting DHT is not bootstrapped. +func NewClusterHost( + ctx context.Context, + cfg *Config, +) (host.Host, *pubsub.PubSub, *dht.IpfsDHT, error) { + + h, err := newHost(ctx, cfg.Secret, cfg.PrivateKey, []ma.Multiaddr{cfg.ListenAddr}) + if err != nil { + return nil, nil, nil, err + } + + psub, err := newPubSub(ctx, h) + if err != nil { + h.Close() + return nil, nil, nil, err + } + + idht, err := newDHT(ctx, h) + if err != nil { + h.Close() + return nil, nil, nil, err + } + + return routedHost(h, idht), psub, idht, nil +} + +func newHost(ctx context.Context, secret []byte, priv crypto.PrivKey, listenAddrs []ma.Multiaddr) (host.Host, error) { var prot ipnet.Protector var err error // Create protector if we have a secret. - if cfg.Secret != nil && len(cfg.Secret) > 0 { + if secret != nil && len(secret) > 0 { var key [32]byte - copy(key[:], cfg.Secret) + copy(key[:], secret) prot, err = pnet.NewV1ProtectorFromBytes(&key) if err != nil { return nil, err @@ -29,13 +60,30 @@ func NewClusterHost(ctx context.Context, cfg *Config) (host.Host, error) { return libp2p.New( ctx, - libp2p.Identity(cfg.PrivateKey), - libp2p.ListenAddrs([]ma.Multiaddr{cfg.ListenAddr}...), + libp2p.Identity(priv), + libp2p.ListenAddrs(listenAddrs...), libp2p.PrivateNetwork(prot), libp2p.NATPortMap(), ) } +func newDHT(ctx context.Context, h host.Host) (*dht.IpfsDHT, error) { + return dht.New(ctx, h) +} + +func newPubSub(ctx context.Context, h host.Host) (*pubsub.PubSub, error) { + return pubsub.NewGossipSub( + ctx, + h, + pubsub.WithMessageSigning(true), + pubsub.WithStrictSignatureVerification(true), + ) +} + +func routedHost(h host.Host, d *dht.IpfsDHT) host.Host { + return routedhost.Wrap(h, d) +} + // EncodeProtectorKey converts a byte slice to its hex string representation. func EncodeProtectorKey(secretBytes []byte) string { return hex.EncodeToString(secretBytes) diff --git a/cmd/ipfs-cluster-ctl/graph.go b/cmd/ipfs-cluster-ctl/graph.go index ee869140..d6a76545 100644 --- a/cmd/ipfs-cluster-ctl/graph.go +++ b/cmd/ipfs-cluster-ctl/graph.go @@ -87,7 +87,7 @@ type dotWriter struct { func (dW *dotWriter) addNode(id string, nT nodeType) error { var node dot.VertexDescription pid, _ := peer.IDB58Decode(id) - node.Label = pid.String() + node.Label = pid.ShortString() switch nT { case tCluster: node.ID = fmt.Sprintf("C%d", len(dW.clusterNodes)) diff --git a/cmd/ipfs-cluster-service/configs.go b/cmd/ipfs-cluster-service/configs.go index 48890d89..546a80d3 100644 --- a/cmd/ipfs-cluster-service/configs.go +++ b/cmd/ipfs-cluster-service/configs.go @@ -8,7 +8,9 @@ import ( "github.com/ipfs/ipfs-cluster/api/ipfsproxy" "github.com/ipfs/ipfs-cluster/api/rest" "github.com/ipfs/ipfs-cluster/config" + "github.com/ipfs/ipfs-cluster/consensus/crdt" "github.com/ipfs/ipfs-cluster/consensus/raft" + "github.com/ipfs/ipfs-cluster/datastore/badger" "github.com/ipfs/ipfs-cluster/informer/disk" "github.com/ipfs/ipfs-cluster/informer/numpin" "github.com/ipfs/ipfs-cluster/ipfsconn/ipfshttp" @@ -23,7 +25,8 @@ type cfgs struct { apiCfg *rest.Config ipfsproxyCfg *ipfsproxy.Config ipfshttpCfg *ipfshttp.Config - consensusCfg *raft.Config + raftCfg *raft.Config + crdtCfg *crdt.Config maptrackerCfg *maptracker.Config statelessTrackerCfg *stateless.Config pubsubmonCfg *pubsubmon.Config @@ -31,6 +34,7 @@ type cfgs struct { numpinInfCfg *numpin.Config metricsCfg *observations.MetricsConfig tracingCfg *observations.TracingConfig + badgerCfg *badger.Config } func makeConfigs() (*config.Manager, *cfgs) { @@ -39,7 +43,8 @@ func makeConfigs() (*config.Manager, *cfgs) { apiCfg := &rest.Config{} ipfsproxyCfg := &ipfsproxy.Config{} ipfshttpCfg := &ipfshttp.Config{} - consensusCfg := &raft.Config{} + raftCfg := &raft.Config{} + crdtCfg := &crdt.Config{} maptrackerCfg := &maptracker.Config{} statelessCfg := &stateless.Config{} pubsubmonCfg := &pubsubmon.Config{} @@ -47,11 +52,13 @@ func makeConfigs() (*config.Manager, *cfgs) { numpinInfCfg := &numpin.Config{} metricsCfg := &observations.MetricsConfig{} tracingCfg := &observations.TracingConfig{} + badgerCfg := &badger.Config{} cfg.RegisterComponent(config.Cluster, clusterCfg) cfg.RegisterComponent(config.API, apiCfg) cfg.RegisterComponent(config.API, ipfsproxyCfg) cfg.RegisterComponent(config.IPFSConn, ipfshttpCfg) - cfg.RegisterComponent(config.Consensus, consensusCfg) + cfg.RegisterComponent(config.Consensus, raftCfg) + cfg.RegisterComponent(config.Consensus, crdtCfg) cfg.RegisterComponent(config.PinTracker, maptrackerCfg) cfg.RegisterComponent(config.PinTracker, statelessCfg) cfg.RegisterComponent(config.Monitor, pubsubmonCfg) @@ -59,12 +66,14 @@ func makeConfigs() (*config.Manager, *cfgs) { cfg.RegisterComponent(config.Informer, numpinInfCfg) cfg.RegisterComponent(config.Observations, metricsCfg) cfg.RegisterComponent(config.Observations, tracingCfg) + cfg.RegisterComponent(config.Datastore, badgerCfg) return cfg, &cfgs{ clusterCfg, apiCfg, ipfsproxyCfg, ipfshttpCfg, - consensusCfg, + raftCfg, + crdtCfg, maptrackerCfg, statelessCfg, pubsubmonCfg, @@ -72,9 +81,16 @@ func makeConfigs() (*config.Manager, *cfgs) { numpinInfCfg, metricsCfg, tracingCfg, + badgerCfg, } } +func makeAndLoadConfigs() (*config.Manager, *cfgs) { + cfgMgr, cfgs := makeConfigs() + checkErr("reading configuration", cfgMgr.LoadJSONFileAndEnv(configPath)) + return cfgMgr, cfgs +} + func saveConfig(cfg *config.Manager) { err := os.MkdirAll(filepath.Dir(configPath), 0700) err = cfg.SaveJSON(configPath) @@ -94,7 +110,8 @@ func propagateTracingConfig(cfgs *cfgs, tracingFlag bool) *cfgs { cfgs.tracingCfg.ClusterPeername = cfgs.clusterCfg.Peername cfgs.tracingCfg.EnableTracing = tracingValue cfgs.clusterCfg.Tracing = tracingValue - cfgs.consensusCfg.Tracing = tracingValue + cfgs.raftCfg.Tracing = tracingValue + cfgs.crdtCfg.Tracing = tracingValue cfgs.apiCfg.Tracing = tracingValue cfgs.ipfshttpCfg.Tracing = tracingValue cfgs.ipfsproxyCfg.Tracing = tracingValue diff --git a/cmd/ipfs-cluster-service/daemon.go b/cmd/ipfs-cluster-service/daemon.go index faeccf74..e02137f3 100644 --- a/cmd/ipfs-cluster-service/daemon.go +++ b/cmd/ipfs-cluster-service/daemon.go @@ -2,20 +2,17 @@ package main import ( "context" - "errors" "os" "os/signal" "syscall" "time" - host "github.com/libp2p/go-libp2p-host" - "github.com/urfave/cli" - ipfscluster "github.com/ipfs/ipfs-cluster" "github.com/ipfs/ipfs-cluster/allocator/ascendalloc" "github.com/ipfs/ipfs-cluster/allocator/descendalloc" "github.com/ipfs/ipfs-cluster/api/ipfsproxy" "github.com/ipfs/ipfs-cluster/api/rest" + "github.com/ipfs/ipfs-cluster/consensus/crdt" "github.com/ipfs/ipfs-cluster/consensus/raft" "github.com/ipfs/ipfs-cluster/informer/disk" "github.com/ipfs/ipfs-cluster/informer/numpin" @@ -25,9 +22,16 @@ import ( "github.com/ipfs/ipfs-cluster/pintracker/maptracker" "github.com/ipfs/ipfs-cluster/pintracker/stateless" "github.com/ipfs/ipfs-cluster/pstoremgr" - "github.com/ipfs/ipfs-cluster/state/mapstate" + ds "github.com/ipfs/go-datastore" + host "github.com/libp2p/go-libp2p-host" + dht "github.com/libp2p/go-libp2p-kad-dht" + peer "github.com/libp2p/go-libp2p-peer" + pubsub "github.com/libp2p/go-libp2p-pubsub" ma "github.com/multiformats/go-multiaddr" + + errors "github.com/pkg/errors" + cli "github.com/urfave/cli" ) func parseBootstraps(flagVal []string) (bootstraps []ma.Multiaddr) { @@ -46,31 +50,16 @@ func daemon(c *cli.Context) error { ctx, cancel := context.WithCancel(context.Background()) defer cancel() - // Load all the configurations - cfgMgr, cfgs := makeConfigs() - - // Run any migrations - if c.Bool("upgrade") { - err := upgrade(ctx) - if err != errNoSnapshot { - checkErr("upgrading state", err) - } // otherwise continue - } - bootstraps := parseBootstraps(c.StringSlice("bootstrap")) // Execution lock - err := locker.lock() - checkErr("acquiring execution lock", err) + locker.lock() defer locker.tryUnlock() // Load all the configurations - // always wait for configuration to be saved + cfgMgr, cfgs := makeAndLoadConfigs() defer cfgMgr.Shutdown() - err = cfgMgr.LoadJSONFileAndEnv(configPath) - checkErr("loading configuration", err) - if c.Bool("stats") { cfgs.metricsCfg.EnableStats = true } @@ -79,8 +68,8 @@ func daemon(c *cli.Context) error { // Cleanup state if bootstrapping raftStaging := false - if len(bootstraps) > 0 { - cleanupState(cfgs.consensusCfg) + if len(bootstraps) > 0 && c.String("consensus") == "raft" { + raft.CleanupRaft(cfgs.raftCfg) raftStaging = true } @@ -101,22 +90,24 @@ func daemon(c *cli.Context) error { return handleSignals(ctx, cluster) } +// createCluster creates all the necessary things to produce the cluster +// object and returns it along the datastore so the lifecycle can be handled +// (the datastore needs to be Closed after shutting down the Cluster). func createCluster( ctx context.Context, c *cli.Context, cfgs *cfgs, raftStaging bool, ) (*ipfscluster.Cluster, error) { - err := observations.SetupMetrics(cfgs.metricsCfg) - checkErr("setting up Metrics", err) - tracer, err := observations.SetupTracing(cfgs.tracingCfg) - checkErr("setting up Tracing", err) - - host, err := ipfscluster.NewClusterHost(ctx, cfgs.clusterCfg) + host, pubsub, dht, err := ipfscluster.NewClusterHost(ctx, cfgs.clusterCfg) checkErr("creating libP2P Host", err) peerstoreMgr := pstoremgr.New(host, cfgs.clusterCfg.GetPeerstorePath()) + // Import peers but do not connect. We cannot connect to peers until + // everything has been created (dht, pubsub, bitswap). Otherwise things + // fail. + // Connections will happen as needed during bootstrap, rpc etc. peerstoreMgr.ImportPeersFromPeerstore(false) api, err := rest.NewAPIWithHost(ctx, cfgs.apiCfg, host) @@ -130,34 +121,63 @@ func createCluster( connector, err := ipfshttp.NewConnector(cfgs.ipfshttpCfg) checkErr("creating IPFS Connector component", err) - state := mapstate.NewMapState() - - err = validateVersion(ctx, cfgs.clusterCfg, cfgs.consensusCfg) - checkErr("validating version", err) - - raftcon, err := raft.NewConsensus( + tracker := setupPinTracker( + c.String("pintracker"), host, - cfgs.consensusCfg, - state, + cfgs.maptrackerCfg, + cfgs.statelessTrackerCfg, + cfgs.clusterCfg.Peername, + ) + + informer, alloc := setupAllocation( + c.String("alloc"), + cfgs.diskInfCfg, + cfgs.numpinInfCfg, + ) + + ipfscluster.ReadyTimeout = cfgs.raftCfg.WaitForLeaderTimeout + 5*time.Second + + err = observations.SetupMetrics(cfgs.metricsCfg) + checkErr("setting up Metrics", err) + + tracer, err := observations.SetupTracing(cfgs.tracingCfg) + checkErr("setting up Tracing", err) + + store := setupDatastore(c.String("consensus"), cfgs) + + cons, err := setupConsensus( + c.String("consensus"), + host, + dht, + pubsub, + cfgs, + store, raftStaging, ) - checkErr("creating consensus component", err) + if err != nil { + store.Close() + checkErr("setting up Consensus", err) + } - tracker := setupPinTracker(c.String("pintracker"), host, cfgs.maptrackerCfg, cfgs.statelessTrackerCfg, cfgs.clusterCfg.Peername) - mon, err := pubsubmon.New(host, cfgs.pubsubmonCfg) - checkErr("creating monitor", err) - logger.Debug("pubsub monitor loaded") - informer, alloc := setupAllocation(c.String("alloc"), cfgs.diskInfCfg, cfgs.numpinInfCfg) + var peersF func(context.Context) ([]peer.ID, error) + if c.String("consensus") == "raft" { + peersF = cons.Peers + } - ipfscluster.ReadyTimeout = cfgs.consensusCfg.WaitForLeaderTimeout + 5*time.Second + mon, err := pubsubmon.New(cfgs.pubsubmonCfg, pubsub, peersF) + if err != nil { + store.Close() + checkErr("setting up PeerMonitor", err) + } return ipfscluster.NewCluster( host, + dht, cfgs.clusterCfg, - raftcon, + store, + cons, apis, connector, - state, tracker, mon, alloc, @@ -271,3 +291,51 @@ func setupPinTracker( return nil } } + +func setupDatastore( + consensus string, + cfgs *cfgs, +) ds.Datastore { + stmgr := newStateManager(consensus, cfgs) + store, err := stmgr.GetStore() + checkErr("creating datastore", err) + return store +} + +func setupConsensus( + name string, + h host.Host, + dht *dht.IpfsDHT, + pubsub *pubsub.PubSub, + cfgs *cfgs, + store ds.Datastore, + raftStaging bool, +) (ipfscluster.Consensus, error) { + switch name { + case "raft": + rft, err := raft.NewConsensus( + h, + cfgs.raftCfg, + store, + raftStaging, + ) + if err != nil { + return nil, errors.Wrap(err, "creating Raft component") + } + return rft, nil + case "crdt": + convrdt, err := crdt.New( + h, + dht, + pubsub, + cfgs.crdtCfg, + store, + ) + if err != nil { + return nil, errors.Wrap(err, "creating CRDT component") + } + return convrdt, nil + default: + return nil, errors.New("unknown consensus component") + } +} diff --git a/cmd/ipfs-cluster-service/lock.go b/cmd/ipfs-cluster-service/lock.go index fa9ca5fb..ccd33232 100644 --- a/cmd/ipfs-cluster-service/lock.go +++ b/cmd/ipfs-cluster-service/lock.go @@ -1,6 +1,7 @@ package main import ( + "errors" "fmt" "io" "os" @@ -22,13 +23,15 @@ type lock struct { path string } -func (l *lock) lock() error { +func (l *lock) lock() { if l.lockCloser != nil { - return fmt.Errorf("cannot acquire lock twice") + checkErr("", errors.New("cannot acquire lock twice")) } - if err := l.checkConfigExists(); err != nil { - return err + if _, err := os.Stat(configPath); os.IsNotExist(err) { + errMsg := "%s config hasn't been initialized. Please run '%s init'" + errMsg = fmt.Sprintf(errMsg, programName, programName) + checkErr("", errors.New(errMsg)) } // set the lock file within this function @@ -37,16 +40,21 @@ func (l *lock) lock() error { if err != nil { logger.Debug(err) l.lockCloser = nil - errStr := fmt.Sprintf(`could not obtain execution lock. If no other process -is running, remove %s, or make sure that the config folder is -writable for the user running ipfs-cluster. Run with -d for more information -about the error`, path.Join(l.path, lockFileName)) - logger.Error(errStr) - return fmt.Errorf("could not obtain execution lock") + errStr := "%s. If no other " + errStr += "%s process is running, remove %s, or make sure " + errStr += "that the config folder is writable for the user " + errStr += "running %s." + errStr = fmt.Sprintf( + errStr, + err, + programName, + path.Join(l.path, lockFileName), + programName, + ) + checkErr("obtaining execution lock", errors.New(errStr)) } - logger.Debug("Success! ipfs-cluster-service lock acquired") + logger.Debugf("%s execution lock acquired", programName) l.lockCloser = lk - return nil } func (l *lock) tryUnlock() error { @@ -59,19 +67,7 @@ func (l *lock) tryUnlock() error { if err != nil { return err } - logger.Debug("Successfully released execution lock") + logger.Debug("successfully released execution lock") l.lockCloser = nil return nil } - -func (l *lock) checkConfigExists() error { - if _, err := os.Stat(l.path); os.IsNotExist(err) { - logger.Error("ipfs-cluster-service config hasn't been initialized.\nPlease run ipfs-cluster-service init") - return err - } - if _, err := os.Stat(fmt.Sprintf("%s/%s", l.path, "service.json")); os.IsNotExist(err) { - logger.Error("ipfs-cluster-service config hasn't been initialized.\nPlease run ipfs-cluster-service init") - return err - } - return nil -} diff --git a/cmd/ipfs-cluster-service/main.go b/cmd/ipfs-cluster-service/main.go index b8e99012..19db6666 100644 --- a/cmd/ipfs-cluster-service/main.go +++ b/cmd/ipfs-cluster-service/main.go @@ -3,7 +3,6 @@ package main import ( "bufio" - "context" "fmt" "io" "os" @@ -11,7 +10,6 @@ import ( "path/filepath" ipfscluster "github.com/ipfs/ipfs-cluster" - "github.com/ipfs/ipfs-cluster/state/mapstate" "github.com/ipfs/ipfs-cluster/version" semver "github.com/blang/semver" @@ -24,6 +22,7 @@ const programName = `ipfs-cluster-service` // flag defaults const ( + defaultConsensus = "raft" defaultAllocation = "disk-freespace" defaultPinTracker = "map" defaultLogLevel = "info" @@ -220,19 +219,28 @@ configuration. if alreadyInitialized { // acquire lock for config folder - err := locker.lock() - checkErr("acquiring execution lock", err) + locker.lock() defer locker.tryUnlock() - if !c.Bool("force") && !yesNoPrompt(fmt.Sprintf("%s\n%s Continue? [y/n]:", stateCleanupPrompt, configurationOverwritePrompt)) { + confirm := fmt.Sprintf( + "%s\n%s Continue? [y/n]:", + stateCleanupPrompt, + configurationOverwritePrompt, + ) + + if !c.Bool("force") && !yesNoPrompt(confirm) { return nil } - err = cfgMgr.LoadJSONFromFile(configPath) + err := cfgMgr.LoadJSONFileAndEnv(configPath) checkErr("reading configuration", err) - err = cleanupState(cfgs.consensusCfg) - checkErr("Cleaning up consensus data", err) + // rafts needs cleanup on re-init because + // the peer ID of this peer changes + // and is no longer part of the old + // peerset. + mgr := newStateManager("raft", cfgs) + checkErr("cleaning up raft data", mgr.Clean()) } // Generate defaults for all registered components @@ -258,7 +266,7 @@ configuration. Flags: []cli.Flag{ cli.BoolFlag{ Name: "upgrade, u", - Usage: "run necessary state migrations before starting cluster service", + Usage: "run state migrations before starting (deprecated/unused)", }, cli.StringSliceFlag{ Name: "bootstrap, j", @@ -269,6 +277,11 @@ configuration. Usage: "remove peer from cluster on exit. Overrides \"leave_on_shutdown\"", Hidden: true, }, + cli.StringFlag{ + Name: "consensus", + Value: defaultConsensus, + Usage: "shared state management provider [raft,crdt]", + }, cli.StringFlag{ Name: "alloc, a", Value: defaultAllocation, @@ -293,60 +306,34 @@ configuration. }, { Name: "state", - Usage: "Manage ipfs-cluster-state", + Usage: "Manage the peer's consensus state (pinset)", Subcommands: []cli.Command{ - { - Name: "version", - Usage: "display the shared state format version", - Action: func(c *cli.Context) error { - fmt.Printf("%d\n", mapstate.Version) - return nil - }, - }, - { - Name: "upgrade", - Usage: "upgrade the IPFS Cluster state to the current version", - Description: ` -This command upgrades the internal state of the ipfs-cluster node -specified in the latest raft snapshot. The state format is migrated from the -version of the snapshot to the version supported by the current cluster version. -To successfully run an upgrade of an entire cluster, shut down each peer without -removal, upgrade state using this command, and restart every peer. -`, - Action: func(c *cli.Context) error { - ctx := context.Background() - err := locker.lock() - checkErr("acquiring execution lock", err) - defer locker.tryUnlock() - - err = upgrade(ctx) - checkErr("upgrading state", err) - return nil - }, - }, { Name: "export", - Usage: "save the IPFS Cluster state to a json file", + Usage: "save the state to a JSON file", Description: ` -This command reads the current cluster state and saves it as json for -human readability and editing. Only state formats compatible with this -version of ipfs-cluster-service can be exported. By default this command -prints the state to stdout. +This command dumps the current cluster pinset (state) as a JSON file. The +resulting file can be used to migrate, restore or backup a Cluster peer. +By default, the state will be printed to stdout. `, Flags: []cli.Flag{ cli.StringFlag{ Name: "file, f", Value: "", - Usage: "sets an output file for exported state", + Usage: "writes to an output file", + }, + cli.StringFlag{ + Name: "consensus", + Value: "raft", + Usage: "consensus component to export data from [raft, crdt]", }, }, Action: func(c *cli.Context) error { - ctx := context.Background() - err := locker.lock() - checkErr("acquiring execution lock", err) + locker.lock() defer locker.tryUnlock() var w io.WriteCloser + var err error outputPath := c.String("file") if outputPath == "" { // Output to stdout @@ -358,88 +345,103 @@ prints the state to stdout. } defer w.Close() - err = export(ctx, w) - checkErr("exporting state", err) + cfgMgr, cfgs := makeAndLoadConfigs() + defer cfgMgr.Shutdown() + mgr := newStateManager(c.String("consensus"), cfgs) + checkErr("exporting state", mgr.ExportState(w)) + logger.Info("state successfully exported") return nil }, }, { Name: "import", - Usage: "load an IPFS Cluster state from an exported state file", + Usage: "load the state from a file produced by 'export'", Description: ` -This command reads in an exported state file storing the state as a persistent -snapshot to be loaded as the cluster state when the cluster peer is restarted. -If an argument is provided, cluster will treat it as the path of the file to -import. If no argument is provided cluster will read json from stdin +This command reads in an exported pinset (state) file and replaces the +existing one. This can be used, for example, to restore a Cluster peer from a +backup. + +If an argument is provided, it will be treated it as the path of the file +to import. If no argument is provided, stdin will be used. `, Flags: []cli.Flag{ cli.BoolFlag{ Name: "force, f", - Usage: "forcefully proceed with replacing the current state with the given one, without prompting", + Usage: "skips confirmation prompt", + }, + cli.StringFlag{ + Name: "consensus", + Value: "raft", + Usage: "consensus component to export data from [raft, crdt]", }, }, Action: func(c *cli.Context) error { - ctx := context.Background() - err := locker.lock() - checkErr("acquiring execution lock", err) + locker.lock() defer locker.tryUnlock() - if !c.Bool("force") { - if !yesNoPrompt("The peer's state will be replaced. Run with -h for details. Continue? [y/n]:") { - return nil - } + confirm := "The pinset (state) of this peer " + confirm += "will be replaced. Continue? [y/n]:" + if !c.Bool("force") && !yesNoPrompt(confirm) { + return nil } // Get the importing file path importFile := c.Args().First() var r io.ReadCloser + var err error if importFile == "" { r = os.Stdin - logger.Info("Reading from stdin, Ctrl-D to finish") + fmt.Println("reading from stdin, Ctrl-D to finish") } else { r, err = os.Open(importFile) checkErr("reading import file", err) } defer r.Close() - err = stateImport(ctx, r) - checkErr("importing state", err) - logger.Info("the given state has been correctly imported to this peer. Make sure all peers have consistent states") + + cfgMgr, cfgs := makeAndLoadConfigs() + defer cfgMgr.Shutdown() + mgr := newStateManager(c.String("consensus"), cfgs) + checkErr("importing state", mgr.ImportState(r)) + logger.Info("state successfully imported. Make sure all peers have consistent states") return nil }, }, { Name: "cleanup", - Usage: "cleanup persistent consensus state so cluster can start afresh", + Usage: "remove persistent data", Description: ` -This command removes the persistent state that is loaded on startup to determine this peer's view of the -cluster state. While it removes the existing state from the load path, one invocation does not permanently remove -this state from disk. This command renames cluster's data folder to .old.0, and rotates other -deprecated data folders to .old., etc for some rotation factor before permanatly deleting -the mth data folder (m currently defaults to 5) +This command removes any persisted consensus data in this peer, including the +current pinset (state). The next start of the peer will be like the first start +to all effects. Peers may need to bootstrap and sync from scratch after this. `, Flags: []cli.Flag{ cli.BoolFlag{ Name: "force, f", - Usage: "forcefully proceed with rotating peer state without prompting", + Usage: "skip confirmation prompt", + }, + cli.StringFlag{ + Name: "consensus", + Value: "raft", + Usage: "consensus component to export data from [raft, crdt]", }, }, Action: func(c *cli.Context) error { - err := locker.lock() - checkErr("acquiring execution lock", err) + locker.lock() defer locker.tryUnlock() - if !c.Bool("force") { - if !yesNoPrompt(fmt.Sprintf("%s Continue? [y/n]:", stateCleanupPrompt)) { - return nil - } + confirm := fmt.Sprintf( + "%s Continue? [y/n]:", + stateCleanupPrompt, + ) + if !c.Bool("force") && !yesNoPrompt(confirm) { + return nil } - cfgMgr, cfgs := makeConfigs() - err = cfgMgr.LoadJSONFileAndEnv(configPath) - checkErr("reading configuration", err) - - err = cleanupState(cfgs.consensusCfg) - checkErr("Cleaning up consensus data", err) + cfgMgr, cfgs := makeAndLoadConfigs() + defer cfgMgr.Shutdown() + mgr := newStateManager(c.String("consensus"), cfgs) + checkErr("cleaning state", mgr.Clean()) + logger.Info("data correctly cleaned up") return nil }, }, diff --git a/cmd/ipfs-cluster-service/state.go b/cmd/ipfs-cluster-service/state.go index 516ee9a3..3f8748e6 100644 --- a/cmd/ipfs-cluster-service/state.go +++ b/cmd/ipfs-cluster-service/state.go @@ -1,188 +1,194 @@ package main import ( - "bytes" "context" "encoding/json" "errors" + "fmt" "io" - "io/ioutil" ipfscluster "github.com/ipfs/ipfs-cluster" "github.com/ipfs/ipfs-cluster/api" + "github.com/ipfs/ipfs-cluster/consensus/crdt" "github.com/ipfs/ipfs-cluster/consensus/raft" + "github.com/ipfs/ipfs-cluster/datastore/badger" + "github.com/ipfs/ipfs-cluster/datastore/inmem" "github.com/ipfs/ipfs-cluster/pstoremgr" "github.com/ipfs/ipfs-cluster/state" - "github.com/ipfs/ipfs-cluster/state/mapstate" - "go.opencensus.io/trace" + + ds "github.com/ipfs/go-datastore" ) -var errNoSnapshot = errors.New("no snapshot found") - -func upgrade(ctx context.Context) error { - ctx, span := trace.StartSpan(ctx, "daemon/upgrade") - defer span.End() - - newState, current, err := restoreStateFromDisk(ctx) - if err != nil { - return err - } - - if current { - logger.Warning("Skipping migration of up-to-date state") - return nil - } - - cfgMgr, cfgs := makeConfigs() - - err = cfgMgr.LoadJSONFileAndEnv(configPath) - if err != nil { - return err - } - - pm := pstoremgr.New(nil, cfgs.clusterCfg.GetPeerstorePath()) - raftPeers := append(ipfscluster.PeersFromMultiaddrs(pm.LoadPeerstore()), cfgs.clusterCfg.ID) - return raft.SnapshotSave(cfgs.consensusCfg, newState, raftPeers) +type stateManager interface { + ImportState(io.Reader) error + ExportState(io.Writer) error + GetStore() (ds.Datastore, error) + Clean() error } -func export(ctx context.Context, w io.Writer) error { - ctx, span := trace.StartSpan(ctx, "daemon/export") - defer span.End() +func newStateManager(consensus string, cfgs *cfgs) stateManager { + switch consensus { + case "raft": + return &raftStateManager{cfgs} + case "crdt": + return &crdtStateManager{cfgs} + case "": + checkErr("", errors.New("unspecified consensus component")) + default: + checkErr("", fmt.Errorf("unknown consensus component '%s'", consensus)) + } + return nil +} - stateToExport, _, err := restoreStateFromDisk(ctx) +type raftStateManager struct { + cfgs *cfgs +} + +func (raftsm *raftStateManager) GetStore() (ds.Datastore, error) { + return inmem.New(), nil +} + +func (raftsm *raftStateManager) getOfflineState(store ds.Datastore) (state.State, error) { + return raft.OfflineState(raftsm.cfgs.raftCfg, store) +} + +func (raftsm *raftStateManager) ImportState(r io.Reader) error { + err := raftsm.Clean() if err != nil { return err } - return exportState(ctx, stateToExport, w) + store, err := raftsm.GetStore() + if err != nil { + return err + } + defer store.Close() + st, err := raftsm.getOfflineState(store) + if err != nil { + return err + } + err = importState(r, st) + if err != nil { + return err + } + pm := pstoremgr.New(nil, raftsm.cfgs.clusterCfg.GetPeerstorePath()) + raftPeers := append( + ipfscluster.PeersFromMultiaddrs(pm.LoadPeerstore()), + raftsm.cfgs.clusterCfg.ID, + ) + return raft.SnapshotSave(raftsm.cfgs.raftCfg, st, raftPeers) } -// restoreStateFromDisk returns a mapstate containing the latest -// snapshot, a flag set to true when the state format has the -// current version and an error -func restoreStateFromDisk(ctx context.Context) (state.State, bool, error) { - ctx, span := trace.StartSpan(ctx, "daemon/restoreStateFromDisk") - defer span.End() - - cfgMgr, cfgs := makeConfigs() - - err := cfgMgr.LoadJSONFileAndEnv(configPath) +func (raftsm *raftStateManager) ExportState(w io.Writer) error { + store, err := raftsm.GetStore() if err != nil { - return nil, false, err - } - - r, snapExists, err := raft.LastStateRaw(cfgs.consensusCfg) - if !snapExists { - err = errNoSnapshot + return err } + defer store.Close() + st, err := raftsm.getOfflineState(store) if err != nil { - return nil, false, err + return err } - - full, err := ioutil.ReadAll(r) - if err != nil { - return nil, false, err - } - - stateFromSnap := mapstate.NewMapState() - // duplicate reader to both check version and migrate - reader1 := bytes.NewReader(full) - err = stateFromSnap.Unmarshal(reader1) - if err != nil { - return nil, false, err - } - if stateFromSnap.GetVersion() == mapstate.Version { - return stateFromSnap, true, nil - } - reader2 := bytes.NewReader(full) - err = stateFromSnap.Migrate(ctx, reader2) - if err != nil { - return nil, false, err - } - - return stateFromSnap, false, nil + return exportState(w, st) } -func stateImport(ctx context.Context, r io.Reader) error { - ctx, span := trace.StartSpan(ctx, "daemon/stateImport") - defer span.End() +func (raftsm *raftStateManager) Clean() error { + return raft.CleanupRaft(raftsm.cfgs.raftCfg) +} - cfgMgr, cfgs := makeConfigs() +type crdtStateManager struct { + cfgs *cfgs +} - err := cfgMgr.LoadJSONFileAndEnv(configPath) +func (crdtsm *crdtStateManager) GetStore() (ds.Datastore, error) { + bds, err := badger.New(crdtsm.cfgs.badgerCfg) + if err != nil { + return nil, err + } + return bds, nil +} + +func (crdtsm *crdtStateManager) getOfflineState(store ds.Datastore) (state.BatchingState, error) { + return crdt.OfflineState(crdtsm.cfgs.crdtCfg, store) +} + +func (crdtsm *crdtStateManager) ImportState(r io.Reader) error { + err := crdtsm.Clean() if err != nil { return err } - pins := make([]*api.Pin, 0) + store, err := crdtsm.GetStore() + if err != nil { + return err + } + defer store.Close() + st, err := crdtsm.getOfflineState(store) + if err != nil { + return err + } + + err = importState(r, st) + if err != nil { + return err + } + + return st.Commit(context.Background()) +} + +func (crdtsm *crdtStateManager) ExportState(w io.Writer) error { + store, err := crdtsm.GetStore() + if err != nil { + return err + } + defer store.Close() + st, err := crdtsm.getOfflineState(store) + if err != nil { + return err + } + return exportState(w, st) +} + +func (crdtsm *crdtStateManager) Clean() error { + store, err := crdtsm.GetStore() + if err != nil { + return err + } + defer store.Close() + return crdt.Clean(context.Background(), crdtsm.cfgs.crdtCfg, store) +} + +func importState(r io.Reader, st state.State) error { + ctx := context.Background() dec := json.NewDecoder(r) - err = dec.Decode(&pins) - if err != nil { - return err - } - - stateToImport := mapstate.NewMapState() - for _, p := range pins { - err = stateToImport.Add(ctx, p) + for { + var pin api.Pin + err := dec.Decode(&pin) + if err == io.EOF { + return nil + } + if err != nil { + return err + } + err = st.Add(ctx, &pin) if err != nil { return err } } - - pm := pstoremgr.New(nil, cfgs.clusterCfg.GetPeerstorePath()) - raftPeers := append(ipfscluster.PeersFromMultiaddrs(pm.LoadPeerstore()), cfgs.clusterCfg.ID) - return raft.SnapshotSave(cfgs.consensusCfg, stateToImport, raftPeers) -} - -func validateVersion(ctx context.Context, cfg *ipfscluster.Config, cCfg *raft.Config) error { - ctx, span := trace.StartSpan(ctx, "daemon/validateVersion") - defer span.End() - - state := mapstate.NewMapState() - r, snapExists, err := raft.LastStateRaw(cCfg) - if !snapExists && err != nil { - logger.Error("error before reading latest snapshot.") - } else if snapExists && err != nil { - logger.Error("error after reading last snapshot. Snapshot potentially corrupt.") - } else if snapExists && err == nil { - err2 := state.Unmarshal(r) - if err2 != nil { - logger.Error("error unmarshalling snapshot. Snapshot potentially corrupt.") - return err2 - } - if state.GetVersion() != mapstate.Version { - logger.Error("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!") - logger.Error("Out of date ipfs-cluster state is saved.") - logger.Error("To migrate to the new version, run ipfs-cluster-service state upgrade.") - logger.Error("To launch a node without this state, rename the consensus data directory.") - logger.Error("Hint: the default is .ipfs-cluster/raft.") - logger.Error("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!") - err = errors.New("outdated state version stored") - } - } // !snapExists && err == nil // no existing state, no check needed - return err } // ExportState saves a json representation of a state -func exportState(ctx context.Context, state state.State, w io.Writer) error { - ctx, span := trace.StartSpan(ctx, "daemon/exportState") - defer span.End() - - // Serialize pins - pins := state.List(ctx) - - // Write json to output file - enc := json.NewEncoder(w) - enc.SetIndent("", " ") - return enc.Encode(pins) -} - -// CleanupState cleans the state -func cleanupState(cCfg *raft.Config) error { - err := raft.CleanupRaft(cCfg.GetDataFolder(), cCfg.BackupsRotate) - if err == nil { - logger.Warningf("the %s folder has been rotated. Next start will use an empty state", cCfg.GetDataFolder()) +func exportState(w io.Writer, st state.State) error { + pins, err := st.List(context.Background()) + if err != nil { + return err } - - return err + enc := json.NewEncoder(w) + for _, pin := range pins { + err := enc.Encode(pin) + if err != nil { + return err + } + } + return nil } diff --git a/config/config.go b/config/config.go index dad3ec79..e7a9b7ec 100644 --- a/config/config.go +++ b/config/config.go @@ -58,6 +58,7 @@ const ( Allocator Informer Observations + Datastore endTypes // keep this at the end ) @@ -178,6 +179,7 @@ type jsonConfig struct { Allocator jsonSection `json:"allocator,omitempty"` Informer jsonSection `json:"informer,omitempty"` Observations jsonSection `json:"observations,omitempty"` + Datastore jsonSection `json:"datastore,omitempty"` } func (jcfg *jsonConfig) getSection(i SectionType) *jsonSection { @@ -200,6 +202,8 @@ func (jcfg *jsonConfig) getSection(i SectionType) *jsonSection { return &jcfg.Informer case Observations: return &jcfg.Observations + case Datastore: + return &jcfg.Datastore default: return nil } diff --git a/config/config_test.go b/config/config_test.go index 3fa39240..dc6bc4ad 100644 --- a/config/config_test.go +++ b/config/config_test.go @@ -97,6 +97,11 @@ func TestManager_ToJSON(t *testing.T) { "mock": { "a": "b" } + }, + "datastore": { + "mock": { + "a": "b" + } } }`) cfgMgr := setupConfigManager() diff --git a/config_test.go b/config_test.go index 90f0ab20..1dabef38 100644 --- a/config_test.go +++ b/config_test.go @@ -3,7 +3,9 @@ package ipfscluster import ( "github.com/ipfs/ipfs-cluster/api/ipfsproxy" "github.com/ipfs/ipfs-cluster/api/rest" + "github.com/ipfs/ipfs-cluster/consensus/crdt" "github.com/ipfs/ipfs-cluster/consensus/raft" + "github.com/ipfs/ipfs-cluster/datastore/badger" "github.com/ipfs/ipfs-cluster/informer/disk" "github.com/ipfs/ipfs-cluster/ipfsconn/ipfshttp" "github.com/ipfs/ipfs-cluster/monitor/pubsubmon" @@ -46,6 +48,15 @@ var testingRaftCfg = []byte(`{ "leader_lease_timeout": "80ms" }`) +var testingCrdtCfg = []byte(`{ + "cluster_name": "crdt-test", + "rebroadcast_interval": "150ms" +}`) + +var testingBadgerCfg = []byte(`{ + "folder": "badgerFromTests" +}`) + var testingAPICfg = []byte(`{ "http_listen_multiaddress": "/ip4/127.0.0.1/tcp/10002", "read_timeout": "0", @@ -85,8 +96,8 @@ var testingIpfsCfg = []byte(`{ var testingTrackerCfg = []byte(` { - "max_pin_queue_size": 4092, - "concurrent_pins": 1 + "max_pin_queue_size": 4092, + "concurrent_pins": 1 } `) @@ -100,37 +111,41 @@ var testingDiskInfCfg = []byte(`{ }`) var testingTracerCfg = []byte(`{ - "enable_tracing": false + "enable_tracing": false }`) -func testingConfigs() (*Config, *rest.Config, *ipfsproxy.Config, *ipfshttp.Config, *raft.Config, *maptracker.Config, *stateless.Config, *pubsubmon.Config, *disk.Config, *observations.TracingConfig) { - clusterCfg, apiCfg, proxyCfg, ipfsCfg, consensusCfg, maptrackerCfg, statelesstrkrCfg, pubsubmonCfg, diskInfCfg, tracingCfg := testingEmptyConfigs() +func testingConfigs() (*Config, *rest.Config, *ipfsproxy.Config, *ipfshttp.Config, *badger.Config, *raft.Config, *crdt.Config, *maptracker.Config, *stateless.Config, *pubsubmon.Config, *disk.Config, *observations.TracingConfig) { + clusterCfg, apiCfg, proxyCfg, ipfsCfg, badgerCfg, raftCfg, crdtCfg, maptrackerCfg, statelesstrkrCfg, pubsubmonCfg, diskInfCfg, tracingCfg := testingEmptyConfigs() clusterCfg.LoadJSON(testingClusterCfg) apiCfg.LoadJSON(testingAPICfg) proxyCfg.LoadJSON(testingProxyCfg) ipfsCfg.LoadJSON(testingIpfsCfg) - consensusCfg.LoadJSON(testingRaftCfg) + badgerCfg.LoadJSON(testingBadgerCfg) + raftCfg.LoadJSON(testingRaftCfg) + crdtCfg.LoadJSON(testingCrdtCfg) maptrackerCfg.LoadJSON(testingTrackerCfg) statelesstrkrCfg.LoadJSON(testingTrackerCfg) pubsubmonCfg.LoadJSON(testingMonCfg) diskInfCfg.LoadJSON(testingDiskInfCfg) tracingCfg.LoadJSON(testingTracerCfg) - return clusterCfg, apiCfg, proxyCfg, ipfsCfg, consensusCfg, maptrackerCfg, statelesstrkrCfg, pubsubmonCfg, diskInfCfg, tracingCfg + return clusterCfg, apiCfg, proxyCfg, ipfsCfg, badgerCfg, raftCfg, crdtCfg, maptrackerCfg, statelesstrkrCfg, pubsubmonCfg, diskInfCfg, tracingCfg } -func testingEmptyConfigs() (*Config, *rest.Config, *ipfsproxy.Config, *ipfshttp.Config, *raft.Config, *maptracker.Config, *stateless.Config, *pubsubmon.Config, *disk.Config, *observations.TracingConfig) { +func testingEmptyConfigs() (*Config, *rest.Config, *ipfsproxy.Config, *ipfshttp.Config, *badger.Config, *raft.Config, *crdt.Config, *maptracker.Config, *stateless.Config, *pubsubmon.Config, *disk.Config, *observations.TracingConfig) { clusterCfg := &Config{} apiCfg := &rest.Config{} proxyCfg := &ipfsproxy.Config{} ipfshttpCfg := &ipfshttp.Config{} - consensusCfg := &raft.Config{} + badgerCfg := &badger.Config{} + raftCfg := &raft.Config{} + crdtCfg := &crdt.Config{} maptrackerCfg := &maptracker.Config{} statelessCfg := &stateless.Config{} pubsubmonCfg := &pubsubmon.Config{} diskInfCfg := &disk.Config{} tracingCfg := &observations.TracingConfig{} - return clusterCfg, apiCfg, proxyCfg, ipfshttpCfg, consensusCfg, maptrackerCfg, statelessCfg, pubsubmonCfg, diskInfCfg, tracingCfg + return clusterCfg, apiCfg, proxyCfg, ipfshttpCfg, badgerCfg, raftCfg, crdtCfg, maptrackerCfg, statelessCfg, pubsubmonCfg, diskInfCfg, tracingCfg } // func TestConfigDefault(t *testing.T) { diff --git a/consensus/crdt/config.go b/consensus/crdt/config.go new file mode 100644 index 00000000..d37c3a72 --- /dev/null +++ b/consensus/crdt/config.go @@ -0,0 +1,152 @@ +package crdt + +import ( + "encoding/json" + "errors" + "fmt" + "time" + + "github.com/kelseyhightower/envconfig" + + "github.com/ipfs/ipfs-cluster/config" +) + +var configKey = "crdt" +var envConfigKey = "cluster_crdt" + +// Default configuration values +var ( + DefaultClusterName = "ipfs-cluster" + DefaultPeersetMetric = "ping" + DefaultDatastoreNamespace = "/c" // from "/crdt" + DefaultRebroadcastInterval = time.Minute +) + +// Config is the configuration object for Consensus. +type Config struct { + config.Saver + + hostShutdown bool + + // The topic we wish to subscribe to + ClusterName string + + // The interval before re-announcing the current state + // to the network when no activity is observed. + RebroadcastInterval time.Duration + + // The name of the metric we use to obtain the peerset (every peer + // with valid metric of this type is part of it). + PeersetMetric string + + // All keys written to the datastore will be namespaced with this prefix + DatastoreNamespace string + + // Tracing enables propagation of contexts across binary boundaries. + Tracing bool +} + +type jsonConfig struct { + ClusterName string `json:"cluster_name"` + RebroadcastInterval string `json:"rebroadcast_interval,omitempty"` + PeersetMetric string `json:"peerset_metric,omitempty"` + DatastoreNamespace string `json:"datastore_namespace,omitempty"` +} + +// ConfigKey returns the section name for this type of configuration. +func (cfg *Config) ConfigKey() string { + return configKey +} + +// Validate returns an error if the configuration has invalid values. +func (cfg *Config) Validate() error { + if cfg.ClusterName == "" { + return errors.New("crdt.cluster_name cannot be empty") + } + + if cfg.PeersetMetric == "" { + return errors.New("crdt.peerset_metric needs a name") + } + + if cfg.RebroadcastInterval <= 0 { + return errors.New("crdt.rebroadcast_interval is invalid") + } + return nil +} + +// LoadJSON takes a raw JSON slice and sets all the configuration fields. +func (cfg *Config) LoadJSON(raw []byte) error { + jcfg := &jsonConfig{} + err := json.Unmarshal(raw, jcfg) + if err != nil { + return fmt.Errorf("error unmarshaling %s config", configKey) + } + + cfg.Default() + + return cfg.applyJSONConfig(jcfg) +} + +func (cfg *Config) applyJSONConfig(jcfg *jsonConfig) error { + cfg.ClusterName = jcfg.ClusterName + config.SetIfNotDefault(jcfg.PeersetMetric, &cfg.PeersetMetric) + config.SetIfNotDefault(jcfg.DatastoreNamespace, &cfg.DatastoreNamespace) + config.ParseDurations( + "crdt", + &config.DurationOpt{Duration: jcfg.RebroadcastInterval, Dst: &cfg.RebroadcastInterval, Name: "rebroadcast_interval"}, + ) + return cfg.Validate() +} + +// ToJSON returns the JSON representation of this configuration. +func (cfg *Config) ToJSON() ([]byte, error) { + jcfg := cfg.toJSONConfig() + + return config.DefaultJSONMarshal(jcfg) +} + +func (cfg *Config) toJSONConfig() *jsonConfig { + jcfg := &jsonConfig{ + ClusterName: cfg.ClusterName, + PeersetMetric: "", + RebroadcastInterval: "", + } + + if cfg.PeersetMetric != DefaultPeersetMetric { + jcfg.PeersetMetric = cfg.PeersetMetric + // otherwise leave empty/hidden + } + + if cfg.DatastoreNamespace != DefaultDatastoreNamespace { + jcfg.DatastoreNamespace = cfg.DatastoreNamespace + // otherwise leave empty/hidden + } + + if cfg.RebroadcastInterval != DefaultRebroadcastInterval { + jcfg.RebroadcastInterval = cfg.RebroadcastInterval.String() + } + + return jcfg +} + +// Default sets the configuration fields to their default values. +func (cfg *Config) Default() error { + cfg.ClusterName = DefaultClusterName + cfg.RebroadcastInterval = DefaultRebroadcastInterval + cfg.PeersetMetric = DefaultPeersetMetric + cfg.DatastoreNamespace = DefaultDatastoreNamespace + return nil +} + +// ApplyEnvVars fills in any Config fields found +// as environment variables. +func (cfg *Config) ApplyEnvVars() error { + jcfg := cfg.toJSONConfig() + + err := envconfig.Process(envConfigKey, jcfg) + if err != nil { + return err + } + + return cfg.applyJSONConfig(jcfg) +} diff --git a/consensus/crdt/config_test.go b/consensus/crdt/config_test.go new file mode 100644 index 00000000..588b0550 --- /dev/null +++ b/consensus/crdt/config_test.go @@ -0,0 +1,71 @@ +package crdt + +import ( + "os" + "testing" +) + +var cfgJSON = []byte(` +{ + "cluster_name": "test" +} +`) + +func TestLoadJSON(t *testing.T) { + cfg := &Config{} + err := cfg.LoadJSON(cfgJSON) + if err != nil { + t.Fatal(err) + } +} + +func TestToJSON(t *testing.T) { + cfg := &Config{} + cfg.LoadJSON(cfgJSON) + newjson, err := cfg.ToJSON() + if err != nil { + t.Fatal(err) + } + + cfg = &Config{} + err = cfg.LoadJSON(newjson) + if err != nil { + t.Fatal(err) + } +} + +func TestDefault(t *testing.T) { + cfg := &Config{} + cfg.Default() + if cfg.Validate() != nil { + t.Fatal("error validating") + } + + cfg.ClusterName = "" + if cfg.Validate() == nil { + t.Fatal("expected error validating") + } + + cfg.Default() + cfg.PeersetMetric = "" + if cfg.Validate() == nil { + t.Fatal("expected error validating") + } + + cfg.Default() + cfg.RebroadcastInterval = 0 + if cfg.Validate() == nil { + t.Fatal("expected error validating") + } +} + +func TestApplyEnvVars(t *testing.T) { + os.Setenv("CLUSTER_CRDT_CLUSTERNAME", "test2") + cfg := &Config{} + cfg.Default() + cfg.ApplyEnvVars() + + if cfg.ClusterName != "test2" { + t.Fatal("failed to override cluster_name with env var") + } +} diff --git a/consensus/crdt/consensus.go b/consensus/crdt/consensus.go new file mode 100644 index 00000000..b5f1fc41 --- /dev/null +++ b/consensus/crdt/consensus.go @@ -0,0 +1,425 @@ +package crdt + +import ( + "context" + "errors" + "sync" + + ipfslite "github.com/hsanjuan/ipfs-lite" + dshelp "github.com/ipfs/go-ipfs-ds-help" + "github.com/ipfs/ipfs-cluster/api" + "github.com/ipfs/ipfs-cluster/state" + "github.com/ipfs/ipfs-cluster/state/dsstate" + multihash "github.com/multiformats/go-multihash" + + ds "github.com/ipfs/go-datastore" + "github.com/ipfs/go-datastore/namespace" + query "github.com/ipfs/go-datastore/query" + crdt "github.com/ipfs/go-ds-crdt" + logging "github.com/ipfs/go-log" + rpc "github.com/libp2p/go-libp2p-gorpc" + host "github.com/libp2p/go-libp2p-host" + dht "github.com/libp2p/go-libp2p-kad-dht" + peer "github.com/libp2p/go-libp2p-peer" + pubsub "github.com/libp2p/go-libp2p-pubsub" +) + +var logger = logging.Logger("crdt") + +var ( + blocksNs = "b" // blockstore namespace +) + +// Common variables for the module. +var ( + ErrNoLeader = errors.New("crdt consensus component does not provide a leader") + ErrRmPeer = errors.New("crdt consensus component cannot remove peers") +) + +// Consensus implement ipfscluster.Consensus and provides the facility to add +// and remove pins from the Cluster shared state. It uses a CRDT-backed +// implementation of go-datastore (go-ds-crdt). +type Consensus struct { + ctx context.Context + cancel context.CancelFunc + + config *Config + + host host.Host + + store ds.Datastore + namespace ds.Key + + state state.State + crdt *crdt.Datastore + + dht *dht.IpfsDHT + pubsub *pubsub.PubSub + + rpcClient *rpc.Client + rpcReady chan struct{} + readyCh chan struct{} + + shutdownLock sync.RWMutex + shutdown bool +} + +// New creates a new crdt Consensus component. The given PubSub will be used to +// broadcast new heads. The given thread-safe datastore will be used to persist +// data and all will be prefixed with cfg.DatastoreNamespace. +func New( + host host.Host, + dht *dht.IpfsDHT, + pubsub *pubsub.PubSub, + cfg *Config, + store ds.Datastore, +) (*Consensus, error) { + err := cfg.Validate() + if err != nil { + return nil, err + } + + ctx, cancel := context.WithCancel(context.Background()) + + css := &Consensus{ + ctx: ctx, + cancel: cancel, + config: cfg, + host: host, + dht: dht, + store: store, + namespace: ds.NewKey(cfg.DatastoreNamespace), + pubsub: pubsub, + rpcReady: make(chan struct{}, 1), + readyCh: make(chan struct{}, 1), + } + + go css.setup() + return css, nil +} + +func (css *Consensus) setup() { + select { + case <-css.ctx.Done(): + return + case <-css.rpcReady: + } + + // Hash the cluster name and produce the topic name from there + // as a way to avoid pubsub topic collisions with other + // pubsub applications potentially when both potentially use + // simple names like "test". + topicName := css.config.ClusterName + topicHash, err := multihash.Sum([]byte(css.config.ClusterName), multihash.MD5, -1) + if err != nil { + logger.Errorf("error hashing topic: %s", err) + } else { + topicName = topicHash.B58String() + } + + // Validate pubsub messages for our topic (only accept + // from trusted sources) + err = css.pubsub.RegisterTopicValidator( + topicName, + func(ctx context.Context, p peer.ID, msg *pubsub.Message) bool { + // This is where peer authentication will go. + return true + }, + ) + if err != nil { + logger.Errorf("error registering topic validator: %s", err) + } + + var blocksDatastore ds.Batching + blocksDatastore = namespace.Wrap(css.store, css.namespace.ChildString(blocksNs)) + + ipfs, err := ipfslite.New( + css.ctx, + blocksDatastore, + css.host, + css.dht, + &ipfslite.Config{ + Offline: false, + }, + ) + if err != nil { + logger.Errorf("error creating ipfs-lite: %s", err) + return + } + + dagSyncer := newLiteDAGSyncer(css.ctx, ipfs) + + broadcaster, err := crdt.NewPubSubBroadcaster( + css.ctx, + css.pubsub, + topicName, // subscription name + ) + if err != nil { + logger.Errorf("error creating broadcaster: %s", err) + return + } + + opts := crdt.DefaultOptions() + opts.RebroadcastInterval = css.config.RebroadcastInterval + opts.Logger = logger + opts.PutHook = func(k ds.Key, v []byte) { + pin := &api.Pin{} + err := pin.ProtoUnmarshal(v) + if err != nil { + logger.Error(err) + return + } + + // TODO: tracing for this context + err = css.rpcClient.CallContext( + css.ctx, + "", + "Cluster", + "Track", + pin, + &struct{}{}, + ) + if err != nil { + logger.Error(err) + } + } + opts.DeleteHook = func(k ds.Key) { + c, err := dshelp.DsKeyToCid(k) + if err != nil { + logger.Error(err, k) + return + } + pin := api.PinCid(c) + + err = css.rpcClient.CallContext( + css.ctx, + "", + "Cluster", + "Untrack", + pin, + &struct{}{}, + ) + if err != nil { + logger.Error(err) + } + } + + crdt, err := crdt.New( + css.store, + css.namespace, + dagSyncer, + broadcaster, + opts, + ) + if err != nil { + logger.Error(err) + return + } + + css.crdt = crdt + + clusterState, err := dsstate.New( + css.crdt, + // unsure if we should set something else but crdt is already + // namespaced and this would only namespace the keys, which only + // complicates things. + "", + dsstate.DefaultHandle(), + ) + if err != nil { + logger.Errorf("error creating cluster state datastore: %s", err) + return + } + css.state = clusterState + css.readyCh <- struct{}{} +} + +// Shutdown closes this component, cancelling the pubsub subscription. +func (css *Consensus) Shutdown(ctx context.Context) error { + css.shutdownLock.Lock() + defer css.shutdownLock.Unlock() + + if css.shutdown { + logger.Debug("already shutdown") + return nil + } + + logger.Info("stopping Consensus component") + + css.cancel() + + // Only close crdt after cancelling the context, otherwise + // the pubsub broadcaster stays on and locks it. + if crdt := css.crdt; crdt != nil { + crdt.Close() + } + + if css.config.hostShutdown { + css.host.Close() + } + + css.shutdown = true + close(css.rpcReady) + return nil +} + +// SetClient gives the component the ability to communicate and +// leaves it ready to use. +func (css *Consensus) SetClient(c *rpc.Client) { + css.rpcClient = c + css.rpcReady <- struct{}{} +} + +// Ready returns a channel which is signalled when the component +// is ready to use. +func (css *Consensus) Ready(ctx context.Context) <-chan struct{} { + return css.readyCh +} + +// LogPin adds a new pin to the shared state. +func (css *Consensus) LogPin(ctx context.Context, pin *api.Pin) error { + return css.state.Add(ctx, pin) +} + +// LogUnpin removes a pin from the shared state. +func (css *Consensus) LogUnpin(ctx context.Context, pin *api.Pin) error { + return css.state.Rm(ctx, pin.Cid) +} + +// Peers returns the current known peerset. It uses +// the monitor component and considers every peer with +// valid known metrics a member. +func (css *Consensus) Peers(ctx context.Context) ([]peer.ID, error) { + var metrics []*api.Metric + + err := css.rpcClient.CallContext( + ctx, + "", + "Cluster", + "PeerMonitorLatestMetrics", + css.config.PeersetMetric, + &metrics, + ) + if err != nil { + return nil, err + } + + var peers []peer.ID + + selfIncluded := false + for _, m := range metrics { + peers = append(peers, m.Peer) + if m.Peer == css.host.ID() { + selfIncluded = true + } + } + + // Always include self + if !selfIncluded { + peers = append(peers, css.host.ID()) + } + + return peers, nil +} + +// WaitForSync is a no-op as it is not necessary to be fully synced for the +// component to be usable. +func (css *Consensus) WaitForSync(ctx context.Context) error { return nil } + +// AddPeer is a no-op as we do not need to do peerset management with +// Merkle-CRDTs. Therefore adding a peer to the peerset means doing nothing. +func (css *Consensus) AddPeer(ctx context.Context, pid peer.ID) error { return nil } + +// RmPeer is a no-op which always errors, as, since we do not do peerset +// management, we also have no ability to remove a peer from it. +func (css *Consensus) RmPeer(ctx context.Context, pid peer.ID) error { + return ErrRmPeer +} + +// State returns the cluster shared state. +func (css *Consensus) State(ctx context.Context) (state.ReadOnly, error) { return css.state, nil } + +// Clean deletes all crdt-consensus datas from the datastore. +func (css *Consensus) Clean(ctx context.Context) error { + return Clean(ctx, css.config, css.store) +} + +// Clean deletes all crdt-consensus datas from the given datastore. +func Clean(ctx context.Context, cfg *Config, store ds.Datastore) error { + logger.Info("cleaning all CRDT data from datastore") + q := query.Query{ + Prefix: cfg.DatastoreNamespace, + KeysOnly: true, + } + + results, err := store.Query(q) + if err != nil { + return err + } + defer results.Close() + + for r := range results.Next() { + if r.Error != nil { + return err + } + k := ds.NewKey(r.Key) + err := store.Delete(k) + if err != nil { + // do not die, continue cleaning + logger.Error(err) + } + } + return nil +} + +// Leader returns ErrNoLeader. +func (css *Consensus) Leader(ctx context.Context) (peer.ID, error) { + return "", ErrNoLeader +} + +// OfflineState returns an offline, read-only batching state using the given +// datastore. Any writes to this state are processed through the given +// ipfs connector (the state is offline as it does not require a +// running cluster peer). +func OfflineState(cfg *Config, store ds.Datastore) (state.BatchingState, error) { + batching, ok := store.(ds.Batching) + if !ok { + return nil, errors.New("must provide a Bathing datastore") + } + opts := crdt.DefaultOptions() + opts.Logger = logger + + var blocksDatastore ds.Batching + blocksDatastore = namespace.Wrap( + batching, + ds.NewKey(cfg.DatastoreNamespace).ChildString(blocksNs), + ) + + ipfs, err := ipfslite.New( + context.Background(), + blocksDatastore, + nil, + nil, + &ipfslite.Config{ + Offline: true, + }, + ) + + if err != nil { + return nil, err + } + + dags := newLiteDAGSyncer(context.Background(), ipfs) + + crdt, err := crdt.New( + batching, + ds.NewKey(cfg.DatastoreNamespace), + dags, + nil, + opts, + ) + if err != nil { + return nil, err + } + return dsstate.NewBatching(crdt, "", dsstate.DefaultHandle()) +} diff --git a/consensus/crdt/consensus_test.go b/consensus/crdt/consensus_test.go new file mode 100644 index 00000000..fc722930 --- /dev/null +++ b/consensus/crdt/consensus_test.go @@ -0,0 +1,293 @@ +package crdt + +import ( + "context" + "fmt" + "testing" + "time" + + cid "github.com/ipfs/go-cid" + "github.com/ipfs/ipfs-cluster/api" + "github.com/ipfs/ipfs-cluster/datastore/inmem" + "github.com/ipfs/ipfs-cluster/test" + + libp2p "github.com/libp2p/go-libp2p" + host "github.com/libp2p/go-libp2p-host" + dht "github.com/libp2p/go-libp2p-kad-dht" + peerstore "github.com/libp2p/go-libp2p-peerstore" + pubsub "github.com/libp2p/go-libp2p-pubsub" + routedhost "github.com/libp2p/go-libp2p/p2p/host/routed" +) + +func makeTestingHost(t *testing.T) (host.Host, *pubsub.PubSub, *dht.IpfsDHT) { + ctx := context.Background() + h, err := libp2p.New( + ctx, + libp2p.ListenAddrStrings("/ip4/127.0.0.1/tcp/0"), + ) + if err != nil { + t.Fatal(err) + } + + psub, err := pubsub.NewGossipSub( + ctx, + h, + pubsub.WithMessageSigning(true), + pubsub.WithStrictSignatureVerification(true), + ) + if err != nil { + h.Close() + t.Fatal(err) + } + + idht, err := dht.New(ctx, h) + if err != nil { + h.Close() + t.Fatal(err) + } + + btstrCfg := dht.BootstrapConfig{ + Queries: 1, + Period: 200 * time.Millisecond, + Timeout: 100 * time.Millisecond, + } + + err = idht.BootstrapWithConfig(ctx, btstrCfg) + if err != nil { + h.Close() + t.Fatal(err) + } + + rHost := routedhost.Wrap(h, idht) + return rHost, psub, idht +} + +func testingConsensus(t *testing.T, idn int) *Consensus { + h, psub, dht := makeTestingHost(t) + + cfg := &Config{} + cfg.Default() + cfg.DatastoreNamespace = fmt.Sprintf("crdttest-%d", idn) + cfg.hostShutdown = true + + cc, err := New(h, dht, psub, cfg, inmem.New()) + if err != nil { + t.Fatal("cannot create Consensus:", err) + } + cc.SetClient(test.NewMockRPCClientWithHost(t, h)) + <-cc.Ready(context.Background()) + return cc +} + +func clean(t *testing.T, cc *Consensus) { + err := cc.Clean(context.Background()) + if err != nil { + t.Error(err) + } +} + +func testPin(c cid.Cid) *api.Pin { + p := api.PinCid(c) + p.ReplicationFactorMin = -1 + p.ReplicationFactorMax = -1 + return p +} + +func TestShutdownConsensus(t *testing.T) { + ctx := context.Background() + cc := testingConsensus(t, 1) + defer clean(t, cc) + err := cc.Shutdown(ctx) + if err != nil { + t.Fatal("Consensus cannot shutdown:", err) + } + err = cc.Shutdown(ctx) // should be fine to shutdown twice + if err != nil { + t.Fatal("Consensus should be able to shutdown several times") + } +} + +func TestConsensusPin(t *testing.T) { + ctx := context.Background() + cc := testingConsensus(t, 1) + defer clean(t, cc) + defer cc.Shutdown(ctx) + + err := cc.LogPin(ctx, testPin(test.Cid1)) + if err != nil { + t.Error(err) + } + + time.Sleep(250 * time.Millisecond) + st, err := cc.State(ctx) + if err != nil { + t.Fatal("error getting state:", err) + } + + pins, err := st.List(ctx) + if err != nil { + t.Fatal(err) + } + if len(pins) != 1 || !pins[0].Cid.Equals(test.Cid1) { + t.Error("the added pin should be in the state") + } +} + +func TestConsensusUnpin(t *testing.T) { + ctx := context.Background() + cc := testingConsensus(t, 1) + defer clean(t, cc) + defer cc.Shutdown(ctx) + + err := cc.LogPin(ctx, testPin(test.Cid1)) + if err != nil { + t.Error(err) + } + + err = cc.LogUnpin(ctx, api.PinCid(test.Cid1)) + if err != nil { + t.Error(err) + } +} + +func TestConsensusUpdate(t *testing.T) { + ctx := context.Background() + cc := testingConsensus(t, 1) + defer clean(t, cc) + defer cc.Shutdown(ctx) + + // Pin first + pin := testPin(test.Cid1) + pin.Type = api.ShardType + err := cc.LogPin(ctx, pin) + if err != nil { + t.Fatal(err) + } + time.Sleep(250 * time.Millisecond) + + // Update pin + pin.Reference = &test.Cid2 + err = cc.LogPin(ctx, pin) + if err != nil { + t.Error(err) + } + + time.Sleep(250 * time.Millisecond) + st, err := cc.State(ctx) + if err != nil { + t.Fatal("error getting state:", err) + } + + pins, err := st.List(ctx) + if err != nil { + t.Fatal(err) + } + if len(pins) != 1 || !pins[0].Cid.Equals(test.Cid1) { + t.Error("the added pin should be in the state") + } + if !pins[0].Reference.Equals(test.Cid2) { + t.Error("pin updated incorrectly") + } +} + +func TestConsensusAddRmPeer(t *testing.T) { + ctx := context.Background() + cc := testingConsensus(t, 1) + cc2 := testingConsensus(t, 2) + defer clean(t, cc) + defer clean(t, cc) + defer cc.Shutdown(ctx) + defer cc2.Shutdown(ctx) + + cc.host.Peerstore().AddAddrs(cc2.host.ID(), cc2.host.Addrs(), peerstore.PermanentAddrTTL) + _, err := cc.host.Network().DialPeer(ctx, cc2.host.ID()) + if err != nil { + t.Fatal(err) + } + time.Sleep(100 * time.Millisecond) + + err = cc.AddPeer(ctx, cc2.host.ID()) + if err != nil { + t.Error("could not add peer:", err) + } + + // Make a pin on peer1 and check it arrived to peer2 + err = cc.LogPin(ctx, testPin(test.Cid1)) + if err != nil { + t.Error(err) + } + + time.Sleep(250 * time.Millisecond) + st, err := cc2.State(ctx) + if err != nil { + t.Fatal("error getting state:", err) + } + + pins, err := st.List(ctx) + if err != nil { + t.Fatal(err) + } + if len(pins) != 1 || !pins[0].Cid.Equals(test.Cid1) { + t.Error("the added pin should be in the state") + } + + err = cc2.RmPeer(ctx, cc.host.ID()) + if err == nil { + t.Error("crdt consensus should not remove pins") + } +} + +func TestPeers(t *testing.T) { + ctx := context.Background() + cc := testingConsensus(t, 1) + defer clean(t, cc) + defer cc.Shutdown(ctx) + + peers, err := cc.Peers(ctx) + if err != nil { + t.Fatal(err) + } + + // 1 is ourselves and the other comes from rpc + // mock PeerMonitorLatestMetrics + if len(peers) != 2 { + t.Error("unexpected number of peers") + } +} + +func TestOfflineState(t *testing.T) { + ctx := context.Background() + cc := testingConsensus(t, 1) + defer clean(t, cc) + defer cc.Shutdown(ctx) + + // Make pin 1 + err := cc.LogPin(ctx, testPin(test.Cid1)) + if err != nil { + t.Error(err) + } + + // Make pin 2 + err = cc.LogPin(ctx, testPin(test.Cid2)) + if err != nil { + t.Error(err) + } + + err = cc.Shutdown(ctx) + if err != nil { + t.Fatal(err) + } + + offlineState, err := OfflineState(cc.config, cc.store) + if err != nil { + t.Fatal(err) + } + + pins, err := offlineState.List(ctx) + if err != nil { + t.Fatal(err) + } + if len(pins) != 2 { + t.Error("there should be two pins in the state") + } +} diff --git a/consensus/crdt/dagsyncer.go b/consensus/crdt/dagsyncer.go new file mode 100644 index 00000000..8828e21c --- /dev/null +++ b/consensus/crdt/dagsyncer.go @@ -0,0 +1,73 @@ +package crdt + +import ( + "context" + "time" + + ipfslite "github.com/hsanjuan/ipfs-lite" + cid "github.com/ipfs/go-cid" + blockstore "github.com/ipfs/go-ipfs-blockstore" + ipld "github.com/ipfs/go-ipld-format" +) + +// A DAGSyncer component implementation using ipfs-lite. +type liteDAGSyncer struct { + getter ipld.NodeGetter + dags ipld.DAGService + + ipfs *ipfslite.Peer + blockstore blockstore.Blockstore + + timeout time.Duration +} + +func newLiteDAGSyncer(ctx context.Context, ipfs *ipfslite.Peer) *liteDAGSyncer { + return &liteDAGSyncer{ + getter: ipfs, // should we use ipfs.Session() ?? + dags: ipfs, + ipfs: ipfs, + blockstore: ipfs.BlockStore(), + timeout: time.Minute, + } +} + +func (lds *liteDAGSyncer) HasBlock(c cid.Cid) (bool, error) { + return lds.ipfs.HasBlock(c) +} + +func (lds *liteDAGSyncer) Get(ctx context.Context, c cid.Cid) (ipld.Node, error) { + ctx, cancel := context.WithTimeout(ctx, lds.timeout) + defer cancel() + return lds.getter.Get(ctx, c) +} + +func (lds *liteDAGSyncer) GetMany(ctx context.Context, cs []cid.Cid) <-chan *ipld.NodeOption { + ctx, cancel := context.WithTimeout(ctx, lds.timeout) + defer cancel() + return lds.getter.GetMany(ctx, cs) +} + +func (lds *liteDAGSyncer) Add(ctx context.Context, n ipld.Node) error { + ctx, cancel := context.WithTimeout(ctx, lds.timeout) + defer cancel() + return lds.dags.Add(ctx, n) +} + +func (lds *liteDAGSyncer) AddMany(ctx context.Context, nds []ipld.Node) error { + ctx, cancel := context.WithTimeout(ctx, lds.timeout) + defer cancel() + return lds.dags.AddMany(ctx, nds) + +} + +func (lds *liteDAGSyncer) Remove(ctx context.Context, c cid.Cid) error { + ctx, cancel := context.WithTimeout(ctx, lds.timeout) + defer cancel() + return lds.dags.Remove(ctx, c) +} + +func (lds *liteDAGSyncer) RemoveMany(ctx context.Context, cs []cid.Cid) error { + ctx, cancel := context.WithTimeout(ctx, lds.timeout) + defer cancel() + return lds.dags.RemoveMany(ctx, cs) +} diff --git a/consensus/raft/config.go b/consensus/raft/config.go index 5cfdf227..8f6ad151 100644 --- a/consensus/raft/config.go +++ b/consensus/raft/config.go @@ -28,6 +28,7 @@ var ( DefaultNetworkTimeout = 10 * time.Second DefaultCommitRetryDelay = 200 * time.Millisecond DefaultBackupsRotate = 6 + DefaultDatastoreNamespace = "/r" // from "/raft" ) // Config allows to configure the Raft Consensus component for ipfs-cluster. @@ -60,12 +61,13 @@ type Config struct { // BackupsRotate specifies the maximum number of Raft's DataFolder // copies that we keep as backups (renaming) after cleanup. BackupsRotate int + // Namespace to use when writing keys to the datastore + DatastoreNamespace string // A Hashicorp Raft's configuration object. RaftConfig *hraft.Config - // Tracing enables propagation of contexts across binary boundary in - // consensus component + // Tracing enables propagation of contexts across binary boundaries. Tracing bool } @@ -101,6 +103,8 @@ type jsonConfig struct { // copies that we keep as backups (renaming) after cleanup. BackupsRotate int `json:"backups_rotate"` + DatastoreNamespace string `json:"datastore_namespace,omitempty"` + // HeartbeatTimeout specifies the time in follower state without // a leader before we attempt an election. HeartbeatTimeout string `json:"heartbeat_timeout,omitempty"` @@ -242,7 +246,7 @@ func (cfg *Config) ToJSON() ([]byte, error) { } func (cfg *Config) toJSONConfig() *jsonConfig { - return &jsonConfig{ + jcfg := &jsonConfig{ DataFolder: cfg.DataFolder, InitPeerset: api.PeersToStrings(cfg.InitPeerset), WaitForLeaderTimeout: cfg.WaitForLeaderTimeout.String(), @@ -259,6 +263,11 @@ func (cfg *Config) toJSONConfig() *jsonConfig { SnapshotThreshold: cfg.RaftConfig.SnapshotThreshold, LeaderLeaseTimeout: cfg.RaftConfig.LeaderLeaseTimeout.String(), } + if cfg.DatastoreNamespace != DefaultDatastoreNamespace { + jcfg.DatastoreNamespace = cfg.DatastoreNamespace + // otherwise leave empty so it gets ommitted. + } + return jcfg } // Default initializes this configuration with working defaults. @@ -270,6 +279,7 @@ func (cfg *Config) Default() error { cfg.CommitRetries = DefaultCommitRetries cfg.CommitRetryDelay = DefaultCommitRetryDelay cfg.BackupsRotate = DefaultBackupsRotate + cfg.DatastoreNamespace = DefaultDatastoreNamespace cfg.RaftConfig = hraft.DefaultConfig() // These options are imposed over any Default Raft Config. diff --git a/consensus/raft/consensus.go b/consensus/raft/consensus.go index ced96cf3..7328f357 100644 --- a/consensus/raft/consensus.go +++ b/consensus/raft/consensus.go @@ -15,7 +15,9 @@ import ( "github.com/ipfs/ipfs-cluster/api" "github.com/ipfs/ipfs-cluster/state" + "github.com/ipfs/ipfs-cluster/state/dsstate" + ds "github.com/ipfs/go-datastore" logging "github.com/ipfs/go-log" consensus "github.com/libp2p/go-libp2p-consensus" rpc "github.com/libp2p/go-libp2p-gorpc" @@ -25,7 +27,7 @@ import ( ma "github.com/multiformats/go-multiaddr" ) -var logger = logging.Logger("consensus") +var logger = logging.Logger("raft") // Consensus handles the work of keeping a shared-state between // the peers of an IPFS Cluster, as well as modifying that state and @@ -50,15 +52,20 @@ type Consensus struct { shutdown bool } -// NewConsensus builds a new ClusterConsensus component using Raft. The state -// is used to initialize the Consensus system, so any information -// in it is discarded once the raft state is loaded. -// The singlePeer parameter controls whether this Raft peer is be expected to -// join a cluster or it should run on its own. +// NewConsensus builds a new ClusterConsensus component using Raft. +// +// Raft saves state snapshots regularly and persists log data in a bolt +// datastore. Therefore, unless memory usage is a concern, it is recommended +// to use an in-memory go-datastore as store parameter. +// +// The staging parameter controls if the Raft peer should start in +// staging mode (used when joining a new Raft peerset with other peers). +// +// The store parameter should be a thread-safe datastore. func NewConsensus( host host.Host, cfg *Config, - state state.State, + store ds.Datastore, staging bool, // this peer must not be bootstrapped if no state exists ) (*Consensus, error) { err := cfg.Validate() @@ -66,9 +73,16 @@ func NewConsensus( return nil, err } - baseOp := &LogOp{tracing: cfg.Tracing} - logger.Debug("starting Consensus and waiting for a leader...") + baseOp := &LogOp{tracing: cfg.Tracing} + state, err := dsstate.New( + store, + cfg.DatastoreNamespace, + dsstate.DefaultHandle(), + ) + if err != nil { + return nil, err + } consensus := libp2praft.NewOpLog(state, baseOp) raft, err := newRaftWrapper(host, cfg, consensus.FSM(), staging) if err != nil { @@ -385,6 +399,7 @@ func (cc *Consensus) AddPeer(ctx context.Context, pid peer.ID) error { // Being here means we are the leader and can commit cc.shutdownLock.RLock() // do not shutdown while committing finalErr = cc.raft.AddPeer(ctx, peer.IDB58Encode(pid)) + cc.shutdownLock.RUnlock() if finalErr != nil { time.Sleep(cc.config.CommitRetryDelay) @@ -426,15 +441,20 @@ func (cc *Consensus) RmPeer(ctx context.Context, pid peer.ID) error { return finalErr } -// State retrieves the current consensus State. It may error -// if no State has been agreed upon or the state is not -// consistent. The returned State is the last agreed-upon -// State known by this node. -func (cc *Consensus) State(ctx context.Context) (state.State, error) { +// State retrieves the current consensus State. It may error if no State has +// been agreed upon or the state is not consistent. The returned State is the +// last agreed-upon State known by this node. No writes are allowed, as all +// writes to the shared state should happen through the Consensus component +// methods. +func (cc *Consensus) State(ctx context.Context) (state.ReadOnly, error) { ctx, span := trace.StartSpan(ctx, "consensus/State") defer span.End() st, err := cc.consensus.GetLogHead() + if err == libp2praft.ErrNoState { + return state.Empty(), nil + } + if err != nil { return nil, err } @@ -456,8 +476,7 @@ func (cc *Consensus) Leader(ctx context.Context) (peer.ID, error) { return raftactor.Leader() } -// Clean removes all raft data from disk. Next time -// a full new peer will be bootstrapped. +// Clean removes the Raft persisted state. func (cc *Consensus) Clean(ctx context.Context) error { ctx, span := trace.StartSpan(ctx, "consensus/Clean") defer span.End() @@ -468,11 +487,7 @@ func (cc *Consensus) Clean(ctx context.Context) error { return errors.New("consensus component is not shutdown") } - err := cc.raft.Clean() - if err != nil { - return err - } - return nil + return CleanupRaft(cc.config) } // Rollback replaces the current agreed-upon @@ -522,3 +537,28 @@ func parsePIDFromMultiaddr(addr ma.Multiaddr) string { } return pidstr } + +// OfflineState state returns a cluster state by reading the Raft data and +// writing it to the given datastore which is then wrapped as a state.State. +// Usually an in-memory datastore suffices. The given datastore should be +// thread-safe. +func OfflineState(cfg *Config, store ds.Datastore) (state.State, error) { + r, snapExists, err := LastStateRaw(cfg) + if err != nil { + return nil, err + } + + st, err := dsstate.New(store, cfg.DatastoreNamespace, dsstate.DefaultHandle()) + if err != nil { + return nil, err + } + if !snapExists { + return st, nil + } + + err = st.Unmarshal(r) + if err != nil { + return nil, err + } + return st, nil +} diff --git a/consensus/raft/consensus_test.go b/consensus/raft/consensus_test.go index b7c8a8a4..3585219f 100644 --- a/consensus/raft/consensus_test.go +++ b/consensus/raft/consensus_test.go @@ -8,7 +8,8 @@ import ( "time" "github.com/ipfs/ipfs-cluster/api" - "github.com/ipfs/ipfs-cluster/state/mapstate" + "github.com/ipfs/ipfs-cluster/datastore/inmem" + "github.com/ipfs/ipfs-cluster/state/dsstate" "github.com/ipfs/ipfs-cluster/test" cid "github.com/ipfs/go-cid" @@ -43,14 +44,13 @@ func testingConsensus(t *testing.T, idn int) *Consensus { ctx := context.Background() cleanRaft(idn) h := makeTestingHost(t) - st := mapstate.NewMapState() cfg := &Config{} cfg.Default() cfg.DataFolder = fmt.Sprintf("raftFolderFromTests-%d", idn) cfg.hostShutdown = true - cc, err := NewConsensus(h, cfg, st, false) + cc, err := NewConsensus(h, cfg, inmem.New(), false) if err != nil { t.Fatal("cannot create Consensus:", err) } @@ -100,7 +100,10 @@ func TestConsensusPin(t *testing.T) { t.Fatal("error getting state:", err) } - pins := st.List(ctx) + pins, err := st.List(ctx) + if err != nil { + t.Fatal(err) + } if len(pins) != 1 || !pins[0].Cid.Equals(test.Cid1) { t.Error("the added pin should be in the state") } @@ -146,7 +149,10 @@ func TestConsensusUpdate(t *testing.T) { t.Fatal("error getting state:", err) } - pins := st.List(ctx) + pins, err := st.List(ctx) + if err != nil { + t.Fatal(err) + } if len(pins) != 1 || !pins[0].Cid.Equals(test.Cid1) { t.Error("the added pin should be in the state") } @@ -298,7 +304,10 @@ func TestRaftLatestSnapshot(t *testing.T) { } // Call raft.LastState and ensure we get the correct state - snapState := mapstate.NewMapState() + snapState, err := dsstate.New(inmem.New(), "", dsstate.DefaultHandle()) + if err != nil { + t.Fatal(err) + } r, snapExists, err := LastStateRaw(cc.config) if !snapExists { t.Fatal("No snapshot found by LastStateRaw") @@ -306,11 +315,14 @@ func TestRaftLatestSnapshot(t *testing.T) { if err != nil { t.Fatal("Error while taking snapshot", err) } - err = snapState.Migrate(ctx, r) + err = snapState.Unmarshal(r) if err != nil { t.Fatal("Snapshot bytes returned could not restore to state: ", err) } - pins := snapState.List(ctx) + pins, err := snapState.List(ctx) + if err != nil { + t.Fatal(err) + } if len(pins) != 3 { t.Fatal("Latest snapshot not read") } diff --git a/consensus/raft/log_op_test.go b/consensus/raft/log_op_test.go index 5d5b20e0..e52f49f2 100644 --- a/consensus/raft/log_op_test.go +++ b/consensus/raft/log_op_test.go @@ -5,7 +5,8 @@ import ( "testing" "github.com/ipfs/ipfs-cluster/api" - "github.com/ipfs/ipfs-cluster/state/mapstate" + "github.com/ipfs/ipfs-cluster/datastore/inmem" + "github.com/ipfs/ipfs-cluster/state/dsstate" "github.com/ipfs/ipfs-cluster/test" ) @@ -20,9 +21,16 @@ func TestApplyToPin(t *testing.T) { defer cleanRaft(1) defer cc.Shutdown(ctx) - st := mapstate.NewMapState() + st, err := dsstate.New(inmem.New(), "", dsstate.DefaultHandle()) + if err != nil { + t.Fatal(err) + } op.ApplyTo(st) - pins := st.List(ctx) + + pins, err := st.List(ctx) + if err != nil { + t.Fatal(err) + } if len(pins) != 1 || !pins[0].Cid.Equals(test.Cid1) { t.Error("the state was not modified correctly") } @@ -39,10 +47,16 @@ func TestApplyToUnpin(t *testing.T) { defer cleanRaft(1) defer cc.Shutdown(ctx) - st := mapstate.NewMapState() + st, err := dsstate.New(inmem.New(), "", dsstate.DefaultHandle()) + if err != nil { + t.Fatal(err) + } st.Add(ctx, testPin(test.Cid1)) op.ApplyTo(st) - pins := st.List(ctx) + pins, err := st.List(ctx) + if err != nil { + t.Fatal(err) + } if len(pins) != 0 { t.Error("the state was not modified correctly") } @@ -63,21 +77,3 @@ func TestApplyToBadState(t *testing.T) { var st interface{} op.ApplyTo(st) } - -// func TestApplyToBadCid(t *testing.T) { -// defer func() { -// if r := recover(); r == nil { -// t.Error("should have recovered an error") -// } -// }() - -// op := &LogOp{ -// Cid: api.PinSerial{Cid: "agadfaegf"}, -// Type: LogOpPin, -// ctx: context.Background(), -// rpcClient: test.NewMockRPCClient(t), -// } - -// st := mapstate.NewMapState() -// op.ApplyTo(st) -// } diff --git a/consensus/raft/logging.go b/consensus/raft/logging.go index 33440b62..cb394869 100644 --- a/consensus/raft/logging.go +++ b/consensus/raft/logging.go @@ -26,7 +26,7 @@ type logForwarder struct { } var raftStdLogger = log.New(&logForwarder{}, "", 0) -var raftLogger = logging.Logger("raft") +var raftLogger = logging.Logger("raftlib") // Write forwards to our go-log logger. // According to https://golang.org/pkg/log/#Logger.Output diff --git a/consensus/raft/raft.go b/consensus/raft/raft.go index 31438f4d..8d1f67fa 100644 --- a/consensus/raft/raft.go +++ b/consensus/raft/raft.go @@ -616,7 +616,7 @@ func SnapshotSave(cfg *Config, newState state.State, pids []peer.ID) error { raftIndex = meta.Index raftTerm = meta.Term srvCfg = meta.Configuration - CleanupRaft(dataFolder, cfg.BackupsRotate) + CleanupRaft(cfg) } else { // Begin the log after the index of a fresh start so that // the snapshot's state propagate's during bootstrap @@ -649,7 +649,10 @@ func SnapshotSave(cfg *Config, newState state.State, pids []peer.ID) error { } // CleanupRaft moves the current data folder to a backup location -func CleanupRaft(dataFolder string, keep int) error { +func CleanupRaft(cfg *Config) error { + dataFolder := cfg.GetDataFolder() + keep := cfg.BackupsRotate + meta, _, err := latestSnapshot(dataFolder) if meta == nil && err == nil { // no snapshots at all. Avoid creating backups @@ -672,7 +675,7 @@ func CleanupRaft(dataFolder string, keep int) error { // only call when Raft is shutdown func (rw *raftWrapper) Clean() error { - return CleanupRaft(rw.config.GetDataFolder(), rw.config.BackupsRotate) + return CleanupRaft(rw.config) } func find(s []string, elem string) bool { diff --git a/datastore/badger/badger.go b/datastore/badger/badger.go new file mode 100644 index 00000000..0201361f --- /dev/null +++ b/datastore/badger/badger.go @@ -0,0 +1,33 @@ +// Package badger provides a configurable BadgerDB go-datastore for use with +// IPFS Cluster. +package badger + +import ( + "os" + + ds "github.com/ipfs/go-datastore" + badgerds "github.com/ipfs/go-ds-badger" + "github.com/pkg/errors" +) + +// New returns a BadgerDB datastore configured with the given +// configuration. +func New(cfg *Config) (ds.Datastore, error) { + folder := cfg.GetFolder() + err := os.MkdirAll(folder, 0700) + if err != nil { + return nil, errors.Wrap(err, "creating badger folder") + } + opts := &badgerds.DefaultOptions + return badgerds.NewDatastore(folder, opts) +} + +// Cleanup deletes the badger datastore. +func Cleanup(cfg *Config) error { + folder := cfg.GetFolder() + if _, err := os.Stat(folder); os.IsNotExist(err) { + return nil + } + return os.RemoveAll(cfg.GetFolder()) + +} diff --git a/datastore/badger/config.go b/datastore/badger/config.go new file mode 100644 index 00000000..20d7e2ef --- /dev/null +++ b/datastore/badger/config.go @@ -0,0 +1,113 @@ +package badger + +import ( + "encoding/json" + "errors" + "path/filepath" + + "github.com/kelseyhightower/envconfig" + + "github.com/ipfs/ipfs-cluster/config" +) + +const configKey = "badger" +const envConfigKey = "cluster_badger" + +// Default values for badger Config +const ( + DefaultSubFolder = "badger" +) + +// Config is used to initialize a BadgerDB datastore. It implements the +// ComponentConfig interface. +type Config struct { + config.Saver + + // The folder for this datastore. Non-absolute paths are relative to + // the base configuration folder. + Folder string +} + +type jsonConfig struct { + Folder string `json:"folder,omitempty"` +} + +// ConfigKey returns a human-friendly identifier for this type of Datastore. +func (cfg *Config) ConfigKey() string { + return configKey +} + +// Default initializes this Config with sensible values. +func (cfg *Config) Default() error { + cfg.Folder = DefaultSubFolder + return nil +} + +// ApplyEnvVars fills in any Config fields found as environment variables. +func (cfg *Config) ApplyEnvVars() error { + jcfg := cfg.toJSONConfig() + + err := envconfig.Process(envConfigKey, jcfg) + if err != nil { + return err + } + + return cfg.applyJSONConfig(jcfg) +} + +// Validate checks that the fields of this Config have working values, +// at least in appearance. +func (cfg *Config) Validate() error { + if cfg.Folder == "" { + return errors.New("folder is unset") + } + + return nil +} + +// LoadJSON reads the fields of this Config from a JSON byteslice as +// generated by ToJSON. +func (cfg *Config) LoadJSON(raw []byte) error { + jcfg := &jsonConfig{} + err := json.Unmarshal(raw, jcfg) + if err != nil { + return err + } + + cfg.Default() + + return cfg.applyJSONConfig(jcfg) +} + +func (cfg *Config) applyJSONConfig(jcfg *jsonConfig) error { + config.SetIfNotDefault(jcfg.Folder, &cfg.Folder) + return cfg.Validate() +} + +// ToJSON generates a JSON-formatted human-friendly representation of this +// Config. +func (cfg *Config) ToJSON() (raw []byte, err error) { + jcfg := cfg.toJSONConfig() + + raw, err = config.DefaultJSONMarshal(jcfg) + return +} + +func (cfg *Config) toJSONConfig() *jsonConfig { + jCfg := &jsonConfig{} + + if cfg.Folder != DefaultSubFolder { + jCfg.Folder = cfg.Folder + } + + return jCfg +} + +// GetFolder returns the BadgerDB folder. +func (cfg *Config) GetFolder() string { + if filepath.IsAbs(cfg.Folder) { + return cfg.Folder + } + + return filepath.Join(cfg.BaseDir, cfg.Folder) +} diff --git a/datastore/inmem/inmem.go b/datastore/inmem/inmem.go new file mode 100644 index 00000000..a6ef5e54 --- /dev/null +++ b/datastore/inmem/inmem.go @@ -0,0 +1,14 @@ +// Package inmem provides a in-memory thread-safe datastore for use with +// Cluster. +package inmem + +import ( + ds "github.com/ipfs/go-datastore" + sync "github.com/ipfs/go-datastore/sync" +) + +// New returns a new thread-safe in-memory go-datastore. +func New() ds.Datastore { + mapDs := ds.NewMapDatastore() + return sync.MutexWrap(mapDs) +} diff --git a/go.mod b/go.mod index fe2bd3ed..0e8c9ea9 100644 --- a/go.mod +++ b/go.mod @@ -1,24 +1,33 @@ module github.com/ipfs/ipfs-cluster require ( + github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9 // indirect github.com/blang/semver v3.5.1+incompatible github.com/boltdb/bolt v1.3.1 // indirect + github.com/btcsuite/btcd v0.0.0-20190410025418-9bfb2ca0346b // indirect + github.com/coreos/go-semver v0.3.0 // indirect + github.com/dgryski/go-farm v0.0.0-20190416075124-e1214b5e05dc // indirect github.com/dustin/go-humanize v1.0.0 github.com/gogo/protobuf v1.2.1 - github.com/golang/protobuf v1.3.0 + github.com/golang/protobuf v1.3.1 github.com/google/uuid v1.1.1 - github.com/gorilla/mux v1.7.0 + github.com/gopherjs/gopherjs v0.0.0-20190411002643-bd77b112433e // indirect + github.com/gorilla/mux v1.7.1 github.com/hashicorp/go-uuid v1.0.1 // indirect github.com/hashicorp/raft v1.0.0 github.com/hashicorp/raft-boltdb v0.0.0-20171010151810-6e5ba93211ea - github.com/hsanjuan/go-libp2p-gostream v0.0.30 - github.com/hsanjuan/go-libp2p-http v0.0.1 - github.com/huin/goupnp v1.0.0 // indirect + github.com/hsanjuan/go-libp2p-gostream v0.0.31 + github.com/hsanjuan/go-libp2p-http v0.0.2 + github.com/hsanjuan/ipfs-lite v0.0.2 + github.com/ipfs/go-bitswap v0.0.4 // indirect github.com/ipfs/go-block-format v0.0.2 github.com/ipfs/go-cid v0.0.1 - github.com/ipfs/go-datastore v0.0.1 + github.com/ipfs/go-datastore v0.0.4 + github.com/ipfs/go-ds-badger v0.0.3 + github.com/ipfs/go-ds-crdt v0.0.7 github.com/ipfs/go-fs-lock v0.0.1 github.com/ipfs/go-ipfs-api v0.0.1 + github.com/ipfs/go-ipfs-blockstore v0.0.1 github.com/ipfs/go-ipfs-chunker v0.0.1 github.com/ipfs/go-ipfs-ds-help v0.0.1 github.com/ipfs/go-ipfs-files v0.0.2 @@ -27,43 +36,56 @@ require ( github.com/ipfs/go-ipld-cbor v0.0.1 github.com/ipfs/go-ipld-format v0.0.1 github.com/ipfs/go-log v0.0.1 - github.com/ipfs/go-merkledag v0.0.1 - github.com/ipfs/go-mfs v0.0.1 - github.com/ipfs/go-path v0.0.1 - github.com/ipfs/go-unixfs v0.0.1 - github.com/jackpal/gateway v1.0.5 // indirect + github.com/ipfs/go-merkledag v0.0.3 + github.com/ipfs/go-mfs v0.0.5 + github.com/ipfs/go-path v0.0.3 + github.com/ipfs/go-unixfs v0.0.4 github.com/kelseyhightower/envconfig v1.3.0 github.com/lanzafame/go-libp2p-ocgorpc v0.0.1 - github.com/libp2p/go-libp2p v0.0.2 + github.com/libp2p/go-libp2p v0.0.13 github.com/libp2p/go-libp2p-consensus v0.0.1 github.com/libp2p/go-libp2p-crypto v0.0.1 + github.com/libp2p/go-libp2p-discovery v0.0.2 // indirect github.com/libp2p/go-libp2p-gorpc v0.0.2 - github.com/libp2p/go-libp2p-host v0.0.1 + github.com/libp2p/go-libp2p-host v0.0.2 + github.com/libp2p/go-libp2p-interface-connmgr v0.0.3 // indirect github.com/libp2p/go-libp2p-interface-pnet v0.0.1 - github.com/libp2p/go-libp2p-kad-dht v0.0.4 - github.com/libp2p/go-libp2p-peer v0.0.1 - github.com/libp2p/go-libp2p-peerstore v0.0.1 + github.com/libp2p/go-libp2p-kad-dht v0.0.8 + github.com/libp2p/go-libp2p-kbucket v0.1.1 // indirect + github.com/libp2p/go-libp2p-peer v0.1.0 + github.com/libp2p/go-libp2p-peerstore v0.0.2 github.com/libp2p/go-libp2p-pnet v0.0.1 github.com/libp2p/go-libp2p-protocol v0.0.1 github.com/libp2p/go-libp2p-pubsub v0.0.1 github.com/libp2p/go-libp2p-raft v0.0.2 - github.com/libp2p/go-ws-transport v0.0.1 - github.com/mattn/go-isatty v0.0.6 // indirect + github.com/libp2p/go-ws-transport v0.0.2 + github.com/minio/sha256-simd v0.0.0-20190328051042-05b4dd3047e5 // indirect github.com/multiformats/go-multiaddr v0.0.2 github.com/multiformats/go-multiaddr-dns v0.0.2 github.com/multiformats/go-multiaddr-net v0.0.1 github.com/multiformats/go-multicodec v0.1.6 - github.com/multiformats/go-multihash v0.0.1 + github.com/multiformats/go-multihash v0.0.3 + github.com/onsi/ginkgo v1.8.0 // indirect + github.com/onsi/gomega v1.5.0 // indirect + github.com/opentracing/opentracing-go v1.1.0 // indirect + github.com/pkg/errors v0.8.1 + github.com/polydawn/refmt v0.0.0-20190408063855-01bf1e26dd14 // indirect github.com/prometheus/client_golang v0.9.3-0.20190127221311-3c4408c8b829 github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90 // indirect github.com/prometheus/procfs v0.0.0-20190306233201-d0f344d83b0c // indirect github.com/rs/cors v1.6.0 + github.com/smartystreets/assertions v0.0.0-20190401211740-f487f9de1cd3 // indirect + github.com/smartystreets/goconvey v0.0.0-20190330032615-68dc04aab96a // indirect + github.com/spaolacci/murmur3 v1.1.0 // indirect github.com/ugorji/go v0.0.0-20171015030454-f26fc641ec9d github.com/urfave/cli v1.20.0 + github.com/warpfork/go-wish v0.0.0-20190328234359-8b3e70f8e830 // indirect github.com/zenground0/go-dot v0.0.0-20180912213407-94a425d4984e - go.opencensus.io v0.19.1 - golang.org/x/sys v0.0.0-20190308023053-584f3b12f43e // indirect - golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2 // indirect - google.golang.org/api v0.1.0 // indirect - google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19 // indirect + go.opencensus.io v0.20.2 + golang.org/x/crypto v0.0.0-20190417170229-92d88b081a49 // indirect + golang.org/x/net v0.0.0-20190415214537-1da14a5a36f2 // indirect + golang.org/x/sync v0.0.0-20190412183630-56d357773e84 // indirect + golang.org/x/sys v0.0.0-20190416152802-12500544f89f // indirect + golang.org/x/xerrors v0.0.0-20190410155217-1f06c39b4373 // indirect + google.golang.org/grpc v1.19.1 // indirect ) diff --git a/go.sum b/go.sum index 882650b8..a2f537f9 100644 --- a/go.sum +++ b/go.sum @@ -1,16 +1,23 @@ cloud.google.com/go v0.26.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= cloud.google.com/go v0.34.0/go.mod h1:aQUYkXzVsufM+DwF1aE+0xfcU+56JwCaLick0ClmMTw= -git.apache.org/thrift.git v0.0.0-20180902110319-2566ecd5d999/go.mod h1:fPE2ZNJGynbRyZ4dJvy6G277gSllfV2HJqblrnkyeyg= git.apache.org/thrift.git v0.12.0 h1:CMxsZlAmxKs+VAZMlDDL0wXciMblJcutQbEe3A9CYUM= git.apache.org/thrift.git v0.12.0/go.mod h1:fPE2ZNJGynbRyZ4dJvy6G277gSllfV2HJqblrnkyeyg= +github.com/AndreasBriese/bbloom v0.0.0-20180913140656-343706a395b7 h1:PqzgE6kAMi81xWQA2QIVxjWkFHptGgC547vchpUbtFo= github.com/AndreasBriese/bbloom v0.0.0-20180913140656-343706a395b7/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= +github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9 h1:HD8gA2tkByhMAwYaFAX9w2l7vxvBQ5NMoxDrkhqhtn4= +github.com/AndreasBriese/bbloom v0.0.0-20190306092124-e2d15f34fcf9/go.mod h1:bOvUY6CB00SOBii9/FifXqc0awNKxLFCL/+pkDPuyl8= github.com/BurntSushi/toml v0.3.1/go.mod h1:xHWCNGjB5oqiDr8zfno3MHue2Ht5sIBksp03qcyfWMU= +github.com/Kubuxu/go-os-helper v0.0.1 h1:EJiD2VUQyh5A9hWJLmc6iWg6yIcJ7jpBcwC8GMGXfDk= github.com/Kubuxu/go-os-helper v0.0.1/go.mod h1:N8B+I7vPCT80IcP58r50u4+gEEcsZETFUpAzWW2ep1Y= +github.com/Shopify/sarama v1.19.0/go.mod h1:FVkBWblsNy7DGZRfXLU0O9RCGt5g3g3yEuWXgklEdEo= +github.com/Shopify/toxiproxy v2.1.4+incompatible/go.mod h1:OXgGpZ6Cli1/URJOF1DMxUHB2q5Ap20/P/eIdh4G0pI= github.com/Stebalien/go-bitfield v0.0.0-20180330043415-076a62f9ce6e h1:2Z+EBRrOJsA3psnUPcEWMIH2EIga1xHflQcr/EZslx8= github.com/Stebalien/go-bitfield v0.0.0-20180330043415-076a62f9ce6e/go.mod h1:3oM7gXIttpYDAJXpVNnSCiUMYBLIZ6cb1t+Ip982MRo= github.com/aead/siphash v1.0.1/go.mod h1:Nywa3cDsYNNK3gaciGTWPwHt0wlpNV15vwmswBAUSII= github.com/alecthomas/template v0.0.0-20160405071501-a0175ee3bccc/go.mod h1:LOuyumcjzFXgccqObfd/Ljyb9UuFJ6TxHnclSeseNhc= github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0= +github.com/apache/thrift v0.12.0 h1:pODnxUFNcjP9UTLZGTdeh+j16A8lJbRvD3rOtrk/7bs= +github.com/apache/thrift v0.12.0/go.mod h1:cp2SuWMxlEZw2r+iP2GNCdIi4C1qmUzdZFSVb+bacwQ= github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da h1:8GUt8eRujhVEGZFFEjBj46YV4rDjvGrNxb0KMWYkL2I= github.com/armon/go-metrics v0.0.0-20180917152333-f0300d1749da/go.mod h1:Q73ZrmVTwzkszR9V5SSuryQ31EELlFMUz1kKyl939pY= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973 h1:xJ4a3vCFaGF/jqvzLMYoU8P317H5OQ+Via4RmuPwCS0= @@ -21,6 +28,8 @@ github.com/boltdb/bolt v1.3.1 h1:JQmyP4ZBrce+ZQu0dY660FMfatumYDLun9hBCUVIkF4= github.com/boltdb/bolt v1.3.1/go.mod h1:clJnj/oiGkjum5o1McbSZDSLxVThjynRyGBgiAx27Ps= github.com/btcsuite/btcd v0.0.0-20190213025234-306aecffea32 h1:qkOC5Gd33k54tobS36cXdAzJbeHaduLtnLQQwNoIi78= github.com/btcsuite/btcd v0.0.0-20190213025234-306aecffea32/go.mod h1:DrZx5ec/dmnfpw9KyYoQyYo7d0KEvTkk/5M/vbZjAr8= +github.com/btcsuite/btcd v0.0.0-20190410025418-9bfb2ca0346b h1:7J7sEce3LgtbMgs7PKcN61gF3b4txM6SjaRoJTSk640= +github.com/btcsuite/btcd v0.0.0-20190410025418-9bfb2ca0346b/go.mod h1:DrZx5ec/dmnfpw9KyYoQyYo7d0KEvTkk/5M/vbZjAr8= github.com/btcsuite/btclog v0.0.0-20170628155309-84c8d2346e9f/go.mod h1:TdznJufoqS23FtqVCzL0ZqgP5MqXbb4fg/WgDys70nA= github.com/btcsuite/btcutil v0.0.0-20190207003914-4c204d697803/go.mod h1:+5NJ2+qvTyV9exUAL/rxXi3DcLg2Ts+ymUAY5y4NvMg= github.com/btcsuite/go-socks v0.0.0-20170105172521-4720035b7bfd/go.mod h1:HHNXQzUsZCxOoE+CPiyCTO6x34Zs86zZUiwtpXoGdtg= @@ -33,17 +42,32 @@ github.com/cheekybits/is v0.0.0-20150225183255-68e9c0620927/go.mod h1:h/aW8ynjgk github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= github.com/coreos/go-semver v0.2.0 h1:3Jm3tLmsgAYcjC+4Up7hJrFBPr+n7rAqYeSw/SZazuY= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= +github.com/coreos/go-semver v0.2.1-0.20180108230905-e214231b295a h1:U0BbGfKnviqVBJQB4etvm+mKx53KfkumNLBt6YeF/0Q= +github.com/coreos/go-semver v0.2.1-0.20180108230905-e214231b295a/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= +github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmfM= +github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= +github.com/cskr/pubsub v1.0.2 h1:vlOzMhl6PFn60gRlTQQsIfVwaPB/B/8MziK8FhEPt/0= +github.com/cskr/pubsub v1.0.2/go.mod h1:/8MzYXk/NJAz782G8RPkFzXTZVu63VotefPnR9TIRis= github.com/davecgh/go-spew v0.0.0-20171005155431-ecdeabc65495/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davidlazar/go-crypto v0.0.0-20170701192655-dcfb0a7ac018 h1:6xT9KW8zLC5IlbaIF5Q7JNieBoACT7iW0YTxQHR0in0= github.com/davidlazar/go-crypto v0.0.0-20170701192655-dcfb0a7ac018/go.mod h1:rQYf4tfk5sSwFsnDg3qYaBxSjsD9S8+59vW0dKUgme4= +github.com/dgraph-io/badger v1.5.5-0.20190226225317-8115aed38f8f h1:6itBiEUtu+gOzXZWn46bM5/qm8LlV6/byR7Yflx/y6M= github.com/dgraph-io/badger v1.5.5-0.20190226225317-8115aed38f8f/go.mod h1:VZxzAIRPHRVNRKRo6AXrX9BJegn6il06VMTZVJYCIjQ= +github.com/dgraph-io/badger v2.0.0-rc.2+incompatible h1:7KPp6xv5+wymkVUbkAnZZXvmDrJlf09m/7u1HG5lAYA= +github.com/dgraph-io/badger v2.0.0-rc.2+incompatible/go.mod h1:VZxzAIRPHRVNRKRo6AXrX9BJegn6il06VMTZVJYCIjQ= +github.com/dgryski/go-farm v0.0.0-20190104051053-3adb47b1fb0f h1:dDxpBYafY/GYpcl+LS4Bn3ziLPuEdGRkRjYAbSlWxSA= github.com/dgryski/go-farm v0.0.0-20190104051053-3adb47b1fb0f/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= +github.com/dgryski/go-farm v0.0.0-20190416075124-e1214b5e05dc h1:VxEJYcOh1LMAdhIiHkofa6UC0PZvCmielUgJXgAAWFU= +github.com/dgryski/go-farm v0.0.0-20190416075124-e1214b5e05dc/go.mod h1:SqUrOPUnsFjfmXRMNPybcSiG0BgUW2AuFH8PAnS2iTw= github.com/dustin/go-humanize v1.0.0 h1:VSnTsYCnlFHaM2/igO1h6X3HA71jcobQuxemgkq4zYo= github.com/dustin/go-humanize v1.0.0/go.mod h1:HtrtbFcZ19U5GC7JDqmcUSB87Iq5E25KnS6fMYU6eOk= -github.com/fd/go-nat v1.0.0 h1:DPyQ97sxA9ThrWYRPcWUz/z9TnpTIGRYODIQc/dy64M= +github.com/eapache/go-resiliency v1.1.0/go.mod h1:kFI+JgMyC7bLPUVY133qvEBtVayf5mFgVsvEsIPBvNs= +github.com/eapache/go-xerial-snappy v0.0.0-20180814174437-776d5712da21/go.mod h1:+020luEh2TKB4/GOp8oxxtq0Daoen/Cii55CzbTV6DU= +github.com/eapache/queue v1.1.0/go.mod h1:6eCeP0CKFpHLu8blIFXhExK/dRa7WDZfr6jVFPTqq+I= +github.com/facebookgo/atomicfile v0.0.0-20151019160806-2de1f203e7d5/go.mod h1:JpoxHjuQauoxiFMl1ie8Xc/7TfLuMZ5eOCONd1sUBHg= github.com/fd/go-nat v1.0.0/go.mod h1:BTBu/CKvMmOMUPkKVef1pngt2WFH/lg7E6yQnulfp6E= github.com/fsnotify/fsnotify v1.4.7 h1:IXs+QLmnXW2CcXuY+8Mzv/fWEsPGWxqefPtCP5CnV9I= github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMoQvtojpjFo= @@ -54,6 +78,7 @@ github.com/go-kit/kit v0.8.0/go.mod h1:xBxKIO96dXMWWy0MnWVtmwkA9/13aqxPnvrjFYMA2 github.com/go-logfmt/logfmt v0.3.0/go.mod h1:Qt1PoO58o5twSAckw1HlFXLmHsOX5/0LbT9GBnD5lWE= github.com/go-stack/stack v1.8.0/go.mod h1:v0f6uXyyMGvRgIKkXu+yp6POWl0qKG85gN/melR3HDY= github.com/gogo/protobuf v1.1.1/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= +github.com/gogo/protobuf v1.2.0/go.mod h1:r8qH/GZQm5c6nD/R0oafs1akxWv10x8SbQlK7atdtwQ= github.com/gogo/protobuf v1.2.1 h1:/s5zKNz0uPFCZ5hddgPdo2TK2TVrUNMn0OOX8/aZMTE= github.com/gogo/protobuf v1.2.1/go.mod h1:hp+jE20tsWTFYpLwKvXlhS1hjn+gTNwPg2I6zVXpSg4= github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b h1:VKtxabqXZkF25pY9ekfRL6a582T4P37/31XEstQ5p58= @@ -64,6 +89,8 @@ github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfb github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.0 h1:kbxbvI4Un1LUWKxufD+BiE6AEExYYgkQLQmLFqA1LFk= github.com/golang/protobuf v1.3.0/go.mod h1:Qd/q+1AKNOZr9uGQzbzCmRO6sUih6GTPZv6a1/R87v0= +github.com/golang/protobuf v1.3.1 h1:YF8+flBXS5eO826T4nzqPrxfhQThhXl0YzfuUPu4SBg= +github.com/golang/protobuf v1.3.1/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/snappy v0.0.0-20180518054509-2e65f85255db/go.mod h1:/XxbfmMg8lxefKM7IXC3fBNl/7bRcc72aCRzEWrmP2Q= github.com/google/go-cmp v0.2.0 h1:+dTQ8DZQJz0Mb/HjFlkptS1FeQ4cWSnN941F8aEG4SQ= github.com/google/go-cmp v0.2.0/go.mod h1:oXzfMopK8JAjlY9xF4vHSVASa0yLyX7SntLO5aqRK0M= @@ -71,11 +98,14 @@ github.com/google/uuid v1.1.1 h1:Gkbcsh/GbpXz7lPftLA3P6TYMwjCLYm83jiFQZF/3gY= github.com/google/uuid v1.1.1/go.mod h1:TIyPZe4MgqvfeYDBFedMoGGpEw/LqOeaOT+nhxU+yHo= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1 h1:EGx4pi6eqNxGaHF6qqu48+N2wcFQ5qg5FXgOdqsJ5d8= github.com/gopherjs/gopherjs v0.0.0-20181017120253-0766667cb4d1/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= -github.com/gorilla/mux v1.7.0 h1:tOSd0UKHQd6urX6ApfOn4XdBMY6Sh1MfxV3kmaazO+U= -github.com/gorilla/mux v1.7.0/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= +github.com/gopherjs/gopherjs v0.0.0-20190411002643-bd77b112433e h1:XWcjeEtTFTOVA9Fs1w7n2XBftk5ib4oZrhzWk0B+3eA= +github.com/gopherjs/gopherjs v0.0.0-20190411002643-bd77b112433e/go.mod h1:wJfORRmW1u3UXTncJ5qlYoELFm8eSnnEO6hX4iZ3EWY= +github.com/gorilla/context v1.1.1/go.mod h1:kBGZzfjB9CEq2AlWe17Uuf7NDRt0dE0s8S51q0aT7Yg= +github.com/gorilla/mux v1.6.2/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= +github.com/gorilla/mux v1.7.1 h1:Dw4jY2nghMMRsh1ol8dv1axHkDwMQK2DHerMNJsIpJU= +github.com/gorilla/mux v1.7.1/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= github.com/gorilla/websocket v1.4.0 h1:WDFjx/TMzVgy9VdMMQi2K2Emtwi2QcUQsztZ/zLaH/Q= github.com/gorilla/websocket v1.4.0/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= -github.com/grpc-ecosystem/grpc-gateway v1.5.0/go.mod h1:RSKVYQBd5MCa4OVpNdGskqpgL2+G+NZTnrVHpWWfpdw= github.com/grpc-ecosystem/grpc-gateway v1.6.2/go.mod h1:RSKVYQBd5MCa4OVpNdGskqpgL2+G+NZTnrVHpWWfpdw= github.com/gxed/hashland/keccakpg v0.0.1 h1:wrk3uMNaMxbXiHibbPO4S0ymqJMm41WiudyFSs7UnsU= github.com/gxed/hashland/keccakpg v0.0.1/go.mod h1:kRzw3HkwxFU1mpmPP8v1WyQzwdGfmKFJ6tItnhQ67kU= @@ -83,10 +113,14 @@ github.com/gxed/hashland/murmur3 v0.0.1 h1:SheiaIt0sda5K+8FLz952/1iWS9zrnKsEJaOJ github.com/gxed/hashland/murmur3 v0.0.1/go.mod h1:KjXop02n4/ckmZSnY2+HKcLud/tcmvhST0bie/0lS48= github.com/gxed/pubsub v0.0.0-20180201040156-26ebdf44f824 h1:TF4mX7zXpeyz/xintezebSa7ZDxAGBnqDwcoobvaz2o= github.com/gxed/pubsub v0.0.0-20180201040156-26ebdf44f824/go.mod h1:OiEWyHgK+CWrmOlVquHaIK1vhpUJydC9m0Je6mhaiNE= +github.com/hashicorp/errwrap v1.0.0 h1:hLrqtEDnRye3+sgx6z4qVLNuviH3MR5aQ0ykNJa/UYA= +github.com/hashicorp/errwrap v1.0.0/go.mod h1:YH+1FKiLXxHSkmPseP+kNlulaMuP3n2brvKWEqk/Jc4= github.com/hashicorp/go-immutable-radix v1.0.0 h1:AKDB1HM5PWEA7i4nhcpwOrO2byshxBjXVn/J/3+z5/0= github.com/hashicorp/go-immutable-radix v1.0.0/go.mod h1:0y9vanUI8NX6FsYoO3zeMjhV/C5i9g4Q3DwcSNZ4P60= github.com/hashicorp/go-msgpack v0.5.3 h1:zKjpN5BK/P5lMYrLmBHdBULWbJ0XpYR+7NGzqkZzoD4= github.com/hashicorp/go-msgpack v0.5.3/go.mod h1:ahLV/dePpqEmjfWmKiqvPkv/twdG7iPBM1vqhUKIvfM= +github.com/hashicorp/go-multierror v1.0.0 h1:iVjPR7a6H0tWELX5NxNe7bYopibicUzc7uPribsnS6o= +github.com/hashicorp/go-multierror v1.0.0/go.mod h1:dHtQlpGsu+cZNNAkkCN/P3hoUDHhCYQXV3UM06sGGrk= github.com/hashicorp/go-uuid v1.0.0 h1:RS8zrF7PhGwyNPOtxSClXXj9HA8feRnJzgnI1RJCSnM= github.com/hashicorp/go-uuid v1.0.0/go.mod h1:6SBZvOh/SIDV7/2o3Jml5SYk/TvGqwFJ/bN7x4byOro= github.com/hashicorp/go-uuid v1.0.1 h1:fv1ep09latC32wFoVwnqcnKJGnMSdBanPczbHAYm1BE= @@ -101,11 +135,12 @@ github.com/hashicorp/raft-boltdb v0.0.0-20171010151810-6e5ba93211ea/go.mod h1:pN github.com/hpcloud/tail v1.0.0 h1:nfCOvKYfkgYP8hkirhJocXT2+zOD8yUNjXaWfTlyFKI= github.com/hpcloud/tail v1.0.0/go.mod h1:ab1qPbhIpdTxEkNHXyeSf5vhxWSCs/tWer42PpOxQnU= github.com/hsanjuan/go-libp2p-gostream v0.0.0-20190227195646-68f5153d3e40/go.mod h1:UYhQHuYJ0UqGSHxq9WbHlDz2qjDcz+upv5KlUY/OLUg= -github.com/hsanjuan/go-libp2p-gostream v0.0.30 h1:8vNjVW55G0i2fw17+Pl+76UG3IQd68jJOiCfTW5JOCQ= -github.com/hsanjuan/go-libp2p-gostream v0.0.30/go.mod h1:jfrdMwo5DXvVXruRjg33ELnc9loyQrLhonOrDZnka2M= -github.com/hsanjuan/go-libp2p-http v0.0.1 h1:BIT66H0+XwuHAqlTP6EUBZ27MwZwBy1Cxx0gEPf7q2Y= -github.com/hsanjuan/go-libp2p-http v0.0.1/go.mod h1:x7UoBuLChSOHvfs2TivH8j7vDy2EQAGU6QQYdRlkbm0= -github.com/huin/goupnp v0.0.0-20180415215157-1395d1447324 h1:PV190X5/DzQ/tbFFG5YpT5mH6q+cHlfgqI5JuRnH9oE= +github.com/hsanjuan/go-libp2p-gostream v0.0.31 h1:9dIgHQPR0VWxhOyTZrbgLzTx0xvZ5rTpmhG9huGEPjY= +github.com/hsanjuan/go-libp2p-gostream v0.0.31/go.mod h1:cWvV5/NQ5XWi0eQZnX/svsAk6NLc4U26pItvj0eDeRk= +github.com/hsanjuan/go-libp2p-http v0.0.2 h1:hviJbUD3h1Ez2FYTUdnRjrkAzn/9i2V/cLZpFPgnuP8= +github.com/hsanjuan/go-libp2p-http v0.0.2/go.mod h1:MynY94gfOZxrw/0lVF4o7vbV2Zr84IC8sLBXmj8F5IE= +github.com/hsanjuan/ipfs-lite v0.0.2 h1:9ytKeCa1yGmqNCOaNNHuI2BvuAfAzxDQ4aHWbLRGsLU= +github.com/hsanjuan/ipfs-lite v0.0.2/go.mod h1:qMgD0e0f26QPyzOHrH5uXqgsQO7U4j5RIsnIqw1Us7s= github.com/huin/goupnp v0.0.0-20180415215157-1395d1447324/go.mod h1:MZ2ZmwcBpvOoJ22IJsc7va19ZwoheaBk43rKg12SKag= github.com/huin/goupnp v1.0.0 h1:wg75sLpL6DZqwHQN6E1Cfk6mtfzS45z8OV+ic+DtHRo= github.com/huin/goupnp v1.0.0/go.mod h1:n9v9KO1tAxYH82qOn+UTIFQDmx5n1Zxd/ClZDMX7Bnc= @@ -114,21 +149,37 @@ github.com/ipfs/bbloom v0.0.1 h1:s7KkiBPfxCeDVo47KySjK0ACPc5GJRUxFpdyWEuDjhw= github.com/ipfs/bbloom v0.0.1/go.mod h1:oqo8CVWsJFMOZqTglBG4wydCE4IQA/G2/SEofB0rjUI= github.com/ipfs/go-bitswap v0.0.1 h1:Xx1ma7TWy9ISOx5zFq5YVQyrTHzUP4GkRPMsZokHxAg= github.com/ipfs/go-bitswap v0.0.1/go.mod h1:z+tP3h+HTJ810n1R5yMy2ccKFffJ2F6Vqm/5Bf7vs2c= +github.com/ipfs/go-bitswap v0.0.3 h1:uFcSI9dkjUn67S7IM60vr2wA27aAvn8o9xYjaQCug3o= +github.com/ipfs/go-bitswap v0.0.3/go.mod h1:jadAZYsP/tcRMl47ZhFxhaNuDQoXawT8iHMg+iFoQbg= +github.com/ipfs/go-bitswap v0.0.4 h1:mrS8jBd+rCgKw7Owx4RM5QBiMi9DBc1Ih9FaEBYM4/M= +github.com/ipfs/go-bitswap v0.0.4/go.mod h1:jadAZYsP/tcRMl47ZhFxhaNuDQoXawT8iHMg+iFoQbg= github.com/ipfs/go-block-format v0.0.1/go.mod h1:DK/YYcsSUIVAFNwo/KZCdIIbpN0ROH/baNLgayt4pFc= github.com/ipfs/go-block-format v0.0.2 h1:qPDvcP19izTjU8rgo6p7gTXZlkMkF5bz5G3fqIsSCPE= github.com/ipfs/go-block-format v0.0.2/go.mod h1:AWR46JfpcObNfg3ok2JHDUfdiHRgWhJgCQF+KIgOPJY= github.com/ipfs/go-blockservice v0.0.1 h1:l6g1hwYDV6vb4bAvTqia6Cvo+zLMOPKel/n0zUX48bc= github.com/ipfs/go-blockservice v0.0.1/go.mod h1:2Ao89U7jV1KIqqNk5EdhSTBG/Pgc1vMFr0bhkx376j4= +github.com/ipfs/go-blockservice v0.0.3 h1:40OvwrxeudTAlUGUAKNYnNPcwQeLtXedjzTWecnUinQ= +github.com/ipfs/go-blockservice v0.0.3/go.mod h1:/NNihwTi6V2Yr6g8wBI+BSwPuURpBRMtYNGrlxZ8KuI= github.com/ipfs/go-cid v0.0.1 h1:GBjWPktLnNyX0JiQCNFpUuUSoMw5KMyqrsejHYlILBE= github.com/ipfs/go-cid v0.0.1/go.mod h1:GHWU/WuQdMPmIosc4Yn1bcCT7dSeX4lBafM7iqUPQvM= github.com/ipfs/go-datastore v0.0.1 h1:AW/KZCScnBWlSb5JbnEnLKFWXL224LBEh/9KXXOrUms= github.com/ipfs/go-datastore v0.0.1/go.mod h1:d4KVXhMt913cLBEI/PXAy6ko+W7e9AhyAKBGh803qeE= +github.com/ipfs/go-datastore v0.0.2/go.mod h1:d4KVXhMt913cLBEI/PXAy6ko+W7e9AhyAKBGh803qeE= +github.com/ipfs/go-datastore v0.0.4 h1:FZgezmkbkMGXal1nXXFMyCKkWNglxK1c4jVqO1Emlso= +github.com/ipfs/go-datastore v0.0.4/go.mod h1:d4KVXhMt913cLBEI/PXAy6ko+W7e9AhyAKBGh803qeE= github.com/ipfs/go-detect-race v0.0.1 h1:qX/xay2W3E4Q1U7d9lNs1sU9nvguX0a7319XbyQ6cOk= github.com/ipfs/go-detect-race v0.0.1/go.mod h1:8BNT7shDZPo99Q74BpGMK+4D8Mn4j46UU0LZ723meps= +github.com/ipfs/go-ds-badger v0.0.2 h1:7ToQt7QByBhOTuZF2USMv+PGlMcBC7FW7FdgQ4FCsoo= github.com/ipfs/go-ds-badger v0.0.2/go.mod h1:Y3QpeSFWQf6MopLTiZD+VT6IC1yZqaGmjvRcKeSGij8= +github.com/ipfs/go-ds-badger v0.0.3 h1:sVYE2YlCzltznTZeAP1S+bp3qipz7VzogfZDtf6tGq0= +github.com/ipfs/go-ds-badger v0.0.3/go.mod h1:7AzMKCsGav0u46HpdLiAEAOqizR1H6AZsjpHpQSPYCQ= +github.com/ipfs/go-ds-crdt v0.0.7 h1:2iZGUNuYcvTt8OKOL4YCpZYIWC39AnfEyFz306moJ8Y= +github.com/ipfs/go-ds-crdt v0.0.7/go.mod h1:RCEASeHwFD3uBhHSDjTcsbTkI76G4Jw6ZxYQQO05b+I= github.com/ipfs/go-ds-leveldb v0.0.1/go.mod h1:feO8V3kubwsEF22n0YRQCffeb79OOYIykR4L04tMOYc= github.com/ipfs/go-fs-lock v0.0.1 h1:XHX8uW4jQBYWHj59XXcjg7BHlHxV9ZOYs6Y43yb7/l0= github.com/ipfs/go-fs-lock v0.0.1/go.mod h1:DNBekbboPKcxs1aukPSaOtFA3QfSdi5C855v0i9XJ8Y= +github.com/ipfs/go-ipfs-addr v0.0.1 h1:DpDFybnho9v3/a1dzJ5KnWdThWD1HrFLpQ+tWIyBaFI= +github.com/ipfs/go-ipfs-addr v0.0.1/go.mod h1:uKTDljHT3Q3SUWzDLp3aYUi8MrY32fgNgogsIa0npjg= github.com/ipfs/go-ipfs-api v0.0.1 h1:4wx4mSgeq5FwMN8LDF7WLwPDKEd+YKjgySrpOJQ2r8o= github.com/ipfs/go-ipfs-api v0.0.1/go.mod h1:0FhXgCzrLu7qNmdxZvgYqD9jFzJxzz1NAVt3OQ0WOIc= github.com/ipfs/go-ipfs-blockstore v0.0.1 h1:O9n3PbmTYZoNhkgkEyrXTznbmktIXif62xLX+8dPHzc= @@ -137,6 +188,8 @@ github.com/ipfs/go-ipfs-blocksutil v0.0.1 h1:Eh/H4pc1hsvhzsQoMEP3Bke/aW5P5rVM1IW github.com/ipfs/go-ipfs-blocksutil v0.0.1/go.mod h1:Yq4M86uIOmxmGPUHv/uI7uKqZNtLb449gwKqXjIsnRk= github.com/ipfs/go-ipfs-chunker v0.0.1 h1:cHUUxKFQ99pozdahi+uSC/3Y6HeRpi9oTeUHbE27SEw= github.com/ipfs/go-ipfs-chunker v0.0.1/go.mod h1:tWewYK0we3+rMbOh7pPFGDyypCtvGcBFymgY4rSDLAw= +github.com/ipfs/go-ipfs-config v0.0.1 h1:6ED08emzI1imdsAjixFi2pEyZxTVD5ECKtCOxLBx+Uc= +github.com/ipfs/go-ipfs-config v0.0.1/go.mod h1:KDbHjNyg4e6LLQSQpkgQMBz6Jf4LXiWAcmnkcwmH0DU= github.com/ipfs/go-ipfs-delay v0.0.0-20181109222059-70721b86a9a8/go.mod h1:8SP1YXK1M1kXuc4KJZINY3TQQ03J2rwBG9QfXmbRPrw= github.com/ipfs/go-ipfs-delay v0.0.1 h1:r/UXYyRcddO6thwOnhiznIAiSvxMECGgtv35Xs1IeRQ= github.com/ipfs/go-ipfs-delay v0.0.1/go.mod h1:8SP1YXK1M1kXuc4KJZINY3TQQ03J2rwBG9QfXmbRPrw= @@ -168,19 +221,20 @@ github.com/ipfs/go-log v0.0.1 h1:9XTUN/rW64BCG1YhPK9Hoy3q8nr4gOmHHBpgFdfw6Lc= github.com/ipfs/go-log v0.0.1/go.mod h1:kL1d2/hzSpI0thNYjiKfjanbVNU+IIGA/WnNESY9leM= github.com/ipfs/go-merkledag v0.0.1 h1:HqvQsqqLvNOgItOy80Sd4T4rHvq6cXtAtrbEoWAON+I= github.com/ipfs/go-merkledag v0.0.1/go.mod h1:CRdtHMROECqaehAGeJ0Wd9TtlmWv/ta5cUnvbTnniEI= +github.com/ipfs/go-merkledag v0.0.3 h1:A5DlOMzqTRDVmdgkf3dzCKCFmVWH4Zqwb0cbYXUs+Ro= +github.com/ipfs/go-merkledag v0.0.3/go.mod h1:Oc5kIXLHokkE1hWGMBHw+oxehkAaTOqtEb7Zbh6BhLA= github.com/ipfs/go-metrics-interface v0.0.1 h1:j+cpbjYvu4R8zbleSs36gvB7jR+wsL2fGD6n0jO4kdg= github.com/ipfs/go-metrics-interface v0.0.1/go.mod h1:6s6euYU4zowdslK0GKHmqaIZ3j/b/tL7HTWtJ4VPgWY= -github.com/ipfs/go-mfs v0.0.1 h1:EcQeaxW2DO3LmRXEtbWCGxhZ7YmMR+X7nzjYChBzG8s= -github.com/ipfs/go-mfs v0.0.1/go.mod h1:rUT0dKNWkKa1T+MobpBL2zANn7p8Y6unXANC0PV2FLk= -github.com/ipfs/go-path v0.0.1 h1:6UskTq8xYVs3zVnHjXDvoCqw22dKWK1BwD1cy1cuHyc= -github.com/ipfs/go-path v0.0.1/go.mod h1:ztzG4iSBN2/CJa93rtHAv/I+mpK+BGALeUoJzhclhw0= +github.com/ipfs/go-mfs v0.0.5 h1:Zdma6fypyTV4iZIX5RxINQ3h+xdgvN+0JRJ5OPs65ok= +github.com/ipfs/go-mfs v0.0.5/go.mod h1:10Hdow7wUbSlIamnOduxeP6MEp58TozZmdnAhugOKz8= +github.com/ipfs/go-path v0.0.3 h1:G/VFcCMXtp36JUPPyytYQ1I3UsBUBf47M//uSdTLnFg= +github.com/ipfs/go-path v0.0.3/go.mod h1:zIRQUez3LuQIU25zFjC2hpBTHimWx7VK5bjZgRLbbdo= github.com/ipfs/go-todocounter v0.0.1 h1:kITWA5ZcQZfrUnDNkRn04Xzh0YFaDFXsoO2A81Eb6Lw= github.com/ipfs/go-todocounter v0.0.1/go.mod h1:l5aErvQc8qKE2r7NDMjmq5UNAvuZy0rC8BHOplkWvZ4= -github.com/ipfs/go-unixfs v0.0.1 h1:CTTGqLxU5+PRkkeA+w1peStqRWFD1Kya+yZgIT4Xy1w= -github.com/ipfs/go-unixfs v0.0.1/go.mod h1:ZlB83nMtxNMx4DAAE5/GixeKN1qHC+xspBksI7Q5NeI= +github.com/ipfs/go-unixfs v0.0.4 h1:IApzQ+SnY0tfjqM7aU2b80CFYLZNHvhLmEZDIWr4e/E= +github.com/ipfs/go-unixfs v0.0.4/go.mod h1:eIo/p9ADu/MFOuyxzwU+Th8D6xoxU//r590vUpWyfz8= github.com/ipfs/go-verifcid v0.0.1 h1:m2HI7zIuR5TFyQ1b79Da5N9dnnCP1vcu2QqawmWlK2E= github.com/ipfs/go-verifcid v0.0.1/go.mod h1:5Hrva5KBeIog4A+UpqlaIU+DEstipcJYQQZc0g37pY0= -github.com/jackpal/gateway v1.0.4 h1:LS5EHkLuQ6jzaHwULi0vL+JO0mU/n4yUtK8oUjHHOlM= github.com/jackpal/gateway v1.0.4/go.mod h1:lTpwd4ACLXmpyiCTRtfiNyVnUmqT9RivzCDQetPfnjA= github.com/jackpal/gateway v1.0.5 h1:qzXWUJfuMdlLMtt0a3Dgt+xkWQiA5itDEITVJtuSwMc= github.com/jackpal/gateway v1.0.5/go.mod h1:lTpwd4ACLXmpyiCTRtfiNyVnUmqT9RivzCDQetPfnjA= @@ -198,6 +252,8 @@ github.com/jessevdk/go-flags v0.0.0-20141203071132-1679536dcc89/go.mod h1:4FA24M github.com/jrick/logrotate v1.0.0/go.mod h1:LNinyqDIJnpAur+b8yyulnQw/wDuN1+BYKlTRt3OuAQ= github.com/jtolds/gls v4.2.1+incompatible h1:fSuqC+Gmlu6l/ZYAoZzx2pyucC8Xza35fpRVWLVmUEE= github.com/jtolds/gls v4.2.1+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= +github.com/jtolds/gls v4.20.0+incompatible h1:xdiiI2gbIgH/gLH7ADydsJ1uDOEzR8yvV7C0MuV77Wo= +github.com/jtolds/gls v4.20.0+incompatible/go.mod h1:QJZ7F/aHp+rZTRtaJ1ow/lLfFfVYBRgL+9YlvaHOwJU= github.com/julienschmidt/httprouter v1.2.0/go.mod h1:SYymIcj16QtmaHHD7aYtjjsJG7VTCxuUUipMqKk8s4w= github.com/kelseyhightower/envconfig v1.3.0 h1:IvRS4f2VcIQy6j4ORGIf9145T/AsUB+oY8LyvN8BXNM= github.com/kelseyhightower/envconfig v1.3.0/go.mod h1:cccZRl6mQpaq41TPp5QxidR+Sa3axMbJDNb//FQX6Gg= @@ -206,6 +262,8 @@ github.com/kisielk/gotool v1.0.0 h1:AV2c/EiW3KqPNT9ZKl07ehoAGi4C5/01Cfbblndcapg= github.com/kisielk/gotool v1.0.0/go.mod h1:XhKaO+MFFWcvkIS/tQcRk01m1F5IRFswLeQ+oQHNcck= github.com/kkdai/bstream v0.0.0-20161212061736-f391b8402d23/go.mod h1:J+Gs4SYgM6CZQHDETBtE9HaSEkGmuNXF86RwHhHUvq4= github.com/konsorten/go-windows-terminal-sequences v1.0.1/go.mod h1:T0+1ngSBFLxvqU3pZ+m/2kptfBszLMUkC4ZK/EgS/cQ= +github.com/koron/go-ssdp v0.0.0-20180514024734-4a0ed625a78b h1:wxtKgYHEncAU00muMD06dzLiahtGM1eouRNOzVV7tdQ= +github.com/koron/go-ssdp v0.0.0-20180514024734-4a0ed625a78b/go.mod h1:5Ky9EC2xfoUKUor0Hjgi2BJhCSXJfMOFlmyYrVKGQMk= github.com/kr/logfmt v0.0.0-20140226030751-b84e30acd515/go.mod h1:+0opPa2QZZtGFBFZlji/RkVcI2GknAs/DXo4wKdlNEc= github.com/kr/pretty v0.1.0 h1:L/CwN0zerZDmRFUapSPitk6f+Q3+0za1rQkzVuMiMFI= github.com/kr/pretty v0.1.0/go.mod h1:dAy3ld7l9f0ibDNOQOHHMYYIIbhfbHSm3C4ZsoJORNo= @@ -227,46 +285,68 @@ github.com/libp2p/go-flow-metrics v0.0.1/go.mod h1:Iv1GH0sG8DtYN3SVJ2eG221wMiNpZ github.com/libp2p/go-libp2p v0.0.1/go.mod h1:bmRs8I0vwn6iRaVssZnJx/epY6WPSKiLoK1vyle4EX0= github.com/libp2p/go-libp2p v0.0.2 h1:+jvgi0Zy3y4TKXJKApchCk3pCBPZf1T54z3+vKie3gw= github.com/libp2p/go-libp2p v0.0.2/go.mod h1:Qu8bWqFXiocPloabFGUcVG4kk94fLvfC8mWTDdFC9wE= +github.com/libp2p/go-libp2p v0.0.3/go.mod h1:Qu8bWqFXiocPloabFGUcVG4kk94fLvfC8mWTDdFC9wE= +github.com/libp2p/go-libp2p v0.0.12/go.mod h1:l8kmLMc4DToz2TtA0aWEaE4/jVMALRKw8Kv2WTR0LHc= +github.com/libp2p/go-libp2p v0.0.13 h1:F8EJzBX3eipwW1yrvlmpkpc3jh5YpGwZiG7zVAt8mjM= +github.com/libp2p/go-libp2p v0.0.13/go.mod h1:99u6nHehR0M9pxJM682dq44GDdowQBcQPxXNTPGoEgU= github.com/libp2p/go-libp2p-autonat v0.0.1/go.mod h1:fs71q5Xk+pdnKU014o2iq1RhMs9/PMaG5zXRFNnIIT4= -github.com/libp2p/go-libp2p-autonat v0.0.2 h1:ilo9QPzNPf1hMkqaPG55yzvhILf5ZtijstJhcii+l3s= github.com/libp2p/go-libp2p-autonat v0.0.2/go.mod h1:fs71q5Xk+pdnKU014o2iq1RhMs9/PMaG5zXRFNnIIT4= +github.com/libp2p/go-libp2p-autonat v0.0.4 h1:cZzdB9KW1ZkHnSjLCB6aFNw47XS4r+SecCVMuVB1xgo= +github.com/libp2p/go-libp2p-autonat v0.0.4/go.mod h1:fs71q5Xk+pdnKU014o2iq1RhMs9/PMaG5zXRFNnIIT4= github.com/libp2p/go-libp2p-blankhost v0.0.1 h1:/mZuuiwntNR8RywnCFlGHLKrKLYne+qciBpQXWqp5fk= github.com/libp2p/go-libp2p-blankhost v0.0.1/go.mod h1:Ibpbw/7cPPYwFb7PACIWdvxxv0t0XCCI10t7czjAjTc= -github.com/libp2p/go-libp2p-circuit v0.0.1 h1:DYbjyQ5ZY3QVAVYZWG4uzBQ6Wmcd1C82Bk8Q/pJlM1I= github.com/libp2p/go-libp2p-circuit v0.0.1/go.mod h1:Dqm0s/BiV63j8EEAs8hr1H5HudqvCAeXxDyic59lCwE= +github.com/libp2p/go-libp2p-circuit v0.0.4 h1:yOgEadnSVFj3e9KLBuLG+edqCImeav0VXxXvcimpOUQ= +github.com/libp2p/go-libp2p-circuit v0.0.4/go.mod h1:p1cHJnB9xnX5/1vZLkXgKwmNEOQQuF/Hp+SkATXnXYk= github.com/libp2p/go-libp2p-consensus v0.0.1 h1:jcVbHRZLwTXU9iT/mPi+Lx4/OrIzq3bU1TbZNhYFCV8= github.com/libp2p/go-libp2p-consensus v0.0.1/go.mod h1:+9Wrfhc5QOqWB0gXI0m6ARlkHfdJpcFXmRU0WoHz4Mo= github.com/libp2p/go-libp2p-crypto v0.0.1 h1:JNQd8CmoGTohO/akqrH16ewsqZpci2CbgYH/LmYl8gw= github.com/libp2p/go-libp2p-crypto v0.0.1/go.mod h1:yJkNyDmO341d5wwXxDUGO0LykUVT72ImHNUqh5D/dBE= github.com/libp2p/go-libp2p-discovery v0.0.1 h1:VkjCKmJQMwpDUwtA8Qc1z3TQAHJgQ5nGQ6cdN0wQXOw= github.com/libp2p/go-libp2p-discovery v0.0.1/go.mod h1:ZkkF9xIFRLA1xCc7bstYFkd80gBGK8Fc1JqGoU2i+zI= +github.com/libp2p/go-libp2p-discovery v0.0.2 h1:Rf+20nsFcCnHo4Kxvf8ofAft75+fW+cXy9FonNVyU/g= +github.com/libp2p/go-libp2p-discovery v0.0.2/go.mod h1:ZkkF9xIFRLA1xCc7bstYFkd80gBGK8Fc1JqGoU2i+zI= github.com/libp2p/go-libp2p-gorpc v0.0.2 h1:b6AFNr9M7RfQew3wz+18afteUpfB7t/7Kvmizoh0rrg= github.com/libp2p/go-libp2p-gorpc v0.0.2/go.mod h1:+6nLxg9kjPOKwJHq5YQwJ5NGBcOxJ4YF9OHIX89TwwE= github.com/libp2p/go-libp2p-host v0.0.1 h1:dnqusU+DheGcdxrE718kG4XgHNuL2n9eEv8Rg5zy8hQ= github.com/libp2p/go-libp2p-host v0.0.1/go.mod h1:qWd+H1yuU0m5CwzAkvbSjqKairayEHdR5MMl7Cwa7Go= +github.com/libp2p/go-libp2p-host v0.0.2 h1:UnyDP7gmaUIYG3EUPnGc54K+KLFmI6V0Ozm+BQU9VQ8= +github.com/libp2p/go-libp2p-host v0.0.2/go.mod h1:JACKb5geZ28rUiChzlzSFRC8XYYcLwsZq38h+a4D4Hs= github.com/libp2p/go-libp2p-interface-connmgr v0.0.1 h1:Q9EkNSLAOF+u90L88qmE9z/fTdjLh8OsJwGw74mkwk4= github.com/libp2p/go-libp2p-interface-connmgr v0.0.1/go.mod h1:GarlRLH0LdeWcLnYM/SaBykKFl9U5JFnbBGruAk/D5k= +github.com/libp2p/go-libp2p-interface-connmgr v0.0.3 h1:uN9FGH9OUJAtQ2G19F60Huu7s3TIYRBaJLUaW0PlCUo= +github.com/libp2p/go-libp2p-interface-connmgr v0.0.3/go.mod h1:GarlRLH0LdeWcLnYM/SaBykKFl9U5JFnbBGruAk/D5k= github.com/libp2p/go-libp2p-interface-pnet v0.0.1 h1:7GnzRrBTJHEsofi1ahFdPN9Si6skwXQE9UqR2S+Pkh8= github.com/libp2p/go-libp2p-interface-pnet v0.0.1/go.mod h1:el9jHpQAXK5dnTpKA4yfCNBZXvrzdOU75zz+C6ryp3k= -github.com/libp2p/go-libp2p-kad-dht v0.0.4 h1:Z+6l5pCD8xXQmqmKKO+OTa4+7Or1uyPTmu49rXNf4oo= -github.com/libp2p/go-libp2p-kad-dht v0.0.4/go.mod h1:oaBflOQcuC8H+SVV0YN26H6AS+wcUEJyjUGV66vXuSY= +github.com/libp2p/go-libp2p-kad-dht v0.0.6/go.mod h1:oaBflOQcuC8H+SVV0YN26H6AS+wcUEJyjUGV66vXuSY= +github.com/libp2p/go-libp2p-kad-dht v0.0.8 h1:oUnAqkCAWYvAoF5TmnDIf4k1fIcipMAFec6sQlTXGKE= +github.com/libp2p/go-libp2p-kad-dht v0.0.8/go.mod h1:fGQfSQWWOxQFB97kETE09lYRLPRKaZZdguIq98fE5PI= github.com/libp2p/go-libp2p-kbucket v0.0.1 h1:7H5hM851hkEpLOFjrVNSrrxo6J4bWrUQxxv+z1JW9xk= github.com/libp2p/go-libp2p-kbucket v0.0.1/go.mod h1:Y0iQDHRTk/ZgM8PC4jExoF+E4j+yXWwRkdldkMa5Xm4= +github.com/libp2p/go-libp2p-kbucket v0.1.1 h1:ZrvW3qCM+lAuv7nrNts/zfEiClq+GZe8OIzX4Vb3Dwo= +github.com/libp2p/go-libp2p-kbucket v0.1.1/go.mod h1:Y0iQDHRTk/ZgM8PC4jExoF+E4j+yXWwRkdldkMa5Xm4= github.com/libp2p/go-libp2p-loggables v0.0.1 h1:HVww9oAnINIxbt69LJNkxD8lnbfgteXR97Xm4p3l9ps= github.com/libp2p/go-libp2p-loggables v0.0.1/go.mod h1:lDipDlBNYbpyqyPX/KcoO+eq0sJYEVR2JgOexcivchg= github.com/libp2p/go-libp2p-metrics v0.0.1 h1:yumdPC/P2VzINdmcKZd0pciSUCpou+s0lwYCjBbzQZU= github.com/libp2p/go-libp2p-metrics v0.0.1/go.mod h1:jQJ95SXXA/K1VZi13h52WZMa9ja78zjyy5rspMsC/08= github.com/libp2p/go-libp2p-nat v0.0.1/go.mod h1:4L6ajyUIlJvx1Cbh5pc6Ma6vMDpKXf3GgLO5u7W0oQ4= -github.com/libp2p/go-libp2p-nat v0.0.2 h1:sKI5hiCsGFhuEKdXMsF9mywQu2qhfoIGX6a+VG6zelE= github.com/libp2p/go-libp2p-nat v0.0.2/go.mod h1:QrjXQSD5Dj4IJOdEcjHRkWTSomyxRo6HnUkf/TfQpLQ= +github.com/libp2p/go-libp2p-nat v0.0.4 h1:+KXK324yaY701On8a0aGjTnw8467kW3ExKcqW2wwmyw= +github.com/libp2p/go-libp2p-nat v0.0.4/go.mod h1:N9Js/zVtAXqaeT99cXgTV9e75KpnWCvVOiGzlcHmBbY= github.com/libp2p/go-libp2p-net v0.0.1 h1:xJ4Vh4yKF/XKb8fd1Ev0ebAGzVjMxXzrxG2kjtU+F5Q= github.com/libp2p/go-libp2p-net v0.0.1/go.mod h1:Yt3zgmlsHOgUWSXmt5V/Jpz9upuJBE8EgNU9DrCcR8c= +github.com/libp2p/go-libp2p-net v0.0.2 h1:qP06u4TYXfl7uW/hzqPhlVVTSA2nw1B/bHBJaUnbh6M= +github.com/libp2p/go-libp2p-net v0.0.2/go.mod h1:Yt3zgmlsHOgUWSXmt5V/Jpz9upuJBE8EgNU9DrCcR8c= github.com/libp2p/go-libp2p-netutil v0.0.1 h1:LgD6+skofkOx8z6odD9+MZHKjupv3ng1u6KRhaADTnA= github.com/libp2p/go-libp2p-netutil v0.0.1/go.mod h1:GdusFvujWZI9Vt0X5BKqwWWmZFxecf9Gt03cKxm2f/Q= github.com/libp2p/go-libp2p-peer v0.0.1 h1:0qwAOljzYewINrU+Kndoc+1jAL7vzY/oY2Go4DCGfyY= github.com/libp2p/go-libp2p-peer v0.0.1/go.mod h1:nXQvOBbwVqoP+T5Y5nCjeH4sP9IX/J0AMzcDUVruVoo= +github.com/libp2p/go-libp2p-peer v0.1.0 h1:9D1St1vqXRkeAhNdDtpt8AivS1bhzA6yH+YWrVXWcWI= +github.com/libp2p/go-libp2p-peer v0.1.0/go.mod h1:nXQvOBbwVqoP+T5Y5nCjeH4sP9IX/J0AMzcDUVruVoo= github.com/libp2p/go-libp2p-peerstore v0.0.1 h1:twKovq8YK5trLrd3nB7PD2Zu9JcyAIdm7Bz9yBWjhq8= github.com/libp2p/go-libp2p-peerstore v0.0.1/go.mod h1:RabLyPVJLuNQ+GFyoEkfi8H4Ti6k/HtZJ7YKgtSq+20= +github.com/libp2p/go-libp2p-peerstore v0.0.2 h1:Lirt3A1Oq11jszJ4SPNBo8chNv61UWXE538KUEGxTVk= +github.com/libp2p/go-libp2p-peerstore v0.0.2/go.mod h1:RabLyPVJLuNQ+GFyoEkfi8H4Ti6k/HtZJ7YKgtSq+20= github.com/libp2p/go-libp2p-pnet v0.0.1 h1:2e5d15M8XplUKsU4Fqrll5eDfqGg/7mHUufLkhbfKHM= github.com/libp2p/go-libp2p-pnet v0.0.1/go.mod h1:bWN8HqdpgCdKnXSCsJhbWjiU3UZFa/tIe4no5jCmHVw= github.com/libp2p/go-libp2p-protocol v0.0.1 h1:+zkEmZ2yFDi5adpVE3t9dqh/N9TbpFWywowzeEzBbLM= @@ -283,6 +363,8 @@ github.com/libp2p/go-libp2p-secio v0.0.1 h1:CqE/RdsizOwItdgLe632iyft/w0tshDLmZGA github.com/libp2p/go-libp2p-secio v0.0.1/go.mod h1:IdG6iQybdcYmbTzxp4J5dwtUEDTOvZrT0opIDVNPrJs= github.com/libp2p/go-libp2p-swarm v0.0.1 h1:Vne+hjaDwXqzgNwQ2vb2YKbnbOTyXjtS47stT66Apc4= github.com/libp2p/go-libp2p-swarm v0.0.1/go.mod h1:mh+KZxkbd3lQnveQ3j2q60BM1Cw2mX36XXQqwfPOShs= +github.com/libp2p/go-libp2p-swarm v0.0.2 h1:cpHHXTeU2IgUu8LPemF7vaLPGtVC6VxMoll2EwqlC+E= +github.com/libp2p/go-libp2p-swarm v0.0.2/go.mod h1:n0cAAcKyndIrJWctQwjqXlAdIPBZzfdpBjx1SSvz30g= github.com/libp2p/go-libp2p-transport v0.0.1/go.mod h1:UzbUs9X+PHOSw7S3ZmeOxfnwaQY5vGDzZmKPod3N3tk= github.com/libp2p/go-libp2p-transport v0.0.4 h1:/CPHQMN75/IQwkhBxxIo6p6PtL3rwFZtlzBROT3e8mw= github.com/libp2p/go-libp2p-transport v0.0.4/go.mod h1:StoY3sx6IqsP6XKoabsPnHCwqKXWUMWU7Rfcsubee/A= @@ -294,26 +376,33 @@ github.com/libp2p/go-mplex v0.0.1 h1:dn2XGSrUxLtz3/8u85bGrwhUEKPX8MOF3lpmcWBZCWc github.com/libp2p/go-mplex v0.0.1/go.mod h1:pK5yMLmOoBR1pNCqDlA2GQrdAVTMkqFalaTWe7l4Yd0= github.com/libp2p/go-msgio v0.0.1 h1:znj97n5FtXGCLDwe9x8jpHmY770SW4WStBGcCDh6GJw= github.com/libp2p/go-msgio v0.0.1/go.mod h1:63lBBgOTDKQL6EWazRMCwXsEeEeK9O2Cd+0+6OOuipQ= +github.com/libp2p/go-nat v0.0.3 h1:l6fKV+p0Xa354EqQOQP+d8CivdLM4kl5GxC1hSc/UeI= +github.com/libp2p/go-nat v0.0.3/go.mod h1:88nUEt0k0JD45Bk93NIwDqjlhiOwOoV36GchpcVc1yI= github.com/libp2p/go-reuseport v0.0.1 h1:7PhkfH73VXfPJYKQ6JwS5I/eVcoyYi9IMNGc6FWpFLw= github.com/libp2p/go-reuseport v0.0.1/go.mod h1:jn6RmB1ufnQwl0Q1f+YxAj8isJgDCQzaaxIFYDhcYEA= github.com/libp2p/go-reuseport-transport v0.0.1 h1:UIRneNxLDmEGNjGHpIiWzSWkZ5bhxMCP9x3Vh7BSc7E= github.com/libp2p/go-reuseport-transport v0.0.1/go.mod h1:YkbSDrvjUVDL6b8XqriyA20obEtsW9BLkuOUyQAOCbs= +github.com/libp2p/go-reuseport-transport v0.0.2 h1:WglMwyXyBu61CMkjCCtnmqNqnjib0GIEjMiHTwR/KN4= +github.com/libp2p/go-reuseport-transport v0.0.2/go.mod h1:YkbSDrvjUVDL6b8XqriyA20obEtsW9BLkuOUyQAOCbs= github.com/libp2p/go-stream-muxer v0.0.1 h1:Ce6e2Pyu+b5MC1k3eeFtAax0pW4gc6MosYSLV05UeLw= github.com/libp2p/go-stream-muxer v0.0.1/go.mod h1:bAo8x7YkSpadMTbtTaxGVHWUQsR/l5MEaHbKaliuT14= github.com/libp2p/go-tcp-transport v0.0.1 h1:WyvJVw2lYAnr6CU+GZZ4oCt06fvORlmvBlFX2+ZpZDM= github.com/libp2p/go-tcp-transport v0.0.1/go.mod h1:mnjg0o0O5TmXUaUIanYPUqkW4+u6mK0en8rlpA6BBTs= +github.com/libp2p/go-tcp-transport v0.0.2 h1:EzOSRaHpyrGpJ5qe+9SaxJM1mrWlkSLFfNTufUkq0lg= +github.com/libp2p/go-tcp-transport v0.0.2/go.mod h1:VjZFHasNJ0QiJQNNwiFDy25qyGWTXQWs8GM5DR4/L1k= github.com/libp2p/go-testutil v0.0.1 h1:Xg+O0G2HIMfHqBOBDcMS1iSZJ3GEcId4qOxCQvsGZHk= github.com/libp2p/go-testutil v0.0.1/go.mod h1:iAcJc/DKJQanJ5ws2V+u5ywdL2n12X1WbbEG+Jjy69I= -github.com/libp2p/go-ws-transport v0.0.1 h1:9ytMqq86Xvp8rcnC/1ZNuH612eXLDglvcu4ZHseJl8s= github.com/libp2p/go-ws-transport v0.0.1/go.mod h1:p3bKjDWHEgtuKKj+2OdPYs5dAPIjtpQGHF2tJfGz7Ww= +github.com/libp2p/go-ws-transport v0.0.2 h1:PtK1AoM16nm96FwPBQoq+4T4t9LdDwOhkB+mdXuGSlg= +github.com/libp2p/go-ws-transport v0.0.2/go.mod h1:p3bKjDWHEgtuKKj+2OdPYs5dAPIjtpQGHF2tJfGz7Ww= github.com/mailru/easyjson v0.0.0-20190221075403-6243d8e04c3f h1:B6PQkurxGG1rqEX96oE14gbj8bqvYC5dtks9r5uGmlE= github.com/mailru/easyjson v0.0.0-20190221075403-6243d8e04c3f/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mattn/go-colorable v0.1.1 h1:G1f5SKeVxmagw/IyvzvtZE4Gybcc4Tr1tf7I8z0XgOg= github.com/mattn/go-colorable v0.1.1/go.mod h1:FuOcm+DKB9mbwrcAfNl7/TZVBZ6rcnceauSikq3lYCQ= github.com/mattn/go-isatty v0.0.5 h1:tHXDdz1cpzGaovsTB+TVB8q90WEokoVmfMqoVcrLUgw= github.com/mattn/go-isatty v0.0.5/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= -github.com/mattn/go-isatty v0.0.6 h1:SrwhHcpV4nWrMGdNcC2kXpMfcBVYGDuTArqyhocJgvA= -github.com/mattn/go-isatty v0.0.6/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= +github.com/mattn/go-isatty v0.0.7 h1:UvyT9uN+3r7yLEYSlJsbQGdsaB/a0DlgWP3pql6iwOc= +github.com/mattn/go-isatty v0.0.7/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hdxcsrc5s= github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/miekg/dns v1.1.4/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3Nrg= @@ -321,10 +410,14 @@ github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1 h1:lYpkrQH5ajf0 github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1/go.mod h1:pD8RvIylQ358TN4wwqatJ8rNavkEINozVn9DtGI3dfQ= github.com/minio/sha256-simd v0.0.0-20190131020904-2d45a736cd16 h1:5W7KhL8HVF3XCFOweFD3BNESdnO8ewyYTFT2R+/b8FQ= github.com/minio/sha256-simd v0.0.0-20190131020904-2d45a736cd16/go.mod h1:2FMWW+8GMoPweT6+pI63m9YE3Lmw4J71hV56Chs1E/U= +github.com/minio/sha256-simd v0.0.0-20190328051042-05b4dd3047e5 h1:l16XLUUJ34wIz+RIvLhSwGvLvKyy+W598b135bJN6mg= +github.com/minio/sha256-simd v0.0.0-20190328051042-05b4dd3047e5/go.mod h1:2FMWW+8GMoPweT6+pI63m9YE3Lmw4J71hV56Chs1E/U= github.com/mitchellh/go-homedir v1.1.0 h1:lukF9ziXFxDFPkA1vsr5zpc1XuPDn/wFntq5mG+4E0Y= github.com/mitchellh/go-homedir v1.1.0/go.mod h1:SfyaCUpYCn1Vlf4IUYiD9fPX4A5wJrkLzIz1N1q0pr0= github.com/mr-tron/base58 v1.1.0 h1:Y51FGVJ91WBqCEabAi5OPUz38eAx8DakuAm5svLcsfQ= github.com/mr-tron/base58 v1.1.0/go.mod h1:xcD2VGqlgYjBdcBLw+TuYLr8afG+Hj8g2eTVqeSzSU8= +github.com/mr-tron/base58 v1.1.1 h1:OJIdWOWYe2l5PQNgimGtuwHY8nDskvJ5vvs//YnzRLs= +github.com/mr-tron/base58 v1.1.1/go.mod h1:xcD2VGqlgYjBdcBLw+TuYLr8afG+Hj8g2eTVqeSzSU8= github.com/multiformats/go-base32 v0.0.3 h1:tw5+NhuwaOjJCC5Pp82QuXbrmLzWg7uxlMFp8Nq/kkI= github.com/multiformats/go-base32 v0.0.3/go.mod h1:pLiuGC8y0QR3Ue4Zug5UzK9LjgbkL8NSQj0zQ5Nz/AA= github.com/multiformats/go-multiaddr v0.0.1 h1:/QUV3VBMDI6pi6xfiw7lr6xhDWWvQKn9udPn68kLSdY= @@ -342,22 +435,31 @@ github.com/multiformats/go-multicodec v0.1.6 h1:4u6lcjbE4VVVoigU4QJSSVYsGVP4j2jt github.com/multiformats/go-multicodec v0.1.6/go.mod h1:lliaRHbcG8q33yf4Ot9BGD7JqR/Za9HE7HTyVyKwrUQ= github.com/multiformats/go-multihash v0.0.1 h1:HHwN1K12I+XllBCrqKnhX949Orn4oawPkegHMu2vDqQ= github.com/multiformats/go-multihash v0.0.1/go.mod h1:w/5tugSrLEbWqlcgJabL3oHFKTwfvkofsjW2Qa1ct4U= +github.com/multiformats/go-multihash v0.0.3 h1:j9FrQUfaGhGQUKHaHjsrCiChsqh+bFR187z8VGq77M0= +github.com/multiformats/go-multihash v0.0.3/go.mod h1:w/5tugSrLEbWqlcgJabL3oHFKTwfvkofsjW2Qa1ct4U= github.com/multiformats/go-multistream v0.0.1 h1:JV4VfSdY9n7ECTtY59/TlSyFCzRILvYx4T4Ws8ZgihU= github.com/multiformats/go-multistream v0.0.1/go.mod h1:fJTiDfXJVmItycydCnNx4+wSzZ5NwG2FEVAI30fiovg= github.com/mwitkow/go-conntrack v0.0.0-20161129095857-cc309e4a2223/go.mod h1:qRWi+5nqEBWmkhHvq77mSJWrCKwh8bxhgT7d/eI7P4U= github.com/onsi/ginkgo v1.6.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/ginkgo v1.7.0 h1:WSHQ+IS43OoUrWtD1/bbclrwK8TTH5hzp+umCiuxHgs= github.com/onsi/ginkgo v1.7.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= +github.com/onsi/ginkgo v1.8.0 h1:VkHVNpR4iVnU8XQR6DBm8BqYjN7CRzw+xKUbVVbbW9w= +github.com/onsi/ginkgo v1.8.0/go.mod h1:lLunBs/Ym6LB5Z9jYTR76FiuTmxDTDusOGeTQH+WWjE= github.com/onsi/gomega v1.4.3 h1:RE1xgDvH7imwFD45h+u2SgIfERHlS2yNG4DObb5BSKU= github.com/onsi/gomega v1.4.3/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= +github.com/onsi/gomega v1.5.0 h1:izbySO9zDPmjJ8rDjLvkA2zJHIo+HkYXHnf7eN7SSyo= +github.com/onsi/gomega v1.5.0/go.mod h1:ex+gbHU/CVuBBDIJjb2X0qEXbFg53c61hWP/1CpauHY= github.com/opentracing/opentracing-go v1.0.2 h1:3jA2P6O1F9UOrWVpwrIo17pu01KWvNWg4X946/Y5Zwg= github.com/opentracing/opentracing-go v1.0.2/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= -github.com/openzipkin/zipkin-go v0.1.1/go.mod h1:NtoC/o8u3JlF1lSlyPNswIbeQH9bJTmOf0Erfk+hxe8= +github.com/opentracing/opentracing-go v1.1.0 h1:pWlfV3Bxv7k65HYwkikxat0+s3pV4bsqf19k25Ur8rU= +github.com/opentracing/opentracing-go v1.1.0/go.mod h1:UkNAQd3GIcIGf0SeVgPpRdFStlNbqXla1AfSYxPUl2o= github.com/openzipkin/zipkin-go v0.1.3/go.mod h1:NtoC/o8u3JlF1lSlyPNswIbeQH9bJTmOf0Erfk+hxe8= +github.com/openzipkin/zipkin-go v0.1.6/go.mod h1:QgAqvLzwWbR/WpD4A3cGpPtJrZXNIiJc5AZX7/PBEpw= github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c h1:Lgl0gzECD8GnQ5QCWA8o6BtfL6mDH5rQgM4/fX3avOs= github.com/pascaldekloe/goe v0.0.0-20180627143212-57f6aae5913c/go.mod h1:lzWF7FIEvWOWxwDKqyGYQf6ZUaNfKdP144TG7ZOy1lc= github.com/philhofer/fwd v1.0.0 h1:UbZqGr5Y38ApvM/V/jEljVxwocdweyH+vmYvRPBnbqQ= github.com/philhofer/fwd v1.0.0/go.mod h1:gk3iGcWd9+svBvR0sR+KPcfE+RNWozjowpeBVG3ZVNU= +github.com/pierrec/lz4 v2.0.5+incompatible/go.mod h1:pdkljMzZIN41W+lC3N2tnIh5sFi+IEE17M5jbnwPHcY= github.com/pkg/errors v0.8.0/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= github.com/pkg/errors v0.8.1 h1:iURUrRGxPUNPdy5/HRSm+Yj6okJ6UtLINN0Q9M4+h3I= github.com/pkg/errors v0.8.1/go.mod h1:bwawxfHBFNV+L2hUp1rHADufV3IMtnDRdf1r5NINEl0= @@ -365,9 +467,10 @@ github.com/pmezard/go-difflib v1.0.0 h1:4DBwDE0NGyQoBHbLQYPwSUPoCMWR5BEzIk/f1lZb github.com/pmezard/go-difflib v1.0.0/go.mod h1:iKH77koFhYxTK1pcRnkKkqfTogsbg7gZNVY4sRDYZ/4= github.com/polydawn/refmt v0.0.0-20190221155625-df39d6c2d992 h1:bzMe+2coZJYHnhGgVlcQKuRy4FSny4ds8dLQjw5P1XE= github.com/polydawn/refmt v0.0.0-20190221155625-df39d6c2d992/go.mod h1:uIp+gprXxxrWSjjklXD+mN4wed/tMfjMMmN/9+JsA9o= +github.com/polydawn/refmt v0.0.0-20190408063855-01bf1e26dd14 h1:2m16U/rLwVaRdz7ANkHtHTodP3zTP3N451MADg64x5k= +github.com/polydawn/refmt v0.0.0-20190408063855-01bf1e26dd14/go.mod h1:uIp+gprXxxrWSjjklXD+mN4wed/tMfjMMmN/9+JsA9o= github.com/pquerna/ffjson v0.0.0-20181028064349-e517b90714f7 h1:gGBSHPOU7g8YjTbhwn+lvFm2VDEhhA+PwDIlstkgSxE= github.com/pquerna/ffjson v0.0.0-20181028064349-e517b90714f7/go.mod h1:YARuvh7BUWHNhzDq2OM5tzR2RiCcN2D7sapiKyCel/M= -github.com/prometheus/client_golang v0.8.0/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.1/go.mod h1:7SWBe2y4D6OKWSNQJUaRYU/AaXPKyh/dDVn+NZz0KFw= github.com/prometheus/client_golang v0.9.3-0.20190127221311-3c4408c8b829 h1:D+CiwcpGTW6pL6bv6KI3KbyEyCKyS+1JWS2h8PNDnGA= github.com/prometheus/client_golang v0.9.3-0.20190127221311-3c4408c8b829/go.mod h1:p2iRAGwDERtqlqzRXnrOVns+ignqQo//hLXqYxZYVNs= @@ -376,24 +479,29 @@ github.com/prometheus/client_model v0.0.0-20190115171406-56726106282f h1:BVwpUVJ github.com/prometheus/client_model v0.0.0-20190115171406-56726106282f/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90 h1:S/YWwWx/RA8rT8tKFRuGUZhuA90OyIBpPCXkcbwU8DE= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/common v0.0.0-20180801064454-c7de2306084e/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.2.0 h1:kUZDBDTdBVBYBj5Tmh2NZLlF60mfjA27rM34b+cVwNU= github.com/prometheus/common v0.2.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= -github.com/prometheus/procfs v0.0.0-20180725123919-05ee40e3a273/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190117184657-bf6a532e95b1 h1:/K3IL0Z1quvmJ7X0A1AwNEK7CRkVK3YwfOU/QAL4WGg= github.com/prometheus/procfs v0.0.0-20190117184657-bf6a532e95b1/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20190306233201-d0f344d83b0c h1:xAaFC6WmfeVufj49LZocAyA0S4FSB8eB/himN+phUR4= github.com/prometheus/procfs v0.0.0-20190306233201-d0f344d83b0c/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= +github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= github.com/rs/cors v1.6.0 h1:G9tHG9lebljV9mfp9SNPDL36nCDxmo3zTlAf1YgvzmI= github.com/rs/cors v1.6.0/go.mod h1:gFx+x8UowdsKA9AchylcLynDq+nNFfI8FkUZdN/jGCU= github.com/sirupsen/logrus v1.2.0/go.mod h1:LxeOpSwHxABJmUn/MG1IvRgCAasNZTLOkJPxbbu5VWo= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d h1:zE9ykElWQ6/NYmHa3jpm/yHnI4xSofP+UP6SpjHcSeM= github.com/smartystreets/assertions v0.0.0-20180927180507-b2de0cb4f26d/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= +github.com/smartystreets/assertions v0.0.0-20190401211740-f487f9de1cd3 h1:hBSHahWMEgzwRyS6dRpxY0XyjZsHyQ61s084wo5PJe0= +github.com/smartystreets/assertions v0.0.0-20190401211740-f487f9de1cd3/go.mod h1:OnSkiWE9lh6wB0YB77sQom3nweQdgAjqCqsofrRNTgc= github.com/smartystreets/goconvey v0.0.0-20190222223459-a17d461953aa h1:E+gaaifzi2xF65PbDmuKI3PhLWY6G5opMLniFq8vmXA= github.com/smartystreets/goconvey v0.0.0-20190222223459-a17d461953aa/go.mod h1:2RVY1rIf+2J2o/IM9+vPq9RzmHDSseB7FoXiSNIUsoU= +github.com/smartystreets/goconvey v0.0.0-20190330032615-68dc04aab96a h1:pa8hGb/2YqsZKovtsgrwcDH1RZhVbTKCjLp47XpqCDs= +github.com/smartystreets/goconvey v0.0.0-20190330032615-68dc04aab96a/go.mod h1:syvi0/a8iFYH4r/RixwvyeAJjdLS9QV7WQ/tjFTllLA= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72 h1:qLC7fQah7D6K1B0ujays3HV9gkFtllcxhzImRR7ArPQ= github.com/spaolacci/murmur3 v0.0.0-20180118202830-f09979ecbc72/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= +github.com/spaolacci/murmur3 v1.1.0 h1:7c1g84S4BPRrfL5Xrdp6fOJ206sU9y293DDHaoy0bLI= +github.com/spaolacci/murmur3 v1.1.0/go.mod h1:JwIasOWyU6f++ZhiEuf87xNszmSA2myDM2Kzu9HwQUA= github.com/stretchr/objx v0.1.0/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/objx v0.1.1/go.mod h1:HFkY916IF+rwdDfMAkV7OtwuqBVzrE8GR6GFx+wExME= github.com/stretchr/testify v1.2.2/go.mod h1:a8OnRcib4nhh0OaRAV+Yts87kKdq0PP7pXfy6kDkUVs= @@ -408,6 +516,8 @@ github.com/urfave/cli v1.20.0 h1:fDqGv3UG/4jbVl/QkFwEdddtEDjh/5Ov6X+0B/3bPaw= github.com/urfave/cli v1.20.0/go.mod h1:70zkFmudgCuE/ngEzBv17Jvp/497gISqfk5gWijbERA= github.com/warpfork/go-wish v0.0.0-20180510122957-5ad1f5abf436 h1:qOpVTI+BrstcjTZLm2Yz/3sOnqkzj3FQoh0g+E5s3Gc= github.com/warpfork/go-wish v0.0.0-20180510122957-5ad1f5abf436/go.mod h1:x6AKhvSSexNrVSrViXSHUEbICjmGXhtgABaHIySUSGw= +github.com/warpfork/go-wish v0.0.0-20190328234359-8b3e70f8e830 h1:8kxMKmKzXXL4Ru1nyhvdms/JjWt+3YLpvRb/bAjO/y0= +github.com/warpfork/go-wish v0.0.0-20190328234359-8b3e70f8e830/go.mod h1:x6AKhvSSexNrVSrViXSHUEbICjmGXhtgABaHIySUSGw= github.com/whyrusleeping/base32 v0.0.0-20170828182744-c30ac30633cc h1:BCPnHtcboadS0DvysUuJXZ4lWVv5Bh5i7+tbIyi+ck4= github.com/whyrusleeping/base32 v0.0.0-20170828182744-c30ac30633cc/go.mod h1:r45hJU7yEoA81k6MWNhpMj/kms0n14dkzkxYHoB96UM= github.com/whyrusleeping/cbor v0.0.0-20171005072247-63513f603b11 h1:5HZfQkwe0mIfyDmc1Em5GqlNRzcdtlv4HTNmdpt7XH0= @@ -440,9 +550,11 @@ github.com/whyrusleeping/yamux v1.1.5 h1:4CK3aUUJQu0qpKZv5gEWJjNOQtdbdDhVVS6PJ+H github.com/whyrusleeping/yamux v1.1.5/go.mod h1:E8LnQQ8HKx5KD29HZFUwM1PxCOdPRzGwur1mcYhXcD8= github.com/zenground0/go-dot v0.0.0-20180912213407-94a425d4984e h1:GN1PUQ/MNDdtiZZhCAnZ4PwTcslUM8qWVz8q2bLkDeM= github.com/zenground0/go-dot v0.0.0-20180912213407-94a425d4984e/go.mod h1:T00FaxHq4SlnicuZFole4yRAgcjWtqbMcUXBfXAYvaI= -go.opencensus.io v0.18.0/go.mod h1:vKdFvxhtzZ9onBp9VKHK8z/sRpBMnKAsufL7wlDrCOA= go.opencensus.io v0.19.1 h1:gPYKQ/GAQYR2ksU+qXNmq3CrOZWT1kkryvW6O0v1acY= go.opencensus.io v0.19.1/go.mod h1:gug0GbSHa8Pafr0d2urOSgoXHZ6x/RUlaiT0d9pqb4A= +go.opencensus.io v0.20.1/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= +go.opencensus.io v0.20.2 h1:NAfh7zF0/3/HqtMvJNZ/RFrSlCE6ZTlHmKfhL/Dm1Jk= +go.opencensus.io v0.20.2/go.mod h1:6WKK9ahsWS3RSO+PY9ZHZUfv2irvY6gN279GOPZjmmk= go4.org v0.0.0-20190218023631-ce4c26f7be8e h1:m9LfARr2VIOW0vsV19kEKp/sWQvZnGobA8JHui/XJoY= go4.org v0.0.0-20190218023631-ce4c26f7be8e/go.mod h1:MkTOUMDaeVYJUOUsaDXIhWPZYa1yOyC1qaOBpL57BhE= golang.org/x/crypto v0.0.0-20170930174604-9419663f5a44/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= @@ -451,11 +563,16 @@ golang.org/x/crypto v0.0.0-20190211182817-74369b46fc67/go.mod h1:6SG95UA2DQfeDnf golang.org/x/crypto v0.0.0-20190225124518-7f87c0fbb88b/go.mod h1:6SG95UA2DQfeDnfUPMdvaQW0Q7yPrPDi9nlGo2tz2b4= golang.org/x/crypto v0.0.0-20190228161510-8dd112bcdc25 h1:jsG6UpNLt9iAsb0S2AGW28DveNzzgmbXR+ENoPjUeIU= golang.org/x/crypto v0.0.0-20190228161510-8dd112bcdc25/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2 h1:VklqNMn3ovrHsnt90PveolxSbWFaJdECFbxSq0Mqo2M= +golang.org/x/crypto v0.0.0-20190308221718-c2843e01d9a2/go.mod h1:djNgcEr1/C05ACkg1iLfiJU5Ep61QUkGW8qpdssI0+w= +golang.org/x/crypto v0.0.0-20190417170229-92d88b081a49 h1:mE9V9RMa141kxdQR3pfZM3mkg0MPyw+FOPpnciBXkbE= +golang.org/x/crypto v0.0.0-20190417170229-92d88b081a49/go.mod h1:WFFai1msRO1wXaEeE5yQxYXgSfI8pQAWXbQop6sCtWE= golang.org/x/exp v0.0.0-20190121172915-509febef88a4/go.mod h1:CJ0aWSM057203Lf6IL+f9T1iT9GByDxfZKAQTCR3kQA= golang.org/x/lint v0.0.0-20180702182130-06c8688daad7/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20181026193005-c67002cb31c3/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20181217174547-8f45f776aaf1/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/lint v0.0.0-20190227174305-5b3e6a55c961/go.mod h1:wehouNa3lNwaWXcvxsM5YxQ5yQlVC4a0KAMCusXpPoU= +golang.org/x/lint v0.0.0-20190301231843-5614ed5bae6f/go.mod h1:UVdnD1Gm6xHRNCYTkRU2/jEulfH38KcIWyp/GAMgvoE= golang.org/x/net v0.0.0-20180524181706-dfa909b99c79/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180724234803-3673e40ba225/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20180826012351-8a410e7b638d/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= @@ -463,24 +580,33 @@ golang.org/x/net v0.0.0-20180906233101-161cd47e91fd/go.mod h1:mL1N/T3taQHkDXs73r golang.org/x/net v0.0.0-20181011144130-49bb7cea24b1/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181106065722-10aee1819953/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20181114220301-adae6a3d119a/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190108225652-1e06a53dbb7e/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190125091013-d26f9f9a57f3/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190213061140-3a22650c66bd/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190227160552-c95aed5357e7 h1:C2F/nMkR/9sfUTpvR3QrjBuTdvMUC/cFajkphs1YLQo= golang.org/x/net v0.0.0-20190227160552-c95aed5357e7/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= golang.org/x/net v0.0.0-20190301231341-16b79f2e4e95 h1:fY7Dsw114eJN4boqzVSbpVHO6rTdhq6/GnXeu+PKnzU= golang.org/x/net v0.0.0-20190301231341-16b79f2e4e95/go.mod h1:mL1N/T3taQHkDXs73rZJwtUhF3w3ftmwwsq0BUmARs4= +golang.org/x/net v0.0.0-20190311183353-d8887717615a h1:oWX7TPOiFAMXLq8o0ikBYfCJVlRHBcsciT5bXOrH628= +golang.org/x/net v0.0.0-20190311183353-d8887717615a/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= +golang.org/x/net v0.0.0-20190415214537-1da14a5a36f2 h1:iC0Y6EDq+rhnAePxGvJs2kzUAYcwESqdcGRPzEUfzTU= +golang.org/x/net v0.0.0-20190415214537-1da14a5a36f2/go.mod h1:t9HGtf8HONx5eT2rtn7q6eTqICYqUVnKs3thJo3Qplg= golang.org/x/oauth2 v0.0.0-20180821212333-d2e6202438be/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= golang.org/x/oauth2 v0.0.0-20181203162652-d668ce993890/go.mod h1:N/0e6XlmueqKjAGxoOufVs8QHGRruUQn6yWY3a++T0U= +golang.org/x/oauth2 v0.0.0-20190226205417-e64efc72b421/go.mod h1:gOpvHmFTYa4IltrdGE7lF6nIHvwfUNPOp7c8zoXwtLw= golang.org/x/sync v0.0.0-20180314180146-1d60e4601c6f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181108010431-42b317875d0f/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4 h1:YUO/7uOKsKeq9UokNS62b8FYywz3ker1l1vDZRCRefw= golang.org/x/sync v0.0.0-20181221193216-37e7f081c4d4/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6 h1:bjcUS9ztw9kFmmIxJInhon/0Is3p+EHBKNgquIzo1OI= golang.org/x/sync v0.0.0-20190227155943-e225da77a7e6/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= +golang.org/x/sync v0.0.0-20190412183630-56d357773e84 h1:IqXQ59gzdXv58Jmm2xn0tSOR9i6HqroaOFRQ3wR/dJQ= +golang.org/x/sync v0.0.0-20190412183630-56d357773e84/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180905080454-ebe1bf3edb33/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180909124046-d0be0721c37e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181116152217-5ac8a444bdc5/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= +golang.org/x/sys v0.0.0-20181122145206-62eef0e2fa9b/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20181218192612-074acd46bca6/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190215142949-d0b11bdaac8a/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190219092855-153ac476189d/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -488,8 +614,9 @@ golang.org/x/sys v0.0.0-20190222072716-a9d3bda3a223/go.mod h1:STP8DvDyc/dI5b8T5h golang.org/x/sys v0.0.0-20190228124157-a34e9553db1e/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20190302025703-b6889370fb10 h1:xQJI9OEiErEQ++DoXOHqEpzsGMrAv2Q2jyCpi7DmfpQ= golang.org/x/sys v0.0.0-20190302025703-b6889370fb10/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= -golang.org/x/sys v0.0.0-20190308023053-584f3b12f43e h1:K7CV15oJ823+HLXQ+M7MSMrUg8LjfqY7O3naO+8Pp/I= -golang.org/x/sys v0.0.0-20190308023053-584f3b12f43e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190403152447-81d4e9dc473e/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20190416152802-12500544f89f h1:1ZH9RnjNgLzh6YrsRp/c6ddZ8Lq0fq9xztNOoWJ2sz4= +golang.org/x/sys v0.0.0-20190416152802-12500544f89f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/text v0.3.0 h1:g61tztE5qeGQ89tm6NTjjM9VPIm088od1l6aSorWRWg= golang.org/x/text v0.3.0/go.mod h1:NqM8EUOU14njkJ3fqMW+pc6Ldnwhi/IjpwHt7yyuwOQ= golang.org/x/text v0.3.1-0.20180807135948-17ff2d5776d2 h1:z99zHgr7hKfrUcX/KsoJk5FJfjTceCKIp96+biqP4To= @@ -500,29 +627,32 @@ golang.org/x/tools v0.0.0-20181219222714-6e267b5cc78e/go.mod h1:n7NCudcB/nEzxVGm golang.org/x/tools v0.0.0-20190114222345-bf090417da8b/go.mod h1:n7NCudcB/nEzxVGmLbDWY5pfWTLqBcC2KZ6jyYvM4mQ= golang.org/x/tools v0.0.0-20190226205152-f727befe758c h1:vamGzbGri8IKo20MQncCuljcQ5uAO6kaCeawQPVblAI= golang.org/x/tools v0.0.0-20190226205152-f727befe758c/go.mod h1:9Yl7xja0Znq3iFh3HoIrodX9oNMXvdceNzlUR8zjMvY= +golang.org/x/tools v0.0.0-20190312170243-e65039ee4138/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= +golang.org/x/tools v0.0.0-20190328211700-ab21143f2384/go.mod h1:LCzVGOaR6xXOjkQ3onu1FJEFr0SW1gC7cKk1uF8kGRs= golang.org/x/xerrors v0.0.0-20190212162355-a5947ffaace3 h1:P6iTFmrTQqWrqLZPX1VMzCUbCRCAUXSUsSpkEOvWzJ0= golang.org/x/xerrors v0.0.0-20190212162355-a5947ffaace3/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= -google.golang.org/api v0.0.0-20180910000450-7ca32eb868bf/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= +golang.org/x/xerrors v0.0.0-20190410155217-1f06c39b4373 h1:PPwnA7z1Pjf7XYaBP9GL1VAMZmcIWyFz7QCMSIIa3Bg= +golang.org/x/xerrors v0.0.0-20190410155217-1f06c39b4373/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= google.golang.org/api v0.0.0-20181220000619-583d854617af h1:iQMS7JKv/0w/iiWf1M49Cg3dmOkBoBZT5KheqPDpaac= google.golang.org/api v0.0.0-20181220000619-583d854617af/go.mod h1:4mhQ8q/RsB7i+udVvVy5NUi08OU8ZlA0gRVgrF7VFY0= -google.golang.org/api v0.1.0 h1:K6z2u68e86TPdSdefXdzvXgR1zEMa+459vBSfWYAZkI= -google.golang.org/api v0.1.0/go.mod h1:UGEZY7KEX120AnNLIHFMKIo4obdJhkp2tPbaPlQx13Y= +google.golang.org/api v0.3.1 h1:oJra/lMfmtm13/rgY/8i3MzjFWYXvQIAKjQ3HqofMk8= +google.golang.org/api v0.3.1/go.mod h1:6wY9I6uQWHQ8EM57III9mq/AjF+i8G65rmVagqKMtkk= google.golang.org/appengine v1.1.0/go.mod h1:EbEs0AVv82hx2wNQdGPgUI5lhzA/G0D9YwlJXL52JkM= google.golang.org/appengine v1.3.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/appengine v1.4.0/go.mod h1:xpcJRLb0r/rnEns0DIKYYv+WjYCduHsrkT7/EB5XEv4= google.golang.org/genproto v0.0.0-20180817151627-c66870c02cf8/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= google.golang.org/genproto v0.0.0-20180831171423-11092d34479b/go.mod h1:JiN7NxoALGmiZfu7CAH4rXhgtRTLTxftemlI0sWmxmc= -google.golang.org/genproto v0.0.0-20181202183823-bd91e49a0898/go.mod h1:7Ep/1NZk928CDR8SjdVbjWNpdIf6nzjE3BTgJDr2Atg= google.golang.org/genproto v0.0.0-20181219182458-5a97ab628bfb h1:dQshZyyJ5W/Xk8myF4GKBak1pZW6EywJuQ8+44EQhGA= google.golang.org/genproto v0.0.0-20181219182458-5a97ab628bfb/go.mod h1:7Ep/1NZk928CDR8SjdVbjWNpdIf6nzjE3BTgJDr2Atg= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19 h1:Lj2SnHtxkRGJDqnGaSjo+CCdIieEnwVazbOXILwQemk= google.golang.org/genproto v0.0.0-20190307195333-5fe7a883aa19/go.mod h1:VzzqZJRnGkLBvHegQrXjBqPurQTc5/KpmUdxsrq26oE= -google.golang.org/grpc v1.14.0/go.mod h1:yo6s7OP7yaDglbqo1J04qKzAhqBH6lvTonzMVmEdcZw= google.golang.org/grpc v1.16.0/go.mod h1:0JHn/cJsOMiMfNA9+DeHDlAU7KAAB5GDlYFpa9MZMio= google.golang.org/grpc v1.17.0 h1:TRJYBgMclJvGYn2rIMjj+h9KtMt5r1Ij7ODVRIZkwhk= google.golang.org/grpc v1.17.0/go.mod h1:6QZJwpn2B+Zp71q/5VxRsJ6NXXVCE5NRUHRo+f3cWCs= google.golang.org/grpc v1.19.0 h1:cfg4PD8YEdSFnm7qLV4++93WcmhH2nIUhMjhdCvl3j8= google.golang.org/grpc v1.19.0/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= +google.golang.org/grpc v1.19.1 h1:TrBcJ1yqAl1G++wO39nD/qtgpsW9/1+QGrluyMGEYgM= +google.golang.org/grpc v1.19.1/go.mod h1:mqu4LbDTu4XGKhr4mRzUsmM4RtVoemTSY81AxZiDr8c= gopkg.in/alecthomas/kingpin.v2 v2.2.6/go.mod h1:FMv+mEhP44yOT+4EoQTLFTRgOQ1FBLkstjWtayDeSgw= gopkg.in/check.v1 v0.0.0-20161208181325-20d25e280405/go.mod h1:Co6ibVJAznAaIkqp8huTwlJQCZ016jof/cbN4VW5Yz0= gopkg.in/check.v1 v1.0.0-20180628173108-788fd7840127 h1:qIbj1fsPNlZgppZ+VLlY7N33q108Sa+fhmuc+sWQYwY= diff --git a/ipfscluster.go b/ipfscluster.go index 67b11b4d..36f9cdff 100644 --- a/ipfscluster.go +++ b/ipfscluster.go @@ -44,7 +44,7 @@ type Consensus interface { LogUnpin(ctx context.Context, c *api.Pin) error AddPeer(ctx context.Context, p peer.ID) error RmPeer(ctx context.Context, p peer.ID) error - State(context.Context) (state.State, error) + State(context.Context) (state.ReadOnly, error) // Provide a node which is responsible to perform // specific tasks which must only run in 1 cluster peer Leader(context.Context) (peer.ID, error) diff --git a/ipfscluster_test.go b/ipfscluster_test.go index 07774820..76fcb0f8 100644 --- a/ipfscluster_test.go +++ b/ipfscluster_test.go @@ -7,31 +7,38 @@ import ( "fmt" "math/rand" "os" + "path/filepath" "sort" "strings" "sync" "testing" "time" + crypto "github.com/libp2p/go-libp2p-crypto" + peerstore "github.com/libp2p/go-libp2p-peerstore" + pubsub "github.com/libp2p/go-libp2p-pubsub" + + dht "github.com/libp2p/go-libp2p-kad-dht" + "github.com/ipfs/ipfs-cluster/allocator/descendalloc" "github.com/ipfs/ipfs-cluster/api" "github.com/ipfs/ipfs-cluster/api/rest" + "github.com/ipfs/ipfs-cluster/consensus/crdt" "github.com/ipfs/ipfs-cluster/consensus/raft" + "github.com/ipfs/ipfs-cluster/datastore/badger" + "github.com/ipfs/ipfs-cluster/datastore/inmem" "github.com/ipfs/ipfs-cluster/informer/disk" "github.com/ipfs/ipfs-cluster/ipfsconn/ipfshttp" "github.com/ipfs/ipfs-cluster/monitor/pubsubmon" "github.com/ipfs/ipfs-cluster/observations" "github.com/ipfs/ipfs-cluster/pintracker/maptracker" "github.com/ipfs/ipfs-cluster/pintracker/stateless" - "github.com/ipfs/ipfs-cluster/state" - "github.com/ipfs/ipfs-cluster/state/mapstate" "github.com/ipfs/ipfs-cluster/test" "github.com/ipfs/ipfs-cluster/version" - crypto "github.com/libp2p/go-libp2p-crypto" + ds "github.com/ipfs/go-datastore" host "github.com/libp2p/go-libp2p-host" peer "github.com/libp2p/go-libp2p-peer" - peerstore "github.com/libp2p/go-libp2p-peerstore" ma "github.com/multiformats/go-multiaddr" ) @@ -45,8 +52,10 @@ var ( logLevel = "CRITICAL" customLogLvlFacilities = logFacilities{} - pmonitor = "pubsub" - ptracker = "map" + ptracker = "map" + consensus = "raft" + + testsFolder = "clusterTestsFolder" // When testing with fixed ports... // clusterPort = 10000 @@ -77,8 +86,8 @@ func init() { flag.StringVar(&logLevel, "loglevel", logLevel, "default log level for tests") flag.IntVar(&nClusters, "nclusters", nClusters, "number of clusters to use") flag.IntVar(&nPins, "npins", nPins, "number of pins to pin/unpin/check") - flag.StringVar(&pmonitor, "monitor", pmonitor, "monitor implementation") flag.StringVar(&ptracker, "tracker", ptracker, "tracker implementation") + flag.StringVar(&consensus, "consensus", consensus, "consensus implementation") flag.Parse() rand.Seed(time.Now().UnixNano()) @@ -104,6 +113,10 @@ func init() { } } ReadyTimeout = 11 * time.Second + + // GossipSub needs to heartbeat to discover newly connected hosts + // This speeds things up a little. + pubsub.GossipSubHeartbeatInterval = 50 * time.Millisecond } func checkErr(t *testing.T, err error) { @@ -121,13 +134,10 @@ func randomBytes() []byte { return bs } -func createComponents(t *testing.T, i int, clusterSecret []byte, staging bool) (host.Host, *Config, *raft.Consensus, []API, IPFSConnector, state.State, PinTracker, PeerMonitor, PinAllocator, Informer, Tracer, *test.IpfsMock) { +func createComponents(t *testing.T, host host.Host, pubsub *pubsub.PubSub, dht *dht.IpfsDHT, i int, staging bool) (*Config, ds.Datastore, Consensus, []API, IPFSConnector, PinTracker, PeerMonitor, PinAllocator, Informer, Tracer, *test.IpfsMock) { ctx := context.Background() - mock := test.NewIpfsMock() - // - //clusterAddr, _ := ma.NewMultiaddr(fmt.Sprintf("/ip4/127.0.0.1/tcp/%d", clusterPort+i)) - // Bind on port 0 - clusterAddr, _ := ma.NewMultiaddr("/ip4/127.0.0.1/tcp/0") + mock := test.NewIpfsMock(t) + //apiAddr, _ := ma.NewMultiaddr(fmt.Sprintf("/ip4/127.0.0.1/tcp/%d", apiPort+i)) // Bind on port 0 apiAddr, _ := ma.NewMultiaddr("/ip4/127.0.0.1/tcp/0") @@ -135,30 +145,27 @@ func createComponents(t *testing.T, i int, clusterSecret []byte, staging bool) ( // proxyAddr, _ := ma.NewMultiaddr(fmt.Sprintf("/ip4/127.0.0.1/tcp/%d", ipfsProxyPort+i)) proxyAddr, _ := ma.NewMultiaddr("/ip4/127.0.0.1/tcp/0") nodeAddr, _ := ma.NewMultiaddr(fmt.Sprintf("/ip4/%s/tcp/%d", mock.Addr, mock.Port)) - priv, pub, err := crypto.GenerateKeyPair(crypto.RSA, 2048) - checkErr(t, err) - pid, err := peer.IDFromPublicKey(pub) - checkErr(t, err) + peername := fmt.Sprintf("peer_%d", i) - clusterCfg, apiCfg, ipfsproxyCfg, ipfshttpCfg, consensusCfg, maptrackerCfg, statelesstrackerCfg, psmonCfg, diskInfCfg, tracingCfg := testingConfigs() + clusterCfg, apiCfg, ipfsproxyCfg, ipfshttpCfg, badgerCfg, raftCfg, crdtCfg, maptrackerCfg, statelesstrackerCfg, psmonCfg, diskInfCfg, tracingCfg := testingConfigs() - clusterCfg.ID = pid + clusterCfg.ID = host.ID() + clusterCfg.PrivateKey = host.Peerstore().PrivKey(host.ID()) clusterCfg.Peername = peername - clusterCfg.PrivateKey = priv - clusterCfg.Secret = clusterSecret - clusterCfg.ListenAddr = clusterAddr clusterCfg.LeaveOnShutdown = false - clusterCfg.SetBaseDir("./e2eTestRaft/" + pid.Pretty()) - - host, err := NewClusterHost(context.Background(), clusterCfg) - checkErr(t, err) + clusterCfg.SetBaseDir(filepath.Join(testsFolder, host.ID().Pretty())) apiCfg.HTTPListenAddr = apiAddr + ipfsproxyCfg.ListenAddr = proxyAddr ipfsproxyCfg.NodeAddr = nodeAddr + ipfshttpCfg.NodeAddr = nodeAddr - consensusCfg.DataFolder = "./e2eTestRaft/" + pid.Pretty() + + raftCfg.DataFolder = filepath.Join(testsFolder, host.ID().Pretty()) + + badgerCfg.Folder = filepath.Join(testsFolder, host.ID().Pretty(), "badger") api, err := rest.NewAPI(ctx, apiCfg) checkErr(t, err) @@ -167,22 +174,52 @@ func createComponents(t *testing.T, i int, clusterSecret []byte, staging bool) ( ipfs, err := ipfshttp.NewConnector(ipfshttpCfg) checkErr(t, err) - state := mapstate.NewMapState() tracker := makePinTracker(t, clusterCfg.ID, maptrackerCfg, statelesstrackerCfg, clusterCfg.Peername) - mon, err := pubsubmon.New(host, psmonCfg) - checkErr(t, err) - alloc := descendalloc.NewAllocator() inf, err := disk.NewInformer(diskInfCfg) checkErr(t, err) - raftCon, err := raft.NewConsensus(host, consensusCfg, state, staging) + + store := makeStore(t, badgerCfg) + cons := makeConsensus(t, store, host, pubsub, dht, raftCfg, staging, crdtCfg) + + var peersF func(context.Context) ([]peer.ID, error) + if consensus == "raft" { + peersF = cons.Peers + } + mon, err := pubsubmon.New(psmonCfg, pubsub, peersF) checkErr(t, err) tracer, err := observations.SetupTracing(tracingCfg) checkErr(t, err) - return host, clusterCfg, raftCon, []API{api, ipfsProxy}, ipfs, state, tracker, mon, alloc, inf, tracer, mock + return clusterCfg, store, cons, []API{api, ipfsProxy}, ipfs, tracker, mon, alloc, inf, tracer, mock +} + +func makeStore(t *testing.T, badgerCfg *badger.Config) ds.Datastore { + switch consensus { + case "crdt": + dstr, err := badger.New(badgerCfg) + checkErr(t, err) + return dstr + default: + return inmem.New() + } +} + +func makeConsensus(t *testing.T, store ds.Datastore, h host.Host, psub *pubsub.PubSub, dht *dht.IpfsDHT, raftCfg *raft.Config, staging bool, crdtCfg *crdt.Config) Consensus { + switch consensus { + case "raft": + raftCon, err := raft.NewConsensus(h, raftCfg, store, staging) + checkErr(t, err) + return raftCon + case "crdt": + crdtCon, err := crdt.New(h, dht, psub, crdtCfg, store) + checkErr(t, err) + return crdtCon + default: + panic("bad consensus") + } } func makePinTracker(t *testing.T, pid peer.ID, mptCfg *maptracker.Config, sptCfg *stateless.Config, peerName string) PinTracker { @@ -198,27 +235,61 @@ func makePinTracker(t *testing.T, pid peer.ID, mptCfg *maptracker.Config, sptCfg return ptrkr } -func createCluster(t *testing.T, host host.Host, clusterCfg *Config, raftCons *raft.Consensus, apis []API, ipfs IPFSConnector, state state.State, tracker PinTracker, mon PeerMonitor, alloc PinAllocator, inf Informer, tracer Tracer) *Cluster { - cl, err := NewCluster(host, clusterCfg, raftCons, apis, ipfs, state, tracker, mon, alloc, inf, tracer) +func createCluster(t *testing.T, host host.Host, dht *dht.IpfsDHT, clusterCfg *Config, store ds.Datastore, consensus Consensus, apis []API, ipfs IPFSConnector, tracker PinTracker, mon PeerMonitor, alloc PinAllocator, inf Informer, tracer Tracer) *Cluster { + cl, err := NewCluster(host, dht, clusterCfg, store, consensus, apis, ipfs, tracker, mon, alloc, inf, tracer) checkErr(t, err) return cl } func createOnePeerCluster(t *testing.T, nth int, clusterSecret []byte) (*Cluster, *test.IpfsMock) { - host, clusterCfg, consensusCfg, api, ipfs, state, tracker, mon, alloc, inf, tracer, mock := createComponents(t, nth, clusterSecret, false) - cl := createCluster(t, host, clusterCfg, consensusCfg, api, ipfs, state, tracker, mon, alloc, inf, tracer) + hosts, pubsubs, dhts := createHosts(t, clusterSecret, 1) + clusterCfg, store, consensus, api, ipfs, tracker, mon, alloc, inf, tracer, mock := createComponents(t, hosts[0], pubsubs[0], dhts[0], nth, false) + cl := createCluster(t, hosts[0], dhts[0], clusterCfg, store, consensus, api, ipfs, tracker, mon, alloc, inf, tracer) <-cl.Ready() return cl, mock } +func createHosts(t *testing.T, clusterSecret []byte, nClusters int) ([]host.Host, []*pubsub.PubSub, []*dht.IpfsDHT) { + hosts := make([]host.Host, nClusters, nClusters) + pubsubs := make([]*pubsub.PubSub, nClusters, nClusters) + dhts := make([]*dht.IpfsDHT, nClusters, nClusters) + + ctx := context.Background() + + clusterAddr, _ := ma.NewMultiaddr("/ip4/127.0.0.1/tcp/0") + + for i := range hosts { + priv, _, err := crypto.GenerateKeyPair(crypto.RSA, 2048) + checkErr(t, err) + h, err := newHost(ctx, clusterSecret, priv, []ma.Multiaddr{clusterAddr}) + checkErr(t, err) + + // DHT needs to be created BEFORE connecting the peers, but + // bootstrapped AFTER + d, err := newDHT(ctx, h) + checkErr(t, err) + dhts[i] = d + + hosts[i] = routedHost(h, d) + + // Pubsub needs to be created BEFORE connecting the peers, + // otherwise they are not picked up. + psub, err := newPubSub(ctx, hosts[i]) + checkErr(t, err) + pubsubs[i] = psub + } + + return hosts, pubsubs, dhts +} + func createClusters(t *testing.T) ([]*Cluster, []*test.IpfsMock) { ctx := context.Background() - os.RemoveAll("./e2eTestRaft") + os.RemoveAll(testsFolder) cfgs := make([]*Config, nClusters, nClusters) - raftCons := make([]*raft.Consensus, nClusters, nClusters) + stores := make([]ds.Datastore, nClusters, nClusters) + cons := make([]Consensus, nClusters, nClusters) apis := make([][]API, nClusters, nClusters) ipfss := make([]IPFSConnector, nClusters, nClusters) - states := make([]state.State, nClusters, nClusters) trackers := make([]PinTracker, nClusters, nClusters) mons := make([]PeerMonitor, nClusters, nClusters) allocs := make([]PinAllocator, nClusters, nClusters) @@ -226,39 +297,26 @@ func createClusters(t *testing.T) ([]*Cluster, []*test.IpfsMock) { tracers := make([]Tracer, nClusters, nClusters) ipfsMocks := make([]*test.IpfsMock, nClusters, nClusters) - hosts := make([]host.Host, nClusters, nClusters) clusters := make([]*Cluster, nClusters, nClusters) // Uncomment when testing with fixed ports // clusterPeers := make([]ma.Multiaddr, nClusters, nClusters) + hosts, pubsubs, dhts := createHosts(t, testingClusterSecret, nClusters) + for i := 0; i < nClusters; i++ { // staging = true for all except first (i==0) - hosts[i], cfgs[i], raftCons[i], apis[i], ipfss[i], states[i], trackers[i], mons[i], allocs[i], infs[i], tracers[i], ipfsMocks[i] = createComponents(t, i, testingClusterSecret, i != 0) - } - - // open connections among all hosts - for _, h := range hosts { - for _, h2 := range hosts { - if h.ID() != h2.ID() { - - h.Peerstore().AddAddrs(h2.ID(), h2.Addrs(), peerstore.PermanentAddrTTL) - _, err := h.Network().DialPeer(context.Background(), h2.ID()) - if err != nil { - t.Fatal(err) - } - } - } + cfgs[i], stores[i], cons[i], apis[i], ipfss[i], trackers[i], mons[i], allocs[i], infs[i], tracers[i], ipfsMocks[i] = createComponents(t, hosts[i], pubsubs[i], dhts[i], i, i != 0) } // Start first node - clusters[0] = createCluster(t, hosts[0], cfgs[0], raftCons[0], apis[0], ipfss[0], states[0], trackers[0], mons[0], allocs[0], infs[0], tracers[0]) + clusters[0] = createCluster(t, hosts[0], dhts[0], cfgs[0], stores[0], cons[0], apis[0], ipfss[0], trackers[0], mons[0], allocs[0], infs[0], tracers[0]) <-clusters[0].Ready() bootstrapAddr := clusterAddr(clusters[0]) // Start the rest and join for i := 1; i < nClusters; i++ { - clusters[i] = createCluster(t, hosts[i], cfgs[i], raftCons[i], apis[i], ipfss[i], states[i], trackers[i], mons[i], allocs[i], infs[i], tracers[i]) + clusters[i] = createCluster(t, hosts[i], dhts[i], cfgs[i], stores[i], cons[i], apis[i], ipfss[i], trackers[i], mons[i], allocs[i], infs[i], tracers[i]) err := clusters[i].Join(ctx, bootstrapAddr) if err != nil { logger.Error(err) @@ -266,6 +324,32 @@ func createClusters(t *testing.T) ([]*Cluster, []*test.IpfsMock) { } <-clusters[i].Ready() } + + // connect all hosts + for _, h := range hosts { + for _, h2 := range hosts { + if h.ID() != h2.ID() { + h.Peerstore().AddAddrs(h2.ID(), h2.Addrs(), peerstore.PermanentAddrTTL) + _, err := h.Network().DialPeer(ctx, h2.ID()) + if err != nil { + t.Log(err) + } + } + + } + } + + // // Bootstrap the DHTs + dhtCfg := dht.BootstrapConfig{ + Queries: 1, + Period: 600 * time.Millisecond, + Timeout: 300 * time.Millisecond, + } + + for _, d := range dhts { + d.BootstrapWithConfig(ctx, dhtCfg) + } + waitForLeader(t, clusters) return clusters, ipfsMocks @@ -280,7 +364,7 @@ func shutdownClusters(t *testing.T, clusters []*Cluster, m []*test.IpfsMock) { } m[i].Close() } - os.RemoveAll("./e2eTestRaft") + os.RemoveAll(testsFolder) } func runF(t *testing.T, clusters []*Cluster, f func(*testing.T, *Cluster)) { @@ -341,6 +425,9 @@ func waitForLeaderAndMetrics(t *testing.T, clusters []*Cluster) { // Makes sure there is a leader and everyone knows about it. func waitForLeader(t *testing.T, clusters []*Cluster) { + if consensus == "crdt" { + return // yai + } ctx := context.Background() timer := time.NewTimer(time.Minute) ticker := time.NewTicker(100 * time.Millisecond) @@ -384,6 +471,8 @@ func TestClustersPeers(t *testing.T) { clusters, mock := createClusters(t) defer shutdownClusters(t, clusters, mock) + delay() + j := rand.Intn(nClusters) // choose a random cluster peer peers := clusters[j].Peers(ctx) @@ -400,6 +489,10 @@ func TestClustersPeers(t *testing.T) { } for _, p := range peers { + if p.Error != "" { + t.Error(p.ID, p.Error) + continue + } peerIDMap[p.ID] = p } @@ -433,13 +526,18 @@ func TestClustersPin(t *testing.T) { if err != nil { t.Errorf("error pinning %s: %s", h, err) } - // Test re-pin - err = clusters[j].Pin(ctx, api.PinCid(h)) - if err != nil { - t.Errorf("error repinning %s: %s", h, err) - } + // // Test re-pin + // err = clusters[j].Pin(ctx, api.PinCid(h)) + // if err != nil { + // t.Errorf("error repinning %s: %s", h, err) + // } + } + switch consensus { + case "crdt": + time.Sleep(20 * time.Second) + default: + delay() } - delay() fpinned := func(t *testing.T, c *Cluster) { status := c.tracker.StatusAll(ctx) for _, v := range status { @@ -454,7 +552,14 @@ func TestClustersPin(t *testing.T) { runF(t, clusters, fpinned) // Unpin everything - pinList := clusters[0].Pins(ctx) + pinList, err := clusters[0].Pins(ctx) + if err != nil { + t.Fatal(err) + } + + if len(pinList) != nPins { + t.Fatalf("pin list has %d but pinned %d", len(pinList), nPins) + } for i := 0; i < len(pinList); i++ { // test re-unpin fails @@ -465,11 +570,11 @@ func TestClustersPin(t *testing.T) { } } delay() - for i := 0; i < nPins; i++ { + for i := 0; i < len(pinList); i++ { j := rand.Intn(nClusters) // choose a random cluster peer err := clusters[j].Unpin(ctx, pinList[i].Cid) if err == nil { - t.Errorf("expected error re-unpinning %s: %s", pinList[i].Cid, err) + t.Errorf("expected error re-unpinning %s", pinList[i].Cid) } } @@ -556,36 +661,53 @@ func TestClustersStatusAllWithErrors(t *testing.T) { t.Fatal("bad status. Expected one item") } - stts := statuses[0] - if len(stts.PeerMap) != nClusters { - t.Error("bad number of peers in status") - } + // Raft and CRDT behave differently here + switch consensus { + case "raft": + // Raft will have all statuses with one of them + // being in ERROR because the peer is off - pid := peer.IDB58Encode(clusters[1].ID(ctx).ID) - errst := stts.PeerMap[pid] + stts := statuses[0] + if len(stts.PeerMap) != nClusters { + t.Error("bad number of peers in status") + } - if !errst.Cid.Equals(h) { - t.Error("errored pinInfo should have a good cid") - } + pid := peer.IDB58Encode(clusters[1].id) + errst := stts.PeerMap[pid] - if errst.Status != api.TrackerStatusClusterError { - t.Error("erroring status should be set to ClusterError") - } + if !errst.Cid.Equals(h) { + t.Error("errored pinInfo should have a good cid") + } - // now check with Cid status - status, err := c.Status(ctx, h) - if err != nil { - t.Error(err) - } + if errst.Status != api.TrackerStatusClusterError { + t.Error("erroring status should be set to ClusterError") + } - pinfo := status.PeerMap[pid] + // now check with Cid status + status, err := c.Status(ctx, h) + if err != nil { + t.Error(err) + } - if pinfo.Status != api.TrackerStatusClusterError { - t.Error("erroring status should be ClusterError") - } + pinfo := status.PeerMap[pid] + + if pinfo.Status != api.TrackerStatusClusterError { + t.Error("erroring status should be ClusterError") + } + + if !pinfo.Cid.Equals(h) { + t.Error("errored status should have a good cid") + } + case "crdt": + // CRDT will not have contacted the offline peer because + // its metric expired and therefore is not in the + // peerset. + if len(statuses[0].PeerMap) != nClusters-1 { + t.Error("expected a different number of statuses") + } + default: + t.Fatal("bad consensus") - if !pinfo.Cid.Equals(h) { - t.Error("errored status should have a good cid") } } @@ -958,7 +1080,10 @@ func TestClustersReplication(t *testing.T) { t.Errorf("Expected 1 remote pin but got %d", numRemote) } - pins := c.Pins(ctx) + pins, err := c.Pins(ctx) + if err != nil { + t.Fatal(err) + } for _, pin := range pins { allocs := pin.Allocations if len(allocs) != nClusters-1 { @@ -1325,7 +1450,11 @@ func TestClustersReplicationRealloc(t *testing.T) { // Let the pin arrive pinDelay() - pin := clusters[j].Pins(ctx)[0] + pinList, err := clusters[j].Pins(ctx) + if err != nil { + t.Fatal(err) + } + pin := pinList[0] allocs := sort.StringSlice(api.PeersToStrings(pin.Allocations)) allocs.Sort() allocsStr := fmt.Sprintf("%s", allocs) @@ -1339,7 +1468,11 @@ func TestClustersReplicationRealloc(t *testing.T) { pinDelay() - pin2 := clusters[j].Pins(ctx)[0] + pinList2, err := clusters[j].Pins(ctx) + if err != nil { + t.Fatal(err) + } + pin2 := pinList2[0] allocs2 := sort.StringSlice(api.PeersToStrings(pin2.Allocations)) allocs2.Sort() allocsStr2 := fmt.Sprintf("%s", allocs2) @@ -1444,6 +1577,11 @@ func TestClustersRebalanceOnPeerDown(t *testing.T) { t.Skip("Need at least 5 peers") } + if consensus == "crdt" { + t.Log("FIXME when re-alloc changes come through") + return + } + clusters, mock := createClusters(t) defer shutdownClusters(t, clusters, mock) for _, c := range clusters { @@ -1501,7 +1639,7 @@ func TestClustersRebalanceOnPeerDown(t *testing.T) { // peers in clusterIDs are fully connected to each other and the expected ipfs // mock connectivity exists. Cluster peers not in clusterIDs are assumed to // be disconnected and the graph should reflect this -func validateClusterGraph(t *testing.T, graph api.ConnectGraph, clusterIDs map[string]struct{}) { +func validateClusterGraph(t *testing.T, graph api.ConnectGraph, clusterIDs map[string]struct{}, peerNum int) { // Check that all cluster peers see each other as peers for id1, peers := range graph.ClusterLinks { if _, ok := clusterIDs[id1]; !ok { @@ -1525,7 +1663,7 @@ func validateClusterGraph(t *testing.T, graph api.ConnectGraph, clusterIDs map[s } } } - if len(graph.ClusterLinks) != nClusters { + if len(graph.ClusterLinks) != peerNum { t.Errorf("Unexpected number of cluster nodes in graph") } @@ -1583,7 +1721,7 @@ func TestClustersGraphConnected(t *testing.T) { id := peer.IDB58Encode(c.ID(ctx).ID) clusterIDs[id] = struct{}{} } - validateClusterGraph(t, graph, clusterIDs) + validateClusterGraph(t, graph, clusterIDs, nClusters) } // Similar to the previous test we get a cluster graph report from a peer. @@ -1630,7 +1768,13 @@ func TestClustersGraphUnhealthy(t *testing.T) { id := peer.IDB58Encode(c.ID(ctx).ID) clusterIDs[id] = struct{}{} } - validateClusterGraph(t, graph, clusterIDs) + peerNum := nClusters + switch consensus { + case "crdt": + peerNum = nClusters - 2 + } + + validateClusterGraph(t, graph, clusterIDs, peerNum) } // Check that the pin is not re-assigned when a node diff --git a/ipfsconn/ipfshttp/ipfshttp.go b/ipfsconn/ipfshttp/ipfshttp.go index 214d48d0..6eddfe15 100644 --- a/ipfsconn/ipfshttp/ipfshttp.go +++ b/ipfsconn/ipfshttp/ipfshttp.go @@ -710,6 +710,27 @@ func (ipfs *Connector) BlockGet(ctx context.Context, c cid.Cid) ([]byte, error) return ipfs.postCtx(ctx, url, "", nil) } +// // FetchRefs asks IPFS to download blocks recursively to the given depth. +// // It discards the response, but waits until it completes. +// func (ipfs *Connector) FetchRefs(ctx context.Context, c cid.Cid, maxDepth int) error { +// ctx, cancel := context.WithTimeout(ipfs.ctx, ipfs.config.PinTimeout) +// defer cancel() + +// q := url.Values{} +// q.Set("recursive", "true") +// q.Set("unique", "false") // same memory on IPFS side +// q.Set("max-depth", fmt.Sprintf("%d", maxDepth)) +// q.Set("arg", c.String()) + +// url := fmt.Sprintf("refs?%s", q.Encode()) +// err := ipfs.postDiscardBodyCtx(ctx, url) +// if err != nil { +// return err +// } +// logger.Debugf("refs for %s sucessfully fetched", c) +// return nil +// } + // Returns true every updateMetricsMod-th time that we // call this function. func (ipfs *Connector) shouldUpdateMetric() bool { diff --git a/ipfsconn/ipfshttp/ipfshttp_test.go b/ipfsconn/ipfshttp/ipfshttp_test.go index 6154d4af..67a1f140 100644 --- a/ipfsconn/ipfshttp/ipfshttp_test.go +++ b/ipfsconn/ipfshttp/ipfshttp_test.go @@ -19,7 +19,7 @@ func init() { } func testIPFSConnector(t *testing.T) (*Connector, *test.IpfsMock) { - mock := test.NewIpfsMock() + mock := test.NewIpfsMock(t) nodeMAddr, _ := ma.NewMultiaddr(fmt.Sprintf("/ip4/%s/tcp/%d", mock.Addr, mock.Port)) diff --git a/logging.go b/logging.go index 054b1ff1..b11e0cdb 100644 --- a/logging.go +++ b/logging.go @@ -4,6 +4,21 @@ import logging "github.com/ipfs/go-log" var logger = logging.Logger("cluster") +var ( + ansiGray = "\033[0;37m" + ansiYellow = "\033[0;33m" +) + +func init() { + // The whole purpose of this is to print the facility name in yellow + // color in the logs because the current blue is very hard to read. + logging.LogFormats["color"] = ansiGray + + "%{time:15:04:05.000} %{color}%{level:5.5s} " + + ansiYellow + "%{module:10.10s}: %{color:reset}%{message} " + + ansiGray + "%{shortfile}%{color:reset}" + logging.SetupLogging() +} + // LoggingFacilities provides a list of logging identifiers // used by cluster and their default logging level. var LoggingFacilities = map[string]string{ @@ -12,9 +27,9 @@ var LoggingFacilities = map[string]string{ "ipfsproxy": "INFO", "ipfshttp": "INFO", "monitor": "INFO", - "mapstate": "INFO", "dsstate": "INFO", - "consensus": "INFO", + "raft": "INFO", + "crdt": "INFO", "pintracker": "INFO", "ascendalloc": "INFO", "diskinfo": "INFO", @@ -33,7 +48,7 @@ var LoggingFacilitiesExtra = map[string]string{ "p2p-gorpc": "CRITICAL", "swarm2": "ERROR", "libp2p-raft": "CRITICAL", - "raft": "ERROR", + "raftlib": "ERROR", } // SetFacilityLogLevel sets the log level for a given module diff --git a/monitor/metrics/checker.go b/monitor/metrics/checker.go index d96dd78f..e8c58973 100644 --- a/monitor/metrics/checker.go +++ b/monitor/metrics/checker.go @@ -32,22 +32,47 @@ func NewChecker(metrics *Store) *Checker { } } -// CheckPeers will trigger alerts for expired metrics belonging to the -// given peerset. +// CheckPeers will trigger alerts all latest metrics from the given peerset +// when they have expired and no alert has been sent before. func (mc *Checker) CheckPeers(peers []peer.ID) error { for _, peer := range peers { for _, metric := range mc.metrics.PeerMetrics(peer) { - if metric.Valid && metric.Expired() { - err := mc.alert(metric.Peer, metric.Name) - if err != nil { - return err - } + err := mc.alertIfExpired(metric) + if err != nil { + return err } } } return nil } +// CheckAll will trigger alerts for all latest metrics when they have expired +// and no alert has been sent before. +func (mc Checker) CheckAll() error { + for _, metric := range mc.metrics.AllMetrics() { + err := mc.alertIfExpired(metric) + if err != nil { + return err + } + } + + return nil +} + +func (mc *Checker) alertIfExpired(metric *api.Metric) error { + if !metric.Expired() { + return nil + } + + err := mc.alert(metric.Peer, metric.Name) + if err != nil { + return err + } + metric.Valid = false + mc.metrics.Add(metric) // invalidate so we don't alert again + return nil +} + func (mc *Checker) alert(pid peer.ID, metricName string) error { alrt := &api.Alert{ Peer: pid, @@ -74,11 +99,15 @@ func (mc *Checker) Watch(ctx context.Context, peersF func(context.Context) ([]pe for { select { case <-ticker.C: - peers, err := peersF(ctx) - if err != nil { - continue + if peersF != nil { + peers, err := peersF(ctx) + if err != nil { + continue + } + mc.CheckPeers(peers) + } else { + mc.CheckAll() } - mc.CheckPeers(peers) case <-ctx.Done(): ticker.Stop() return diff --git a/monitor/metrics/store.go b/monitor/metrics/store.go index afc0337b..5ac04667 100644 --- a/monitor/metrics/store.go +++ b/monitor/metrics/store.go @@ -47,9 +47,9 @@ func (mtrs *Store) Add(m *api.Metric) { window.Add(m) } -// Latest returns all the last known valid metrics. A metric is valid -// if it has not expired. -func (mtrs *Store) Latest(name string) []*api.Metric { +// LatestValid returns all the last known valid metrics of a given type. A metric +// is valid if it has not expired. +func (mtrs *Store) LatestValid(name string) []*api.Metric { mtrs.mux.RLock() defer mtrs.mux.RUnlock() @@ -69,6 +69,26 @@ func (mtrs *Store) Latest(name string) []*api.Metric { return metrics } +// AllMetrics returns the latest metrics for all peers and metrics types. It +// may return expired metrics. +func (mtrs *Store) AllMetrics() []*api.Metric { + mtrs.mux.RLock() + defer mtrs.mux.RUnlock() + + result := make([]*api.Metric, 0) + + for _, byPeer := range mtrs.byName { + for _, window := range byPeer { + metric, err := window.Latest() + if err != nil || !metric.Valid { + continue + } + result = append(result, metric) + } + } + return result +} + // PeerMetrics returns the latest metrics for a given peer ID for // all known metrics types. It may return expired metrics. func (mtrs *Store) PeerMetrics(pid peer.ID) []*api.Metric { @@ -83,7 +103,7 @@ func (mtrs *Store) PeerMetrics(pid peer.ID) []*api.Metric { continue } metric, err := window.Latest() - if err != nil { + if err != nil || !metric.Valid { continue } result = append(result, metric) diff --git a/monitor/metrics/store_test.go b/monitor/metrics/store_test.go index 0c9e5e0b..f8ab061c 100644 --- a/monitor/metrics/store_test.go +++ b/monitor/metrics/store_test.go @@ -20,14 +20,14 @@ func TestStoreLatest(t *testing.T) { metr.SetTTL(200 * time.Millisecond) store.Add(metr) - latest := store.Latest("test") + latest := store.LatestValid("test") if len(latest) != 1 { t.Error("expected 1 metric") } time.Sleep(220 * time.Millisecond) - latest = store.Latest("test") + latest = store.LatestValid("test") if len(latest) != 0 { t.Error("expected no metrics") } diff --git a/monitor/pubsubmon/pubsubmon.go b/monitor/pubsubmon/pubsubmon.go index 02b652e2..ff396ca8 100644 --- a/monitor/pubsubmon/pubsubmon.go +++ b/monitor/pubsubmon/pubsubmon.go @@ -14,7 +14,6 @@ import ( logging "github.com/ipfs/go-log" rpc "github.com/libp2p/go-libp2p-gorpc" - host "github.com/libp2p/go-libp2p-host" peer "github.com/libp2p/go-libp2p-peer" pubsub "github.com/libp2p/go-libp2p-pubsub" msgpack "github.com/multiformats/go-multicodec/msgpack" @@ -35,9 +34,9 @@ type Monitor struct { rpcClient *rpc.Client rpcReady chan struct{} - host host.Host pubsub *pubsub.PubSub subscription *pubsub.Subscription + peers PeersFunc metrics *metrics.Store checker *metrics.Checker @@ -49,8 +48,18 @@ type Monitor struct { wg sync.WaitGroup } -// New creates a new PubSub monitor, using the given host and config. -func New(h host.Host, cfg *Config) (*Monitor, error) { +// PeersFunc allows the Monitor to filter and discard metrics +// that do not belong to a given peerset. +type PeersFunc func(context.Context) ([]peer.ID, error) + +// New creates a new PubSub monitor, using the given host, config and +// PeersFunc. The PeersFunc can be nil. In this case, no metric filtering is +// done based on peers (any peer is considered part of the peerset). +func New( + cfg *Config, + psub *pubsub.PubSub, + peers PeersFunc, +) (*Monitor, error) { err := cfg.Validate() if err != nil { return nil, err @@ -61,13 +70,7 @@ func New(h host.Host, cfg *Config) (*Monitor, error) { mtrs := metrics.NewStore() checker := metrics.NewChecker(mtrs) - pubsub, err := pubsub.NewGossipSub(ctx, h) - if err != nil { - cancel() - return nil, err - } - - subscription, err := pubsub.Subscribe(PubsubTopic) + subscription, err := psub.Subscribe(PubsubTopic) if err != nil { cancel() return nil, err @@ -78,9 +81,9 @@ func New(h host.Host, cfg *Config) (*Monitor, error) { cancel: cancel, rpcReady: make(chan struct{}, 1), - host: h, - pubsub: pubsub, + pubsub: psub, subscription: subscription, + peers: peers, metrics: mtrs, checker: checker, @@ -95,7 +98,7 @@ func (mon *Monitor) run() { select { case <-mon.rpcReady: go mon.logFromPubsub() - go mon.checker.Watch(mon.ctx, mon.getPeers, mon.config.CheckInterval) + go mon.checker.Watch(mon.ctx, mon.peers, mon.config.CheckInterval) case <-mon.ctx.Done(): } } @@ -213,36 +216,21 @@ func (mon *Monitor) PublishMetric(ctx context.Context, m *api.Metric) error { return nil } -// getPeers gets the current list of peers from the consensus component -func (mon *Monitor) getPeers(ctx context.Context) ([]peer.ID, error) { - ctx, span := trace.StartSpan(ctx, "monitor/pubsub/getPeers") - defer span.End() - - var peers []peer.ID - err := mon.rpcClient.CallContext( - ctx, - "", - "Cluster", - "ConsensusPeers", - struct{}{}, - &peers, - ) - if err != nil { - logger.Error(err) - } - return peers, err -} - // LatestMetrics returns last known VALID metrics of a given type. A metric -// is only valid if it has not expired and belongs to a current cluster peers. +// is only valid if it has not expired and belongs to a current cluster peer. func (mon *Monitor) LatestMetrics(ctx context.Context, name string) []*api.Metric { ctx, span := trace.StartSpan(ctx, "monitor/pubsub/LatestMetrics") defer span.End() - latest := mon.metrics.Latest(name) + latest := mon.metrics.LatestValid(name) - // Make sure we only return metrics in the current peerset - peers, err := mon.getPeers(ctx) + if mon.peers == nil { + return latest + } + + // Make sure we only return metrics in the current peerset if we have + // a peerset provider. + peers, err := mon.peers(ctx) if err != nil { return []*api.Metric{} } diff --git a/monitor/pubsubmon/pubsubmon_test.go b/monitor/pubsubmon/pubsubmon_test.go index 4c4b33fe..0e1ce10b 100644 --- a/monitor/pubsubmon/pubsubmon_test.go +++ b/monitor/pubsubmon/pubsubmon_test.go @@ -14,6 +14,8 @@ import ( peerstore "github.com/libp2p/go-libp2p-peerstore" pubsub "github.com/libp2p/go-libp2p-pubsub" + host "github.com/libp2p/go-libp2p-host" + "github.com/ipfs/ipfs-cluster/api" "github.com/ipfs/ipfs-cluster/test" ) @@ -55,7 +57,11 @@ func (mf *metricFactory) count() int { return mf.counter } -func testPeerMonitor(t *testing.T) (*Monitor, func()) { +func peers(ctx context.Context) ([]peer.ID, error) { + return []peer.ID{test.PeerID1, test.PeerID2, test.PeerID3}, nil +} + +func testPeerMonitor(t *testing.T) (*Monitor, host.Host, func()) { ctx := context.Background() h, err := libp2p.New( context.Background(), @@ -65,11 +71,22 @@ func testPeerMonitor(t *testing.T) (*Monitor, func()) { t.Fatal(err) } + psub, err := pubsub.NewGossipSub( + ctx, + h, + pubsub.WithMessageSigning(true), + pubsub.WithStrictSignatureVerification(true), + ) + if err != nil { + h.Close() + t.Fatal(err) + } + mock := test.NewMockRPCClientWithHost(t, h) cfg := &Config{} cfg.Default() cfg.CheckInterval = 2 * time.Second - mon, err := New(h, cfg) + mon, err := New(cfg, psub, peers) if err != nil { t.Fatal(err) } @@ -80,12 +97,12 @@ func testPeerMonitor(t *testing.T) (*Monitor, func()) { h.Close() } - return mon, shutdownF + return mon, h, shutdownF } func TestPeerMonitorShutdown(t *testing.T) { ctx := context.Background() - pm, shutdown := testPeerMonitor(t) + pm, _, shutdown := testPeerMonitor(t) defer shutdown() err := pm.Shutdown(ctx) @@ -101,7 +118,7 @@ func TestPeerMonitorShutdown(t *testing.T) { func TestLogMetricConcurrent(t *testing.T) { ctx := context.Background() - pm, shutdown := testPeerMonitor(t) + pm, _, shutdown := testPeerMonitor(t) defer shutdown() var wg sync.WaitGroup @@ -160,7 +177,7 @@ func TestLogMetricConcurrent(t *testing.T) { func TestPeerMonitorLogMetric(t *testing.T) { ctx := context.Background() - pm, shutdown := testPeerMonitor(t) + pm, _, shutdown := testPeerMonitor(t) defer shutdown() mf := newMetricFactory() @@ -216,19 +233,19 @@ func TestPeerMonitorLogMetric(t *testing.T) { func TestPeerMonitorPublishMetric(t *testing.T) { ctx := context.Background() - pm, shutdown := testPeerMonitor(t) + pm, host, shutdown := testPeerMonitor(t) defer shutdown() - pm2, shutdown2 := testPeerMonitor(t) + pm2, host2, shutdown2 := testPeerMonitor(t) defer shutdown2() time.Sleep(200 * time.Millisecond) - err := pm.host.Connect( + err := host.Connect( context.Background(), peerstore.PeerInfo{ - ID: pm2.host.ID(), - Addrs: pm2.host.Addrs(), + ID: host2.ID(), + Addrs: host2.Addrs(), }, ) if err != nil { @@ -250,9 +267,9 @@ func TestPeerMonitorPublishMetric(t *testing.T) { checkMetric := func(t *testing.T, pm *Monitor) { latestMetrics := pm.LatestMetrics(ctx, "test") if len(latestMetrics) != 1 { - t.Fatal(pm.host.ID(), "expected 1 published metric") + t.Fatal(host.ID(), "expected 1 published metric") } - t.Log(pm.host.ID(), "received metric") + t.Log(host.ID(), "received metric") receivedMetric := latestMetrics[0] if receivedMetric.Peer != metric.Peer || @@ -272,7 +289,7 @@ func TestPeerMonitorPublishMetric(t *testing.T) { func TestPeerMonitorAlerts(t *testing.T) { ctx := context.Background() - pm, shutdown := testPeerMonitor(t) + pm, _, shutdown := testPeerMonitor(t) defer shutdown() mf := newMetricFactory() @@ -282,8 +299,8 @@ func TestPeerMonitorAlerts(t *testing.T) { time.Sleep(time.Second) timeout := time.NewTimer(time.Second * 5) - // it should alert twice at least. Alert re-occurrs. - for i := 0; i < 2; i++ { + // it should alert once. + for i := 0; i < 1; i++ { select { case <-timeout.C: t.Fatal("should have thrown an alert by now") diff --git a/package.json b/package.json index b3f2175e..1028ccc6 100644 --- a/package.json +++ b/package.json @@ -167,6 +167,12 @@ "hash": "QmSSeQqc5QeuefkaM6JFV5tSF9knLUkXKVhW1eYRiqe72W", "name": "uuid", "version": "0.1.0" + }, + { + "author": "hsanjuan", + "hash": "QmWqSMkd9LSYahr9NQvrxoZ4sCzkGctQstqfAYKepzukS6", + "name": "go-ds-crdt", + "version": "0.0.1" } ], "gxVersion": "0.11.0", diff --git a/peer_manager_test.go b/peer_manager_test.go index 5f8f2553..f71a45fd 100644 --- a/peer_manager_test.go +++ b/peer_manager_test.go @@ -11,12 +11,13 @@ import ( "github.com/ipfs/ipfs-cluster/test" cid "github.com/ipfs/go-cid" + host "github.com/libp2p/go-libp2p-host" peer "github.com/libp2p/go-libp2p-peer" peerstore "github.com/libp2p/go-libp2p-peerstore" ma "github.com/multiformats/go-multiaddr" ) -func peerManagerClusters(t *testing.T) ([]*Cluster, []*test.IpfsMock) { +func peerManagerClusters(t *testing.T) ([]*Cluster, []*test.IpfsMock, host.Host) { cls := make([]*Cluster, nClusters, nClusters) mocks := make([]*test.IpfsMock, nClusters, nClusters) var wg sync.WaitGroup @@ -31,22 +32,36 @@ func peerManagerClusters(t *testing.T) ([]*Cluster, []*test.IpfsMock) { } wg.Wait() - // This allows discovery - // PeerAdd won't work without this. - for i := 1; i < nClusters; i++ { + // Create a config + cfg := &Config{} + cfg.Default() + listen, _ := ma.NewMultiaddr("/ip4/127.0.0.1/tcp/0") + cfg.ListenAddr = listen + cfg.Secret = testingClusterSecret + + // Create a bootstrapping libp2p host + h, _, dht, err := NewClusterHost(context.Background(), cfg) + if err != nil { + t.Fatal(err) + } + + // Connect all peers to that host. This will allow that they + // can discover each others via DHT. + for i := 0; i < nClusters; i++ { err := cls[i].host.Connect( context.Background(), peerstore.PeerInfo{ - ID: cls[0].id, - Addrs: cls[0].host.Addrs(), + ID: h.ID(), + Addrs: h.Addrs(), }, ) if err != nil { t.Fatal(err) } } + dht.Bootstrap(context.Background()) - return cls, mocks + return cls, mocks, h } func clusterAddr(c *Cluster) ma.Multiaddr { @@ -62,8 +77,9 @@ func clusterAddr(c *Cluster) ma.Multiaddr { func TestClustersPeerAdd(t *testing.T) { ctx := context.Background() - clusters, mocks := peerManagerClusters(t) + clusters, mocks, boot := peerManagerClusters(t) defer shutdownClusters(t, clusters, mocks) + defer boot.Close() if len(clusters) < 2 { t.Skip("need at least 2 nodes for this test") @@ -99,7 +115,10 @@ func TestClustersPeerAdd(t *testing.T) { } // Check that they are part of the consensus - pins := c.Pins(ctx) + pins, err := c.Pins(ctx) + if err != nil { + t.Fatal(err) + } if len(pins) != 1 { t.Log(pins) t.Error("expected 1 pin everywhere") @@ -139,8 +158,9 @@ func TestClustersPeerAdd(t *testing.T) { func TestClustersJoinBadPeer(t *testing.T) { ctx := context.Background() - clusters, mocks := peerManagerClusters(t) + clusters, mocks, boot := peerManagerClusters(t) defer shutdownClusters(t, clusters, mocks) + defer boot.Close() if len(clusters) < 2 { t.Skip("need at least 2 nodes for this test") @@ -168,8 +188,9 @@ func TestClustersJoinBadPeer(t *testing.T) { func TestClustersPeerAddInUnhealthyCluster(t *testing.T) { ctx := context.Background() - clusters, mocks := peerManagerClusters(t) + clusters, mocks, boot := peerManagerClusters(t) defer shutdownClusters(t, clusters, mocks) + defer boot.Close() if len(clusters) < 3 { t.Skip("need at least 3 nodes for this test") @@ -187,17 +208,37 @@ func TestClustersPeerAddInUnhealthyCluster(t *testing.T) { if err != nil { t.Error("Shutdown should be clean: ", err) } - delay() // This makes sure the leader realizes - //that it's not leader anymore. Otherwise it commits fine. - _, err = clusters[0].PeerAdd(ctx, clusters[2].id) + switch consensus { + case "raft": + delay() // This makes sure the leader realizes that it's not + // leader anymore. Otherwise it commits fine. - if err == nil { - t.Error("expected an error") - } + _, err = clusters[0].PeerAdd(ctx, clusters[2].id) - ids = clusters[0].Peers(ctx) - if len(ids) != 2 { - t.Error("cluster should still have 2 peers") + if err == nil { + t.Error("expected an error") + } + + ids = clusters[0].Peers(ctx) + if len(ids) != 2 { + t.Error("cluster should still have 2 peers") + } + case "crdt": + // crdt does not really care whether we add or remove + + delay() // let metrics expire + _, err = clusters[0].PeerAdd(ctx, clusters[2].id) + + if err != nil { + t.Error(err) + } + + ids = clusters[0].Peers(ctx) + if len(ids) != 2 { + t.Error("cluster should have 2 peers after removing and adding 1") + } + default: + t.Fatal("bad consensus") } } @@ -210,29 +251,37 @@ func TestClustersPeerRemove(t *testing.T) { t.Skip("test needs at least 2 clusters") } - p := clusters[1].ID(ctx).ID - err := clusters[0].PeerRemove(ctx, p) - if err != nil { - t.Error(err) - } + switch consensus { + case "crdt": + // Peer Rm is a no op. + return + case "raft": + p := clusters[1].ID(ctx).ID + err := clusters[0].PeerRemove(ctx, p) + if err != nil { + t.Error(err) + } - delay() + delay() - f := func(t *testing.T, c *Cluster) { - if c.ID(ctx).ID == p { //This is the removed cluster - _, ok := <-c.Done() - if ok { - t.Error("removed peer should have exited") - } - } else { - ids := c.Peers(ctx) - if len(ids) != nClusters-1 { - t.Error("should have removed 1 peer") + f := func(t *testing.T, c *Cluster) { + if c.ID(ctx).ID == p { //This is the removed cluster + _, ok := <-c.Done() + if ok { + t.Error("removed peer should have exited") + } + } else { + ids := c.Peers(ctx) + if len(ids) != nClusters-1 { + t.Error("should have removed 1 peer") + } } } - } - runF(t, clusters, f) + runF(t, clusters, f) + default: + t.Fatal("bad consensus") + } } func TestClustersPeerRemoveSelf(t *testing.T) { @@ -241,30 +290,39 @@ func TestClustersPeerRemoveSelf(t *testing.T) { clusters, mocks := createClusters(t) defer shutdownClusters(t, clusters, mocks) - for i := 0; i < len(clusters); i++ { - waitForLeaderAndMetrics(t, clusters) - peers := clusters[i].Peers(ctx) - t.Logf("Current cluster size: %d", len(peers)) - if len(peers) != (len(clusters) - i) { - t.Fatal("Previous peers not removed correctly") - } - err := clusters[i].PeerRemove(ctx, clusters[i].ID(ctx).ID) - // Last peer member won't be able to remove itself - // In this case, we shut it down. - if err != nil { - if i != len(clusters)-1 { //not last - t.Error(err) - } else { - err := clusters[i].Shutdown(ctx) - if err != nil { - t.Fatal(err) + switch consensus { + case "crdt": + // remove is a no op in CRDTs + return + + case "raft": + for i := 0; i < len(clusters); i++ { + waitForLeaderAndMetrics(t, clusters) + peers := clusters[i].Peers(ctx) + t.Logf("Current cluster size: %d", len(peers)) + if len(peers) != (len(clusters) - i) { + t.Fatal("Previous peers not removed correctly") + } + err := clusters[i].PeerRemove(ctx, clusters[i].ID(ctx).ID) + // Last peer member won't be able to remove itself + // In this case, we shut it down. + if err != nil { + if i != len(clusters)-1 { //not last + t.Error(err) + } else { + err := clusters[i].Shutdown(ctx) + if err != nil { + t.Fatal(err) + } } } + _, more := <-clusters[i].Done() + if more { + t.Error("should be done") + } } - _, more := <-clusters[i].Done() - if more { - t.Error("should be done") - } + default: + t.Fatal("bad consensus") } } @@ -276,47 +334,55 @@ func TestClustersPeerRemoveLeader(t *testing.T) { clusters, mocks := createClusters(t) defer shutdownClusters(t, clusters, mocks) - findLeader := func() *Cluster { - var l peer.ID - for _, c := range clusters { - if !c.shutdownB { - waitForLeaderAndMetrics(t, clusters) - l, _ = c.consensus.Leader(ctx) - } - } - for _, c := range clusters { - if c.id == l { - return c - } - } - return nil - } + switch consensus { + case "crdt": + return + case "raft": - for i := 0; i < len(clusters); i++ { - leader := findLeader() - peers := leader.Peers(ctx) - t.Logf("Current cluster size: %d", len(peers)) - if len(peers) != (len(clusters) - i) { - t.Fatal("Previous peers not removed correctly") - } - err := leader.PeerRemove(ctx, leader.id) - // Last peer member won't be able to remove itself - // In this case, we shut it down. - if err != nil { - if i != len(clusters)-1 { //not last - t.Error(err) - } else { - err := leader.Shutdown(ctx) - if err != nil { - t.Fatal(err) + findLeader := func() *Cluster { + var l peer.ID + for _, c := range clusters { + if !c.shutdownB { + waitForLeaderAndMetrics(t, clusters) + l, _ = c.consensus.Leader(ctx) } } + for _, c := range clusters { + if c.id == l { + return c + } + } + return nil } - _, more := <-leader.Done() - if more { - t.Error("should be done") + + for i := 0; i < len(clusters); i++ { + leader := findLeader() + peers := leader.Peers(ctx) + t.Logf("Current cluster size: %d", len(peers)) + if len(peers) != (len(clusters) - i) { + t.Fatal("Previous peers not removed correctly") + } + err := leader.PeerRemove(ctx, leader.id) + // Last peer member won't be able to remove itself + // In this case, we shut it down. + if err != nil { + if i != len(clusters)-1 { //not last + t.Error(err) + } else { + err := leader.Shutdown(ctx) + if err != nil { + t.Fatal(err) + } + } + } + _, more := <-leader.Done() + if more { + t.Error("should be done") + } + time.Sleep(time.Second / 2) } - time.Sleep(time.Second / 2) + default: + t.Fatal("bad consensus") } } @@ -325,6 +391,11 @@ func TestClustersPeerRemoveReallocsPins(t *testing.T) { clusters, mocks := createClusters(t) defer shutdownClusters(t, clusters, mocks) + if consensus == "crdt" { + t.Log("FIXME when re-alloc changes come through") + return + } + if len(clusters) < 3 { t.Skip("test needs at least 3 clusters") } @@ -381,7 +452,10 @@ func TestClustersPeerRemoveReallocsPins(t *testing.T) { // Find out which pins are associated to the leader. interestingCids := []cid.Cid{} - pins := leader.Pins(ctx) + pins, err := leader.Pins(ctx) + if err != nil { + t.Fatal(err) + } if len(pins) != nClusters { t.Fatal("expected number of tracked pins to be nClusters") } @@ -422,8 +496,9 @@ func TestClustersPeerRemoveReallocsPins(t *testing.T) { func TestClustersPeerJoin(t *testing.T) { ctx := context.Background() - clusters, mocks := peerManagerClusters(t) + clusters, mocks, boot := peerManagerClusters(t) defer shutdownClusters(t, clusters, mocks) + defer boot.Close() if len(clusters) < 3 { t.Skip("test needs at least 3 clusters") @@ -453,7 +528,10 @@ func TestClustersPeerJoin(t *testing.T) { if len(peers) != nClusters { t.Error("all peers should be connected") } - pins := c.Pins(ctx) + pins, err := c.Pins(ctx) + if err != nil { + t.Fatal(err) + } if len(pins) != 1 || !pins[0].Cid.Equals(hash) { t.Error("all peers should have pinned the cid") } @@ -463,8 +541,9 @@ func TestClustersPeerJoin(t *testing.T) { func TestClustersPeerJoinAllAtOnce(t *testing.T) { ctx := context.Background() - clusters, mocks := peerManagerClusters(t) + clusters, mocks, boot := peerManagerClusters(t) defer shutdownClusters(t, clusters, mocks) + defer boot.Close() if len(clusters) < 2 { t.Skip("test needs at least 2 clusters") @@ -487,7 +566,10 @@ func TestClustersPeerJoinAllAtOnce(t *testing.T) { if len(peers) != nClusters { t.Error("all peers should be connected") } - pins := c.Pins(ctx) + pins, err := c.Pins(ctx) + if err != nil { + t.Fatal(err) + } if len(pins) != 1 || !pins[0].Cid.Equals(hash) { t.Error("all peers should have pinned the cid") } @@ -497,9 +579,9 @@ func TestClustersPeerJoinAllAtOnce(t *testing.T) { // This test fails a lot when re-use port is not available (MacOS, Windows) // func TestClustersPeerJoinAllAtOnceWithRandomBootstrap(t *testing.T) { -// clusters, mocks := peerManagerClusters(t) +// clusters, mocks,boot := peerManagerClusters(t) // defer shutdownClusters(t, clusters, mocks) - +// defer boot.Close() // if len(clusters) < 3 { // t.Skip("test needs at least 3 clusters") // } @@ -547,8 +629,9 @@ func TestClustersPeerJoinAllAtOnce(t *testing.T) { // Tests that a peer catches up on the state correctly after rejoining func TestClustersPeerRejoin(t *testing.T) { ctx := context.Background() - clusters, mocks := peerManagerClusters(t) + clusters, mocks, boot := peerManagerClusters(t) defer shutdownClusters(t, clusters, mocks) + defer boot.Close() // pin something in c0 pin1 := test.Cid1 @@ -586,7 +669,7 @@ func TestClustersPeerRejoin(t *testing.T) { // Forget peer so we can re-add one in same address/port f := func(t *testing.T, c *Cluster) { - c.peerManager.RmPeer(clusters[0].id) + c.peerManager.RmPeer(clusters[0].id) // errors ignore for crdts } runF(t, clusters[1:], f) diff --git a/pintracker/maptracker/maptracker.go b/pintracker/maptracker/maptracker.go index 3b4d8393..b86b19b0 100644 --- a/pintracker/maptracker/maptracker.go +++ b/pintracker/maptracker/maptracker.go @@ -233,7 +233,7 @@ func (mpt *MapPinTracker) Untrack(ctx context.Context, c cid.Cid) error { ctx, span := trace.StartSpan(ctx, "tracker/map/Untrack") defer span.End() - logger.Debugf("untracking %s", c) + logger.Infof("untracking %s", c) return mpt.enqueue(ctx, api.PinCid(c), optracker.OperationUnpin, mpt.unpinCh) } diff --git a/pnet_test.go b/pnet_test.go index 83454172..afc13e1b 100644 --- a/pnet_test.go +++ b/pnet_test.go @@ -36,9 +36,10 @@ func TestClusterSecretFormat(t *testing.T) { func TestSimplePNet(t *testing.T) { ctx := context.Background() - clusters, mocks := peerManagerClusters(t) + clusters, mocks, boot := peerManagerClusters(t) defer cleanRaft() defer shutdownClusters(t, clusters, mocks) + defer boot.Close() if len(clusters) < 2 { t.Skip("need at least 2 nodes for this test") diff --git a/pstoremgr/pstoremgr.go b/pstoremgr/pstoremgr.go index 69793d58..d6c102c7 100644 --- a/pstoremgr/pstoremgr.go +++ b/pstoremgr/pstoremgr.go @@ -79,9 +79,11 @@ func (pm *Manager) ImportPeer(addr ma.Multiaddr, connect bool) error { pm.ImportPeers(resolvedAddrs, connect) } if connect { - ctx, cancel := context.WithTimeout(pm.ctx, ConnectTimeout) - defer cancel() - pm.host.Network().DialPeer(ctx, pid) + go func() { + ctx, cancel := context.WithTimeout(pm.ctx, ConnectTimeout) + defer cancel() + pm.host.Network().DialPeer(ctx, pid) + }() } return nil } @@ -210,7 +212,7 @@ func (pm *Manager) SavePeerstore(addrs []api.Multiaddr) { f, err := os.Create(pm.peerstorePath) if err != nil { - logger.Errorf( + logger.Warningf( "could not save peer addresses to %s: %s", pm.peerstorePath, err, diff --git a/rpc_api.go b/rpc_api.go index 913f04fe..fa36f1ef 100644 --- a/rpc_api.go +++ b/rpc_api.go @@ -66,7 +66,10 @@ func (rpcapi *RPCAPI) UnpinPath(ctx context.Context, in *api.PinPath, out *api.P // Pins runs Cluster.Pins(). func (rpcapi *RPCAPI) Pins(ctx context.Context, in struct{}, out *[]*api.Pin) error { - cidList := rpcapi.c.Pins(ctx) + cidList, err := rpcapi.c.Pins(ctx) + if err != nil { + return err + } *out = cidList return nil } diff --git a/sharness/t0050-service-state-upgrade-from-current.sh b/sharness/t0050-service-state-upgrade-from-current.sh deleted file mode 100755 index 744c77ba..00000000 --- a/sharness/t0050-service-state-upgrade-from-current.sh +++ /dev/null @@ -1,32 +0,0 @@ -#!/bin/bash - -test_description="Test service state 'upgrade' from current version" - -. lib/test-lib.sh - -test_ipfs_init -test_cluster_init - -test_expect_success IPFS,CLUSTER "cluster-service state upgrade works" ' - cid=`docker exec ipfs sh -c "echo testing | ipfs add -q"` && - ipfs-cluster-ctl pin add "$cid" && - sleep 5 && - cluster_kill && - sleep 5 && - ipfs-cluster-service --debug --config "test-config" state upgrade -' - -# previous test kills the cluster, we need to re-start -# if done inside the test, we lose debugging output -cluster_start - -test_expect_success IPFS,CLUSTER "state is preserved after migration" ' - cid=`docker exec ipfs sh -c "echo testing | ipfs add -q"` && - ipfs-cluster-ctl pin ls "$cid" | grep -q "$cid" && - ipfs-cluster-ctl status "$cid" | grep -q -i "PINNED" -' - -test_clean_ipfs -test_clean_cluster - -test_done diff --git a/sharness/t0051-service-state-upgrade-from-old.sh b/sharness/t0051-service-state-upgrade-from-old.sh deleted file mode 100755 index 082a05e6..00000000 --- a/sharness/t0051-service-state-upgrade-from-old.sh +++ /dev/null @@ -1,35 +0,0 @@ -#!/bin/bash - -test_description="Test service state upgrade v1 -> v2 and v2 -> v2" - -. lib/test-lib.sh - -test_ipfs_init -test_cluster_init -test_confirm_v1State - -# Make a pin and shutdown to force a snapshot. Modify snapshot files to specify -# a snapshot of v1 state pinning "test" (it's easier than taking a new one each -# time with the correct metadata). Upgrade state, check that the correct -# pin cid is in the state -test_expect_success IPFS,CLUSTER,V1STATE,JQ "cluster-service loads v1 state correctly" ' - cid=`docker exec ipfs sh -c "echo test | ipfs add -q"` && - cid2=`docker exec ipfs sh -c "echo testing | ipfs add -q"` && - ipfs-cluster-ctl pin add "$cid2" && - cluster_kill && - sleep 15 && - SNAP_DIR=`find test-config/raft/snapshots/ -maxdepth 1 -mindepth 1 | head -n 1` && - cp v1State "$SNAP_DIR/state.bin" && - cat "$SNAP_DIR/meta.json" | jq --arg CRC "$V1_CRC" '"'"'.CRC = $CRC'"'"' > tmp.json && - cp tmp.json "$SNAP_DIR/meta.json" && - ipfs-cluster-service --debug --config "test-config" state upgrade && - cluster_start && - ipfs-cluster-ctl pin ls "$cid" | grep -q "$cid" && - ipfs-cluster-ctl status "$cid" | grep -q -i "PINNED" -' - -test_clean_ipfs -test_clean_cluster - - -test_done diff --git a/sharness/t0052-service-state-export.sh b/sharness/t0052-service-state-export.sh index 47ed462a..e11348d0 100755 --- a/sharness/t0052-service-state-export.sh +++ b/sharness/t0052-service-state-export.sh @@ -7,16 +7,6 @@ test_description="Test service state export" test_ipfs_init test_cluster_init - -test_expect_success IPFS,CLUSTER "state export fails without snapshots" ' - cluster_kill && - sleep 5 && - test_expect_code 1 ipfs-cluster-service --debug --config "test-config" state export -' - -test_clean_cluster -test_cluster_init - test_expect_success IPFS,CLUSTER,JQ "state export saves the correct state to expected file" ' cid=`docker exec ipfs sh -c "echo test_52 | ipfs add -q"` && ipfs-cluster-ctl pin add "$cid" && @@ -24,7 +14,7 @@ test_expect_success IPFS,CLUSTER,JQ "state export saves the correct state to exp cluster_kill && sleep 5 && ipfs-cluster-service --debug --config "test-config" state export -f export.json && [ -f export.json ] && - jq -r ".[] | .cid | .[\"/\"]" export.json | grep -q "$cid" + jq -r ".cid | .[\"/\"]" export.json | grep -q "$cid" ' test_clean_ipfs diff --git a/sharness/test_data/importState b/sharness/test_data/importState index dcb0b533..f3a2fea6 100644 --- a/sharness/test_data/importState +++ b/sharness/test_data/importState @@ -1,9 +1,8 @@ -[ - { - "cid": { "/": "QmbrCtydGyPeHiLURSPMqrvE5mCgMCwFYq3UD4XLCeAYw6"}, - "name": "", - "allocations": [], - "replication_factor_min": -1, - "replication_factor_max": -1 - } -] +{ + "cid": { "/": "QmbrCtydGyPeHiLURSPMqrvE5mCgMCwFYq3UD4XLCeAYw6"}, + "name": "", + "allocations": [], + "replication_factor_min": -1, + "replication_factor_max": -1 +} + diff --git a/sharness/test_data/v1State b/sharness/test_data/v1State deleted file mode 100755 index 7a81195e8a5cf54a6c9e62070d1c6de9d9cced6b..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 79 zcmZSN)XyzWFGx(z=4x6NkeTP3SkQQjK`$^jH9t2kEz~b9H?Y#&%sJ4#($X&_x6m~& i&m+{bINv$7!YR)^)Ht;&z$46~VR=|;QE_H|9wPu<&>T