Query nodes table using AS OF SYSTEM TIME '-10s' (by default) when on CRDB to alleviate contention on the nodes table and minimize CRDB retries. Queries for standard uploads are already cached, and node lookups for graceful exit uploads has retry logic so it isn't necessary for the nodes returned to be current.
The chief segment health models we've come up with are the "immediate
danger" model and the "survivability" model. The former calculates the
chance of losing a segment becoming lost in the next time period (using
the CDF of the binomial distribution to estimate the chance of x nodes
failing in that period), while the latter estimates the number of
iterations for which a segment can be expected to survive (using the
mean of the negative binomial distribution). The immediate danger model
was a promising one for comparing segment health across segments with
different RS parameters, as it is more precisely what we want to
prevent, but it turns out that practically all segments in production
have infinite health, as the chance of losing segments with any
reasonable estimate of node failure rate is smaller than DBL_EPSILON,
the smallest possible difference from 1.0 representable in a float64
(about 1e-16).
Leaving aside the wisdom of worrying about the repair of segments that
have less than a 1e-16 chance of being lost, we want to be extremely
conservative and proactive in our repair efforts, and the health of the
segments we have been repairing thus far also evaluates to infinity
under the immediate danger model. Thus, we find ourselves reaching for
an alternative.
Dr. Ben saves the day: the survivability model is a reasonably close
approximation of the immediate danger model, and even better, it is
far simpler to calculate and yields manageable values for real-world
segments. The downside to it is that it requires as input an estimate
of the total number of active nodes.
This change replaces the segment health calculation to use the
survivability model, and reinstates the call to SegmentHealth() where it
was reverted. It gets estimates for the total number of active nodes by
leveraging the reliability cache.
Change-Id: Ia5d9b9031b9f6cf0fa7b9005a7011609415527dc
A few weeks ago it was discovered that the segment health function
was not working as expected with production values. As a bandaid,
we decided to insert the number of healthy pieces into the segment
health column. This should have effectively reverted our means of
prioritizing repair to the previous implementation.
However, it turns out that the bandaid was placed into the code which
removes items from the irreparable db and inserts them into the repair
queue.
This change: insert number of healthy pieces into the repair queue in the
method, RemoteSegment
Change-Id: Iabfc7984df0a928066b69e9aecb6f615253f1ad2
There is a new checker field called statsCollector. This contains
a map of stats pointers where the key is a stringified redundancy
scheme. stats contains all tagged monkit metrics. These metrics exist
under the key name, "tagged_repair_stats", which is tagged with the
name of each metric and a corresponding rs scheme.
As the metainfo observer works on a segment, it checks statsCollector
for a stats corresponding to the segment's redundancy scheme. If one
doesn't exist, it is created and chained to the monkit scope. Now we can call
Observe, Inc, etc on the fields just like before, and they have tags!
durabilityStats has also been renamed to aggregateStats.
At the end of the metainfo loop, we insert the aggregateStats totals into the
corresponding stats fields for metric reporting.
Change-Id: I8aa1918351d246a8ef818b9712ed4cb39d1ea9c6
We migrated satelliteDB off of Postgres and over to CockroachDB (crdb), but there was way too high contention for the injuredsegments table so we had to rollback to Postgres for the repair queue. A couple things contributed to this problem:
1) crdb doesn't support `FOR UPDATE SKIP LOCKED`
2) the original crdb Select query was doing 2 full table scans and not using any indexes
3) the SLC Satellite (where we were doing the migration) was running 48 repair worker processes, each of which run up to 5 goroutines which all are trying to select out of the repair queue and this was causing a ton of contention.
The changes in this PR should help to reduce that contention and improve performance on CRDB.
The changes include:
1) Use an update/set query instead of select/update to capitalize on the new `UPDATE` implicit row locking ability in CRDB.
- Details: As of CRDB v20.2.2, there is implicit row locking with update/set queries (contention reduction and performance gains are described in this blog post: https://www.cockroachlabs.com/blog/when-and-why-to-use-select-for-update-in-cockroachdb/).
2) Remove the `ORDER BY` clause since this was causing a full table scan and also prevented the use of the row locking capability.
- While long term it is very important to `ORDER BY segment_health`, the change here is only suppose to be a temporary bandaid to get us migrated over to CRDB quickly. Since segment_health has been set to infinity for some time now (re: https://review.dev.storj.io/c/storj/storj/+/3224), it seems like it might be ok to continue not making use of this for the short term. However, long term this needs to be fixed with a redesign of the repair workers, possible in the trusted delegated repair design (https://review.dev.storj.io/c/storj/storj/+/2602) or something similar to what is recommended here on how to implement a queue on CRDB https://dev.to/ajwerner/quick-and-easy-exactly-once-distributed-work-queues-using-serializable-transactions-jdp, or migrate to rabbit MQ priority queue or something similar..
This PRs improved query uses the index to avoid full scans and also locks the row its going to update and CRDB retries for us if there are any lock errors.
Change-Id: Id29faad2186627872fbeb0f31536c4f55f860f23
the immediate need is to be able to move the repair queue back out
of cockroach if we can't save it.
Change-Id: If26001a4e6804f6bb8713b4aee7e4fd6254dc326
We did not test the SegmentHealth function with actual production
values, and it turns out that values such as 52 healthy, 35 minimum
result in +Inf segment health - so pretty much all segments put into the
repair queue have the same health, which means we effectively aren't
sorting by health.
This change inserts numHealthy as segment health into the database so
the segments are ordered as they were before. We need to refine the
SegmentHealth function before we can support multi RS.
Change-Id: Ief19bbfee3594c5dfe94ca606bc930f05f85ff74
Rather than having a single repair override value, we will now support
repair override values based on a particular segment's RS scheme.
The new format for RS override values is
"k/o/n-override,k/o/n-override..."
Change-Id: Ieb422638446ef3a9357d59b2d279ee941367604d
Firstly, this changes the repair functionality to return Canceled errors
when a repair is canceled during the Get phase. Previously, because we
do not track individual errors per piece, this would just show up as a
failure to download enough pieces to repair the segment, which would
cause the segment to be added to the IrreparableDB, which is entirely
unhelpful.
Then, ignore Canceled errors in the return value of the repair worker.
Apparently, when the worker returns an error, that makes Cobra exit the
program with a nonzero exit code, which causes some piece of our
deployment automation to freak out and page people. And when we ask the
repair worker to shut down, "canceled" errors are what we _expect_, not
an error case.
Change-Id: Ia3eb1c60a8d6ec5d09e7cef55dea523be28e8435
We plan to add support for a new Reed-Solomon scheme soon, but our
repair queue orders segments by least number of healthy pieces first.
With a second RS scheme, fewer healthy pieces will not necessarily
correlate to lower health.
This change just adds the new column in a migration. A separate change
will add the new health function.
Right now, since we only support one RS scheme, behavior will not
change. Number of healthy pieces is being inserted as "segment health"
until the new health function is merged.
Segment health is calculated with a new priority function created in
commit 3e5640359. In order to use the function, a new config value is
added, called NodeFailureRate, representing the approximate probability
of any individual node going down in the duration of one checker run.
Change-Id: I51c4202203faf52528d923befbe886dbf86d02f2
The current monkit reporting for "remote_segments_lost" is not usable for
triggering alerts, as it has reported no data. To allow alerting, two new
metrics "checker_segments_below_min_req" and "repairer_segments_below_min_req"
will increment by zero on each segment unless it is below the minimum
required piece count. The two metrics report what is found by the checker
and the repairer respectively.
Change-Id: I98a68bb189eaf68a833d25cf5db9e68df535b9d7
Make metainfo.RSConfig a valid pflag config value. This allows us to
configure the RSConfig as a string like k/m/o/n-shareSize, which makes
having multiple supported RS schemes easier in the future.
RS-related config values that are no longer needed have been removed
(MinTotalThreshold, MaxTotalThreshold, MaxBufferMem, Verify).
Change-Id: I0178ae467dcf4375c504e7202f31443d627c15e1
A change was made to use a metabase.SegmentKey (a byte slice alias)
as the last seen item to iterate through the irreparable DB in a
for loop. However, this SegmentKey was not initialized, thus it was
nil. This caused the DB query to return nothing, and healthy segments
could not be cleaned out of the irreparable DB.
Change-Id: Idb30d6fef6113a30a27158d548f62c7443e65a81
With the new overlay.AuditOutcome type for offline audits, the
IsUp field is redundant. If AuditOutcome != AuditOffline, then
the node is online.
In addition to removing the field itself, other changes needed
to be made regarding the relationship between 'uptime' and 'audits'.
Previously, uptime and audit outcome were completely separated. For
example, it was possible to update a node's stats to give it a
successful/failed/unknown audit while simultaneously indicating that
the node was offline by setting IsUp to false. This is no longer possible
under this changeset. Some test which did this have been changed slightly
in order to pass.
Also add new benchmarks for UpdateStats and BatchUpdateStats with different
audit outcomes.
Change-Id: I998892d615850b1f138dc62f9b050f720ea0926b
As part of the Metainfo Refactoring, we need to make the Metainfo Loop
working with both the current PointerDB and the new Metabase. Thus, the
Metainfo Loop should pass to the Observer interface more specific Object
and Segment types instead of pb.Pointer.
After this change, there are still a couple of use cases that require
access to the pb.Pointer (hence we have it as a field in the
metainfo.Segment type):
1. Expired Deletion Service
2. Repair Service
It would require additional refactoring in these two services before we
are able to clean this.
Change-Id: Ib3eb6b7507ed89d5ba745ffbb6b37524ef10ed9f
Repair workers prioritize the most unhealthy segments. This has the consequence that when we
finally begin to reach the end of the queue, a good portion of the remaining segments are
healthy again as their nodes have come back online. This makes it appear that there are more
injured segments than there actually are.
solution:
Any time the checker observes an injured segment it inserts it into the repair queue or
updates it if it already exists. Therefore, we can determine which segments are no longer
injured if they were not inserted or updated by the last checker iteration. To do this we
add a new column to the injured segments table, updated_at, which is set to the current time
when a segment is inserted or updated. At the end of the checker iteration, we can delete any
items where updated_at < checker start.
Change-Id: I76a98487a4a845fab2fbc677638a732a95057a94
Another change which is a part of refactoring to replace path parameter
(string/[]byte) with key paramter (metabase.SegmentKey)
Change-Id: I617878442442e5d59bbe5c995f913c3c93c16928
* The audit worker wants to get items from the queue and process them.
* The audit chore wants to create new queues and swap them in when the
old queue has been processed.
This change adds a "Queues" struct which handles the concurrency
issues around the worker fetching a queue and the chore swapping a new
queue in. It simplifies the logic of the "Queue" struct to its bare
bones, so that it behaves like a normal queue with no need to understand
the details of swapping and worker/chore interactions.
Change-Id: Ic3689ede97a528e7590e98338cedddfa51794e1b
This change removes the overlay function FindStorageNodesForRepair,
which skips using the node selection cache and hits the database
directly. Otherwise, it is functionally identical to
FindStorageNodesForUpload, which checks the node selection cache first.
When selecting nodes for PUT_REPAIRs, we now call
FindStorageNodesForUpload instead of FindStorageNodesForRepair to reduce
database load.
Change-Id: If34e109695b2ed2b8fb6759115bf769a3459684e
* Do not swap the active audit queue with the pending audit queue until
the active audit queue is empty.
* Do not begin creating a new pending audit queue until the existing
pending audit queue has been swapped to the active queue.
Change-Id: I81db5bfa01458edb8cdbe71f5baeebdcb1b94317
It feels weird having a repairer configuration part of order services.
Let's have a single source of truth for it.
Change-Id: I24f7c897aec80f3293f8af24876cbb6733d85a0b
Inside CreateGetRepairOrderLimits we pass in a list of healthy pieces,
but when we query node info from this list we apply the "reliable" filter
again. We sometimes end up with nodes which at first were healthy, but then
became unhealthy, and thus can be repaired, but we do not update the 'unhealthyPieces'
list with these nodes.
This causes an error, 'piece to add already exists', as we fail to remove these
pieces from the pointer before replacing them with repaired pieces.
Change-Id: I6e2445f342ac117ded30351fa7e5e523c9ec26bd
What:
Use the github.com/jackc/pgx postgresql driver in place of
github.com/lib/pq.
Why:
github.com/lib/pq has some problems with error handling and context
cancellations (i.e. it might even issue queries or DML statements more
than once! see https://github.com/lib/pq/issues/939). The
github.com/jackx/pgx library appears not to have these problems, and
also appears to be better engineered and implemented (in particular, it
doesn't use "exceptions by panic"). It should also give us some
performance improvements in some cases, and even more so if we can use
it directly instead of going through the database/sql layer.
Change-Id: Ia696d220f340a097dee9550a312d37de14ed2044
Adds monkit tracing for ecrepairer.downloadAndVerifyPiece and
ecrepairer.putPiece so we can get more accurate estimates of node
performance during repair.
Change-Id: Ic05025bf3c493bb3d6f5d325d090c5b7c9e5465d
This will speed up the Put step of repair by not waiting to time out for
a handful of slow nodes, at the expense of a slightly less durable
pointer. It will still repair to the optimal threshold, but not every
node that is selected will end up in the pointer.
Change-Id: I02a0658e3fe6fc0383f26af0f50a065b8b11a651
* add monkit stat new_remote_segments_needing_repair, which reports the
number of new unhealthy segments in the repair queue since the previous
checker iteration
Change-Id: I2f10266006fdd6406ece50f4759b91382059dcc3
Sometimes nodes who have gracefully exited will still be holding pieces
according to the satellite. This has some unintended side effects
currently, such as nodes getting disqualified after having successfully
exited.
* When the audit reporter attempts to update node stats, do not update
stats (alpha, beta, suspension, disqualification) if the node has
finished graceful exit (audit/reporter_test.go TestGracefullyExitedNotUpdated)
* Treat gracefully exited nodes as "not reputable" so that the repairer
and checker do not count them as healthy (overlay/statdb_test.go
TestKnownUnreliableOrOffline, repair/repair_test.go
TestRepairGracefullyExited)
Change-Id: I1920d60dd35de5b2385a9b06989397628a2f1272
* Delete expired segments in expired segments service using metainfo
loop
* Add test to verify expired segments service deletes expired segments
* Ignore expired segments in checker observer
* Modify checker tests to verify that expired segments are ignored
* Ignore expired segments in segment repairer and drop from repair queue
* Add repair test to verify that a segment that expires after being
added to the repair queue is ignored and dropped from the repair queue
Change-Id: Ib2b0934db525fef58325583d2a7ca859b88ea60d
* satellite: update log levels
Change-Id: I86bc32e042d742af6dbc469a294291a2e667e81f
* log version on start up for every service
Change-Id: Ic128bb9c5ac52d4dc6d6c4cb3059fbad73f5d3de
* Use monkit for tracking failed ip resolutions
Change-Id: Ia5aa71d315515e0c5f62c98d9d115ef984cd50c2
* fix compile errors
Change-Id: Ia33c8b6e34e780bd1115120dc347a439d99e83bf
* add request limit value to storage node rpc err
Change-Id: I1ad6706a60237928e29da300d96a1bafa94156e5
* we cant track storage node ids in monkit metrics so lets use logging to track that for expired orders
Change-Id: I1cc1d240b29019ae2f8c774792765df3cbeac887
* fix build errs
Change-Id: I6d0ffe058e9a38b7ed031c85a29440f3d68e8d47
Add flag to satellite repairer, "InMemoryRepair" that allows the
satellite to decide whether to download the entire segment being
repaired into memory (this is what the satellite already does), or to
download it into temporary files on disk that will be read from in the
upload phase of repair.
This should help with handling high repair traffic on satellites that
cannot afford to spend 64mb of memory per repair worker.
Updates tests to test repair for both in memory and to disk.
Change-Id: Iddf591e165621497c98533d45bfea3c28b08a194
Make sure that suspended nodes are treated appropriately by the overlay
cache. This means we should expect the following behavior:
* suspended nodes (vetted or not) should not be selected for uploading
new segments
* suspended nodes should be treated by the checker and repairer as
"unhealthy", and should be removed upon successful repair
This commit also removes unused overlay functionality.
Fixes a bug with commit 8b72181a1f where
the audit reporter was automatically suspending nodes regardless of
audit outcome (see test added).
Tests:
* updates repair tests to ensure that a suspended node is treated as
unhealthy and will be removed from the pointer on successful repair
* updates overlay tests for KnownUnreliableOrOffline and KnownReliable
to expect suspended nodes to be considered "unreliable"
* adds satellitedb test that ensures overlay.SelectStorageNodes and
overlay.SelectNewStorageNodes do not include suspended nodes
* adds audit reporter test to ensure that different audit outcomes
result in the correct suspended/disqualified states
Change-Id: I40dba67278c8e8d2ce0bcec5e0a5cb6e4ce2f561
* change overlay.UpdateStats to allow a third audit outcome. Now it can
handle successful, failed, and unknown audits.
* when "unknown audit reputation"
(unknownAuditAlpha/(unknownAuditAlpha+unknownAuditBeta)) falls below the
DQ threshold, put node into suspension.
* when unknown audit reputation goes above the DQ threshold, remove node
from suspension.
* record unknown audits from audit reporter.
* add basic tests around unknown audits and suspension.
Change-Id: I125f06f3af52e8a29ba48dc19361821a9ff1daa1
My understanding is that the nodes table has the following fields:
- `address` field which can be a hostname or an IP
- `last_net` field that is the /24 subnet of the IP resolved from the address
This PR does the following:
1) add back the `last_ip` field to the nodes table
2) for uplink operations remove the calls that the satellite makes to `lookupNodeAddress` (which makes the DNS calls to resolve the IP from the hostname) and instead use the data stored in the nodes table `last_ip` field. This means that the IP that the satellite sends to the uplink for the storage nodes could be approx 1 hr stale. In the short term this is fine, next we will be adding changes so that the storage node pushes any IP changes to the satellite in real time.
3) use the address field for repair and audit since we want them to still make DNS calls to confirm the IP is up to date
4) try to reduce confusion about hostname, ip, subnet, and address in the code base
Change-Id: I96ce0d8bb78303f82483d0701bc79544b74057ac
Previously, we were simply discarding rows from the repair queue when
they couldn't be repaired (either because the overlay said too many
nodes were down, or because we failed to download enough pieces).
Now, such segments will be put into the irreparableDB for further
and (hopefully) more focused attention.
This change also better differentiates some error cases from Repair()
for monitoring purposes.
Change-Id: I82a52a6da50c948ddd651048e2a39cb4b1e6df5c
In production, the satellite is overriding the default repair threshold
(35) to a higher value (52). In some places in the checker and
irreparable processes, the repair threshold on the redundancy scheme is
used in place of the override value. This fixes those cases.
Change-Id: Ie7387217d9fb3886f050b5e5b67be51f276196de
The migration was broken into one migration per table to reduce table locking and reduce the
chances of failure due to SQL timeouts.
Of the 14 fields that lacked time zones, only the 3 named 'interval_start` seemed to have non-UTC data in them.
These fields are fixed in the migration by removing the +00 and adding AT TIME ZONE current_setting('TIMEZONE')
Field with good data are migrated by adding AT TIME ZONE 'UTC'
Note that postgres's timezone() is different than cockroach's timezone() so AT TIME ZONE is used.
https://storjlabs.atlassian.net/browse/SM-104
Change-Id: I410f2f1d7c11b143f17844347f37e6f4b1e70fce
On satellite, remove all references to free_bandwidth column in nodes table.
On storage node, remove references to AllocatedBandwidth and MinimumBandwidth and mark as deprecated.
Protobuf message, NodeCapacity, is left intact for backwards compatibility.
Once this is released to all satellites, we can drop the column from the DB.
Change-Id: I2ff6c6537fc9008a0c5588e951afea58ede85838
Add a column to the repair queue table in the satellite db for healthy
piece count. When an item is selected from the repair queue, the least
durable segment that has not been attempted in the past hour should be
selected first. This prevents our repairer from getting stuck doing work
on segments that are close to the repair threshold while allowing
segments that are more unhealthy to degrade further.
The migration also clears the repair queue so that the migration runs
quickly and we can properly account for segment health in future repair
work.
We do not select items off the repair queue that have been attempted in
the past six hours. This was changed from on hour to allow us time to
try a wider variety of segments when the repair queue is very large.
Change-Id: Iaf183f1e5fd45cd792a52e3563a3e43a2b9f410b
This new repair timeout (configured as TotalTimeout) will include both
the time to download pieces and the time to upload pieces, as well as
the time to pop the segment from the repair queue.
This is a move from Github PR #3645.
Change-Id: I47d618f57285845d8473fcd285f7d9be9b4318c8
Currently SNs report their free disk space once per hour. If a node
becomes full, it has to wait until the next contact cycle begins to
report; all the while receiving and failing upload requests. By increasing
the minimum required disk space, we can give the storage nodes more time
to report their space before the completely fill up. This change goes
hand-in-hand with another change we want to implement: trigger capacity
report on SN immediately upon falling below threshold.
Change-Id: I12f778286c6c3f582438b0e2949765ac43325e27
this commit updates our monkit dependency to the v3 version where
it outputs in an influx style. this makes discovery much easier
as many tools are built to look at it this way.
graphite and rothko will suffer some due to no longer being a tree
based on dots. hopefully time will exist to update rothko to
index based on the new metric format.
it adds an influx output for the statreceiver so that we can
write to influxdb v1 or v2 directly.
Change-Id: Iae9f9494a6d29cfbd1f932a5e71a891b490415ff
With this change RS configuration will be set on satellite. Uplink with
get RS values with BeginObject request and will use it. For backward
compatibility and to avoid super large change redundancy scheme stored
with bucket is not touched. This can be done in future.
Change-Id: Ia5f76fc10c37e2c44e4f7b8754f28eafe1f97eff
everyone was importing it as dbx anyway. why should it be
named satellitedb? so yeah just pass the "-p dbx" flag.
Change-Id: I5efa669f4f00f196b38a9acd0d402009475a936f
With the new storage node downtime tracking feature, we need remove current uptime reputation configs: UptimeReputationAlpha, UptimeReputationBeta, and
UptimeReputationDQ. This is the first step of removing the uptime
reputation columns from satellitedb
Change-Id: Ie8fab13295dbf545e33aeda0c4306cda4ba54e36
Remove direct dependency on uplink.RSConfig, this simplifies
moving the config file without introducing weird dependencies.
Change-Id: I7fd2a145401e0205d7047631df9d2810241efeec
* satellite/nodeselection: dont select nodes that havent checked in for a while
* change testplanet online window to one minute
* remove satellite reconfigure online window = 0 in repair tests
* pass timestamp into UpdateCheckIn
* change timestamp to timestamptz
* edit tests to set last_contact_success to 4 hours ago
* fix syntax error
* remove check for last_contact_success > last_contact_failure in IsOnline
We don't use reverse listing in any of our code, outside of tests, and
it is only exposed through libuplink in the
lib/uplink.(*Project).ListBuckets() API. We also don't know of any users
who might have a need for reverse listing through ListBuckets().
Since one of our prospective pointerdb backends can not support
backwards iteration, and because of the above considerations, we are
going to remove the reverse listing feature.
Change-Id: I8d2a1f33d01ee70b79918d584b8c671f57eef2a0
all of the packages and tests work with both grpc and
drpc. we'll probably need to do some jenkins pipelines
to run the tests with drpc as well.
most of the changes are really due to a bit of cleanup
of the pkg/transport.Client api into an rpc.Dialer in
the spirit of a net.Dialer. now that we don't need
observers, we can pass around stateless configuration
to everything rather than stateful things that issue
observations. it also adds a DialAddressID for the
case where we don't have a pb.Node, but we do have an
address and want to assert some ID. this happened
pretty frequently, and now there's no more weird
contortions creating custom tls options, etc.
a lot of the other changes are being consistent/using
the abstractions in the rpc package to do rpc style
things like finding peer information, or checking
status codes.
Change-Id: Ief62875e21d80a21b3c56a5a37f45887679f9412
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).
* update audit status as failed for nodes that failed piece hash verification
* remove comment
* fix lint error
* add test
* fix format
* use named return value for Get
* add comments
* add more better comment
* format
* add outline for ECRepairer
* add description of process in TODO comments
* begin download/getting hash for a single piece
* verify piece hash and order limit during download
* fix download piece
* begin filling out ESREpair. Get
* wip move ecclient.Repair to ecrepairer.Repair
* pass satellite signee into repairer
* reconstruct original stripe from pieces
* move rebuildStripe()
* calculate piece size differently, increment successful count
* fix shares slices initialization
* rename stripeData to segment
* do not pad reader in Repair()
* temp debug
* create unsafeRSScheme
* use decode reader
* rename file name to be all lowercase
* make repair downloader async
* declare condition variable inside Get method
* set downloadAndVerifyPiece's in-memory buffer to be share size
* update unusedLimits var
* address comments
* remove unnecessary comments
* move initialization of segmentRepaire to be outside of repairer service
* use ReadAll during download
* remove dots and move hashing to after validating for order limit signature
* wip test
* make sure files exactly at min threshold are repaired
* remove unused code
* use corrput data and write back to storagenode
* only create corrupted node and piece ids once
* add comment
* address nat's comment
* fix linting and checker_test
* update comment
* add comments
* remove "copied from ecclient" comments
* add clarification comments in ec.Repair
* Added batch update stats for recordAuditSuccessStatus
* Added batch update stats to recordAuditFailStatus
* added configurable batch size
* build individual update/delete statements so the statements can be batched into 1 call to the DB
* notified #config-changes channel and ran make update-satellite-config-lock
* updated tests to use batch update stats
* rename pkg/linksharing to linksharing
* rename pkg/httpserver to linksharing/httpserver
* rename pkg/eestream to uplink/eestream
* rename pkg/stream to uplink/stream
* rename pkg/metainfo/kvmetainfo to uplink/metainfo/kvmetainfo
* rename pkg/auth/signing to pkg/signing
* rename pkg/storage to uplink/storage
* rename pkg/accounting to satellite/accounting
* rename pkg/audit to satellite/audit
* rename pkg/certdb to satellite/certdb
* rename pkg/discovery to satellite/discovery
* rename pkg/overlay to satellite/overlay
* rename pkg/datarepair to satellite/repair