storj/storagenode/orders/service.go

510 lines
14 KiB
Go
Raw Normal View History

// Copyright (C) 2019 Storj Labs, Inc.
// See LICENSE for copying information.
package orders
import (
"context"
"errors"
2019-03-21 13:24:26 +00:00
"io"
"math/rand"
"sync"
"time"
"github.com/spacemonkeygo/monkit/v3"
"github.com/zeebo/errs"
"go.uber.org/zap"
2019-03-21 13:24:26 +00:00
"golang.org/x/sync/errgroup"
"storj.io/common/pb"
"storj.io/common/rpc"
"storj.io/common/storj"
"storj.io/common/sync2"
"storj.io/storj/storagenode/orders/ordersfile"
"storj.io/storj/storagenode/trust"
)
var (
// OrderError represents errors with orders.
OrderError = errs.Class("order")
// OrderNotFoundError is the error returned when an order is not found.
OrderNotFoundError = errs.Class("order not found")
mon = monkit.Package()
)
2019-03-21 13:24:26 +00:00
// ArchivedInfo contains full information about an archived order.
type ArchivedInfo struct {
Limit *pb.OrderLimit
Order *pb.Order
2019-03-21 13:24:26 +00:00
Status Status
ArchivedAt time.Time
}
// Status is the archival status of the order.
type Status byte
// Statuses for satellite responses.
const (
StatusUnsent Status = iota
StatusAccepted
StatusRejected
)
// ArchiveRequest defines arguments for archiving a single order.
type ArchiveRequest struct {
Satellite storj.NodeID
Serial storj.SerialNumber
Status Status
}
// DB implements storing orders for sending to the satellite.
2019-09-10 14:24:16 +01:00
//
// architecture: Database
type DB interface {
// Enqueue inserts order to the list of orders needing to be sent to the satellite.
Enqueue(ctx context.Context, info *ordersfile.Info) error
// ListUnsent returns orders that haven't been sent yet.
ListUnsent(ctx context.Context, limit int) ([]*ordersfile.Info, error)
2019-03-21 13:24:26 +00:00
// ListUnsentBySatellite returns orders that haven't been sent yet grouped by satellite.
ListUnsentBySatellite(ctx context.Context) (map[storj.NodeID][]*ordersfile.Info, error)
2019-03-21 13:24:26 +00:00
// Archive marks order as being handled.
Archive(ctx context.Context, archivedAt time.Time, requests ...ArchiveRequest) error
2019-03-21 13:24:26 +00:00
// ListArchived returns orders that have been sent.
ListArchived(ctx context.Context, limit int) ([]*ArchivedInfo, error)
storagenode: live tracking of order window usage This change accomplishes multiple things: 1. Instead of having a max in flight time, which means we effectively have a minimum bandwidth for uploads and downloads, we keep track of what windows have active requests happening in them. 2. We don't double check when we save the order to see if it is too old: by then, it's too late. A malicious uplink could just submit orders outside of the grace window and receive all the data, but the node would just not commit it, so the uplink gets free traffic. Because the endpoints also check for the order being too old, this would be a very tight race that depends on knowledge of the node system clock, but best to not have the race exist. Instead, we piggy back off of the in flight tracking and do the check when we start to handle the order, and commit at the end. 3. Change the functions that send orders and list unsent orders to accept a time at which that operation is happening. This way, in tests, we can pretend we're listing or sending far into the future after the windows are available to send, rather than exposing test functions to modify internal state about the grace period to get the desired effect. This brings tests closer to actual usage in production. 4. Change the calculation for if an order is allowed to be enqueued due to the grace period to just look at the order creation time, rather than some computation involving the window it will be in. In this way, you can easily answer the question of "will this order be accepted?" by asking "is it older than X?" where X is the grace period. 5. Increases the frequency we check to send up orders to once every 5 minutes instead of once every hour because we already have hour-long buffering due to the windows. This decreases the maximum latency that an order will be reported back to the satellite by 55 minutes. Change-Id: Ie08b90d139d45ee89b82347e191a2f8db1b88036
2020-08-12 20:01:43 +01:00
// CleanArchive deletes all entries older than the before time.
CleanArchive(ctx context.Context, deleteBefore time.Time) (int, error)
}
// Config defines configuration for sending orders.
type Config struct {
storagenode: live tracking of order window usage This change accomplishes multiple things: 1. Instead of having a max in flight time, which means we effectively have a minimum bandwidth for uploads and downloads, we keep track of what windows have active requests happening in them. 2. We don't double check when we save the order to see if it is too old: by then, it's too late. A malicious uplink could just submit orders outside of the grace window and receive all the data, but the node would just not commit it, so the uplink gets free traffic. Because the endpoints also check for the order being too old, this would be a very tight race that depends on knowledge of the node system clock, but best to not have the race exist. Instead, we piggy back off of the in flight tracking and do the check when we start to handle the order, and commit at the end. 3. Change the functions that send orders and list unsent orders to accept a time at which that operation is happening. This way, in tests, we can pretend we're listing or sending far into the future after the windows are available to send, rather than exposing test functions to modify internal state about the grace period to get the desired effect. This brings tests closer to actual usage in production. 4. Change the calculation for if an order is allowed to be enqueued due to the grace period to just look at the order creation time, rather than some computation involving the window it will be in. In this way, you can easily answer the question of "will this order be accepted?" by asking "is it older than X?" where X is the grace period. 5. Increases the frequency we check to send up orders to once every 5 minutes instead of once every hour because we already have hour-long buffering due to the windows. This decreases the maximum latency that an order will be reported back to the satellite by 55 minutes. Change-Id: Ie08b90d139d45ee89b82347e191a2f8db1b88036
2020-08-12 20:01:43 +01:00
MaxSleep time.Duration `help:"maximum duration to wait before trying to send orders" releaseDefault:"30s" devDefault:"1s"`
SenderInterval time.Duration `help:"duration between sending" releaseDefault:"1h0m0s" devDefault:"30s"`
SenderTimeout time.Duration `help:"timeout for sending" default:"1h0m0s"`
SenderDialTimeout time.Duration `help:"timeout for dialing satellite during sending orders" default:"1m0s"`
storagenode: live tracking of order window usage This change accomplishes multiple things: 1. Instead of having a max in flight time, which means we effectively have a minimum bandwidth for uploads and downloads, we keep track of what windows have active requests happening in them. 2. We don't double check when we save the order to see if it is too old: by then, it's too late. A malicious uplink could just submit orders outside of the grace window and receive all the data, but the node would just not commit it, so the uplink gets free traffic. Because the endpoints also check for the order being too old, this would be a very tight race that depends on knowledge of the node system clock, but best to not have the race exist. Instead, we piggy back off of the in flight tracking and do the check when we start to handle the order, and commit at the end. 3. Change the functions that send orders and list unsent orders to accept a time at which that operation is happening. This way, in tests, we can pretend we're listing or sending far into the future after the windows are available to send, rather than exposing test functions to modify internal state about the grace period to get the desired effect. This brings tests closer to actual usage in production. 4. Change the calculation for if an order is allowed to be enqueued due to the grace period to just look at the order creation time, rather than some computation involving the window it will be in. In this way, you can easily answer the question of "will this order be accepted?" by asking "is it older than X?" where X is the grace period. 5. Increases the frequency we check to send up orders to once every 5 minutes instead of once every hour because we already have hour-long buffering due to the windows. This decreases the maximum latency that an order will be reported back to the satellite by 55 minutes. Change-Id: Ie08b90d139d45ee89b82347e191a2f8db1b88036
2020-08-12 20:01:43 +01:00
CleanupInterval time.Duration `help:"duration between archive cleanups" default:"5m0s"`
ArchiveTTL time.Duration `help:"length of time to archive orders before deletion" default:"168h0m0s"` // 7 days
Path string `help:"path to store order limit files in" default:"$CONFDIR/orders"`
}
// Service sends every interval unsent orders to the satellite.
2019-09-10 14:24:16 +01:00
//
// architecture: Chore
type Service struct {
log *zap.Logger
config Config
dialer rpc.Dialer
ordersStore *FileStore
orders DB
trust *trust.Pool
2019-03-21 13:24:26 +00:00
Sender *sync2.Cycle
Cleanup *sync2.Cycle
}
// NewService creates an order service.
func NewService(log *zap.Logger, dialer rpc.Dialer, ordersStore *FileStore, orders DB, trust *trust.Pool, config Config) *Service {
return &Service{
log: log,
dialer: dialer,
ordersStore: ordersStore,
orders: orders,
config: config,
trust: trust,
2019-03-21 13:24:26 +00:00
Sender: sync2.NewCycle(config.SenderInterval),
Cleanup: sync2.NewCycle(config.CleanupInterval),
}
}
2019-03-21 13:24:26 +00:00
// Run sends orders on every interval to the appropriate satellites.
func (service *Service) Run(ctx context.Context) (err error) {
defer mon.Task()(&ctx)(&err)
var group errgroup.Group
service.Sender.Start(ctx, &group, func(ctx context.Context) error {
if err := service.sleep(ctx); err != nil {
return err
}
storagenode: live tracking of order window usage This change accomplishes multiple things: 1. Instead of having a max in flight time, which means we effectively have a minimum bandwidth for uploads and downloads, we keep track of what windows have active requests happening in them. 2. We don't double check when we save the order to see if it is too old: by then, it's too late. A malicious uplink could just submit orders outside of the grace window and receive all the data, but the node would just not commit it, so the uplink gets free traffic. Because the endpoints also check for the order being too old, this would be a very tight race that depends on knowledge of the node system clock, but best to not have the race exist. Instead, we piggy back off of the in flight tracking and do the check when we start to handle the order, and commit at the end. 3. Change the functions that send orders and list unsent orders to accept a time at which that operation is happening. This way, in tests, we can pretend we're listing or sending far into the future after the windows are available to send, rather than exposing test functions to modify internal state about the grace period to get the desired effect. This brings tests closer to actual usage in production. 4. Change the calculation for if an order is allowed to be enqueued due to the grace period to just look at the order creation time, rather than some computation involving the window it will be in. In this way, you can easily answer the question of "will this order be accepted?" by asking "is it older than X?" where X is the grace period. 5. Increases the frequency we check to send up orders to once every 5 minutes instead of once every hour because we already have hour-long buffering due to the windows. This decreases the maximum latency that an order will be reported back to the satellite by 55 minutes. Change-Id: Ie08b90d139d45ee89b82347e191a2f8db1b88036
2020-08-12 20:01:43 +01:00
service.SendOrders(ctx, time.Now())
return nil
})
service.Cleanup.Start(ctx, &group, func(ctx context.Context) error {
if err := service.sleep(ctx); err != nil {
return err
}
storagenode: live tracking of order window usage This change accomplishes multiple things: 1. Instead of having a max in flight time, which means we effectively have a minimum bandwidth for uploads and downloads, we keep track of what windows have active requests happening in them. 2. We don't double check when we save the order to see if it is too old: by then, it's too late. A malicious uplink could just submit orders outside of the grace window and receive all the data, but the node would just not commit it, so the uplink gets free traffic. Because the endpoints also check for the order being too old, this would be a very tight race that depends on knowledge of the node system clock, but best to not have the race exist. Instead, we piggy back off of the in flight tracking and do the check when we start to handle the order, and commit at the end. 3. Change the functions that send orders and list unsent orders to accept a time at which that operation is happening. This way, in tests, we can pretend we're listing or sending far into the future after the windows are available to send, rather than exposing test functions to modify internal state about the grace period to get the desired effect. This brings tests closer to actual usage in production. 4. Change the calculation for if an order is allowed to be enqueued due to the grace period to just look at the order creation time, rather than some computation involving the window it will be in. In this way, you can easily answer the question of "will this order be accepted?" by asking "is it older than X?" where X is the grace period. 5. Increases the frequency we check to send up orders to once every 5 minutes instead of once every hour because we already have hour-long buffering due to the windows. This decreases the maximum latency that an order will be reported back to the satellite by 55 minutes. Change-Id: Ie08b90d139d45ee89b82347e191a2f8db1b88036
2020-08-12 20:01:43 +01:00
err := service.CleanArchive(ctx, time.Now().Add(-service.config.ArchiveTTL))
if err != nil {
service.log.Error("clean archive failed", zap.Error(err))
}
return nil
})
return group.Wait()
}
storagenode: live tracking of order window usage This change accomplishes multiple things: 1. Instead of having a max in flight time, which means we effectively have a minimum bandwidth for uploads and downloads, we keep track of what windows have active requests happening in them. 2. We don't double check when we save the order to see if it is too old: by then, it's too late. A malicious uplink could just submit orders outside of the grace window and receive all the data, but the node would just not commit it, so the uplink gets free traffic. Because the endpoints also check for the order being too old, this would be a very tight race that depends on knowledge of the node system clock, but best to not have the race exist. Instead, we piggy back off of the in flight tracking and do the check when we start to handle the order, and commit at the end. 3. Change the functions that send orders and list unsent orders to accept a time at which that operation is happening. This way, in tests, we can pretend we're listing or sending far into the future after the windows are available to send, rather than exposing test functions to modify internal state about the grace period to get the desired effect. This brings tests closer to actual usage in production. 4. Change the calculation for if an order is allowed to be enqueued due to the grace period to just look at the order creation time, rather than some computation involving the window it will be in. In this way, you can easily answer the question of "will this order be accepted?" by asking "is it older than X?" where X is the grace period. 5. Increases the frequency we check to send up orders to once every 5 minutes instead of once every hour because we already have hour-long buffering due to the windows. This decreases the maximum latency that an order will be reported back to the satellite by 55 minutes. Change-Id: Ie08b90d139d45ee89b82347e191a2f8db1b88036
2020-08-12 20:01:43 +01:00
// CleanArchive removes all archived orders that were archived before the deleteBefore time.
func (service *Service) CleanArchive(ctx context.Context, deleteBefore time.Time) (err error) {
defer mon.Task()(&ctx)(&err)
service.log.Debug("cleaning")
storagenode: live tracking of order window usage This change accomplishes multiple things: 1. Instead of having a max in flight time, which means we effectively have a minimum bandwidth for uploads and downloads, we keep track of what windows have active requests happening in them. 2. We don't double check when we save the order to see if it is too old: by then, it's too late. A malicious uplink could just submit orders outside of the grace window and receive all the data, but the node would just not commit it, so the uplink gets free traffic. Because the endpoints also check for the order being too old, this would be a very tight race that depends on knowledge of the node system clock, but best to not have the race exist. Instead, we piggy back off of the in flight tracking and do the check when we start to handle the order, and commit at the end. 3. Change the functions that send orders and list unsent orders to accept a time at which that operation is happening. This way, in tests, we can pretend we're listing or sending far into the future after the windows are available to send, rather than exposing test functions to modify internal state about the grace period to get the desired effect. This brings tests closer to actual usage in production. 4. Change the calculation for if an order is allowed to be enqueued due to the grace period to just look at the order creation time, rather than some computation involving the window it will be in. In this way, you can easily answer the question of "will this order be accepted?" by asking "is it older than X?" where X is the grace period. 5. Increases the frequency we check to send up orders to once every 5 minutes instead of once every hour because we already have hour-long buffering due to the windows. This decreases the maximum latency that an order will be reported back to the satellite by 55 minutes. Change-Id: Ie08b90d139d45ee89b82347e191a2f8db1b88036
2020-08-12 20:01:43 +01:00
deleted, err := service.orders.CleanArchive(ctx, deleteBefore)
if err != nil {
service.log.Error("cleaning DB archive", zap.Error(err))
return nil
}
storagenode: live tracking of order window usage This change accomplishes multiple things: 1. Instead of having a max in flight time, which means we effectively have a minimum bandwidth for uploads and downloads, we keep track of what windows have active requests happening in them. 2. We don't double check when we save the order to see if it is too old: by then, it's too late. A malicious uplink could just submit orders outside of the grace window and receive all the data, but the node would just not commit it, so the uplink gets free traffic. Because the endpoints also check for the order being too old, this would be a very tight race that depends on knowledge of the node system clock, but best to not have the race exist. Instead, we piggy back off of the in flight tracking and do the check when we start to handle the order, and commit at the end. 3. Change the functions that send orders and list unsent orders to accept a time at which that operation is happening. This way, in tests, we can pretend we're listing or sending far into the future after the windows are available to send, rather than exposing test functions to modify internal state about the grace period to get the desired effect. This brings tests closer to actual usage in production. 4. Change the calculation for if an order is allowed to be enqueued due to the grace period to just look at the order creation time, rather than some computation involving the window it will be in. In this way, you can easily answer the question of "will this order be accepted?" by asking "is it older than X?" where X is the grace period. 5. Increases the frequency we check to send up orders to once every 5 minutes instead of once every hour because we already have hour-long buffering due to the windows. This decreases the maximum latency that an order will be reported back to the satellite by 55 minutes. Change-Id: Ie08b90d139d45ee89b82347e191a2f8db1b88036
2020-08-12 20:01:43 +01:00
err = service.ordersStore.CleanArchive(deleteBefore)
if err != nil {
service.log.Error("cleaning filestore archive", zap.Error(err))
return nil
}
service.log.Debug("cleanup finished", zap.Int("items deleted", deleted))
return nil
}
2019-03-21 13:24:26 +00:00
storagenode: live tracking of order window usage This change accomplishes multiple things: 1. Instead of having a max in flight time, which means we effectively have a minimum bandwidth for uploads and downloads, we keep track of what windows have active requests happening in them. 2. We don't double check when we save the order to see if it is too old: by then, it's too late. A malicious uplink could just submit orders outside of the grace window and receive all the data, but the node would just not commit it, so the uplink gets free traffic. Because the endpoints also check for the order being too old, this would be a very tight race that depends on knowledge of the node system clock, but best to not have the race exist. Instead, we piggy back off of the in flight tracking and do the check when we start to handle the order, and commit at the end. 3. Change the functions that send orders and list unsent orders to accept a time at which that operation is happening. This way, in tests, we can pretend we're listing or sending far into the future after the windows are available to send, rather than exposing test functions to modify internal state about the grace period to get the desired effect. This brings tests closer to actual usage in production. 4. Change the calculation for if an order is allowed to be enqueued due to the grace period to just look at the order creation time, rather than some computation involving the window it will be in. In this way, you can easily answer the question of "will this order be accepted?" by asking "is it older than X?" where X is the grace period. 5. Increases the frequency we check to send up orders to once every 5 minutes instead of once every hour because we already have hour-long buffering due to the windows. This decreases the maximum latency that an order will be reported back to the satellite by 55 minutes. Change-Id: Ie08b90d139d45ee89b82347e191a2f8db1b88036
2020-08-12 20:01:43 +01:00
// SendOrders sends the orders using now as the current time.
func (service *Service) SendOrders(ctx context.Context, now time.Time) {
defer mon.Task()(&ctx)(nil)
service.log.Debug("sending")
2019-03-21 13:24:26 +00:00
// If there are orders in the database, send from there.
// Otherwise, send from the filestore.
service.sendOrdersFromDB(ctx)
storagenode: live tracking of order window usage This change accomplishes multiple things: 1. Instead of having a max in flight time, which means we effectively have a minimum bandwidth for uploads and downloads, we keep track of what windows have active requests happening in them. 2. We don't double check when we save the order to see if it is too old: by then, it's too late. A malicious uplink could just submit orders outside of the grace window and receive all the data, but the node would just not commit it, so the uplink gets free traffic. Because the endpoints also check for the order being too old, this would be a very tight race that depends on knowledge of the node system clock, but best to not have the race exist. Instead, we piggy back off of the in flight tracking and do the check when we start to handle the order, and commit at the end. 3. Change the functions that send orders and list unsent orders to accept a time at which that operation is happening. This way, in tests, we can pretend we're listing or sending far into the future after the windows are available to send, rather than exposing test functions to modify internal state about the grace period to get the desired effect. This brings tests closer to actual usage in production. 4. Change the calculation for if an order is allowed to be enqueued due to the grace period to just look at the order creation time, rather than some computation involving the window it will be in. In this way, you can easily answer the question of "will this order be accepted?" by asking "is it older than X?" where X is the grace period. 5. Increases the frequency we check to send up orders to once every 5 minutes instead of once every hour because we already have hour-long buffering due to the windows. This decreases the maximum latency that an order will be reported back to the satellite by 55 minutes. Change-Id: Ie08b90d139d45ee89b82347e191a2f8db1b88036
2020-08-12 20:01:43 +01:00
service.sendOrdersFromFileStore(ctx, now)
}
func (service *Service) sendOrdersFromDB(ctx context.Context) (hasOrders bool) {
defer mon.Task()(&ctx)(nil)
const batchSize = 1000
hasOrders = true
ordersBySatellite, err := service.orders.ListUnsentBySatellite(ctx)
if err != nil {
if ordersBySatellite == nil {
service.log.Error("listing orders", zap.Error(err))
hasOrders = false
return hasOrders
}
service.log.Warn("DB contains invalid marshalled orders", zap.Error(err))
}
2019-03-21 13:24:26 +00:00
requests := make(chan ArchiveRequest, batchSize)
var batchGroup errgroup.Group
batchGroup.Go(func() error { return service.handleBatches(ctx, requests) })
if len(ordersBySatellite) > 0 {
var group errgroup.Group
ctx, cancel := context.WithTimeout(ctx, service.config.SenderTimeout)
defer cancel()
for satelliteID, orders := range ordersBySatellite {
satelliteID, orders := satelliteID, orders
group.Go(func() error {
service.Settle(ctx, satelliteID, orders, requests)
return nil
})
2019-03-21 13:24:26 +00:00
}
_ = group.Wait() // doesn't return errors
} else {
service.log.Debug("no orders to send")
hasOrders = false
}
2019-03-21 13:24:26 +00:00
close(requests)
err = batchGroup.Wait()
if err != nil {
service.log.Error("archiving orders", zap.Error(err))
}
return hasOrders
2019-03-21 13:24:26 +00:00
}
// Settle uploads orders to the satellite.
func (service *Service) Settle(ctx context.Context, satelliteID storj.NodeID, orders []*ordersfile.Info, requests chan ArchiveRequest) {
log := service.log.Named(satelliteID.String())
err := service.settle(ctx, log, satelliteID, orders, requests)
if err != nil {
log.Error("failed to settle orders", zap.Error(err))
}
}
func (service *Service) handleBatches(ctx context.Context, requests chan ArchiveRequest) (err error) {
defer mon.Task()(&ctx)(&err)
// In case anything goes wrong, discard everything from the channel.
defer func() {
for range requests {
}
}()
buffer := make([]ArchiveRequest, 0, cap(requests))
archive := func(ctx context.Context, archivedAt time.Time, requests ...ArchiveRequest) error {
if err := service.orders.Archive(ctx, time.Now().UTC(), buffer...); err != nil {
if !OrderNotFoundError.Has(err) {
return err
}
service.log.Warn("some unsent order aren't in the DB", zap.Error(err))
}
return nil
}
for request := range requests {
buffer = append(buffer, request)
if len(buffer) < cap(buffer) {
continue
}
if err := archive(ctx, time.Now().UTC(), buffer...); err != nil {
return err
}
buffer = buffer[:0]
}
if len(buffer) > 0 {
return archive(ctx, time.Now().UTC(), buffer...)
}
return nil
}
func (service *Service) settle(ctx context.Context, log *zap.Logger, satelliteID storj.NodeID, orders []*ordersfile.Info, requests chan ArchiveRequest) (err error) {
defer mon.Task()(&ctx)(&err)
2019-03-21 13:24:26 +00:00
log.Info("sending", zap.Int("count", len(orders)))
defer log.Info("finished")
nodeurl, err := service.trust.GetNodeURL(ctx, satelliteID)
2019-03-21 13:24:26 +00:00
if err != nil {
return OrderError.New("unable to get satellite address: %w", err)
}
2019-03-21 13:24:26 +00:00
conn, err := service.dialer.DialNodeURL(ctx, nodeurl)
2019-03-21 13:24:26 +00:00
if err != nil {
return OrderError.New("unable to connect to the satellite: %w", err)
2019-03-21 13:24:26 +00:00
}
defer func() { err = errs.Combine(err, conn.Close()) }()
2019-03-21 13:24:26 +00:00
stream, err := pb.NewDRPCOrdersClient(conn).Settlement(ctx)
2019-03-21 13:24:26 +00:00
if err != nil {
return OrderError.New("failed to start settlement: %w", err)
2019-03-21 13:24:26 +00:00
}
var group errgroup.Group
var sendErrors errs.Group
2019-03-21 13:24:26 +00:00
group.Go(func() error {
for _, order := range orders {
req := pb.SettlementRequest{
2019-03-21 13:24:26 +00:00
Limit: order.Limit,
Order: order.Order,
}
err := stream.Send(&req)
2019-03-21 13:24:26 +00:00
if err != nil {
err = OrderError.New("sending settlement agreements returned an error: %w", err)
log.Error("rpc client when sending new orders settlements",
zap.Error(err),
zap.Any("request", req),
)
sendErrors.Add(err)
return nil
2019-03-21 13:24:26 +00:00
}
}
err := stream.CloseSend()
if err != nil {
err = OrderError.New("CloseSend settlement agreements returned an error: %w", err)
log.Error("rpc client error when closing sender ", zap.Error(err))
sendErrors.Add(err)
}
return nil
2019-03-21 13:24:26 +00:00
})
var errList errs.Group
2019-03-21 13:24:26 +00:00
for {
response, err := stream.Recv()
2019-03-21 13:24:26 +00:00
if err != nil {
if errors.Is(err, io.EOF) {
2019-03-21 13:24:26 +00:00
break
}
err = OrderError.New("failed to receive settlement response: %w", err)
log.Error("rpc client error when receiving new order settlements", zap.Error(err))
errList.Add(err)
2019-03-21 13:24:26 +00:00
break
}
var status Status
2019-03-21 13:24:26 +00:00
switch response.Status {
case pb.SettlementResponse_ACCEPTED:
status = StatusAccepted
2019-03-21 13:24:26 +00:00
case pb.SettlementResponse_REJECTED:
status = StatusRejected
2019-03-21 13:24:26 +00:00
default:
err := OrderError.New("unexpected settlement status response: %d", response.Status)
log.Error("rpc client received an unexpected new orders settlement status",
zap.Error(err), zap.Any("response", response),
)
errList.Add(err)
continue
}
requests <- ArchiveRequest{
Satellite: satelliteID,
Serial: response.SerialNumber,
Status: status,
2019-03-21 13:24:26 +00:00
}
}
// errors of this group are reported to sendErrors and it always return nil
_ = group.Wait()
errList.Add(sendErrors...)
return errList.Err()
2019-03-21 13:24:26 +00:00
}
storagenode: live tracking of order window usage This change accomplishes multiple things: 1. Instead of having a max in flight time, which means we effectively have a minimum bandwidth for uploads and downloads, we keep track of what windows have active requests happening in them. 2. We don't double check when we save the order to see if it is too old: by then, it's too late. A malicious uplink could just submit orders outside of the grace window and receive all the data, but the node would just not commit it, so the uplink gets free traffic. Because the endpoints also check for the order being too old, this would be a very tight race that depends on knowledge of the node system clock, but best to not have the race exist. Instead, we piggy back off of the in flight tracking and do the check when we start to handle the order, and commit at the end. 3. Change the functions that send orders and list unsent orders to accept a time at which that operation is happening. This way, in tests, we can pretend we're listing or sending far into the future after the windows are available to send, rather than exposing test functions to modify internal state about the grace period to get the desired effect. This brings tests closer to actual usage in production. 4. Change the calculation for if an order is allowed to be enqueued due to the grace period to just look at the order creation time, rather than some computation involving the window it will be in. In this way, you can easily answer the question of "will this order be accepted?" by asking "is it older than X?" where X is the grace period. 5. Increases the frequency we check to send up orders to once every 5 minutes instead of once every hour because we already have hour-long buffering due to the windows. This decreases the maximum latency that an order will be reported back to the satellite by 55 minutes. Change-Id: Ie08b90d139d45ee89b82347e191a2f8db1b88036
2020-08-12 20:01:43 +01:00
func (service *Service) sendOrdersFromFileStore(ctx context.Context, now time.Time) {
defer mon.Task()(&ctx)(nil)
errorSatellites := make(map[storj.NodeID]struct{})
var errorSatellitesMu sync.Mutex
// Continue sending until there are no more windows to send, or all relevant satellites are offline.
for {
ordersBySatellite, err := service.ordersStore.ListUnsentBySatellite(ctx, now)
if err != nil {
service.log.Error("listing orders", zap.Error(err))
}
if len(ordersBySatellite) == 0 {
service.log.Debug("no orders to send")
break
}
var group errgroup.Group
attemptedSatellites := 0
ctx, cancel := context.WithTimeout(ctx, service.config.SenderTimeout)
defer cancel()
for satelliteID, unsentInfo := range ordersBySatellite {
satelliteID, unsentInfo := satelliteID, unsentInfo
if _, ok := errorSatellites[satelliteID]; ok {
continue
}
attemptedSatellites++
group.Go(func() error {
log := service.log.Named(satelliteID.String())
status, err := service.settleWindow(ctx, log, satelliteID, unsentInfo.InfoList)
if err != nil {
// satellite returned an error, but settlement was not explicitly rejected; we want to retry later
errorSatellitesMu.Lock()
errorSatellites[satelliteID] = struct{}{}
errorSatellitesMu.Unlock()
log.Error("failed to settle orders for satellite", zap.String("satellite ID", satelliteID.String()), zap.Error(err))
return nil
}
err = service.ordersStore.Archive(satelliteID, unsentInfo, time.Now().UTC(), status)
if err != nil {
log.Error("failed to archive orders", zap.Error(err))
return nil
}
return nil
})
}
_ = group.Wait() // doesn't return errors
// if all satellites that orders need to be sent to are offline, exit and try again later.
if attemptedSatellites == 0 {
break
}
}
}
func (service *Service) settleWindow(ctx context.Context, log *zap.Logger, satelliteID storj.NodeID, orders []*ordersfile.Info) (status pb.SettlementWithWindowResponse_Status, err error) {
defer mon.Task()(&ctx)(&err)
log.Info("sending", zap.Int("count", len(orders)))
defer log.Info("finished")
nodeurl, err := service.trust.GetNodeURL(ctx, satelliteID)
if err != nil {
return 0, OrderError.New("unable to get satellite address: %w", err)
}
conn, err := service.dialer.DialNodeURL(ctx, nodeurl)
if err != nil {
return 0, OrderError.New("unable to connect to the satellite: %w", err)
}
defer func() { err = errs.Combine(err, conn.Close()) }()
stream, err := pb.NewDRPCOrdersClient(conn).SettlementWithWindow(ctx)
if err != nil {
return 0, OrderError.New("failed to start settlement: %w", err)
}
for _, order := range orders {
req := pb.SettlementRequest{
Limit: order.Limit,
Order: order.Order,
}
err := stream.Send(&req)
if err != nil {
err = OrderError.New("sending settlement agreements returned an error: %w", err)
log.Error("rpc client when sending new orders settlements",
zap.Error(err),
zap.Any("request", req),
)
return 0, err
}
}
res, err := stream.CloseAndRecv()
if err != nil {
err = OrderError.New("CloseAndRecv settlement agreements returned an error: %w", err)
log.Error("rpc client error when closing sender ", zap.Error(err))
return 0, err
}
return res.Status, nil
}
// sleep for random interval in [0;maxSleep).
// Returns an error if context was cancelled.
func (service *Service) sleep(ctx context.Context) error {
if service.config.MaxSleep <= 0 {
return nil
}
jitter := time.Duration(rand.Int63n(int64(service.config.MaxSleep)))
if !sync2.Sleep(ctx, jitter) {
return ctx.Err()
}
return nil
}
2019-03-21 13:24:26 +00:00
// Close stops the sending service.
func (service *Service) Close() error {
service.Sender.Close()
service.Cleanup.Close()
2019-03-21 13:24:26 +00:00
return nil
}