satellite/satellitedb: use queue for orders to get back fast billing

This change adds two new tables to process orders as fast as we used
to but in an asynchronous manner and with hopefully less storage
usage. This should help scale on cockroach, but limits us to one
worker. It lays the groundwork for the order processing pipeline to
be queue rather than database driven.

For more details, see the added fast billing changes blueprint.

It also fixes the orders db so that all the timestamps that are
passed to columns that do not contain a time zone are converted to
UTC at the last possible opportunity, making it less likely to use
the APIs incorrectly. We really should migrate to include timezones
on all of our timestamp columns.

Change-Id: Ibfda8e7a3d5972b7798fb61b31ff56419c64ea35
This commit is contained in:
Jeff Wendling 2020-02-13 17:03:41 -07:00 committed by jens
parent dca6fcbe28
commit f671eb2beb
13 changed files with 1698 additions and 339 deletions

View File

@ -0,0 +1,81 @@
# Fast Billing Changes
## Problem Statement
Cockroach has some interesting performance characteristics.
1. Doing a blind write is fast
2. Batching writes into single transactions is fast
3. Doing reads and writes in the same transaction is slow
4. Doing reads at a given timestamp is non-blocking and fast
We hit some of these performance issues with the `used_serials` table: in order to check if we should include some serial into some bandwidth rollup, we had to issue a read in the same transaction as the eventual write. Also, since we had multiple APIs responsible for this, there was contention.
In order to address those performance issues, we started using the `reported_serials` table that was only ever blindly written to. In order to do blind writes and avoid double spending issues (a node submitting the same order multiple times), a reported order is not added into a bandwidth rollup until after it expires. Unfortunately, this causes a lot of confusion for a number of reasons.
1. End of month payouts do not include the last 7-8 days of bandwidth
2. Display of used bandwidth lags behind actual usage 7-8 days
Some constraints on the solution space happen due to our payment processing system having a number of steps to it. After orders are received, we eventually insert them into the bandwidth rollups tables, which are eventually read to be inserted into an accounting table. That accounting table is used for storagenode payments at the end of the month. The accounting table only includes entries that it has never included before based on creation timestamp, so you cannot insert into the past or update existing rows of the bandwidth rollups tables without also updating how the accounting table works.
## Proposed Solution
We create a new `pending_serial_queue` table to function as a queue of serials to process sent by storage nodes. Any order processing just blindly upserts into this table. The primary key will be on `( storage_node_id, bucket_id, serial_number )` which means that we don't necessarily consume them in the order they have been inserted, but we do get good prefix compression with cockroach.
We bring back a `consumed_serials` table which functions much like the older `used_serials` table to record which serials have been added into a rollup. It has a primary key of `( storage_node_id, serial_number )` to allow for quick point lookups and has an index on `expires_at` in order to allow for efficient deletes.
The core process consumes `pending_serial_queue` to do inserts into `consumed_serial`. Since there is only ever one core process doing this, we are given much flexibility in how to do the transactions (for example, we can do any size of transaction, or partition them into read-only and write-only transactions.) It first queries `pending_serial_queue` in pages (each page in its own transaction) for any values. While batching up the pages into a values to write, it has a read-only transaction open querying `consumed_serials` to ensure it does not double account, building a batch of writes into `storagenode_bandwidth_rollups`, `bucket_bandwidth_rollups`, and `reported_consumed_serials`.
At the end of a page, if the batches are large enough, a new transaction issues the blind upserts. It then issues a delete to the `pending_serial_queue` table for all entries that were part of the batch. Note that this does not need to be in the same transaction: since it was inserted into `consumed_serial`, we know that it will not be accounted for again.
Eventually, some process deletes from `consumed_serials` when they are definitely expired.
The essence of this solution is to go back to how we were doing it with `used_serials` except asynchronously and with a single worker so that we can do it with high performance and nearly the same characteristics with respect to when values are included for billing. It allows full control over transaction size and the read/write characteristics of each transaction.
## Benefits
- APIs can blindly upsert into `pending_serial_queue`, allowing high scalability
- Full control over transactions means we can tune sizes without code changes
- Using smaller transactions allows getting better monitoring data on rate of progress
- The core consumer will quickly include data that has been reported again
- Does not require changes to any of the other systems (accounting, dashboards, etc.) to have their behavior restored
- Does not modify existing tables, just adds new ones.
## Downsides
- It is racy if we need to have more than one consumer to keep up, but this can be fixed at some complexity cost with sharding the `pending_serial_queue` table if necessary.
- More temporary space used with `consumed_serials`, but hopefully this is offset by the prefix compression.
## Q/A
- Why not use kafka or some other queue for `pending_serial_queue`?
That'd be fine, and whatever the consumer of the queue is should be agnostic to how the queue is implemented. The fastest implementation will be one that uses the current database we have, and if we like the design of the system, changing where the serials get inserted to is an easy detail to change.
If the database can handle the load, I'd prefer not to have to spin up and maintain a new service and learn the operation challenges involved as we head into production. If the database cannot handle the load, the current system, while flawed, does not lose payment.
## Appendix
### dbx models
model pending_serial_queue (
table pending_serial_queue
key storage_node_id bucket_id serial_number
field storage_node_id blob
field bucket_id blob
field serial_number blob
field action uint
field settled uint64
field expires_at timestamp
)
model consumed_serial (
key storage_node_id serial_number
index ( fields expires_at )
field storage_node_id blob
field serial_number blob
field expires_at timestamp
)

View File

@ -7,11 +7,14 @@ import (
"context"
"time"
"github.com/skyrings/skyring-common/tools/uuid"
"github.com/spacemonkeygo/monkit/v3"
"github.com/zeebo/errs"
"go.uber.org/zap"
"storj.io/common/pb"
"storj.io/common/sync2"
"storj.io/storj/pkg/storj"
"storj.io/storj/satellite/orders"
)
@ -24,7 +27,7 @@ var (
// Config is a configuration struct for the Chore.
type Config struct {
Interval time.Duration `help:"how often to flush the reported serial rollups to the database" devDefault:"5m" releaseDefault:"24h"`
Interval time.Duration `help:"how often to flush the reported serial rollups to the database" default:"5m"`
}
// Chore for flushing reported serials to the database as rollups.
@ -67,21 +70,198 @@ func (chore *Chore) Close() error {
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
for {
done, err := chore.runOnceHelper(ctx, now)
if err != nil {
return errs.Wrap(err)
}
if done {
return nil
}
}
}
// TODO: jeeze make configurable
const (
defaultQueueBatchSize = 1000
defaultRollupBatchSize = 1000
defaultConsumedSerialsBatchSize = 10000
)
func (chore *Chore) readWork(ctx context.Context, now time.Time, tx orders.Transaction, queue orders.Queue) (
bucketRollups []orders.BucketBandwidthRollup,
storagenodeRollups []orders.StoragenodeBandwidthRollup,
consumedSerials []orders.ConsumedSerial,
done bool, err error,
) {
defer mon.Task()(&ctx)(&err)
// Variables and types to keep track of bucket bandwidth rollups
type bucketKey struct {
projectID uuid.UUID
bucketName string
action pb.PieceAction
}
byBucket := make(map[bucketKey]uint64)
// Variables and types to keep track of storagenode bandwidth rollups
type storagenodeKey struct {
nodeID storj.NodeID
action pb.PieceAction
}
byStoragenode := make(map[storagenodeKey]uint64)
// Variables to keep track of which serial numbers were consumed
type consumedSerialKey struct {
nodeID storj.NodeID
serialNumber storj.SerialNumber
}
seenConsumedSerials := make(map[consumedSerialKey]struct{})
// Loop until our batch is big enough, but not too big in any dimension.
for len(byBucket) < defaultRollupBatchSize &&
len(byStoragenode) < defaultRollupBatchSize &&
len(seenConsumedSerials) < defaultConsumedSerialsBatchSize {
// Get a batch of pending serials from the queue.
pendingSerials, err := queue.GetPendingSerialsBatch(ctx, defaultQueueBatchSize)
if err != nil {
return nil, nil, nil, false, errs.Wrap(err)
}
for _, row := range pendingSerials {
row := row
// If we have seen this serial inside of this function already, don't
// count it again and record it now.
key := consumedSerialKey{
nodeID: row.NodeID,
serialNumber: row.SerialNumber,
}
if _, exists := seenConsumedSerials[key]; exists {
continue
}
seenConsumedSerials[key] = struct{}{}
// If the serial already exists in the database already, don't count
// it again.
exists, err := tx.HasConsumedSerial(ctx, row.NodeID, row.SerialNumber)
if err != nil {
return nil, nil, nil, false, errs.Wrap(err)
}
if exists {
continue
}
// If the order is expired, don't consume it. It may have already made its
// way into consumed_serials and some process deleted it, allowing a double
// spend. This does mean we have to be careful to keep up with the queue or
// risk only getting to serials after they are expired.
if row.ExpiresAt.Before(now) {
continue
}
// Parse the node id, project id, and bucket name from the reported serial.
projectID, bucketName, err := orders.SplitBucketID(row.BucketID)
if err != nil {
chore.log.Error("bad row inserted into reported serials",
zap.Binary("bucket_id", row.BucketID),
zap.String("node_id", row.NodeID.String()),
zap.String("serial_number", row.SerialNumber.String()))
continue
}
action := pb.PieceAction(row.Action)
settled := row.Settled
// Update our batch state to include it.
byBucket[bucketKey{
projectID: *projectID,
bucketName: string(bucketName),
action: action,
}] += settled
byStoragenode[storagenodeKey{
nodeID: row.NodeID,
action: action,
}] += settled
consumedSerials = append(consumedSerials, orders.ConsumedSerial{
NodeID: row.NodeID,
SerialNumber: row.SerialNumber,
ExpiresAt: row.ExpiresAt,
})
}
// If we didn't get a full batch, the queue must have run out. We should signal
// this fact to our caller so that they can stop looping.
if len(pendingSerials) != defaultQueueBatchSize {
done = true
break
}
}
return Error.Wrap(chore.db.WithTransaction(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
}))
// Convert bucket rollups into a slice.
for key, settled := range byBucket {
bucketRollups = append(bucketRollups, orders.BucketBandwidthRollup{
ProjectID: key.projectID,
BucketName: key.bucketName,
Action: key.action,
Settled: int64(settled),
})
}
// Convert storagenode rollups into a slice.
for key, settled := range byStoragenode {
storagenodeRollups = append(storagenodeRollups, orders.StoragenodeBandwidthRollup{
NodeID: key.nodeID,
Action: key.action,
Settled: int64(settled),
})
}
chore.log.Debug("Read work",
zap.Int("bucket_rollups", len(bucketRollups)),
zap.Int("storagenode_rollups", len(storagenodeRollups)),
zap.Int("consumed_serials", len(consumedSerials)),
)
return bucketRollups, storagenodeRollups, consumedSerials, done, nil
}
func (chore *Chore) runOnceHelper(ctx context.Context, now time.Time) (done bool, err error) {
defer mon.Task()(&ctx)(&err)
err = chore.db.WithQueue(ctx, func(ctx context.Context, queue orders.Queue) error {
var (
bucketRollups []orders.BucketBandwidthRollup
storagenodeRollups []orders.StoragenodeBandwidthRollup
consumedSerials []orders.ConsumedSerial
)
// Read the work we should insert in its own transaction.
err := chore.db.WithTransaction(ctx, func(ctx context.Context, tx orders.Transaction) error {
bucketRollups, storagenodeRollups, consumedSerials, done, err = chore.readWork(ctx, now, tx, queue)
return errs.Wrap(err)
})
if err != nil {
return errs.Wrap(err)
}
// Now that we have work, write it all in its own transaction.
return errs.Wrap(chore.db.WithTransaction(ctx, func(ctx context.Context, tx orders.Transaction) error {
now := time.Now()
if err := tx.UpdateBucketBandwidthBatch(ctx, now, bucketRollups); err != nil {
return errs.Wrap(err)
}
if err := tx.UpdateStoragenodeBandwidthBatch(ctx, now, storagenodeRollups); err != nil {
return errs.Wrap(err)
}
if err := tx.CreateConsumedSerialsBatch(ctx, consumedSerials); err != nil {
return errs.Wrap(err)
}
return nil
}))
})
return done, errs.Wrap(err)
}

View File

@ -57,13 +57,22 @@ func (chore *Chore) deleteExpiredSerials(ctx context.Context) (err error) {
defer mon.Task()(&ctx)(&err)
chore.log.Debug("deleting expired serial numbers")
deleted, err := chore.orders.DeleteExpiredSerials(ctx, time.Now().UTC())
now := time.Now()
deleted, err := chore.orders.DeleteExpiredSerials(ctx, now)
if err != nil {
chore.log.Error("deleting expired serial numbers", zap.Error(err))
return nil
} else {
chore.log.Debug("expired serials deleted", zap.Int("items deleted", deleted))
}
deleted, err = chore.orders.DeleteExpiredConsumedSerials(ctx, now)
if err != nil {
chore.log.Error("deleting expired serial numbers", zap.Error(err))
} else {
chore.log.Debug("expired serials deleted", zap.Int("items deleted", deleted))
}
chore.log.Debug("expired serials deleted", zap.Int("items deleted", deleted))
return nil
}

View File

@ -34,8 +34,10 @@ type DB interface {
UseSerialNumber(ctx context.Context, serialNumber storj.SerialNumber, storageNodeID storj.NodeID) ([]byte, error)
// UnuseSerialNumber removes pair serial number -> storage node id from database
UnuseSerialNumber(ctx context.Context, serialNumber storj.SerialNumber, storageNodeID storj.NodeID) error
// DeleteExpiredSerials deletes all expired serials in serial_number and used_serials table.
// DeleteExpiredSerials deletes all expired serials in serial_number, used_serials, and consumed_serials table.
DeleteExpiredSerials(ctx context.Context, now time.Time) (_ int, err error)
// DeleteExpiredConsumedSerials deletes all expired serials in the consumed_serials table.
DeleteExpiredConsumedSerials(ctx context.Context, now time.Time) (_ int, err error)
// UpdateBucketBandwidthAllocation updates 'allocated' bandwidth for given bucket
UpdateBucketBandwidthAllocation(ctx context.Context, projectID uuid.UUID, bucketName []byte, action pb.PieceAction, amount int64, intervalStart time.Time) error
@ -53,23 +55,51 @@ type DB interface {
GetStorageNodeBandwidth(ctx context.Context, nodeID storj.NodeID, from, to time.Time) (int64, error)
// ProcessOrders takes a list of order requests and processes them in a batch
ProcessOrders(ctx context.Context, requests []*ProcessOrderRequest, observedAt time.Time) (responses []*ProcessOrderResponse, err 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)
ProcessOrders(ctx context.Context, requests []*ProcessOrderRequest) (responses []*ProcessOrderResponse, err error)
// WithTransaction runs the callback and provides it with a Transaction.
WithTransaction(ctx context.Context, cb func(ctx context.Context, tx Transaction) error) error
// WithQueue TODO: DOCS
WithQueue(ctx context.Context, cb func(ctx context.Context, queue Queue) error) error
}
// 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)
// CreateConsumedSerialsBatch TODO: DOCS
CreateConsumedSerialsBatch(ctx context.Context, consumedSerials []ConsumedSerial) (err error)
// HasConsumedSerial TODO: DOCS
HasConsumedSerial(ctx context.Context, nodeID storj.NodeID, serialNumber storj.SerialNumber) (bool, error)
}
// Queue TODO: DOCS
type Queue interface {
// GetPendingSerialsBatch TODO: DOCS
GetPendingSerialsBatch(ctx context.Context, size int) ([]PendingSerial, error)
}
// ConsumedSerial TODO: DOCS
type ConsumedSerial struct {
NodeID storj.NodeID
SerialNumber storj.SerialNumber
ExpiresAt time.Time
}
// PendingSerial is a serial number reported by a storagenode waiting to be
// settled
type PendingSerial struct {
NodeID storj.NodeID
BucketID []byte
Action uint
SerialNumber storj.SerialNumber
ExpiresAt time.Time
Settled uint64
}
var (
@ -316,7 +346,7 @@ func (endpoint *Endpoint) doSettlement(stream settlementStream) (err error) {
func (endpoint *Endpoint) processOrders(ctx context.Context, stream settlementStream, requests []*ProcessOrderRequest) (err error) {
defer mon.Task()(&ctx)(&err)
responses, err := endpoint.DB.ProcessOrders(ctx, requests, time.Now())
responses, err := endpoint.DB.ProcessOrders(ctx, requests)
if err != nil {
return err
}

View File

@ -12,6 +12,7 @@ import (
"github.com/skyrings/skyring-common/tools/uuid"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"go.uber.org/zap/zaptest"
"storj.io/common/memory"
"storj.io/common/pb"
@ -20,6 +21,7 @@ import (
"storj.io/common/testrand"
"storj.io/storj/private/testplanet"
"storj.io/storj/satellite"
"storj.io/storj/satellite/accounting/reportedrollup"
"storj.io/storj/satellite/orders"
"storj.io/storj/satellite/satellitedb/satellitedbtest"
)
@ -125,8 +127,11 @@ func TestUploadDownloadBandwidth(t *testing.T) {
Satellite: testplanet.ReconfigureRS(2, 3, 4, 4),
},
}, func(t *testing.T, ctx *testcontext.Context, planet *testplanet.Planet) {
wayInTheFuture := time.Now().UTC().Add(1000 * time.Hour)
hourBeforeTheFuture := wayInTheFuture.Add(-time.Hour)
now := time.Now()
beforeRollup := now.Add(-time.Hour - time.Second)
afterRollup := now.Add(time.Hour + time.Second)
bucketName := "testbucket"
planet.Satellites[0].Audit.Worker.Loop.Pause()
for _, storageNode := range planet.StorageNodes {
@ -135,10 +140,10 @@ func TestUploadDownloadBandwidth(t *testing.T) {
expectedData := testrand.Bytes(50 * memory.KiB)
err := planet.Uplinks[0].Upload(ctx, planet.Satellites[0], "testbucket", "test/path", expectedData)
err := planet.Uplinks[0].Upload(ctx, planet.Satellites[0], bucketName, "test/path", expectedData)
require.NoError(t, err)
data, err := planet.Uplinks[0].Download(ctx, planet.Satellites[0], "testbucket", "test/path")
data, err := planet.Uplinks[0].Download(ctx, planet.Satellites[0], bucketName, "test/path")
require.NoError(t, err)
require.Equal(t, expectedData, data)
@ -162,21 +167,20 @@ 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))
require.NoError(t, reportedRollupChore.RunOnce(ctx, now))
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"), hourBeforeTheFuture, wayInTheFuture)
bucketBandwidth, err := ordersDB.GetBucketBandwidth(ctx, projects[0].ID, []byte(bucketName), beforeRollup, afterRollup)
require.NoError(t, err)
require.Equal(t, expectedBucketBandwidth, bucketBandwidth)
for _, storageNode := range planet.StorageNodes {
nodeBandwidth, err := ordersDB.GetStorageNodeBandwidth(ctx, storageNode.ID(), hourBeforeTheFuture, wayInTheFuture)
nodeBandwidth, err := ordersDB.GetStorageNodeBandwidth(ctx, storageNode.ID(), beforeRollup, afterRollup)
require.NoError(t, err)
require.Equal(t, expectedStorageBandwidth[storageNode.ID()], nodeBandwidth)
}
@ -190,8 +194,10 @@ func TestMultiProjectUploadDownloadBandwidth(t *testing.T) {
Satellite: testplanet.ReconfigureRS(2, 3, 4, 4),
},
}, func(t *testing.T, ctx *testcontext.Context, planet *testplanet.Planet) {
wayInTheFuture := time.Now().UTC().Add(1000 * time.Hour)
hourBeforeTheFuture := wayInTheFuture.Add(-time.Hour)
now := time.Now()
beforeRollup := now.Add(-time.Hour - time.Second)
afterRollup := now.Add(time.Hour + time.Second)
planet.Satellites[0].Audit.Worker.Loop.Pause()
for _, storageNode := range planet.StorageNodes {
@ -223,19 +229,26 @@ func TestMultiProjectUploadDownloadBandwidth(t *testing.T) {
// 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))
require.NoError(t, reportedRollupChore.RunOnce(ctx, now))
// Query and ensure that there's no data recorded for the bucket from the other project
ordersDB := planet.Satellites[0].DB.Orders()
uplink0Project := planet.Uplinks[0].ProjectID[planet.Satellites[0].ID()]
uplink1Project := planet.Uplinks[1].ProjectID[planet.Satellites[0].ID()]
wrongBucketBandwidth, err := ordersDB.GetBucketBandwidth(ctx, uplink0Project, []byte("testbucket1"), hourBeforeTheFuture, wayInTheFuture)
wrongBucketBandwidth, err := ordersDB.GetBucketBandwidth(ctx, uplink0Project, []byte("testbucket1"), beforeRollup, afterRollup)
require.NoError(t, err)
require.Equal(t, int64(0), wrongBucketBandwidth)
wrongBucketBandwidth, err = ordersDB.GetBucketBandwidth(ctx, uplink1Project, []byte("testbucket0"), hourBeforeTheFuture, wayInTheFuture)
rightBucketBandwidth, err := ordersDB.GetBucketBandwidth(ctx, uplink0Project, []byte("testbucket0"), beforeRollup, afterRollup)
require.NoError(t, err)
require.Greater(t, rightBucketBandwidth, int64(0))
wrongBucketBandwidth, err = ordersDB.GetBucketBandwidth(ctx, uplink1Project, []byte("testbucket0"), beforeRollup, afterRollup)
require.NoError(t, err)
require.Equal(t, int64(0), wrongBucketBandwidth)
rightBucketBandwidth, err = ordersDB.GetBucketBandwidth(ctx, uplink1Project, []byte("testbucket1"), beforeRollup, afterRollup)
require.NoError(t, err)
require.Greater(t, rightBucketBandwidth, int64(0))
})
}
@ -302,7 +315,7 @@ func BenchmarkOrders(b *testing.B) {
for i := 0; i < b.N; i++ {
requests := buildBenchmarkData(ctx, b, db, snID, bucketID, c)
_, err := db.Orders().ProcessOrders(ctx, requests, time.Now())
_, err := db.Orders().ProcessOrders(ctx, requests)
assert.NoError(b, err)
}
})
@ -341,11 +354,26 @@ func buildBenchmarkData(ctx context.Context, b *testing.B, db satellite.DB, stor
func TestProcessOrders(t *testing.T) {
satellitedbtest.Run(t, func(ctx *testcontext.Context, t *testing.T, db satellite.DB) {
ordersDB := db.Orders()
chore := reportedrollup.NewChore(zaptest.NewLogger(t), ordersDB, reportedrollup.Config{})
invalidSerial := storj.SerialNumber{1}
serialNum := storj.SerialNumber{2}
serialNum2 := storj.SerialNumber{3}
projectID, _ := uuid.New()
now := time.Now()
beforeRollup := now.Add(-time.Hour - time.Second)
afterRollup := now.Add(time.Hour + time.Second)
// assertion helpers
checkBucketBandwidth := func(bucket string, amount int64) {
settled, err := ordersDB.GetBucketBandwidth(ctx, *projectID, []byte(bucket), beforeRollup, afterRollup)
require.NoError(t, err)
require.Equal(t, amount, settled)
}
checkStoragenodeBandwidth := func(node storj.NodeID, amount int64) {
settled, err := ordersDB.GetStorageNodeBandwidth(ctx, node, beforeRollup, afterRollup)
require.NoError(t, err)
require.Equal(t, amount, settled)
}
// setup: create serial number records
err := ordersDB.CreateSerialInfo(ctx, serialNum, []byte(projectID.String()+"/b"), now.AddDate(0, 0, 1))
@ -355,7 +383,8 @@ func TestProcessOrders(t *testing.T) {
var requests []*orders.ProcessOrderRequest
t.Run("process one order and confirm we get the correct response", func(t *testing.T) {
// process one order and confirm we get the correct response
{
requests = append(requests, &orders.ProcessOrderRequest{
Order: &pb.Order{
SerialNumber: serialNum,
@ -368,7 +397,7 @@ func TestProcessOrders(t *testing.T) {
OrderExpiration: now.AddDate(0, 0, 3),
},
})
actualResponses, err := ordersDB.ProcessOrders(ctx, requests, now.Add(time.Second))
actualResponses, err := ordersDB.ProcessOrders(ctx, requests)
require.NoError(t, err)
expectedResponses := []*orders.ProcessOrderResponse{
{
@ -377,9 +406,10 @@ func TestProcessOrders(t *testing.T) {
},
}
assert.Equal(t, expectedResponses, actualResponses)
})
}
t.Run("process two orders from different storagenodes and confirm there is an error", func(t *testing.T) {
// process two orders from different storagenodes and confirm there is an error
{
requests = append(requests, &orders.ProcessOrderRequest{
Order: &pb.Order{
SerialNumber: serialNum2,
@ -391,49 +421,30 @@ func TestProcessOrders(t *testing.T) {
Action: pb.PieceAction_PUT,
OrderExpiration: now.AddDate(0, 0, 1)},
})
_, err = ordersDB.ProcessOrders(ctx, requests, now.Add(time.Second))
_, err = ordersDB.ProcessOrders(ctx, requests)
require.Error(t, err, "different storage nodes")
})
}
t.Run("process two orders from same storagenodes and confirm we get two responses", func(t *testing.T) {
// process two orders from same storagenodes and confirm we get two responses
{
requests[0].OrderLimit.StorageNodeId = storj.NodeID{2}
actualResponses, err := ordersDB.ProcessOrders(ctx, requests, now.Add(time.Second))
actualResponses, err := ordersDB.ProcessOrders(ctx, requests)
require.NoError(t, err)
assert.Equal(t, 2, len(actualResponses))
})
}
t.Run("confirm the correct data from processing orders was written to reported_serials table", func(t *testing.T) {
bbr, snr, err := ordersDB.GetBillableBandwidth(ctx, now.AddDate(0, 0, 3))
require.NoError(t, err)
assert.Equal(t, 1, len(bbr))
expected := []orders.BucketBandwidthRollup{
{
ProjectID: *projectID,
BucketName: "c",
Action: pb.PieceAction_PUT,
Inline: 0,
Allocated: 0,
Settled: 200,
},
}
assert.Equal(t, expected, bbr)
assert.Equal(t, 1, len(snr))
expectedRollup := []orders.StoragenodeBandwidthRollup{
{
NodeID: storj.NodeID{2},
Action: pb.PieceAction_PUT,
Allocated: 0,
Settled: 200,
},
}
assert.Equal(t, expectedRollup, snr)
bbr, snr, err = ordersDB.GetBillableBandwidth(ctx, now.AddDate(0, 0, 5))
require.NoError(t, err)
assert.Equal(t, 2, len(bbr))
assert.Equal(t, 3, len(snr))
})
// confirm the correct data from processing orders was written and consumed
{
require.NoError(t, chore.RunOnce(ctx, now))
t.Run("confirm invalid order at index 0 does not result in a SQL error", func(t *testing.T) {
checkBucketBandwidth("b", 200)
checkBucketBandwidth("c", 200)
checkStoragenodeBandwidth(storj.NodeID{1}, 100)
checkStoragenodeBandwidth(storj.NodeID{2}, 300)
}
// confirm invalid order at index 0 does not result in a SQL error
{
requests := []*orders.ProcessOrderRequest{
{
Order: &pb.Order{
@ -460,12 +471,13 @@ func TestProcessOrders(t *testing.T) {
},
},
}
responses, err := ordersDB.ProcessOrders(ctx, requests, now.Add(time.Second))
responses, err := ordersDB.ProcessOrders(ctx, requests)
require.NoError(t, err)
assert.Equal(t, pb.SettlementResponse_REJECTED, responses[0].Status)
})
}
t.Run("in case of conflicting ProcessOrderRequests, later one wins", func(t *testing.T) {
// in case of conflicting ProcessOrderRequests, what has been recorded already wins
{
// unique nodeID so the other tests here don't interfere
nodeID := testrand.NodeID()
requests := []*orders.ProcessOrderRequest{
@ -494,7 +506,7 @@ func TestProcessOrders(t *testing.T) {
},
},
}
responses, err := ordersDB.ProcessOrders(ctx, requests, now.Add(time.Second))
responses, err := ordersDB.ProcessOrders(ctx, requests)
require.NoError(t, err)
require.Equal(t, pb.SettlementResponse_ACCEPTED, responses[0].Status)
require.Equal(t, pb.SettlementResponse_ACCEPTED, responses[1].Status)
@ -525,23 +537,19 @@ func TestProcessOrders(t *testing.T) {
},
},
}
responses, err = ordersDB.ProcessOrders(ctx, requests, now.Add(time.Second))
responses, err = ordersDB.ProcessOrders(ctx, requests)
require.NoError(t, err)
require.Equal(t, pb.SettlementResponse_ACCEPTED, responses[0].Status)
require.Equal(t, pb.SettlementResponse_ACCEPTED, responses[1].Status)
_, storagenodeRollups, err := ordersDB.GetBillableBandwidth(ctx, now.AddDate(0, 0, 10))
require.NoError(t, err)
found := false
for _, rollup := range storagenodeRollups {
if rollup.NodeID == nodeID {
require.Equal(t, pb.PieceAction_GET, rollup.Action)
require.Equal(t, int64(501), rollup.Settled)
found = true
}
}
require.True(t, found)
})
require.NoError(t, chore.RunOnce(ctx, now))
checkBucketBandwidth("b", 201)
checkBucketBandwidth("c", 700)
checkStoragenodeBandwidth(storj.NodeID{1}, 100)
checkStoragenodeBandwidth(storj.NodeID{2}, 300)
checkStoragenodeBandwidth(nodeID, 501)
}
})
}

View File

@ -459,14 +459,11 @@ 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.
//
// DEPRECATED! vvvvvvvvvvvvvvvvvvvvvvvvvvv
//
model reported_serial (
key expires_at storage_node_id bucket_id action serial_number
field expires_at timestamp // ceil'd to the next day
@ -479,9 +476,43 @@ model reported_serial (
field observed_at timestamp
)
create reported_serial ( noreturn, replace )
read paged ( select reported_serial, where reported_serial.expires_at <= ? )
delete reported_serial ( where reported_serial.expires_at <= ? )
//
// DEPRECATED! ^^^^^^^^^^^^^^^^^^^^^^^^^^
//
model pending_serial_queue (
table pending_serial_queue
key storage_node_id bucket_id serial_number
field storage_node_id blob
field bucket_id blob
field serial_number blob
field action uint
field settled uint64
field expires_at timestamp
)
create pending_serial_queue ( noreturn, replace )
read paged ( select pending_serial_queue )
model consumed_serial (
key storage_node_id serial_number
index ( fields expires_at )
field storage_node_id blob
field serial_number blob
field expires_at timestamp
)
create consumed_serial ( noreturn )
delete consumed_serial ( where consumed_serial.expires_at <= ? )
read has (
select consumed_serial
where consumed_serial.storage_node_id = ?
where consumed_serial.serial_number = ?
)
// --- bucket accounting tables --- //

View File

@ -52,6 +52,12 @@ CREATE TABLE coinpayments_transactions (
created_at timestamp with time zone NOT NULL,
PRIMARY KEY ( id )
);
CREATE TABLE consumed_serials (
storage_node_id bytea NOT NULL,
serial_number bytea NOT NULL,
expires_at timestamp with time zone NOT NULL,
PRIMARY KEY ( storage_node_id, serial_number )
);
CREATE TABLE coupons (
id bytea NOT NULL,
project_id bytea NOT NULL,
@ -200,6 +206,15 @@ CREATE TABLE pending_audits (
path bytea NOT NULL,
PRIMARY KEY ( node_id )
);
CREATE TABLE pending_serial_queue (
storage_node_id bytea NOT NULL,
bucket_id bytea NOT NULL,
serial_number bytea NOT NULL,
action integer NOT NULL,
settled bigint NOT NULL,
expires_at timestamp with time zone NOT NULL,
PRIMARY KEY ( storage_node_id, bucket_id, serial_number )
);
CREATE TABLE projects (
id bytea NOT NULL,
name text NOT NULL,
@ -374,6 +389,7 @@ CREATE TABLE user_credits (
PRIMARY KEY ( id ),
UNIQUE ( id, offer_id )
);
CREATE INDEX consumed_serials_expires_at_index ON consumed_serials ( expires_at );
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 );

View File

@ -324,6 +324,12 @@ CREATE TABLE coinpayments_transactions (
created_at timestamp with time zone NOT NULL,
PRIMARY KEY ( id )
);
CREATE TABLE consumed_serials (
storage_node_id bytea NOT NULL,
serial_number bytea NOT NULL,
expires_at timestamp with time zone NOT NULL,
PRIMARY KEY ( storage_node_id, serial_number )
);
CREATE TABLE coupons (
id bytea NOT NULL,
project_id bytea NOT NULL,
@ -472,6 +478,15 @@ CREATE TABLE pending_audits (
path bytea NOT NULL,
PRIMARY KEY ( node_id )
);
CREATE TABLE pending_serial_queue (
storage_node_id bytea NOT NULL,
bucket_id bytea NOT NULL,
serial_number bytea NOT NULL,
action integer NOT NULL,
settled bigint NOT NULL,
expires_at timestamp with time zone NOT NULL,
PRIMARY KEY ( storage_node_id, bucket_id, serial_number )
);
CREATE TABLE projects (
id bytea NOT NULL,
name text NOT NULL,
@ -646,6 +661,7 @@ CREATE TABLE user_credits (
PRIMARY KEY ( id ),
UNIQUE ( id, offer_id )
);
CREATE INDEX consumed_serials_expires_at_index ON consumed_serials ( expires_at );
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 );
@ -767,6 +783,12 @@ CREATE TABLE coinpayments_transactions (
created_at timestamp with time zone NOT NULL,
PRIMARY KEY ( id )
);
CREATE TABLE consumed_serials (
storage_node_id bytea NOT NULL,
serial_number bytea NOT NULL,
expires_at timestamp with time zone NOT NULL,
PRIMARY KEY ( storage_node_id, serial_number )
);
CREATE TABLE coupons (
id bytea NOT NULL,
project_id bytea NOT NULL,
@ -915,6 +937,15 @@ CREATE TABLE pending_audits (
path bytea NOT NULL,
PRIMARY KEY ( node_id )
);
CREATE TABLE pending_serial_queue (
storage_node_id bytea NOT NULL,
bucket_id bytea NOT NULL,
serial_number bytea NOT NULL,
action integer NOT NULL,
settled bigint NOT NULL,
expires_at timestamp with time zone NOT NULL,
PRIMARY KEY ( storage_node_id, bucket_id, serial_number )
);
CREATE TABLE projects (
id bytea NOT NULL,
name text NOT NULL,
@ -1089,6 +1120,7 @@ CREATE TABLE user_credits (
PRIMARY KEY ( id ),
UNIQUE ( id, offer_id )
);
CREATE INDEX consumed_serials_expires_at_index ON consumed_serials ( expires_at );
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 );
@ -1945,6 +1977,74 @@ func (f CoinpaymentsTransaction_CreatedAt_Field) value() interface{} {
func (CoinpaymentsTransaction_CreatedAt_Field) _Column() string { return "created_at" }
type ConsumedSerial struct {
StorageNodeId []byte
SerialNumber []byte
ExpiresAt time.Time
}
func (ConsumedSerial) _Table() string { return "consumed_serials" }
type ConsumedSerial_Update_Fields struct {
}
type ConsumedSerial_StorageNodeId_Field struct {
_set bool
_null bool
_value []byte
}
func ConsumedSerial_StorageNodeId(v []byte) ConsumedSerial_StorageNodeId_Field {
return ConsumedSerial_StorageNodeId_Field{_set: true, _value: v}
}
func (f ConsumedSerial_StorageNodeId_Field) value() interface{} {
if !f._set || f._null {
return nil
}
return f._value
}
func (ConsumedSerial_StorageNodeId_Field) _Column() string { return "storage_node_id" }
type ConsumedSerial_SerialNumber_Field struct {
_set bool
_null bool
_value []byte
}
func ConsumedSerial_SerialNumber(v []byte) ConsumedSerial_SerialNumber_Field {
return ConsumedSerial_SerialNumber_Field{_set: true, _value: v}
}
func (f ConsumedSerial_SerialNumber_Field) value() interface{} {
if !f._set || f._null {
return nil
}
return f._value
}
func (ConsumedSerial_SerialNumber_Field) _Column() string { return "serial_number" }
type ConsumedSerial_ExpiresAt_Field struct {
_set bool
_null bool
_value time.Time
}
func ConsumedSerial_ExpiresAt(v time.Time) ConsumedSerial_ExpiresAt_Field {
return ConsumedSerial_ExpiresAt_Field{_set: true, _value: v}
}
func (f ConsumedSerial_ExpiresAt_Field) value() interface{} {
if !f._set || f._null {
return nil
}
return f._value
}
func (ConsumedSerial_ExpiresAt_Field) _Column() string { return "expires_at" }
type Coupon struct {
Id []byte
ProjectId []byte
@ -4527,6 +4627,134 @@ func (f PendingAudits_Path_Field) value() interface{} {
func (PendingAudits_Path_Field) _Column() string { return "path" }
type PendingSerialQueue struct {
StorageNodeId []byte
BucketId []byte
SerialNumber []byte
Action uint
Settled uint64
ExpiresAt time.Time
}
func (PendingSerialQueue) _Table() string { return "pending_serial_queue" }
type PendingSerialQueue_Update_Fields struct {
}
type PendingSerialQueue_StorageNodeId_Field struct {
_set bool
_null bool
_value []byte
}
func PendingSerialQueue_StorageNodeId(v []byte) PendingSerialQueue_StorageNodeId_Field {
return PendingSerialQueue_StorageNodeId_Field{_set: true, _value: v}
}
func (f PendingSerialQueue_StorageNodeId_Field) value() interface{} {
if !f._set || f._null {
return nil
}
return f._value
}
func (PendingSerialQueue_StorageNodeId_Field) _Column() string { return "storage_node_id" }
type PendingSerialQueue_BucketId_Field struct {
_set bool
_null bool
_value []byte
}
func PendingSerialQueue_BucketId(v []byte) PendingSerialQueue_BucketId_Field {
return PendingSerialQueue_BucketId_Field{_set: true, _value: v}
}
func (f PendingSerialQueue_BucketId_Field) value() interface{} {
if !f._set || f._null {
return nil
}
return f._value
}
func (PendingSerialQueue_BucketId_Field) _Column() string { return "bucket_id" }
type PendingSerialQueue_SerialNumber_Field struct {
_set bool
_null bool
_value []byte
}
func PendingSerialQueue_SerialNumber(v []byte) PendingSerialQueue_SerialNumber_Field {
return PendingSerialQueue_SerialNumber_Field{_set: true, _value: v}
}
func (f PendingSerialQueue_SerialNumber_Field) value() interface{} {
if !f._set || f._null {
return nil
}
return f._value
}
func (PendingSerialQueue_SerialNumber_Field) _Column() string { return "serial_number" }
type PendingSerialQueue_Action_Field struct {
_set bool
_null bool
_value uint
}
func PendingSerialQueue_Action(v uint) PendingSerialQueue_Action_Field {
return PendingSerialQueue_Action_Field{_set: true, _value: v}
}
func (f PendingSerialQueue_Action_Field) value() interface{} {
if !f._set || f._null {
return nil
}
return f._value
}
func (PendingSerialQueue_Action_Field) _Column() string { return "action" }
type PendingSerialQueue_Settled_Field struct {
_set bool
_null bool
_value uint64
}
func PendingSerialQueue_Settled(v uint64) PendingSerialQueue_Settled_Field {
return PendingSerialQueue_Settled_Field{_set: true, _value: v}
}
func (f PendingSerialQueue_Settled_Field) value() interface{} {
if !f._set || f._null {
return nil
}
return f._value
}
func (PendingSerialQueue_Settled_Field) _Column() string { return "settled" }
type PendingSerialQueue_ExpiresAt_Field struct {
_set bool
_null bool
_value time.Time
}
func PendingSerialQueue_ExpiresAt(v time.Time) PendingSerialQueue_ExpiresAt_Field {
return PendingSerialQueue_ExpiresAt_Field{_set: true, _value: v}
}
func (f PendingSerialQueue_ExpiresAt_Field) value() interface{} {
if !f._set || f._null {
return nil
}
return f._value
}
func (PendingSerialQueue_ExpiresAt_Field) _Column() string { return "expires_at" }
type Project struct {
Id []byte
Name string
@ -7382,11 +7610,9 @@ type LeafSerialNumber_Row struct {
LeafSerialNumber []byte
}
type Paged_ReportedSerial_By_ExpiresAt_LessOrEqual_Continuation struct {
_value_expires_at time.Time
type Paged_PendingSerialQueue_Continuation struct {
_value_storage_node_id []byte
_value_bucket_id []byte
_value_action uint
_value_serial_number []byte
_set bool
}
@ -7860,28 +8086,52 @@ 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) (
func (obj *postgresImpl) ReplaceNoReturn_PendingSerialQueue(ctx context.Context,
pending_serial_queue_storage_node_id PendingSerialQueue_StorageNodeId_Field,
pending_serial_queue_bucket_id PendingSerialQueue_BucketId_Field,
pending_serial_queue_serial_number PendingSerialQueue_SerialNumber_Field,
pending_serial_queue_action PendingSerialQueue_Action_Field,
pending_serial_queue_settled PendingSerialQueue_Settled_Field,
pending_serial_queue_expires_at PendingSerialQueue_ExpiresAt_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()
__storage_node_id_val := pending_serial_queue_storage_node_id.value()
__bucket_id_val := pending_serial_queue_bucket_id.value()
__serial_number_val := pending_serial_queue_serial_number.value()
__action_val := pending_serial_queue_action.value()
__settled_val := pending_serial_queue_settled.value()
__expires_at_val := pending_serial_queue_expires_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 __embed_stmt = __sqlbundle_Literal("INSERT INTO pending_serial_queue ( storage_node_id, bucket_id, serial_number, action, settled, expires_at ) VALUES ( ?, ?, ?, ?, ?, ? ) ON CONFLICT ( storage_node_id, bucket_id, serial_number ) DO UPDATE SET storage_node_id = EXCLUDED.storage_node_id, bucket_id = EXCLUDED.bucket_id, serial_number = EXCLUDED.serial_number, action = EXCLUDED.action, settled = EXCLUDED.settled, expires_at = EXCLUDED.expires_at")
var __values []interface{}
__values = append(__values, __expires_at_val, __storage_node_id_val, __bucket_id_val, __action_val, __serial_number_val, __settled_val, __observed_at_val)
__values = append(__values, __storage_node_id_val, __bucket_id_val, __serial_number_val, __action_val, __settled_val, __expires_at_val)
var __stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
obj.logStmt(__stmt, __values...)
_, err = obj.driver.ExecContext(ctx, __stmt, __values...)
if err != nil {
return obj.makeErr(err)
}
return nil
}
func (obj *postgresImpl) CreateNoReturn_ConsumedSerial(ctx context.Context,
consumed_serial_storage_node_id ConsumedSerial_StorageNodeId_Field,
consumed_serial_serial_number ConsumedSerial_SerialNumber_Field,
consumed_serial_expires_at ConsumedSerial_ExpiresAt_Field) (
err error) {
defer mon.Task()(&ctx)(&err)
__storage_node_id_val := consumed_serial_storage_node_id.value()
__serial_number_val := consumed_serial_serial_number.value()
__expires_at_val := consumed_serial_expires_at.value()
var __embed_stmt = __sqlbundle_Literal("INSERT INTO consumed_serials ( storage_node_id, serial_number, expires_at ) VALUES ( ?, ?, ? )")
var __values []interface{}
__values = append(__values, __storage_node_id_val, __serial_number_val, __expires_at_val)
var __stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
obj.logStmt(__stmt, __values...)
@ -9559,22 +9809,20 @@ 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) {
func (obj *postgresImpl) Paged_PendingSerialQueue(ctx context.Context,
limit int, start *Paged_PendingSerialQueue_Continuation) (
rows []*PendingSerialQueue, next *Paged_PendingSerialQueue_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, reported_serials.storage_node_id, reported_serials.bucket_id, reported_serials.action, 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_stmt = __sqlbundle_Literal("SELECT pending_serial_queue.storage_node_id, pending_serial_queue.bucket_id, pending_serial_queue.serial_number, pending_serial_queue.action, pending_serial_queue.settled, pending_serial_queue.expires_at, pending_serial_queue.storage_node_id, pending_serial_queue.bucket_id, pending_serial_queue.serial_number FROM pending_serial_queue WHERE (pending_serial_queue.storage_node_id, pending_serial_queue.bucket_id, pending_serial_queue.serial_number) > (?, ?, ?) ORDER BY pending_serial_queue.storage_node_id, pending_serial_queue.bucket_id, pending_serial_queue.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 __embed_first_stmt = __sqlbundle_Literal("SELECT pending_serial_queue.storage_node_id, pending_serial_queue.bucket_id, pending_serial_queue.serial_number, pending_serial_queue.action, pending_serial_queue.settled, pending_serial_queue.expires_at, pending_serial_queue.storage_node_id, pending_serial_queue.bucket_id, pending_serial_queue.serial_number FROM pending_serial_queue ORDER BY pending_serial_queue.storage_node_id, pending_serial_queue.bucket_id, pending_serial_queue.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_storage_node_id, start._value_bucket_id, start._value_action, start._value_serial_number, limit)
__values = append(__values, start._value_storage_node_id, start._value_bucket_id, start._value_serial_number, limit)
__stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
} else {
__values = append(__values, limit)
@ -9588,16 +9836,16 @@ func (obj *postgresImpl) Paged_ReportedSerial_By_ExpiresAt_LessOrEqual(ctx conte
}
defer __rows.Close()
var __continuation Paged_ReportedSerial_By_ExpiresAt_LessOrEqual_Continuation
var __continuation Paged_PendingSerialQueue_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)
pending_serial_queue := &PendingSerialQueue{}
err = __rows.Scan(&pending_serial_queue.StorageNodeId, &pending_serial_queue.BucketId, &pending_serial_queue.SerialNumber, &pending_serial_queue.Action, &pending_serial_queue.Settled, &pending_serial_queue.ExpiresAt, &__continuation._value_storage_node_id, &__continuation._value_bucket_id, &__continuation._value_serial_number)
if err != nil {
return nil, nil, obj.makeErr(err)
}
rows = append(rows, reported_serial)
rows = append(rows, pending_serial_queue)
next = &__continuation
}
if err := __rows.Err(); err != nil {
@ -9608,6 +9856,28 @@ func (obj *postgresImpl) Paged_ReportedSerial_By_ExpiresAt_LessOrEqual(ctx conte
}
func (obj *postgresImpl) Has_ConsumedSerial_By_StorageNodeId_And_SerialNumber(ctx context.Context,
consumed_serial_storage_node_id ConsumedSerial_StorageNodeId_Field,
consumed_serial_serial_number ConsumedSerial_SerialNumber_Field) (
has bool, err error) {
defer mon.Task()(&ctx)(&err)
var __embed_stmt = __sqlbundle_Literal("SELECT EXISTS( SELECT 1 FROM consumed_serials WHERE consumed_serials.storage_node_id = ? AND consumed_serials.serial_number = ? )")
var __values []interface{}
__values = append(__values, consumed_serial_storage_node_id.value(), consumed_serial_serial_number.value())
var __stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
obj.logStmt(__stmt, __values...)
err = obj.driver.QueryRowContext(ctx, __stmt, __values...).Scan(&has)
if err != nil {
return false, obj.makeErr(err)
}
return has, 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,
@ -12507,15 +12777,15 @@ 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) (
func (obj *postgresImpl) Delete_ConsumedSerial_By_ExpiresAt_LessOrEqual(ctx context.Context,
consumed_serial_expires_at_less_or_equal ConsumedSerial_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 __embed_stmt = __sqlbundle_Literal("DELETE FROM consumed_serials WHERE consumed_serials.expires_at <= ?")
var __values []interface{}
__values = append(__values, reported_serial_expires_at_less_or_equal.value())
__values = append(__values, consumed_serial_expires_at_less_or_equal.value())
var __stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
obj.logStmt(__stmt, __values...)
@ -13006,6 +13276,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.ExecContext(ctx, "DELETE FROM pending_serial_queue;")
if err != nil {
return 0, obj.makeErr(err)
}
__count, err = __res.RowsAffected()
if err != nil {
return 0, obj.makeErr(err)
@ -13136,6 +13416,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.ExecContext(ctx, "DELETE FROM consumed_serials;")
if err != nil {
return 0, obj.makeErr(err)
}
__count, err = __res.RowsAffected()
if err != nil {
return 0, obj.makeErr(err)
@ -13657,28 +13947,52 @@ 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) (
func (obj *cockroachImpl) ReplaceNoReturn_PendingSerialQueue(ctx context.Context,
pending_serial_queue_storage_node_id PendingSerialQueue_StorageNodeId_Field,
pending_serial_queue_bucket_id PendingSerialQueue_BucketId_Field,
pending_serial_queue_serial_number PendingSerialQueue_SerialNumber_Field,
pending_serial_queue_action PendingSerialQueue_Action_Field,
pending_serial_queue_settled PendingSerialQueue_Settled_Field,
pending_serial_queue_expires_at PendingSerialQueue_ExpiresAt_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()
__storage_node_id_val := pending_serial_queue_storage_node_id.value()
__bucket_id_val := pending_serial_queue_bucket_id.value()
__serial_number_val := pending_serial_queue_serial_number.value()
__action_val := pending_serial_queue_action.value()
__settled_val := pending_serial_queue_settled.value()
__expires_at_val := pending_serial_queue_expires_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 __embed_stmt = __sqlbundle_Literal("UPSERT INTO pending_serial_queue ( storage_node_id, bucket_id, serial_number, action, settled, expires_at ) VALUES ( ?, ?, ?, ?, ?, ? )")
var __values []interface{}
__values = append(__values, __expires_at_val, __storage_node_id_val, __bucket_id_val, __action_val, __serial_number_val, __settled_val, __observed_at_val)
__values = append(__values, __storage_node_id_val, __bucket_id_val, __serial_number_val, __action_val, __settled_val, __expires_at_val)
var __stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
obj.logStmt(__stmt, __values...)
_, err = obj.driver.ExecContext(ctx, __stmt, __values...)
if err != nil {
return obj.makeErr(err)
}
return nil
}
func (obj *cockroachImpl) CreateNoReturn_ConsumedSerial(ctx context.Context,
consumed_serial_storage_node_id ConsumedSerial_StorageNodeId_Field,
consumed_serial_serial_number ConsumedSerial_SerialNumber_Field,
consumed_serial_expires_at ConsumedSerial_ExpiresAt_Field) (
err error) {
defer mon.Task()(&ctx)(&err)
__storage_node_id_val := consumed_serial_storage_node_id.value()
__serial_number_val := consumed_serial_serial_number.value()
__expires_at_val := consumed_serial_expires_at.value()
var __embed_stmt = __sqlbundle_Literal("INSERT INTO consumed_serials ( storage_node_id, serial_number, expires_at ) VALUES ( ?, ?, ? )")
var __values []interface{}
__values = append(__values, __storage_node_id_val, __serial_number_val, __expires_at_val)
var __stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
obj.logStmt(__stmt, __values...)
@ -15356,22 +15670,20 @@ 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) {
func (obj *cockroachImpl) Paged_PendingSerialQueue(ctx context.Context,
limit int, start *Paged_PendingSerialQueue_Continuation) (
rows []*PendingSerialQueue, next *Paged_PendingSerialQueue_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, reported_serials.storage_node_id, reported_serials.bucket_id, reported_serials.action, 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_stmt = __sqlbundle_Literal("SELECT pending_serial_queue.storage_node_id, pending_serial_queue.bucket_id, pending_serial_queue.serial_number, pending_serial_queue.action, pending_serial_queue.settled, pending_serial_queue.expires_at, pending_serial_queue.storage_node_id, pending_serial_queue.bucket_id, pending_serial_queue.serial_number FROM pending_serial_queue WHERE (pending_serial_queue.storage_node_id, pending_serial_queue.bucket_id, pending_serial_queue.serial_number) > (?, ?, ?) ORDER BY pending_serial_queue.storage_node_id, pending_serial_queue.bucket_id, pending_serial_queue.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 __embed_first_stmt = __sqlbundle_Literal("SELECT pending_serial_queue.storage_node_id, pending_serial_queue.bucket_id, pending_serial_queue.serial_number, pending_serial_queue.action, pending_serial_queue.settled, pending_serial_queue.expires_at, pending_serial_queue.storage_node_id, pending_serial_queue.bucket_id, pending_serial_queue.serial_number FROM pending_serial_queue ORDER BY pending_serial_queue.storage_node_id, pending_serial_queue.bucket_id, pending_serial_queue.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_storage_node_id, start._value_bucket_id, start._value_action, start._value_serial_number, limit)
__values = append(__values, start._value_storage_node_id, start._value_bucket_id, start._value_serial_number, limit)
__stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
} else {
__values = append(__values, limit)
@ -15385,16 +15697,16 @@ func (obj *cockroachImpl) Paged_ReportedSerial_By_ExpiresAt_LessOrEqual(ctx cont
}
defer __rows.Close()
var __continuation Paged_ReportedSerial_By_ExpiresAt_LessOrEqual_Continuation
var __continuation Paged_PendingSerialQueue_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)
pending_serial_queue := &PendingSerialQueue{}
err = __rows.Scan(&pending_serial_queue.StorageNodeId, &pending_serial_queue.BucketId, &pending_serial_queue.SerialNumber, &pending_serial_queue.Action, &pending_serial_queue.Settled, &pending_serial_queue.ExpiresAt, &__continuation._value_storage_node_id, &__continuation._value_bucket_id, &__continuation._value_serial_number)
if err != nil {
return nil, nil, obj.makeErr(err)
}
rows = append(rows, reported_serial)
rows = append(rows, pending_serial_queue)
next = &__continuation
}
if err := __rows.Err(); err != nil {
@ -15405,6 +15717,28 @@ func (obj *cockroachImpl) Paged_ReportedSerial_By_ExpiresAt_LessOrEqual(ctx cont
}
func (obj *cockroachImpl) Has_ConsumedSerial_By_StorageNodeId_And_SerialNumber(ctx context.Context,
consumed_serial_storage_node_id ConsumedSerial_StorageNodeId_Field,
consumed_serial_serial_number ConsumedSerial_SerialNumber_Field) (
has bool, err error) {
defer mon.Task()(&ctx)(&err)
var __embed_stmt = __sqlbundle_Literal("SELECT EXISTS( SELECT 1 FROM consumed_serials WHERE consumed_serials.storage_node_id = ? AND consumed_serials.serial_number = ? )")
var __values []interface{}
__values = append(__values, consumed_serial_storage_node_id.value(), consumed_serial_serial_number.value())
var __stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
obj.logStmt(__stmt, __values...)
err = obj.driver.QueryRowContext(ctx, __stmt, __values...).Scan(&has)
if err != nil {
return false, obj.makeErr(err)
}
return has, 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,
@ -18304,15 +18638,15 @@ 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) (
func (obj *cockroachImpl) Delete_ConsumedSerial_By_ExpiresAt_LessOrEqual(ctx context.Context,
consumed_serial_expires_at_less_or_equal ConsumedSerial_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 __embed_stmt = __sqlbundle_Literal("DELETE FROM consumed_serials WHERE consumed_serials.expires_at <= ?")
var __values []interface{}
__values = append(__values, reported_serial_expires_at_less_or_equal.value())
__values = append(__values, consumed_serial_expires_at_less_or_equal.value())
var __stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
obj.logStmt(__stmt, __values...)
@ -18803,6 +19137,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.ExecContext(ctx, "DELETE FROM pending_serial_queue;")
if err != nil {
return 0, obj.makeErr(err)
}
__count, err = __res.RowsAffected()
if err != nil {
return 0, obj.makeErr(err)
@ -18933,6 +19277,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.ExecContext(ctx, "DELETE FROM consumed_serials;")
if err != nil {
return 0, obj.makeErr(err)
}
__count, err = __res.RowsAffected()
if err != nil {
return 0, obj.makeErr(err)
@ -19355,6 +19709,19 @@ func (rx *Rx) CreateNoReturn_BucketStorageTally(ctx context.Context,
}
func (rx *Rx) CreateNoReturn_ConsumedSerial(ctx context.Context,
consumed_serial_storage_node_id ConsumedSerial_StorageNodeId_Field,
consumed_serial_serial_number ConsumedSerial_SerialNumber_Field,
consumed_serial_expires_at ConsumedSerial_ExpiresAt_Field) (
err error) {
var tx *Tx
if tx, err = rx.getTx(ctx); err != nil {
return
}
return tx.CreateNoReturn_ConsumedSerial(ctx, consumed_serial_storage_node_id, consumed_serial_serial_number, consumed_serial_expires_at)
}
func (rx *Rx) CreateNoReturn_GracefulExitProgress(ctx context.Context,
graceful_exit_progress_node_id GracefulExitProgress_NodeId_Field,
graceful_exit_progress_bytes_transferred GracefulExitProgress_BytesTransferred_Field) (
@ -19852,6 +20219,17 @@ func (rx *Rx) Delete_BucketMetainfo_By_ProjectId_And_Name(ctx context.Context,
return tx.Delete_BucketMetainfo_By_ProjectId_And_Name(ctx, bucket_metainfo_project_id, bucket_metainfo_name)
}
func (rx *Rx) Delete_ConsumedSerial_By_ExpiresAt_LessOrEqual(ctx context.Context,
consumed_serial_expires_at_less_or_equal ConsumedSerial_ExpiresAt_Field) (
count int64, err error) {
var tx *Tx
if tx, err = rx.getTx(ctx); err != nil {
return
}
return tx.Delete_ConsumedSerial_By_ExpiresAt_LessOrEqual(ctx, consumed_serial_expires_at_less_or_equal)
}
func (rx *Rx) Delete_Coupon_By_Id(ctx context.Context,
coupon_id Coupon_Id_Field) (
deleted bool, err error) {
@ -19957,17 +20335,6 @@ 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) {
@ -20393,6 +20760,17 @@ func (rx *Rx) Get_ValueAttribution_By_ProjectId_And_BucketName(ctx context.Conte
return tx.Get_ValueAttribution_By_ProjectId_And_BucketName(ctx, value_attribution_project_id, value_attribution_bucket_name)
}
func (rx *Rx) Has_ConsumedSerial_By_StorageNodeId_And_SerialNumber(ctx context.Context,
consumed_serial_storage_node_id ConsumedSerial_StorageNodeId_Field,
consumed_serial_serial_number ConsumedSerial_SerialNumber_Field) (
has bool, err error) {
var tx *Tx
if tx, err = rx.getTx(ctx); err != nil {
return
}
return tx.Has_ConsumedSerial_By_StorageNodeId_And_SerialNumber(ctx, consumed_serial_storage_node_id, consumed_serial_serial_number)
}
func (rx *Rx) Limited_BucketMetainfo_By_ProjectId_And_Name_GreaterOrEqual_OrderBy_Asc_Name(ctx context.Context,
bucket_metainfo_project_id BucketMetainfo_ProjectId_Field,
bucket_metainfo_name_greater_or_equal BucketMetainfo_Name_Field,
@ -20564,31 +20942,29 @@ 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) {
func (rx *Rx) Paged_PendingSerialQueue(ctx context.Context,
limit int, start *Paged_PendingSerialQueue_Continuation) (
rows []*PendingSerialQueue, next *Paged_PendingSerialQueue_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)
return tx.Paged_PendingSerialQueue(ctx, 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) (
func (rx *Rx) ReplaceNoReturn_PendingSerialQueue(ctx context.Context,
pending_serial_queue_storage_node_id PendingSerialQueue_StorageNodeId_Field,
pending_serial_queue_bucket_id PendingSerialQueue_BucketId_Field,
pending_serial_queue_serial_number PendingSerialQueue_SerialNumber_Field,
pending_serial_queue_action PendingSerialQueue_Action_Field,
pending_serial_queue_settled PendingSerialQueue_Settled_Field,
pending_serial_queue_expires_at PendingSerialQueue_ExpiresAt_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)
return tx.ReplaceNoReturn_PendingSerialQueue(ctx, pending_serial_queue_storage_node_id, pending_serial_queue_bucket_id, pending_serial_queue_serial_number, pending_serial_queue_action, pending_serial_queue_settled, pending_serial_queue_expires_at)
}
@ -20954,6 +21330,12 @@ type Methods interface {
bucket_storage_tally_metadata_size BucketStorageTally_MetadataSize_Field) (
err error)
CreateNoReturn_ConsumedSerial(ctx context.Context,
consumed_serial_storage_node_id ConsumedSerial_StorageNodeId_Field,
consumed_serial_serial_number ConsumedSerial_SerialNumber_Field,
consumed_serial_expires_at ConsumedSerial_ExpiresAt_Field) (
err error)
CreateNoReturn_GracefulExitProgress(ctx context.Context,
graceful_exit_progress_node_id GracefulExitProgress_NodeId_Field,
graceful_exit_progress_bytes_transferred GracefulExitProgress_BytesTransferred_Field) (
@ -21223,6 +21605,10 @@ type Methods interface {
bucket_metainfo_name BucketMetainfo_Name_Field) (
deleted bool, err error)
Delete_ConsumedSerial_By_ExpiresAt_LessOrEqual(ctx context.Context,
consumed_serial_expires_at_less_or_equal ConsumedSerial_ExpiresAt_Field) (
count int64, err error)
Delete_Coupon_By_Id(ctx context.Context,
coupon_id Coupon_Id_Field) (
deleted bool, err error)
@ -21266,10 +21652,6 @@ 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)
@ -21448,6 +21830,11 @@ type Methods interface {
value_attribution_bucket_name ValueAttribution_BucketName_Field) (
value_attribution *ValueAttribution, err error)
Has_ConsumedSerial_By_StorageNodeId_And_SerialNumber(ctx context.Context,
consumed_serial_storage_node_id ConsumedSerial_StorageNodeId_Field,
consumed_serial_serial_number ConsumedSerial_SerialNumber_Field) (
has bool, err error)
Limited_BucketMetainfo_By_ProjectId_And_Name_GreaterOrEqual_OrderBy_Asc_Name(ctx context.Context,
bucket_metainfo_project_id BucketMetainfo_ProjectId_Field,
bucket_metainfo_name_greater_or_equal BucketMetainfo_Name_Field,
@ -21529,19 +21916,17 @@ 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)
Paged_PendingSerialQueue(ctx context.Context,
limit int, start *Paged_PendingSerialQueue_Continuation) (
rows []*PendingSerialQueue, next *Paged_PendingSerialQueue_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) (
ReplaceNoReturn_PendingSerialQueue(ctx context.Context,
pending_serial_queue_storage_node_id PendingSerialQueue_StorageNodeId_Field,
pending_serial_queue_bucket_id PendingSerialQueue_BucketId_Field,
pending_serial_queue_serial_number PendingSerialQueue_SerialNumber_Field,
pending_serial_queue_action PendingSerialQueue_Action_Field,
pending_serial_queue_settled PendingSerialQueue_Settled_Field,
pending_serial_queue_expires_at PendingSerialQueue_ExpiresAt_Field) (
err error)
UpdateNoReturn_AccountingTimestamps_By_Name(ctx context.Context,

View File

@ -52,6 +52,12 @@ CREATE TABLE coinpayments_transactions (
created_at timestamp with time zone NOT NULL,
PRIMARY KEY ( id )
);
CREATE TABLE consumed_serials (
storage_node_id bytea NOT NULL,
serial_number bytea NOT NULL,
expires_at timestamp with time zone NOT NULL,
PRIMARY KEY ( storage_node_id, serial_number )
);
CREATE TABLE coupons (
id bytea NOT NULL,
project_id bytea NOT NULL,
@ -200,6 +206,15 @@ CREATE TABLE pending_audits (
path bytea NOT NULL,
PRIMARY KEY ( node_id )
);
CREATE TABLE pending_serial_queue (
storage_node_id bytea NOT NULL,
bucket_id bytea NOT NULL,
serial_number bytea NOT NULL,
action integer NOT NULL,
settled bigint NOT NULL,
expires_at timestamp with time zone NOT NULL,
PRIMARY KEY ( storage_node_id, bucket_id, serial_number )
);
CREATE TABLE projects (
id bytea NOT NULL,
name text NOT NULL,
@ -374,6 +389,7 @@ CREATE TABLE user_credits (
PRIMARY KEY ( id ),
UNIQUE ( id, offer_id )
);
CREATE INDEX consumed_serials_expires_at_index ON consumed_serials ( expires_at );
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 );

View File

@ -759,6 +759,28 @@ func (db *satelliteDB) PostgresMigration() *migrate.Migration {
);`,
},
},
{
DB: db.DB,
Description: "Create consumed serials tables",
Version: 82,
Action: migrate.SQL{
`CREATE TABLE consumed_serials (
storage_node_id bytea NOT NULL,
serial_number bytea NOT NULL,
expires_at timestamp with time zone NOT NULL,
PRIMARY KEY ( storage_node_id, serial_number )
);`,
`CREATE TABLE pending_serial_queue (
storage_node_id bytea NOT NULL,
bucket_id bytea NOT NULL,
serial_number bytea NOT NULL,
action integer NOT NULL,
settled bigint NOT NULL,
expires_at timestamp with time zone NOT NULL,
PRIMARY KEY ( storage_node_id, bucket_id, serial_number )
);`,
},
},
},
}
}

View File

@ -37,18 +37,31 @@ type ordersDB struct {
// CreateSerialInfo creates serial number entry in database.
func (db *ordersDB) CreateSerialInfo(ctx context.Context, serialNumber storj.SerialNumber, bucketID []byte, limitExpiration time.Time) (err error) {
defer mon.Task()(&ctx)(&err)
return db.db.CreateNoReturn_SerialNumber(
ctx,
dbx.SerialNumber_SerialNumber(serialNumber.Bytes()),
dbx.SerialNumber_BucketId(bucketID),
dbx.SerialNumber_ExpiresAt(limitExpiration),
dbx.SerialNumber_ExpiresAt(limitExpiration.UTC()),
)
}
// DeleteExpiredSerials deletes all expired serials in serial_number and used_serials table.
func (db *ordersDB) DeleteExpiredSerials(ctx context.Context, now time.Time) (_ int, err error) {
defer mon.Task()(&ctx)(&err)
count, err := db.db.Delete_SerialNumber_By_ExpiresAt_LessOrEqual(ctx, dbx.SerialNumber_ExpiresAt(now))
count, err := db.db.Delete_SerialNumber_By_ExpiresAt_LessOrEqual(ctx, dbx.SerialNumber_ExpiresAt(now.UTC()))
if err != nil {
return 0, err
}
return int(count), nil
}
// DeleteExpiredConsumedSerials deletes all expired serials in the consumed_serials table.
func (db *ordersDB) DeleteExpiredConsumedSerials(ctx context.Context, now time.Time) (_ int, err error) {
defer mon.Task()(&ctx, now)(&err)
count, err := db.db.Delete_ConsumedSerial_By_ExpiresAt_LessOrEqual(ctx, dbx.ConsumedSerial_ExpiresAt(now))
if err != nil {
return 0, err
}
@ -60,6 +73,7 @@ func (db *ordersDB) DeleteExpiredSerials(ctx context.Context, now time.Time) (_
// It returns the bucket ID associated to serialNumber.
func (db *ordersDB) UseSerialNumber(ctx context.Context, serialNumber storj.SerialNumber, storageNodeID storj.NodeID) (_ []byte, err error) {
defer mon.Task()(&ctx)(&err)
statement := db.db.Rebind(
`INSERT INTO used_serials (serial_number_id, storage_node_id)
SELECT id, ? FROM serial_numbers WHERE serial_number = ?`,
@ -88,6 +102,7 @@ func (db *ordersDB) UseSerialNumber(ctx context.Context, serialNumber storj.Seri
// UpdateBucketBandwidthAllocation updates 'allocated' bandwidth for given bucket.
func (db *ordersDB) UpdateBucketBandwidthAllocation(ctx context.Context, projectID uuid.UUID, bucketName []byte, action pb.PieceAction, amount int64, intervalStart time.Time) (err error) {
defer mon.Task()(&ctx)(&err)
statement := db.db.Rebind(
`INSERT INTO bucket_bandwidth_rollups (bucket_name, project_id, interval_start, interval_seconds, action, inline, allocated, settled)
VALUES (?, ?, ?, ?, ?, ?, ?, ?)
@ -95,7 +110,7 @@ func (db *ordersDB) UpdateBucketBandwidthAllocation(ctx context.Context, project
DO UPDATE SET allocated = bucket_bandwidth_rollups.allocated + ?`,
)
_, err = db.db.ExecContext(ctx, statement,
bucketName, projectID[:], intervalStart, defaultIntervalSeconds, action, 0, uint64(amount), 0, uint64(amount),
bucketName, projectID[:], intervalStart.UTC(), defaultIntervalSeconds, action, 0, uint64(amount), 0, uint64(amount),
)
if err != nil {
return err
@ -107,6 +122,7 @@ func (db *ordersDB) UpdateBucketBandwidthAllocation(ctx context.Context, project
// UpdateBucketBandwidthSettle updates 'settled' bandwidth for given bucket.
func (db *ordersDB) UpdateBucketBandwidthSettle(ctx context.Context, projectID uuid.UUID, bucketName []byte, action pb.PieceAction, amount int64, intervalStart time.Time) (err error) {
defer mon.Task()(&ctx)(&err)
statement := db.db.Rebind(
`INSERT INTO bucket_bandwidth_rollups (bucket_name, project_id, interval_start, interval_seconds, action, inline, allocated, settled)
VALUES (?, ?, ?, ?, ?, ?, ?, ?)
@ -114,7 +130,7 @@ func (db *ordersDB) UpdateBucketBandwidthSettle(ctx context.Context, projectID u
DO UPDATE SET settled = bucket_bandwidth_rollups.settled + ?`,
)
_, err = db.db.ExecContext(ctx, statement,
bucketName, projectID[:], intervalStart, defaultIntervalSeconds, action, 0, 0, uint64(amount), uint64(amount),
bucketName, projectID[:], intervalStart.UTC(), defaultIntervalSeconds, action, 0, 0, uint64(amount), uint64(amount),
)
if err != nil {
return err
@ -125,6 +141,7 @@ func (db *ordersDB) UpdateBucketBandwidthSettle(ctx context.Context, projectID u
// UpdateBucketBandwidthInline updates 'inline' bandwidth for given bucket.
func (db *ordersDB) UpdateBucketBandwidthInline(ctx context.Context, projectID uuid.UUID, bucketName []byte, action pb.PieceAction, amount int64, intervalStart time.Time) (err error) {
defer mon.Task()(&ctx)(&err)
statement := db.db.Rebind(
`INSERT INTO bucket_bandwidth_rollups (bucket_name, project_id, interval_start, interval_seconds, action, inline, allocated, settled)
VALUES (?, ?, ?, ?, ?, ?, ?, ?)
@ -132,7 +149,7 @@ func (db *ordersDB) UpdateBucketBandwidthInline(ctx context.Context, projectID u
DO UPDATE SET inline = bucket_bandwidth_rollups.inline + ?`,
)
_, err = db.db.ExecContext(ctx, statement,
bucketName, projectID[:], intervalStart, defaultIntervalSeconds, action, uint64(amount), 0, 0, uint64(amount),
bucketName, projectID[:], intervalStart.UTC(), defaultIntervalSeconds, action, uint64(amount), 0, 0, uint64(amount),
)
if err != nil {
return err
@ -143,6 +160,7 @@ func (db *ordersDB) UpdateBucketBandwidthInline(ctx context.Context, projectID u
// UpdateStoragenodeBandwidthSettle updates 'settled' bandwidth for given storage node for the given intervalStart time.
func (db *ordersDB) UpdateStoragenodeBandwidthSettle(ctx context.Context, storageNode storj.NodeID, action pb.PieceAction, amount int64, intervalStart time.Time) (err error) {
defer mon.Task()(&ctx)(&err)
statement := db.db.Rebind(
`INSERT INTO storagenode_bandwidth_rollups (storagenode_id, interval_start, interval_seconds, action, settled)
VALUES (?, ?, ?, ?, ?)
@ -150,7 +168,7 @@ func (db *ordersDB) UpdateStoragenodeBandwidthSettle(ctx context.Context, storag
DO UPDATE SET settled = storagenode_bandwidth_rollups.settled + ?`,
)
_, err = db.db.ExecContext(ctx, statement,
storageNode.Bytes(), intervalStart, defaultIntervalSeconds, action, uint64(amount), uint64(amount),
storageNode.Bytes(), intervalStart.UTC(), defaultIntervalSeconds, action, uint64(amount), uint64(amount),
)
if err != nil {
return err
@ -161,9 +179,10 @@ func (db *ordersDB) UpdateStoragenodeBandwidthSettle(ctx context.Context, storag
// GetBucketBandwidth gets total bucket bandwidth from period of time.
func (db *ordersDB) GetBucketBandwidth(ctx context.Context, projectID uuid.UUID, bucketName []byte, from, to time.Time) (_ int64, err error) {
defer mon.Task()(&ctx)(&err)
var sum *int64
query := `SELECT SUM(settled) FROM bucket_bandwidth_rollups WHERE bucket_name = ? AND project_id = ? AND interval_start > ? AND interval_start <= ?`
err = db.db.QueryRow(ctx, db.db.Rebind(query), bucketName, projectID[:], from, to).Scan(&sum)
err = db.db.QueryRow(ctx, db.db.Rebind(query), bucketName, projectID[:], from.UTC(), to.UTC()).Scan(&sum)
if err == sql.ErrNoRows || sum == nil {
return 0, nil
}
@ -173,9 +192,10 @@ func (db *ordersDB) GetBucketBandwidth(ctx context.Context, projectID uuid.UUID,
// GetStorageNodeBandwidth gets total storage node bandwidth from period of time.
func (db *ordersDB) GetStorageNodeBandwidth(ctx context.Context, nodeID storj.NodeID, from, to time.Time) (_ int64, err error) {
defer mon.Task()(&ctx)(&err)
var sum *int64
query := `SELECT SUM(settled) FROM storagenode_bandwidth_rollups WHERE storagenode_id = ? AND interval_start > ? AND interval_start <= ?`
err = db.db.QueryRow(ctx, db.db.Rebind(query), nodeID.Bytes(), from, to).Scan(&sum)
err = db.db.QueryRow(ctx, db.db.Rebind(query), nodeID.Bytes(), from.UTC(), to.UTC()).Scan(&sum)
if err == sql.ErrNoRows || sum == nil {
return 0, nil
}
@ -185,16 +205,17 @@ func (db *ordersDB) GetStorageNodeBandwidth(ctx context.Context, nodeID storj.No
// UnuseSerialNumber removes pair serial number -> storage node id from database.
func (db *ordersDB) UnuseSerialNumber(ctx context.Context, serialNumber storj.SerialNumber, storageNodeID storj.NodeID) (err error) {
defer mon.Task()(&ctx)(&err)
statement := `DELETE FROM used_serials WHERE storage_node_id = ? AND
serial_number_id IN (SELECT id FROM serial_numbers WHERE serial_number = ?)`
_, err = db.db.ExecContext(ctx, db.db.Rebind(statement), storageNodeID.Bytes(), serialNumber.Bytes())
return err
}
// ProcessOrders take a list of order requests and "settles" them in one transaction.
// ProcessOrders take a list of order requests and inserts them into the pending serials queue.
//
// ProcessOrders requires that all orders come from the same storage node.
func (db *ordersDB) ProcessOrders(ctx context.Context, requests []*orders.ProcessOrderRequest, observedAt time.Time) (responses []*orders.ProcessOrderResponse, err error) {
func (db *ordersDB) ProcessOrders(ctx context.Context, requests []*orders.ProcessOrderRequest) (responses []*orders.ProcessOrderResponse, err error) {
defer mon.Task()(&ctx)(&err)
if len(requests) == 0 {
@ -266,7 +287,7 @@ func (db *ordersDB) ProcessOrders(ctx context.Context, requests []*orders.Proces
})
continue
}
expiresAtArray = append(expiresAtArray, roundToNextDay(request.OrderLimit.OrderExpiration))
expiresAtArray = append(expiresAtArray, request.OrderLimit.OrderExpiration)
bucketIDArray = append(bucketIDArray, bucketIDs[i])
actionArray = append(actionArray, request.OrderLimit.Action)
serialNumCopy := request.Order.SerialNumber
@ -283,34 +304,46 @@ func (db *ordersDB) ProcessOrders(ctx context.Context, requests []*orders.Proces
switch db.db.implementation {
case dbutil.Postgres:
stmt = `
INSERT INTO reported_serials (
expires_at, storage_node_id, bucket_id, action, serial_number, settled, observed_at
INSERT INTO pending_serial_queue (
storage_node_id, bucket_id, serial_number, action, settled, expires_at
)
SELECT unnest($1::timestamptz[]), $2::bytea, unnest($3::bytea[]), unnest($4::integer[]), unnest($5::bytea[]), unnest($6::bigint[]), $7::timestamptz
ON CONFLICT ( expires_at, storage_node_id, bucket_id, action, serial_number )
SELECT
$1::bytea,
unnest($2::bytea[]),
unnest($3::bytea[]),
unnest($4::integer[]),
unnest($5::bigint[]),
unnest($6::timestamptz[])
ON CONFLICT ( storage_node_id, bucket_id, serial_number )
DO UPDATE SET
action = EXCLUDED.action,
settled = EXCLUDED.settled,
observed_at = EXCLUDED.observed_at
expires_at = EXCLUDED.expires_at
`
case dbutil.Cockroach:
stmt = `
UPSERT INTO reported_serials (
expires_at, storage_node_id, bucket_id, action, serial_number, settled, observed_at
UPSERT INTO pending_serial_queue (
storage_node_id, bucket_id, serial_number, action, settled, expires_at
)
SELECT unnest($1::timestamptz[]), $2::bytea, unnest($3::bytea[]), unnest($4::integer[]), unnest($5::bytea[]), unnest($6::bigint[]), $7::timestamptz
SELECT
$1::bytea,
unnest($2::bytea[]),
unnest($3::bytea[]),
unnest($4::integer[]),
unnest($5::bigint[]),
unnest($6::timestamptz[])
`
default:
return nil, Error.New("invalid dbType: %v", db.db.driver)
}
_, err = db.db.ExecContext(ctx, stmt,
pq.Array(expiresAtArray),
storageNodeID.Bytes(),
pq.ByteaArray(bucketIDArray),
pq.Array(actionArray),
pq.ByteaArray(serialNumArray),
pq.Array(actionArray),
pq.Array(settledArray),
observedAt.UTC(),
pq.Array(expiresAtArray),
)
if err != nil {
return nil, Error.Wrap(err)
@ -318,105 +351,13 @@ func (db *ordersDB) ProcessOrders(ctx context.Context, requests []*orders.Proces
return responses, nil
}
func roundToNextDay(t time.Time) time.Time {
t = t.UTC()
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
db *satelliteDB
log *zap.Logger
}
@ -424,7 +365,7 @@ func (db *ordersDB) WithTransaction(ctx context.Context, cb func(ctx context.Con
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})
return cb(ctx, &ordersDBTx{tx: tx, db: db.db, log: db.db.log})
})
}
@ -528,11 +469,154 @@ func (tx *ordersDBTx) UpdateStoragenodeBandwidthBatch(ctx context.Context, inter
return err
}
// DeleteExpiredReportedSerials deletes any expired reported serials as of expiredThreshold.
func (tx *ordersDBTx) DeleteExpiredReportedSerials(ctx context.Context, expiredThreshold time.Time) (err error) {
// CreateConsumedSerialsBatch TODO: DOCS
func (tx *ordersDBTx) CreateConsumedSerialsBatch(ctx context.Context, consumedSerials []orders.ConsumedSerial) (err error) {
defer mon.Task()(&ctx)(&err)
_, err = tx.tx.Delete_ReportedSerial_By_ExpiresAt_LessOrEqual(ctx,
dbx.ReportedSerial_ExpiresAt(expiredThreshold))
if len(consumedSerials) == 0 {
return nil
}
var storageNodeIDSlice [][]byte
var serialNumberSlice [][]byte
var expiresAtSlice []time.Time
for _, consumedSerial := range consumedSerials {
consumedSerial := consumedSerial
storageNodeIDSlice = append(storageNodeIDSlice, consumedSerial.NodeID.Bytes())
serialNumberSlice = append(serialNumberSlice, consumedSerial.SerialNumber.Bytes())
expiresAtSlice = append(expiresAtSlice, consumedSerial.ExpiresAt)
}
var stmt string
switch tx.db.implementation {
case dbutil.Postgres:
stmt = `
INSERT INTO consumed_serials (
storage_node_id, serial_number, expires_at
)
SELECT unnest($1::bytea[]), unnest($2::bytea[]), unnest($3::timestamptz[])
ON CONFLICT ( storage_node_id, serial_number ) DO NOTHING
`
case dbutil.Cockroach:
stmt = `
UPSERT INTO consumed_serials (
storage_node_id, serial_number, expires_at
)
SELECT unnest($1::bytea[]), unnest($2::bytea[]), unnest($3::timestamptz[])
`
default:
return Error.New("invalid dbType: %v", tx.db.driver)
}
_, err = tx.tx.Tx.ExecContext(ctx, stmt,
pq.ByteaArray(storageNodeIDSlice),
pq.ByteaArray(serialNumberSlice),
pq.Array(expiresAtSlice),
)
return Error.Wrap(err)
}
func (tx *ordersDBTx) HasConsumedSerial(ctx context.Context, nodeID storj.NodeID, serialNumber storj.SerialNumber) (exists bool, err error) {
defer mon.Task()(&ctx)(&err)
exists, err = tx.tx.Has_ConsumedSerial_By_StorageNodeId_And_SerialNumber(ctx,
dbx.ConsumedSerial_StorageNodeId(nodeID.Bytes()),
dbx.ConsumedSerial_SerialNumber(serialNumber.Bytes()))
return exists, Error.Wrap(err)
}
//
// transaction/batch methods
//
type ordersDBQueue struct {
db *satelliteDB
log *zap.Logger
produced []orders.PendingSerial
cont *dbx.Paged_PendingSerialQueue_Continuation
}
func (db *ordersDB) WithQueue(ctx context.Context, cb func(ctx context.Context, queue orders.Queue) error) (err error) {
defer mon.Task()(&ctx)(&err)
queue := &ordersDBQueue{
db: db.db,
log: db.db.log,
}
err = cb(ctx, queue)
if err != nil {
return errs.Wrap(err)
}
var (
storageNodeSlice [][]byte
bucketIDSlice [][]byte
serialNumberSlice [][]byte
)
for _, reportedSerial := range queue.produced {
reportedSerial := reportedSerial
storageNodeSlice = append(storageNodeSlice, reportedSerial.NodeID.Bytes())
bucketIDSlice = append(bucketIDSlice, reportedSerial.BucketID)
serialNumberSlice = append(serialNumberSlice, reportedSerial.SerialNumber.Bytes())
}
_, err = db.db.ExecContext(ctx, `
DELETE FROM pending_serial_queue WHERE (
storage_node_id, bucket_id, serial_number
) IN (
SELECT
unnest($1::bytea[]),
unnest($2::bytea[]),
unnest($3::bytea[])
)
`,
pq.ByteaArray(storageNodeSlice),
pq.ByteaArray(bucketIDSlice),
pq.ByteaArray(serialNumberSlice))
return Error.Wrap(err)
}
func (queue *ordersDBQueue) GetPendingSerialsBatch(ctx context.Context, size int) (pendingSerials []orders.PendingSerial, err error) {
defer mon.Task()(&ctx)(&err)
rows, cont, err := queue.db.Paged_PendingSerialQueue(ctx, size, queue.cont)
if err != nil {
return nil, Error.Wrap(err)
}
pendingSerials = make([]orders.PendingSerial, 0, len(rows))
for _, row := range rows {
nodeID, err := storj.NodeIDFromBytes(row.StorageNodeId)
if err != nil {
queue.log.Error("Invalid storage node id in pending serials queue",
zap.Binary("id", row.StorageNodeId),
zap.Error(errs.Wrap(err)))
continue
}
serialNumber, err := storj.SerialNumberFromBytes(row.SerialNumber)
if err != nil {
queue.log.Error("Invalid serial number in pending serials queue",
zap.Binary("id", row.SerialNumber),
zap.Error(errs.Wrap(err)))
continue
}
pendingSerials = append(pendingSerials, orders.PendingSerial{
ExpiresAt: row.ExpiresAt,
NodeID: nodeID,
BucketID: row.BucketId,
Action: row.Action,
SerialNumber: serialNumber,
Settled: row.Settled,
})
}
queue.produced = append(queue.produced, pendingSerials...)
queue.cont = cont
return pendingSerials, nil
}

View File

@ -0,0 +1,497 @@
-- 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 consumed_serials (
storage_node_id bytea NOT NULL,
serial_number bytea NOT NULL,
expires_at timestamp with time zone NOT NULL,
PRIMARY KEY ( storage_node_id, serial_number )
);
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 credits (
user_id bytea NOT NULL,
transaction_id text NOT NULL,
amount bigint NOT NULL,
created_at timestamp with time zone NOT NULL,
PRIMARY KEY ( transaction_id )
);
CREATE TABLE credits_spendings (
id bytea NOT NULL,
user_id bytea NOT NULL,
project_id bytea NOT NULL,
amount bigint NOT NULL,
status integer NOT NULL,
created_at timestamp with time zone NOT NULL,
PRIMARY KEY ( id )
);
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 pending_serial_queue (
storage_node_id bytea NOT NULL,
bucket_id bytea NOT NULL,
serial_number bytea NOT NULL,
action integer NOT NULL,
settled bigint NOT NULL,
expires_at timestamp with time zone NOT NULL,
PRIMARY KEY ( storage_node_id, bucket_id, serial_number )
);
CREATE TABLE projects (
id bytea NOT NULL,
name text NOT NULL,
description text NOT NULL,
usage_limit bigint NOT NULL,
rate_limit integer,
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 with time zone 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 with time zone 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 consumed_serials_expires_at_index ON consumed_serials ( expires_at );
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 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 "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');
INSERT INTO "reported_serials" ("expires_at", "storage_node_id", "bucket_id", "action", "serial_number", "settled", "observed_at") VALUES ('2020-01-11 08:00:00.000000+00', 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', E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\014/testbucket'::bytea, 1, E'0123456701234567'::bytea, 100, '2020-01-11 08:00:00.000000+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 "projects"("id", "name", "description", "usage_limit", "rate_limit", "partner_id", "owner_id", "created_at") VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\347'::bytea, 'projName1', 'Test project 1', 0, 2000000, NULL, E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, '2020-01-15 08:28:24.636949+00');
INSERT INTO "credits" ("user_id", "transaction_id", "amount", "created_at") VALUES (E'\\362\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\014'::bytea, 'transactionID', 10, '2019-06-01 08:28:24.267934+00');
INSERT INTO "credits_spendings" ("id", "user_id", "project_id", "amount", "status", "created_at") VALUES (E'\\362\\342\\363\\371>+F\\256\\263\\300\\275|\\342N\\347\\014'::bytea, 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, 0, '2019-06-01 09:28:24.267934+00');
-- NEW DATA --
INSERT INTO "pending_serial_queue" ("storage_node_id", "bucket_id", "serial_number", "action", "settled", "expires_at") 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', E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\014/testbucket'::bytea, E'5123456701234567'::bytea, 1, 100, '2020-01-11 08:00:00.000000+00');
INSERT INTO "consumed_serials" ("storage_node_id", "serial_number", "expires_at") 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', E'1234567012345678'::bytea, '2020-01-12 08:00:00.000000+00');

View File

@ -445,7 +445,7 @@ identity.key-path: /root/.local/share/storj/identity/satellite/identity.key
# repairer.timeout: 5m0s
# how often to flush the reported serial rollups to the database
# reported-rollup.interval: 24h0m0s
# reported-rollup.interval: 5m0s
# option for deleting tallies after they are rolled up
# rollup.delete-tallies: true