2019-01-24 16:26:36 +00:00
|
|
|
// Copyright (C) 2019 Storj Labs, Inc.
|
2018-11-03 12:17:14 +00:00
|
|
|
// See LICENSE for copying information
|
|
|
|
|
|
|
|
// Package testplanet implements the full network wiring for testing
|
|
|
|
package testplanet
|
|
|
|
|
|
|
|
import (
|
|
|
|
"context"
|
2018-11-19 20:39:25 +00:00
|
|
|
"errors"
|
2019-01-10 13:13:27 +00:00
|
|
|
"io"
|
2018-11-03 12:17:14 +00:00
|
|
|
"io/ioutil"
|
|
|
|
"net"
|
|
|
|
"os"
|
|
|
|
"path/filepath"
|
2018-11-15 08:57:47 +00:00
|
|
|
"strconv"
|
2019-01-10 13:13:27 +00:00
|
|
|
"strings"
|
2019-02-04 15:40:37 +00:00
|
|
|
"sync"
|
2018-11-20 16:54:52 +00:00
|
|
|
"time"
|
2018-11-03 12:17:14 +00:00
|
|
|
|
2019-01-10 13:13:27 +00:00
|
|
|
"github.com/zeebo/errs"
|
2018-11-03 12:17:14 +00:00
|
|
|
"go.uber.org/zap"
|
2018-11-15 08:57:47 +00:00
|
|
|
"go.uber.org/zap/zaptest"
|
2018-11-03 12:17:14 +00:00
|
|
|
"google.golang.org/grpc"
|
|
|
|
|
2019-01-23 15:48:46 +00:00
|
|
|
"storj.io/storj/bootstrap"
|
|
|
|
"storj.io/storj/bootstrap/bootstrapdb"
|
2018-11-03 12:17:14 +00:00
|
|
|
"storj.io/storj/internal/memory"
|
2019-01-23 19:58:44 +00:00
|
|
|
"storj.io/storj/pkg/accounting/rollup"
|
|
|
|
"storj.io/storj/pkg/accounting/tally"
|
|
|
|
"storj.io/storj/pkg/audit"
|
2019-01-18 13:54:08 +00:00
|
|
|
"storj.io/storj/pkg/bwagreement"
|
|
|
|
"storj.io/storj/pkg/datarepair/checker"
|
|
|
|
"storj.io/storj/pkg/datarepair/repairer"
|
|
|
|
"storj.io/storj/pkg/discovery"
|
2019-01-30 20:47:21 +00:00
|
|
|
"storj.io/storj/pkg/identity"
|
2019-01-10 13:13:27 +00:00
|
|
|
"storj.io/storj/pkg/kademlia"
|
2018-11-19 14:40:01 +00:00
|
|
|
"storj.io/storj/pkg/node"
|
2018-11-19 20:39:25 +00:00
|
|
|
"storj.io/storj/pkg/overlay"
|
2018-11-03 12:17:14 +00:00
|
|
|
"storj.io/storj/pkg/pb"
|
2019-01-23 10:39:03 +00:00
|
|
|
"storj.io/storj/pkg/peertls"
|
2019-01-10 13:13:27 +00:00
|
|
|
"storj.io/storj/pkg/piecestore/psserver"
|
2018-11-03 12:17:14 +00:00
|
|
|
"storj.io/storj/pkg/pointerdb"
|
2019-01-23 10:39:03 +00:00
|
|
|
"storj.io/storj/pkg/server"
|
2019-01-10 13:13:27 +00:00
|
|
|
"storj.io/storj/pkg/storj"
|
2019-01-18 13:54:08 +00:00
|
|
|
"storj.io/storj/satellite"
|
2019-01-24 16:26:36 +00:00
|
|
|
"storj.io/storj/satellite/console/consoleweb"
|
2019-01-18 13:54:08 +00:00
|
|
|
"storj.io/storj/satellite/satellitedb"
|
2019-01-10 13:13:27 +00:00
|
|
|
"storj.io/storj/storagenode"
|
|
|
|
"storj.io/storj/storagenode/storagenodedb"
|
2018-11-03 12:17:14 +00:00
|
|
|
)
|
|
|
|
|
2019-01-10 13:13:27 +00:00
|
|
|
// Peer represents one of StorageNode or Satellite
|
|
|
|
type Peer interface {
|
|
|
|
ID() storj.NodeID
|
|
|
|
Addr() string
|
|
|
|
Local() pb.Node
|
|
|
|
|
|
|
|
Run(context.Context) error
|
|
|
|
Close() error
|
|
|
|
|
|
|
|
NewNodeClient() (node.Client, error)
|
|
|
|
}
|
|
|
|
|
2019-02-01 13:32:28 +00:00
|
|
|
// Config describes planet configuration
|
|
|
|
type Config struct {
|
|
|
|
SatelliteCount int
|
|
|
|
StorageNodeCount int
|
|
|
|
UplinkCount int
|
|
|
|
|
|
|
|
Identities *Identities
|
|
|
|
Reconfigure Reconfigure
|
|
|
|
}
|
|
|
|
|
|
|
|
// Reconfigure allows to change node configurations
|
|
|
|
type Reconfigure struct {
|
2019-02-04 20:37:46 +00:00
|
|
|
NewBootstrapDB func(index int) (bootstrap.DB, error)
|
|
|
|
Bootstrap func(index int, config *bootstrap.Config)
|
|
|
|
|
|
|
|
NewSatelliteDB func(index int) (satellite.DB, error)
|
|
|
|
Satellite func(index int, config *satellite.Config)
|
|
|
|
|
|
|
|
NewStorageNodeDB func(index int) (storagenode.DB, error)
|
|
|
|
StorageNode func(index int, config *storagenode.Config)
|
2019-02-01 13:32:28 +00:00
|
|
|
}
|
|
|
|
|
2018-11-03 12:17:14 +00:00
|
|
|
// Planet is a full storj system setup.
|
|
|
|
type Planet struct {
|
2018-11-15 08:57:47 +00:00
|
|
|
log *zap.Logger
|
2019-02-01 13:32:28 +00:00
|
|
|
config Config
|
2018-11-03 12:17:14 +00:00
|
|
|
directory string // TODO: ensure that everything is in-memory to speed things up
|
2018-11-19 20:39:25 +00:00
|
|
|
started bool
|
2018-11-03 12:17:14 +00:00
|
|
|
|
2019-02-04 15:40:37 +00:00
|
|
|
peers []closablePeer
|
2019-01-10 13:13:27 +00:00
|
|
|
databases []io.Closer
|
2019-02-04 16:56:10 +00:00
|
|
|
uplinks []*Uplink
|
2018-11-03 12:17:14 +00:00
|
|
|
|
2019-01-23 15:48:46 +00:00
|
|
|
Bootstrap *bootstrap.Peer
|
2019-01-18 13:54:08 +00:00
|
|
|
Satellites []*satellite.Peer
|
2019-01-10 13:13:27 +00:00
|
|
|
StorageNodes []*storagenode.Peer
|
2019-02-04 16:56:10 +00:00
|
|
|
Uplinks []*Uplink
|
2018-11-03 12:17:14 +00:00
|
|
|
|
|
|
|
identities *Identities
|
2019-01-25 22:33:20 +00:00
|
|
|
|
|
|
|
cancel func()
|
2018-11-03 12:17:14 +00:00
|
|
|
}
|
|
|
|
|
2019-02-04 15:40:37 +00:00
|
|
|
type closablePeer struct {
|
|
|
|
peer Peer
|
|
|
|
|
|
|
|
ctx context.Context
|
|
|
|
cancel func()
|
|
|
|
|
|
|
|
close sync.Once
|
|
|
|
err error
|
|
|
|
}
|
|
|
|
|
|
|
|
// Close closes safely the peer.
|
|
|
|
func (peer *closablePeer) Close() error {
|
|
|
|
peer.cancel()
|
|
|
|
peer.close.Do(func() {
|
|
|
|
peer.err = peer.peer.Close()
|
|
|
|
})
|
|
|
|
return peer.err
|
|
|
|
}
|
|
|
|
|
2018-11-03 12:17:14 +00:00
|
|
|
// New creates a new full system with the given number of nodes.
|
2018-11-15 08:57:47 +00:00
|
|
|
func New(t zaptest.TestingT, satelliteCount, storageNodeCount, uplinkCount int) (*Planet, error) {
|
|
|
|
var log *zap.Logger
|
|
|
|
if t == nil {
|
|
|
|
log = zap.NewNop()
|
|
|
|
} else {
|
|
|
|
log = zaptest.NewLogger(t)
|
|
|
|
}
|
|
|
|
|
2019-01-02 18:07:49 +00:00
|
|
|
return NewWithLogger(log, satelliteCount, storageNodeCount, uplinkCount)
|
|
|
|
}
|
|
|
|
|
|
|
|
// NewWithLogger creates a new full system with the given number of nodes.
|
|
|
|
func NewWithLogger(log *zap.Logger, satelliteCount, storageNodeCount, uplinkCount int) (*Planet, error) {
|
2019-02-01 13:32:28 +00:00
|
|
|
return NewCustom(log, Config{
|
|
|
|
SatelliteCount: satelliteCount,
|
|
|
|
StorageNodeCount: storageNodeCount,
|
|
|
|
UplinkCount: uplinkCount,
|
|
|
|
})
|
|
|
|
}
|
|
|
|
|
|
|
|
// NewCustom creates a new full system with the specified configuration.
|
|
|
|
func NewCustom(log *zap.Logger, config Config) (*Planet, error) {
|
|
|
|
if config.Identities == nil {
|
|
|
|
config.Identities = pregeneratedIdentities
|
|
|
|
}
|
|
|
|
|
2018-11-03 12:17:14 +00:00
|
|
|
planet := &Planet{
|
2018-11-15 08:57:47 +00:00
|
|
|
log: log,
|
2019-02-01 13:32:28 +00:00
|
|
|
config: config,
|
|
|
|
identities: config.Identities,
|
2018-11-03 12:17:14 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
var err error
|
|
|
|
planet.directory, err = ioutil.TempDir("", "planet")
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2019-01-23 15:48:46 +00:00
|
|
|
planet.Bootstrap, err = planet.newBootstrap()
|
|
|
|
if err != nil {
|
2019-01-24 20:28:06 +00:00
|
|
|
return nil, errs.Combine(err, planet.Shutdown())
|
2019-01-23 15:48:46 +00:00
|
|
|
}
|
|
|
|
|
2019-02-01 13:32:28 +00:00
|
|
|
planet.Satellites, err = planet.newSatellites(config.SatelliteCount)
|
2018-11-03 12:17:14 +00:00
|
|
|
if err != nil {
|
2019-01-24 20:28:06 +00:00
|
|
|
return nil, errs.Combine(err, planet.Shutdown())
|
2018-11-03 12:17:14 +00:00
|
|
|
}
|
|
|
|
|
2019-02-01 13:32:28 +00:00
|
|
|
planet.StorageNodes, err = planet.newStorageNodes(config.StorageNodeCount)
|
2018-11-03 12:17:14 +00:00
|
|
|
if err != nil {
|
2019-01-24 20:28:06 +00:00
|
|
|
return nil, errs.Combine(err, planet.Shutdown())
|
2018-11-03 12:17:14 +00:00
|
|
|
}
|
|
|
|
|
2019-02-04 16:56:10 +00:00
|
|
|
planet.Uplinks, err = planet.newUplinks("uplink", config.UplinkCount, config.StorageNodeCount)
|
2018-11-03 12:17:14 +00:00
|
|
|
if err != nil {
|
2019-01-24 20:28:06 +00:00
|
|
|
return nil, errs.Combine(err, planet.Shutdown())
|
2018-11-03 12:17:14 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
// init Satellites
|
2019-01-18 13:54:08 +00:00
|
|
|
for _, satellite := range planet.Satellites {
|
2019-02-01 13:32:28 +00:00
|
|
|
if len(satellite.Kademlia.Service.GetBootstrapNodes()) == 0 {
|
|
|
|
satellite.Kademlia.Service.SetBootstrapNodes([]pb.Node{planet.Bootstrap.Local()})
|
|
|
|
}
|
2018-11-03 12:17:14 +00:00
|
|
|
}
|
|
|
|
// init storage nodes
|
2019-01-10 13:13:27 +00:00
|
|
|
for _, storageNode := range planet.StorageNodes {
|
2019-02-01 13:32:28 +00:00
|
|
|
if len(storageNode.Kademlia.Service.GetBootstrapNodes()) == 0 {
|
|
|
|
storageNode.Kademlia.Service.SetBootstrapNodes([]pb.Node{planet.Bootstrap.Local()})
|
|
|
|
}
|
2018-11-03 12:17:14 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return planet, nil
|
|
|
|
}
|
|
|
|
|
|
|
|
// Start starts all the nodes.
|
|
|
|
func (planet *Planet) Start(ctx context.Context) {
|
2019-01-25 22:33:20 +00:00
|
|
|
ctx, cancel := context.WithCancel(ctx)
|
|
|
|
planet.cancel = cancel
|
|
|
|
|
2019-02-04 15:40:37 +00:00
|
|
|
for i := range planet.peers {
|
|
|
|
peer := &planet.peers[i]
|
|
|
|
peer.ctx, peer.cancel = context.WithCancel(ctx)
|
|
|
|
go func(peer *closablePeer) {
|
|
|
|
err := peer.peer.Run(peer.ctx)
|
2019-01-10 13:13:27 +00:00
|
|
|
if err == grpc.ErrServerStopped {
|
|
|
|
err = nil
|
|
|
|
}
|
|
|
|
if err != nil {
|
|
|
|
// TODO: better error handling
|
|
|
|
panic(err)
|
|
|
|
}
|
|
|
|
}(peer)
|
|
|
|
}
|
|
|
|
|
2018-11-19 20:39:25 +00:00
|
|
|
planet.started = true
|
2019-01-25 22:33:20 +00:00
|
|
|
|
|
|
|
for _, peer := range planet.Satellites {
|
|
|
|
peer.Kademlia.Service.WaitForBootstrap()
|
|
|
|
}
|
|
|
|
for _, peer := range planet.StorageNodes {
|
|
|
|
peer.Kademlia.Service.WaitForBootstrap()
|
|
|
|
}
|
2018-11-03 12:17:14 +00:00
|
|
|
}
|
|
|
|
|
2019-02-04 15:40:37 +00:00
|
|
|
// StopPeer stops a single peer in the planet
|
|
|
|
func (planet *Planet) StopPeer(peer Peer) error {
|
|
|
|
for i := range planet.peers {
|
|
|
|
p := &planet.peers[i]
|
|
|
|
if p.peer == peer {
|
|
|
|
return p.Close()
|
|
|
|
}
|
|
|
|
}
|
|
|
|
return errors.New("unknown peer")
|
|
|
|
}
|
|
|
|
|
2018-12-12 15:40:33 +00:00
|
|
|
// Size returns number of nodes in the network
|
2019-02-04 16:56:10 +00:00
|
|
|
func (planet *Planet) Size() int { return len(planet.uplinks) + len(planet.peers) }
|
2018-12-12 15:40:33 +00:00
|
|
|
|
2018-11-03 12:17:14 +00:00
|
|
|
// Shutdown shuts down all the nodes and deletes temporary directories.
|
|
|
|
func (planet *Planet) Shutdown() error {
|
2018-11-19 20:39:25 +00:00
|
|
|
if !planet.started {
|
2019-01-25 22:33:20 +00:00
|
|
|
return errors.New("Start was never called")
|
2018-11-19 20:39:25 +00:00
|
|
|
}
|
2019-01-25 22:33:20 +00:00
|
|
|
planet.cancel()
|
2018-11-19 20:39:25 +00:00
|
|
|
|
2019-01-25 22:33:20 +00:00
|
|
|
var errlist errs.Group
|
2018-11-03 12:17:14 +00:00
|
|
|
// shutdown in reverse order
|
2019-02-04 16:56:10 +00:00
|
|
|
for i := len(planet.uplinks) - 1; i >= 0; i-- {
|
|
|
|
node := planet.uplinks[i]
|
2019-01-10 13:13:27 +00:00
|
|
|
errlist.Add(node.Shutdown())
|
|
|
|
}
|
|
|
|
for i := len(planet.peers) - 1; i >= 0; i-- {
|
2019-02-04 15:40:37 +00:00
|
|
|
peer := &planet.peers[i]
|
2019-01-10 13:13:27 +00:00
|
|
|
errlist.Add(peer.Close())
|
|
|
|
}
|
|
|
|
for _, db := range planet.databases {
|
|
|
|
errlist.Add(db.Close())
|
2018-11-03 12:17:14 +00:00
|
|
|
}
|
2019-01-10 13:13:27 +00:00
|
|
|
|
|
|
|
errlist.Add(os.RemoveAll(planet.directory))
|
|
|
|
return errlist.Err()
|
2018-11-03 12:17:14 +00:00
|
|
|
}
|
|
|
|
|
2019-01-18 13:54:08 +00:00
|
|
|
// newUplinks creates initializes uplinks
|
2019-02-04 16:56:10 +00:00
|
|
|
func (planet *Planet) newUplinks(prefix string, count, storageNodeCount int) ([]*Uplink, error) {
|
|
|
|
var xs []*Uplink
|
2018-11-03 12:17:14 +00:00
|
|
|
for i := 0; i < count; i++ {
|
2019-02-04 16:56:10 +00:00
|
|
|
uplink, err := planet.newUplink(prefix+strconv.Itoa(i), storageNodeCount)
|
2018-11-03 12:17:14 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2019-02-04 16:56:10 +00:00
|
|
|
xs = append(xs, uplink)
|
2018-11-03 12:17:14 +00:00
|
|
|
}
|
|
|
|
|
|
|
|
return xs, nil
|
|
|
|
}
|
|
|
|
|
2019-01-18 13:54:08 +00:00
|
|
|
// newSatellites initializes satellites
|
|
|
|
func (planet *Planet) newSatellites(count int) ([]*satellite.Peer, error) {
|
|
|
|
var xs []*satellite.Peer
|
|
|
|
defer func() {
|
|
|
|
for _, x := range xs {
|
2019-02-04 15:40:37 +00:00
|
|
|
planet.peers = append(planet.peers, closablePeer{peer: x})
|
2019-01-18 13:54:08 +00:00
|
|
|
}
|
|
|
|
}()
|
|
|
|
|
|
|
|
for i := 0; i < count; i++ {
|
|
|
|
prefix := "satellite" + strconv.Itoa(i)
|
|
|
|
log := planet.log.Named(prefix)
|
|
|
|
|
|
|
|
storageDir := filepath.Join(planet.directory, prefix)
|
|
|
|
if err := os.MkdirAll(storageDir, 0700); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
identity, err := planet.NewIdentity()
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2019-02-04 20:37:46 +00:00
|
|
|
var db satellite.DB
|
|
|
|
if planet.config.Reconfigure.NewSatelliteDB != nil {
|
|
|
|
db, err = planet.config.Reconfigure.NewSatelliteDB(i)
|
|
|
|
} else {
|
|
|
|
db, err = satellitedb.NewInMemory()
|
|
|
|
}
|
2019-01-18 13:54:08 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
err = db.CreateTables()
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
planet.databases = append(planet.databases, db)
|
|
|
|
|
|
|
|
config := satellite.Config{
|
2019-01-23 19:58:44 +00:00
|
|
|
Server: server.Config{
|
|
|
|
Address: "127.0.0.1:0",
|
|
|
|
RevocationDBURL: "bolt://" + filepath.Join(planet.directory, "revocation.db"),
|
|
|
|
UsePeerCAWhitelist: false, // TODO: enable
|
|
|
|
Extensions: peertls.TLSExtConfig{
|
|
|
|
Revocation: true,
|
|
|
|
WhitelistSignedLeaf: false,
|
|
|
|
},
|
|
|
|
},
|
2019-01-18 13:54:08 +00:00
|
|
|
Kademlia: kademlia.Config{
|
|
|
|
Alpha: 5,
|
|
|
|
DBPath: storageDir, // TODO: replace with master db
|
|
|
|
Operator: kademlia.OperatorConfig{
|
|
|
|
Email: prefix + "@example.com",
|
|
|
|
Wallet: "0x" + strings.Repeat("00", 20),
|
|
|
|
},
|
|
|
|
},
|
|
|
|
Overlay: overlay.Config{
|
|
|
|
RefreshInterval: 30 * time.Second,
|
|
|
|
Node: overlay.NodeSelectionConfig{
|
2019-01-29 19:42:43 +00:00
|
|
|
UptimeRatio: 0,
|
|
|
|
UptimeCount: 0,
|
|
|
|
AuditSuccessRatio: 0,
|
|
|
|
AuditCount: 0,
|
|
|
|
NewNodeAuditThreshold: 0,
|
|
|
|
NewNodePercentage: 0,
|
2019-01-18 13:54:08 +00:00
|
|
|
},
|
|
|
|
},
|
|
|
|
Discovery: discovery.Config{
|
|
|
|
RefreshInterval: 1 * time.Second,
|
2019-01-30 16:29:18 +00:00
|
|
|
RefreshLimit: 100,
|
2019-01-18 13:54:08 +00:00
|
|
|
},
|
|
|
|
PointerDB: pointerdb.Config{
|
|
|
|
DatabaseURL: "bolt://" + filepath.Join(storageDir, "pointers.db"),
|
|
|
|
MinRemoteSegmentSize: 0, // TODO: fix tests to work with 1024
|
|
|
|
MaxInlineSegmentSize: 8000,
|
|
|
|
Overlay: true,
|
|
|
|
BwExpiration: 45,
|
|
|
|
},
|
|
|
|
BwAgreement: bwagreement.Config{},
|
|
|
|
Checker: checker.Config{
|
|
|
|
Interval: 30 * time.Second,
|
|
|
|
},
|
|
|
|
Repairer: repairer.Config{
|
|
|
|
MaxRepair: 10,
|
|
|
|
Interval: time.Hour,
|
|
|
|
OverlayAddr: "", // overridden in satellite.New
|
|
|
|
PointerDBAddr: "", // overridden in satellite.New
|
|
|
|
MaxBufferMem: 4 * memory.MB,
|
|
|
|
APIKey: "",
|
|
|
|
},
|
2019-01-23 19:58:44 +00:00
|
|
|
Audit: audit.Config{
|
|
|
|
MaxRetriesStatDB: 0,
|
|
|
|
Interval: 30 * time.Second,
|
|
|
|
},
|
|
|
|
Tally: tally.Config{
|
|
|
|
Interval: 30 * time.Second,
|
|
|
|
},
|
|
|
|
Rollup: rollup.Config{
|
|
|
|
Interval: 120 * time.Second,
|
|
|
|
},
|
2019-01-24 16:26:36 +00:00
|
|
|
Console: consoleweb.Config{
|
|
|
|
Address: "127.0.0.1:0",
|
|
|
|
},
|
2019-01-18 13:54:08 +00:00
|
|
|
}
|
2019-02-01 13:32:28 +00:00
|
|
|
if planet.config.Reconfigure.Satellite != nil {
|
2019-02-04 20:37:46 +00:00
|
|
|
planet.config.Reconfigure.Satellite(i, &config)
|
2019-02-01 13:32:28 +00:00
|
|
|
}
|
2019-01-18 13:54:08 +00:00
|
|
|
|
2019-01-24 16:26:36 +00:00
|
|
|
// TODO: for development only
|
|
|
|
config.Console.StaticDir = "./web/satellite"
|
|
|
|
|
2019-01-18 13:54:08 +00:00
|
|
|
peer, err := satellite.New(log, identity, db, &config)
|
|
|
|
if err != nil {
|
|
|
|
return xs, err
|
|
|
|
}
|
|
|
|
|
|
|
|
log.Debug("id=" + peer.ID().String() + " addr=" + peer.Addr())
|
|
|
|
xs = append(xs, peer)
|
|
|
|
}
|
|
|
|
return xs, nil
|
|
|
|
}
|
|
|
|
|
2019-01-10 13:13:27 +00:00
|
|
|
// newStorageNodes initializes storage nodes
|
|
|
|
func (planet *Planet) newStorageNodes(count int) ([]*storagenode.Peer, error) {
|
|
|
|
var xs []*storagenode.Peer
|
|
|
|
defer func() {
|
|
|
|
for _, x := range xs {
|
2019-02-04 15:40:37 +00:00
|
|
|
planet.peers = append(planet.peers, closablePeer{peer: x})
|
2019-01-10 13:13:27 +00:00
|
|
|
}
|
|
|
|
}()
|
|
|
|
|
|
|
|
for i := 0; i < count; i++ {
|
|
|
|
prefix := "storage" + strconv.Itoa(i)
|
|
|
|
log := planet.log.Named(prefix)
|
|
|
|
storageDir := filepath.Join(planet.directory, prefix)
|
|
|
|
|
2019-01-18 13:54:08 +00:00
|
|
|
if err := os.MkdirAll(storageDir, 0700); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2019-01-10 13:13:27 +00:00
|
|
|
identity, err := planet.NewIdentity()
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2019-02-04 20:37:46 +00:00
|
|
|
var db storagenode.DB
|
|
|
|
if planet.config.Reconfigure.NewStorageNodeDB != nil {
|
|
|
|
db, err = planet.config.Reconfigure.NewStorageNodeDB(i)
|
|
|
|
} else {
|
|
|
|
db, err = storagenodedb.NewInMemory(storageDir)
|
|
|
|
}
|
2019-01-10 13:13:27 +00:00
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2019-01-18 13:54:08 +00:00
|
|
|
|
2019-01-24 20:28:06 +00:00
|
|
|
err = db.CreateTables()
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2019-01-18 13:54:08 +00:00
|
|
|
|
2019-01-10 13:13:27 +00:00
|
|
|
planet.databases = append(planet.databases, db)
|
|
|
|
|
|
|
|
config := storagenode.Config{
|
2019-01-23 10:39:03 +00:00
|
|
|
Server: server.Config{
|
|
|
|
Address: "127.0.0.1:0",
|
2019-01-23 15:48:46 +00:00
|
|
|
RevocationDBURL: "bolt://" + filepath.Join(storageDir, "revocation.db"),
|
2019-01-23 10:39:03 +00:00
|
|
|
UsePeerCAWhitelist: false, // TODO: enable
|
|
|
|
Extensions: peertls.TLSExtConfig{
|
|
|
|
Revocation: true,
|
|
|
|
WhitelistSignedLeaf: false,
|
|
|
|
},
|
|
|
|
},
|
2019-01-10 13:13:27 +00:00
|
|
|
Kademlia: kademlia.Config{
|
2019-01-18 13:54:08 +00:00
|
|
|
Alpha: 5,
|
|
|
|
DBPath: storageDir, // TODO: replace with master db
|
2019-01-10 13:13:27 +00:00
|
|
|
Operator: kademlia.OperatorConfig{
|
|
|
|
Email: prefix + "@example.com",
|
|
|
|
Wallet: "0x" + strings.Repeat("00", 20),
|
|
|
|
},
|
|
|
|
},
|
|
|
|
Storage: psserver.Config{
|
2019-01-29 15:41:01 +00:00
|
|
|
Path: "", // TODO: this argument won't be needed with master storagenodedb
|
|
|
|
AllocatedDiskSpace: memory.TB,
|
|
|
|
AllocatedBandwidth: memory.TB,
|
|
|
|
KBucketRefreshInterval: time.Hour,
|
|
|
|
|
2019-01-23 10:39:03 +00:00
|
|
|
AgreementSenderCheckInterval: time.Hour,
|
2019-01-29 15:41:01 +00:00
|
|
|
CollectorInterval: time.Hour,
|
2019-01-10 13:13:27 +00:00
|
|
|
},
|
|
|
|
}
|
2019-02-01 13:32:28 +00:00
|
|
|
if planet.config.Reconfigure.StorageNode != nil {
|
2019-02-04 20:37:46 +00:00
|
|
|
planet.config.Reconfigure.StorageNode(i, &config)
|
2019-02-01 13:32:28 +00:00
|
|
|
}
|
2019-01-10 13:13:27 +00:00
|
|
|
|
|
|
|
peer, err := storagenode.New(log, identity, db, config)
|
|
|
|
if err != nil {
|
|
|
|
return xs, err
|
|
|
|
}
|
|
|
|
|
|
|
|
log.Debug("id=" + peer.ID().String() + " addr=" + peer.Addr())
|
|
|
|
xs = append(xs, peer)
|
|
|
|
}
|
|
|
|
return xs, nil
|
|
|
|
}
|
|
|
|
|
2019-01-23 15:48:46 +00:00
|
|
|
// newBootstrap initializes the bootstrap node
|
|
|
|
func (planet *Planet) newBootstrap() (peer *bootstrap.Peer, err error) {
|
|
|
|
defer func() {
|
2019-02-04 15:40:37 +00:00
|
|
|
planet.peers = append(planet.peers, closablePeer{peer: peer})
|
2019-01-23 15:48:46 +00:00
|
|
|
}()
|
|
|
|
|
|
|
|
prefix := "bootstrap"
|
|
|
|
log := planet.log.Named(prefix)
|
|
|
|
dbDir := filepath.Join(planet.directory, prefix)
|
|
|
|
|
|
|
|
if err := os.MkdirAll(dbDir, 0700); err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
identity, err := planet.NewIdentity()
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
2019-02-04 20:37:46 +00:00
|
|
|
var db bootstrap.DB
|
|
|
|
if planet.config.Reconfigure.NewBootstrapDB != nil {
|
|
|
|
db, err = planet.config.Reconfigure.NewBootstrapDB(0)
|
|
|
|
} else {
|
|
|
|
db, err = bootstrapdb.NewInMemory(dbDir)
|
2019-01-23 15:48:46 +00:00
|
|
|
}
|
|
|
|
|
2019-01-24 20:28:06 +00:00
|
|
|
err = db.CreateTables()
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
2019-01-23 15:48:46 +00:00
|
|
|
|
|
|
|
planet.databases = append(planet.databases, db)
|
|
|
|
|
|
|
|
config := bootstrap.Config{
|
|
|
|
Server: server.Config{
|
|
|
|
Address: "127.0.0.1:0",
|
|
|
|
RevocationDBURL: "bolt://" + filepath.Join(dbDir, "revocation.db"),
|
|
|
|
UsePeerCAWhitelist: false, // TODO: enable
|
|
|
|
Extensions: peertls.TLSExtConfig{
|
|
|
|
Revocation: true,
|
|
|
|
WhitelistSignedLeaf: false,
|
|
|
|
},
|
|
|
|
},
|
|
|
|
Kademlia: kademlia.Config{
|
|
|
|
Alpha: 5,
|
|
|
|
DBPath: dbDir, // TODO: replace with master db
|
|
|
|
Operator: kademlia.OperatorConfig{
|
|
|
|
Email: prefix + "@example.com",
|
|
|
|
Wallet: "0x" + strings.Repeat("00", 20),
|
|
|
|
},
|
|
|
|
},
|
|
|
|
}
|
2019-02-01 13:32:28 +00:00
|
|
|
if planet.config.Reconfigure.Bootstrap != nil {
|
2019-02-04 20:37:46 +00:00
|
|
|
planet.config.Reconfigure.Bootstrap(0, &config)
|
2019-02-01 13:32:28 +00:00
|
|
|
}
|
2019-01-23 15:48:46 +00:00
|
|
|
|
|
|
|
peer, err = bootstrap.New(log, identity, db, config)
|
|
|
|
if err != nil {
|
|
|
|
return nil, err
|
|
|
|
}
|
|
|
|
|
|
|
|
log.Debug("id=" + peer.ID().String() + " addr=" + peer.Addr())
|
|
|
|
|
|
|
|
return peer, nil
|
|
|
|
}
|
|
|
|
|
2019-02-01 13:32:28 +00:00
|
|
|
// Identities returns the identity provider for this planet.
|
|
|
|
func (planet *Planet) Identities() *Identities {
|
|
|
|
return planet.identities
|
|
|
|
}
|
|
|
|
|
2019-01-02 18:07:49 +00:00
|
|
|
// NewIdentity creates a new identity for a node
|
2019-01-30 20:47:21 +00:00
|
|
|
func (planet *Planet) NewIdentity() (*identity.FullIdentity, error) {
|
2018-11-03 12:17:14 +00:00
|
|
|
return planet.identities.NewIdentity()
|
|
|
|
}
|
|
|
|
|
2019-01-18 13:54:08 +00:00
|
|
|
// NewListener creates a new listener
|
|
|
|
func (planet *Planet) NewListener() (net.Listener, error) {
|
2018-11-03 12:17:14 +00:00
|
|
|
return net.Listen("tcp", "127.0.0.1:0")
|
|
|
|
}
|