storj/private/testplanet/planet.go

317 lines
7.2 KiB
Go
Raw Normal View History

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"
"errors"
2019-01-10 13:13:27 +00:00
"io"
2018-11-03 12:17:14 +00:00
"io/ioutil"
"net"
"os"
"path/filepath"
"sync"
"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"
2019-02-06 13:19:14 +00:00
"golang.org/x/sync/errgroup"
2018-11-03 12:17:14 +00:00
"storj.io/common/identity"
"storj.io/common/identity/testidentity"
"storj.io/common/storj"
"storj.io/storj/pkg/server"
"storj.io/storj/private/dbutil/pgutil"
"storj.io/storj/satellite/overlay"
"storj.io/storj/satellite/satellitedb/satellitedbtest"
2019-01-10 13:13:27 +00:00
"storj.io/storj/storagenode"
Add Versioning Server (#1576) * Initial Webserver Draft for Version Controlling * Rename type to avoid confusion * Move Function Calls into Version Package * Fix Linting and Language Typos * Fix Linting and Spelling Mistakes * Include Copyright * Include Copyright * Adjust Version-Control Server to return list of Versions * Linting * Improve Request Handling and Readability * Add Configuration File Option Add Systemd Service file * Add Logging to File * Smaller Changes * Add Semantic Versioning and refuses outdated Software from Startup (#1612) * implements internal Semantic Version library * adds version logging + reporting to process * Advance SemVer struct for easier handling * Add Accepted Version Store * Fix Function * Restructure * Type Conversion * Handle Version String properly * Add Note about array index * Set temporary Default Version * Add Copyright * Adding Version to Dashboard * Adding Version Info Log * Renaming and adding CheckerProcess * Iteration Sync * Iteration V2 * linting * made LogAndReportVersion a go routine * Refactor to Go Routine * Add Context to Go Routine and allow Operation if Lookup to Control Server fails * Handle Unmarshal properly * Linting * Relocate Version Checks * Relocating Version Check and specified default Version for now * Linting Error Prevention * Refuse Startup on outdated Version * Add Startup Check Function * Straighten Logging * Dont force Shutdown if --dev flag is set * Create full Service/Peer Structure for ControlServer * Linting * Straighting Naming * Finish VersionControl Service Layout * Improve Error Handling * Change Listening Address * Move Checker Function * Remove VersionControl Peer * Linting * Linting * Create VersionClient Service * Renaming * Add Version Client to Peer Definitions * Linting and Renaming * Linting * Remove Transport Checks for now * Move to Client Side Flag * Remove check * Linting * Transport Client Version Intro * Adding Version Client to Transport Client * Add missing parameter * Adding Version Check, to set Allowed = true * Set Default to true, testing * Restructuring Code * Uplink Changes * Add more proper Defaults * Renaming of Version struct * Dont pass Service use Pointer * Set Defaults for Versioning Checks * Put HTTP Server in go routine * Add Versioncontrol to Storj-Sim * Testplanet Fixes * Linting * Add Error Handling and new Server Struct * Move Lock slightly * Reduce Race Potentials * Remove unnecessary files * Linting * Add Proper Transport Handling * small fixes * add fence for allowed check * Add Startup Version Check and Service Naming * make errormessage private * Add Comments about VersionedClient * Linting * Remove Checks that refuse outgoing connections * Remove release cmd * Add Release Script * Linting * Update to use correct Values * Move vars private and set minimum default versions for testing builds * Remove VersionedClient * Better Error Handling and naked return removal * Straighten the Regex and string conversion * Change Check to allows testplanet and storj-sim to run without the need to pass an LDFlag * Cosmetic Change to Dashboard * Cleanup Returns and remove commented code * Remove Version Check if no build options are passed in * Pass in Config Values instead of Pointers * Handle missed Error * Update Endpoint URL * Change Type of Release Flag * Add additional Logging * Remove Versions Logging of other Services * minor fixes Change-Id: I5cc04a410ea6b2008d14dffd63eb5f36dd348a8b
2019-04-03 20:13:39 +01:00
"storj.io/storj/versioncontrol"
2018-11-03 12:17:14 +00:00
)
const defaultInterval = 15 * time.Second
2019-01-10 13:13:27 +00:00
// Peer represents one of StorageNode or Satellite
type Peer interface {
ID() storj.NodeID
Addr() string
URL() storj.NodeURL
2019-04-22 10:07:50 +01:00
Local() overlay.NodeDossier
2019-01-10 13:13:27 +00:00
Run(context.Context) error
Close() error
}
2019-02-01 13:32:28 +00:00
// Config describes planet configuration
type Config struct {
SatelliteCount int
StorageNodeCount int
UplinkCount int
2019-04-08 19:15:19 +01:00
IdentityVersion *storj.IDVersion
Reconfigure Reconfigure
Name string
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 {
id string
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
2019-02-06 13:19:14 +00:00
started bool
shutdown bool
2018-11-03 12:17:14 +00:00
peers []closablePeer
2019-01-10 13:13:27 +00:00
databases []io.Closer
uplinks []*Uplink
2018-11-03 12:17:14 +00:00
Add Versioning Server (#1576) * Initial Webserver Draft for Version Controlling * Rename type to avoid confusion * Move Function Calls into Version Package * Fix Linting and Language Typos * Fix Linting and Spelling Mistakes * Include Copyright * Include Copyright * Adjust Version-Control Server to return list of Versions * Linting * Improve Request Handling and Readability * Add Configuration File Option Add Systemd Service file * Add Logging to File * Smaller Changes * Add Semantic Versioning and refuses outdated Software from Startup (#1612) * implements internal Semantic Version library * adds version logging + reporting to process * Advance SemVer struct for easier handling * Add Accepted Version Store * Fix Function * Restructure * Type Conversion * Handle Version String properly * Add Note about array index * Set temporary Default Version * Add Copyright * Adding Version to Dashboard * Adding Version Info Log * Renaming and adding CheckerProcess * Iteration Sync * Iteration V2 * linting * made LogAndReportVersion a go routine * Refactor to Go Routine * Add Context to Go Routine and allow Operation if Lookup to Control Server fails * Handle Unmarshal properly * Linting * Relocate Version Checks * Relocating Version Check and specified default Version for now * Linting Error Prevention * Refuse Startup on outdated Version * Add Startup Check Function * Straighten Logging * Dont force Shutdown if --dev flag is set * Create full Service/Peer Structure for ControlServer * Linting * Straighting Naming * Finish VersionControl Service Layout * Improve Error Handling * Change Listening Address * Move Checker Function * Remove VersionControl Peer * Linting * Linting * Create VersionClient Service * Renaming * Add Version Client to Peer Definitions * Linting and Renaming * Linting * Remove Transport Checks for now * Move to Client Side Flag * Remove check * Linting * Transport Client Version Intro * Adding Version Client to Transport Client * Add missing parameter * Adding Version Check, to set Allowed = true * Set Default to true, testing * Restructuring Code * Uplink Changes * Add more proper Defaults * Renaming of Version struct * Dont pass Service use Pointer * Set Defaults for Versioning Checks * Put HTTP Server in go routine * Add Versioncontrol to Storj-Sim * Testplanet Fixes * Linting * Add Error Handling and new Server Struct * Move Lock slightly * Reduce Race Potentials * Remove unnecessary files * Linting * Add Proper Transport Handling * small fixes * add fence for allowed check * Add Startup Version Check and Service Naming * make errormessage private * Add Comments about VersionedClient * Linting * Remove Checks that refuse outgoing connections * Remove release cmd * Add Release Script * Linting * Update to use correct Values * Move vars private and set minimum default versions for testing builds * Remove VersionedClient * Better Error Handling and naked return removal * Straighten the Regex and string conversion * Change Check to allows testplanet and storj-sim to run without the need to pass an LDFlag * Cosmetic Change to Dashboard * Cleanup Returns and remove commented code * Remove Version Check if no build options are passed in * Pass in Config Values instead of Pointers * Handle missed Error * Update Endpoint URL * Change Type of Release Flag * Add additional Logging * Remove Versions Logging of other Services * minor fixes Change-Id: I5cc04a410ea6b2008d14dffd63eb5f36dd348a8b
2019-04-03 20:13:39 +01:00
VersionControl *versioncontrol.Peer
Satellites []*SatelliteSystem
Add Versioning Server (#1576) * Initial Webserver Draft for Version Controlling * Rename type to avoid confusion * Move Function Calls into Version Package * Fix Linting and Language Typos * Fix Linting and Spelling Mistakes * Include Copyright * Include Copyright * Adjust Version-Control Server to return list of Versions * Linting * Improve Request Handling and Readability * Add Configuration File Option Add Systemd Service file * Add Logging to File * Smaller Changes * Add Semantic Versioning and refuses outdated Software from Startup (#1612) * implements internal Semantic Version library * adds version logging + reporting to process * Advance SemVer struct for easier handling * Add Accepted Version Store * Fix Function * Restructure * Type Conversion * Handle Version String properly * Add Note about array index * Set temporary Default Version * Add Copyright * Adding Version to Dashboard * Adding Version Info Log * Renaming and adding CheckerProcess * Iteration Sync * Iteration V2 * linting * made LogAndReportVersion a go routine * Refactor to Go Routine * Add Context to Go Routine and allow Operation if Lookup to Control Server fails * Handle Unmarshal properly * Linting * Relocate Version Checks * Relocating Version Check and specified default Version for now * Linting Error Prevention * Refuse Startup on outdated Version * Add Startup Check Function * Straighten Logging * Dont force Shutdown if --dev flag is set * Create full Service/Peer Structure for ControlServer * Linting * Straighting Naming * Finish VersionControl Service Layout * Improve Error Handling * Change Listening Address * Move Checker Function * Remove VersionControl Peer * Linting * Linting * Create VersionClient Service * Renaming * Add Version Client to Peer Definitions * Linting and Renaming * Linting * Remove Transport Checks for now * Move to Client Side Flag * Remove check * Linting * Transport Client Version Intro * Adding Version Client to Transport Client * Add missing parameter * Adding Version Check, to set Allowed = true * Set Default to true, testing * Restructuring Code * Uplink Changes * Add more proper Defaults * Renaming of Version struct * Dont pass Service use Pointer * Set Defaults for Versioning Checks * Put HTTP Server in go routine * Add Versioncontrol to Storj-Sim * Testplanet Fixes * Linting * Add Error Handling and new Server Struct * Move Lock slightly * Reduce Race Potentials * Remove unnecessary files * Linting * Add Proper Transport Handling * small fixes * add fence for allowed check * Add Startup Version Check and Service Naming * make errormessage private * Add Comments about VersionedClient * Linting * Remove Checks that refuse outgoing connections * Remove release cmd * Add Release Script * Linting * Update to use correct Values * Move vars private and set minimum default versions for testing builds * Remove VersionedClient * Better Error Handling and naked return removal * Straighten the Regex and string conversion * Change Check to allows testplanet and storj-sim to run without the need to pass an LDFlag * Cosmetic Change to Dashboard * Cleanup Returns and remove commented code * Remove Version Check if no build options are passed in * Pass in Config Values instead of Pointers * Handle missed Error * Update Endpoint URL * Change Type of Release Flag * Add additional Logging * Remove Versions Logging of other Services * minor fixes Change-Id: I5cc04a410ea6b2008d14dffd63eb5f36dd348a8b
2019-04-03 20:13:39 +01:00
StorageNodes []*storagenode.Peer
Uplinks []*Uplink
2018-11-03 12:17:14 +00:00
ReferralManager *server.Server
2019-04-08 19:15:19 +01:00
identities *testidentity.Identities
whitelistPath string // TODO: in-memory
2019-01-25 22:33:20 +00:00
2019-02-06 13:19:14 +00:00
run errgroup.Group
2019-01-25 22:33:20 +00:00
cancel func()
2018-11-03 12:17:14 +00:00
}
type closablePeer struct {
peer Peer
ctx context.Context
cancel func()
close sync.Once
closed chan error
err error
}
func newClosablePeer(peer Peer) closablePeer {
return closablePeer{
peer: peer,
closed: make(chan error, 1),
}
}
// Close closes safely the peer.
func (peer *closablePeer) Close() error {
peer.cancel()
peer.close.Do(func() {
peer.err = peer.peer.Close()
<-peer.closed
})
return peer.err
}
2019-02-01 13:32:28 +00:00
// NewCustom creates a new full system with the specified configuration.
func NewCustom(log *zap.Logger, config Config) (*Planet, error) {
// Clear error in the beginning to avoid issues down the line.
if err := satellitedbtest.PostgresDefined(); err != nil {
return nil, err
}
2019-04-08 19:15:19 +01:00
if config.IdentityVersion == nil {
version := storj.LatestIDVersion()
config.IdentityVersion = &version
}
2019-02-01 13:32:28 +00:00
2018-11-03 12:17:14 +00:00
planet := &Planet{
log: log,
id: config.Name + "/" + pgutil.CreateRandomTestingSchemaName(6),
config: config,
}
if config.Reconfigure.Identities != nil {
planet.identities = config.Reconfigure.Identities(log, *config.IdentityVersion)
} else {
planet.identities = testidentity.NewPregeneratedSignedIdentities(*config.IdentityVersion)
2018-11-03 12:17:14 +00:00
}
var err error
planet.directory, err = ioutil.TempDir("", "planet")
if err != nil {
return nil, err
}
2019-04-08 19:15:19 +01:00
whitelistPath, err := planet.WriteWhitelist(*config.IdentityVersion)
if err != nil {
return nil, err
}
planet.whitelistPath = whitelistPath
Add Versioning Server (#1576) * Initial Webserver Draft for Version Controlling * Rename type to avoid confusion * Move Function Calls into Version Package * Fix Linting and Language Typos * Fix Linting and Spelling Mistakes * Include Copyright * Include Copyright * Adjust Version-Control Server to return list of Versions * Linting * Improve Request Handling and Readability * Add Configuration File Option Add Systemd Service file * Add Logging to File * Smaller Changes * Add Semantic Versioning and refuses outdated Software from Startup (#1612) * implements internal Semantic Version library * adds version logging + reporting to process * Advance SemVer struct for easier handling * Add Accepted Version Store * Fix Function * Restructure * Type Conversion * Handle Version String properly * Add Note about array index * Set temporary Default Version * Add Copyright * Adding Version to Dashboard * Adding Version Info Log * Renaming and adding CheckerProcess * Iteration Sync * Iteration V2 * linting * made LogAndReportVersion a go routine * Refactor to Go Routine * Add Context to Go Routine and allow Operation if Lookup to Control Server fails * Handle Unmarshal properly * Linting * Relocate Version Checks * Relocating Version Check and specified default Version for now * Linting Error Prevention * Refuse Startup on outdated Version * Add Startup Check Function * Straighten Logging * Dont force Shutdown if --dev flag is set * Create full Service/Peer Structure for ControlServer * Linting * Straighting Naming * Finish VersionControl Service Layout * Improve Error Handling * Change Listening Address * Move Checker Function * Remove VersionControl Peer * Linting * Linting * Create VersionClient Service * Renaming * Add Version Client to Peer Definitions * Linting and Renaming * Linting * Remove Transport Checks for now * Move to Client Side Flag * Remove check * Linting * Transport Client Version Intro * Adding Version Client to Transport Client * Add missing parameter * Adding Version Check, to set Allowed = true * Set Default to true, testing * Restructuring Code * Uplink Changes * Add more proper Defaults * Renaming of Version struct * Dont pass Service use Pointer * Set Defaults for Versioning Checks * Put HTTP Server in go routine * Add Versioncontrol to Storj-Sim * Testplanet Fixes * Linting * Add Error Handling and new Server Struct * Move Lock slightly * Reduce Race Potentials * Remove unnecessary files * Linting * Add Proper Transport Handling * small fixes * add fence for allowed check * Add Startup Version Check and Service Naming * make errormessage private * Add Comments about VersionedClient * Linting * Remove Checks that refuse outgoing connections * Remove release cmd * Add Release Script * Linting * Update to use correct Values * Move vars private and set minimum default versions for testing builds * Remove VersionedClient * Better Error Handling and naked return removal * Straighten the Regex and string conversion * Change Check to allows testplanet and storj-sim to run without the need to pass an LDFlag * Cosmetic Change to Dashboard * Cleanup Returns and remove commented code * Remove Version Check if no build options are passed in * Pass in Config Values instead of Pointers * Handle missed Error * Update Endpoint URL * Change Type of Release Flag * Add additional Logging * Remove Versions Logging of other Services * minor fixes Change-Id: I5cc04a410ea6b2008d14dffd63eb5f36dd348a8b
2019-04-03 20:13:39 +01:00
planet.VersionControl, err = planet.newVersionControlServer()
if err != nil {
return nil, errs.Combine(err, planet.Shutdown())
}
planet.ReferralManager, err = planet.newReferralManager()
if err != nil {
return nil, errs.Combine(err, planet.Shutdown())
}
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
}
whitelistedSatellites := make(storj.NodeURLs, 0, len(planet.Satellites))
for _, satellite := range planet.Satellites {
whitelistedSatellites = append(whitelistedSatellites, satellite.URL())
}
planet.StorageNodes, err = planet.newStorageNodes(config.StorageNodeCount, whitelistedSatellites)
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
}
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
}
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
Add Versioning Server (#1576) * Initial Webserver Draft for Version Controlling * Rename type to avoid confusion * Move Function Calls into Version Package * Fix Linting and Language Typos * Fix Linting and Spelling Mistakes * Include Copyright * Include Copyright * Adjust Version-Control Server to return list of Versions * Linting * Improve Request Handling and Readability * Add Configuration File Option Add Systemd Service file * Add Logging to File * Smaller Changes * Add Semantic Versioning and refuses outdated Software from Startup (#1612) * implements internal Semantic Version library * adds version logging + reporting to process * Advance SemVer struct for easier handling * Add Accepted Version Store * Fix Function * Restructure * Type Conversion * Handle Version String properly * Add Note about array index * Set temporary Default Version * Add Copyright * Adding Version to Dashboard * Adding Version Info Log * Renaming and adding CheckerProcess * Iteration Sync * Iteration V2 * linting * made LogAndReportVersion a go routine * Refactor to Go Routine * Add Context to Go Routine and allow Operation if Lookup to Control Server fails * Handle Unmarshal properly * Linting * Relocate Version Checks * Relocating Version Check and specified default Version for now * Linting Error Prevention * Refuse Startup on outdated Version * Add Startup Check Function * Straighten Logging * Dont force Shutdown if --dev flag is set * Create full Service/Peer Structure for ControlServer * Linting * Straighting Naming * Finish VersionControl Service Layout * Improve Error Handling * Change Listening Address * Move Checker Function * Remove VersionControl Peer * Linting * Linting * Create VersionClient Service * Renaming * Add Version Client to Peer Definitions * Linting and Renaming * Linting * Remove Transport Checks for now * Move to Client Side Flag * Remove check * Linting * Transport Client Version Intro * Adding Version Client to Transport Client * Add missing parameter * Adding Version Check, to set Allowed = true * Set Default to true, testing * Restructuring Code * Uplink Changes * Add more proper Defaults * Renaming of Version struct * Dont pass Service use Pointer * Set Defaults for Versioning Checks * Put HTTP Server in go routine * Add Versioncontrol to Storj-Sim * Testplanet Fixes * Linting * Add Error Handling and new Server Struct * Move Lock slightly * Reduce Race Potentials * Remove unnecessary files * Linting * Add Proper Transport Handling * small fixes * add fence for allowed check * Add Startup Version Check and Service Naming * make errormessage private * Add Comments about VersionedClient * Linting * Remove Checks that refuse outgoing connections * Remove release cmd * Add Release Script * Linting * Update to use correct Values * Move vars private and set minimum default versions for testing builds * Remove VersionedClient * Better Error Handling and naked return removal * Straighten the Regex and string conversion * Change Check to allows testplanet and storj-sim to run without the need to pass an LDFlag * Cosmetic Change to Dashboard * Cleanup Returns and remove commented code * Remove Version Check if no build options are passed in * Pass in Config Values instead of Pointers * Handle missed Error * Update Endpoint URL * Change Type of Release Flag * Add additional Logging * Remove Versions Logging of other Services * minor fixes Change-Id: I5cc04a410ea6b2008d14dffd63eb5f36dd348a8b
2019-04-03 20:13:39 +01:00
planet.run.Go(func() error {
return planet.VersionControl.Run(ctx)
})
if planet.ReferralManager != nil {
planet.run.Go(func() error {
return planet.ReferralManager.Run(ctx)
})
}
for i := range planet.peers {
peer := &planet.peers[i]
peer.ctx, peer.cancel = context.WithCancel(ctx)
2019-02-06 13:19:14 +00:00
planet.run.Go(func() error {
err := peer.peer.Run(peer.ctx)
peer.closed <- err
close(peer.closed)
return err
2019-02-06 13:19:14 +00:00
})
2019-01-10 13:13:27 +00:00
}
var group errgroup.Group
2019-04-22 10:07:50 +01:00
for _, peer := range planet.StorageNodes {
peer := peer
group.Go(func() error {
peer.Storage2.Monitor.Loop.TriggerWait()
peer.Contact.Chore.TriggerWait(ctx)
return nil
})
2019-01-25 22:33:20 +00:00
}
_ = group.Wait()
2019-04-22 10:07:50 +01:00
Remove Kademlia dependencies from Satellite and Storagenode (#2966) What: cmd/inspector/main.go: removes kad commands internal/testplanet/planet.go: Waits for contact chore to finish satellite/contact/nodesservice.go: creates an empty nodes service implementation satellite/contact/service.go: implements Local and FetchInfo methods & adds external address config value satellite/discovery/service.go: replaces kad.FetchInfo with contact.FetchInfo in Refresh() & removes Discover() satellite/peer.go: sets up contact service and endpoints storagenode/console/service.go: replaces nodeID with contact.Local() storagenode/contact/chore.go: replaces routing table with contact service storagenode/contact/nodesservice.go: creates empty implementation for ping and request info nodes service & implements RequestInfo method storagenode/contact/service.go: creates a service to return the local node and update its own capacity storagenode/monitor/monitor.go: uses contact service in place of routing table storagenode/operator.go: moves operatorconfig from kad into its own setup storagenode/peer.go: sets up contact service, chore, pingstats and endpoints satellite/overlay/config.go: changes NodeSelectionConfig.OnlineWindow default to 4hr to allow for accurate repair selection Removes kademlia setups in: cmd/storagenode/main.go cmd/storj-sim/network.go internal/testplane/planet.go internal/testplanet/satellite.go internal/testplanet/storagenode.go satellite/peer.go scripts/test-sim-backwards.sh scripts/testdata/satellite-config.yaml.lock storagenode/inspector/inspector.go storagenode/peer.go storagenode/storagenodedb/database.go Why: Replacing Kademlia Please describe the tests: • internal/testplanet/planet_test.go: TestBasic: assert that the storagenode can check in with the satellite without any errors TestContact: test that all nodes get inserted into both satellites' overlay cache during testplanet setup • satellite/contact/contact_test.go: TestFetchInfo: Tests that the FetchInfo method returns the correct info • storagenode/contact/contact_test.go: TestNodeInfoUpdated: tests that the contact chore updates the node information TestRequestInfoEndpoint: tests that the Request info endpoint returns the correct info Please describe the performance impact: Node discovery should be at least slightly more performant since each node connects directly to each satellite and no longer needs to wait for bootstrapping. It probably won't be faster in real time on start up since each node waits a random amount of time (less than 1 hr) to initialize its first connection (jitter).
2019-09-19 20:56:34 +01:00
planet.started = true
2018-11-03 12:17:14 +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
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 {
if !planet.started {
2019-01-25 22:33:20 +00:00
return errors.New("Start was never called")
}
2019-02-06 13:19:14 +00:00
if planet.shutdown {
panic("double Shutdown")
}
planet.shutdown = true
2019-01-25 22:33:20 +00:00
planet.cancel()
2019-01-25 22:33:20 +00:00
var errlist errs.Group
2019-02-06 13:19:14 +00:00
ctx, cancel := context.WithCancel(context.Background())
go func() {
// TODO: add diagnostics to see what hasn't been properly shut down
timer := time.NewTimer(30 * time.Second)
defer timer.Stop()
select {
case <-timer.C:
panic("planet took too long to shutdown")
case <-ctx.Done():
}
}()
errlist.Add(planet.run.Wait())
cancel()
2018-11-03 12:17:14 +00:00
// shutdown in reverse order
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-- {
peer := &planet.peers[i]
2019-01-10 13:13:27 +00:00
errlist.Add(peer.Close())
}
for i := len(planet.databases) - 1; i >= 0; i-- {
db := planet.databases[i]
2019-01-10 13:13:27 +00:00
errlist.Add(db.Close())
2018-11-03 12:17:14 +00:00
}
if planet.ReferralManager != nil {
errlist.Add(planet.ReferralManager.Close())
}
Add Versioning Server (#1576) * Initial Webserver Draft for Version Controlling * Rename type to avoid confusion * Move Function Calls into Version Package * Fix Linting and Language Typos * Fix Linting and Spelling Mistakes * Include Copyright * Include Copyright * Adjust Version-Control Server to return list of Versions * Linting * Improve Request Handling and Readability * Add Configuration File Option Add Systemd Service file * Add Logging to File * Smaller Changes * Add Semantic Versioning and refuses outdated Software from Startup (#1612) * implements internal Semantic Version library * adds version logging + reporting to process * Advance SemVer struct for easier handling * Add Accepted Version Store * Fix Function * Restructure * Type Conversion * Handle Version String properly * Add Note about array index * Set temporary Default Version * Add Copyright * Adding Version to Dashboard * Adding Version Info Log * Renaming and adding CheckerProcess * Iteration Sync * Iteration V2 * linting * made LogAndReportVersion a go routine * Refactor to Go Routine * Add Context to Go Routine and allow Operation if Lookup to Control Server fails * Handle Unmarshal properly * Linting * Relocate Version Checks * Relocating Version Check and specified default Version for now * Linting Error Prevention * Refuse Startup on outdated Version * Add Startup Check Function * Straighten Logging * Dont force Shutdown if --dev flag is set * Create full Service/Peer Structure for ControlServer * Linting * Straighting Naming * Finish VersionControl Service Layout * Improve Error Handling * Change Listening Address * Move Checker Function * Remove VersionControl Peer * Linting * Linting * Create VersionClient Service * Renaming * Add Version Client to Peer Definitions * Linting and Renaming * Linting * Remove Transport Checks for now * Move to Client Side Flag * Remove check * Linting * Transport Client Version Intro * Adding Version Client to Transport Client * Add missing parameter * Adding Version Check, to set Allowed = true * Set Default to true, testing * Restructuring Code * Uplink Changes * Add more proper Defaults * Renaming of Version struct * Dont pass Service use Pointer * Set Defaults for Versioning Checks * Put HTTP Server in go routine * Add Versioncontrol to Storj-Sim * Testplanet Fixes * Linting * Add Error Handling and new Server Struct * Move Lock slightly * Reduce Race Potentials * Remove unnecessary files * Linting * Add Proper Transport Handling * small fixes * add fence for allowed check * Add Startup Version Check and Service Naming * make errormessage private * Add Comments about VersionedClient * Linting * Remove Checks that refuse outgoing connections * Remove release cmd * Add Release Script * Linting * Update to use correct Values * Move vars private and set minimum default versions for testing builds * Remove VersionedClient * Better Error Handling and naked return removal * Straighten the Regex and string conversion * Change Check to allows testplanet and storj-sim to run without the need to pass an LDFlag * Cosmetic Change to Dashboard * Cleanup Returns and remove commented code * Remove Version Check if no build options are passed in * Pass in Config Values instead of Pointers * Handle missed Error * Update Endpoint URL * Change Type of Release Flag * Add additional Logging * Remove Versions Logging of other Services * minor fixes Change-Id: I5cc04a410ea6b2008d14dffd63eb5f36dd348a8b
2019-04-03 20:13:39 +01:00
errlist.Add(planet.VersionControl.Close())
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-02-01 13:32:28 +00:00
// Identities returns the identity provider for this planet.
2019-04-08 19:15:19 +01:00
func (planet *Planet) Identities() *testidentity.Identities {
2019-02-01 13:32:28 +00:00
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()
}
Satellite Peer (#1034) * add satellite peer * Add overlay * reorganize kademlia * add RunRefresh * add refresh to storagenode.Peer * add discovery * add agreements and metainfo * rename * add datarepair checker * add repair * add todo notes for audit * add testing interface * add into testplanet * fixes * fix compilation errors * fix compilation errors * make testplanet run * remove audit refrences * ensure that audit tests run * dev * checker tests compilable * fix discovery * fix compilation * fix * fix * dev * fix * disable auth * fixes * revert go.mod/sum * fix linter errors * fix * fix copyright * Add address param for SN dashboard (#1076) * Rename storj-sdk to storj-sim (#1078) * Storagenode logs and config improvements (#1075) * Add more info to SN logs * remove config-dir from user config * add output where config was stored * add message for successful connection * fix linter * remove storage.path from user config * resolve config path * move success message to info * log improvements * Remove captplanet (#1070) * pkg/server: include production cert (#1082) Change-Id: Ie8e6fe78550be83c3bd797db7a1e58d37c684792 * Generate Payments Report (#1079) * memory.Size: autoformat sizes based on value entropy (#1081) * Jj/bytes (#1085) * run tally and rollup * sets dev default tally and rollup intervals * nonessential storj-sim edits (#1086) * Closing context doesn't stop storage node (#1084) * Print when cancelled * Close properly * Don't log nil * Don't print error when closing dashboard * Fix panic in inspector if ping fails (#1088) * Consolidate identity management to identity cli commands (#1083) * Consolidate identity management: Move identity cretaion/signing out of storagenode setup command. * fixes * linters * Consolidate identity management: Move identity cretaion/signing out of storagenode setup command. * fixes * sava backups before saving signed certs * add "-prebuilt-test-cmds" test flag * linters * prepare cli tests for travis * linter fixes * more fixes * linter gods * sp/sdk/sim * remove ca.difficulty * remove unused difficulty * return setup to its rightful place * wip travis * Revert "wip travis" This reverts commit 56834849dcf066d3cc0a4f139033fc3f6d7188ca. * typo in travis.yaml * remove tests * remove more * make it only create one identity at a time for consistency * add config-dir for consitency * add identity creation to storj-sim * add flags * simplify * fix nolint and compile * prevent overwrite and pass difficulty, concurrency, and parent creds * goimports
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")
}
// WriteWhitelist writes the pregenerated signer's CA cert to a "CA whitelist", PEM-encoded.
2019-04-08 19:15:19 +01:00
func (planet *Planet) WriteWhitelist(version storj.IDVersion) (string, error) {
whitelistPath := filepath.Join(planet.directory, "whitelist.pem")
2019-04-08 19:15:19 +01:00
signer := testidentity.NewPregeneratedSigner(version)
err := identity.PeerCAConfig{
CertPath: whitelistPath,
}.Save(signer.PeerCA())
return whitelistPath, err
}