Dependency ugprade: changes needed

* Libp2p protectors no longer needed, use PSK directly
* Generate cluster 32-byte secret here (helper gone from pnet)
* Switch to go-log/v2 in all places
* DHT bootstrapping not needed. Adjust DHT options for tests.
* Do not rely on dissappeared CidToDsKey and DsKeyToCid functions fro dshelp.
* Disable QUIC (does not support private networks)
* Fix tests: autodiscovery started working properly
This commit is contained in:
Hector Sanjuan 2020-03-13 21:40:02 +01:00
parent 8ced9b16de
commit b3853caf36
50 changed files with 179 additions and 205 deletions

View File

@ -3,7 +3,7 @@ os:
- linux - linux
# - osx # - osx
go: go:
- '1.13.x' - '1.14.x'
services: services:
- docker - docker

View File

@ -1,4 +1,4 @@
FROM golang:1.13-stretch AS builder FROM golang:1.14-stretch AS builder
MAINTAINER Hector Sanjuan <hector@protocol.ai> MAINTAINER Hector Sanjuan <hector@protocol.ai>
# This dockerfile builds and runs ipfs-cluster-service. # This dockerfile builds and runs ipfs-cluster-service.

View File

@ -1,4 +1,4 @@
FROM golang:1.13-stretch AS builder FROM golang:1.14-stretch AS builder
MAINTAINER Hector Sanjuan <hector@protocol.ai> MAINTAINER Hector Sanjuan <hector@protocol.ai>
# This dockerfile builds cluster and runs it along with go-ipfs. # This dockerfile builds cluster and runs it along with go-ipfs.

View File

@ -1,4 +1,4 @@
FROM golang:1.13-stretch AS builder FROM golang:1.14-stretch AS builder
MAINTAINER Hector Sanjuan <hector@protocol.ai> MAINTAINER Hector Sanjuan <hector@protocol.ai>
# This build state just builds the cluster binaries # This build state just builds the cluster binaries

View File

@ -14,7 +14,7 @@ import (
cid "github.com/ipfs/go-cid" cid "github.com/ipfs/go-cid"
files "github.com/ipfs/go-ipfs-files" files "github.com/ipfs/go-ipfs-files"
ipld "github.com/ipfs/go-ipld-format" ipld "github.com/ipfs/go-ipld-format"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
merkledag "github.com/ipfs/go-merkledag" merkledag "github.com/ipfs/go-merkledag"
multihash "github.com/multiformats/go-multihash" multihash "github.com/multiformats/go-multihash"
) )

View File

@ -14,7 +14,7 @@ import (
"github.com/ipfs/ipfs-cluster/api" "github.com/ipfs/ipfs-cluster/api"
cid "github.com/ipfs/go-cid" cid "github.com/ipfs/go-cid"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
rpc "github.com/libp2p/go-libp2p-gorpc" rpc "github.com/libp2p/go-libp2p-gorpc"
) )

View File

@ -16,7 +16,7 @@ import (
files "github.com/ipfs/go-ipfs-files" files "github.com/ipfs/go-ipfs-files"
posinfo "github.com/ipfs/go-ipfs-posinfo" posinfo "github.com/ipfs/go-ipfs-posinfo"
ipld "github.com/ipfs/go-ipld-format" ipld "github.com/ipfs/go-ipld-format"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
dag "github.com/ipfs/go-merkledag" dag "github.com/ipfs/go-merkledag"
mfs "github.com/ipfs/go-mfs" mfs "github.com/ipfs/go-mfs"
unixfs "github.com/ipfs/go-unixfs" unixfs "github.com/ipfs/go-unixfs"

View File

@ -16,7 +16,7 @@ import (
humanize "github.com/dustin/go-humanize" humanize "github.com/dustin/go-humanize"
cid "github.com/ipfs/go-cid" cid "github.com/ipfs/go-cid"
ipld "github.com/ipfs/go-ipld-format" ipld "github.com/ipfs/go-ipld-format"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
rpc "github.com/libp2p/go-libp2p-gorpc" rpc "github.com/libp2p/go-libp2p-gorpc"
) )
@ -83,7 +83,7 @@ func (dgs *DAGService) Finalize(ctx context.Context, dataRoot cid.Cid) (cid.Cid,
} }
if !lastCid.Equals(dataRoot) { if !lastCid.Equals(dataRoot) {
logger.Warningf("the last added CID (%s) is not the IPFS data root (%s). This is only normal when adding a single file without wrapping in directory.", lastCid, dataRoot) logger.Warnf("the last added CID (%s) is not the IPFS data root (%s). This is only normal when adding a single file without wrapping in directory.", lastCid, dataRoot)
} }
clusterDAGNodes, err := makeDAG(ctx, dgs.shards) clusterDAGNodes, err := makeDAG(ctx, dgs.shards)

View File

@ -12,7 +12,7 @@ import (
"github.com/ipfs/ipfs-cluster/test" "github.com/ipfs/ipfs-cluster/test"
cid "github.com/ipfs/go-cid" cid "github.com/ipfs/go-cid"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
rpc "github.com/libp2p/go-libp2p-gorpc" rpc "github.com/libp2p/go-libp2p-gorpc"
) )

View File

@ -41,7 +41,7 @@ func newShard(ctx context.Context, rpc *rpc.Client, opts api.PinOptions) (*shard
} }
if opts.ReplicationFactorMin < 0 { if opts.ReplicationFactorMin < 0 {
logger.Warning("Shard is set to replicate everywhere ,which doesn't make sense for sharding") logger.Warn("Shard is set to replicate everywhere ,which doesn't make sense for sharding")
} }
// TODO (hector): get latest metrics for allocations, adjust sizeLimit // TODO (hector): get latest metrics for allocations, adjust sizeLimit

View File

@ -11,7 +11,7 @@ import (
cid "github.com/ipfs/go-cid" cid "github.com/ipfs/go-cid"
ipld "github.com/ipfs/go-ipld-format" ipld "github.com/ipfs/go-ipld-format"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
rpc "github.com/libp2p/go-libp2p-gorpc" rpc "github.com/libp2p/go-libp2p-gorpc"
) )

View File

@ -90,7 +90,7 @@ func (ba *BlockAdder) AddMany(ctx context.Context, nodes []ipld.Node) error {
func ipldNodeToNodeWithMeta(n ipld.Node) *api.NodeWithMeta { func ipldNodeToNodeWithMeta(n ipld.Node) *api.NodeWithMeta {
size, err := n.Size() size, err := n.Size()
if err != nil { if err != nil {
logger.Warning(err) logger.Warn(err)
} }
return &api.NodeWithMeta{ return &api.NodeWithMeta{

View File

@ -11,7 +11,7 @@ import (
"github.com/ipfs/ipfs-cluster/api" "github.com/ipfs/ipfs-cluster/api"
cid "github.com/ipfs/go-cid" cid "github.com/ipfs/go-cid"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
rpc "github.com/libp2p/go-libp2p-gorpc" rpc "github.com/libp2p/go-libp2p-gorpc"
) )

View File

@ -11,7 +11,7 @@ import (
"github.com/ipfs/ipfs-cluster/api" "github.com/ipfs/ipfs-cluster/api"
cid "github.com/ipfs/go-cid" cid "github.com/ipfs/go-cid"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
rpc "github.com/libp2p/go-libp2p-gorpc" rpc "github.com/libp2p/go-libp2p-gorpc"
) )

View File

@ -22,7 +22,7 @@ import (
handlers "github.com/gorilla/handlers" handlers "github.com/gorilla/handlers"
mux "github.com/gorilla/mux" mux "github.com/gorilla/mux"
cid "github.com/ipfs/go-cid" cid "github.com/ipfs/go-cid"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
path "github.com/ipfs/go-path" path "github.com/ipfs/go-path"
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
rpc "github.com/libp2p/go-libp2p-gorpc" rpc "github.com/libp2p/go-libp2p-gorpc"
@ -560,7 +560,7 @@ func (proxy *Server) addHandler(w http.ResponseWriter, r *http.Request) {
params.Layout = "trickle" params.Layout = "trickle"
} }
logger.Warningf("Proxy/add does not support all IPFS params. Current options: %+v", params) logger.Warnf("Proxy/add does not support all IPFS params. Current options: %+v", params)
outputTransform := func(in *api.AddedOutput) interface{} { outputTransform := func(in *api.AddedOutput) interface{} {
r := &ipfsAddResp{ r := &ipfsAddResp{

View File

@ -19,7 +19,7 @@ import (
"github.com/ipfs/ipfs-cluster/api" "github.com/ipfs/ipfs-cluster/api"
"github.com/ipfs/ipfs-cluster/test" "github.com/ipfs/ipfs-cluster/test"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
ma "github.com/multiformats/go-multiaddr" ma "github.com/multiformats/go-multiaddr"
) )

View File

@ -15,9 +15,10 @@ import (
cid "github.com/ipfs/go-cid" cid "github.com/ipfs/go-cid"
shell "github.com/ipfs/go-ipfs-api" shell "github.com/ipfs/go-ipfs-api"
files "github.com/ipfs/go-ipfs-files" files "github.com/ipfs/go-ipfs-files"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
host "github.com/libp2p/go-libp2p-core/host" host "github.com/libp2p/go-libp2p-core/host"
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
pnet "github.com/libp2p/go-libp2p-core/pnet"
ma "github.com/multiformats/go-multiaddr" ma "github.com/multiformats/go-multiaddr"
madns "github.com/multiformats/go-multiaddr-dns" madns "github.com/multiformats/go-multiaddr-dns"
manet "github.com/multiformats/go-multiaddr-net" manet "github.com/multiformats/go-multiaddr-net"
@ -145,7 +146,7 @@ type Config struct {
// If APIAddr is provided, and the peer uses private networks (pnet), // If APIAddr is provided, and the peer uses private networks (pnet),
// then we need to provide the key. If the peer is the cluster peer, // then we need to provide the key. If the peer is the cluster peer,
// this corresponds to the cluster secret. // this corresponds to the cluster secret.
ProtectorKey []byte ProtectorKey pnet.PSK
// ProxyAddr is used to obtain a go-ipfs-api Shell instance pointing // ProxyAddr is used to obtain a go-ipfs-api Shell instance pointing
// to the ipfs proxy endpoint of ipfs-cluster. If empty, the location // to the ipfs proxy endpoint of ipfs-cluster. If empty, the location

View File

@ -11,7 +11,7 @@ import (
libp2p "github.com/libp2p/go-libp2p" libp2p "github.com/libp2p/go-libp2p"
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
pnet "github.com/libp2p/go-libp2p-pnet" pnet "github.com/libp2p/go-libp2p-core/pnet"
ma "github.com/multiformats/go-multiaddr" ma "github.com/multiformats/go-multiaddr"
) )
@ -23,16 +23,12 @@ func testAPI(t *testing.T) *rest.API {
cfg := &rest.Config{} cfg := &rest.Config{}
cfg.Default() cfg.Default()
cfg.HTTPListenAddr = []ma.Multiaddr{apiMAddr} cfg.HTTPListenAddr = []ma.Multiaddr{apiMAddr}
var secret [32]byte secret := make(pnet.PSK, 32, 32)
prot, err := pnet.NewV1ProtectorFromBytes(&secret)
if err != nil {
t.Fatal(err)
}
h, err := libp2p.New( h, err := libp2p.New(
context.Background(), context.Background(),
libp2p.ListenAddrs(apiMAddr), libp2p.ListenAddrs(apiMAddr),
libp2p.PrivateNetwork(prot), libp2p.PrivateNetwork(secret),
) )
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)

View File

@ -11,10 +11,7 @@ import (
libp2p "github.com/libp2p/go-libp2p" libp2p "github.com/libp2p/go-libp2p"
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
peerstore "github.com/libp2p/go-libp2p-core/peerstore" peerstore "github.com/libp2p/go-libp2p-core/peerstore"
ipnet "github.com/libp2p/go-libp2p-core/pnet"
p2phttp "github.com/libp2p/go-libp2p-http" p2phttp "github.com/libp2p/go-libp2p-http"
pnet "github.com/libp2p/go-libp2p-pnet"
libp2pquic "github.com/libp2p/go-libp2p-quic-transport"
secio "github.com/libp2p/go-libp2p-secio" secio "github.com/libp2p/go-libp2p-secio"
libp2ptls "github.com/libp2p/go-libp2p-tls" libp2ptls "github.com/libp2p/go-libp2p-tls"
madns "github.com/multiformats/go-multiaddr-dns" madns "github.com/multiformats/go-multiaddr-dns"
@ -54,25 +51,18 @@ func (c *defaultClient) enableLibp2p() error {
return errors.New("APIAddr only includes a Peer ID") return errors.New("APIAddr only includes a Peer ID")
} }
var prot ipnet.Protector
if c.config.ProtectorKey != nil && len(c.config.ProtectorKey) > 0 { if c.config.ProtectorKey != nil && len(c.config.ProtectorKey) > 0 {
if len(c.config.ProtectorKey) != 32 { if len(c.config.ProtectorKey) != 32 {
return errors.New("length of ProtectorKey should be 32") return errors.New("length of ProtectorKey should be 32")
} }
var key [32]byte
copy(key[:], c.config.ProtectorKey)
prot, err = pnet.NewV1ProtectorFromBytes(&key)
if err != nil {
return err
}
} }
h, err := libp2p.New(c.ctx, h, err := libp2p.New(c.ctx,
libp2p.PrivateNetwork(prot), libp2p.PrivateNetwork(c.config.ProtectorKey),
libp2p.Security(libp2ptls.ID, libp2ptls.New), libp2p.Security(libp2ptls.ID, libp2ptls.New),
libp2p.Security(secio.ID, secio.New), libp2p.Security(secio.ID, secio.New),
libp2p.Transport(libp2pquic.NewTransport), // TODO: quic does not support private networks
//libp2p.Transport(libp2pquic.NewTransport),
libp2p.DefaultTransports, libp2p.DefaultTransports,
) )
if err != nil { if err != nil {

View File

@ -27,7 +27,7 @@ import (
"github.com/ipfs/ipfs-cluster/state" "github.com/ipfs/ipfs-cluster/state"
cid "github.com/ipfs/go-cid" cid "github.com/ipfs/go-cid"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
gopath "github.com/ipfs/go-path" gopath "github.com/ipfs/go-path"
libp2p "github.com/libp2p/go-libp2p" libp2p "github.com/libp2p/go-libp2p"
host "github.com/libp2p/go-libp2p-core/host" host "github.com/libp2p/go-libp2p-core/host"

View File

@ -20,7 +20,7 @@ import (
pb "github.com/ipfs/ipfs-cluster/api/pb" pb "github.com/ipfs/ipfs-cluster/api/pb"
cid "github.com/ipfs/go-cid" cid "github.com/ipfs/go-cid"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
protocol "github.com/libp2p/go-libp2p-core/protocol" protocol "github.com/libp2p/go-libp2p-core/protocol"
multiaddr "github.com/multiformats/go-multiaddr" multiaddr "github.com/multiformats/go-multiaddr"

View File

@ -182,13 +182,10 @@ func NewCluster(
// visible as peers without having to wait for them to send one. // visible as peers without having to wait for them to send one.
for _, p := range connectedPeers { for _, p := range connectedPeers {
if err := c.logPingMetric(ctx, p); err != nil { if err := c.logPingMetric(ctx, p); err != nil {
logger.Warning(err) logger.Warn(err)
} }
} }
// Bootstrap the DHT now that we possibly have some connections
c.dht.Bootstrap(c.ctx)
// After setupRPC components can do their tasks with a fully operative // After setupRPC components can do their tasks with a fully operative
// routed libp2p host with some connections and a working DHT (hopefully). // routed libp2p host with some connections and a working DHT (hopefully).
err = c.setupRPC() err = c.setupRPC()
@ -401,7 +398,7 @@ func (c *Cluster) alertsHandler() {
continue continue
} }
logger.Warningf("metric alert for %s: Peer: %s.", alrt.MetricName, alrt.Peer) logger.Warnf("metric alert for %s: Peer: %s.", alrt.MetricName, alrt.Peer)
if alrt.MetricName != pingMetricName { if alrt.MetricName != pingMetricName {
continue // only handle ping alerts continue // only handle ping alerts
} }
@ -413,18 +410,18 @@ func (c *Cluster) alertsHandler() {
cState, err := c.consensus.State(c.ctx) cState, err := c.consensus.State(c.ctx)
if err != nil { if err != nil {
logger.Warning(err) logger.Warn(err)
return return
} }
list, err := cState.List(c.ctx) list, err := cState.List(c.ctx)
if err != nil { if err != nil {
logger.Warning(err) logger.Warn(err)
return return
} }
for _, pin := range list { for _, pin := range list {
if len(pin.Allocations) == 1 && containsPeer(pin.Allocations, alrt.Peer) { if len(pin.Allocations) == 1 && containsPeer(pin.Allocations, alrt.Peer) {
logger.Warning("a pin with only one allocation cannot be repinned") logger.Warn("a pin with only one allocation cannot be repinned")
logger.Warning("to make repinning possible, pin with a replication factor of 2+") logger.Warn("to make repinning possible, pin with a replication factor of 2+")
continue continue
} }
if c.shouldPeerRepinCid(alrt.Peer, pin) { if c.shouldPeerRepinCid(alrt.Peer, pin) {
@ -517,18 +514,18 @@ func (c *Cluster) vacatePeer(ctx context.Context, p peer.ID) {
defer span.End() defer span.End()
if c.config.DisableRepinning { if c.config.DisableRepinning {
logger.Warningf("repinning is disabled. Will not re-allocate cids from %s", p.Pretty()) logger.Warnf("repinning is disabled. Will not re-allocate cids from %s", p.Pretty())
return return
} }
cState, err := c.consensus.State(ctx) cState, err := c.consensus.State(ctx)
if err != nil { if err != nil {
logger.Warning(err) logger.Warn(err)
return return
} }
list, err := cState.List(ctx) list, err := cState.List(ctx)
if err != nil { if err != nil {
logger.Warning(err) logger.Warn(err)
return return
} }
for _, pin := range list { for _, pin := range list {
@ -707,7 +704,7 @@ func (c *Cluster) Shutdown(ctx context.Context) error {
_, err := c.consensus.Peers(ctx) _, err := c.consensus.Peers(ctx)
if err == nil { if err == nil {
// best effort // best effort
logger.Warning("attempting to leave the cluster. This may take some seconds") logger.Warn("attempting to leave the cluster. This may take some seconds")
err := c.consensus.RmPeer(ctx, c.id) err := c.consensus.RmPeer(ctx, c.id)
if err != nil { if err != nil {
logger.Error("leaving cluster: " + err.Error()) logger.Error("leaving cluster: " + err.Error())
@ -950,18 +947,18 @@ func (c *Cluster) Join(ctx context.Context, addr ma.Multiaddr) error {
// we know that peer since we have metrics for it without // we know that peer since we have metrics for it without
// having to wait for the next metric round. // having to wait for the next metric round.
if err := c.logPingMetric(ctx, pid); err != nil { if err := c.logPingMetric(ctx, pid); err != nil {
logger.Warning(err) logger.Warn(err)
} }
// Broadcast our metrics to the world // Broadcast our metrics to the world
_, err = c.sendInformersMetrics(ctx) _, err = c.sendInformersMetrics(ctx)
if err != nil { if err != nil {
logger.Warning(err) logger.Warn(err)
} }
_, err = c.sendPingMetric(ctx) _, err = c.sendPingMetric(ctx)
if err != nil { if err != nil {
logger.Warning(err) logger.Warn(err)
} }
// We need to trigger a DHT bootstrap asap for this peer to not be // We need to trigger a DHT bootstrap asap for this peer to not be
@ -969,8 +966,18 @@ func (c *Cluster) Join(ctx context.Context, addr ma.Multiaddr) error {
// by triggering 1 round of bootstrap in the background. // by triggering 1 round of bootstrap in the background.
// Note that our regular bootstrap process is still running in the // Note that our regular bootstrap process is still running in the
// background since we created the cluster. // background since we created the cluster.
c.wg.Add(1)
go func() { go func() {
c.dht.BootstrapOnce(ctx, dht.DefaultBootstrapConfig) defer c.wg.Done()
select {
case err := <-c.dht.RefreshRoutingTable():
if err != nil {
logger.Error(err)
}
return
case <-c.ctx.Done():
return
}
}() }()
// ConnectSwarms in the background after a while, when we have likely // ConnectSwarms in the background after a while, when we have likely

View File

@ -1,6 +1,7 @@
package ipfscluster package ipfscluster
import ( import (
"crypto/rand"
"encoding/hex" "encoding/hex"
"encoding/json" "encoding/json"
"errors" "errors"
@ -14,7 +15,7 @@ import (
"github.com/ipfs/ipfs-cluster/config" "github.com/ipfs/ipfs-cluster/config"
ipfsconfig "github.com/ipfs/go-ipfs-config" ipfsconfig "github.com/ipfs/go-ipfs-config"
pnet "github.com/libp2p/go-libp2p-pnet" pnet "github.com/libp2p/go-libp2p-core/pnet"
ma "github.com/multiformats/go-multiaddr" ma "github.com/multiformats/go-multiaddr"
"github.com/kelseyhightower/envconfig" "github.com/kelseyhightower/envconfig"
@ -25,7 +26,8 @@ const configKey = "cluster"
// DefaultListenAddrs contains TCP and QUIC listen addresses. // DefaultListenAddrs contains TCP and QUIC listen addresses.
var DefaultListenAddrs = []string{ var DefaultListenAddrs = []string{
"/ip4/0.0.0.0/tcp/9096", "/ip4/0.0.0.0/tcp/9096",
"/ip4/0.0.0.0/udp/9096/quic", // TODO: QUIC does not support private networks
// "/ip4/0.0.0.0/udp/9096/quic",
} }
// Configuration defaults // Configuration defaults
@ -67,7 +69,7 @@ type Config struct {
// Cluster secret for private network. Peers will be in the same cluster if and // Cluster secret for private network. Peers will be in the same cluster if and
// only if they have the same ClusterSecret. The cluster secret must be exactly // only if they have the same ClusterSecret. The cluster secret must be exactly
// 64 characters and contain only hexadecimal characters (`[0-9a-f]`). // 64 characters and contain only hexadecimal characters (`[0-9a-f]`).
Secret []byte Secret pnet.PSK
// RPCPolicy defines access control to RPC endpoints. // RPCPolicy defines access control to RPC endpoints.
RPCPolicy map[string]RPCEndpointType RPCPolicy map[string]RPCEndpointType
@ -202,14 +204,16 @@ func (cfg *Config) ConfigKey() string {
func (cfg *Config) Default() error { func (cfg *Config) Default() error {
cfg.setDefaults() cfg.setDefaults()
// cluster secret clusterSecret := make([]byte, 32, 32)
clusterSecret, err := pnet.GenerateV1Bytes() n, err := rand.Read(clusterSecret)
if err != nil { if err != nil {
return err return err
} }
cfg.Secret = (*clusterSecret)[:] if n != 32 {
// -- return errors.New("did not generate 32-byte secret")
}
cfg.Secret = clusterSecret
return nil return nil
} }
@ -329,7 +333,7 @@ func isRPCPolicyValid(p map[string]RPCEndpointType) error {
} }
} }
if len(p) != total { if len(p) != total {
logger.Warning("defined RPC policy has more entries than needed") logger.Warn("defined RPC policy has more entries than needed")
} }
return nil return nil
} }
@ -535,7 +539,7 @@ func DecodeClusterSecret(hexSecret string) ([]byte, error) {
} }
switch secretLen := len(secret); secretLen { switch secretLen := len(secret); secretLen {
case 0: case 0:
logger.Warning("Cluster secret is empty, cluster will start on unprotected network.") logger.Warn("Cluster secret is empty, cluster will start on unprotected network.")
return nil, nil return nil, nil
case 32: case 32:
return secret, nil return secret, nil

View File

@ -14,9 +14,7 @@ import (
crypto "github.com/libp2p/go-libp2p-crypto" crypto "github.com/libp2p/go-libp2p-crypto"
host "github.com/libp2p/go-libp2p-host" host "github.com/libp2p/go-libp2p-host"
dht "github.com/libp2p/go-libp2p-kad-dht" dht "github.com/libp2p/go-libp2p-kad-dht"
pnet "github.com/libp2p/go-libp2p-pnet"
pubsub "github.com/libp2p/go-libp2p-pubsub" pubsub "github.com/libp2p/go-libp2p-pubsub"
libp2pquic "github.com/libp2p/go-libp2p-quic-transport"
secio "github.com/libp2p/go-libp2p-secio" secio "github.com/libp2p/go-libp2p-secio"
libp2ptls "github.com/libp2p/go-libp2p-tls" libp2ptls "github.com/libp2p/go-libp2p-tls"
routedhost "github.com/libp2p/go-libp2p/p2p/host/routed" routedhost "github.com/libp2p/go-libp2p/p2p/host/routed"
@ -66,14 +64,9 @@ func NewClusterHost(
libp2p.EnableAutoRelay(), libp2p.EnableAutoRelay(),
} }
prot, err := newProtector(cfg.Secret)
if err != nil {
return nil, nil, nil, err
}
h, err := newHost( h, err := newHost(
ctx, ctx,
prot, cfg.Secret,
ident.PrivateKey, ident.PrivateKey,
opts..., opts...,
) )
@ -88,7 +81,7 @@ func NewClusterHost(
} }
// needed for auto relay // needed for auto relay
_, err = autonat.NewAutoNATService(ctx, h, baseOpts(prot)...) _, err = autonat.NewAutoNATService(ctx, h, baseOpts(cfg.Secret)...)
if err != nil { if err != nil {
h.Close() h.Close()
return nil, nil, nil, err return nil, nil, nil, err
@ -99,11 +92,11 @@ func NewClusterHost(
// newHost creates a base cluster host without dht, pubsub, relay or nat etc. // newHost creates a base cluster host without dht, pubsub, relay or nat etc.
// mostly used for testing. // mostly used for testing.
func newHost(ctx context.Context, prot corepnet.Protector, priv crypto.PrivKey, opts ...libp2p.Option) (host.Host, error) { func newHost(ctx context.Context, psk corepnet.PSK, priv crypto.PrivKey, opts ...libp2p.Option) (host.Host, error) {
finalOpts := []libp2p.Option{ finalOpts := []libp2p.Option{
libp2p.Identity(priv), libp2p.Identity(priv),
} }
finalOpts = append(finalOpts, baseOpts(prot)...) finalOpts = append(finalOpts, baseOpts(psk)...)
finalOpts = append(finalOpts, opts...) finalOpts = append(finalOpts, opts...)
h, err := libp2p.New( h, err := libp2p.New(
@ -117,27 +110,17 @@ func newHost(ctx context.Context, prot corepnet.Protector, priv crypto.PrivKey,
return h, nil return h, nil
} }
func baseOpts(prot corepnet.Protector) []libp2p.Option { func baseOpts(psk corepnet.PSK) []libp2p.Option {
return []libp2p.Option{ return []libp2p.Option{
libp2p.PrivateNetwork(prot), libp2p.PrivateNetwork(psk),
libp2p.Security(libp2ptls.ID, libp2ptls.New), libp2p.Security(libp2ptls.ID, libp2ptls.New),
libp2p.Security(secio.ID, secio.New), libp2p.Security(secio.ID, secio.New),
libp2p.Transport(libp2pquic.NewTransport), // TODO: quic does not support private networks
//libp2p.Transport(libp2pquic.NewTransport),
libp2p.DefaultTransports, libp2p.DefaultTransports,
} }
} }
func newProtector(secret []byte) (corepnet.Protector, error) {
// Create protector if we have a secret.
if len(secret) == 0 {
return nil, nil
}
var key [32]byte
copy(key[:], secret)
return pnet.NewV1ProtectorFromBytes(&key)
}
func newDHT(ctx context.Context, h host.Host) (*dht.IpfsDHT, error) { func newDHT(ctx context.Context, h host.Host) (*dht.IpfsDHT, error) {
return dht.New(ctx, h) return dht.New(ctx, h)
} }

View File

@ -16,7 +16,7 @@ import (
"github.com/ipfs/ipfs-cluster/api/rest/client" "github.com/ipfs/ipfs-cluster/api/rest/client"
cid "github.com/ipfs/go-cid" cid "github.com/ipfs/go-cid"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
ma "github.com/multiformats/go-multiaddr" ma "github.com/multiformats/go-multiaddr"
@ -160,7 +160,7 @@ requires authorization. implies --https, which you can disable with --force-http
cfg.Timeout = time.Duration(c.Int("timeout")) * time.Second cfg.Timeout = time.Duration(c.Int("timeout")) * time.Second
if client.IsPeerAddress(cfg.APIAddr) && c.Bool("https") { if client.IsPeerAddress(cfg.APIAddr) && c.Bool("https") {
logger.Warning("Using libp2p-http. SSL flags will be ignored") logger.Warn("Using libp2p-http. SSL flags will be ignored")
} }
cfg.SSL = c.Bool("https") cfg.SSL = c.Bool("https")
@ -169,7 +169,7 @@ requires authorization. implies --https, which you can disable with --force-http
cfg.Username = user cfg.Username = user
cfg.Password = pass cfg.Password = pass
if user != "" && !cfg.SSL && !c.Bool("force-http") { if user != "" && !cfg.SSL && !c.Bool("force-http") {
logger.Warning("SSL automatically enabled with basic auth credentials. Set \"force-http\" to disable") logger.Warn("SSL automatically enabled with basic auth credentials. Set \"force-http\" to disable")
cfg.SSL = true cfg.SSL = true
} }

View File

@ -20,7 +20,7 @@ import (
ma "github.com/multiformats/go-multiaddr" ma "github.com/multiformats/go-multiaddr"
semver "github.com/blang/semver" semver "github.com/blang/semver"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
cli "github.com/urfave/cli" cli "github.com/urfave/cli"
) )

View File

@ -14,7 +14,7 @@ import (
"sync" "sync"
"time" "time"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
) )
var logger = logging.Logger("config") var logger = logging.Logger("config")

View File

@ -22,7 +22,7 @@ func (sv *Saver) NotifySave() {
select { select {
case sv.save <- struct{}{}: case sv.save <- struct{}{}:
default: default:
logger.Warning("configuration save channel full") logger.Warn("configuration save channel full")
} }
} }

View File

@ -59,7 +59,7 @@ func (c *Cluster) ConnectGraph() (api.ConnectGraph, error) {
cg.IDtoPeername[p] = pID.Peername cg.IDtoPeername[p] = pID.Peername
// IPFS connections // IPFS connections
if !selfConnection { if !selfConnection {
logger.Warningf("cluster peer %s not its own peer. No ipfs info ", p) logger.Warnf("cluster peer %s not its own peer. No ipfs info ", p)
continue continue
} }
c.recordIPFSLinks(&cg, pID) c.recordIPFSLinks(&cg, pID)
@ -89,7 +89,7 @@ func (c *Cluster) recordClusterLinks(cg *api.ConnectGraph, p string, peers []*ap
func (c *Cluster) recordIPFSLinks(cg *api.ConnectGraph, pID *api.ID) { func (c *Cluster) recordIPFSLinks(cg *api.ConnectGraph, pID *api.ID) {
ipfsID := pID.IPFS.ID ipfsID := pID.IPFS.ID
if pID.IPFS.Error != "" { // Only setting ipfs connections when no error occurs if pID.IPFS.Error != "" { // Only setting ipfs connections when no error occurs
logger.Warningf("ipfs id: %s has error: %s. Skipping swarm connections", ipfsID.Pretty(), pID.IPFS.Error) logger.Warnf("ipfs id: %s has error: %s. Skipping swarm connections", ipfsID.Pretty(), pID.IPFS.Error)
return return
} }
@ -97,7 +97,7 @@ func (c *Cluster) recordIPFSLinks(cg *api.ConnectGraph, pID *api.ID) {
ipfsPid := peer.IDB58Encode(ipfsID) ipfsPid := peer.IDB58Encode(ipfsID)
if _, ok := cg.IPFSLinks[pid]; ok { if _, ok := cg.IPFSLinks[pid]; ok {
logger.Warningf("ipfs id: %s already recorded, one ipfs daemon in use by multiple cluster peers", ipfsID.Pretty()) logger.Warnf("ipfs id: %s already recorded, one ipfs daemon in use by multiple cluster peers", ipfsID.Pretty())
} }
cg.ClustertoIPFS[pid] = ipfsID cg.ClustertoIPFS[pid] = ipfsID
cg.IPFSLinks[ipfsPid] = make([]peer.ID, 0) cg.IPFSLinks[ipfsPid] = make([]peer.ID, 0)

View File

@ -7,6 +7,7 @@ import (
"sync" "sync"
"time" "time"
"github.com/ipfs/go-cid"
"github.com/ipfs/ipfs-cluster/api" "github.com/ipfs/ipfs-cluster/api"
"github.com/ipfs/ipfs-cluster/pstoremgr" "github.com/ipfs/ipfs-cluster/pstoremgr"
"github.com/ipfs/ipfs-cluster/state" "github.com/ipfs/ipfs-cluster/state"
@ -17,7 +18,7 @@ import (
query "github.com/ipfs/go-datastore/query" query "github.com/ipfs/go-datastore/query"
crdt "github.com/ipfs/go-ds-crdt" crdt "github.com/ipfs/go-ds-crdt"
dshelp "github.com/ipfs/go-ipfs-ds-help" dshelp "github.com/ipfs/go-ipfs-ds-help"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
host "github.com/libp2p/go-libp2p-core/host" host "github.com/libp2p/go-libp2p-core/host"
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
peerstore "github.com/libp2p/go-libp2p-core/peerstore" peerstore "github.com/libp2p/go-libp2p-core/peerstore"
@ -212,11 +213,17 @@ func (css *Consensus) setup() {
ctx, span := trace.StartSpan(css.ctx, "crdt/DeleteHook") ctx, span := trace.StartSpan(css.ctx, "crdt/DeleteHook")
defer span.End() defer span.End()
c, err := dshelp.DsKeyToCid(k) kb, err := dshelp.BinaryFromDsKey(k)
if err != nil { if err != nil {
logger.Error(err, k) logger.Error(err, k)
return return
} }
c, err := cid.Cast(kb)
if err != nil {
logger.Error(err, k)
return
}
pin := api.PinCid(c) pin := api.PinCid(c)
err = css.rpcClient.CallContext( err = css.rpcClient.CallContext(

View File

@ -15,6 +15,7 @@ import (
host "github.com/libp2p/go-libp2p-core/host" host "github.com/libp2p/go-libp2p-core/host"
peerstore "github.com/libp2p/go-libp2p-core/peerstore" peerstore "github.com/libp2p/go-libp2p-core/peerstore"
dht "github.com/libp2p/go-libp2p-kad-dht" dht "github.com/libp2p/go-libp2p-kad-dht"
dhtopts "github.com/libp2p/go-libp2p-kad-dht/opts"
pubsub "github.com/libp2p/go-libp2p-pubsub" pubsub "github.com/libp2p/go-libp2p-pubsub"
routedhost "github.com/libp2p/go-libp2p/p2p/host/routed" routedhost "github.com/libp2p/go-libp2p/p2p/host/routed"
) )
@ -40,19 +41,10 @@ func makeTestingHost(t *testing.T) (host.Host, *pubsub.PubSub, *dht.IpfsDHT) {
t.Fatal(err) t.Fatal(err)
} }
idht, err := dht.New(ctx, h) idht, err := dht.New(ctx, h,
if err != nil { dhtopts.RoutingTableRefreshPeriod(200*time.Millisecond),
h.Close() dhtopts.RoutingTableRefreshQueryTimeout(100*time.Millisecond),
t.Fatal(err) )
}
btstrCfg := dht.BootstrapConfig{
Queries: 1,
Period: 200 * time.Millisecond,
Timeout: 100 * time.Millisecond,
}
err = idht.BootstrapWithConfig(ctx, btstrCfg)
if err != nil { if err != nil {
h.Close() h.Close()
t.Fatal(err) t.Fatal(err)

View File

@ -199,7 +199,7 @@ func (cfg *Config) applyJSONConfig(jcfg *jsonConfig) error {
parseDuration := func(txt string) time.Duration { parseDuration := func(txt string) time.Duration {
d, _ := time.ParseDuration(txt) d, _ := time.ParseDuration(txt)
if txt != "" && d == 0 { if txt != "" && d == 0 {
logger.Warningf("%s is not a valid duration. Default will be used", txt) logger.Warnf("%s is not a valid duration. Default will be used", txt)
} }
return d return d
} }

View File

@ -15,7 +15,7 @@ import (
"github.com/ipfs/ipfs-cluster/state/dsstate" "github.com/ipfs/ipfs-cluster/state/dsstate"
ds "github.com/ipfs/go-datastore" ds "github.com/ipfs/go-datastore"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
consensus "github.com/libp2p/go-libp2p-consensus" consensus "github.com/libp2p/go-libp2p-consensus"
host "github.com/libp2p/go-libp2p-core/host" host "github.com/libp2p/go-libp2p-core/host"
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
@ -263,7 +263,7 @@ func (cc *Consensus) redirectToLeader(method string, arg interface{}) (bool, err
// No leader, wait for one // No leader, wait for one
if err != nil { if err != nil {
logger.Warning("there seems to be no leader. Waiting for one") logger.Warn("there seems to be no leader. Waiting for one")
rctx, cancel := context.WithTimeout( rctx, cancel := context.WithTimeout(
ctx, ctx,
cc.config.WaitForLeaderTimeout, cc.config.WaitForLeaderTimeout,

View File

@ -8,7 +8,7 @@ import (
"time" "time"
hclog "github.com/hashicorp/go-hclog" hclog "github.com/hashicorp/go-hclog"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
) )
const ( const (
@ -52,6 +52,21 @@ func (log *hcLogToLogger) format(msg string, args []interface{}) string {
return name + msg + argstr return name + msg + argstr
} }
func (log *hcLogToLogger) Log(level hclog.Level, msg string, args ...interface{}) {
switch level {
case hclog.Trace, hclog.Debug:
log.Debug(msg, args)
case hclog.NoLevel, hclog.Info:
log.Info(msg, args)
case hclog.Warn:
log.Warn(msg, args)
case hclog.Error:
log.Error(msg, args)
default:
log.Warn(msg, args)
}
}
func (log *hcLogToLogger) Trace(msg string, args ...interface{}) { func (log *hcLogToLogger) Trace(msg string, args ...interface{}) {
raftLogger.Debug(log.format(msg, args)) raftLogger.Debug(log.format(msg, args))
} }
@ -65,7 +80,7 @@ func (log *hcLogToLogger) Info(msg string, args ...interface{}) {
} }
func (log *hcLogToLogger) Warn(msg string, args ...interface{}) { func (log *hcLogToLogger) Warn(msg string, args ...interface{}) {
raftLogger.Warning(log.format(msg, args)) raftLogger.Warn(log.format(msg, args))
} }
func (log *hcLogToLogger) Error(msg string, args ...interface{}) { func (log *hcLogToLogger) Error(msg string, args ...interface{}) {
@ -134,7 +149,7 @@ type logForwarder struct {
var raftStdLogger = log.New(&logForwarder{}, "", 0) var raftStdLogger = log.New(&logForwarder{}, "", 0)
// Write forwards to our go-log logger. // Write forwards to our go-log/v2 logger.
// According to https://golang.org/pkg/log/#Logger.Output // According to https://golang.org/pkg/log/#Logger.Output
// it is called per line. // it is called per line.
func (fw *logForwarder) Write(p []byte) (n int, e error) { func (fw *logForwarder) Write(p []byte) (n int, e error) {
@ -206,7 +221,7 @@ func (fw *logForwarder) log(t int, msg string) {
case info: case info:
raftLogger.Info(msg) raftLogger.Info(msg)
case warn: case warn:
raftLogger.Warning(msg) raftLogger.Warn(msg)
case err: case err:
raftLogger.Error(msg) raftLogger.Error(msg)
default: default:

View File

@ -393,7 +393,7 @@ func (rw *raftWrapper) snapshotOnShutdown() error {
err := rw.WaitForUpdates(ctx) err := rw.WaitForUpdates(ctx)
cancel() cancel()
if err != nil { if err != nil {
logger.Warning("timed out waiting for state updates before shutdown. Snapshotting may fail") logger.Warn("timed out waiting for state updates before shutdown. Snapshotting may fail")
done = true // let's not wait for updates again done = true // let's not wait for updates again
} }
@ -408,7 +408,7 @@ func (rw *raftWrapper) snapshotOnShutdown() error {
if done { if done {
break break
} }
logger.Warningf("retrying to snapshot (%d/%d)...", i+1, maxShutdownSnapshotRetries) logger.Warnf("retrying to snapshot (%d/%d)...", i+1, maxShutdownSnapshotRetries)
} }
return err return err
} }
@ -654,9 +654,9 @@ func CleanupRaft(cfg *Config) error {
dbh := newDataBackupHelper(dataFolder, keep) dbh := newDataBackupHelper(dataFolder, keep)
err = dbh.makeBackup() err = dbh.makeBackup()
if err != nil { if err != nil {
logger.Warning(err) logger.Warn(err)
logger.Warning("the state could not be cleaned properly") logger.Warn("the state could not be cleaned properly")
logger.Warning("manual intervention may be needed before starting cluster again") logger.Warn("manual intervention may be needed before starting cluster again")
} }
return nil return nil
} }

4
go.mod
View File

@ -31,7 +31,7 @@ require (
github.com/ipfs/go-ipfs-util v0.0.1 github.com/ipfs/go-ipfs-util v0.0.1
github.com/ipfs/go-ipld-cbor v0.0.4 github.com/ipfs/go-ipld-cbor v0.0.4
github.com/ipfs/go-ipld-format v0.0.2 github.com/ipfs/go-ipld-format v0.0.2
github.com/ipfs/go-log v1.0.2 github.com/ipfs/go-log/v2 v2.0.2
github.com/ipfs/go-merkledag v0.3.1 github.com/ipfs/go-merkledag v0.3.1
github.com/ipfs/go-mfs v0.1.1 github.com/ipfs/go-mfs v0.1.1
github.com/ipfs/go-path v0.0.7 github.com/ipfs/go-path v0.0.7
@ -78,4 +78,4 @@ require (
gonum.org/v1/plot v0.0.0-20190615073203-9aa86143727f gonum.org/v1/plot v0.0.0-20190615073203-9aa86143727f
) )
go 1.13 go 1.14

View File

@ -8,7 +8,7 @@ import (
"github.com/ipfs/ipfs-cluster/api" "github.com/ipfs/ipfs-cluster/api"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
rpc "github.com/libp2p/go-libp2p-gorpc" rpc "github.com/libp2p/go-libp2p-gorpc"
"go.opencensus.io/trace" "go.opencensus.io/trace"

View File

@ -40,6 +40,7 @@ import (
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
peerstore "github.com/libp2p/go-libp2p-core/peerstore" peerstore "github.com/libp2p/go-libp2p-core/peerstore"
dht "github.com/libp2p/go-libp2p-kad-dht" dht "github.com/libp2p/go-libp2p-kad-dht"
dhtopts "github.com/libp2p/go-libp2p-kad-dht/opts"
pubsub "github.com/libp2p/go-libp2p-pubsub" pubsub "github.com/libp2p/go-libp2p-pubsub"
ma "github.com/multiformats/go-multiaddr" ma "github.com/multiformats/go-multiaddr"
) )
@ -288,14 +289,15 @@ func createHosts(t *testing.T, clusterSecret []byte, nClusters int) ([]host.Host
dhts := make([]*dht.IpfsDHT, nClusters, nClusters) dhts := make([]*dht.IpfsDHT, nClusters, nClusters)
tcpaddr, _ := ma.NewMultiaddr("/ip4/127.0.0.1/tcp/0") tcpaddr, _ := ma.NewMultiaddr("/ip4/127.0.0.1/tcp/0")
quicAddr, _ := ma.NewMultiaddr("/ip4/127.0.0.1/udp/0/quic") //quicAddr, _ := ma.NewMultiaddr("/ip4/127.0.0.1/udp/0/quic")
for i := range hosts { for i := range hosts {
priv, _, err := crypto.GenerateKeyPair(crypto.RSA, 2048) priv, _, err := crypto.GenerateKeyPair(crypto.RSA, 2048)
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
h, p, d := createHost(t, priv, clusterSecret, []ma.Multiaddr{quicAddr, tcpaddr}) //h, p, d := createHost(t, priv, clusterSecret, []ma.Multiaddr{quicAddr, tcpaddr})
h, p, d := createHost(t, priv, clusterSecret, []ma.Multiaddr{tcpaddr})
hosts[i] = h hosts[i] = h
dhts[i] = d dhts[i] = d
pubsubs[i] = p pubsubs[i] = p
@ -306,12 +308,8 @@ func createHosts(t *testing.T, clusterSecret []byte, nClusters int) ([]host.Host
func createHost(t *testing.T, priv crypto.PrivKey, clusterSecret []byte, listen []ma.Multiaddr) (host.Host, *pubsub.PubSub, *dht.IpfsDHT) { func createHost(t *testing.T, priv crypto.PrivKey, clusterSecret []byte, listen []ma.Multiaddr) (host.Host, *pubsub.PubSub, *dht.IpfsDHT) {
ctx := context.Background() ctx := context.Background()
prot, err := newProtector(clusterSecret)
if err != nil {
t.Fatal(err)
}
h, err := newHost(ctx, prot, priv, libp2p.ListenAddrs(listen...)) h, err := newHost(ctx, clusterSecret, priv, libp2p.ListenAddrs(listen...))
if err != nil { if err != nil {
t.Fatal(err) t.Fatal(err)
} }
@ -333,12 +331,10 @@ func createHost(t *testing.T, priv crypto.PrivKey, clusterSecret []byte, listen
} }
func newTestDHT(ctx context.Context, h host.Host) (*dht.IpfsDHT, error) { func newTestDHT(ctx context.Context, h host.Host) (*dht.IpfsDHT, error) {
return newDHT(ctx, h) return dht.New(ctx, h,
// TODO: when new dht options are released dhtopts.RoutingTableRefreshPeriod(600*time.Millisecond),
// return dht.New(ctx, h, dhtopts.Bootstrap(dhtopts.BootstrapConfig{ dhtopts.RoutingTableRefreshQueryTimeout(300*time.Millisecond),
// Timeout: 300 * time.Millisecond, )
// SelfQueryInterval: 300 * time.Millisecond,
// }))
} }
func createClusters(t *testing.T) ([]*Cluster, []*test.IpfsMock) { func createClusters(t *testing.T) ([]*Cluster, []*test.IpfsMock) {
@ -398,17 +394,6 @@ func createClusters(t *testing.T) ([]*Cluster, []*test.IpfsMock) {
} }
} }
// // 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) waitForLeader(t, clusters)
waitForClustersHealthy(t, clusters) waitForClustersHealthy(t, clusters)

View File

@ -21,7 +21,7 @@ import (
cid "github.com/ipfs/go-cid" cid "github.com/ipfs/go-cid"
files "github.com/ipfs/go-ipfs-files" files "github.com/ipfs/go-ipfs-files"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
gopath "github.com/ipfs/go-path" gopath "github.com/ipfs/go-path"
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
rpc "github.com/libp2p/go-libp2p-gorpc" rpc "github.com/libp2p/go-libp2p-gorpc"

View File

@ -7,7 +7,7 @@ import (
"testing" "testing"
"time" "time"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
ma "github.com/multiformats/go-multiaddr" ma "github.com/multiformats/go-multiaddr"
merkledag "github.com/ipfs/go-merkledag" merkledag "github.com/ipfs/go-merkledag"

View File

@ -1,26 +1,11 @@
package ipfscluster package ipfscluster
import ( import (
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
) )
var logger = logging.Logger("cluster") 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 // LoggingFacilities provides a list of logging identifiers
// used by cluster and their default logging level. // used by cluster and their default logging level.
var LoggingFacilities = map[string]string{ var LoggingFacilities = map[string]string{

View File

@ -9,7 +9,7 @@ import (
"sync" "sync"
"time" "time"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
"github.com/ipfs/ipfs-cluster/api" "github.com/ipfs/ipfs-cluster/api"
) )

View File

@ -11,7 +11,7 @@ import (
"github.com/ipfs/ipfs-cluster/api" "github.com/ipfs/ipfs-cluster/api"
"github.com/ipfs/ipfs-cluster/monitor/metrics" "github.com/ipfs/ipfs-cluster/monitor/metrics"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
rpc "github.com/libp2p/go-libp2p-gorpc" rpc "github.com/libp2p/go-libp2p-gorpc"
pubsub "github.com/libp2p/go-libp2p-pubsub" pubsub "github.com/libp2p/go-libp2p-pubsub"
@ -160,7 +160,7 @@ func (mon *Monitor) Shutdown(ctx context.Context) error {
defer mon.shutdownLock.Unlock() defer mon.shutdownLock.Unlock()
if mon.shutdown { if mon.shutdown {
logger.Warning("Monitor already shut down") logger.Warn("Monitor already shut down")
return nil return nil
} }
@ -190,7 +190,7 @@ func (mon *Monitor) PublishMetric(ctx context.Context, m *api.Metric) error {
defer span.End() defer span.End()
if m.Discard() { if m.Discard() {
logger.Warningf("discarding invalid metric: %+v", m) logger.Warnf("discarding invalid metric: %+v", m)
return nil return nil
} }

View File

@ -5,7 +5,7 @@ import (
"go.opencensus.io/stats/view" "go.opencensus.io/stats/view"
"go.opencensus.io/tag" "go.opencensus.io/tag"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
) )
var logger = logging.Logger("observations") var logger = logging.Logger("observations")

View File

@ -15,7 +15,6 @@ import (
cid "github.com/ipfs/go-cid" cid "github.com/ipfs/go-cid"
host "github.com/libp2p/go-libp2p-core/host" host "github.com/libp2p/go-libp2p-core/host"
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
dht "github.com/libp2p/go-libp2p-kad-dht"
ma "github.com/multiformats/go-multiaddr" ma "github.com/multiformats/go-multiaddr"
) )
@ -46,7 +45,7 @@ func peerManagerClusters(t *testing.T) ([]*Cluster, []*test.IpfsMock, host.Host)
cfg.ListenAddr = []ma.Multiaddr{listen} cfg.ListenAddr = []ma.Multiaddr{listen}
cfg.Secret = testingClusterSecret cfg.Secret = testingClusterSecret
h, _, idht := createHost(t, ident.PrivateKey, testingClusterSecret, cfg.ListenAddr) h, _, _ := createHost(t, ident.PrivateKey, testingClusterSecret, cfg.ListenAddr)
// Connect host to all peers. This will allow that they can discover // Connect host to all peers. This will allow that they can discover
// each others via DHT. // each others via DHT.
@ -62,14 +61,6 @@ func peerManagerClusters(t *testing.T) ([]*Cluster, []*test.IpfsMock, host.Host)
t.Fatal(err) t.Fatal(err)
} }
} }
ctx := context.Background()
dhtCfg := dht.BootstrapConfig{
Queries: 1,
Period: 600 * time.Millisecond,
Timeout: 300 * time.Millisecond,
}
idht.BootstrapWithConfig(ctx, dhtCfg)
return cls, mocks, h return cls, mocks, h
} }
@ -208,8 +199,10 @@ func TestClustersPeerAddInUnhealthyCluster(t *testing.T) {
_, err := clusters[0].PeerAdd(ctx, clusters[1].id) _, err := clusters[0].PeerAdd(ctx, clusters[1].id)
ttlDelay() ttlDelay()
ids := clusters[1].Peers(ctx) ids := clusters[1].Peers(ctx)
if len(ids) != 2 { // raft will have only 2 peers
t.Error("expected 2 peers") // crdt will have all peers autodiscovered by now
if len(ids) < 2 {
t.Error("expected at least 2 peers")
} }
// Now we shutdown the one member of the running cluster // Now we shutdown the one member of the running cluster
@ -245,8 +238,8 @@ func TestClustersPeerAddInUnhealthyCluster(t *testing.T) {
ttlDelay() ttlDelay()
ids = clusters[0].Peers(ctx) ids = clusters[0].Peers(ctx)
if len(ids) != 2 { if len(ids) < 2 {
t.Error("cluster should have 2 peers after removing and adding 1") t.Error("cluster should have at least 2 peers after removing and adding 1")
} }
default: default:
t.Fatal("bad consensus") t.Fatal("bad consensus")

View File

@ -15,7 +15,7 @@ import (
"github.com/ipfs/ipfs-cluster/api" "github.com/ipfs/ipfs-cluster/api"
cid "github.com/ipfs/go-cid" cid "github.com/ipfs/go-cid"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
"go.opencensus.io/trace" "go.opencensus.io/trace"

View File

@ -14,7 +14,7 @@ import (
"github.com/ipfs/ipfs-cluster/state" "github.com/ipfs/ipfs-cluster/state"
cid "github.com/ipfs/go-cid" cid "github.com/ipfs/go-cid"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
rpc "github.com/libp2p/go-libp2p-gorpc" rpc "github.com/libp2p/go-libp2p-gorpc"

View File

@ -54,8 +54,10 @@ func TestSimplePNet(t *testing.T) {
if len(clusters[0].Peers(ctx)) != len(clusters[1].Peers(ctx)) { if len(clusters[0].Peers(ctx)) != len(clusters[1].Peers(ctx)) {
t.Fatal("Expected same number of peers") t.Fatal("Expected same number of peers")
} }
if len(clusters[0].Peers(ctx)) != 2 { if len(clusters[0].Peers(ctx)) < 2 {
t.Fatal("Expected 2 peers") // crdt mode has auto discovered all peers at this point.
// Raft mode has 2 peers only.
t.Fatal("Expected at least 2 peers")
} }
} }

View File

@ -14,7 +14,7 @@ import (
"sync" "sync"
"time" "time"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
host "github.com/libp2p/go-libp2p-core/host" host "github.com/libp2p/go-libp2p-core/host"
net "github.com/libp2p/go-libp2p-core/network" net "github.com/libp2p/go-libp2p-core/network"
peer "github.com/libp2p/go-libp2p-core/peer" peer "github.com/libp2p/go-libp2p-core/peer"
@ -265,13 +265,13 @@ func (pm *Manager) SavePeerstore(pinfos []peer.AddrInfo) error {
for _, pinfo := range pinfos { for _, pinfo := range pinfos {
if len(pinfo.Addrs) == 0 { if len(pinfo.Addrs) == 0 {
logger.Warning("address info does not have any multiaddresses") logger.Warn("address info does not have any multiaddresses")
continue continue
} }
addrs, err := peer.AddrInfoToP2pAddrs(&pinfo) addrs, err := peer.AddrInfoToP2pAddrs(&pinfo)
if err != nil { if err != nil {
logger.Warning(err) logger.Warn(err)
continue continue
} }
for _, a := range addrs { for _, a := range addrs {

View File

@ -13,7 +13,7 @@ import (
ds "github.com/ipfs/go-datastore" ds "github.com/ipfs/go-datastore"
query "github.com/ipfs/go-datastore/query" query "github.com/ipfs/go-datastore/query"
dshelp "github.com/ipfs/go-ipfs-ds-help" dshelp "github.com/ipfs/go-ipfs-ds-help"
logging "github.com/ipfs/go-log" logging "github.com/ipfs/go-log/v2"
codec "github.com/ugorji/go/codec" codec "github.com/ugorji/go/codec"
trace "go.opencensus.io/trace" trace "go.opencensus.io/trace"
@ -148,7 +148,7 @@ func (st *State) List(ctx context.Context) ([]*api.Pin, error) {
k := ds.NewKey(r.Key) k := ds.NewKey(r.Key)
ci, err := st.unkey(k) ci, err := st.unkey(k)
if err != nil { if err != nil {
logger.Warning("bad key (ignoring). key: ", k, "error: ", err) logger.Warn("bad key (ignoring). key: ", k, "error: ", err)
continue continue
} }
@ -236,15 +236,29 @@ func (st *State) Unmarshal(r io.Reader) error {
return nil return nil
} }
// convert Cid to /namespace/cidKey // used to be on go-ipfs-ds-help
func cidToDsKey(c cid.Cid) ds.Key {
return dshelp.NewKeyFromBinary(c.Bytes())
}
// used to be on go-ipfs-ds-help
func dsKeyToCid(k ds.Key) (cid.Cid, error) {
kb, err := dshelp.BinaryFromDsKey(k)
if err != nil {
return cid.Undef, err
}
return cid.Cast(kb)
}
// convert Cid to /namespace/cid1Key
func (st *State) key(c cid.Cid) ds.Key { func (st *State) key(c cid.Cid) ds.Key {
k := dshelp.CidToDsKey(c) k := cidToDsKey(c)
return st.namespace.Child(k) return st.namespace.Child(k)
} }
// convert /namespace/cidKey to Cid // convert /namespace/cidKey to Cid
func (st *State) unkey(k ds.Key) (cid.Cid, error) { func (st *State) unkey(k ds.Key) (cid.Cid, error) {
return dshelp.DsKeyToCid(ds.NewKey(k.BaseNamespace())) return dsKeyToCid(ds.NewKey(k.BaseNamespace()))
} }
// this decides how a Pin object is serialized to be stored in the // this decides how a Pin object is serialized to be stored in the