satellite/satellitedb: reported_serials table for processing orders
this commit introduces the reported_serials table. its purpose is to allow for blind writes into it as nodes report in so that we have minimal contention. in order to continue to accurately account for used bandwidth, though, we cannot immediately add the settled amount. if we did, we would have to give up on blind writes. the table's primary key is structured precisely so that we can quickly find expired orders and so that we maximally benefit from rocksdb path prefix compression. we do this by rounding the expires at time forward to the next day, effectively giving us storagenode petnames for free. and since there's no secondary index or foreign key constraints, this design should use significantly less space than the current used_serials table while also reducing contention. after inserting the orders into the table, we have a chore that periodically consumes all of the expired orders in it and inserts them into the existing rollups tables. this is as if we changed the nodes to report as the order expired rather than as soon as possible, so the belief in correctness of the refactor is higher. since we are able to process large batches of orders (typically a day's worth), we can use the code to maximally batch inserts into the rollup tables to make inserts as friendly as possible to cockroach. Change-Id: I25d609ca2679b8331979184f16c6d46d4f74c1a6
This commit is contained in:
parent
9da16b1d9e
commit
78c6d5bb32
@ -188,7 +188,9 @@ func cmdRun(cmd *cobra.Command, args []string) (err error) {
|
||||
zap.S().Fatal(err)
|
||||
}
|
||||
|
||||
db, err := satellitedb.New(log.Named("db"), runCfg.Database, satellitedb.Options{})
|
||||
db, err := satellitedb.New(log.Named("db"), runCfg.Database, satellitedb.Options{
|
||||
ReportedRollupsReadBatchSize: runCfg.Orders.SettlementBatchSize,
|
||||
})
|
||||
if err != nil {
|
||||
return errs.New("Error starting master database on satellite: %+v", err)
|
||||
}
|
||||
|
@ -33,6 +33,7 @@ import (
|
||||
"storj.io/storj/satellite"
|
||||
"storj.io/storj/satellite/accounting"
|
||||
"storj.io/storj/satellite/accounting/live"
|
||||
"storj.io/storj/satellite/accounting/reportedrollup"
|
||||
"storj.io/storj/satellite/accounting/rollup"
|
||||
"storj.io/storj/satellite/accounting/tally"
|
||||
"storj.io/storj/satellite/audit"
|
||||
@ -125,9 +126,10 @@ type SatelliteSystem struct {
|
||||
}
|
||||
|
||||
Accounting struct {
|
||||
Tally *tally.Service
|
||||
Rollup *rollup.Service
|
||||
ProjectUsage *accounting.Service
|
||||
Tally *tally.Service
|
||||
Rollup *rollup.Service
|
||||
ProjectUsage *accounting.Service
|
||||
ReportedRollup *reportedrollup.Chore
|
||||
}
|
||||
|
||||
LiveAccounting struct {
|
||||
@ -368,6 +370,9 @@ func (planet *Planet) newSatellites(count int) ([]*SatelliteSystem, error) {
|
||||
MaxAlphaUsage: 25 * memory.GB,
|
||||
DeleteTallies: false,
|
||||
},
|
||||
ReportedRollup: reportedrollup.Config{
|
||||
Interval: defaultInterval,
|
||||
},
|
||||
Mail: mailservice.Config{
|
||||
SMTPServerAddress: "smtp.mail.test:587",
|
||||
From: "Labs <storj@mail.test>",
|
||||
@ -517,6 +522,7 @@ func createNewSystem(log *zap.Logger, peer *satellite.Core, api *satellite.API,
|
||||
system.Accounting.Tally = peer.Accounting.Tally
|
||||
system.Accounting.Rollup = peer.Accounting.Rollup
|
||||
system.Accounting.ProjectUsage = peer.Accounting.ProjectUsage
|
||||
system.Accounting.ReportedRollup = peer.Accounting.ReportedRollupChore
|
||||
|
||||
system.Marketing.Listener = api.Marketing.Listener
|
||||
system.Marketing.Endpoint = api.Marketing.Endpoint
|
||||
|
87
satellite/accounting/reportedrollup/chore.go
Normal file
87
satellite/accounting/reportedrollup/chore.go
Normal file
@ -0,0 +1,87 @@
|
||||
// Copyright (C) 2019 Storj Labs, Inc.
|
||||
// See LICENSE for copying information.
|
||||
|
||||
package reportedrollup
|
||||
|
||||
import (
|
||||
"context"
|
||||
"time"
|
||||
|
||||
"github.com/zeebo/errs"
|
||||
"go.uber.org/zap"
|
||||
"gopkg.in/spacemonkeygo/monkit.v2"
|
||||
|
||||
"storj.io/common/sync2"
|
||||
"storj.io/storj/satellite/orders"
|
||||
)
|
||||
|
||||
var (
|
||||
mon = monkit.Package()
|
||||
|
||||
// Error is the error class for this package
|
||||
Error = errs.Class("reportedrollup")
|
||||
)
|
||||
|
||||
// Config is a configuration struct for the Core.
|
||||
type Config struct {
|
||||
Interval time.Duration `help:"how often to flush the reported serial rollups to the database" devDefault:"5m" releaseDefault:"24h"`
|
||||
}
|
||||
|
||||
// Chore for flushing reported serials to the database as rollups.
|
||||
//
|
||||
// architecture: Chore
|
||||
type Chore struct {
|
||||
log *zap.Logger
|
||||
db orders.DB
|
||||
Loop *sync2.Cycle
|
||||
}
|
||||
|
||||
// NewChore creates new chore for flushing the reported serials to the database as rollups.
|
||||
func NewChore(log *zap.Logger, db orders.DB, config Config) *Chore {
|
||||
return &Chore{
|
||||
log: log,
|
||||
db: db,
|
||||
Loop: sync2.NewCycle(config.Interval),
|
||||
}
|
||||
}
|
||||
|
||||
// Run starts the reported rollups chore.
|
||||
func (chore *Chore) Run(ctx context.Context) (err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
return chore.Loop.Run(ctx, func(ctx context.Context) error {
|
||||
err := chore.RunOnce(ctx, time.Now())
|
||||
if err != nil {
|
||||
chore.log.Error("error flushing reported rollups", zap.Error(err))
|
||||
}
|
||||
return nil
|
||||
})
|
||||
}
|
||||
|
||||
// Close stops the reported rollups chore.
|
||||
func (chore *Chore) Close() error {
|
||||
chore.Loop.Close()
|
||||
return nil
|
||||
}
|
||||
|
||||
// RunOnce finds expired bandwidth as of 'now' and inserts rollups into the appropriate tables.
|
||||
func (chore *Chore) RunOnce(ctx context.Context, now time.Time) (err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
|
||||
bucketRollups, storagenodeRollups, err := chore.db.GetBillableBandwidth(ctx, now)
|
||||
if err != nil {
|
||||
return err
|
||||
}
|
||||
|
||||
return Error.Wrap(chore.db.ExecuteInTx(ctx, func(ctx context.Context, tx orders.Transaction) error {
|
||||
if err := tx.UpdateBucketBandwidthBatch(ctx, now, bucketRollups); err != nil {
|
||||
return Error.Wrap(err)
|
||||
}
|
||||
if err := tx.UpdateStoragenodeBandwidthBatch(ctx, now, storagenodeRollups); err != nil {
|
||||
return Error.Wrap(err)
|
||||
}
|
||||
if err := tx.DeleteExpiredReportedSerials(ctx, now); err != nil {
|
||||
return Error.Wrap(err)
|
||||
}
|
||||
return nil
|
||||
}))
|
||||
}
|
@ -21,6 +21,7 @@ import (
|
||||
"storj.io/storj/private/version"
|
||||
version_checker "storj.io/storj/private/version/checker"
|
||||
"storj.io/storj/satellite/accounting"
|
||||
"storj.io/storj/satellite/accounting/reportedrollup"
|
||||
"storj.io/storj/satellite/accounting/rollup"
|
||||
"storj.io/storj/satellite/accounting/tally"
|
||||
"storj.io/storj/satellite/audit"
|
||||
@ -96,9 +97,10 @@ type Core struct {
|
||||
}
|
||||
|
||||
Accounting struct {
|
||||
Tally *tally.Service
|
||||
Rollup *rollup.Service
|
||||
ProjectUsage *accounting.Service
|
||||
Tally *tally.Service
|
||||
Rollup *rollup.Service
|
||||
ProjectUsage *accounting.Service
|
||||
ReportedRollupChore *reportedrollup.Chore
|
||||
}
|
||||
|
||||
LiveAccounting struct {
|
||||
@ -308,6 +310,7 @@ func New(log *zap.Logger, full *identity.FullIdentity, db DB, pointerDB metainfo
|
||||
{ // setup accounting
|
||||
peer.Accounting.Tally = tally.New(peer.Log.Named("tally"), peer.DB.StoragenodeAccounting(), peer.DB.ProjectAccounting(), peer.LiveAccounting.Cache, peer.Metainfo.Loop, config.Tally.Interval)
|
||||
peer.Accounting.Rollup = rollup.New(peer.Log.Named("rollup"), peer.DB.StoragenodeAccounting(), config.Rollup.Interval, config.Rollup.DeleteTallies)
|
||||
peer.Accounting.ReportedRollupChore = reportedrollup.NewChore(peer.Log.Named("reportedrollup"), peer.DB.Orders(), config.ReportedRollup)
|
||||
}
|
||||
|
||||
// TODO: remove in future, should be in API
|
||||
@ -405,6 +408,9 @@ func (peer *Core) Run(ctx context.Context) (err error) {
|
||||
group.Go(func() error {
|
||||
return errs2.IgnoreCanceled(peer.Accounting.Rollup.Run(ctx))
|
||||
})
|
||||
group.Go(func() error {
|
||||
return errs2.IgnoreCanceled(peer.Accounting.ReportedRollupChore.Run(ctx))
|
||||
})
|
||||
group.Go(func() error {
|
||||
return errs2.IgnoreCanceled(peer.Audit.Worker.Run(ctx))
|
||||
})
|
||||
@ -473,6 +479,9 @@ func (peer *Core) Close() error {
|
||||
errlist.Add(peer.Audit.Worker.Close())
|
||||
}
|
||||
|
||||
if peer.Accounting.Rollup != nil {
|
||||
errlist.Add(peer.Accounting.ReportedRollupChore.Close())
|
||||
}
|
||||
if peer.Accounting.Rollup != nil {
|
||||
errlist.Add(peer.Accounting.Rollup.Close())
|
||||
}
|
||||
|
@ -4,8 +4,10 @@
|
||||
package orders
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"io"
|
||||
"sort"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
@ -52,17 +54,21 @@ type DB interface {
|
||||
// ProcessOrders takes a list of order requests and processes them in a batch
|
||||
ProcessOrders(ctx context.Context, requests []*ProcessOrderRequest) (responses []*ProcessOrderResponse, err error)
|
||||
|
||||
// UpdateBucketBandwidthBatch updates all the bandwidth rollups in the database
|
||||
UpdateBucketBandwidthBatch(ctx context.Context, intervalStart time.Time, rollups []BandwidthRollup) error
|
||||
// GetBillableBandwidth gets total billable (expired reported serial) bandwidth for nodes and buckets for all actions.
|
||||
GetBillableBandwidth(ctx context.Context, now time.Time) (bucketRollups []BucketBandwidthRollup, storagenodeRollups []StoragenodeBandwidthRollup, err error)
|
||||
|
||||
// ExecuteInTx runs the callback and provides it with a Transaction.
|
||||
ExecuteInTx(ctx context.Context, cb func(ctx context.Context, tx Transaction) error) error
|
||||
}
|
||||
|
||||
// BandwidthRollup contains all the info needed for a bucket bandwidth rollup
|
||||
type BandwidthRollup struct {
|
||||
ProjectID uuid.UUID
|
||||
BucketName string
|
||||
Action pb.PieceAction
|
||||
Inline int64
|
||||
Allocated int64
|
||||
// Transaction represents a database transaction but with higher level actions.
|
||||
type Transaction interface {
|
||||
// UpdateBucketBandwidthBatch updates all the bandwidth rollups in the database
|
||||
UpdateBucketBandwidthBatch(ctx context.Context, intervalStart time.Time, rollups []BucketBandwidthRollup) error
|
||||
// UpdateStoragenodeBandwidthBatch updates all the bandwidth rollups in the database
|
||||
UpdateStoragenodeBandwidthBatch(ctx context.Context, intervalStart time.Time, rollups []StoragenodeBandwidthRollup) error
|
||||
// DeleteExpiredReportedSerials deletes any expired reported serials as of now.
|
||||
DeleteExpiredReportedSerials(ctx context.Context, now time.Time) (err error)
|
||||
}
|
||||
|
||||
var (
|
||||
@ -74,6 +80,66 @@ var (
|
||||
mon = monkit.Package()
|
||||
)
|
||||
|
||||
// BucketBandwidthRollup contains all the info needed for a bucket bandwidth rollup
|
||||
type BucketBandwidthRollup struct {
|
||||
ProjectID uuid.UUID
|
||||
BucketName string
|
||||
Action pb.PieceAction
|
||||
Inline int64
|
||||
Allocated int64
|
||||
Settled int64
|
||||
}
|
||||
|
||||
// SortBucketBandwidthRollups sorts the rollups
|
||||
func SortBucketBandwidthRollups(rollups []BucketBandwidthRollup) {
|
||||
sort.SliceStable(rollups, func(i, j int) bool {
|
||||
uuidCompare := bytes.Compare(rollups[i].ProjectID[:], rollups[j].ProjectID[:])
|
||||
switch {
|
||||
case uuidCompare == -1:
|
||||
return true
|
||||
case uuidCompare == 1:
|
||||
return false
|
||||
case rollups[i].BucketName < rollups[j].BucketName:
|
||||
return true
|
||||
case rollups[i].BucketName > rollups[j].BucketName:
|
||||
return false
|
||||
case rollups[i].Action < rollups[j].Action:
|
||||
return true
|
||||
case rollups[i].Action > rollups[j].Action:
|
||||
return false
|
||||
default:
|
||||
return false
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
// StoragenodeBandwidthRollup contains all the info needed for a storagenode bandwidth rollup
|
||||
type StoragenodeBandwidthRollup struct {
|
||||
NodeID storj.NodeID
|
||||
Action pb.PieceAction
|
||||
Allocated int64
|
||||
Settled int64
|
||||
}
|
||||
|
||||
// SortStoragenodeBandwidthRollups sorts the rollups
|
||||
func SortStoragenodeBandwidthRollups(rollups []StoragenodeBandwidthRollup) {
|
||||
sort.SliceStable(rollups, func(i, j int) bool {
|
||||
nodeCompare := bytes.Compare(rollups[i].NodeID.Bytes(), rollups[j].NodeID.Bytes())
|
||||
switch {
|
||||
case nodeCompare == -1:
|
||||
return true
|
||||
case nodeCompare == 1:
|
||||
return false
|
||||
case rollups[i].Action < rollups[j].Action:
|
||||
return true
|
||||
case rollups[i].Action > rollups[j].Action:
|
||||
return false
|
||||
default:
|
||||
return false
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
// ProcessOrderRequest for batch order processing
|
||||
type ProcessOrderRequest struct {
|
||||
Order *pb.Order
|
||||
|
@ -22,7 +22,6 @@ import (
|
||||
"storj.io/storj/satellite"
|
||||
"storj.io/storj/satellite/orders"
|
||||
"storj.io/storj/satellite/satellitedb/satellitedbtest"
|
||||
snorders "storj.io/storj/storagenode/orders"
|
||||
)
|
||||
|
||||
func TestSendingReceivingOrders(t *testing.T) {
|
||||
@ -69,67 +68,6 @@ func TestSendingReceivingOrders(t *testing.T) {
|
||||
})
|
||||
}
|
||||
|
||||
func TestSendingReceivingDuplicateOrders(t *testing.T) {
|
||||
// test happy path
|
||||
testplanet.Run(t, testplanet.Config{
|
||||
SatelliteCount: 1, StorageNodeCount: 6, UplinkCount: 1,
|
||||
}, func(t *testing.T, ctx *testcontext.Context, planet *testplanet.Planet) {
|
||||
planet.Satellites[0].Audit.Worker.Loop.Pause()
|
||||
for _, storageNode := range planet.StorageNodes {
|
||||
storageNode.Storage2.Orders.Sender.Pause()
|
||||
}
|
||||
|
||||
expectedData := testrand.Bytes(50 * memory.KiB)
|
||||
|
||||
redundancy := noLongTailRedundancy(planet)
|
||||
err := planet.Uplinks[0].UploadWithConfig(ctx, planet.Satellites[0], &redundancy, "testbucket", "test/path", expectedData)
|
||||
require.NoError(t, err)
|
||||
|
||||
sumBeforeSend := 0
|
||||
usedOne := false
|
||||
for _, storageNode := range planet.StorageNodes {
|
||||
infos, err := storageNode.DB.Orders().ListUnsent(ctx, 10)
|
||||
require.NoError(t, err)
|
||||
sumBeforeSend += len(infos)
|
||||
|
||||
if len(infos) > 0 && !usedOne {
|
||||
_, err := planet.Satellites[0].DB.Orders().UseSerialNumber(ctx, infos[0].Order.SerialNumber, infos[0].Limit.StorageNodeId)
|
||||
require.NoError(t, err)
|
||||
usedOne = true
|
||||
}
|
||||
|
||||
}
|
||||
require.NotZero(t, sumBeforeSend)
|
||||
|
||||
sumUnsent := 0
|
||||
rejected := 0
|
||||
accepted := 0
|
||||
|
||||
for _, storageNode := range planet.StorageNodes {
|
||||
storageNode.Storage2.Orders.Sender.TriggerWait()
|
||||
|
||||
infos, err := storageNode.DB.Orders().ListUnsent(ctx, 10)
|
||||
require.NoError(t, err)
|
||||
sumUnsent += len(infos)
|
||||
|
||||
archivedInfos, err := storageNode.DB.Orders().ListArchived(ctx, sumBeforeSend)
|
||||
require.NoError(t, err)
|
||||
|
||||
for _, archived := range archivedInfos {
|
||||
if archived.Status == snorders.StatusRejected {
|
||||
rejected++
|
||||
} else if archived.Status == snorders.StatusAccepted {
|
||||
accepted++
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
require.Zero(t, sumUnsent)
|
||||
require.Equal(t, 1, rejected)
|
||||
require.Equal(t, sumBeforeSend-1, accepted)
|
||||
})
|
||||
}
|
||||
|
||||
func TestUnableToSendOrders(t *testing.T) {
|
||||
// test sending when satellite is unavailable
|
||||
testplanet.Run(t, testplanet.Config{
|
||||
@ -180,7 +118,8 @@ func TestUploadDownloadBandwidth(t *testing.T) {
|
||||
testplanet.Run(t, testplanet.Config{
|
||||
SatelliteCount: 1, StorageNodeCount: 6, UplinkCount: 1,
|
||||
}, func(t *testing.T, ctx *testcontext.Context, planet *testplanet.Planet) {
|
||||
hourBeforeTest := time.Now().UTC().Add(-time.Hour)
|
||||
wayInTheFuture := time.Now().UTC().Add(1000 * time.Hour)
|
||||
hourBeforeTheFuture := wayInTheFuture.Add(-time.Hour)
|
||||
planet.Satellites[0].Audit.Worker.Loop.Pause()
|
||||
|
||||
for _, storageNode := range planet.StorageNodes {
|
||||
@ -217,17 +156,21 @@ func TestUploadDownloadBandwidth(t *testing.T) {
|
||||
storageNode.Storage2.Orders.Sender.TriggerWait()
|
||||
}
|
||||
|
||||
// Run the chore as if we were far in the future so that the orders are expired.
|
||||
reportedRollupChore := planet.Satellites[0].Core.Accounting.ReportedRollupChore
|
||||
require.NoError(t, reportedRollupChore.RunOnce(ctx, wayInTheFuture))
|
||||
|
||||
projects, err := planet.Satellites[0].DB.Console().Projects().GetAll(ctx)
|
||||
require.NoError(t, err)
|
||||
|
||||
ordersDB := planet.Satellites[0].DB.Orders()
|
||||
|
||||
bucketBandwidth, err := ordersDB.GetBucketBandwidth(ctx, projects[0].ID, []byte("testbucket"), hourBeforeTest, time.Now().UTC())
|
||||
bucketBandwidth, err := ordersDB.GetBucketBandwidth(ctx, projects[0].ID, []byte("testbucket"), hourBeforeTheFuture, wayInTheFuture)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, expectedBucketBandwidth, bucketBandwidth)
|
||||
|
||||
for _, storageNode := range planet.StorageNodes {
|
||||
nodeBandwidth, err := ordersDB.GetStorageNodeBandwidth(ctx, storageNode.ID(), hourBeforeTest, time.Now().UTC())
|
||||
nodeBandwidth, err := ordersDB.GetStorageNodeBandwidth(ctx, storageNode.ID(), hourBeforeTheFuture, wayInTheFuture)
|
||||
require.NoError(t, err)
|
||||
require.Equal(t, expectedStorageBandwidth[storageNode.ID()], nodeBandwidth)
|
||||
}
|
||||
|
@ -4,9 +4,7 @@
|
||||
package orders
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"sort"
|
||||
"sync"
|
||||
"time"
|
||||
|
||||
@ -90,9 +88,9 @@ func (cache *RollupsWriteCache) FlushToDB(ctx context.Context) {
|
||||
func (cache *RollupsWriteCache) flushToDB(ctx context.Context, pendingRollups RollupData, latestTime time.Time, oldSize int) {
|
||||
defer mon.Task()(&ctx)(nil)
|
||||
|
||||
rollups := make([]BandwidthRollup, 0, oldSize)
|
||||
rollups := make([]BucketBandwidthRollup, 0, oldSize)
|
||||
for cacheKey, cacheData := range pendingRollups {
|
||||
rollups = append(rollups, BandwidthRollup{
|
||||
rollups = append(rollups, BucketBandwidthRollup{
|
||||
ProjectID: cacheKey.ProjectID,
|
||||
BucketName: cacheKey.BucketName,
|
||||
Action: cacheKey.Action,
|
||||
@ -101,9 +99,9 @@ func (cache *RollupsWriteCache) flushToDB(ctx context.Context, pendingRollups Ro
|
||||
})
|
||||
}
|
||||
|
||||
SortRollups(rollups)
|
||||
|
||||
err := cache.DB.UpdateBucketBandwidthBatch(ctx, latestTime.UTC(), rollups)
|
||||
err := cache.DB.ExecuteInTx(ctx, func(ctx context.Context, tx Transaction) error {
|
||||
return tx.UpdateBucketBandwidthBatch(ctx, latestTime, rollups)
|
||||
})
|
||||
if err != nil {
|
||||
cache.log.Error("MONEY LOST! Bucket bandwidth rollup batch flush failed.", zap.Error(err))
|
||||
}
|
||||
@ -115,29 +113,6 @@ func (cache *RollupsWriteCache) flushToDB(ctx context.Context, pendingRollups Ro
|
||||
completion.Release()
|
||||
}
|
||||
|
||||
// SortRollups sorts the rollups
|
||||
func SortRollups(rollups []BandwidthRollup) {
|
||||
sort.SliceStable(rollups, func(i, j int) bool {
|
||||
uuidCompare := bytes.Compare(rollups[i].ProjectID[:], rollups[j].ProjectID[:])
|
||||
switch {
|
||||
case uuidCompare == -1:
|
||||
return true
|
||||
case uuidCompare == 1:
|
||||
return false
|
||||
case rollups[i].BucketName < rollups[j].BucketName:
|
||||
return true
|
||||
case rollups[i].BucketName > rollups[j].BucketName:
|
||||
return false
|
||||
case rollups[i].Action < rollups[j].Action:
|
||||
return true
|
||||
case rollups[i].Action > rollups[j].Action:
|
||||
return false
|
||||
default:
|
||||
return false
|
||||
}
|
||||
})
|
||||
}
|
||||
|
||||
func (cache *RollupsWriteCache) updateCacheValue(ctx context.Context, projectID uuid.UUID, bucketName []byte, action pb.PieceAction, allocated, inline int64, intervalStart time.Time) {
|
||||
defer mon.Task()(&ctx)(nil)
|
||||
|
||||
|
@ -138,6 +138,8 @@ func TestUpdateBucketBandwidthAllocation(t *testing.T) {
|
||||
SatelliteCount: 1,
|
||||
},
|
||||
func(t *testing.T, ctx *testcontext.Context, planet *testplanet.Planet) {
|
||||
// don't let the loop flush our cache while we're checking it
|
||||
planet.Satellites[0].Orders.Chore.Loop.Pause()
|
||||
ordersDB := planet.Satellites[0].Orders.DB
|
||||
|
||||
// setup: check there is nothing in the cache to start
|
||||
@ -195,7 +197,7 @@ func TestUpdateBucketBandwidthAllocation(t *testing.T) {
|
||||
}
|
||||
|
||||
func TestSortRollups(t *testing.T) {
|
||||
rollups := []orders.BandwidthRollup{
|
||||
rollups := []orders.BucketBandwidthRollup{
|
||||
{
|
||||
ProjectID: uuid.UUID{1},
|
||||
BucketName: "a",
|
||||
@ -233,7 +235,7 @@ func TestSortRollups(t *testing.T) {
|
||||
},
|
||||
}
|
||||
|
||||
expRollups := []orders.BandwidthRollup{
|
||||
expRollups := []orders.BucketBandwidthRollup{
|
||||
{
|
||||
ProjectID: uuid.UUID{1},
|
||||
BucketName: "a",
|
||||
@ -272,6 +274,6 @@ func TestSortRollups(t *testing.T) {
|
||||
}
|
||||
|
||||
assert.NotEqual(t, expRollups, rollups)
|
||||
orders.SortRollups(rollups)
|
||||
orders.SortBucketBandwidthRollups(rollups)
|
||||
assert.Equal(t, expRollups, rollups)
|
||||
}
|
||||
|
@ -25,10 +25,11 @@ var ErrDownloadFailedNotEnoughPieces = errs.Class("not enough pieces for downloa
|
||||
|
||||
// Config is a configuration struct for orders Service.
|
||||
type Config struct {
|
||||
Expiration time.Duration `help:"how long until an order expires" default:"168h"` // 7 days
|
||||
SettlementBatchSize int `help:"how many orders to batch per transaction" default:"250"`
|
||||
FlushBatchSize int `help:"how many items in the rollups write cache before they are flushed to the database" devDefault:"20" releaseDefault:"10000"`
|
||||
FlushInterval time.Duration `help:"how often to flush the rollups write cache to the database" devDefault:"30s" releaseDefault:"1m"`
|
||||
Expiration time.Duration `help:"how long until an order expires" default:"168h"` // 7 days
|
||||
SettlementBatchSize int `help:"how many orders to batch per transaction" default:"250"`
|
||||
FlushBatchSize int `help:"how many items in the rollups write cache before they are flushed to the database" devDefault:"20" releaseDefault:"10000"`
|
||||
FlushInterval time.Duration `help:"how often to flush the rollups write cache to the database" devDefault:"30s" releaseDefault:"1m"`
|
||||
ReportedRollupsReadBatchSize int `help:"how many records to read in a single transaction when calculating billable bandwidth" default:"1000"`
|
||||
}
|
||||
|
||||
// Service for creating order limits.
|
||||
|
@ -13,6 +13,7 @@ import (
|
||||
version_checker "storj.io/storj/private/version/checker"
|
||||
"storj.io/storj/satellite/accounting"
|
||||
"storj.io/storj/satellite/accounting/live"
|
||||
"storj.io/storj/satellite/accounting/reportedrollup"
|
||||
"storj.io/storj/satellite/accounting/rollup"
|
||||
"storj.io/storj/satellite/accounting/tally"
|
||||
"storj.io/storj/satellite/attribution"
|
||||
@ -107,6 +108,7 @@ type Config struct {
|
||||
Tally tally.Config
|
||||
Rollup rollup.Config
|
||||
LiveAccounting live.Config
|
||||
ReportedRollup reportedrollup.Config
|
||||
|
||||
Mail mailservice.Config
|
||||
|
||||
|
@ -51,6 +51,10 @@ type satelliteDB struct {
|
||||
// Options includes options for how a satelliteDB runs
|
||||
type Options struct {
|
||||
APIKeysLRUOptions cache.Options
|
||||
|
||||
// How many records to read in a single transaction when asked for all of the
|
||||
// billable bandwidth from the reported serials table.
|
||||
ReportedRollupsReadBatchSize int
|
||||
}
|
||||
|
||||
var _ dbx.DBMethods = &satelliteDB{}
|
||||
@ -150,7 +154,7 @@ func (db *satelliteDB) Rewards() rewards.DB {
|
||||
|
||||
// Orders returns database for storing orders
|
||||
func (db *satelliteDB) Orders() orders.DB {
|
||||
return &ordersDB{db: db}
|
||||
return &ordersDB{db: db, reportedRollupsReadBatchSize: db.opts.ReportedRollupsReadBatchSize}
|
||||
}
|
||||
|
||||
// Containment returns database for storing pending audit info
|
||||
|
@ -458,6 +458,30 @@ delete serial_number (
|
||||
// for preventing duplicate serial numbers
|
||||
create used_serial ( noreturn )
|
||||
|
||||
model reported_serial (
|
||||
// all of our access patterns care about if the serial is expired or not so we
|
||||
// should put expires_at first in the primary key to get efficient scans. but,
|
||||
// we also want to have rocksdb prefix compression to apply as much as possible
|
||||
// to reduce data size for numerous benefits. so, we ceil the expires_at field
|
||||
// to the next day so that we only have to store each storage node id only once
|
||||
// per day. the other fields follow the same logic.
|
||||
|
||||
key expires_at storage_node_id bucket_id action serial_number
|
||||
|
||||
field expires_at utimestamp // ceil'd to the next day
|
||||
field storage_node_id blob // involved node id
|
||||
field bucket_id blob // involved project id and bucket name
|
||||
field action uint // action (get, put, audit, etc. see protobuf)
|
||||
field serial_number blob // identifies order (required for uniqueness)
|
||||
|
||||
field settled uint64
|
||||
field observed_at utimestamp
|
||||
)
|
||||
|
||||
create reported_serial ( noreturn, replace )
|
||||
read paged ( select reported_serial, where reported_serial.expires_at <= ? )
|
||||
delete reported_serial ( where reported_serial.expires_at <= ? )
|
||||
|
||||
// --- bucket accounting tables --- //
|
||||
|
||||
model bucket_bandwidth_rollup (
|
||||
|
@ -202,6 +202,16 @@ CREATE TABLE registration_tokens (
|
||||
PRIMARY KEY ( secret ),
|
||||
UNIQUE ( owner_id )
|
||||
);
|
||||
CREATE TABLE reported_serials (
|
||||
expires_at timestamp NOT NULL,
|
||||
storage_node_id bytea NOT NULL,
|
||||
bucket_id bytea NOT NULL,
|
||||
action integer NOT NULL,
|
||||
serial_number bytea NOT NULL,
|
||||
settled bigint NOT NULL,
|
||||
observed_at timestamp NOT NULL,
|
||||
PRIMARY KEY ( expires_at, storage_node_id, bucket_id, action, serial_number )
|
||||
);
|
||||
CREATE TABLE reset_password_tokens (
|
||||
secret bytea NOT NULL,
|
||||
owner_id bytea NOT NULL,
|
||||
|
@ -472,6 +472,16 @@ CREATE TABLE registration_tokens (
|
||||
PRIMARY KEY ( secret ),
|
||||
UNIQUE ( owner_id )
|
||||
);
|
||||
CREATE TABLE reported_serials (
|
||||
expires_at timestamp NOT NULL,
|
||||
storage_node_id bytea NOT NULL,
|
||||
bucket_id bytea NOT NULL,
|
||||
action integer NOT NULL,
|
||||
serial_number bytea NOT NULL,
|
||||
settled bigint NOT NULL,
|
||||
observed_at timestamp NOT NULL,
|
||||
PRIMARY KEY ( expires_at, storage_node_id, bucket_id, action, serial_number )
|
||||
);
|
||||
CREATE TABLE reset_password_tokens (
|
||||
secret bytea NOT NULL,
|
||||
owner_id bytea NOT NULL,
|
||||
@ -890,6 +900,16 @@ CREATE TABLE registration_tokens (
|
||||
PRIMARY KEY ( secret ),
|
||||
UNIQUE ( owner_id )
|
||||
);
|
||||
CREATE TABLE reported_serials (
|
||||
expires_at timestamp NOT NULL,
|
||||
storage_node_id bytea NOT NULL,
|
||||
bucket_id bytea NOT NULL,
|
||||
action integer NOT NULL,
|
||||
serial_number bytea NOT NULL,
|
||||
settled bigint NOT NULL,
|
||||
observed_at timestamp NOT NULL,
|
||||
PRIMARY KEY ( expires_at, storage_node_id, bucket_id, action, serial_number )
|
||||
);
|
||||
CREATE TABLE reset_password_tokens (
|
||||
secret bytea NOT NULL,
|
||||
owner_id bytea NOT NULL,
|
||||
@ -4531,6 +4551,156 @@ func (f RegistrationToken_CreatedAt_Field) value() interface{} {
|
||||
|
||||
func (RegistrationToken_CreatedAt_Field) _Column() string { return "created_at" }
|
||||
|
||||
type ReportedSerial struct {
|
||||
ExpiresAt time.Time
|
||||
StorageNodeId []byte
|
||||
BucketId []byte
|
||||
Action uint
|
||||
SerialNumber []byte
|
||||
Settled uint64
|
||||
ObservedAt time.Time
|
||||
}
|
||||
|
||||
func (ReportedSerial) _Table() string { return "reported_serials" }
|
||||
|
||||
type ReportedSerial_Update_Fields struct {
|
||||
}
|
||||
|
||||
type ReportedSerial_ExpiresAt_Field struct {
|
||||
_set bool
|
||||
_null bool
|
||||
_value time.Time
|
||||
}
|
||||
|
||||
func ReportedSerial_ExpiresAt(v time.Time) ReportedSerial_ExpiresAt_Field {
|
||||
v = toUTC(v)
|
||||
return ReportedSerial_ExpiresAt_Field{_set: true, _value: v}
|
||||
}
|
||||
|
||||
func (f ReportedSerial_ExpiresAt_Field) value() interface{} {
|
||||
if !f._set || f._null {
|
||||
return nil
|
||||
}
|
||||
return f._value
|
||||
}
|
||||
|
||||
func (ReportedSerial_ExpiresAt_Field) _Column() string { return "expires_at" }
|
||||
|
||||
type ReportedSerial_StorageNodeId_Field struct {
|
||||
_set bool
|
||||
_null bool
|
||||
_value []byte
|
||||
}
|
||||
|
||||
func ReportedSerial_StorageNodeId(v []byte) ReportedSerial_StorageNodeId_Field {
|
||||
return ReportedSerial_StorageNodeId_Field{_set: true, _value: v}
|
||||
}
|
||||
|
||||
func (f ReportedSerial_StorageNodeId_Field) value() interface{} {
|
||||
if !f._set || f._null {
|
||||
return nil
|
||||
}
|
||||
return f._value
|
||||
}
|
||||
|
||||
func (ReportedSerial_StorageNodeId_Field) _Column() string { return "storage_node_id" }
|
||||
|
||||
type ReportedSerial_BucketId_Field struct {
|
||||
_set bool
|
||||
_null bool
|
||||
_value []byte
|
||||
}
|
||||
|
||||
func ReportedSerial_BucketId(v []byte) ReportedSerial_BucketId_Field {
|
||||
return ReportedSerial_BucketId_Field{_set: true, _value: v}
|
||||
}
|
||||
|
||||
func (f ReportedSerial_BucketId_Field) value() interface{} {
|
||||
if !f._set || f._null {
|
||||
return nil
|
||||
}
|
||||
return f._value
|
||||
}
|
||||
|
||||
func (ReportedSerial_BucketId_Field) _Column() string { return "bucket_id" }
|
||||
|
||||
type ReportedSerial_Action_Field struct {
|
||||
_set bool
|
||||
_null bool
|
||||
_value uint
|
||||
}
|
||||
|
||||
func ReportedSerial_Action(v uint) ReportedSerial_Action_Field {
|
||||
return ReportedSerial_Action_Field{_set: true, _value: v}
|
||||
}
|
||||
|
||||
func (f ReportedSerial_Action_Field) value() interface{} {
|
||||
if !f._set || f._null {
|
||||
return nil
|
||||
}
|
||||
return f._value
|
||||
}
|
||||
|
||||
func (ReportedSerial_Action_Field) _Column() string { return "action" }
|
||||
|
||||
type ReportedSerial_SerialNumber_Field struct {
|
||||
_set bool
|
||||
_null bool
|
||||
_value []byte
|
||||
}
|
||||
|
||||
func ReportedSerial_SerialNumber(v []byte) ReportedSerial_SerialNumber_Field {
|
||||
return ReportedSerial_SerialNumber_Field{_set: true, _value: v}
|
||||
}
|
||||
|
||||
func (f ReportedSerial_SerialNumber_Field) value() interface{} {
|
||||
if !f._set || f._null {
|
||||
return nil
|
||||
}
|
||||
return f._value
|
||||
}
|
||||
|
||||
func (ReportedSerial_SerialNumber_Field) _Column() string { return "serial_number" }
|
||||
|
||||
type ReportedSerial_Settled_Field struct {
|
||||
_set bool
|
||||
_null bool
|
||||
_value uint64
|
||||
}
|
||||
|
||||
func ReportedSerial_Settled(v uint64) ReportedSerial_Settled_Field {
|
||||
return ReportedSerial_Settled_Field{_set: true, _value: v}
|
||||
}
|
||||
|
||||
func (f ReportedSerial_Settled_Field) value() interface{} {
|
||||
if !f._set || f._null {
|
||||
return nil
|
||||
}
|
||||
return f._value
|
||||
}
|
||||
|
||||
func (ReportedSerial_Settled_Field) _Column() string { return "settled" }
|
||||
|
||||
type ReportedSerial_ObservedAt_Field struct {
|
||||
_set bool
|
||||
_null bool
|
||||
_value time.Time
|
||||
}
|
||||
|
||||
func ReportedSerial_ObservedAt(v time.Time) ReportedSerial_ObservedAt_Field {
|
||||
v = toUTC(v)
|
||||
return ReportedSerial_ObservedAt_Field{_set: true, _value: v}
|
||||
}
|
||||
|
||||
func (f ReportedSerial_ObservedAt_Field) value() interface{} {
|
||||
if !f._set || f._null {
|
||||
return nil
|
||||
}
|
||||
return f._value
|
||||
}
|
||||
|
||||
func (ReportedSerial_ObservedAt_Field) _Column() string { return "observed_at" }
|
||||
|
||||
type ResetPasswordToken struct {
|
||||
Secret []byte
|
||||
OwnerId []byte
|
||||
@ -6906,6 +7076,15 @@ type LeafSerialNumber_Row struct {
|
||||
LeafSerialNumber []byte
|
||||
}
|
||||
|
||||
type Paged_ReportedSerial_By_ExpiresAt_LessOrEqual_Continuation struct {
|
||||
_value_expires_at time.Time
|
||||
_value_storage_node_id []byte
|
||||
_value_bucket_id []byte
|
||||
_value_action uint
|
||||
_value_serial_number []byte
|
||||
_set bool
|
||||
}
|
||||
|
||||
type UsageLimit_Row struct {
|
||||
UsageLimit int64
|
||||
}
|
||||
@ -7335,6 +7514,37 @@ func (obj *postgresImpl) CreateNoReturn_UsedSerial(ctx context.Context,
|
||||
|
||||
}
|
||||
|
||||
func (obj *postgresImpl) ReplaceNoReturn_ReportedSerial(ctx context.Context,
|
||||
reported_serial_expires_at ReportedSerial_ExpiresAt_Field,
|
||||
reported_serial_storage_node_id ReportedSerial_StorageNodeId_Field,
|
||||
reported_serial_bucket_id ReportedSerial_BucketId_Field,
|
||||
reported_serial_action ReportedSerial_Action_Field,
|
||||
reported_serial_serial_number ReportedSerial_SerialNumber_Field,
|
||||
reported_serial_settled ReportedSerial_Settled_Field,
|
||||
reported_serial_observed_at ReportedSerial_ObservedAt_Field) (
|
||||
err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
__expires_at_val := reported_serial_expires_at.value()
|
||||
__storage_node_id_val := reported_serial_storage_node_id.value()
|
||||
__bucket_id_val := reported_serial_bucket_id.value()
|
||||
__action_val := reported_serial_action.value()
|
||||
__serial_number_val := reported_serial_serial_number.value()
|
||||
__settled_val := reported_serial_settled.value()
|
||||
__observed_at_val := reported_serial_observed_at.value()
|
||||
|
||||
var __embed_stmt = __sqlbundle_Literal("INSERT INTO reported_serials ( expires_at, storage_node_id, bucket_id, action, serial_number, settled, observed_at ) VALUES ( ?, ?, ?, ?, ?, ?, ? ) ON CONFLICT ( expires_at, storage_node_id, bucket_id, action, serial_number ) DO UPDATE SET expires_at = EXCLUDED.expires_at, storage_node_id = EXCLUDED.storage_node_id, bucket_id = EXCLUDED.bucket_id, action = EXCLUDED.action, serial_number = EXCLUDED.serial_number, settled = EXCLUDED.settled, observed_at = EXCLUDED.observed_at")
|
||||
|
||||
var __stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
|
||||
obj.logStmt(__stmt, __expires_at_val, __storage_node_id_val, __bucket_id_val, __action_val, __serial_number_val, __settled_val, __observed_at_val)
|
||||
|
||||
_, err = obj.driver.Exec(__stmt, __expires_at_val, __storage_node_id_val, __bucket_id_val, __action_val, __serial_number_val, __settled_val, __observed_at_val)
|
||||
if err != nil {
|
||||
return obj.makeErr(err)
|
||||
}
|
||||
return nil
|
||||
|
||||
}
|
||||
|
||||
func (obj *postgresImpl) CreateNoReturn_BucketStorageTally(ctx context.Context,
|
||||
bucket_storage_tally_bucket_name BucketStorageTally_BucketName_Field,
|
||||
bucket_storage_tally_project_id BucketStorageTally_ProjectId_Field,
|
||||
@ -8882,6 +9092,55 @@ func (obj *postgresImpl) Find_SerialNumber_By_SerialNumber(ctx context.Context,
|
||||
|
||||
}
|
||||
|
||||
func (obj *postgresImpl) Paged_ReportedSerial_By_ExpiresAt_LessOrEqual(ctx context.Context,
|
||||
reported_serial_expires_at_less_or_equal ReportedSerial_ExpiresAt_Field,
|
||||
limit int, start *Paged_ReportedSerial_By_ExpiresAt_LessOrEqual_Continuation) (
|
||||
rows []*ReportedSerial, next *Paged_ReportedSerial_By_ExpiresAt_LessOrEqual_Continuation, err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
|
||||
var __embed_stmt = __sqlbundle_Literal("SELECT reported_serials.expires_at, reported_serials.storage_node_id, reported_serials.bucket_id, reported_serials.action, reported_serials.serial_number, reported_serials.settled, reported_serials.observed_at, reported_serials.expires_at, reported_serials.storage_node_id, reported_serials.bucket_id, reported_serials.action, reported_serials.serial_number FROM reported_serials WHERE reported_serials.expires_at <= ? AND (reported_serials.expires_at > ? OR (reported_serials.expires_at = ? AND (reported_serials.storage_node_id > ? OR (reported_serials.storage_node_id = ? AND (reported_serials.bucket_id > ? OR (reported_serials.bucket_id = ? AND (reported_serials.action > ? OR (reported_serials.action = ? AND reported_serials.serial_number > ?)))))))) ORDER BY reported_serials.expires_at, reported_serials.storage_node_id, reported_serials.bucket_id, reported_serials.action, reported_serials.serial_number LIMIT ?")
|
||||
|
||||
var __embed_first_stmt = __sqlbundle_Literal("SELECT reported_serials.expires_at, reported_serials.storage_node_id, reported_serials.bucket_id, reported_serials.action, reported_serials.serial_number, reported_serials.settled, reported_serials.observed_at, reported_serials.expires_at, reported_serials.storage_node_id, reported_serials.bucket_id, reported_serials.action, reported_serials.serial_number FROM reported_serials WHERE reported_serials.expires_at <= ? ORDER BY reported_serials.expires_at, reported_serials.storage_node_id, reported_serials.bucket_id, reported_serials.action, reported_serials.serial_number LIMIT ?")
|
||||
|
||||
var __values []interface{}
|
||||
__values = append(__values, reported_serial_expires_at_less_or_equal.value())
|
||||
|
||||
var __stmt string
|
||||
if start != nil && start._set {
|
||||
__values = append(__values, start._value_expires_at, start._value_expires_at, start._value_storage_node_id, start._value_storage_node_id, start._value_bucket_id, start._value_bucket_id, start._value_action, start._value_action, start._value_serial_number, limit)
|
||||
__stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
|
||||
} else {
|
||||
__values = append(__values, limit)
|
||||
__stmt = __sqlbundle_Render(obj.dialect, __embed_first_stmt)
|
||||
}
|
||||
obj.logStmt(__stmt, __values...)
|
||||
|
||||
__rows, err := obj.driver.Query(__stmt, __values...)
|
||||
if err != nil {
|
||||
return nil, nil, obj.makeErr(err)
|
||||
}
|
||||
defer __rows.Close()
|
||||
|
||||
var __continuation Paged_ReportedSerial_By_ExpiresAt_LessOrEqual_Continuation
|
||||
__continuation._set = true
|
||||
|
||||
for __rows.Next() {
|
||||
reported_serial := &ReportedSerial{}
|
||||
err = __rows.Scan(&reported_serial.ExpiresAt, &reported_serial.StorageNodeId, &reported_serial.BucketId, &reported_serial.Action, &reported_serial.SerialNumber, &reported_serial.Settled, &reported_serial.ObservedAt, &__continuation._value_expires_at, &__continuation._value_storage_node_id, &__continuation._value_bucket_id, &__continuation._value_action, &__continuation._value_serial_number)
|
||||
if err != nil {
|
||||
return nil, nil, obj.makeErr(err)
|
||||
}
|
||||
rows = append(rows, reported_serial)
|
||||
next = &__continuation
|
||||
}
|
||||
if err := __rows.Err(); err != nil {
|
||||
return nil, nil, obj.makeErr(err)
|
||||
}
|
||||
|
||||
return rows, next, nil
|
||||
|
||||
}
|
||||
|
||||
func (obj *postgresImpl) Find_BucketBandwidthRollup_By_BucketName_And_ProjectId_And_IntervalStart_And_Action(ctx context.Context,
|
||||
bucket_bandwidth_rollup_bucket_name BucketBandwidthRollup_BucketName_Field,
|
||||
bucket_bandwidth_rollup_project_id BucketBandwidthRollup_ProjectId_Field,
|
||||
@ -11569,6 +11828,33 @@ func (obj *postgresImpl) Delete_SerialNumber_By_ExpiresAt_LessOrEqual(ctx contex
|
||||
|
||||
}
|
||||
|
||||
func (obj *postgresImpl) Delete_ReportedSerial_By_ExpiresAt_LessOrEqual(ctx context.Context,
|
||||
reported_serial_expires_at_less_or_equal ReportedSerial_ExpiresAt_Field) (
|
||||
count int64, err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
|
||||
var __embed_stmt = __sqlbundle_Literal("DELETE FROM reported_serials WHERE reported_serials.expires_at <= ?")
|
||||
|
||||
var __values []interface{}
|
||||
__values = append(__values, reported_serial_expires_at_less_or_equal.value())
|
||||
|
||||
var __stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
|
||||
obj.logStmt(__stmt, __values...)
|
||||
|
||||
__res, err := obj.driver.Exec(__stmt, __values...)
|
||||
if err != nil {
|
||||
return 0, obj.makeErr(err)
|
||||
}
|
||||
|
||||
count, err = __res.RowsAffected()
|
||||
if err != nil {
|
||||
return 0, obj.makeErr(err)
|
||||
}
|
||||
|
||||
return count, nil
|
||||
|
||||
}
|
||||
|
||||
func (obj *postgresImpl) Delete_StoragenodeStorageTally_By_Id(ctx context.Context,
|
||||
storagenode_storage_tally_id StoragenodeStorageTally_Id_Field) (
|
||||
deleted bool, err error) {
|
||||
@ -12011,6 +12297,16 @@ func (obj *postgresImpl) deleteAll(ctx context.Context) (count int64, err error)
|
||||
return 0, obj.makeErr(err)
|
||||
}
|
||||
|
||||
__count, err = __res.RowsAffected()
|
||||
if err != nil {
|
||||
return 0, obj.makeErr(err)
|
||||
}
|
||||
count += __count
|
||||
__res, err = obj.driver.Exec("DELETE FROM reported_serials;")
|
||||
if err != nil {
|
||||
return 0, obj.makeErr(err)
|
||||
}
|
||||
|
||||
__count, err = __res.RowsAffected()
|
||||
if err != nil {
|
||||
return 0, obj.makeErr(err)
|
||||
@ -12622,6 +12918,37 @@ func (obj *cockroachImpl) CreateNoReturn_UsedSerial(ctx context.Context,
|
||||
|
||||
}
|
||||
|
||||
func (obj *cockroachImpl) ReplaceNoReturn_ReportedSerial(ctx context.Context,
|
||||
reported_serial_expires_at ReportedSerial_ExpiresAt_Field,
|
||||
reported_serial_storage_node_id ReportedSerial_StorageNodeId_Field,
|
||||
reported_serial_bucket_id ReportedSerial_BucketId_Field,
|
||||
reported_serial_action ReportedSerial_Action_Field,
|
||||
reported_serial_serial_number ReportedSerial_SerialNumber_Field,
|
||||
reported_serial_settled ReportedSerial_Settled_Field,
|
||||
reported_serial_observed_at ReportedSerial_ObservedAt_Field) (
|
||||
err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
__expires_at_val := reported_serial_expires_at.value()
|
||||
__storage_node_id_val := reported_serial_storage_node_id.value()
|
||||
__bucket_id_val := reported_serial_bucket_id.value()
|
||||
__action_val := reported_serial_action.value()
|
||||
__serial_number_val := reported_serial_serial_number.value()
|
||||
__settled_val := reported_serial_settled.value()
|
||||
__observed_at_val := reported_serial_observed_at.value()
|
||||
|
||||
var __embed_stmt = __sqlbundle_Literal("UPSERT INTO reported_serials ( expires_at, storage_node_id, bucket_id, action, serial_number, settled, observed_at ) VALUES ( ?, ?, ?, ?, ?, ?, ? )")
|
||||
|
||||
var __stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
|
||||
obj.logStmt(__stmt, __expires_at_val, __storage_node_id_val, __bucket_id_val, __action_val, __serial_number_val, __settled_val, __observed_at_val)
|
||||
|
||||
_, err = obj.driver.Exec(__stmt, __expires_at_val, __storage_node_id_val, __bucket_id_val, __action_val, __serial_number_val, __settled_val, __observed_at_val)
|
||||
if err != nil {
|
||||
return obj.makeErr(err)
|
||||
}
|
||||
return nil
|
||||
|
||||
}
|
||||
|
||||
func (obj *cockroachImpl) CreateNoReturn_BucketStorageTally(ctx context.Context,
|
||||
bucket_storage_tally_bucket_name BucketStorageTally_BucketName_Field,
|
||||
bucket_storage_tally_project_id BucketStorageTally_ProjectId_Field,
|
||||
@ -14169,6 +14496,55 @@ func (obj *cockroachImpl) Find_SerialNumber_By_SerialNumber(ctx context.Context,
|
||||
|
||||
}
|
||||
|
||||
func (obj *cockroachImpl) Paged_ReportedSerial_By_ExpiresAt_LessOrEqual(ctx context.Context,
|
||||
reported_serial_expires_at_less_or_equal ReportedSerial_ExpiresAt_Field,
|
||||
limit int, start *Paged_ReportedSerial_By_ExpiresAt_LessOrEqual_Continuation) (
|
||||
rows []*ReportedSerial, next *Paged_ReportedSerial_By_ExpiresAt_LessOrEqual_Continuation, err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
|
||||
var __embed_stmt = __sqlbundle_Literal("SELECT reported_serials.expires_at, reported_serials.storage_node_id, reported_serials.bucket_id, reported_serials.action, reported_serials.serial_number, reported_serials.settled, reported_serials.observed_at, reported_serials.expires_at, reported_serials.storage_node_id, reported_serials.bucket_id, reported_serials.action, reported_serials.serial_number FROM reported_serials WHERE reported_serials.expires_at <= ? AND (reported_serials.expires_at > ? OR (reported_serials.expires_at = ? AND (reported_serials.storage_node_id > ? OR (reported_serials.storage_node_id = ? AND (reported_serials.bucket_id > ? OR (reported_serials.bucket_id = ? AND (reported_serials.action > ? OR (reported_serials.action = ? AND reported_serials.serial_number > ?)))))))) ORDER BY reported_serials.expires_at, reported_serials.storage_node_id, reported_serials.bucket_id, reported_serials.action, reported_serials.serial_number LIMIT ?")
|
||||
|
||||
var __embed_first_stmt = __sqlbundle_Literal("SELECT reported_serials.expires_at, reported_serials.storage_node_id, reported_serials.bucket_id, reported_serials.action, reported_serials.serial_number, reported_serials.settled, reported_serials.observed_at, reported_serials.expires_at, reported_serials.storage_node_id, reported_serials.bucket_id, reported_serials.action, reported_serials.serial_number FROM reported_serials WHERE reported_serials.expires_at <= ? ORDER BY reported_serials.expires_at, reported_serials.storage_node_id, reported_serials.bucket_id, reported_serials.action, reported_serials.serial_number LIMIT ?")
|
||||
|
||||
var __values []interface{}
|
||||
__values = append(__values, reported_serial_expires_at_less_or_equal.value())
|
||||
|
||||
var __stmt string
|
||||
if start != nil && start._set {
|
||||
__values = append(__values, start._value_expires_at, start._value_expires_at, start._value_storage_node_id, start._value_storage_node_id, start._value_bucket_id, start._value_bucket_id, start._value_action, start._value_action, start._value_serial_number, limit)
|
||||
__stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
|
||||
} else {
|
||||
__values = append(__values, limit)
|
||||
__stmt = __sqlbundle_Render(obj.dialect, __embed_first_stmt)
|
||||
}
|
||||
obj.logStmt(__stmt, __values...)
|
||||
|
||||
__rows, err := obj.driver.Query(__stmt, __values...)
|
||||
if err != nil {
|
||||
return nil, nil, obj.makeErr(err)
|
||||
}
|
||||
defer __rows.Close()
|
||||
|
||||
var __continuation Paged_ReportedSerial_By_ExpiresAt_LessOrEqual_Continuation
|
||||
__continuation._set = true
|
||||
|
||||
for __rows.Next() {
|
||||
reported_serial := &ReportedSerial{}
|
||||
err = __rows.Scan(&reported_serial.ExpiresAt, &reported_serial.StorageNodeId, &reported_serial.BucketId, &reported_serial.Action, &reported_serial.SerialNumber, &reported_serial.Settled, &reported_serial.ObservedAt, &__continuation._value_expires_at, &__continuation._value_storage_node_id, &__continuation._value_bucket_id, &__continuation._value_action, &__continuation._value_serial_number)
|
||||
if err != nil {
|
||||
return nil, nil, obj.makeErr(err)
|
||||
}
|
||||
rows = append(rows, reported_serial)
|
||||
next = &__continuation
|
||||
}
|
||||
if err := __rows.Err(); err != nil {
|
||||
return nil, nil, obj.makeErr(err)
|
||||
}
|
||||
|
||||
return rows, next, nil
|
||||
|
||||
}
|
||||
|
||||
func (obj *cockroachImpl) Find_BucketBandwidthRollup_By_BucketName_And_ProjectId_And_IntervalStart_And_Action(ctx context.Context,
|
||||
bucket_bandwidth_rollup_bucket_name BucketBandwidthRollup_BucketName_Field,
|
||||
bucket_bandwidth_rollup_project_id BucketBandwidthRollup_ProjectId_Field,
|
||||
@ -16856,6 +17232,33 @@ func (obj *cockroachImpl) Delete_SerialNumber_By_ExpiresAt_LessOrEqual(ctx conte
|
||||
|
||||
}
|
||||
|
||||
func (obj *cockroachImpl) Delete_ReportedSerial_By_ExpiresAt_LessOrEqual(ctx context.Context,
|
||||
reported_serial_expires_at_less_or_equal ReportedSerial_ExpiresAt_Field) (
|
||||
count int64, err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
|
||||
var __embed_stmt = __sqlbundle_Literal("DELETE FROM reported_serials WHERE reported_serials.expires_at <= ?")
|
||||
|
||||
var __values []interface{}
|
||||
__values = append(__values, reported_serial_expires_at_less_or_equal.value())
|
||||
|
||||
var __stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
|
||||
obj.logStmt(__stmt, __values...)
|
||||
|
||||
__res, err := obj.driver.Exec(__stmt, __values...)
|
||||
if err != nil {
|
||||
return 0, obj.makeErr(err)
|
||||
}
|
||||
|
||||
count, err = __res.RowsAffected()
|
||||
if err != nil {
|
||||
return 0, obj.makeErr(err)
|
||||
}
|
||||
|
||||
return count, nil
|
||||
|
||||
}
|
||||
|
||||
func (obj *cockroachImpl) Delete_StoragenodeStorageTally_By_Id(ctx context.Context,
|
||||
storagenode_storage_tally_id StoragenodeStorageTally_Id_Field) (
|
||||
deleted bool, err error) {
|
||||
@ -17298,6 +17701,16 @@ func (obj *cockroachImpl) deleteAll(ctx context.Context) (count int64, err error
|
||||
return 0, obj.makeErr(err)
|
||||
}
|
||||
|
||||
__count, err = __res.RowsAffected()
|
||||
if err != nil {
|
||||
return 0, obj.makeErr(err)
|
||||
}
|
||||
count += __count
|
||||
__res, err = obj.driver.Exec("DELETE FROM reported_serials;")
|
||||
if err != nil {
|
||||
return 0, obj.makeErr(err)
|
||||
}
|
||||
|
||||
__count, err = __res.RowsAffected()
|
||||
if err != nil {
|
||||
return 0, obj.makeErr(err)
|
||||
@ -18404,6 +18817,17 @@ func (rx *Rx) Delete_Project_By_Id(ctx context.Context,
|
||||
return tx.Delete_Project_By_Id(ctx, project_id)
|
||||
}
|
||||
|
||||
func (rx *Rx) Delete_ReportedSerial_By_ExpiresAt_LessOrEqual(ctx context.Context,
|
||||
reported_serial_expires_at_less_or_equal ReportedSerial_ExpiresAt_Field) (
|
||||
count int64, err error) {
|
||||
var tx *Tx
|
||||
if tx, err = rx.getTx(ctx); err != nil {
|
||||
return
|
||||
}
|
||||
return tx.Delete_ReportedSerial_By_ExpiresAt_LessOrEqual(ctx, reported_serial_expires_at_less_or_equal)
|
||||
|
||||
}
|
||||
|
||||
func (rx *Rx) Delete_ResetPasswordToken_By_Secret(ctx context.Context,
|
||||
reset_password_token_secret ResetPasswordToken_Secret_Field) (
|
||||
deleted bool, err error) {
|
||||
@ -18966,6 +19390,34 @@ func (rx *Rx) Limited_StripecoinpaymentsInvoiceProjectRecord_By_CreatedAt_LessOr
|
||||
return tx.Limited_StripecoinpaymentsInvoiceProjectRecord_By_CreatedAt_LessOrEqual_And_State_OrderBy_Desc_CreatedAt(ctx, stripecoinpayments_invoice_project_record_created_at_less_or_equal, stripecoinpayments_invoice_project_record_state, limit, offset)
|
||||
}
|
||||
|
||||
func (rx *Rx) Paged_ReportedSerial_By_ExpiresAt_LessOrEqual(ctx context.Context,
|
||||
reported_serial_expires_at_less_or_equal ReportedSerial_ExpiresAt_Field,
|
||||
limit int, start *Paged_ReportedSerial_By_ExpiresAt_LessOrEqual_Continuation) (
|
||||
rows []*ReportedSerial, next *Paged_ReportedSerial_By_ExpiresAt_LessOrEqual_Continuation, err error) {
|
||||
var tx *Tx
|
||||
if tx, err = rx.getTx(ctx); err != nil {
|
||||
return
|
||||
}
|
||||
return tx.Paged_ReportedSerial_By_ExpiresAt_LessOrEqual(ctx, reported_serial_expires_at_less_or_equal, limit, start)
|
||||
}
|
||||
|
||||
func (rx *Rx) ReplaceNoReturn_ReportedSerial(ctx context.Context,
|
||||
reported_serial_expires_at ReportedSerial_ExpiresAt_Field,
|
||||
reported_serial_storage_node_id ReportedSerial_StorageNodeId_Field,
|
||||
reported_serial_bucket_id ReportedSerial_BucketId_Field,
|
||||
reported_serial_action ReportedSerial_Action_Field,
|
||||
reported_serial_serial_number ReportedSerial_SerialNumber_Field,
|
||||
reported_serial_settled ReportedSerial_Settled_Field,
|
||||
reported_serial_observed_at ReportedSerial_ObservedAt_Field) (
|
||||
err error) {
|
||||
var tx *Tx
|
||||
if tx, err = rx.getTx(ctx); err != nil {
|
||||
return
|
||||
}
|
||||
return tx.ReplaceNoReturn_ReportedSerial(ctx, reported_serial_expires_at, reported_serial_storage_node_id, reported_serial_bucket_id, reported_serial_action, reported_serial_serial_number, reported_serial_settled, reported_serial_observed_at)
|
||||
|
||||
}
|
||||
|
||||
func (rx *Rx) UpdateNoReturn_AccountingTimestamps_By_Name(ctx context.Context,
|
||||
accounting_timestamps_name AccountingTimestamps_Name_Field,
|
||||
update AccountingTimestamps_Update_Fields) (
|
||||
@ -19607,6 +20059,10 @@ type Methods interface {
|
||||
project_id Project_Id_Field) (
|
||||
deleted bool, err error)
|
||||
|
||||
Delete_ReportedSerial_By_ExpiresAt_LessOrEqual(ctx context.Context,
|
||||
reported_serial_expires_at_less_or_equal ReportedSerial_ExpiresAt_Field) (
|
||||
count int64, err error)
|
||||
|
||||
Delete_ResetPasswordToken_By_Secret(ctx context.Context,
|
||||
reset_password_token_secret ResetPasswordToken_Secret_Field) (
|
||||
deleted bool, err error)
|
||||
@ -19850,6 +20306,21 @@ type Methods interface {
|
||||
limit int, offset int64) (
|
||||
rows []*StripecoinpaymentsInvoiceProjectRecord, err error)
|
||||
|
||||
Paged_ReportedSerial_By_ExpiresAt_LessOrEqual(ctx context.Context,
|
||||
reported_serial_expires_at_less_or_equal ReportedSerial_ExpiresAt_Field,
|
||||
limit int, start *Paged_ReportedSerial_By_ExpiresAt_LessOrEqual_Continuation) (
|
||||
rows []*ReportedSerial, next *Paged_ReportedSerial_By_ExpiresAt_LessOrEqual_Continuation, err error)
|
||||
|
||||
ReplaceNoReturn_ReportedSerial(ctx context.Context,
|
||||
reported_serial_expires_at ReportedSerial_ExpiresAt_Field,
|
||||
reported_serial_storage_node_id ReportedSerial_StorageNodeId_Field,
|
||||
reported_serial_bucket_id ReportedSerial_BucketId_Field,
|
||||
reported_serial_action ReportedSerial_Action_Field,
|
||||
reported_serial_serial_number ReportedSerial_SerialNumber_Field,
|
||||
reported_serial_settled ReportedSerial_Settled_Field,
|
||||
reported_serial_observed_at ReportedSerial_ObservedAt_Field) (
|
||||
err error)
|
||||
|
||||
UpdateNoReturn_AccountingTimestamps_By_Name(ctx context.Context,
|
||||
accounting_timestamps_name AccountingTimestamps_Name_Field,
|
||||
update AccountingTimestamps_Update_Fields) (
|
||||
|
@ -202,6 +202,16 @@ CREATE TABLE registration_tokens (
|
||||
PRIMARY KEY ( secret ),
|
||||
UNIQUE ( owner_id )
|
||||
);
|
||||
CREATE TABLE reported_serials (
|
||||
expires_at timestamp NOT NULL,
|
||||
storage_node_id bytea NOT NULL,
|
||||
bucket_id bytea NOT NULL,
|
||||
action integer NOT NULL,
|
||||
serial_number bytea NOT NULL,
|
||||
settled bigint NOT NULL,
|
||||
observed_at timestamp NOT NULL,
|
||||
PRIMARY KEY ( expires_at, storage_node_id, bucket_id, action, serial_number )
|
||||
);
|
||||
CREATE TABLE reset_password_tokens (
|
||||
secret bytea NOT NULL,
|
||||
owner_id bytea NOT NULL,
|
||||
|
@ -595,6 +595,23 @@ func (db *satelliteDB) PostgresMigration() *migrate.Migration {
|
||||
);`,
|
||||
},
|
||||
},
|
||||
{
|
||||
DB: db.DB,
|
||||
Description: "Create reported_serials table for faster order processing",
|
||||
Version: 77,
|
||||
Action: migrate.SQL{
|
||||
`CREATE TABLE reported_serials (
|
||||
expires_at timestamp NOT NULL,
|
||||
storage_node_id bytea NOT NULL,
|
||||
bucket_id bytea NOT NULL,
|
||||
action integer NOT NULL,
|
||||
serial_number bytea NOT NULL,
|
||||
settled bigint NOT NULL,
|
||||
observed_at timestamp NOT NULL,
|
||||
PRIMARY KEY ( expires_at, storage_node_id, bucket_id, action, serial_number )
|
||||
);`,
|
||||
},
|
||||
},
|
||||
},
|
||||
}
|
||||
}
|
||||
|
@ -4,7 +4,6 @@
|
||||
package satellitedb
|
||||
|
||||
import (
|
||||
"bytes"
|
||||
"context"
|
||||
"database/sql"
|
||||
"fmt"
|
||||
@ -32,6 +31,8 @@ var (
|
||||
|
||||
type ordersDB struct {
|
||||
db *satelliteDB
|
||||
|
||||
reportedRollupsReadBatchSize int
|
||||
}
|
||||
|
||||
// CreateSerialInfo creates serial number entry in database
|
||||
@ -165,7 +166,7 @@ func (db *ordersDB) GetBucketBandwidth(ctx context.Context, projectID uuid.UUID,
|
||||
if err == sql.ErrNoRows || sum == nil {
|
||||
return 0, nil
|
||||
}
|
||||
return *sum, err
|
||||
return *sum, Error.Wrap(err)
|
||||
}
|
||||
|
||||
// GetStorageNodeBandwidth gets total storage node bandwidth from period of time
|
||||
@ -196,7 +197,7 @@ func (db *ordersDB) ProcessOrders(ctx context.Context, requests []*orders.Proces
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
|
||||
if len(requests) == 0 {
|
||||
return nil, err
|
||||
return nil, nil
|
||||
}
|
||||
|
||||
// check that all requests are from the same storage node
|
||||
@ -212,173 +213,195 @@ func (db *ordersDB) ProcessOrders(ctx context.Context, requests []*orders.Proces
|
||||
return requests[i].OrderLimit.SerialNumber.Less(requests[k].OrderLimit.SerialNumber)
|
||||
})
|
||||
|
||||
// do a read only transaction to get all the project id/bucket ids
|
||||
var bucketIDs [][]byte
|
||||
err = db.db.WithTx(ctx, func(ctx context.Context, tx *dbx.Tx) error {
|
||||
responses, err = db.processOrdersInTx(ctx, requests, storageNodeID, time.Now(), tx.Tx)
|
||||
return err
|
||||
for _, request := range requests {
|
||||
row, err := tx.Find_SerialNumber_By_SerialNumber(ctx,
|
||||
dbx.SerialNumber_SerialNumber(request.Order.SerialNumber.Bytes()))
|
||||
if err != nil {
|
||||
return Error.Wrap(err)
|
||||
}
|
||||
if row != nil {
|
||||
bucketIDs = append(bucketIDs, row.BucketId)
|
||||
} else {
|
||||
bucketIDs = append(bucketIDs, nil)
|
||||
}
|
||||
}
|
||||
return nil
|
||||
})
|
||||
return responses, errs.Wrap(err)
|
||||
}
|
||||
|
||||
func (db *ordersDB) processOrdersInTx(ctx context.Context, requests []*orders.ProcessOrderRequest, storageNodeID storj.NodeID, now time.Time, tx *sql.Tx) (responses []*orders.ProcessOrderResponse, err error) {
|
||||
now = now.UTC()
|
||||
intervalStart := time.Date(now.Year(), now.Month(), now.Day(), now.Hour(), 0, 0, 0, now.Location())
|
||||
|
||||
rejected := make(map[storj.SerialNumber]bool)
|
||||
bucketBySerial := make(map[storj.SerialNumber][]byte)
|
||||
|
||||
// load the bucket id and insert into used serials table
|
||||
for _, request := range requests {
|
||||
row := tx.QueryRow(db.db.Rebind(`
|
||||
SELECT id, bucket_id
|
||||
FROM serial_numbers
|
||||
WHERE serial_number = ?
|
||||
`), request.OrderLimit.SerialNumber)
|
||||
|
||||
var serialNumberID int64
|
||||
var bucketID []byte
|
||||
if err := row.Scan(&serialNumberID, &bucketID); err == sql.ErrNoRows {
|
||||
rejected[request.OrderLimit.SerialNumber] = true
|
||||
continue
|
||||
} else if err != nil {
|
||||
return nil, Error.Wrap(err)
|
||||
}
|
||||
|
||||
var result sql.Result
|
||||
var count int64
|
||||
|
||||
// try to insert the serial number
|
||||
result, err = tx.Exec(db.db.Rebind(`
|
||||
INSERT INTO used_serials(serial_number_id, storage_node_id)
|
||||
VALUES (?, ?)
|
||||
ON CONFLICT DO NOTHING
|
||||
`), serialNumberID, storageNodeID)
|
||||
if err != nil {
|
||||
return nil, Error.Wrap(err)
|
||||
}
|
||||
|
||||
// if we didn't update any rows, then it must already exist
|
||||
count, err = result.RowsAffected()
|
||||
if err != nil {
|
||||
return nil, Error.Wrap(err)
|
||||
}
|
||||
if count == 0 {
|
||||
rejected[request.OrderLimit.SerialNumber] = true
|
||||
continue
|
||||
}
|
||||
|
||||
bucketBySerial[request.OrderLimit.SerialNumber] = bucketID
|
||||
if err != nil {
|
||||
return nil, Error.Wrap(err)
|
||||
}
|
||||
|
||||
// add up amount by action
|
||||
var largestAction pb.PieceAction
|
||||
amountByAction := map[pb.PieceAction]int64{}
|
||||
for _, request := range requests {
|
||||
if rejected[request.OrderLimit.SerialNumber] {
|
||||
continue
|
||||
}
|
||||
limit, order := request.OrderLimit, request.Order
|
||||
amountByAction[limit.Action] += order.Amount
|
||||
if largestAction < limit.Action {
|
||||
largestAction = limit.Action
|
||||
}
|
||||
}
|
||||
// perform all of the upserts into accounted serials table
|
||||
err = db.db.WithTx(ctx, func(ctx context.Context, tx *dbx.Tx) error {
|
||||
now := time.Now()
|
||||
|
||||
// do action updates for storage node
|
||||
for action := pb.PieceAction(0); action <= largestAction; action++ {
|
||||
amount := amountByAction[action]
|
||||
if amount == 0 {
|
||||
continue
|
||||
}
|
||||
for i, request := range requests {
|
||||
if bucketIDs[i] == nil {
|
||||
responses = append(responses, &orders.ProcessOrderResponse{
|
||||
SerialNumber: request.Order.SerialNumber,
|
||||
Status: pb.SettlementResponse_REJECTED,
|
||||
})
|
||||
continue
|
||||
}
|
||||
|
||||
_, err := tx.Exec(db.db.Rebind(`
|
||||
INSERT INTO storagenode_bandwidth_rollups
|
||||
(storagenode_id, interval_start, interval_seconds, action, settled)
|
||||
VALUES (?, ?, ?, ?, ?)
|
||||
ON CONFLICT (storagenode_id, interval_start, action)
|
||||
DO UPDATE SET settled = storagenode_bandwidth_rollups.settled + ?
|
||||
`), storageNodeID, intervalStart, defaultIntervalSeconds, action, amount, amount)
|
||||
if err != nil {
|
||||
return nil, Error.Wrap(err)
|
||||
}
|
||||
}
|
||||
// TODO: put them all in a single query?
|
||||
err = tx.ReplaceNoReturn_ReportedSerial(ctx,
|
||||
dbx.ReportedSerial_ExpiresAt(roundToNextDay(request.OrderLimit.OrderExpiration)),
|
||||
dbx.ReportedSerial_StorageNodeId(storageNodeID.Bytes()),
|
||||
dbx.ReportedSerial_BucketId(bucketIDs[i]),
|
||||
dbx.ReportedSerial_Action(uint(request.OrderLimit.Action)),
|
||||
dbx.ReportedSerial_SerialNumber(request.Order.SerialNumber.Bytes()),
|
||||
dbx.ReportedSerial_Settled(uint64(request.Order.Amount)),
|
||||
dbx.ReportedSerial_ObservedAt(now))
|
||||
if err != nil {
|
||||
return Error.Wrap(err)
|
||||
}
|
||||
|
||||
// sort bucket updates
|
||||
type bucketUpdate struct {
|
||||
bucketID []byte
|
||||
action pb.PieceAction
|
||||
amount int64
|
||||
}
|
||||
var bucketUpdates []bucketUpdate
|
||||
for _, request := range requests {
|
||||
if rejected[request.OrderLimit.SerialNumber] {
|
||||
continue
|
||||
}
|
||||
limit, order := request.OrderLimit, request.Order
|
||||
|
||||
bucketUpdates = append(bucketUpdates, bucketUpdate{
|
||||
bucketID: bucketBySerial[limit.SerialNumber],
|
||||
action: limit.Action,
|
||||
amount: order.Amount,
|
||||
})
|
||||
}
|
||||
|
||||
sort.Slice(bucketUpdates, func(i, k int) bool {
|
||||
compare := bytes.Compare(bucketUpdates[i].bucketID, bucketUpdates[k].bucketID)
|
||||
if compare == 0 {
|
||||
return bucketUpdates[i].action < bucketUpdates[k].action
|
||||
}
|
||||
return compare < 0
|
||||
})
|
||||
|
||||
// do bucket updates
|
||||
for _, update := range bucketUpdates {
|
||||
projectID, bucketName, err := orders.SplitBucketID(update.bucketID)
|
||||
if err != nil {
|
||||
return nil, errs.Wrap(err)
|
||||
}
|
||||
|
||||
_, err = tx.Exec(db.db.Rebind(`
|
||||
INSERT INTO bucket_bandwidth_rollups
|
||||
(bucket_name, project_id, interval_start, interval_seconds, action, inline, allocated, settled)
|
||||
VALUES (?, ?, ?, ?, ?, ?, ?, ?)
|
||||
ON CONFLICT (bucket_name, project_id, interval_start, action)
|
||||
DO UPDATE SET settled = bucket_bandwidth_rollups.settled + ?
|
||||
`), bucketName, (*projectID)[:], intervalStart, defaultIntervalSeconds, update.action, 0, 0, update.amount, update.amount)
|
||||
if err != nil {
|
||||
return nil, Error.Wrap(err)
|
||||
}
|
||||
}
|
||||
|
||||
for _, request := range requests {
|
||||
if !rejected[request.OrderLimit.SerialNumber] {
|
||||
responses = append(responses, &orders.ProcessOrderResponse{
|
||||
SerialNumber: request.OrderLimit.SerialNumber,
|
||||
SerialNumber: request.Order.SerialNumber,
|
||||
Status: pb.SettlementResponse_ACCEPTED,
|
||||
})
|
||||
} else {
|
||||
responses = append(responses, &orders.ProcessOrderResponse{
|
||||
SerialNumber: request.OrderLimit.SerialNumber,
|
||||
Status: pb.SettlementResponse_REJECTED,
|
||||
})
|
||||
}
|
||||
|
||||
return nil
|
||||
})
|
||||
if err != nil {
|
||||
return nil, Error.Wrap(err)
|
||||
}
|
||||
return responses, nil
|
||||
}
|
||||
|
||||
func (db *ordersDB) UpdateBucketBandwidthBatch(ctx context.Context, intervalStart time.Time, rollups []orders.BandwidthRollup) (err error) {
|
||||
func roundToNextDay(t time.Time) time.Time {
|
||||
return time.Date(t.Year(), t.Month(), t.Day(), 0, 0, 0, 0, t.Location()).AddDate(0, 0, 1)
|
||||
}
|
||||
|
||||
// GetBillableBandwidth gets total billable (expired consumed serial) bandwidth for nodes and buckets for all actions.
|
||||
func (db *ordersDB) GetBillableBandwidth(ctx context.Context, now time.Time) (
|
||||
bucketRollups []orders.BucketBandwidthRollup, storagenodeRollups []orders.StoragenodeBandwidthRollup, err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
|
||||
batchSize := db.reportedRollupsReadBatchSize
|
||||
if batchSize <= 0 {
|
||||
batchSize = 1000
|
||||
}
|
||||
|
||||
type storagenodeKey struct {
|
||||
nodeID storj.NodeID
|
||||
action pb.PieceAction
|
||||
}
|
||||
byStoragenode := make(map[storagenodeKey]uint64)
|
||||
|
||||
type bucketKey struct {
|
||||
projectID uuid.UUID
|
||||
bucketName string
|
||||
action pb.PieceAction
|
||||
}
|
||||
byBucket := make(map[bucketKey]uint64)
|
||||
|
||||
var token *dbx.Paged_ReportedSerial_By_ExpiresAt_LessOrEqual_Continuation
|
||||
var rows []*dbx.ReportedSerial
|
||||
|
||||
for {
|
||||
// We explicitly use a new transaction each time because we don't need the guarantees and
|
||||
// because we don't want a transaction reading for 1000 years.
|
||||
rows, token, err = db.db.Paged_ReportedSerial_By_ExpiresAt_LessOrEqual(ctx,
|
||||
dbx.ReportedSerial_ExpiresAt(now), batchSize, token)
|
||||
if err != nil {
|
||||
return nil, nil, Error.Wrap(err)
|
||||
}
|
||||
|
||||
for _, row := range rows {
|
||||
nodeID, err := storj.NodeIDFromBytes(row.StorageNodeId)
|
||||
if err != nil {
|
||||
db.db.log.Error("bad row inserted into reported serials",
|
||||
zap.Binary("storagenode_id", row.StorageNodeId))
|
||||
continue
|
||||
}
|
||||
projectID, bucketName, err := orders.SplitBucketID(row.BucketId)
|
||||
if err != nil {
|
||||
db.db.log.Error("bad row inserted into reported serials",
|
||||
zap.Binary("bucket_id", row.BucketId))
|
||||
continue
|
||||
}
|
||||
action := pb.PieceAction(row.Action)
|
||||
settled := row.Settled
|
||||
|
||||
byStoragenode[storagenodeKey{
|
||||
nodeID: nodeID,
|
||||
action: action,
|
||||
}] += settled
|
||||
|
||||
byBucket[bucketKey{
|
||||
projectID: *projectID,
|
||||
bucketName: string(bucketName),
|
||||
action: action,
|
||||
}] += settled
|
||||
}
|
||||
|
||||
if token == nil {
|
||||
break
|
||||
}
|
||||
}
|
||||
|
||||
for key, settled := range byBucket {
|
||||
bucketRollups = append(bucketRollups, orders.BucketBandwidthRollup{
|
||||
ProjectID: key.projectID,
|
||||
BucketName: key.bucketName,
|
||||
Action: key.action,
|
||||
Settled: int64(settled),
|
||||
})
|
||||
}
|
||||
|
||||
for key, settled := range byStoragenode {
|
||||
storagenodeRollups = append(storagenodeRollups, orders.StoragenodeBandwidthRollup{
|
||||
NodeID: key.nodeID,
|
||||
Action: key.action,
|
||||
Settled: int64(settled),
|
||||
})
|
||||
}
|
||||
|
||||
return bucketRollups, storagenodeRollups, nil
|
||||
}
|
||||
|
||||
//
|
||||
// transaction/batch methods
|
||||
//
|
||||
|
||||
type ordersDBTx struct {
|
||||
tx *dbx.Tx
|
||||
log *zap.Logger
|
||||
}
|
||||
|
||||
func (db *ordersDB) ExecuteInTx(ctx context.Context, cb func(ctx context.Context, tx orders.Transaction) error) (err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
|
||||
return db.db.WithTx(ctx, func(ctx context.Context, tx *dbx.Tx) error {
|
||||
return cb(ctx, &ordersDBTx{tx: tx, log: db.db.log})
|
||||
})
|
||||
}
|
||||
|
||||
func (tx *ordersDBTx) UpdateBucketBandwidthBatch(ctx context.Context, intervalStart time.Time, rollups []orders.BucketBandwidthRollup) (err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
|
||||
if len(rollups) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
orders.SortBucketBandwidthRollups(rollups)
|
||||
|
||||
const stmtBegin = `
|
||||
INSERT INTO bucket_bandwidth_rollups (bucket_name, project_id, interval_start, interval_seconds, action, inline, allocated, settled)
|
||||
VALUES
|
||||
`
|
||||
const stmtEnd = `
|
||||
ON CONFLICT(bucket_name, project_id, interval_start, action)
|
||||
DO UPDATE SET allocated = bucket_bandwidth_rollups.allocated + EXCLUDED.allocated, inline = bucket_bandwidth_rollups.inline + EXCLUDED.inline
|
||||
DO UPDATE SET
|
||||
allocated = bucket_bandwidth_rollups.allocated + EXCLUDED.allocated,
|
||||
inline = bucket_bandwidth_rollups.inline + EXCLUDED.inline,
|
||||
settled = bucket_bandwidth_rollups.settled + EXCLUDED.settled
|
||||
`
|
||||
|
||||
intervalStart = intervalStart.UTC()
|
||||
@ -411,13 +434,77 @@ func (db *ordersDB) UpdateBucketBandwidthBatch(ctx context.Context, intervalStar
|
||||
args = append(args, lastBucketName)
|
||||
bucketNameArgNum = len(args)
|
||||
}
|
||||
args = append(args, rollup.Action, rollup.Inline, rollup.Allocated)
|
||||
args = append(args, rollup.Action, rollup.Inline, rollup.Allocated, rollup.Settled)
|
||||
|
||||
stmt.WriteString(fmt.Sprintf(
|
||||
"($%d,$%d,$1,%d,$%d,$%d,$%d,0)",
|
||||
"($%d,$%d,$1,%d,$%d,$%d,$%d,$%d)",
|
||||
bucketNameArgNum,
|
||||
projectIDArgNum,
|
||||
defaultIntervalSeconds,
|
||||
len(args)-3,
|
||||
len(args)-2,
|
||||
len(args)-1,
|
||||
len(args),
|
||||
))
|
||||
}
|
||||
stmt.WriteString(stmtEnd)
|
||||
|
||||
_, err = tx.tx.Tx.ExecContext(ctx, stmt.String(), args...)
|
||||
if err != nil {
|
||||
tx.log.Error("Bucket bandwidth rollup batch flush failed.", zap.Error(err))
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
func (tx *ordersDBTx) UpdateStoragenodeBandwidthBatch(ctx context.Context, intervalStart time.Time, rollups []orders.StoragenodeBandwidthRollup) (err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
|
||||
if len(rollups) == 0 {
|
||||
return nil
|
||||
}
|
||||
|
||||
orders.SortStoragenodeBandwidthRollups(rollups)
|
||||
|
||||
const stmtBegin = `
|
||||
INSERT INTO storagenode_bandwidth_rollups (storagenode_id, interval_start, interval_seconds, action, allocated, settled)
|
||||
VALUES
|
||||
`
|
||||
const stmtEnd = `
|
||||
ON CONFLICT(storagenode_id, interval_start, action)
|
||||
DO UPDATE SET
|
||||
allocated = storagenode_bandwidth_rollups.allocated + EXCLUDED.allocated,
|
||||
settled = storagenode_bandwidth_rollups.settled + EXCLUDED.settled
|
||||
`
|
||||
|
||||
intervalStart = intervalStart.UTC()
|
||||
intervalStart = time.Date(intervalStart.Year(), intervalStart.Month(), intervalStart.Day(), intervalStart.Hour(), 0, 0, 0, time.UTC)
|
||||
|
||||
var lastNodeID storj.NodeID
|
||||
var nodeIDArgNum int
|
||||
var args []interface{}
|
||||
|
||||
var stmt strings.Builder
|
||||
stmt.WriteString(stmtBegin)
|
||||
|
||||
args = append(args, intervalStart)
|
||||
for i, rollup := range rollups {
|
||||
if i > 0 {
|
||||
stmt.WriteString(",")
|
||||
}
|
||||
if lastNodeID != rollup.NodeID {
|
||||
lastNodeID = rollup.NodeID
|
||||
// take the slice over rollup.ProjectID, because it is going to stay
|
||||
// the same up to the ExecContext call, whereas lastProjectID is likely
|
||||
// to be overwritten
|
||||
args = append(args, rollup.NodeID.Bytes())
|
||||
nodeIDArgNum = len(args)
|
||||
}
|
||||
args = append(args, rollup.Action, rollup.Allocated, rollup.Settled)
|
||||
|
||||
stmt.WriteString(fmt.Sprintf(
|
||||
"($%d,$1,%d,$%d,$%d,$%d)",
|
||||
nodeIDArgNum,
|
||||
defaultIntervalSeconds,
|
||||
len(args)-2,
|
||||
len(args)-1,
|
||||
len(args),
|
||||
@ -425,9 +512,18 @@ func (db *ordersDB) UpdateBucketBandwidthBatch(ctx context.Context, intervalStar
|
||||
}
|
||||
stmt.WriteString(stmtEnd)
|
||||
|
||||
_, err = db.db.ExecContext(ctx, stmt.String(), args...)
|
||||
_, err = tx.tx.Tx.ExecContext(ctx, stmt.String(), args...)
|
||||
if err != nil {
|
||||
db.db.log.Error("Bucket bandwidth rollup batch flush failed.", zap.Error(err))
|
||||
tx.log.Error("Storagenode bandwidth rollup batch flush failed.", zap.Error(err))
|
||||
}
|
||||
return err
|
||||
}
|
||||
|
||||
// DeleteExpiredReportedSerials deletes any expired reported serials as of expiredThreshold.
|
||||
func (tx *ordersDBTx) DeleteExpiredReportedSerials(ctx context.Context, expiredThreshold time.Time) (err error) {
|
||||
defer mon.Task()(&ctx)(&err)
|
||||
|
||||
_, err = tx.tx.Delete_ReportedSerial_By_ExpiresAt_LessOrEqual(ctx,
|
||||
dbx.ReportedSerial_ExpiresAt(expiredThreshold))
|
||||
return Error.Wrap(err)
|
||||
}
|
||||
|
454
satellite/satellitedb/testdata/postgres.v77.sql
vendored
Normal file
454
satellite/satellitedb/testdata/postgres.v77.sql
vendored
Normal file
@ -0,0 +1,454 @@
|
||||
-- AUTOGENERATED BY storj.io/dbx
|
||||
-- DO NOT EDIT
|
||||
CREATE TABLE accounting_rollups (
|
||||
id bigserial NOT NULL,
|
||||
node_id bytea NOT NULL,
|
||||
start_time timestamp with time zone NOT NULL,
|
||||
put_total bigint NOT NULL,
|
||||
get_total bigint NOT NULL,
|
||||
get_audit_total bigint NOT NULL,
|
||||
get_repair_total bigint NOT NULL,
|
||||
put_repair_total bigint NOT NULL,
|
||||
at_rest_total double precision NOT NULL,
|
||||
PRIMARY KEY ( id )
|
||||
);
|
||||
CREATE TABLE accounting_timestamps (
|
||||
name text NOT NULL,
|
||||
value timestamp with time zone NOT NULL,
|
||||
PRIMARY KEY ( name )
|
||||
);
|
||||
CREATE TABLE bucket_bandwidth_rollups (
|
||||
bucket_name bytea NOT NULL,
|
||||
project_id bytea NOT NULL,
|
||||
interval_start timestamp NOT NULL,
|
||||
interval_seconds integer NOT NULL,
|
||||
action integer NOT NULL,
|
||||
inline bigint NOT NULL,
|
||||
allocated bigint NOT NULL,
|
||||
settled bigint NOT NULL,
|
||||
PRIMARY KEY ( bucket_name, project_id, interval_start, action )
|
||||
);
|
||||
CREATE TABLE bucket_storage_tallies (
|
||||
bucket_name bytea NOT NULL,
|
||||
project_id bytea NOT NULL,
|
||||
interval_start timestamp NOT NULL,
|
||||
inline bigint NOT NULL,
|
||||
remote bigint NOT NULL,
|
||||
remote_segments_count integer NOT NULL,
|
||||
inline_segments_count integer NOT NULL,
|
||||
object_count integer NOT NULL,
|
||||
metadata_size bigint NOT NULL,
|
||||
PRIMARY KEY ( bucket_name, project_id, interval_start )
|
||||
);
|
||||
CREATE TABLE coinpayments_transactions (
|
||||
id text NOT NULL,
|
||||
user_id bytea NOT NULL,
|
||||
address text NOT NULL,
|
||||
amount bytea NOT NULL,
|
||||
received bytea NOT NULL,
|
||||
status integer NOT NULL,
|
||||
key text NOT NULL,
|
||||
timeout integer NOT NULL,
|
||||
created_at timestamp with time zone NOT NULL,
|
||||
PRIMARY KEY ( id )
|
||||
);
|
||||
CREATE TABLE coupons (
|
||||
id bytea NOT NULL,
|
||||
project_id bytea NOT NULL,
|
||||
user_id bytea NOT NULL,
|
||||
amount bigint NOT NULL,
|
||||
description text NOT NULL,
|
||||
type integer NOT NULL,
|
||||
status integer NOT NULL,
|
||||
duration bigint NOT NULL,
|
||||
created_at timestamp with time zone NOT NULL,
|
||||
PRIMARY KEY ( id )
|
||||
);
|
||||
CREATE TABLE coupon_usages (
|
||||
coupon_id bytea NOT NULL,
|
||||
amount bigint NOT NULL,
|
||||
status integer NOT NULL,
|
||||
period timestamp with time zone NOT NULL,
|
||||
PRIMARY KEY ( coupon_id, period )
|
||||
);
|
||||
CREATE TABLE graceful_exit_progress (
|
||||
node_id bytea NOT NULL,
|
||||
bytes_transferred bigint NOT NULL,
|
||||
pieces_transferred bigint NOT NULL,
|
||||
pieces_failed bigint NOT NULL,
|
||||
updated_at timestamp NOT NULL,
|
||||
PRIMARY KEY ( node_id )
|
||||
);
|
||||
CREATE TABLE graceful_exit_transfer_queue (
|
||||
node_id bytea NOT NULL,
|
||||
path bytea NOT NULL,
|
||||
piece_num integer NOT NULL,
|
||||
root_piece_id bytea,
|
||||
durability_ratio double precision NOT NULL,
|
||||
queued_at timestamp NOT NULL,
|
||||
requested_at timestamp,
|
||||
last_failed_at timestamp,
|
||||
last_failed_code integer,
|
||||
failed_count integer,
|
||||
finished_at timestamp,
|
||||
order_limit_send_count integer NOT NULL,
|
||||
PRIMARY KEY ( node_id, path, piece_num )
|
||||
);
|
||||
CREATE TABLE injuredsegments (
|
||||
path bytea NOT NULL,
|
||||
data bytea NOT NULL,
|
||||
attempted timestamp,
|
||||
PRIMARY KEY ( path )
|
||||
);
|
||||
CREATE TABLE irreparabledbs (
|
||||
segmentpath bytea NOT NULL,
|
||||
segmentdetail bytea NOT NULL,
|
||||
pieces_lost_count bigint NOT NULL,
|
||||
seg_damaged_unix_sec bigint NOT NULL,
|
||||
repair_attempt_count bigint NOT NULL,
|
||||
PRIMARY KEY ( segmentpath )
|
||||
);
|
||||
CREATE TABLE nodes (
|
||||
id bytea NOT NULL,
|
||||
address text NOT NULL,
|
||||
last_net text NOT NULL,
|
||||
protocol integer NOT NULL,
|
||||
type integer NOT NULL,
|
||||
email text NOT NULL,
|
||||
wallet text NOT NULL,
|
||||
free_bandwidth bigint NOT NULL,
|
||||
free_disk bigint NOT NULL,
|
||||
piece_count bigint NOT NULL,
|
||||
major bigint NOT NULL,
|
||||
minor bigint NOT NULL,
|
||||
patch bigint NOT NULL,
|
||||
hash text NOT NULL,
|
||||
timestamp timestamp with time zone NOT NULL,
|
||||
release boolean NOT NULL,
|
||||
latency_90 bigint NOT NULL,
|
||||
audit_success_count bigint NOT NULL,
|
||||
total_audit_count bigint NOT NULL,
|
||||
uptime_success_count bigint NOT NULL,
|
||||
total_uptime_count bigint NOT NULL,
|
||||
created_at timestamp with time zone NOT NULL,
|
||||
updated_at timestamp with time zone NOT NULL,
|
||||
last_contact_success timestamp with time zone NOT NULL,
|
||||
last_contact_failure timestamp with time zone NOT NULL,
|
||||
contained boolean NOT NULL,
|
||||
disqualified timestamp with time zone,
|
||||
audit_reputation_alpha double precision NOT NULL,
|
||||
audit_reputation_beta double precision NOT NULL,
|
||||
uptime_reputation_alpha double precision NOT NULL,
|
||||
uptime_reputation_beta double precision NOT NULL,
|
||||
exit_initiated_at timestamp,
|
||||
exit_loop_completed_at timestamp,
|
||||
exit_finished_at timestamp,
|
||||
exit_success boolean NOT NULL,
|
||||
PRIMARY KEY ( id )
|
||||
);
|
||||
CREATE TABLE nodes_offline_times (
|
||||
node_id bytea NOT NULL,
|
||||
tracked_at timestamp with time zone NOT NULL,
|
||||
seconds integer NOT NULL,
|
||||
PRIMARY KEY ( node_id, tracked_at )
|
||||
);
|
||||
CREATE TABLE offers (
|
||||
id serial NOT NULL,
|
||||
name text NOT NULL,
|
||||
description text NOT NULL,
|
||||
award_credit_in_cents integer NOT NULL,
|
||||
invitee_credit_in_cents integer NOT NULL,
|
||||
award_credit_duration_days integer,
|
||||
invitee_credit_duration_days integer,
|
||||
redeemable_cap integer,
|
||||
expires_at timestamp with time zone NOT NULL,
|
||||
created_at timestamp with time zone NOT NULL,
|
||||
status integer NOT NULL,
|
||||
type integer NOT NULL,
|
||||
PRIMARY KEY ( id )
|
||||
);
|
||||
CREATE TABLE peer_identities (
|
||||
node_id bytea NOT NULL,
|
||||
leaf_serial_number bytea NOT NULL,
|
||||
chain bytea NOT NULL,
|
||||
updated_at timestamp with time zone NOT NULL,
|
||||
PRIMARY KEY ( node_id )
|
||||
);
|
||||
CREATE TABLE pending_audits (
|
||||
node_id bytea NOT NULL,
|
||||
piece_id bytea NOT NULL,
|
||||
stripe_index bigint NOT NULL,
|
||||
share_size bigint NOT NULL,
|
||||
expected_share_hash bytea NOT NULL,
|
||||
reverify_count bigint NOT NULL,
|
||||
path bytea NOT NULL,
|
||||
PRIMARY KEY ( node_id )
|
||||
);
|
||||
CREATE TABLE projects (
|
||||
id bytea NOT NULL,
|
||||
name text NOT NULL,
|
||||
description text NOT NULL,
|
||||
usage_limit bigint NOT NULL,
|
||||
partner_id bytea,
|
||||
owner_id bytea NOT NULL,
|
||||
created_at timestamp with time zone NOT NULL,
|
||||
PRIMARY KEY ( id )
|
||||
);
|
||||
CREATE TABLE registration_tokens (
|
||||
secret bytea NOT NULL,
|
||||
owner_id bytea,
|
||||
project_limit integer NOT NULL,
|
||||
created_at timestamp with time zone NOT NULL,
|
||||
PRIMARY KEY ( secret ),
|
||||
UNIQUE ( owner_id )
|
||||
);
|
||||
CREATE TABLE reported_serials (
|
||||
expires_at timestamp NOT NULL,
|
||||
storage_node_id bytea NOT NULL,
|
||||
bucket_id bytea NOT NULL,
|
||||
action integer NOT NULL,
|
||||
serial_number bytea NOT NULL,
|
||||
settled bigint NOT NULL,
|
||||
observed_at timestamp NOT NULL,
|
||||
PRIMARY KEY ( expires_at, storage_node_id, bucket_id, action, serial_number )
|
||||
);
|
||||
CREATE TABLE reset_password_tokens (
|
||||
secret bytea NOT NULL,
|
||||
owner_id bytea NOT NULL,
|
||||
created_at timestamp with time zone NOT NULL,
|
||||
PRIMARY KEY ( secret ),
|
||||
UNIQUE ( owner_id )
|
||||
);
|
||||
CREATE TABLE serial_numbers (
|
||||
id serial NOT NULL,
|
||||
serial_number bytea NOT NULL,
|
||||
bucket_id bytea NOT NULL,
|
||||
expires_at timestamp NOT NULL,
|
||||
PRIMARY KEY ( id )
|
||||
);
|
||||
CREATE TABLE storagenode_bandwidth_rollups (
|
||||
storagenode_id bytea NOT NULL,
|
||||
interval_start timestamp NOT NULL,
|
||||
interval_seconds integer NOT NULL,
|
||||
action integer NOT NULL,
|
||||
allocated bigint DEFAULT 0,
|
||||
settled bigint NOT NULL,
|
||||
PRIMARY KEY ( storagenode_id, interval_start, action )
|
||||
);
|
||||
CREATE TABLE storagenode_storage_tallies (
|
||||
id bigserial NOT NULL,
|
||||
node_id bytea NOT NULL,
|
||||
interval_end_time timestamp with time zone NOT NULL,
|
||||
data_total double precision NOT NULL,
|
||||
PRIMARY KEY ( id )
|
||||
);
|
||||
CREATE TABLE stripe_customers (
|
||||
user_id bytea NOT NULL,
|
||||
customer_id text NOT NULL,
|
||||
created_at timestamp with time zone NOT NULL,
|
||||
PRIMARY KEY ( user_id ),
|
||||
UNIQUE ( customer_id )
|
||||
);
|
||||
CREATE TABLE stripecoinpayments_invoice_project_records (
|
||||
id bytea NOT NULL,
|
||||
project_id bytea NOT NULL,
|
||||
storage double precision NOT NULL,
|
||||
egress bigint NOT NULL,
|
||||
objects bigint NOT NULL,
|
||||
period_start timestamp with time zone NOT NULL,
|
||||
period_end timestamp with time zone NOT NULL,
|
||||
state integer NOT NULL,
|
||||
created_at timestamp with time zone NOT NULL,
|
||||
PRIMARY KEY ( id ),
|
||||
UNIQUE ( project_id, period_start, period_end )
|
||||
);
|
||||
CREATE TABLE stripecoinpayments_tx_conversion_rates (
|
||||
tx_id text NOT NULL,
|
||||
rate bytea NOT NULL,
|
||||
created_at timestamp with time zone NOT NULL,
|
||||
PRIMARY KEY ( tx_id )
|
||||
);
|
||||
CREATE TABLE users (
|
||||
id bytea NOT NULL,
|
||||
email text NOT NULL,
|
||||
normalized_email text NOT NULL,
|
||||
full_name text NOT NULL,
|
||||
short_name text,
|
||||
password_hash bytea NOT NULL,
|
||||
status integer NOT NULL,
|
||||
partner_id bytea,
|
||||
created_at timestamp with time zone NOT NULL,
|
||||
PRIMARY KEY ( id )
|
||||
);
|
||||
CREATE TABLE value_attributions (
|
||||
project_id bytea NOT NULL,
|
||||
bucket_name bytea NOT NULL,
|
||||
partner_id bytea NOT NULL,
|
||||
last_updated timestamp NOT NULL,
|
||||
PRIMARY KEY ( project_id, bucket_name )
|
||||
);
|
||||
CREATE TABLE api_keys (
|
||||
id bytea NOT NULL,
|
||||
project_id bytea NOT NULL REFERENCES projects( id ) ON DELETE CASCADE,
|
||||
head bytea NOT NULL,
|
||||
name text NOT NULL,
|
||||
secret bytea NOT NULL,
|
||||
partner_id bytea,
|
||||
created_at timestamp with time zone NOT NULL,
|
||||
PRIMARY KEY ( id ),
|
||||
UNIQUE ( head ),
|
||||
UNIQUE ( name, project_id )
|
||||
);
|
||||
CREATE TABLE bucket_metainfos (
|
||||
id bytea NOT NULL,
|
||||
project_id bytea NOT NULL REFERENCES projects( id ),
|
||||
name bytea NOT NULL,
|
||||
partner_id bytea,
|
||||
path_cipher integer NOT NULL,
|
||||
created_at timestamp with time zone NOT NULL,
|
||||
default_segment_size integer NOT NULL,
|
||||
default_encryption_cipher_suite integer NOT NULL,
|
||||
default_encryption_block_size integer NOT NULL,
|
||||
default_redundancy_algorithm integer NOT NULL,
|
||||
default_redundancy_share_size integer NOT NULL,
|
||||
default_redundancy_required_shares integer NOT NULL,
|
||||
default_redundancy_repair_shares integer NOT NULL,
|
||||
default_redundancy_optimal_shares integer NOT NULL,
|
||||
default_redundancy_total_shares integer NOT NULL,
|
||||
PRIMARY KEY ( id ),
|
||||
UNIQUE ( name, project_id )
|
||||
);
|
||||
CREATE TABLE project_invoice_stamps (
|
||||
project_id bytea NOT NULL REFERENCES projects( id ) ON DELETE CASCADE,
|
||||
invoice_id bytea NOT NULL,
|
||||
start_date timestamp with time zone NOT NULL,
|
||||
end_date timestamp with time zone NOT NULL,
|
||||
created_at timestamp with time zone NOT NULL,
|
||||
PRIMARY KEY ( project_id, start_date, end_date ),
|
||||
UNIQUE ( invoice_id )
|
||||
);
|
||||
CREATE TABLE project_members (
|
||||
member_id bytea NOT NULL REFERENCES users( id ) ON DELETE CASCADE,
|
||||
project_id bytea NOT NULL REFERENCES projects( id ) ON DELETE CASCADE,
|
||||
created_at timestamp with time zone NOT NULL,
|
||||
PRIMARY KEY ( member_id, project_id )
|
||||
);
|
||||
CREATE TABLE stripecoinpayments_apply_balance_intents (
|
||||
tx_id text NOT NULL REFERENCES coinpayments_transactions( id ) ON DELETE CASCADE,
|
||||
state integer NOT NULL,
|
||||
created_at timestamp with time zone NOT NULL,
|
||||
PRIMARY KEY ( tx_id )
|
||||
);
|
||||
CREATE TABLE used_serials (
|
||||
serial_number_id integer NOT NULL REFERENCES serial_numbers( id ) ON DELETE CASCADE,
|
||||
storage_node_id bytea NOT NULL,
|
||||
PRIMARY KEY ( serial_number_id, storage_node_id )
|
||||
);
|
||||
CREATE TABLE user_credits (
|
||||
id serial NOT NULL,
|
||||
user_id bytea NOT NULL REFERENCES users( id ) ON DELETE CASCADE,
|
||||
offer_id integer NOT NULL REFERENCES offers( id ),
|
||||
referred_by bytea REFERENCES users( id ) ON DELETE SET NULL,
|
||||
type text NOT NULL,
|
||||
credits_earned_in_cents integer NOT NULL,
|
||||
credits_used_in_cents integer NOT NULL,
|
||||
expires_at timestamp with time zone NOT NULL,
|
||||
created_at timestamp with time zone NOT NULL,
|
||||
PRIMARY KEY ( id ),
|
||||
UNIQUE ( id, offer_id )
|
||||
);
|
||||
CREATE INDEX bucket_name_project_id_interval_start_interval_seconds ON bucket_bandwidth_rollups ( bucket_name, project_id, interval_start, interval_seconds );
|
||||
CREATE INDEX injuredsegments_attempted_index ON injuredsegments ( attempted );
|
||||
CREATE INDEX node_last_ip ON nodes ( last_net );
|
||||
CREATE INDEX nodes_offline_times_node_id_index ON nodes_offline_times ( node_id );
|
||||
CREATE UNIQUE INDEX serial_number ON serial_numbers ( serial_number );
|
||||
CREATE INDEX serial_numbers_expires_at_index ON serial_numbers ( expires_at );
|
||||
CREATE INDEX storagenode_id_interval_start_interval_seconds ON storagenode_bandwidth_rollups ( storagenode_id, interval_start, interval_seconds );
|
||||
CREATE UNIQUE INDEX credits_earned_user_id_offer_id ON user_credits ( id, offer_id );
|
||||
|
||||
INSERT INTO "accounting_rollups"("id", "node_id", "start_time", "put_total", "get_total", "get_audit_total", "get_repair_total", "put_repair_total", "at_rest_total") VALUES (1, E'\\367M\\177\\251]t/\\022\\256\\214\\265\\025\\224\\204:\\217\\212\\0102<\\321\\374\\020&\\271Qc\\325\\261\\354\\246\\233'::bytea, '2019-02-09 00:00:00+00', 1000, 2000, 3000, 4000, 0, 5000);
|
||||
|
||||
INSERT INTO "accounting_timestamps" VALUES ('LastAtRestTally', '0001-01-01 00:00:00+00');
|
||||
INSERT INTO "accounting_timestamps" VALUES ('LastRollup', '0001-01-01 00:00:00+00');
|
||||
INSERT INTO "accounting_timestamps" VALUES ('LastBandwidthTally', '0001-01-01 00:00:00+00');
|
||||
|
||||
INSERT INTO "nodes"("id", "address", "last_net", "protocol", "type", "email", "wallet", "free_bandwidth", "free_disk", "piece_count", "major", "minor", "patch", "hash", "timestamp", "release","latency_90", "audit_success_count", "total_audit_count", "uptime_success_count", "total_uptime_count", "created_at", "updated_at", "last_contact_success", "last_contact_failure", "contained", "disqualified", "audit_reputation_alpha", "audit_reputation_beta", "uptime_reputation_alpha", "uptime_reputation_beta", "exit_success") VALUES (E'\\153\\313\\233\\074\\327\\177\\136\\070\\346\\001', '127.0.0.1:55516', '', 0, 4, '', '', -1, -1, 0, 0, 1, 0, '', 'epoch', false, 0, 0, 5, 0, 5, '2019-02-14 08:07:31.028103+00', '2019-02-14 08:07:31.108963+00', 'epoch', 'epoch', false, NULL, 50, 0, 100, 5, false);
|
||||
INSERT INTO "nodes"("id", "address", "last_net", "protocol", "type", "email", "wallet", "free_bandwidth", "free_disk", "piece_count", "major", "minor", "patch", "hash", "timestamp", "release","latency_90", "audit_success_count", "total_audit_count", "uptime_success_count", "total_uptime_count", "created_at", "updated_at", "last_contact_success", "last_contact_failure", "contained", "disqualified", "audit_reputation_alpha", "audit_reputation_beta", "uptime_reputation_alpha", "uptime_reputation_beta", "exit_success") VALUES (E'\\006\\223\\250R\\221\\005\\365\\377v>0\\266\\365\\216\\255?\\347\\244\\371?2\\264\\262\\230\\007<\\001\\262\\263\\237\\247n', '127.0.0.1:55518', '', 0, 4, '', '', -1, -1, 0, 0, 1, 0, '', 'epoch', false, 0, 0, 0, 3, 3, '2019-02-14 08:07:31.028103+00', '2019-02-14 08:07:31.108963+00', 'epoch', 'epoch', false, NULL, 50, 0, 100, 0, false);
|
||||
INSERT INTO "nodes"("id", "address", "last_net", "protocol", "type", "email", "wallet", "free_bandwidth", "free_disk", "piece_count", "major", "minor", "patch", "hash", "timestamp", "release","latency_90", "audit_success_count", "total_audit_count", "uptime_success_count", "total_uptime_count", "created_at", "updated_at", "last_contact_success", "last_contact_failure", "contained", "disqualified", "audit_reputation_alpha", "audit_reputation_beta", "uptime_reputation_alpha", "uptime_reputation_beta", "exit_success") VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\014', '127.0.0.1:55517', '', 0, 4, '', '', -1, -1, 0, 0, 1, 0, '', 'epoch', false, 0, 0, 0, 0, 0, '2019-02-14 08:07:31.028103+00', '2019-02-14 08:07:31.108963+00', 'epoch', 'epoch', false, NULL, 50, 0, 100, 0, false);
|
||||
INSERT INTO "nodes"("id", "address", "last_net", "protocol", "type", "email", "wallet", "free_bandwidth", "free_disk", "piece_count", "major", "minor", "patch", "hash", "timestamp", "release","latency_90", "audit_success_count", "total_audit_count", "uptime_success_count", "total_uptime_count", "created_at", "updated_at", "last_contact_success", "last_contact_failure", "contained", "disqualified", "audit_reputation_alpha", "audit_reputation_beta", "uptime_reputation_alpha", "uptime_reputation_beta", "exit_success") VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\015', '127.0.0.1:55519', '', 0, 4, '', '', -1, -1, 0, 0, 1, 0, '', 'epoch', false, 0, 1, 2, 1, 2, '2019-02-14 08:07:31.028103+00', '2019-02-14 08:07:31.108963+00', 'epoch', 'epoch', false, NULL, 50, 0, 100, 1, false);
|
||||
INSERT INTO "nodes"("id", "address", "last_net", "protocol", "type", "email", "wallet", "free_bandwidth", "free_disk", "piece_count", "major", "minor", "patch", "hash", "timestamp", "release","latency_90", "audit_success_count", "total_audit_count", "uptime_success_count", "total_uptime_count", "created_at", "updated_at", "last_contact_success", "last_contact_failure", "contained", "disqualified", "audit_reputation_alpha", "audit_reputation_beta", "uptime_reputation_alpha", "uptime_reputation_beta", "exit_success") VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\016', '127.0.0.1:55520', '', 0, 4, '', '', -1, -1, 0, 0, 1, 0, '', 'epoch', false, 0, 300, 400, 300, 400, '2019-02-14 08:07:31.028103+00', '2019-02-14 08:07:31.108963+00', 'epoch', 'epoch', false, NULL, 300, 0, 300, 100, false);
|
||||
|
||||
INSERT INTO "users"("id", "full_name", "short_name", "email", "normalized_email", "password_hash", "status", "partner_id", "created_at") VALUES (E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, 'Noahson', 'William', '1email1@mail.test', '1EMAIL1@MAIL.TEST', E'some_readable_hash'::bytea, 1, NULL, '2019-02-14 08:28:24.614594+00');
|
||||
INSERT INTO "projects"("id", "name", "description", "usage_limit", "partner_id", "owner_id", "created_at") VALUES (E'\\022\\217/\\014\\376!K\\023\\276\\031\\311}m\\236\\205\\300'::bytea, 'ProjectName', 'projects description', 0, NULL, E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, '2019-02-14 08:28:24.254934+00');
|
||||
|
||||
INSERT INTO "projects"("id", "name", "description", "usage_limit", "partner_id", "owner_id", "created_at") VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\014'::bytea, 'projName1', 'Test project 1', 0, NULL, E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, '2019-02-14 08:28:24.636949+00');
|
||||
INSERT INTO "project_members"("member_id", "project_id", "created_at") VALUES (E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\014'::bytea, '2019-02-14 08:28:24.677953+00');
|
||||
INSERT INTO "project_members"("member_id", "project_id", "created_at") VALUES (E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, E'\\022\\217/\\014\\376!K\\023\\276\\031\\311}m\\236\\205\\300'::bytea, '2019-02-13 08:28:24.677953+00');
|
||||
|
||||
INSERT INTO "irreparabledbs" ("segmentpath", "segmentdetail", "pieces_lost_count", "seg_damaged_unix_sec", "repair_attempt_count") VALUES ('\x49616d5365676d656e746b6579696e666f30', '\x49616d5365676d656e7464657461696c696e666f30', 10, 1550159554, 10);
|
||||
|
||||
INSERT INTO "injuredsegments" ("path", "data") VALUES ('0', '\x0a0130120100');
|
||||
INSERT INTO "injuredsegments" ("path", "data") VALUES ('here''s/a/great/path', '\x0a136865726527732f612f67726561742f70617468120a0102030405060708090a');
|
||||
INSERT INTO "injuredsegments" ("path", "data") VALUES ('yet/another/cool/path', '\x0a157965742f616e6f746865722f636f6f6c2f70617468120a0102030405060708090a');
|
||||
INSERT INTO "injuredsegments" ("path", "data") VALUES ('so/many/iconic/paths/to/choose/from', '\x0a23736f2f6d616e792f69636f6e69632f70617468732f746f2f63686f6f73652f66726f6d120a0102030405060708090a');
|
||||
|
||||
INSERT INTO "registration_tokens" ("secret", "owner_id", "project_limit", "created_at") VALUES (E'\\070\\127\\144\\013\\332\\344\\102\\376\\306\\056\\303\\130\\106\\132\\321\\276\\321\\274\\170\\264\\054\\333\\221\\116\\154\\221\\335\\070\\220\\146\\344\\216'::bytea, null, 1, '2019-02-14 08:28:24.677953+00');
|
||||
|
||||
INSERT INTO "serial_numbers" ("id", "serial_number", "bucket_id", "expires_at") VALUES (1, E'0123456701234567'::bytea, E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\014/testbucket'::bytea, '2019-03-06 08:28:24.677953+00');
|
||||
INSERT INTO "used_serials" ("serial_number_id", "storage_node_id") VALUES (1, E'\\006\\223\\250R\\221\\005\\365\\377v>0\\266\\365\\216\\255?\\347\\244\\371?2\\264\\262\\230\\007<\\001\\262\\263\\237\\247n');
|
||||
|
||||
INSERT INTO "storagenode_bandwidth_rollups" ("storagenode_id", "interval_start", "interval_seconds", "action", "allocated", "settled") VALUES (E'\\006\\223\\250R\\221\\005\\365\\377v>0\\266\\365\\216\\255?\\347\\244\\371?2\\264\\262\\230\\007<\\001\\262\\263\\237\\247n', '2019-03-06 08:00:00.000000+00', 3600, 1, 1024, 2024);
|
||||
INSERT INTO "storagenode_storage_tallies" VALUES (1, E'\\3510\\323\\225"~\\036<\\342\\330m\\0253Jhr\\246\\233K\\246#\\2303\\351\\256\\275j\\212UM\\362\\207', '2019-02-14 08:16:57.812849+00', 1000);
|
||||
|
||||
INSERT INTO "bucket_bandwidth_rollups" ("bucket_name", "project_id", "interval_start", "interval_seconds", "action", "inline", "allocated", "settled") VALUES (E'testbucket'::bytea, E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\014'::bytea,'2019-03-06 08:00:00.000000+00', 3600, 1, 1024, 2024, 3024);
|
||||
INSERT INTO "bucket_storage_tallies" ("bucket_name", "project_id", "interval_start", "inline", "remote", "remote_segments_count", "inline_segments_count", "object_count", "metadata_size") VALUES (E'testbucket'::bytea, E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\014'::bytea,'2019-03-06 08:00:00.000000+00', 4024, 5024, 0, 0, 0, 0);
|
||||
INSERT INTO "bucket_bandwidth_rollups" ("bucket_name", "project_id", "interval_start", "interval_seconds", "action", "inline", "allocated", "settled") VALUES (E'testbucket'::bytea, E'\\170\\160\\157\\370\\274\\366\\113\\364\\272\\235\\301\\243\\321\\102\\321\\136'::bytea,'2019-03-06 08:00:00.000000+00', 3600, 1, 1024, 2024, 3024);
|
||||
INSERT INTO "bucket_storage_tallies" ("bucket_name", "project_id", "interval_start", "inline", "remote", "remote_segments_count", "inline_segments_count", "object_count", "metadata_size") VALUES (E'testbucket'::bytea, E'\\170\\160\\157\\370\\274\\366\\113\\364\\272\\235\\301\\243\\321\\102\\321\\136'::bytea,'2019-03-06 08:00:00.000000+00', 4024, 5024, 0, 0, 0, 0);
|
||||
|
||||
INSERT INTO "reset_password_tokens" ("secret", "owner_id", "created_at") VALUES (E'\\070\\127\\144\\013\\332\\344\\102\\376\\306\\056\\303\\130\\106\\132\\321\\276\\321\\274\\170\\264\\054\\333\\221\\116\\154\\221\\335\\070\\220\\146\\344\\216'::bytea, E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, '2019-05-08 08:28:24.677953+00');
|
||||
|
||||
INSERT INTO "offers" ("id", "name", "description", "award_credit_in_cents", "invitee_credit_in_cents", "expires_at", "created_at", "status", "type", "award_credit_duration_days", "invitee_credit_duration_days") VALUES (1, 'Default referral offer', 'Is active when no other active referral offer', 300, 600, '2119-03-14 08:28:24.636949+00', '2019-07-14 08:28:24.636949+00', 1, 2, 365, 14);
|
||||
INSERT INTO "offers" ("id", "name", "description", "award_credit_in_cents", "invitee_credit_in_cents", "expires_at", "created_at", "status", "type", "award_credit_duration_days", "invitee_credit_duration_days") VALUES (2, 'Default free credit offer', 'Is active when no active free credit offer', 0, 300, '2119-03-14 08:28:24.636949+00', '2019-07-14 08:28:24.636949+00', 1, 1, NULL, 14);
|
||||
|
||||
INSERT INTO "api_keys" ("id", "project_id", "head", "name", "secret", "partner_id", "created_at") VALUES (E'\\334/\\302;\\225\\355O\\323\\276f\\247\\354/6\\241\\033'::bytea, E'\\022\\217/\\014\\376!K\\023\\276\\031\\311}m\\236\\205\\300'::bytea, E'\\111\\142\\147\\304\\132\\375\\070\\163\\270\\160\\251\\370\\126\\063\\351\\037\\257\\071\\143\\375\\351\\320\\253\\232\\220\\260\\075\\173\\306\\307\\115\\136'::bytea, 'key 2', E'\\254\\011\\315\\333\\273\\365\\001\\071\\024\\154\\253\\332\\301\\216\\361\\074\\221\\367\\251\\231\\274\\333\\300\\367\\001\\272\\327\\111\\315\\123\\042\\016'::bytea, NULL, '2019-02-14 08:28:24.267934+00');
|
||||
|
||||
INSERT INTO "project_invoice_stamps" ("project_id", "invoice_id", "start_date", "end_date", "created_at") VALUES (E'\\022\\217/\\014\\376!K\\023\\276\\031\\311}m\\236\\205\\300'::bytea, E'\\363\\311\\033w\\222\\303,'::bytea, '2019-06-01 08:28:24.267934+00', '2019-06-29 08:28:24.267934+00', '2019-06-01 08:28:24.267934+00');
|
||||
|
||||
INSERT INTO "value_attributions" ("project_id", "bucket_name", "partner_id", "last_updated") VALUES (E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, E''::bytea, E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\014'::bytea,'2019-02-14 08:07:31.028103+00');
|
||||
|
||||
INSERT INTO "user_credits" ("id", "user_id", "offer_id", "referred_by", "credits_earned_in_cents", "credits_used_in_cents", "type", "expires_at", "created_at") VALUES (1, E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, 1, E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, 200, 0, 'invalid', '2019-10-01 08:28:24.267934+00', '2019-06-01 08:28:24.267934+00');
|
||||
|
||||
INSERT INTO "bucket_metainfos" ("id", "project_id", "name", "partner_id", "created_at", "path_cipher", "default_segment_size", "default_encryption_cipher_suite", "default_encryption_block_size", "default_redundancy_algorithm", "default_redundancy_share_size", "default_redundancy_required_shares", "default_redundancy_repair_shares", "default_redundancy_optimal_shares", "default_redundancy_total_shares") VALUES (E'\\334/\\302;\\225\\355O\\323\\276f\\247\\354/6\\241\\033'::bytea, E'\\022\\217/\\014\\376!K\\023\\276\\031\\311}m\\236\\205\\300'::bytea, E'testbucketuniquename'::bytea, NULL, '2019-06-14 08:28:24.677953+00', 1, 65536, 1, 8192, 1, 4096, 4, 6, 8, 10);
|
||||
|
||||
INSERT INTO "pending_audits" ("node_id", "piece_id", "stripe_index", "share_size", "expected_share_hash", "reverify_count", "path") VALUES (E'\\153\\313\\233\\074\\327\\177\\136\\070\\346\\001'::bytea, E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, 5, 1024, E'\\070\\127\\144\\013\\332\\344\\102\\376\\306\\056\\303\\130\\106\\132\\321\\276\\321\\274\\170\\264\\054\\333\\221\\116\\154\\221\\335\\070\\220\\146\\344\\216'::bytea, 1, 'not null');
|
||||
|
||||
INSERT INTO "peer_identities" VALUES (E'\\334/\\302;\\225\\355O\\323\\276f\\247\\354/6\\241\\033'::bytea, E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\014'::bytea, E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, '2019-02-14 08:07:31.335028+00');
|
||||
|
||||
INSERT INTO "graceful_exit_progress" ("node_id", "bytes_transferred", "pieces_transferred", "pieces_failed", "updated_at") VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\016', 1000000000000000, 0, 0, '2019-09-12 10:07:31.028103');
|
||||
INSERT INTO "graceful_exit_transfer_queue" ("node_id", "path", "piece_num", "durability_ratio", "queued_at", "requested_at", "last_failed_at", "last_failed_code", "failed_count", "finished_at", "order_limit_send_count") VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\016', E'f8419768-5baa-4901-b3ba-62808013ec45/s0/test3/\\240\\243\\223n\\334~b}\\2624)\\250m\\201\\202\\235\\276\\361\\3304\\323\\352\\311\\361\\353;\\326\\311', 8, 1.0, '2019-09-12 10:07:31.028103', '2019-09-12 10:07:32.028103', null, null, 0, '2019-09-12 10:07:33.028103', 0);
|
||||
INSERT INTO "graceful_exit_transfer_queue" ("node_id", "path", "piece_num", "durability_ratio", "queued_at", "requested_at", "last_failed_at", "last_failed_code", "failed_count", "finished_at", "order_limit_send_count") VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\016', E'f8419768-5baa-4901-b3ba-62808013ec45/s0/test3/\\240\\243\\223n\\334~b}\\2624)\\250m\\201\\202\\235\\276\\361\\3304\\323\\352\\311\\361\\353;\\326\\312', 8, 1.0, '2019-09-12 10:07:31.028103', '2019-09-12 10:07:32.028103', null, null, 0, '2019-09-12 10:07:33.028103', 0);
|
||||
|
||||
INSERT INTO "stripe_customers" ("user_id", "customer_id", "created_at") VALUES (E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, 'stripe_id', '2019-06-01 08:28:24.267934+00');
|
||||
|
||||
INSERT INTO "graceful_exit_transfer_queue" ("node_id", "path", "piece_num", "durability_ratio", "queued_at", "requested_at", "last_failed_at", "last_failed_code", "failed_count", "finished_at", "order_limit_send_count") VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\016', E'f8419768-5baa-4901-b3ba-62808013ec45/s0/test3/\\240\\243\\223n\\334~b}\\2624)\\250m\\201\\202\\235\\276\\361\\3304\\323\\352\\311\\361\\353;\\326\\311', 9, 1.0, '2019-09-12 10:07:31.028103', '2019-09-12 10:07:32.028103', null, null, 0, '2019-09-12 10:07:33.028103', 0);
|
||||
INSERT INTO "graceful_exit_transfer_queue" ("node_id", "path", "piece_num", "durability_ratio", "queued_at", "requested_at", "last_failed_at", "last_failed_code", "failed_count", "finished_at", "order_limit_send_count") VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\016', E'f8419768-5baa-4901-b3ba-62808013ec45/s0/test3/\\240\\243\\223n\\334~b}\\2624)\\250m\\201\\202\\235\\276\\361\\3304\\323\\352\\311\\361\\353;\\326\\312', 9, 1.0, '2019-09-12 10:07:31.028103', '2019-09-12 10:07:32.028103', null, null, 0, '2019-09-12 10:07:33.028103', 0);
|
||||
|
||||
INSERT INTO "stripecoinpayments_invoice_project_records"("id", "project_id", "storage", "egress", "objects", "period_start", "period_end", "state", "created_at") VALUES (E'\\022\\217/\\014\\376!K\\023\\276\\031\\311}m\\236\\205\\300'::bytea, E'\\021\\217/\\014\\376!K\\023\\276\\031\\311}m\\236\\205\\300'::bytea, 0, 0, 0, '2019-06-01 08:28:24.267934+00', '2019-06-01 08:28:24.267934+00', 0, '2019-06-01 08:28:24.267934+00');
|
||||
|
||||
INSERT INTO "graceful_exit_transfer_queue" ("node_id", "path", "piece_num", "root_piece_id", "durability_ratio", "queued_at", "requested_at", "last_failed_at", "last_failed_code", "failed_count", "finished_at", "order_limit_send_count") VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\016', E'f8419768-5baa-4901-b3ba-62808013ec45/s0/test3/\\240\\243\\223n\\334~b}\\2624)\\250m\\201\\202\\235\\276\\361\\3304\\323\\352\\311\\361\\353;\\326\\311', 10, E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, 1.0, '2019-09-12 10:07:31.028103', '2019-09-12 10:07:32.028103', null, null, 0, '2019-09-12 10:07:33.028103', 0);
|
||||
|
||||
INSERT INTO "stripecoinpayments_tx_conversion_rates" ("tx_id", "rate", "created_at") VALUES ('tx_id', E'\\363\\311\\033w\\222\\303Ci,'::bytea, '2019-06-01 08:28:24.267934+00');
|
||||
|
||||
INSERT INTO "coinpayments_transactions" ("id", "user_id", "address", "amount", "received", "status", "key", "timeout", "created_at") VALUES ('tx_id', E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, 'address', E'\\363\\311\\033w'::bytea, E'\\363\\311\\033w'::bytea, 1, 'key', 60, '2019-06-01 08:28:24.267934+00');
|
||||
INSERT INTO "stripecoinpayments_apply_balance_intents" ("tx_id", "state", "created_at") VALUES ('tx_id', 0, '2019-06-01 08:28:24.267934+00');
|
||||
|
||||
INSERT INTO "nodes_offline_times" ("node_id", "tracked_at", "seconds") VALUES (E'\\153\\313\\233\\074\\327\\177\\136\\070\\346\\001'::bytea, '2019-06-01 09:28:24.267934+00', 3600);
|
||||
INSERT INTO "nodes_offline_times" ("node_id", "tracked_at", "seconds") VALUES (E'\\153\\313\\233\\074\\327\\177\\136\\070\\346\\001'::bytea, '2017-06-01 09:28:24.267934+00', 100);
|
||||
INSERT INTO "nodes_offline_times" ("node_id", "tracked_at", "seconds") VALUES (E'\\006\\223\\250R\\221\\005\\365\\377v>0\\266\\365\\216\\255?\\347\\244\\371?2\\264\\262\\230\\007<\\001\\262\\263\\237\\247n'::bytea, '2019-06-01 09:28:24.267934+00', 3600);
|
||||
|
||||
INSERT INTO "storagenode_bandwidth_rollups" ("storagenode_id", "interval_start", "interval_seconds", "action", "settled") VALUES (E'\\006\\223\\250R\\221\\005\\365\\377v>0\\266\\365\\216\\255?\\347\\244\\371?2\\264\\262\\230\\007<\\001\\262\\263\\237\\247n', '2020-01-11 08:00:00.000000+00', 3600, 1, 2024);
|
||||
|
||||
INSERT INTO "coupons" ("id", "project_id", "user_id", "amount", "description", "type", "status", "duration", "created_at") VALUES (E'\\362\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\014'::bytea, E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\014'::bytea, E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, 50, 'description', 0, 0, 2, '2019-06-01 08:28:24.267934+00');
|
||||
INSERT INTO "coupon_usages" ("coupon_id", "amount", "status", "period") VALUES (E'\\362\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\014'::bytea, 22, 0, '2019-06-01 09:28:24.267934+00');
|
||||
|
||||
-- NEW DATA --
|
6
scripts/testdata/satellite-config.yaml.lock
vendored
6
scripts/testdata/satellite-config.yaml.lock
vendored
@ -304,6 +304,9 @@ identity.key-path: /root/.local/share/storj/identity/satellite/identity.key
|
||||
# how often to flush the rollups write cache to the database
|
||||
# orders.flush-interval: 1m0s
|
||||
|
||||
# how many records to read in a single transaction when calculating billable bandwidth
|
||||
# orders.reported-rollups-read-batch-size: 1000
|
||||
|
||||
# how many orders to batch per transaction
|
||||
# orders.settlement-batch-size: 250
|
||||
|
||||
@ -402,6 +405,9 @@ identity.key-path: /root/.local/share/storj/identity/satellite/identity.key
|
||||
# time limit for uploading repaired pieces to new storage nodes
|
||||
# repairer.timeout: 5m0s
|
||||
|
||||
# how often to flush the reported serial rollups to the database
|
||||
# reported-rollup.interval: 24h0m0s
|
||||
|
||||
# option for deleting tallies after they are rolled up
|
||||
# rollup.delete-tallies: true
|
||||
|
||||
|
Loading…
Reference in New Issue
Block a user