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
# - osx
go:
- '1.13.x'
- '1.14.x'
services:
- 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>
# 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>
# 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>
# This build state just builds the cluster binaries

View File

@ -14,7 +14,7 @@ import (
cid "github.com/ipfs/go-cid"
files "github.com/ipfs/go-ipfs-files"
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"
multihash "github.com/multiformats/go-multihash"
)

View File

@ -14,7 +14,7 @@ import (
"github.com/ipfs/ipfs-cluster/api"
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"
)

View File

@ -16,7 +16,7 @@ import (
files "github.com/ipfs/go-ipfs-files"
posinfo "github.com/ipfs/go-ipfs-posinfo"
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"
mfs "github.com/ipfs/go-mfs"
unixfs "github.com/ipfs/go-unixfs"

View File

@ -16,7 +16,7 @@ import (
humanize "github.com/dustin/go-humanize"
cid "github.com/ipfs/go-cid"
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"
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) {
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)

View File

@ -12,7 +12,7 @@ import (
"github.com/ipfs/ipfs-cluster/test"
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"
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 {
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

View File

@ -11,7 +11,7 @@ import (
cid "github.com/ipfs/go-cid"
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"
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 {
size, err := n.Size()
if err != nil {
logger.Warning(err)
logger.Warn(err)
}
return &api.NodeWithMeta{

View File

@ -11,7 +11,7 @@ import (
"github.com/ipfs/ipfs-cluster/api"
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"
rpc "github.com/libp2p/go-libp2p-gorpc"
)

View File

@ -11,7 +11,7 @@ import (
"github.com/ipfs/ipfs-cluster/api"
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"
rpc "github.com/libp2p/go-libp2p-gorpc"
)

View File

@ -22,7 +22,7 @@ import (
handlers "github.com/gorilla/handlers"
mux "github.com/gorilla/mux"
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"
peer "github.com/libp2p/go-libp2p-core/peer"
rpc "github.com/libp2p/go-libp2p-gorpc"
@ -560,7 +560,7 @@ func (proxy *Server) addHandler(w http.ResponseWriter, r *http.Request) {
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{} {
r := &ipfsAddResp{

View File

@ -19,7 +19,7 @@ import (
"github.com/ipfs/ipfs-cluster/api"
"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"
)

View File

@ -15,9 +15,10 @@ import (
cid "github.com/ipfs/go-cid"
shell "github.com/ipfs/go-ipfs-api"
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"
peer "github.com/libp2p/go-libp2p-core/peer"
pnet "github.com/libp2p/go-libp2p-core/pnet"
ma "github.com/multiformats/go-multiaddr"
madns "github.com/multiformats/go-multiaddr-dns"
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),
// then we need to provide the key. If the peer is the cluster peer,
// this corresponds to the cluster secret.
ProtectorKey []byte
ProtectorKey pnet.PSK
// ProxyAddr is used to obtain a go-ipfs-api Shell instance pointing
// 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"
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"
)
@ -23,16 +23,12 @@ func testAPI(t *testing.T) *rest.API {
cfg := &rest.Config{}
cfg.Default()
cfg.HTTPListenAddr = []ma.Multiaddr{apiMAddr}
var secret [32]byte
prot, err := pnet.NewV1ProtectorFromBytes(&secret)
if err != nil {
t.Fatal(err)
}
secret := make(pnet.PSK, 32, 32)
h, err := libp2p.New(
context.Background(),
libp2p.ListenAddrs(apiMAddr),
libp2p.PrivateNetwork(prot),
libp2p.PrivateNetwork(secret),
)
if err != nil {
t.Fatal(err)

View File

@ -11,10 +11,7 @@ import (
libp2p "github.com/libp2p/go-libp2p"
peer "github.com/libp2p/go-libp2p-core/peer"
peerstore "github.com/libp2p/go-libp2p-core/peerstore"
ipnet "github.com/libp2p/go-libp2p-core/pnet"
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"
libp2ptls "github.com/libp2p/go-libp2p-tls"
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")
}
var prot ipnet.Protector
if c.config.ProtectorKey != nil && len(c.config.ProtectorKey) > 0 {
if len(c.config.ProtectorKey) != 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,
libp2p.PrivateNetwork(prot),
libp2p.PrivateNetwork(c.config.ProtectorKey),
libp2p.Security(libp2ptls.ID, libp2ptls.New),
libp2p.Security(secio.ID, secio.New),
libp2p.Transport(libp2pquic.NewTransport),
// TODO: quic does not support private networks
//libp2p.Transport(libp2pquic.NewTransport),
libp2p.DefaultTransports,
)
if err != nil {

View File

@ -27,7 +27,7 @@ import (
"github.com/ipfs/ipfs-cluster/state"
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"
libp2p "github.com/libp2p/go-libp2p"
host "github.com/libp2p/go-libp2p-core/host"

View File

@ -20,7 +20,7 @@ import (
pb "github.com/ipfs/ipfs-cluster/api/pb"
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"
protocol "github.com/libp2p/go-libp2p-core/protocol"
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.
for _, p := range connectedPeers {
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
// routed libp2p host with some connections and a working DHT (hopefully).
err = c.setupRPC()
@ -401,7 +398,7 @@ func (c *Cluster) alertsHandler() {
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 {
continue // only handle ping alerts
}
@ -413,18 +410,18 @@ func (c *Cluster) alertsHandler() {
cState, err := c.consensus.State(c.ctx)
if err != nil {
logger.Warning(err)
logger.Warn(err)
return
}
list, err := cState.List(c.ctx)
if err != nil {
logger.Warning(err)
logger.Warn(err)
return
}
for _, pin := range list {
if len(pin.Allocations) == 1 && containsPeer(pin.Allocations, alrt.Peer) {
logger.Warning("a pin with only one allocation cannot be repinned")
logger.Warning("to make repinning possible, pin with a replication factor of 2+")
logger.Warn("a pin with only one allocation cannot be repinned")
logger.Warn("to make repinning possible, pin with a replication factor of 2+")
continue
}
if c.shouldPeerRepinCid(alrt.Peer, pin) {
@ -517,18 +514,18 @@ func (c *Cluster) vacatePeer(ctx context.Context, p peer.ID) {
defer span.End()
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
}
cState, err := c.consensus.State(ctx)
if err != nil {
logger.Warning(err)
logger.Warn(err)
return
}
list, err := cState.List(ctx)
if err != nil {
logger.Warning(err)
logger.Warn(err)
return
}
for _, pin := range list {
@ -707,7 +704,7 @@ func (c *Cluster) Shutdown(ctx context.Context) error {
_, err := c.consensus.Peers(ctx)
if err == nil {
// 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)
if err != nil {
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
// having to wait for the next metric round.
if err := c.logPingMetric(ctx, pid); err != nil {
logger.Warning(err)
logger.Warn(err)
}
// Broadcast our metrics to the world
_, err = c.sendInformersMetrics(ctx)
if err != nil {
logger.Warning(err)
logger.Warn(err)
}
_, err = c.sendPingMetric(ctx)
if err != nil {
logger.Warning(err)
logger.Warn(err)
}
// 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.
// Note that our regular bootstrap process is still running in the
// background since we created the cluster.
c.wg.Add(1)
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

View File

@ -1,6 +1,7 @@
package ipfscluster
import (
"crypto/rand"
"encoding/hex"
"encoding/json"
"errors"
@ -14,7 +15,7 @@ import (
"github.com/ipfs/ipfs-cluster/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"
"github.com/kelseyhightower/envconfig"
@ -25,7 +26,8 @@ const configKey = "cluster"
// DefaultListenAddrs contains TCP and QUIC listen addresses.
var DefaultListenAddrs = []string{
"/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
@ -67,7 +69,7 @@ type Config struct {
// 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
// 64 characters and contain only hexadecimal characters (`[0-9a-f]`).
Secret []byte
Secret pnet.PSK
// RPCPolicy defines access control to RPC endpoints.
RPCPolicy map[string]RPCEndpointType
@ -202,14 +204,16 @@ func (cfg *Config) ConfigKey() string {
func (cfg *Config) Default() error {
cfg.setDefaults()
// cluster secret
clusterSecret, err := pnet.GenerateV1Bytes()
clusterSecret := make([]byte, 32, 32)
n, err := rand.Read(clusterSecret)
if err != nil {
return err
}
cfg.Secret = (*clusterSecret)[:]
// --
if n != 32 {
return errors.New("did not generate 32-byte secret")
}
cfg.Secret = clusterSecret
return nil
}
@ -329,7 +333,7 @@ func isRPCPolicyValid(p map[string]RPCEndpointType) error {
}
}
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
}
@ -535,7 +539,7 @@ func DecodeClusterSecret(hexSecret string) ([]byte, error) {
}
switch secretLen := len(secret); secretLen {
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
case 32:
return secret, nil

View File

@ -14,9 +14,7 @@ import (
crypto "github.com/libp2p/go-libp2p-crypto"
host "github.com/libp2p/go-libp2p-host"
dht "github.com/libp2p/go-libp2p-kad-dht"
pnet "github.com/libp2p/go-libp2p-pnet"
pubsub "github.com/libp2p/go-libp2p-pubsub"
libp2pquic "github.com/libp2p/go-libp2p-quic-transport"
secio "github.com/libp2p/go-libp2p-secio"
libp2ptls "github.com/libp2p/go-libp2p-tls"
routedhost "github.com/libp2p/go-libp2p/p2p/host/routed"
@ -66,14 +64,9 @@ func NewClusterHost(
libp2p.EnableAutoRelay(),
}
prot, err := newProtector(cfg.Secret)
if err != nil {
return nil, nil, nil, err
}
h, err := newHost(
ctx,
prot,
cfg.Secret,
ident.PrivateKey,
opts...,
)
@ -88,7 +81,7 @@ func NewClusterHost(
}
// needed for auto relay
_, err = autonat.NewAutoNATService(ctx, h, baseOpts(prot)...)
_, err = autonat.NewAutoNATService(ctx, h, baseOpts(cfg.Secret)...)
if err != nil {
h.Close()
return nil, nil, nil, err
@ -99,11 +92,11 @@ func NewClusterHost(
// newHost creates a base cluster host without dht, pubsub, relay or nat etc.
// 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{
libp2p.Identity(priv),
}
finalOpts = append(finalOpts, baseOpts(prot)...)
finalOpts = append(finalOpts, baseOpts(psk)...)
finalOpts = append(finalOpts, opts...)
h, err := libp2p.New(
@ -117,27 +110,17 @@ func newHost(ctx context.Context, prot corepnet.Protector, priv crypto.PrivKey,
return h, nil
}
func baseOpts(prot corepnet.Protector) []libp2p.Option {
func baseOpts(psk corepnet.PSK) []libp2p.Option {
return []libp2p.Option{
libp2p.PrivateNetwork(prot),
libp2p.PrivateNetwork(psk),
libp2p.Security(libp2ptls.ID, libp2ptls.New),
libp2p.Security(secio.ID, secio.New),
libp2p.Transport(libp2pquic.NewTransport),
// TODO: quic does not support private networks
//libp2p.Transport(libp2pquic.NewTransport),
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) {
return dht.New(ctx, h)
}

View File

@ -16,7 +16,7 @@ import (
"github.com/ipfs/ipfs-cluster/api/rest/client"
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"
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
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")
@ -169,7 +169,7 @@ requires authorization. implies --https, which you can disable with --force-http
cfg.Username = user
cfg.Password = pass
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
}

View File

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

View File

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

View File

@ -22,7 +22,7 @@ func (sv *Saver) NotifySave() {
select {
case sv.save <- struct{}{}:
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
// IPFS connections
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
}
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) {
ipfsID := pID.IPFS.ID
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
}
@ -97,7 +97,7 @@ func (c *Cluster) recordIPFSLinks(cg *api.ConnectGraph, pID *api.ID) {
ipfsPid := peer.IDB58Encode(ipfsID)
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.IPFSLinks[ipfsPid] = make([]peer.ID, 0)

View File

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

View File

@ -15,6 +15,7 @@ import (
host "github.com/libp2p/go-libp2p-core/host"
peerstore "github.com/libp2p/go-libp2p-core/peerstore"
dht "github.com/libp2p/go-libp2p-kad-dht"
dhtopts "github.com/libp2p/go-libp2p-kad-dht/opts"
pubsub "github.com/libp2p/go-libp2p-pubsub"
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)
}
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)
idht, err := dht.New(ctx, h,
dhtopts.RoutingTableRefreshPeriod(200*time.Millisecond),
dhtopts.RoutingTableRefreshQueryTimeout(100*time.Millisecond),
)
if err != nil {
h.Close()
t.Fatal(err)

View File

@ -199,7 +199,7 @@ func (cfg *Config) applyJSONConfig(jcfg *jsonConfig) error {
parseDuration := func(txt string) time.Duration {
d, _ := time.ParseDuration(txt)
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
}

View File

@ -15,7 +15,7 @@ import (
"github.com/ipfs/ipfs-cluster/state/dsstate"
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"
host "github.com/libp2p/go-libp2p-core/host"
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
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(
ctx,
cc.config.WaitForLeaderTimeout,

View File

@ -8,7 +8,7 @@ import (
"time"
hclog "github.com/hashicorp/go-hclog"
logging "github.com/ipfs/go-log"
logging "github.com/ipfs/go-log/v2"
)
const (
@ -52,6 +52,21 @@ func (log *hcLogToLogger) format(msg string, args []interface{}) string {
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{}) {
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{}) {
raftLogger.Warning(log.format(msg, args))
raftLogger.Warn(log.format(msg, args))
}
func (log *hcLogToLogger) Error(msg string, args ...interface{}) {
@ -134,7 +149,7 @@ type logForwarder struct {
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
// it is called per line.
func (fw *logForwarder) Write(p []byte) (n int, e error) {
@ -206,7 +221,7 @@ func (fw *logForwarder) log(t int, msg string) {
case info:
raftLogger.Info(msg)
case warn:
raftLogger.Warning(msg)
raftLogger.Warn(msg)
case err:
raftLogger.Error(msg)
default:

View File

@ -393,7 +393,7 @@ func (rw *raftWrapper) snapshotOnShutdown() error {
err := rw.WaitForUpdates(ctx)
cancel()
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
}
@ -408,7 +408,7 @@ func (rw *raftWrapper) snapshotOnShutdown() error {
if done {
break
}
logger.Warningf("retrying to snapshot (%d/%d)...", i+1, maxShutdownSnapshotRetries)
logger.Warnf("retrying to snapshot (%d/%d)...", i+1, maxShutdownSnapshotRetries)
}
return err
}
@ -654,9 +654,9 @@ func CleanupRaft(cfg *Config) error {
dbh := newDataBackupHelper(dataFolder, keep)
err = dbh.makeBackup()
if err != nil {
logger.Warning(err)
logger.Warning("the state could not be cleaned properly")
logger.Warning("manual intervention may be needed before starting cluster again")
logger.Warn(err)
logger.Warn("the state could not be cleaned properly")
logger.Warn("manual intervention may be needed before starting cluster again")
}
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-ipld-cbor v0.0.4
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-mfs v0.1.1
github.com/ipfs/go-path v0.0.7
@ -78,4 +78,4 @@ require (
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"
logging "github.com/ipfs/go-log"
logging "github.com/ipfs/go-log/v2"
rpc "github.com/libp2p/go-libp2p-gorpc"
"go.opencensus.io/trace"

View File

@ -40,6 +40,7 @@ import (
peer "github.com/libp2p/go-libp2p-core/peer"
peerstore "github.com/libp2p/go-libp2p-core/peerstore"
dht "github.com/libp2p/go-libp2p-kad-dht"
dhtopts "github.com/libp2p/go-libp2p-kad-dht/opts"
pubsub "github.com/libp2p/go-libp2p-pubsub"
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)
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 {
priv, _, err := crypto.GenerateKeyPair(crypto.RSA, 2048)
if err != nil {
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
dhts[i] = d
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) {
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 {
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) {
return newDHT(ctx, h)
// TODO: when new dht options are released
// return dht.New(ctx, h, dhtopts.Bootstrap(dhtopts.BootstrapConfig{
// Timeout: 300 * time.Millisecond,
// SelfQueryInterval: 300 * time.Millisecond,
// }))
return dht.New(ctx, h,
dhtopts.RoutingTableRefreshPeriod(600*time.Millisecond),
dhtopts.RoutingTableRefreshQueryTimeout(300*time.Millisecond),
)
}
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)
waitForClustersHealthy(t, clusters)

View File

@ -21,7 +21,7 @@ import (
cid "github.com/ipfs/go-cid"
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"
peer "github.com/libp2p/go-libp2p-core/peer"
rpc "github.com/libp2p/go-libp2p-gorpc"

View File

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

View File

@ -1,26 +1,11 @@
package ipfscluster
import (
logging "github.com/ipfs/go-log"
logging "github.com/ipfs/go-log/v2"
)
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{

View File

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

View File

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

View File

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

View File

@ -15,7 +15,6 @@ import (
cid "github.com/ipfs/go-cid"
host "github.com/libp2p/go-libp2p-core/host"
peer "github.com/libp2p/go-libp2p-core/peer"
dht "github.com/libp2p/go-libp2p-kad-dht"
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.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
// each others via DHT.
@ -62,14 +61,6 @@ func peerManagerClusters(t *testing.T) ([]*Cluster, []*test.IpfsMock, host.Host)
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
}
@ -208,8 +199,10 @@ func TestClustersPeerAddInUnhealthyCluster(t *testing.T) {
_, err := clusters[0].PeerAdd(ctx, clusters[1].id)
ttlDelay()
ids := clusters[1].Peers(ctx)
if len(ids) != 2 {
t.Error("expected 2 peers")
// raft will have only 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
@ -245,8 +238,8 @@ func TestClustersPeerAddInUnhealthyCluster(t *testing.T) {
ttlDelay()
ids = clusters[0].Peers(ctx)
if len(ids) != 2 {
t.Error("cluster should have 2 peers after removing and adding 1")
if len(ids) < 2 {
t.Error("cluster should have at least 2 peers after removing and adding 1")
}
default:
t.Fatal("bad consensus")

View File

@ -15,7 +15,7 @@ import (
"github.com/ipfs/ipfs-cluster/api"
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"
"go.opencensus.io/trace"

View File

@ -14,7 +14,7 @@ import (
"github.com/ipfs/ipfs-cluster/state"
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"
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)) {
t.Fatal("Expected same number of peers")
}
if len(clusters[0].Peers(ctx)) != 2 {
t.Fatal("Expected 2 peers")
if len(clusters[0].Peers(ctx)) < 2 {
// 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"
"time"
logging "github.com/ipfs/go-log"
logging "github.com/ipfs/go-log/v2"
host "github.com/libp2p/go-libp2p-core/host"
net "github.com/libp2p/go-libp2p-core/network"
peer "github.com/libp2p/go-libp2p-core/peer"
@ -265,13 +265,13 @@ func (pm *Manager) SavePeerstore(pinfos []peer.AddrInfo) error {
for _, pinfo := range pinfos {
if len(pinfo.Addrs) == 0 {
logger.Warning("address info does not have any multiaddresses")
logger.Warn("address info does not have any multiaddresses")
continue
}
addrs, err := peer.AddrInfoToP2pAddrs(&pinfo)
if err != nil {
logger.Warning(err)
logger.Warn(err)
continue
}
for _, a := range addrs {

View File

@ -13,7 +13,7 @@ import (
ds "github.com/ipfs/go-datastore"
query "github.com/ipfs/go-datastore/query"
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"
trace "go.opencensus.io/trace"
@ -148,7 +148,7 @@ func (st *State) List(ctx context.Context) ([]*api.Pin, error) {
k := ds.NewKey(r.Key)
ci, err := st.unkey(k)
if err != nil {
logger.Warning("bad key (ignoring). key: ", k, "error: ", err)
logger.Warn("bad key (ignoring). key: ", k, "error: ", err)
continue
}
@ -236,15 +236,29 @@ func (st *State) Unmarshal(r io.Reader) error {
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 {
k := dshelp.CidToDsKey(c)
k := cidToDsKey(c)
return st.namespace.Child(k)
}
// convert /namespace/cidKey to Cid
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