satellite/satellitedb: stop using _gob columns

This sets the corresponding _numeric columns to be NOT NULL (it has been
verified manually that there are no more NULL _numeric values on any
known satellites, and it should be impossible with current code to get
new NULL values in the _numeric columns.

We can't drop the _gob columns immediately, as there will still be code
running that expects them, but once this version is deployed we can
finally drop them and be totally done with this crazy 5-step migration.

Change-Id: I518302528d972090d56b3eedc815656610ac8e73
This commit is contained in:
paul cannon 2022-03-24 11:19:47 -05:00
parent 0018d62837
commit d8733ddd40
17 changed files with 793 additions and 1286 deletions

View File

@ -484,8 +484,6 @@ func New(log *zap.Logger, full *identity.FullIdentity, db DB,
service,
pc.StripeCoinPayments.TransactionUpdateInterval,
pc.StripeCoinPayments.AccountBalanceUpdateInterval,
pc.StripeCoinPayments.GobFloatMigrationBatchInterval,
pc.StripeCoinPayments.GobFloatMigrationBatchSize,
)
peer.Services.Add(lifecycle.Item{
Name: "payments.stripe:service",

View File

@ -11,7 +11,6 @@ import (
"go.uber.org/zap"
"golang.org/x/sync/errgroup"
"storj.io/common/errs2"
"storj.io/common/sync2"
)
@ -27,24 +26,16 @@ type Chore struct {
service *Service
TransactionCycle *sync2.Cycle
AccountBalanceCycle *sync2.Cycle
// temporary! remove once all gob-encoded big.Float records are gone from DBs on all satellites:
TransactionMigrationCycle *sync2.Cycle
ConversionRateMigrationCycle *sync2.Cycle
migrationBatchSize int
}
// NewChore creates new clearing loop chore.
// TODO: uncomment new interval when coupons will be finished.
func NewChore(log *zap.Logger, service *Service, txInterval, accBalanceInterval, migrationBatchInterval time.Duration, migrationBatchSize int) *Chore {
func NewChore(log *zap.Logger, service *Service, txInterval, accBalanceInterval time.Duration) *Chore {
return &Chore{
log: log,
service: service,
TransactionCycle: sync2.NewCycle(txInterval),
AccountBalanceCycle: sync2.NewCycle(accBalanceInterval),
TransactionMigrationCycle: sync2.NewCycle(migrationBatchInterval),
ConversionRateMigrationCycle: sync2.NewCycle(migrationBatchInterval),
migrationBatchSize: migrationBatchSize,
log: log,
service: service,
TransactionCycle: sync2.NewCycle(txInterval),
AccountBalanceCycle: sync2.NewCycle(accBalanceInterval),
}
}
@ -77,54 +68,6 @@ func (chore *Chore) Run(ctx context.Context) (err error) {
},
)
var transactionMigrationNextRange string
var transactionMigrationDone bool
chore.TransactionMigrationCycle.Start(ctx, &group,
func(ctx context.Context) (err error) {
if transactionMigrationDone {
mon.Event("coinpayments_transactions_gob_encoded_big_float_migration_done")
return nil
}
var migrated int
migrated, transactionMigrationNextRange, err = chore.service.db.Transactions().MigrateGobFloatTransactionRecords(ctx, transactionMigrationNextRange, chore.migrationBatchSize)
mon.Meter("coinpayments_transactions_gob_encoded_big_float_rows_migrated").Mark(migrated)
if transactionMigrationNextRange == "" {
transactionMigrationDone = true
}
if err != nil {
if !errs2.IsCanceled(err) {
chore.log.Error("gob-encoded big.Float transaction migration chore failed", zap.Error(ErrChore.Wrap(err)))
}
return err
}
return nil
},
)
var conversionRateMigrationNextRange string
var conversionRateMigrationDone bool
chore.ConversionRateMigrationCycle.Start(ctx, &group,
func(ctx context.Context) (err error) {
if conversionRateMigrationDone {
mon.Event("stripecoinpayments_tx_conversion_rates_gob_encoded_big_float_migration_done")
return nil
}
var migrated int
migrated, conversionRateMigrationNextRange, err = chore.service.db.Transactions().MigrateGobFloatConversionRateRecords(ctx, conversionRateMigrationNextRange, chore.migrationBatchSize)
mon.Meter("stripecoinpayments_tx_conversion_rates_gob_encoded_big_float_rows_migrated").Mark(migrated)
if conversionRateMigrationNextRange == "" {
conversionRateMigrationDone = true
}
if err != nil {
if !errs2.IsCanceled(err) {
chore.log.Error("gob-encoded big.Float conversion rate migration chore failed", zap.Error(ErrChore.Wrap(err)))
}
return err
}
return nil
},
)
return ErrChore.Wrap(group.Wait())
}
@ -134,7 +77,5 @@ func (chore *Chore) Close() (err error) {
chore.TransactionCycle.Close()
chore.AccountBalanceCycle.Close()
chore.TransactionMigrationCycle.Close()
chore.ConversionRateMigrationCycle.Close()
return nil
}

View File

@ -47,10 +47,6 @@ type Config struct {
ConversionRatesCycleInterval time.Duration `help:"amount of time we wait before running next conversion rates update loop" default:"10m" testDefault:"$TESTINTERVAL"`
AutoAdvance bool `help:"toogle autoadvance feature for invoice creation" default:"false"`
ListingLimit int `help:"sets the maximum amount of items before we start paging on requests" default:"100" hidden:"true"`
// temporary! remove after all gob-encoded big.Float values are out of all satellite DBs.
GobFloatMigrationBatchInterval time.Duration `help:"amount of time to wait between gob-encoded big.Float database migration batches" default:"1m" testDefault:"$TESTINTERVAL" hidden:"true"`
GobFloatMigrationBatchSize int `help:"number of rows with gob-encoded big.Float values to migrate at once" default:"100" testDefault:"10" hidden:"true"`
}
// Service is an implementation for payment service via Stripe and Coinpayments.

View File

@ -39,11 +39,6 @@ type TransactionsDB interface {
ListPending(ctx context.Context, offset int64, limit int, before time.Time) (TransactionsPage, error)
// ListUnapplied returns TransactionsPage with completed transaction that should be applied to account balance.
ListUnapplied(ctx context.Context, offset int64, limit int, before time.Time) (TransactionsPage, error)
// MigrateGobFloatTransactionRecords is a strictly-temporary task that will, with time, eliminate gob-encoded big.Float records from the coinpayments_transactions table. It should be called repeatedly, passing back nextRangeStart for the next rangeStart parameter, until it encounters an error or returns nextRangeStart = "".
MigrateGobFloatTransactionRecords(ctx context.Context, rangeStart string, limit int) (migrated int, nextRangeStart string, err error)
// MigrateGobFloatConversionRateRecords is a strictly-temporary task that will, with time, eliminate gob-encoded big.Float records from the stripecoinpayments_tx_conversion_rates table. It should be called repeatedly, passing back nextRangeStart for the next rangeStart parameter, until it encounters an error or returns nextRangeStart = "".
MigrateGobFloatConversionRateRecords(ctx context.Context, rangeStart string, limit int) (migrated int, nextRangeStart string, err error)
}
// Transaction defines coinpayments transaction info that is stored in the DB.

View File

@ -4,13 +4,8 @@
package stripecoinpayments_test
import (
"context"
"encoding/base64"
"errors"
"fmt"
"math/big"
"math/rand"
"sort"
"sync"
"testing"
"time"
@ -427,186 +422,3 @@ func TestTransactions_ApplyTransactionBalance(t *testing.T) {
require.EqualValues(t, "0.2", cbt.Metadata["storj_usd_rate"])
})
}
func TestDatabaseGobEncodedBigFloatTransactionMigration(t *testing.T) {
satellitedbtest.Run(t, func(ctx *testcontext.Context, t *testing.T, db satellite.DB) {
transactions := db.StripeCoinPayments().Transactions()
testTransactions, ok := transactions.(interface {
ForTestingOnlyInsertGobTransaction(ctx context.Context, tx stripecoinpayments.Transaction) (time.Time, error)
})
require.Truef(t, ok, "transactions object of type %T does not have the needed test hook method", transactions)
// make some random records, insert in db
const numRecords = 100
asInserted := make([]stripecoinpayments.Transaction, 0, numRecords)
for x := 0; x < numRecords; x++ {
tx := stripecoinpayments.Transaction{
ID: coinpayments.TransactionID(fmt.Sprintf("transaction%05d", x)),
Status: coinpayments.Status(x % 2), // 0 (pending) or 1 (received)
AccountID: testrand.UUID(),
Amount: monetary.AmountFromBaseUnits(testrand.Int63n(1e15), monetary.StorjToken),
Received: monetary.AmountFromBaseUnits(testrand.Int63n(1e15), monetary.StorjToken),
Address: fmt.Sprintf("%x", testrand.Bytes(20)),
Key: fmt.Sprintf("%x", testrand.Bytes(20)),
}
createTime, err := testTransactions.ForTestingOnlyInsertGobTransaction(ctx, tx)
require.NoError(t, err)
tx.CreatedAt = createTime
asInserted = append(asInserted, tx)
}
// run migration in a number of batches
const (
numBatches = 6
batchSize = (numRecords + numBatches - 1) / numBatches
)
var (
totalMigrated int
batchesSoFar int
rangeStart string
)
// check that batches work as expected
for {
migrated, nextRangeStart, err := transactions.MigrateGobFloatTransactionRecords(ctx, rangeStart, batchSize)
require.NoError(t, err)
batchesSoFar++
totalMigrated += migrated
// no interference from other db clients, so all should succeed
if migrated < batchSize {
// we expect this to be the last batch, so nextRangeStart should be the empty string
assert.Equal(t, "", nextRangeStart)
assert.Equal(t, numRecords, totalMigrated)
if migrated == 0 {
// it took an extra batch to find out we were done, because batchSize % numRecords == 0
assert.Equal(t, numBatches+1, batchesSoFar)
} else {
assert.Equal(t, numBatches, batchesSoFar)
}
break
}
assert.NotNil(t, nextRangeStart)
assert.Equal(t, batchSize, migrated)
assert.LessOrEqual(t, totalMigrated, numRecords)
require.Less(t, rangeStart, nextRangeStart)
rangeStart = nextRangeStart
}
// read results back in and ensure values are still as expected
transactionsPage, err := transactions.ListPending(ctx, 0, numRecords+1, time.Now())
require.NoError(t, err)
assert.Len(t, transactionsPage.Transactions, numRecords)
assert.False(t, transactionsPage.Next)
// sort the output values to make comparison simple (they're
// currently sorted by created_at, which might be the same
// ordering we want, but it's not guaranteed, as pg and crdb
// only have time values to microsecond resolution).
sort.Slice(transactionsPage.Transactions, func(i, j int) bool {
return transactionsPage.Transactions[i].ID < transactionsPage.Transactions[j].ID
})
assert.Equal(t, asInserted, transactionsPage.Transactions)
})
}
func TestDatabaseGobEncodedBigFloatConversionRateMigration(t *testing.T) {
satellitedbtest.Run(t, func(ctx *testcontext.Context, t *testing.T, db satellite.DB) {
transactions := db.StripeCoinPayments().Transactions()
testTransactions, ok := transactions.(interface {
ForTestingOnlyInsertGobConversionRate(ctx context.Context, txID coinpayments.TransactionID, rate decimal.Decimal) error
})
require.Truef(t, ok, "transactions object of type %T does not have the needed test hook method", transactions)
// make some random records, insert in db
const numRecords = 100
type rateRecord struct {
txID coinpayments.TransactionID
rate decimal.Decimal
}
asInserted := make([]rateRecord, 0, numRecords)
for x := 0; x < numRecords; x++ {
rateDecimal := mustRandomDecimalNumber(4, 11)
rr := rateRecord{
txID: coinpayments.TransactionID(fmt.Sprintf("transaction%05d", x)),
rate: rateDecimal,
}
err := testTransactions.ForTestingOnlyInsertGobConversionRate(ctx, rr.txID, rr.rate)
require.NoError(t, err)
asInserted = append(asInserted, rr)
}
// run migration in a number of batches
const (
numBatches = 6
batchSize = (numRecords + numBatches - 1) / numBatches
)
var (
totalMigrated int
batchesSoFar int
rangeStart string
)
// check that batches work as expected
for {
migrated, nextRangeStart, err := transactions.MigrateGobFloatConversionRateRecords(ctx, rangeStart, batchSize)
require.NoError(t, err)
batchesSoFar++
totalMigrated += migrated
// no interference from other db clients, so all should succeed
if migrated < batchSize {
// we expect this to be the last batch, so nextRangeStart should be the empty string
assert.Equal(t, "", nextRangeStart)
assert.Equal(t, numRecords, totalMigrated)
if migrated == 0 {
// it took an extra batch to find out we were done, because batchSize % numRecords == 0
assert.Equal(t, numBatches+1, batchesSoFar)
} else {
assert.Equal(t, numBatches, batchesSoFar)
}
break
}
assert.NotNil(t, nextRangeStart)
assert.Equal(t, batchSize, migrated)
assert.LessOrEqual(t, totalMigrated, numRecords)
require.Less(t, rangeStart, nextRangeStart)
rangeStart = nextRangeStart
}
// read results back in and ensure values are still as expected
for n := 0; n < numRecords; n++ {
rate, err := transactions.GetLockedRate(ctx, asInserted[n].txID)
require.NoError(t, err)
assert.Truef(t, asInserted[n].rate.Equal(rate), "for row %d: expected=%s, got=%s", n, asInserted[n].rate.String(), rate.String())
}
})
}
func mustRandomDecimalNumber(wholePartDigits, fractionalPartDigits int) decimal.Decimal {
decimalNumber, err := randomDecimalNumber(wholePartDigits, fractionalPartDigits)
if err != nil {
panic(err)
}
return decimalNumber
}
func randomDecimalNumber(wholePartDigits, fractionalPartDigits int) (decimal.Decimal, error) {
wholePart := randomNumberWithNDigits(wholePartDigits)
fractionalPart := randomNumberWithNDigits(fractionalPartDigits)
numberAsString := fmt.Sprintf("%d.%0*d", wholePart, fractionalPartDigits, fractionalPart)
return decimal.NewFromString(numberAsString)
}
func randomNumberWithNDigits(numDigits int) *big.Int {
randomSource := rand.New(rand.NewSource(rand.Int63()))
num := big.NewInt(10)
num.Exp(num, big.NewInt(int64(numDigits)), nil)
return num.Rand(randomSource, num)
}

View File

@ -1,484 +0,0 @@
// Copyright (C) 2021 Storj Labs, Inc.
// See LICENSE for copying information.
package satellitedb
import (
"context"
"fmt"
"math/big"
"time"
"github.com/shopspring/decimal"
"github.com/zeebo/errs"
"storj.io/private/dbutil/cockroachutil"
"storj.io/storj/satellite/payments/coinpayments"
"storj.io/storj/satellite/payments/monetary"
"storj.io/storj/satellite/payments/stripecoinpayments"
"storj.io/storj/satellite/satellitedb/dbx"
)
type transactionToMigrate struct {
id string
amountGob []byte
amountNumeric *int64
receivedGob []byte
receivedNumeric *int64
status coinpayments.Status
}
// getTransactionsToMigrate fetches the data from up to limit rows in the
// coinpayments_transactions table which still have gob-encoded big.Float
// values in them. Querying starts at idRangeStart and proceeds in
// lexicographical order by the id column.
func (db *coinPaymentsTransactions) getTransactionsToMigrate(ctx context.Context, idRangeStart string, limit int) (toMigrate []transactionToMigrate, err error) {
defer mon.Task()(&ctx)(&err)
rows, err := db.db.QueryContext(ctx, `
SELECT id, amount_gob, amount_numeric, received_gob, received_numeric, status
FROM coinpayments_transactions
WHERE (amount_gob IS NOT NULL OR received_gob IS NOT NULL)
AND id >= $1::text
ORDER BY id
LIMIT $2
`, idRangeStart, limit)
if err != nil {
return nil, Error.New("could not issue transaction migration collection query: %v", err)
}
defer func() { err = errs.Combine(err, rows.Close()) }()
var xactsToMigrate []transactionToMigrate
for rows.Next() {
xact := transactionToMigrate{}
err = rows.Scan(&xact.id, &xact.amountGob, &xact.amountNumeric, &xact.receivedGob, &xact.receivedNumeric, &xact.status)
if err != nil {
return nil, Error.New("could not read results of transaction migration collect query: %v", err)
}
xactsToMigrate = append(xactsToMigrate, xact)
}
if err := rows.Err(); err != nil {
return nil, Error.Wrap(err)
}
return xactsToMigrate, nil
}
// getTransactionsToMigrateWithRetry calls getTransactionsToMigrate in a loop
// until a result is found without any "retry needed" error being returned.
func (db *coinPaymentsTransactions) getTransactionsToMigrateWithRetry(ctx context.Context, idRangeStart string, limit int) (toMigrate []transactionToMigrate, err error) {
defer mon.Task()(&ctx)(&err)
for {
toMigrate, err = db.getTransactionsToMigrate(ctx, idRangeStart, limit)
if err != nil {
if cockroachutil.NeedsRetry(err) {
continue
}
return nil, err
}
break
}
return toMigrate, nil
}
// migrateGobFloatTransaction replaces gob-encoded big.Float values for one
// specific row in the coinpayments_transactions table with plain integers (in
// the base units of the currency for the column). Either the amount_gob or
// received_gob columns, or both, might be non-null, indicating the need for a
// replacement in the corresponding amount_numeric or received_numeric
// columns.
//
// This is implemented as a compare-and-swap, so that if any relevant changes
// occur on the target row since the time that we fetched it, this migration
// will not occur. Instead, wasMigrated will be returned as false, so that a
// future query can select the row for migration again if needed.
func (db *coinPaymentsTransactions) migrateGobFloatTransaction(ctx context.Context, transaction transactionToMigrate) (wasMigrated bool, err error) {
defer mon.Task()(&ctx)(&err)
currency := monetary.StorjToken
args := []interface{}{
transaction.id,
transaction.status,
}
transactionIDArg := "$1"
transactionStatusArg := "$2"
var amountSetNewValue, amountGobOldValue, amountNumericOldValue string
var receivedSetNewValue, receivedGobOldValue, receivedNumericOldValue string
if transaction.amountGob == nil {
amountGobOldValue = "IS NULL"
} else {
amount, err := monetaryAmountFromGobEncodedBigFloat(transaction.amountGob, currency)
if err != nil {
return false, Error.New("invalid gob-encoded amount in amount_gob column under transaction id %x: %w", transaction.id, err)
}
args = append(args, amount.BaseUnits())
amountSetNewValue = fmt.Sprintf(", amount_numeric = $%d", len(args))
args = append(args, transaction.amountGob)
amountGobOldValue = fmt.Sprintf("= $%d::bytea", len(args))
}
if transaction.amountNumeric == nil {
amountNumericOldValue = "IS NULL"
} else {
args = append(args, *transaction.amountNumeric)
amountNumericOldValue = fmt.Sprintf("= $%d", len(args))
}
if transaction.receivedGob == nil {
receivedGobOldValue = "IS NULL"
} else {
received, err := monetaryAmountFromGobEncodedBigFloat(transaction.receivedGob, currency)
if err != nil {
return false, Error.New("invalid gob-encoded amount in received_gob column under transaction id %x: %w", transaction.id, err)
}
args = append(args, received.BaseUnits())
receivedSetNewValue = fmt.Sprintf(", received_numeric = $%d", len(args))
args = append(args, transaction.receivedGob)
receivedGobOldValue = fmt.Sprintf("= $%d::bytea", len(args))
}
if transaction.receivedNumeric == nil {
receivedNumericOldValue = "IS NULL"
} else {
args = append(args, *transaction.receivedNumeric)
receivedNumericOldValue = fmt.Sprintf("= $%d", len(args))
}
updateQuery := fmt.Sprintf(`
UPDATE coinpayments_transactions
SET amount_gob = NULL, received_gob = NULL%s%s
WHERE id = %s
AND status = %s
AND amount_gob %s
AND amount_numeric %s
AND received_gob %s
AND received_numeric %s
`,
amountSetNewValue, receivedSetNewValue,
transactionIDArg, transactionStatusArg,
amountGobOldValue, amountNumericOldValue,
receivedGobOldValue, receivedNumericOldValue)
result, err := db.db.ExecContext(ctx, updateQuery, args...)
if err != nil {
return false, Error.New("failed to update coinpayments_transactions row %x: %w", transaction.id, err)
}
// if zero rows were updated, then the row with this ID was changed by
// something else before this migration got to it. we'll want to try
// again with the next read query.
numAffected, err := result.RowsAffected()
if err != nil {
return false, Error.New("could not get number of rows affected: %w", err)
}
return numAffected == 1, nil
}
// migrateGobFloatTransactionWithRetry calls migrateGobFloatTransaction in a
// loop until it succeeds without any "retry needed" error being returned.
func (db *coinPaymentsTransactions) migrateGobFloatTransactionWithRetry(ctx context.Context, transaction transactionToMigrate) (wasMigrated bool, err error) {
defer mon.Task()(&ctx)(&err)
for {
wasMigrated, err = db.migrateGobFloatTransaction(ctx, transaction)
if err != nil {
if cockroachutil.NeedsRetry(err) {
continue
}
}
return wasMigrated, err
}
}
// MigrateGobFloatTransactionRecords is a strictly-temporary task that will,
// with time, eliminate gob-encoded big.Float records from the
// coinpayments_transactions table. It should be called repeatedly, passing back
// nextRangeStart for the next rangeStart parameter, until it encounters an
// error or returns nextRangeStart = "".
func (db *coinPaymentsTransactions) MigrateGobFloatTransactionRecords(ctx context.Context, rangeStart string, limit int) (migrated int, nextRangeStart string, err error) {
defer mon.Task()(&ctx)(&err)
xactsToMigrate, err := db.getTransactionsToMigrateWithRetry(ctx, rangeStart, limit)
if err != nil {
// some sort of internal error or programming error
return 0, "", err
}
if len(xactsToMigrate) == 0 {
// all rows are migrated!
return 0, "", nil
}
for _, xact := range xactsToMigrate {
wasMigrated, err := db.migrateGobFloatTransactionWithRetry(ctx, xact)
if err != nil {
// some sort of internal error or programming error
return migrated, "", err
}
if wasMigrated {
migrated++
} else if nextRangeStart == "" {
// Start here with the next call so that we can try again
// (unless we are already going to start at an earlier point)
nextRangeStart = xact.id
}
}
// if nextRangeStart != "", then we need to redo some rows, and it's already
// set appropriately.
if nextRangeStart == "" {
// if len(xactsToMigrate) < limit, then this is the last range and we've
// completed the migration (leave nextRangeStart as "").
if len(xactsToMigrate) == limit {
// next time we can start after the last ID we just processed
nextRangeStart = xactsToMigrate[len(xactsToMigrate)-1].id
}
}
return migrated, nextRangeStart, nil
}
type conversionRateToMigrate struct {
txID string
rateGob []byte
rateNumeric *float64
}
// getConversionRatesToMigrate fetches the data from up to limit rows in the
// stripecoinpayments_tx_conversion_rates table which still have gob-encoded
// big.Float values in them. Querying starts at txidRangeStart and proceeds in
// lexicographical order by the tx_id column.
func (db *coinPaymentsTransactions) getConversionRatesToMigrate(ctx context.Context, txidRangeStart string, limit int) (toMigrate []conversionRateToMigrate, err error) {
defer mon.Task()(&ctx)(&err)
rows, err := db.db.QueryContext(ctx, `
SELECT tx_id, rate_gob, rate_numeric
FROM stripecoinpayments_tx_conversion_rates
WHERE rate_gob IS NOT NULL
AND tx_id >= $1::text
ORDER BY tx_id
LIMIT $2
`, txidRangeStart, limit)
if err != nil {
return nil, Error.New("could not issue conversion rate migration collection query: %v", err)
}
defer func() { err = errs.Combine(err, rows.Close()) }()
var ratesToMigrate []conversionRateToMigrate
for rows.Next() {
rateRecord := conversionRateToMigrate{}
err = rows.Scan(&rateRecord.txID, &rateRecord.rateGob, &rateRecord.rateNumeric)
if err != nil {
return nil, Error.New("could not read results of conversion rate migration collect query: %v", err)
}
ratesToMigrate = append(ratesToMigrate, rateRecord)
}
if err := rows.Err(); err != nil {
return nil, Error.Wrap(err)
}
return ratesToMigrate, nil
}
// getConversionRatesToMigrateWithRetry calls getConversionRatesToMigrate in a loop
// until a result is found without any "retry needed" error being returned.
func (db *coinPaymentsTransactions) getConversionRatesToMigrateWithRetry(ctx context.Context, idRangeStart string, limit int) (toMigrate []conversionRateToMigrate, err error) {
defer mon.Task()(&ctx)(&err)
for {
toMigrate, err = db.getConversionRatesToMigrate(ctx, idRangeStart, limit)
if err != nil {
if cockroachutil.NeedsRetry(err) {
continue
}
return nil, err
}
break
}
return toMigrate, nil
}
// migrateGobFloatConversionRate replaces gob-encoded big.Float values for one
// specific row in the stripecoinpayments_tx_conversion_rates table with DOUBLE
// PRECISION values.
//
// This is implemented as a compare-and-swap, so that if any relevant changes
// occur on the target row since the time that we fetched it, this migration
// will not occur. Instead, wasMigrated will be returned as false, so that a
// future query can select the row for migration again if needed.
func (db *coinPaymentsTransactions) migrateGobFloatConversionRate(ctx context.Context, rateRecord conversionRateToMigrate) (wasMigrated bool, err error) {
defer mon.Task()(&ctx)(&err)
args := []interface{}{rateRecord.txID}
transactionIDArg := "$1::text"
var rateSetNewValue, rateGobOldValue, rateNumericOldValue string
if rateRecord.rateGob == nil {
rateGobOldValue = "IS NULL"
} else {
var rateBigFloat big.Float
if err = rateBigFloat.GobDecode(rateRecord.rateGob); err != nil {
return false, Error.New("invalid gob-encoded rate in stripecoinpayments_tx_conversion_rates table tx_id = %x: %w", rateRecord.txID, err)
}
rateDecimal, err := monetary.DecimalFromBigFloat(&rateBigFloat)
if err != nil {
return false, Error.New("gob-encoded rate in stripecoinpayments_tx_conversion_rates table (tx_id = %x) cannot be converted to decimal.Decimal: %s: %w", rateRecord.txID, rateBigFloat.String(), err)
}
rateFloat64, exact := rateDecimal.Float64()
if !exact {
// see comment on exactness in (*coinPaymentsTransactions).LockRate()
delta, _ := rateDecimal.Sub(decimal.NewFromFloat(rateFloat64)).Float64()
mon.FloatVal("inexact-float64-exchange-rate-delta").Observe(delta)
}
args = append(args, rateFloat64)
rateSetNewValue = fmt.Sprintf(", rate_numeric = $%d", len(args))
args = append(args, rateRecord.rateGob)
rateGobOldValue = fmt.Sprintf("= $%d::bytea", len(args))
}
if rateRecord.rateNumeric == nil {
rateNumericOldValue = "IS NULL"
} else {
args = append(args, *rateRecord.rateNumeric)
rateNumericOldValue = fmt.Sprintf("= $%d", len(args))
}
updateQuery := fmt.Sprintf(`
UPDATE stripecoinpayments_tx_conversion_rates
SET rate_gob = NULL%s
WHERE tx_id = %s
AND rate_gob %s
AND rate_numeric %s
`,
rateSetNewValue,
transactionIDArg,
rateGobOldValue,
rateNumericOldValue,
)
result, err := db.db.ExecContext(ctx, updateQuery, args...)
if err != nil {
return false, Error.New("failed to update stripecoinpayments_tx_conversion_rates row %x: %w", rateRecord.txID, err)
}
// if zero rows were updated, then the row with this ID was changed by
// something else before this migration got to it. we'll want to try
// again with the next read query.
numAffected, err := result.RowsAffected()
if err != nil {
return false, Error.New("could not get number of rows affected: %w", err)
}
return numAffected == 1, nil
}
// migrateGobFloatConversionRateWithRetry calls migrateGobFloatConversionRate
// in a loop until it succeeds without any "retry needed" error being returned.
func (db *coinPaymentsTransactions) migrateGobFloatConversionRateWithRetry(ctx context.Context, rateRecord conversionRateToMigrate) (wasMigrated bool, err error) {
defer mon.Task()(&ctx)(&err)
for {
wasMigrated, err = db.migrateGobFloatConversionRate(ctx, rateRecord)
if err != nil {
if cockroachutil.NeedsRetry(err) {
continue
}
}
return wasMigrated, err
}
}
// MigrateGobFloatConversionRateRecords is a strictly-temporary task that will,
// with time, eliminate gob-encoded big.Float records from the
// stripecoinpayments_tx_conversion_rates table. It should be called repeatedly,
// passing back nextRangeStart for the next rangeStart parameter, until it
// encounters an error or returns nextRangeStart = "".
func (db *coinPaymentsTransactions) MigrateGobFloatConversionRateRecords(ctx context.Context, rangeStart string, limit int) (migrated int, nextRangeStart string, err error) {
defer mon.Task()(&ctx)(&err)
ratesToMigrate, err := db.getConversionRatesToMigrateWithRetry(ctx, rangeStart, limit)
if err != nil {
// some sort of internal error or programming error
return 0, "", err
}
if len(ratesToMigrate) == 0 {
// all rows are migrated!
return 0, "", nil
}
for _, rateRecord := range ratesToMigrate {
wasMigrated, err := db.migrateGobFloatConversionRateWithRetry(ctx, rateRecord)
if err != nil {
// some sort of internal error or programming error
return migrated, "", err
}
if wasMigrated {
migrated++
} else if nextRangeStart == "" {
// Start here with the next call so that we can try again
// (unless we are already going to start at an earlier point)
nextRangeStart = rateRecord.txID
}
}
// if nextRangeStart != "", then we need to redo some rows, and it's already
// set appropriately.
if nextRangeStart == "" {
// if len(ratesToMigrate) < limit, then this is the last range and we've
// completed the migration (leave nextRangeStart as "").
if len(ratesToMigrate) == limit {
// next time we can start after the last ID we just processed
nextRangeStart = ratesToMigrate[len(ratesToMigrate)-1].txID
}
}
return migrated, nextRangeStart, nil
}
// MonetaryAmountToGobEncodedBigFloat converts a monetary.Amount to a gob-encoded
// big.Float.
func MonetaryAmountToGobEncodedBigFloat(amount monetary.Amount) ([]byte, error) {
asString := amount.AsDecimal().String()
asBigFloat, ok := big.NewFloat(0).SetString(asString)
if !ok {
return nil, Error.New("failed to assign %q to a big.Float", asString)
}
gobEncoded, err := asBigFloat.GobEncode()
if err != nil {
return nil, Error.Wrap(err)
}
return gobEncoded, nil
}
func (db *coinPaymentsTransactions) ForTestingOnlyInsertGobTransaction(ctx context.Context, tx stripecoinpayments.Transaction) (createdAt time.Time, err error) {
amountGob, err := MonetaryAmountToGobEncodedBigFloat(tx.Amount)
if err != nil {
return time.Time{}, err
}
receivedGob, err := MonetaryAmountToGobEncodedBigFloat(tx.Received)
if err != nil {
return time.Time{}, err
}
record, err := db.db.Create_CoinpaymentsTransaction(ctx,
dbx.CoinpaymentsTransaction_Id(tx.ID.String()),
dbx.CoinpaymentsTransaction_UserId(tx.AccountID[:]),
dbx.CoinpaymentsTransaction_Address(tx.Address),
dbx.CoinpaymentsTransaction_Status(tx.Status.Int()),
dbx.CoinpaymentsTransaction_Key(tx.Key),
dbx.CoinpaymentsTransaction_Timeout(int(tx.Timeout.Seconds())),
dbx.CoinpaymentsTransaction_Create_Fields{
AmountGob: dbx.CoinpaymentsTransaction_AmountGob(amountGob),
ReceivedGob: dbx.CoinpaymentsTransaction_ReceivedGob(receivedGob),
})
return record.CreatedAt, Error.Wrap(err)
}
func (db *coinPaymentsTransactions) ForTestingOnlyInsertGobConversionRate(ctx context.Context, txID coinpayments.TransactionID, rate decimal.Decimal) error {
gobEncoded, err := rate.BigFloat().GobEncode()
if err != nil {
return Error.Wrap(err)
}
_, err = db.db.Create_StripecoinpaymentsTxConversionRate(ctx,
dbx.StripecoinpaymentsTxConversionRate_TxId(txID.String()),
dbx.StripecoinpaymentsTxConversionRate_Create_Fields{
RateGob: dbx.StripecoinpaymentsTxConversionRate_RateGob(gobEncoded),
})
return Error.Wrap(err)
}
func (db *coinPaymentsTransactions) ForTestingOnlyGetDBHandle() *dbx.DB {
return db.db.DB
}

View File

@ -1,148 +0,0 @@
// Copyright (C) 2021 Storj Labs, Inc.
// See LICENSE for copying information.
package satellitedb_test
import (
"context"
"fmt"
"sync"
"testing"
"time"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"golang.org/x/sync/errgroup"
"storj.io/common/testcontext"
"storj.io/common/testrand"
"storj.io/storj/satellite"
"storj.io/storj/satellite/payments/coinpayments"
"storj.io/storj/satellite/payments/monetary"
"storj.io/storj/satellite/payments/stripecoinpayments"
"storj.io/storj/satellite/satellitedb"
"storj.io/storj/satellite/satellitedb/dbx"
"storj.io/storj/satellite/satellitedb/satellitedbtest"
)
func TestGobFloatMigrationCompareAndSwapBehavior(t *testing.T) {
satellitedbtest.Run(t, func(ctx *testcontext.Context, t *testing.T, db satellite.DB) {
transactionsDB := db.StripeCoinPayments().Transactions()
testTransactions, ok := transactionsDB.(interface {
ForTestingOnlyInsertGobTransaction(ctx context.Context, tx stripecoinpayments.Transaction) (time.Time, error)
ForTestingOnlyGetDBHandle() *dbx.DB
})
require.Truef(t, ok, "db object of type %T s not a *coinPaymentsTransactions", transactionsDB)
// make some random records, insert in db
const numRecords = 100
asInserted := make([]stripecoinpayments.Transaction, 0, numRecords)
for x := 0; x < numRecords; x++ {
tx := stripecoinpayments.Transaction{
ID: coinpayments.TransactionID(fmt.Sprintf("transaction%05d", x)),
Status: coinpayments.Status(x % 2), // 0 (pending) or 1 (received)
AccountID: testrand.UUID(),
Amount: monetary.AmountFromBaseUnits(testrand.Int63n(1e15), monetary.StorjToken),
Received: monetary.AmountFromBaseUnits(testrand.Int63n(1e15), monetary.StorjToken),
Address: fmt.Sprintf("%x", testrand.Bytes(20)),
Key: fmt.Sprintf("%x", testrand.Bytes(20)),
}
createTime, err := testTransactions.ForTestingOnlyInsertGobTransaction(ctx, tx)
require.NoError(t, err)
tx.CreatedAt = createTime
asInserted = append(asInserted, tx)
}
// In multiple goroutines, try to change one particular record
// in the db as fast as we can, while we are trying to migrate
// that record at the same time. This should (at least
// sometimes) cause the migration to be retried, because the
// underlying value changed.
var (
amountStoredValue = asInserted[0].Amount
valueMutex sync.Mutex
testDoneYet = false
testDoneYetMutex sync.Mutex
group errgroup.Group
dbHandle = testTransactions.ForTestingOnlyGetDBHandle()
)
group.Go(func() error {
for {
testDoneYetMutex.Lock()
areWeDone := testDoneYet
testDoneYetMutex.Unlock()
if areWeDone {
break
}
newAmount := monetary.AmountFromBaseUnits(testrand.Int63n(1e15), monetary.StorjToken)
newAmountGob, err := satellitedb.MonetaryAmountToGobEncodedBigFloat(newAmount)
if err != nil {
return err
}
result, err := dbHandle.ExecContext(ctx, `
UPDATE coinpayments_transactions
SET amount_gob = $1
WHERE id = $2 AND amount_gob IS NOT NULL
`, newAmountGob, asInserted[0].ID)
if err != nil {
return satellitedb.Error.New("could not set amount_gob: %w", err)
}
rowsAffected, err := result.RowsAffected()
if err != nil {
return satellitedb.Error.New("could not get rows affected: %w", err)
}
if rowsAffected < 1 {
// migration must have happened!
break
}
valueMutex.Lock()
amountStoredValue = newAmount
valueMutex.Unlock()
}
return nil
})
totalMigrated := 0
for {
numMigrated, nextRangeStart, err := transactionsDB.MigrateGobFloatTransactionRecords(ctx, "", numRecords+1)
totalMigrated += numMigrated
require.NoError(t, err)
if nextRangeStart == "" {
break
}
}
assert.Equal(t, numRecords, totalMigrated)
testDoneYetMutex.Lock()
testDoneYet = true
testDoneYetMutex.Unlock()
err := group.Wait()
require.NoError(t, err)
// the final value as changed by the changer goroutine
valueMutex.Lock()
finalValue := amountStoredValue
valueMutex.Unlock()
// fetch the numeric value (as migrated) from the db
row := dbHandle.QueryRowContext(ctx, `
SELECT amount_gob, amount_numeric
FROM coinpayments_transactions
WHERE id = $1
`, asInserted[0].ID)
var amountGob []byte
var amountNumeric int64
err = row.Scan(&amountGob, &amountNumeric)
require.NoError(t, err)
assert.Nil(t, amountGob)
amountFromDB := monetary.AmountFromBaseUnits(amountNumeric, monetary.StorjToken)
assert.Truef(t, finalValue.Equal(amountFromDB), "%s != %s", finalValue, amountFromDB)
})
}

View File

@ -5,7 +5,6 @@ package satellitedb
import (
"context"
"math/big"
"time"
"github.com/shopspring/decimal"
@ -51,13 +50,11 @@ func (db *coinPaymentsTransactions) Insert(ctx context.Context, tx stripecoinpay
dbx.CoinpaymentsTransaction_Id(tx.ID.String()),
dbx.CoinpaymentsTransaction_UserId(tx.AccountID[:]),
dbx.CoinpaymentsTransaction_Address(tx.Address),
dbx.CoinpaymentsTransaction_AmountNumeric(tx.Amount.BaseUnits()),
dbx.CoinpaymentsTransaction_ReceivedNumeric(tx.Received.BaseUnits()),
dbx.CoinpaymentsTransaction_Status(tx.Status.Int()),
dbx.CoinpaymentsTransaction_Key(tx.Key),
dbx.CoinpaymentsTransaction_Timeout(int(tx.Timeout.Seconds())),
dbx.CoinpaymentsTransaction_Create_Fields{
AmountNumeric: dbx.CoinpaymentsTransaction_AmountNumeric(tx.Amount.BaseUnits()),
ReceivedNumeric: dbx.CoinpaymentsTransaction_ReceivedNumeric(tx.Received.BaseUnits()),
},
)
if err != nil {
return time.Time{}, err
@ -79,7 +76,6 @@ func (db *coinPaymentsTransactions) Update(ctx context.Context, updates []stripe
dbx.CoinpaymentsTransaction_Id(update.TransactionID.String()),
dbx.CoinpaymentsTransaction_Update_Fields{
ReceivedNumeric: dbx.CoinpaymentsTransaction_ReceivedNumeric(update.Received.BaseUnits()),
ReceivedGob: dbx.CoinpaymentsTransaction_ReceivedGob_Null(),
Status: dbx.CoinpaymentsTransaction_Status(update.Status.Int()),
},
)
@ -173,9 +169,8 @@ func (db *coinPaymentsTransactions) LockRate(ctx context.Context, id coinpayment
_, err = db.db.Create_StripecoinpaymentsTxConversionRate(ctx,
dbx.StripecoinpaymentsTxConversionRate_TxId(id.String()),
dbx.StripecoinpaymentsTxConversionRate_Create_Fields{
RateNumeric: dbx.StripecoinpaymentsTxConversionRate_RateNumeric(rateFloat),
})
dbx.StripecoinpaymentsTxConversionRate_RateNumeric(rateFloat),
)
return Error.Wrap(err)
}
@ -190,17 +185,7 @@ func (db *coinPaymentsTransactions) GetLockedRate(ctx context.Context, id coinpa
return decimal.Decimal{}, err
}
if dbxRate.RateNumeric == nil {
// This row does not have a numeric rate value yet
var rateF big.Float
if err = rateF.GobDecode(dbxRate.RateGob); err != nil {
return decimal.Decimal{}, Error.Wrap(err)
}
rate, err = monetary.DecimalFromBigFloat(&rateF)
return rate, Error.Wrap(err)
}
rate = decimal.NewFromFloat(*dbxRate.RateNumeric)
rate = decimal.NewFromFloat(dbxRate.RateNumeric)
return rate, nil
}
@ -236,9 +221,7 @@ func (db *coinPaymentsTransactions) ListPending(ctx context.Context, offset int6
id,
user_id,
address,
amount_gob,
amount_numeric,
received_gob,
received_numeric,
status,
key,
@ -263,14 +246,12 @@ func (db *coinPaymentsTransactions) ListPending(ctx context.Context, offset int6
for rows.Next() {
var id, address string
var userID uuid.UUID
var amountGob, receivedGob []byte
var amountNumeric, receivedNumeric *int64
var amount, received monetary.Amount
var amount, received *int64
var status int
var key string
var createdAt time.Time
err := rows.Scan(&id, &userID, &address, &amountGob, &amountNumeric, &receivedGob, &receivedNumeric, &status, &key, &createdAt)
err := rows.Scan(&id, &userID, &address, &amount, &received, &status, &key, &createdAt)
if err != nil {
return stripecoinpayments.TransactionsPage{}, Error.Wrap(err)
}
@ -279,32 +260,13 @@ func (db *coinPaymentsTransactions) ListPending(ctx context.Context, offset int6
// in the database.
currency := monetary.StorjToken
if amountNumeric == nil {
// 'amount' in this row hasn't yet been updated to a numeric value
amount, err = monetaryAmountFromGobEncodedBigFloat(amountGob, currency)
if err != nil {
return stripecoinpayments.TransactionsPage{}, Error.New("invalid gob encoding in amount_gob under transaction id %x: %v", id, err)
}
} else {
amount = monetary.AmountFromBaseUnits(*amountNumeric, currency)
}
if receivedNumeric == nil {
// 'received' in this row hasn't yet been updated to a numeric value
received, err = monetaryAmountFromGobEncodedBigFloat(receivedGob, currency)
if err != nil {
return stripecoinpayments.TransactionsPage{}, Error.New("invalid gob encoding in received_gob under transaction id %x: %v", id, err)
}
} else {
received = monetary.AmountFromBaseUnits(*receivedNumeric, currency)
}
page.Transactions = append(page.Transactions,
stripecoinpayments.Transaction{
ID: coinpayments.TransactionID(id),
AccountID: userID,
Address: address,
Amount: amount,
Received: received,
Amount: monetary.AmountFromBaseUnits(*amount, currency),
Received: monetary.AmountFromBaseUnits(*received, currency),
Status: coinpayments.Status(status),
Key: key,
CreatedAt: createdAt,
@ -333,9 +295,7 @@ func (db *coinPaymentsTransactions) ListUnapplied(ctx context.Context, offset in
txs.id,
txs.user_id,
txs.address,
txs.amount_gob,
txs.amount_numeric,
txs.received_gob,
txs.received_numeric,
txs.status,
txs.key,
@ -360,13 +320,12 @@ func (db *coinPaymentsTransactions) ListUnapplied(ctx context.Context, offset in
for rows.Next() {
var id, address string
var userID uuid.UUID
var amountGob, receivedGob []byte
var amountNumeric, receivedNumeric *int64
var status int
var key string
var createdAt time.Time
err := rows.Scan(&id, &userID, &address, &amountGob, &amountNumeric, &receivedGob, &receivedNumeric, &status, &key, &createdAt)
err := rows.Scan(&id, &userID, &address, &amountNumeric, &receivedNumeric, &status, &key, &createdAt)
if err != nil {
return stripecoinpayments.TransactionsPage{}, err
}
@ -375,33 +334,13 @@ func (db *coinPaymentsTransactions) ListUnapplied(ctx context.Context, offset in
// in the database.
currency := monetary.StorjToken
var amount, received monetary.Amount
if amountNumeric == nil {
// 'amount' in this row hasn't yet been updated to a numeric value
amount, err = monetaryAmountFromGobEncodedBigFloat(amountGob, currency)
if err != nil {
return stripecoinpayments.TransactionsPage{}, Error.New("invalid gob encoding in amount_gob under transaction id %x: %v", id, err)
}
} else {
amount = monetary.AmountFromBaseUnits(*amountNumeric, currency)
}
if receivedNumeric == nil {
// 'received' in this row hasn't yet been updated to a numeric value
received, err = monetaryAmountFromGobEncodedBigFloat(receivedGob, currency)
if err != nil {
return stripecoinpayments.TransactionsPage{}, Error.New("invalid gob encoding in received_gob under transaction id %x: %v", id, err)
}
} else {
received = monetary.AmountFromBaseUnits(*receivedNumeric, currency)
}
page.Transactions = append(page.Transactions,
stripecoinpayments.Transaction{
ID: coinpayments.TransactionID(id),
AccountID: userID,
Address: address,
Amount: amount,
Received: received,
Amount: monetary.AmountFromBaseUnits(*amountNumeric, currency),
Received: monetary.AmountFromBaseUnits(*receivedNumeric, currency),
Status: coinpayments.Status(status),
Key: key,
CreatedAt: createdAt,
@ -433,42 +372,15 @@ func fromDBXCoinpaymentsTransaction(dbxCPTX *dbx.CoinpaymentsTransaction) (*stri
// in the database.
currency := monetary.StorjToken
var amount, received monetary.Amount
if dbxCPTX.AmountNumeric == nil {
amount, err = monetaryAmountFromGobEncodedBigFloat(dbxCPTX.AmountGob, currency)
if err != nil {
return nil, Error.New("amount column: %v", err)
}
} else {
amount = monetary.AmountFromBaseUnits(*dbxCPTX.AmountNumeric, currency)
}
if dbxCPTX.ReceivedNumeric == nil {
received, err = monetaryAmountFromGobEncodedBigFloat(dbxCPTX.ReceivedGob, currency)
if err != nil {
return nil, Error.New("received column: %v", err)
}
} else {
received = monetary.AmountFromBaseUnits(*dbxCPTX.ReceivedNumeric, currency)
}
return &stripecoinpayments.Transaction{
ID: coinpayments.TransactionID(dbxCPTX.Id),
AccountID: userID,
Address: dbxCPTX.Address,
Amount: amount,
Received: received,
Amount: monetary.AmountFromBaseUnits(dbxCPTX.AmountNumeric, currency),
Received: monetary.AmountFromBaseUnits(dbxCPTX.ReceivedNumeric, currency),
Status: coinpayments.Status(dbxCPTX.Status),
Key: dbxCPTX.Key,
Timeout: time.Second * time.Duration(dbxCPTX.Timeout),
CreatedAt: dbxCPTX.CreatedAt,
}, nil
}
func monetaryAmountFromGobEncodedBigFloat(encoded []byte, currency *monetary.Currency) (_ monetary.Amount, err error) {
var bf big.Float
if err := bf.GobDecode(encoded); err != nil {
return monetary.Amount{}, Error.Wrap(err)
}
return monetary.AmountFromBigFloat(&bf, currency)
}

View File

@ -1110,10 +1110,8 @@ model coinpayments_transaction (
field id text
field user_id blob
field address text
field amount_gob blob ( nullable, updatable )
field amount_numeric int64 ( nullable, updatable )
field received_gob blob ( nullable, updatable )
field received_numeric int64 ( nullable, updatable )
field amount_numeric int64
field received_numeric int64 ( updatable )
field status int ( updatable )
field key text
field timeout int
@ -1178,8 +1176,7 @@ model stripecoinpayments_tx_conversion_rate (
key tx_id
field tx_id text
field rate_gob blob ( nullable )
field rate_numeric float64 ( nullable )
field rate_numeric float64
field created_at timestamp ( autoinsert )
)

View File

@ -376,10 +376,8 @@ CREATE TABLE coinpayments_transactions (
id text NOT NULL,
user_id bytea NOT NULL,
address text NOT NULL,
amount_gob bytea,
amount_numeric bigint,
received_gob bytea,
received_numeric bigint,
amount_numeric bigint NOT NULL,
received_numeric bigint NOT NULL,
status integer NOT NULL,
key text NOT NULL,
timeout integer NOT NULL,
@ -718,8 +716,7 @@ CREATE TABLE stripecoinpayments_invoice_project_records (
);
CREATE TABLE stripecoinpayments_tx_conversion_rates (
tx_id text NOT NULL,
rate_gob bytea,
rate_numeric double precision,
rate_numeric double precision NOT NULL,
created_at timestamp with time zone NOT NULL,
PRIMARY KEY ( tx_id )
);
@ -999,10 +996,8 @@ CREATE TABLE coinpayments_transactions (
id text NOT NULL,
user_id bytea NOT NULL,
address text NOT NULL,
amount_gob bytea,
amount_numeric bigint,
received_gob bytea,
received_numeric bigint,
amount_numeric bigint NOT NULL,
received_numeric bigint NOT NULL,
status integer NOT NULL,
key text NOT NULL,
timeout integer NOT NULL,
@ -1341,8 +1336,7 @@ CREATE TABLE stripecoinpayments_invoice_project_records (
);
CREATE TABLE stripecoinpayments_tx_conversion_rates (
tx_id text NOT NULL,
rate_gob bytea,
rate_numeric double precision,
rate_numeric double precision NOT NULL,
created_at timestamp with time zone NOT NULL,
PRIMARY KEY ( tx_id )
);
@ -2326,10 +2320,8 @@ type CoinpaymentsTransaction struct {
Id string
UserId []byte
Address string
AmountGob []byte
AmountNumeric *int64
ReceivedGob []byte
ReceivedNumeric *int64
AmountNumeric int64
ReceivedNumeric int64
Status int
Key string
Timeout int
@ -2338,17 +2330,7 @@ type CoinpaymentsTransaction struct {
func (CoinpaymentsTransaction) _Table() string { return "coinpayments_transactions" }
type CoinpaymentsTransaction_Create_Fields struct {
AmountGob CoinpaymentsTransaction_AmountGob_Field
AmountNumeric CoinpaymentsTransaction_AmountNumeric_Field
ReceivedGob CoinpaymentsTransaction_ReceivedGob_Field
ReceivedNumeric CoinpaymentsTransaction_ReceivedNumeric_Field
}
type CoinpaymentsTransaction_Update_Fields struct {
AmountGob CoinpaymentsTransaction_AmountGob_Field
AmountNumeric CoinpaymentsTransaction_AmountNumeric_Field
ReceivedGob CoinpaymentsTransaction_ReceivedGob_Field
ReceivedNumeric CoinpaymentsTransaction_ReceivedNumeric_Field
Status CoinpaymentsTransaction_Status_Field
}
@ -2410,63 +2392,14 @@ func (f CoinpaymentsTransaction_Address_Field) value() interface{} {
func (CoinpaymentsTransaction_Address_Field) _Column() string { return "address" }
type CoinpaymentsTransaction_AmountGob_Field struct {
_set bool
_null bool
_value []byte
}
func CoinpaymentsTransaction_AmountGob(v []byte) CoinpaymentsTransaction_AmountGob_Field {
return CoinpaymentsTransaction_AmountGob_Field{_set: true, _value: v}
}
func CoinpaymentsTransaction_AmountGob_Raw(v []byte) CoinpaymentsTransaction_AmountGob_Field {
if v == nil {
return CoinpaymentsTransaction_AmountGob_Null()
}
return CoinpaymentsTransaction_AmountGob(v)
}
func CoinpaymentsTransaction_AmountGob_Null() CoinpaymentsTransaction_AmountGob_Field {
return CoinpaymentsTransaction_AmountGob_Field{_set: true, _null: true}
}
func (f CoinpaymentsTransaction_AmountGob_Field) isnull() bool {
return !f._set || f._null || f._value == nil
}
func (f CoinpaymentsTransaction_AmountGob_Field) value() interface{} {
if !f._set || f._null {
return nil
}
return f._value
}
func (CoinpaymentsTransaction_AmountGob_Field) _Column() string { return "amount_gob" }
type CoinpaymentsTransaction_AmountNumeric_Field struct {
_set bool
_null bool
_value *int64
_value int64
}
func CoinpaymentsTransaction_AmountNumeric(v int64) CoinpaymentsTransaction_AmountNumeric_Field {
return CoinpaymentsTransaction_AmountNumeric_Field{_set: true, _value: &v}
}
func CoinpaymentsTransaction_AmountNumeric_Raw(v *int64) CoinpaymentsTransaction_AmountNumeric_Field {
if v == nil {
return CoinpaymentsTransaction_AmountNumeric_Null()
}
return CoinpaymentsTransaction_AmountNumeric(*v)
}
func CoinpaymentsTransaction_AmountNumeric_Null() CoinpaymentsTransaction_AmountNumeric_Field {
return CoinpaymentsTransaction_AmountNumeric_Field{_set: true, _null: true}
}
func (f CoinpaymentsTransaction_AmountNumeric_Field) isnull() bool {
return !f._set || f._null || f._value == nil
return CoinpaymentsTransaction_AmountNumeric_Field{_set: true, _value: v}
}
func (f CoinpaymentsTransaction_AmountNumeric_Field) value() interface{} {
@ -2478,63 +2411,14 @@ func (f CoinpaymentsTransaction_AmountNumeric_Field) value() interface{} {
func (CoinpaymentsTransaction_AmountNumeric_Field) _Column() string { return "amount_numeric" }
type CoinpaymentsTransaction_ReceivedGob_Field struct {
_set bool
_null bool
_value []byte
}
func CoinpaymentsTransaction_ReceivedGob(v []byte) CoinpaymentsTransaction_ReceivedGob_Field {
return CoinpaymentsTransaction_ReceivedGob_Field{_set: true, _value: v}
}
func CoinpaymentsTransaction_ReceivedGob_Raw(v []byte) CoinpaymentsTransaction_ReceivedGob_Field {
if v == nil {
return CoinpaymentsTransaction_ReceivedGob_Null()
}
return CoinpaymentsTransaction_ReceivedGob(v)
}
func CoinpaymentsTransaction_ReceivedGob_Null() CoinpaymentsTransaction_ReceivedGob_Field {
return CoinpaymentsTransaction_ReceivedGob_Field{_set: true, _null: true}
}
func (f CoinpaymentsTransaction_ReceivedGob_Field) isnull() bool {
return !f._set || f._null || f._value == nil
}
func (f CoinpaymentsTransaction_ReceivedGob_Field) value() interface{} {
if !f._set || f._null {
return nil
}
return f._value
}
func (CoinpaymentsTransaction_ReceivedGob_Field) _Column() string { return "received_gob" }
type CoinpaymentsTransaction_ReceivedNumeric_Field struct {
_set bool
_null bool
_value *int64
_value int64
}
func CoinpaymentsTransaction_ReceivedNumeric(v int64) CoinpaymentsTransaction_ReceivedNumeric_Field {
return CoinpaymentsTransaction_ReceivedNumeric_Field{_set: true, _value: &v}
}
func CoinpaymentsTransaction_ReceivedNumeric_Raw(v *int64) CoinpaymentsTransaction_ReceivedNumeric_Field {
if v == nil {
return CoinpaymentsTransaction_ReceivedNumeric_Null()
}
return CoinpaymentsTransaction_ReceivedNumeric(*v)
}
func CoinpaymentsTransaction_ReceivedNumeric_Null() CoinpaymentsTransaction_ReceivedNumeric_Field {
return CoinpaymentsTransaction_ReceivedNumeric_Field{_set: true, _null: true}
}
func (f CoinpaymentsTransaction_ReceivedNumeric_Field) isnull() bool {
return !f._set || f._null || f._value == nil
return CoinpaymentsTransaction_ReceivedNumeric_Field{_set: true, _value: v}
}
func (f CoinpaymentsTransaction_ReceivedNumeric_Field) value() interface{} {
@ -8491,8 +8375,7 @@ func (StripecoinpaymentsInvoiceProjectRecord_CreatedAt_Field) _Column() string {
type StripecoinpaymentsTxConversionRate struct {
TxId string
RateGob []byte
RateNumeric *float64
RateNumeric float64
CreatedAt time.Time
}
@ -8500,11 +8383,6 @@ func (StripecoinpaymentsTxConversionRate) _Table() string {
return "stripecoinpayments_tx_conversion_rates"
}
type StripecoinpaymentsTxConversionRate_Create_Fields struct {
RateGob StripecoinpaymentsTxConversionRate_RateGob_Field
RateNumeric StripecoinpaymentsTxConversionRate_RateNumeric_Field
}
type StripecoinpaymentsTxConversionRate_Update_Fields struct {
}
@ -8527,63 +8405,14 @@ func (f StripecoinpaymentsTxConversionRate_TxId_Field) value() interface{} {
func (StripecoinpaymentsTxConversionRate_TxId_Field) _Column() string { return "tx_id" }
type StripecoinpaymentsTxConversionRate_RateGob_Field struct {
_set bool
_null bool
_value []byte
}
func StripecoinpaymentsTxConversionRate_RateGob(v []byte) StripecoinpaymentsTxConversionRate_RateGob_Field {
return StripecoinpaymentsTxConversionRate_RateGob_Field{_set: true, _value: v}
}
func StripecoinpaymentsTxConversionRate_RateGob_Raw(v []byte) StripecoinpaymentsTxConversionRate_RateGob_Field {
if v == nil {
return StripecoinpaymentsTxConversionRate_RateGob_Null()
}
return StripecoinpaymentsTxConversionRate_RateGob(v)
}
func StripecoinpaymentsTxConversionRate_RateGob_Null() StripecoinpaymentsTxConversionRate_RateGob_Field {
return StripecoinpaymentsTxConversionRate_RateGob_Field{_set: true, _null: true}
}
func (f StripecoinpaymentsTxConversionRate_RateGob_Field) isnull() bool {
return !f._set || f._null || f._value == nil
}
func (f StripecoinpaymentsTxConversionRate_RateGob_Field) value() interface{} {
if !f._set || f._null {
return nil
}
return f._value
}
func (StripecoinpaymentsTxConversionRate_RateGob_Field) _Column() string { return "rate_gob" }
type StripecoinpaymentsTxConversionRate_RateNumeric_Field struct {
_set bool
_null bool
_value *float64
_value float64
}
func StripecoinpaymentsTxConversionRate_RateNumeric(v float64) StripecoinpaymentsTxConversionRate_RateNumeric_Field {
return StripecoinpaymentsTxConversionRate_RateNumeric_Field{_set: true, _value: &v}
}
func StripecoinpaymentsTxConversionRate_RateNumeric_Raw(v *float64) StripecoinpaymentsTxConversionRate_RateNumeric_Field {
if v == nil {
return StripecoinpaymentsTxConversionRate_RateNumeric_Null()
}
return StripecoinpaymentsTxConversionRate_RateNumeric(*v)
}
func StripecoinpaymentsTxConversionRate_RateNumeric_Null() StripecoinpaymentsTxConversionRate_RateNumeric_Field {
return StripecoinpaymentsTxConversionRate_RateNumeric_Field{_set: true, _null: true}
}
func (f StripecoinpaymentsTxConversionRate_RateNumeric_Field) isnull() bool {
return !f._set || f._null || f._value == nil
return StripecoinpaymentsTxConversionRate_RateNumeric_Field{_set: true, _value: v}
}
func (f StripecoinpaymentsTxConversionRate_RateNumeric_Field) value() interface{} {
@ -11720,10 +11549,11 @@ func (obj *pgxImpl) Create_CoinpaymentsTransaction(ctx context.Context,
coinpayments_transaction_id CoinpaymentsTransaction_Id_Field,
coinpayments_transaction_user_id CoinpaymentsTransaction_UserId_Field,
coinpayments_transaction_address CoinpaymentsTransaction_Address_Field,
coinpayments_transaction_amount_numeric CoinpaymentsTransaction_AmountNumeric_Field,
coinpayments_transaction_received_numeric CoinpaymentsTransaction_ReceivedNumeric_Field,
coinpayments_transaction_status CoinpaymentsTransaction_Status_Field,
coinpayments_transaction_key CoinpaymentsTransaction_Key_Field,
coinpayments_transaction_timeout CoinpaymentsTransaction_Timeout_Field,
optional CoinpaymentsTransaction_Create_Fields) (
coinpayments_transaction_timeout CoinpaymentsTransaction_Timeout_Field) (
coinpayments_transaction *CoinpaymentsTransaction, err error) {
defer mon.Task()(&ctx)(&err)
@ -11731,25 +11561,23 @@ func (obj *pgxImpl) Create_CoinpaymentsTransaction(ctx context.Context,
__id_val := coinpayments_transaction_id.value()
__user_id_val := coinpayments_transaction_user_id.value()
__address_val := coinpayments_transaction_address.value()
__amount_gob_val := optional.AmountGob.value()
__amount_numeric_val := optional.AmountNumeric.value()
__received_gob_val := optional.ReceivedGob.value()
__received_numeric_val := optional.ReceivedNumeric.value()
__amount_numeric_val := coinpayments_transaction_amount_numeric.value()
__received_numeric_val := coinpayments_transaction_received_numeric.value()
__status_val := coinpayments_transaction_status.value()
__key_val := coinpayments_transaction_key.value()
__timeout_val := coinpayments_transaction_timeout.value()
__created_at_val := __now
var __embed_stmt = __sqlbundle_Literal("INSERT INTO coinpayments_transactions ( id, user_id, address, amount_gob, amount_numeric, received_gob, received_numeric, status, key, timeout, created_at ) VALUES ( ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ? ) RETURNING coinpayments_transactions.id, coinpayments_transactions.user_id, coinpayments_transactions.address, coinpayments_transactions.amount_gob, coinpayments_transactions.amount_numeric, coinpayments_transactions.received_gob, coinpayments_transactions.received_numeric, coinpayments_transactions.status, coinpayments_transactions.key, coinpayments_transactions.timeout, coinpayments_transactions.created_at")
var __embed_stmt = __sqlbundle_Literal("INSERT INTO coinpayments_transactions ( id, user_id, address, amount_numeric, received_numeric, status, key, timeout, created_at ) VALUES ( ?, ?, ?, ?, ?, ?, ?, ?, ? ) RETURNING coinpayments_transactions.id, coinpayments_transactions.user_id, coinpayments_transactions.address, coinpayments_transactions.amount_numeric, coinpayments_transactions.received_numeric, coinpayments_transactions.status, coinpayments_transactions.key, coinpayments_transactions.timeout, coinpayments_transactions.created_at")
var __values []interface{}
__values = append(__values, __id_val, __user_id_val, __address_val, __amount_gob_val, __amount_numeric_val, __received_gob_val, __received_numeric_val, __status_val, __key_val, __timeout_val, __created_at_val)
__values = append(__values, __id_val, __user_id_val, __address_val, __amount_numeric_val, __received_numeric_val, __status_val, __key_val, __timeout_val, __created_at_val)
var __stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
obj.logStmt(__stmt, __values...)
coinpayments_transaction = &CoinpaymentsTransaction{}
err = obj.queryRowContext(ctx, __stmt, __values...).Scan(&coinpayments_transaction.Id, &coinpayments_transaction.UserId, &coinpayments_transaction.Address, &coinpayments_transaction.AmountGob, &coinpayments_transaction.AmountNumeric, &coinpayments_transaction.ReceivedGob, &coinpayments_transaction.ReceivedNumeric, &coinpayments_transaction.Status, &coinpayments_transaction.Key, &coinpayments_transaction.Timeout, &coinpayments_transaction.CreatedAt)
err = obj.queryRowContext(ctx, __stmt, __values...).Scan(&coinpayments_transaction.Id, &coinpayments_transaction.UserId, &coinpayments_transaction.Address, &coinpayments_transaction.AmountNumeric, &coinpayments_transaction.ReceivedNumeric, &coinpayments_transaction.Status, &coinpayments_transaction.Key, &coinpayments_transaction.Timeout, &coinpayments_transaction.CreatedAt)
if err != nil {
return nil, obj.makeErr(err)
}
@ -11800,26 +11628,25 @@ func (obj *pgxImpl) Create_StripecoinpaymentsInvoiceProjectRecord(ctx context.Co
func (obj *pgxImpl) Create_StripecoinpaymentsTxConversionRate(ctx context.Context,
stripecoinpayments_tx_conversion_rate_tx_id StripecoinpaymentsTxConversionRate_TxId_Field,
optional StripecoinpaymentsTxConversionRate_Create_Fields) (
stripecoinpayments_tx_conversion_rate_rate_numeric StripecoinpaymentsTxConversionRate_RateNumeric_Field) (
stripecoinpayments_tx_conversion_rate *StripecoinpaymentsTxConversionRate, err error) {
defer mon.Task()(&ctx)(&err)
__now := obj.db.Hooks.Now().UTC()
__tx_id_val := stripecoinpayments_tx_conversion_rate_tx_id.value()
__rate_gob_val := optional.RateGob.value()
__rate_numeric_val := optional.RateNumeric.value()
__rate_numeric_val := stripecoinpayments_tx_conversion_rate_rate_numeric.value()
__created_at_val := __now
var __embed_stmt = __sqlbundle_Literal("INSERT INTO stripecoinpayments_tx_conversion_rates ( tx_id, rate_gob, rate_numeric, created_at ) VALUES ( ?, ?, ?, ? ) RETURNING stripecoinpayments_tx_conversion_rates.tx_id, stripecoinpayments_tx_conversion_rates.rate_gob, stripecoinpayments_tx_conversion_rates.rate_numeric, stripecoinpayments_tx_conversion_rates.created_at")
var __embed_stmt = __sqlbundle_Literal("INSERT INTO stripecoinpayments_tx_conversion_rates ( tx_id, rate_numeric, created_at ) VALUES ( ?, ?, ? ) RETURNING stripecoinpayments_tx_conversion_rates.tx_id, stripecoinpayments_tx_conversion_rates.rate_numeric, stripecoinpayments_tx_conversion_rates.created_at")
var __values []interface{}
__values = append(__values, __tx_id_val, __rate_gob_val, __rate_numeric_val, __created_at_val)
__values = append(__values, __tx_id_val, __rate_numeric_val, __created_at_val)
var __stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
obj.logStmt(__stmt, __values...)
stripecoinpayments_tx_conversion_rate = &StripecoinpaymentsTxConversionRate{}
err = obj.queryRowContext(ctx, __stmt, __values...).Scan(&stripecoinpayments_tx_conversion_rate.TxId, &stripecoinpayments_tx_conversion_rate.RateGob, &stripecoinpayments_tx_conversion_rate.RateNumeric, &stripecoinpayments_tx_conversion_rate.CreatedAt)
err = obj.queryRowContext(ctx, __stmt, __values...).Scan(&stripecoinpayments_tx_conversion_rate.TxId, &stripecoinpayments_tx_conversion_rate.RateNumeric, &stripecoinpayments_tx_conversion_rate.CreatedAt)
if err != nil {
return nil, obj.makeErr(err)
}
@ -14289,7 +14116,7 @@ func (obj *pgxImpl) All_CoinpaymentsTransaction_By_UserId_OrderBy_Desc_CreatedAt
rows []*CoinpaymentsTransaction, err error) {
defer mon.Task()(&ctx)(&err)
var __embed_stmt = __sqlbundle_Literal("SELECT coinpayments_transactions.id, coinpayments_transactions.user_id, coinpayments_transactions.address, coinpayments_transactions.amount_gob, coinpayments_transactions.amount_numeric, coinpayments_transactions.received_gob, coinpayments_transactions.received_numeric, coinpayments_transactions.status, coinpayments_transactions.key, coinpayments_transactions.timeout, coinpayments_transactions.created_at FROM coinpayments_transactions WHERE coinpayments_transactions.user_id = ? ORDER BY coinpayments_transactions.created_at DESC")
var __embed_stmt = __sqlbundle_Literal("SELECT coinpayments_transactions.id, coinpayments_transactions.user_id, coinpayments_transactions.address, coinpayments_transactions.amount_numeric, coinpayments_transactions.received_numeric, coinpayments_transactions.status, coinpayments_transactions.key, coinpayments_transactions.timeout, coinpayments_transactions.created_at FROM coinpayments_transactions WHERE coinpayments_transactions.user_id = ? ORDER BY coinpayments_transactions.created_at DESC")
var __values []interface{}
__values = append(__values, coinpayments_transaction_user_id.value())
@ -14307,7 +14134,7 @@ func (obj *pgxImpl) All_CoinpaymentsTransaction_By_UserId_OrderBy_Desc_CreatedAt
for __rows.Next() {
coinpayments_transaction := &CoinpaymentsTransaction{}
err = __rows.Scan(&coinpayments_transaction.Id, &coinpayments_transaction.UserId, &coinpayments_transaction.Address, &coinpayments_transaction.AmountGob, &coinpayments_transaction.AmountNumeric, &coinpayments_transaction.ReceivedGob, &coinpayments_transaction.ReceivedNumeric, &coinpayments_transaction.Status, &coinpayments_transaction.Key, &coinpayments_transaction.Timeout, &coinpayments_transaction.CreatedAt)
err = __rows.Scan(&coinpayments_transaction.Id, &coinpayments_transaction.UserId, &coinpayments_transaction.Address, &coinpayments_transaction.AmountNumeric, &coinpayments_transaction.ReceivedNumeric, &coinpayments_transaction.Status, &coinpayments_transaction.Key, &coinpayments_transaction.Timeout, &coinpayments_transaction.CreatedAt)
if err != nil {
return nil, err
}
@ -14409,7 +14236,7 @@ func (obj *pgxImpl) Get_StripecoinpaymentsTxConversionRate_By_TxId(ctx context.C
stripecoinpayments_tx_conversion_rate *StripecoinpaymentsTxConversionRate, err error) {
defer mon.Task()(&ctx)(&err)
var __embed_stmt = __sqlbundle_Literal("SELECT stripecoinpayments_tx_conversion_rates.tx_id, stripecoinpayments_tx_conversion_rates.rate_gob, stripecoinpayments_tx_conversion_rates.rate_numeric, stripecoinpayments_tx_conversion_rates.created_at FROM stripecoinpayments_tx_conversion_rates WHERE stripecoinpayments_tx_conversion_rates.tx_id = ?")
var __embed_stmt = __sqlbundle_Literal("SELECT stripecoinpayments_tx_conversion_rates.tx_id, stripecoinpayments_tx_conversion_rates.rate_numeric, stripecoinpayments_tx_conversion_rates.created_at FROM stripecoinpayments_tx_conversion_rates WHERE stripecoinpayments_tx_conversion_rates.tx_id = ?")
var __values []interface{}
__values = append(__values, stripecoinpayments_tx_conversion_rate_tx_id.value())
@ -14418,7 +14245,7 @@ func (obj *pgxImpl) Get_StripecoinpaymentsTxConversionRate_By_TxId(ctx context.C
obj.logStmt(__stmt, __values...)
stripecoinpayments_tx_conversion_rate = &StripecoinpaymentsTxConversionRate{}
err = obj.queryRowContext(ctx, __stmt, __values...).Scan(&stripecoinpayments_tx_conversion_rate.TxId, &stripecoinpayments_tx_conversion_rate.RateGob, &stripecoinpayments_tx_conversion_rate.RateNumeric, &stripecoinpayments_tx_conversion_rate.CreatedAt)
err = obj.queryRowContext(ctx, __stmt, __values...).Scan(&stripecoinpayments_tx_conversion_rate.TxId, &stripecoinpayments_tx_conversion_rate.RateNumeric, &stripecoinpayments_tx_conversion_rate.CreatedAt)
if err != nil {
return (*StripecoinpaymentsTxConversionRate)(nil), obj.makeErr(err)
}
@ -16209,27 +16036,12 @@ func (obj *pgxImpl) Update_CoinpaymentsTransaction_By_Id(ctx context.Context,
defer mon.Task()(&ctx)(&err)
var __sets = &__sqlbundle_Hole{}
var __embed_stmt = __sqlbundle_Literals{Join: "", SQLs: []__sqlbundle_SQL{__sqlbundle_Literal("UPDATE coinpayments_transactions SET "), __sets, __sqlbundle_Literal(" WHERE coinpayments_transactions.id = ? RETURNING coinpayments_transactions.id, coinpayments_transactions.user_id, coinpayments_transactions.address, coinpayments_transactions.amount_gob, coinpayments_transactions.amount_numeric, coinpayments_transactions.received_gob, coinpayments_transactions.received_numeric, coinpayments_transactions.status, coinpayments_transactions.key, coinpayments_transactions.timeout, coinpayments_transactions.created_at")}}
var __embed_stmt = __sqlbundle_Literals{Join: "", SQLs: []__sqlbundle_SQL{__sqlbundle_Literal("UPDATE coinpayments_transactions SET "), __sets, __sqlbundle_Literal(" WHERE coinpayments_transactions.id = ? RETURNING coinpayments_transactions.id, coinpayments_transactions.user_id, coinpayments_transactions.address, coinpayments_transactions.amount_numeric, coinpayments_transactions.received_numeric, coinpayments_transactions.status, coinpayments_transactions.key, coinpayments_transactions.timeout, coinpayments_transactions.created_at")}}
__sets_sql := __sqlbundle_Literals{Join: ", "}
var __values []interface{}
var __args []interface{}
if update.AmountGob._set {
__values = append(__values, update.AmountGob.value())
__sets_sql.SQLs = append(__sets_sql.SQLs, __sqlbundle_Literal("amount_gob = ?"))
}
if update.AmountNumeric._set {
__values = append(__values, update.AmountNumeric.value())
__sets_sql.SQLs = append(__sets_sql.SQLs, __sqlbundle_Literal("amount_numeric = ?"))
}
if update.ReceivedGob._set {
__values = append(__values, update.ReceivedGob.value())
__sets_sql.SQLs = append(__sets_sql.SQLs, __sqlbundle_Literal("received_gob = ?"))
}
if update.ReceivedNumeric._set {
__values = append(__values, update.ReceivedNumeric.value())
__sets_sql.SQLs = append(__sets_sql.SQLs, __sqlbundle_Literal("received_numeric = ?"))
@ -16253,7 +16065,7 @@ func (obj *pgxImpl) Update_CoinpaymentsTransaction_By_Id(ctx context.Context,
obj.logStmt(__stmt, __values...)
coinpayments_transaction = &CoinpaymentsTransaction{}
err = obj.queryRowContext(ctx, __stmt, __values...).Scan(&coinpayments_transaction.Id, &coinpayments_transaction.UserId, &coinpayments_transaction.Address, &coinpayments_transaction.AmountGob, &coinpayments_transaction.AmountNumeric, &coinpayments_transaction.ReceivedGob, &coinpayments_transaction.ReceivedNumeric, &coinpayments_transaction.Status, &coinpayments_transaction.Key, &coinpayments_transaction.Timeout, &coinpayments_transaction.CreatedAt)
err = obj.queryRowContext(ctx, __stmt, __values...).Scan(&coinpayments_transaction.Id, &coinpayments_transaction.UserId, &coinpayments_transaction.Address, &coinpayments_transaction.AmountNumeric, &coinpayments_transaction.ReceivedNumeric, &coinpayments_transaction.Status, &coinpayments_transaction.Key, &coinpayments_transaction.Timeout, &coinpayments_transaction.CreatedAt)
if err == sql.ErrNoRows {
return nil, nil
}
@ -18113,10 +17925,11 @@ func (obj *pgxcockroachImpl) Create_CoinpaymentsTransaction(ctx context.Context,
coinpayments_transaction_id CoinpaymentsTransaction_Id_Field,
coinpayments_transaction_user_id CoinpaymentsTransaction_UserId_Field,
coinpayments_transaction_address CoinpaymentsTransaction_Address_Field,
coinpayments_transaction_amount_numeric CoinpaymentsTransaction_AmountNumeric_Field,
coinpayments_transaction_received_numeric CoinpaymentsTransaction_ReceivedNumeric_Field,
coinpayments_transaction_status CoinpaymentsTransaction_Status_Field,
coinpayments_transaction_key CoinpaymentsTransaction_Key_Field,
coinpayments_transaction_timeout CoinpaymentsTransaction_Timeout_Field,
optional CoinpaymentsTransaction_Create_Fields) (
coinpayments_transaction_timeout CoinpaymentsTransaction_Timeout_Field) (
coinpayments_transaction *CoinpaymentsTransaction, err error) {
defer mon.Task()(&ctx)(&err)
@ -18124,25 +17937,23 @@ func (obj *pgxcockroachImpl) Create_CoinpaymentsTransaction(ctx context.Context,
__id_val := coinpayments_transaction_id.value()
__user_id_val := coinpayments_transaction_user_id.value()
__address_val := coinpayments_transaction_address.value()
__amount_gob_val := optional.AmountGob.value()
__amount_numeric_val := optional.AmountNumeric.value()
__received_gob_val := optional.ReceivedGob.value()
__received_numeric_val := optional.ReceivedNumeric.value()
__amount_numeric_val := coinpayments_transaction_amount_numeric.value()
__received_numeric_val := coinpayments_transaction_received_numeric.value()
__status_val := coinpayments_transaction_status.value()
__key_val := coinpayments_transaction_key.value()
__timeout_val := coinpayments_transaction_timeout.value()
__created_at_val := __now
var __embed_stmt = __sqlbundle_Literal("INSERT INTO coinpayments_transactions ( id, user_id, address, amount_gob, amount_numeric, received_gob, received_numeric, status, key, timeout, created_at ) VALUES ( ?, ?, ?, ?, ?, ?, ?, ?, ?, ?, ? ) RETURNING coinpayments_transactions.id, coinpayments_transactions.user_id, coinpayments_transactions.address, coinpayments_transactions.amount_gob, coinpayments_transactions.amount_numeric, coinpayments_transactions.received_gob, coinpayments_transactions.received_numeric, coinpayments_transactions.status, coinpayments_transactions.key, coinpayments_transactions.timeout, coinpayments_transactions.created_at")
var __embed_stmt = __sqlbundle_Literal("INSERT INTO coinpayments_transactions ( id, user_id, address, amount_numeric, received_numeric, status, key, timeout, created_at ) VALUES ( ?, ?, ?, ?, ?, ?, ?, ?, ? ) RETURNING coinpayments_transactions.id, coinpayments_transactions.user_id, coinpayments_transactions.address, coinpayments_transactions.amount_numeric, coinpayments_transactions.received_numeric, coinpayments_transactions.status, coinpayments_transactions.key, coinpayments_transactions.timeout, coinpayments_transactions.created_at")
var __values []interface{}
__values = append(__values, __id_val, __user_id_val, __address_val, __amount_gob_val, __amount_numeric_val, __received_gob_val, __received_numeric_val, __status_val, __key_val, __timeout_val, __created_at_val)
__values = append(__values, __id_val, __user_id_val, __address_val, __amount_numeric_val, __received_numeric_val, __status_val, __key_val, __timeout_val, __created_at_val)
var __stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
obj.logStmt(__stmt, __values...)
coinpayments_transaction = &CoinpaymentsTransaction{}
err = obj.queryRowContext(ctx, __stmt, __values...).Scan(&coinpayments_transaction.Id, &coinpayments_transaction.UserId, &coinpayments_transaction.Address, &coinpayments_transaction.AmountGob, &coinpayments_transaction.AmountNumeric, &coinpayments_transaction.ReceivedGob, &coinpayments_transaction.ReceivedNumeric, &coinpayments_transaction.Status, &coinpayments_transaction.Key, &coinpayments_transaction.Timeout, &coinpayments_transaction.CreatedAt)
err = obj.queryRowContext(ctx, __stmt, __values...).Scan(&coinpayments_transaction.Id, &coinpayments_transaction.UserId, &coinpayments_transaction.Address, &coinpayments_transaction.AmountNumeric, &coinpayments_transaction.ReceivedNumeric, &coinpayments_transaction.Status, &coinpayments_transaction.Key, &coinpayments_transaction.Timeout, &coinpayments_transaction.CreatedAt)
if err != nil {
return nil, obj.makeErr(err)
}
@ -18193,26 +18004,25 @@ func (obj *pgxcockroachImpl) Create_StripecoinpaymentsInvoiceProjectRecord(ctx c
func (obj *pgxcockroachImpl) Create_StripecoinpaymentsTxConversionRate(ctx context.Context,
stripecoinpayments_tx_conversion_rate_tx_id StripecoinpaymentsTxConversionRate_TxId_Field,
optional StripecoinpaymentsTxConversionRate_Create_Fields) (
stripecoinpayments_tx_conversion_rate_rate_numeric StripecoinpaymentsTxConversionRate_RateNumeric_Field) (
stripecoinpayments_tx_conversion_rate *StripecoinpaymentsTxConversionRate, err error) {
defer mon.Task()(&ctx)(&err)
__now := obj.db.Hooks.Now().UTC()
__tx_id_val := stripecoinpayments_tx_conversion_rate_tx_id.value()
__rate_gob_val := optional.RateGob.value()
__rate_numeric_val := optional.RateNumeric.value()
__rate_numeric_val := stripecoinpayments_tx_conversion_rate_rate_numeric.value()
__created_at_val := __now
var __embed_stmt = __sqlbundle_Literal("INSERT INTO stripecoinpayments_tx_conversion_rates ( tx_id, rate_gob, rate_numeric, created_at ) VALUES ( ?, ?, ?, ? ) RETURNING stripecoinpayments_tx_conversion_rates.tx_id, stripecoinpayments_tx_conversion_rates.rate_gob, stripecoinpayments_tx_conversion_rates.rate_numeric, stripecoinpayments_tx_conversion_rates.created_at")
var __embed_stmt = __sqlbundle_Literal("INSERT INTO stripecoinpayments_tx_conversion_rates ( tx_id, rate_numeric, created_at ) VALUES ( ?, ?, ? ) RETURNING stripecoinpayments_tx_conversion_rates.tx_id, stripecoinpayments_tx_conversion_rates.rate_numeric, stripecoinpayments_tx_conversion_rates.created_at")
var __values []interface{}
__values = append(__values, __tx_id_val, __rate_gob_val, __rate_numeric_val, __created_at_val)
__values = append(__values, __tx_id_val, __rate_numeric_val, __created_at_val)
var __stmt = __sqlbundle_Render(obj.dialect, __embed_stmt)
obj.logStmt(__stmt, __values...)
stripecoinpayments_tx_conversion_rate = &StripecoinpaymentsTxConversionRate{}
err = obj.queryRowContext(ctx, __stmt, __values...).Scan(&stripecoinpayments_tx_conversion_rate.TxId, &stripecoinpayments_tx_conversion_rate.RateGob, &stripecoinpayments_tx_conversion_rate.RateNumeric, &stripecoinpayments_tx_conversion_rate.CreatedAt)
err = obj.queryRowContext(ctx, __stmt, __values...).Scan(&stripecoinpayments_tx_conversion_rate.TxId, &stripecoinpayments_tx_conversion_rate.RateNumeric, &stripecoinpayments_tx_conversion_rate.CreatedAt)
if err != nil {
return nil, obj.makeErr(err)
}
@ -20682,7 +20492,7 @@ func (obj *pgxcockroachImpl) All_CoinpaymentsTransaction_By_UserId_OrderBy_Desc_
rows []*CoinpaymentsTransaction, err error) {
defer mon.Task()(&ctx)(&err)
var __embed_stmt = __sqlbundle_Literal("SELECT coinpayments_transactions.id, coinpayments_transactions.user_id, coinpayments_transactions.address, coinpayments_transactions.amount_gob, coinpayments_transactions.amount_numeric, coinpayments_transactions.received_gob, coinpayments_transactions.received_numeric, coinpayments_transactions.status, coinpayments_transactions.key, coinpayments_transactions.timeout, coinpayments_transactions.created_at FROM coinpayments_transactions WHERE coinpayments_transactions.user_id = ? ORDER BY coinpayments_transactions.created_at DESC")
var __embed_stmt = __sqlbundle_Literal("SELECT coinpayments_transactions.id, coinpayments_transactions.user_id, coinpayments_transactions.address, coinpayments_transactions.amount_numeric, coinpayments_transactions.received_numeric, coinpayments_transactions.status, coinpayments_transactions.key, coinpayments_transactions.timeout, coinpayments_transactions.created_at FROM coinpayments_transactions WHERE coinpayments_transactions.user_id = ? ORDER BY coinpayments_transactions.created_at DESC")
var __values []interface{}
__values = append(__values, coinpayments_transaction_user_id.value())
@ -20700,7 +20510,7 @@ func (obj *pgxcockroachImpl) All_CoinpaymentsTransaction_By_UserId_OrderBy_Desc_
for __rows.Next() {
coinpayments_transaction := &CoinpaymentsTransaction{}
err = __rows.Scan(&coinpayments_transaction.Id, &coinpayments_transaction.UserId, &coinpayments_transaction.Address, &coinpayments_transaction.AmountGob, &coinpayments_transaction.AmountNumeric, &coinpayments_transaction.ReceivedGob, &coinpayments_transaction.ReceivedNumeric, &coinpayments_transaction.Status, &coinpayments_transaction.Key, &coinpayments_transaction.Timeout, &coinpayments_transaction.CreatedAt)
err = __rows.Scan(&coinpayments_transaction.Id, &coinpayments_transaction.UserId, &coinpayments_transaction.Address, &coinpayments_transaction.AmountNumeric, &coinpayments_transaction.ReceivedNumeric, &coinpayments_transaction.Status, &coinpayments_transaction.Key, &coinpayments_transaction.Timeout, &coinpayments_transaction.CreatedAt)
if err != nil {
return nil, err
}
@ -20802,7 +20612,7 @@ func (obj *pgxcockroachImpl) Get_StripecoinpaymentsTxConversionRate_By_TxId(ctx
stripecoinpayments_tx_conversion_rate *StripecoinpaymentsTxConversionRate, err error) {
defer mon.Task()(&ctx)(&err)
var __embed_stmt = __sqlbundle_Literal("SELECT stripecoinpayments_tx_conversion_rates.tx_id, stripecoinpayments_tx_conversion_rates.rate_gob, stripecoinpayments_tx_conversion_rates.rate_numeric, stripecoinpayments_tx_conversion_rates.created_at FROM stripecoinpayments_tx_conversion_rates WHERE stripecoinpayments_tx_conversion_rates.tx_id = ?")
var __embed_stmt = __sqlbundle_Literal("SELECT stripecoinpayments_tx_conversion_rates.tx_id, stripecoinpayments_tx_conversion_rates.rate_numeric, stripecoinpayments_tx_conversion_rates.created_at FROM stripecoinpayments_tx_conversion_rates WHERE stripecoinpayments_tx_conversion_rates.tx_id = ?")
var __values []interface{}
__values = append(__values, stripecoinpayments_tx_conversion_rate_tx_id.value())
@ -20811,7 +20621,7 @@ func (obj *pgxcockroachImpl) Get_StripecoinpaymentsTxConversionRate_By_TxId(ctx
obj.logStmt(__stmt, __values...)
stripecoinpayments_tx_conversion_rate = &StripecoinpaymentsTxConversionRate{}
err = obj.queryRowContext(ctx, __stmt, __values...).Scan(&stripecoinpayments_tx_conversion_rate.TxId, &stripecoinpayments_tx_conversion_rate.RateGob, &stripecoinpayments_tx_conversion_rate.RateNumeric, &stripecoinpayments_tx_conversion_rate.CreatedAt)
err = obj.queryRowContext(ctx, __stmt, __values...).Scan(&stripecoinpayments_tx_conversion_rate.TxId, &stripecoinpayments_tx_conversion_rate.RateNumeric, &stripecoinpayments_tx_conversion_rate.CreatedAt)
if err != nil {
return (*StripecoinpaymentsTxConversionRate)(nil), obj.makeErr(err)
}
@ -22602,27 +22412,12 @@ func (obj *pgxcockroachImpl) Update_CoinpaymentsTransaction_By_Id(ctx context.Co
defer mon.Task()(&ctx)(&err)
var __sets = &__sqlbundle_Hole{}
var __embed_stmt = __sqlbundle_Literals{Join: "", SQLs: []__sqlbundle_SQL{__sqlbundle_Literal("UPDATE coinpayments_transactions SET "), __sets, __sqlbundle_Literal(" WHERE coinpayments_transactions.id = ? RETURNING coinpayments_transactions.id, coinpayments_transactions.user_id, coinpayments_transactions.address, coinpayments_transactions.amount_gob, coinpayments_transactions.amount_numeric, coinpayments_transactions.received_gob, coinpayments_transactions.received_numeric, coinpayments_transactions.status, coinpayments_transactions.key, coinpayments_transactions.timeout, coinpayments_transactions.created_at")}}
var __embed_stmt = __sqlbundle_Literals{Join: "", SQLs: []__sqlbundle_SQL{__sqlbundle_Literal("UPDATE coinpayments_transactions SET "), __sets, __sqlbundle_Literal(" WHERE coinpayments_transactions.id = ? RETURNING coinpayments_transactions.id, coinpayments_transactions.user_id, coinpayments_transactions.address, coinpayments_transactions.amount_numeric, coinpayments_transactions.received_numeric, coinpayments_transactions.status, coinpayments_transactions.key, coinpayments_transactions.timeout, coinpayments_transactions.created_at")}}
__sets_sql := __sqlbundle_Literals{Join: ", "}
var __values []interface{}
var __args []interface{}
if update.AmountGob._set {
__values = append(__values, update.AmountGob.value())
__sets_sql.SQLs = append(__sets_sql.SQLs, __sqlbundle_Literal("amount_gob = ?"))
}
if update.AmountNumeric._set {
__values = append(__values, update.AmountNumeric.value())
__sets_sql.SQLs = append(__sets_sql.SQLs, __sqlbundle_Literal("amount_numeric = ?"))
}
if update.ReceivedGob._set {
__values = append(__values, update.ReceivedGob.value())
__sets_sql.SQLs = append(__sets_sql.SQLs, __sqlbundle_Literal("received_gob = ?"))
}
if update.ReceivedNumeric._set {
__values = append(__values, update.ReceivedNumeric.value())
__sets_sql.SQLs = append(__sets_sql.SQLs, __sqlbundle_Literal("received_numeric = ?"))
@ -22646,7 +22441,7 @@ func (obj *pgxcockroachImpl) Update_CoinpaymentsTransaction_By_Id(ctx context.Co
obj.logStmt(__stmt, __values...)
coinpayments_transaction = &CoinpaymentsTransaction{}
err = obj.queryRowContext(ctx, __stmt, __values...).Scan(&coinpayments_transaction.Id, &coinpayments_transaction.UserId, &coinpayments_transaction.Address, &coinpayments_transaction.AmountGob, &coinpayments_transaction.AmountNumeric, &coinpayments_transaction.ReceivedGob, &coinpayments_transaction.ReceivedNumeric, &coinpayments_transaction.Status, &coinpayments_transaction.Key, &coinpayments_transaction.Timeout, &coinpayments_transaction.CreatedAt)
err = obj.queryRowContext(ctx, __stmt, __values...).Scan(&coinpayments_transaction.Id, &coinpayments_transaction.UserId, &coinpayments_transaction.Address, &coinpayments_transaction.AmountNumeric, &coinpayments_transaction.ReceivedNumeric, &coinpayments_transaction.Status, &coinpayments_transaction.Key, &coinpayments_transaction.Timeout, &coinpayments_transaction.CreatedAt)
if err == sql.ErrNoRows {
return nil, nil
}
@ -24126,16 +23921,17 @@ func (rx *Rx) Create_CoinpaymentsTransaction(ctx context.Context,
coinpayments_transaction_id CoinpaymentsTransaction_Id_Field,
coinpayments_transaction_user_id CoinpaymentsTransaction_UserId_Field,
coinpayments_transaction_address CoinpaymentsTransaction_Address_Field,
coinpayments_transaction_amount_numeric CoinpaymentsTransaction_AmountNumeric_Field,
coinpayments_transaction_received_numeric CoinpaymentsTransaction_ReceivedNumeric_Field,
coinpayments_transaction_status CoinpaymentsTransaction_Status_Field,
coinpayments_transaction_key CoinpaymentsTransaction_Key_Field,
coinpayments_transaction_timeout CoinpaymentsTransaction_Timeout_Field,
optional CoinpaymentsTransaction_Create_Fields) (
coinpayments_transaction_timeout CoinpaymentsTransaction_Timeout_Field) (
coinpayments_transaction *CoinpaymentsTransaction, err error) {
var tx *Tx
if tx, err = rx.getTx(ctx); err != nil {
return
}
return tx.Create_CoinpaymentsTransaction(ctx, coinpayments_transaction_id, coinpayments_transaction_user_id, coinpayments_transaction_address, coinpayments_transaction_status, coinpayments_transaction_key, coinpayments_transaction_timeout, optional)
return tx.Create_CoinpaymentsTransaction(ctx, coinpayments_transaction_id, coinpayments_transaction_user_id, coinpayments_transaction_address, coinpayments_transaction_amount_numeric, coinpayments_transaction_received_numeric, coinpayments_transaction_status, coinpayments_transaction_key, coinpayments_transaction_timeout)
}
@ -24353,13 +24149,13 @@ func (rx *Rx) Create_StripecoinpaymentsInvoiceProjectRecord(ctx context.Context,
func (rx *Rx) Create_StripecoinpaymentsTxConversionRate(ctx context.Context,
stripecoinpayments_tx_conversion_rate_tx_id StripecoinpaymentsTxConversionRate_TxId_Field,
optional StripecoinpaymentsTxConversionRate_Create_Fields) (
stripecoinpayments_tx_conversion_rate_rate_numeric StripecoinpaymentsTxConversionRate_RateNumeric_Field) (
stripecoinpayments_tx_conversion_rate *StripecoinpaymentsTxConversionRate, err error) {
var tx *Tx
if tx, err = rx.getTx(ctx); err != nil {
return
}
return tx.Create_StripecoinpaymentsTxConversionRate(ctx, stripecoinpayments_tx_conversion_rate_tx_id, optional)
return tx.Create_StripecoinpaymentsTxConversionRate(ctx, stripecoinpayments_tx_conversion_rate_tx_id, stripecoinpayments_tx_conversion_rate_rate_numeric)
}
@ -25580,10 +25376,11 @@ type Methods interface {
coinpayments_transaction_id CoinpaymentsTransaction_Id_Field,
coinpayments_transaction_user_id CoinpaymentsTransaction_UserId_Field,
coinpayments_transaction_address CoinpaymentsTransaction_Address_Field,
coinpayments_transaction_amount_numeric CoinpaymentsTransaction_AmountNumeric_Field,
coinpayments_transaction_received_numeric CoinpaymentsTransaction_ReceivedNumeric_Field,
coinpayments_transaction_status CoinpaymentsTransaction_Status_Field,
coinpayments_transaction_key CoinpaymentsTransaction_Key_Field,
coinpayments_transaction_timeout CoinpaymentsTransaction_Timeout_Field,
optional CoinpaymentsTransaction_Create_Fields) (
coinpayments_transaction_timeout CoinpaymentsTransaction_Timeout_Field) (
coinpayments_transaction *CoinpaymentsTransaction, err error)
Create_Coupon(ctx context.Context,
@ -25702,7 +25499,7 @@ type Methods interface {
Create_StripecoinpaymentsTxConversionRate(ctx context.Context,
stripecoinpayments_tx_conversion_rate_tx_id StripecoinpaymentsTxConversionRate_TxId_Field,
optional StripecoinpaymentsTxConversionRate_Create_Fields) (
stripecoinpayments_tx_conversion_rate_rate_numeric StripecoinpaymentsTxConversionRate_RateNumeric_Field) (
stripecoinpayments_tx_conversion_rate *StripecoinpaymentsTxConversionRate, err error)
Create_User(ctx context.Context,

View File

@ -56,10 +56,8 @@ CREATE TABLE coinpayments_transactions (
id text NOT NULL,
user_id bytea NOT NULL,
address text NOT NULL,
amount_gob bytea,
amount_numeric bigint,
received_gob bytea,
received_numeric bigint,
amount_numeric bigint NOT NULL,
received_numeric bigint NOT NULL,
status integer NOT NULL,
key text NOT NULL,
timeout integer NOT NULL,
@ -398,8 +396,7 @@ CREATE TABLE stripecoinpayments_invoice_project_records (
);
CREATE TABLE stripecoinpayments_tx_conversion_rates (
tx_id text NOT NULL,
rate_gob bytea,
rate_numeric double precision,
rate_numeric double precision NOT NULL,
created_at timestamp with time zone NOT NULL,
PRIMARY KEY ( tx_id )
);

View File

@ -56,10 +56,8 @@ CREATE TABLE coinpayments_transactions (
id text NOT NULL,
user_id bytea NOT NULL,
address text NOT NULL,
amount_gob bytea,
amount_numeric bigint,
received_gob bytea,
received_numeric bigint,
amount_numeric bigint NOT NULL,
received_numeric bigint NOT NULL,
status integer NOT NULL,
key text NOT NULL,
timeout integer NOT NULL,
@ -398,8 +396,7 @@ CREATE TABLE stripecoinpayments_invoice_project_records (
);
CREATE TABLE stripecoinpayments_tx_conversion_rates (
tx_id text NOT NULL,
rate_gob bytea,
rate_numeric double precision,
rate_numeric double precision NOT NULL,
created_at timestamp with time zone NOT NULL,
PRIMARY KEY ( tx_id )
);

View File

@ -1840,6 +1840,16 @@ func (db *satelliteDB) PostgresMigration() *migrate.Migration {
WHERE owner_id IN (SELECT id FROM users WHERE paid_tier = true);`,
},
},
{
DB: &db.migrationDB,
Description: "make _numeric fields not null (all are now populated)",
Version: 191,
Action: migrate.SQL{
`ALTER TABLE coinpayments_transactions ALTER COLUMN amount_numeric SET NOT NULL;`,
`ALTER TABLE coinpayments_transactions ALTER COLUMN received_numeric SET NOT NULL;`,
`ALTER TABLE stripecoinpayments_tx_conversion_rates ALTER COLUMN rate_numeric SET NOT NULL;`,
},
},
// NB: after updating testdata in `testdata`, run
// `go generate` to update `migratez.go`.
},

View File

@ -294,6 +294,17 @@ func migrateTest(t *testing.T, connStr string) {
reputations.RemoveColumn("suspended")
}
// TODO(thepaul): remove these exceptions on adding migration to remove _gob columns
coinpaymentsTransactions, ok := finalSchema.FindTable("coinpayments_transactions")
if ok {
coinpaymentsTransactions.RemoveColumn("amount_gob")
coinpaymentsTransactions.RemoveColumn("received_gob")
}
conversionRates, ok := finalSchema.FindTable("stripecoinpayments_tx_conversion_rates")
if ok {
conversionRates.RemoveColumn("rate_gob")
}
// verify that we also match the dbx version
require.Equal(t, dbxschema, finalSchema, "result of all migration scripts did not match dbx schema")
}

View File

@ -13,7 +13,7 @@ func (db *satelliteDB) testMigration() *migrate.Migration {
{
DB: &db.migrationDB,
Description: "Testing setup",
Version: 190,
Version: 191,
Action: migrate.SQL{`-- AUTOGENERATED BY storj.io/dbx
-- DO NOT EDIT
CREATE TABLE accounting_rollups (
@ -73,9 +73,9 @@ CREATE TABLE coinpayments_transactions (
user_id bytea NOT NULL,
address text NOT NULL,
amount_gob bytea,
amount_numeric int8,
amount_numeric int8 NOT NULL,
received_gob bytea,
received_numeric int8,
received_numeric int8 NOT NULL,
status integer NOT NULL,
key text NOT NULL,
timeout integer NOT NULL,
@ -417,7 +417,7 @@ CREATE TABLE stripecoinpayments_invoice_project_records (
CREATE TABLE stripecoinpayments_tx_conversion_rates (
tx_id text NOT NULL,
rate_gob bytea,
rate_numeric double precision,
rate_numeric double precision NOT NULL,
created_at timestamp with time zone NOT NULL,
PRIMARY KEY ( tx_id )
);

View File

@ -667,4 +667,10 @@ INSERT INTO "oauth_tokens"("client_id", "user_id", "scope", "kind", "token", "cr
INSERT INTO "coinpayments_transactions" ("id", "user_id", "address", "amount_gob", "amount_numeric", "received_gob", "received_numeric", "status", "key", "timeout", "created_at") VALUES ('different_tx_id_from_before', E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, 'address', E'\\x0112000000400000000b9cc6615b299c5b96'::bytea, 125419938429, E'\\x011200000040ffffffe6abcc77118461cefd'::bytea, 1, 1, 'key', 60, '2021-07-28 20:24:11.932313-05');
INSERT INTO "stripecoinpayments_tx_conversion_rates" ("tx_id", "rate_gob", "rate_numeric", "created_at") VALUES ('different_tx_id_from_before', E'\\x01020000004000000002c3890fdaa221774ec3a4'::bytea, 3.14159265359, '2021-07-28 20:24:11.932313-05');
-- NEW DATA --
-- NEW DATA --
-- this simulates the migration which would have been carried out (outside of SQL) by the satellite core
UPDATE coinpayments_transactions SET amount_gob = NULL, received_gob = NULL, amount_numeric = 1411112222, received_numeric = 1311112222 WHERE id = 'tx_id';
UPDATE coinpayments_transactions SET amount_gob = NULL, received_gob = NULL WHERE id = 'different_tx_id_from_before';
UPDATE stripecoinpayments_tx_conversion_rates SET rate_gob = NULL, rate_numeric = '1.929883831' WHERE tx_id = 'tx_id';
UPDATE stripecoinpayments_tx_conversion_rates SET rate_gob = NULL WHERE tx_id = 'different_tx_id_from_before';

View File

@ -0,0 +1,670 @@
-- AUTOGENERATED BY storj.io/dbx
-- DO NOT EDIT
CREATE TABLE accounting_rollups (
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 ( node_id, start_time )
);
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 with time zone 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_bandwidth_rollup_archives (
bucket_name bytea NOT NULL,
project_id bytea NOT NULL,
interval_start timestamp with time zone 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 with time zone NOT NULL,
total_bytes bigint NOT NULL DEFAULT 0,
inline bigint NOT NULL,
remote bigint NOT NULL,
total_segments_count integer NOT NULL DEFAULT 0,
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_gob bytea,
amount_numeric int8 NOT NULL,
received_gob bytea,
received_numeric int8 NOT NULL,
status integer NOT NULL,
key text NOT NULL,
timeout integer NOT NULL,
created_at timestamp with time zone NOT NULL,
PRIMARY KEY ( id )
);
CREATE TABLE coupons (
id bytea NOT NULL,
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,
billing_periods bigint,
coupon_code_name text,
created_at timestamp with time zone NOT NULL,
PRIMARY KEY ( id )
);
CREATE TABLE coupon_codes (
id bytea NOT NULL,
name text NOT NULL,
amount bigint NOT NULL,
description text NOT NULL,
type integer NOT NULL,
billing_periods bigint,
created_at timestamp with time zone NOT NULL,
PRIMARY KEY ( id ),
UNIQUE ( name )
);
CREATE TABLE coupon_usages (
coupon_id bytea NOT NULL,
amount bigint NOT NULL,
status integer NOT NULL,
period timestamp with time zone NOT NULL,
PRIMARY KEY ( coupon_id, period )
);
CREATE TABLE graceful_exit_progress (
node_id bytea NOT NULL,
bytes_transferred bigint NOT NULL,
pieces_transferred bigint NOT NULL DEFAULT 0,
pieces_failed bigint NOT NULL DEFAULT 0,
updated_at timestamp with time zone NOT NULL,
PRIMARY KEY ( node_id )
);
CREATE TABLE graceful_exit_segment_transfer_queue (
node_id bytea NOT NULL,
stream_id bytea NOT NULL,
position bigint NOT NULL,
piece_num integer NOT NULL,
root_piece_id bytea,
durability_ratio double precision NOT NULL,
queued_at timestamp with time zone NOT NULL,
requested_at timestamp with time zone,
last_failed_at timestamp with time zone,
last_failed_code integer,
failed_count integer,
finished_at timestamp with time zone,
order_limit_send_count integer NOT NULL DEFAULT 0,
PRIMARY KEY ( node_id, stream_id, position, piece_num )
);
CREATE TABLE nodes (
id bytea NOT NULL,
address text NOT NULL DEFAULT '',
last_net text NOT NULL,
last_ip_port text,
country_code text,
protocol integer NOT NULL DEFAULT 0,
type integer NOT NULL DEFAULT 0,
email text NOT NULL,
wallet text NOT NULL,
wallet_features text NOT NULL DEFAULT '',
free_disk bigint NOT NULL DEFAULT -1,
piece_count bigint NOT NULL DEFAULT 0,
major bigint NOT NULL DEFAULT 0,
minor bigint NOT NULL DEFAULT 0,
patch bigint NOT NULL DEFAULT 0,
hash text NOT NULL DEFAULT '',
timestamp timestamp with time zone NOT NULL DEFAULT '0001-01-01 00:00:00+00',
release boolean NOT NULL DEFAULT false,
latency_90 bigint NOT NULL DEFAULT 0,
vetted_at timestamp with time zone,
created_at timestamp with time zone NOT NULL DEFAULT current_timestamp,
updated_at timestamp with time zone NOT NULL DEFAULT current_timestamp,
last_contact_success timestamp with time zone NOT NULL DEFAULT 'epoch',
last_contact_failure timestamp with time zone NOT NULL DEFAULT 'epoch',
disqualified timestamp with time zone,
disqualification_reason integer,
suspended timestamp with time zone,
unknown_audit_suspended timestamp with time zone,
offline_suspended timestamp with time zone,
under_review timestamp with time zone,
exit_initiated_at timestamp with time zone,
exit_loop_completed_at timestamp with time zone,
exit_finished_at timestamp with time zone,
exit_success boolean NOT NULL DEFAULT false,
PRIMARY KEY ( id )
);
CREATE TABLE node_api_versions (
id bytea NOT NULL,
api_version integer NOT NULL,
created_at timestamp with time zone NOT NULL,
updated_at timestamp with time zone NOT NULL,
PRIMARY KEY ( id )
);
CREATE TABLE oauth_clients (
id bytea NOT NULL,
encrypted_secret bytea NOT NULL,
redirect_url text NOT NULL,
user_id bytea NOT NULL,
app_name text NOT NULL,
app_logo_url text NOT NULL,
PRIMARY KEY ( id )
);
CREATE TABLE oauth_codes (
client_id bytea NOT NULL,
user_id bytea NOT NULL,
scope text NOT NULL,
redirect_url text NOT NULL,
challenge text NOT NULL,
challenge_method text NOT NULL,
code text NOT NULL,
created_at timestamp with time zone NOT NULL,
expires_at timestamp with time zone NOT NULL,
claimed_at timestamp with time zone,
PRIMARY KEY ( code )
);
CREATE TABLE oauth_tokens (
client_id bytea NOT NULL,
user_id bytea NOT NULL,
scope text NOT NULL,
kind integer NOT NULL,
token bytea NOT NULL,
created_at timestamp with time zone NOT NULL,
expires_at timestamp with time zone NOT NULL,
PRIMARY KEY ( token )
);
CREATE TABLE offers (
id serial NOT NULL,
name text NOT NULL,
description text NOT NULL,
award_credit_in_cents integer NOT NULL DEFAULT 0,
invitee_credit_in_cents integer NOT NULL DEFAULT 0,
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 projects (
id bytea NOT NULL,
name text NOT NULL,
description text NOT NULL,
usage_limit bigint,
bandwidth_limit bigint,
segment_limit bigint DEFAULT 1000000,
rate_limit integer,
burst_limit integer,
max_buckets integer,
partner_id bytea,
user_agent bytea,
owner_id bytea NOT NULL,
created_at timestamp with time zone NOT NULL,
PRIMARY KEY ( id )
);
CREATE TABLE project_bandwidth_daily_rollups (
project_id bytea NOT NULL,
interval_day date NOT NULL,
egress_allocated bigint NOT NULL,
egress_settled bigint NOT NULL,
egress_dead bigint NOT NULL DEFAULT 0,
PRIMARY KEY ( project_id, interval_day )
);
CREATE TABLE project_bandwidth_rollups (
project_id bytea NOT NULL,
interval_month date NOT NULL,
egress_allocated bigint NOT NULL,
PRIMARY KEY ( project_id, interval_month )
);
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 repair_queue (
stream_id bytea NOT NULL,
position bigint NOT NULL,
attempted_at timestamp with time zone,
updated_at timestamp with time zone NOT NULL DEFAULT current_timestamp,
inserted_at timestamp with time zone NOT NULL DEFAULT current_timestamp,
segment_health double precision NOT NULL DEFAULT 1,
PRIMARY KEY ( stream_id, position )
);
CREATE TABLE reputations (
id bytea NOT NULL,
audit_success_count bigint NOT NULL DEFAULT 0,
total_audit_count bigint NOT NULL DEFAULT 0,
vetted_at timestamp with time zone,
created_at timestamp with time zone NOT NULL DEFAULT current_timestamp,
updated_at timestamp with time zone NOT NULL DEFAULT current_timestamp,
disqualified timestamp with time zone,
suspended timestamp with time zone,
unknown_audit_suspended timestamp with time zone,
offline_suspended timestamp with time zone,
under_review timestamp with time zone,
online_score double precision NOT NULL DEFAULT 1,
audit_history bytea NOT NULL,
audit_reputation_alpha double precision NOT NULL DEFAULT 1,
audit_reputation_beta double precision NOT NULL DEFAULT 0,
unknown_audit_reputation_alpha double precision NOT NULL DEFAULT 1,
unknown_audit_reputation_beta double precision NOT NULL DEFAULT 0,
PRIMARY KEY ( id )
);
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 revocations (
revoked bytea NOT NULL,
api_key_id bytea NOT NULL,
PRIMARY KEY ( revoked )
);
CREATE TABLE segment_pending_audits (
node_id bytea NOT NULL,
stream_id bytea NOT NULL,
position bigint 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,
PRIMARY KEY ( node_id )
);
CREATE TABLE storagenode_bandwidth_rollups (
storagenode_id bytea NOT NULL,
interval_start timestamp with time zone 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_bandwidth_rollup_archives (
storagenode_id bytea NOT NULL,
interval_start timestamp with time zone 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_bandwidth_rollups_phase2 (
storagenode_id bytea NOT NULL,
interval_start timestamp with time zone 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_payments (
id bigserial NOT NULL,
created_at timestamp with time zone NOT NULL,
node_id bytea NOT NULL,
period text NOT NULL,
amount bigint NOT NULL,
receipt text,
notes text,
PRIMARY KEY ( id )
);
CREATE TABLE storagenode_paystubs (
period text NOT NULL,
node_id bytea NOT NULL,
created_at timestamp with time zone NOT NULL,
codes text NOT NULL,
usage_at_rest double precision NOT NULL,
usage_get bigint NOT NULL,
usage_put bigint NOT NULL,
usage_get_repair bigint NOT NULL,
usage_put_repair bigint NOT NULL,
usage_get_audit bigint NOT NULL,
comp_at_rest bigint NOT NULL,
comp_get bigint NOT NULL,
comp_put bigint NOT NULL,
comp_get_repair bigint NOT NULL,
comp_put_repair bigint NOT NULL,
comp_get_audit bigint NOT NULL,
surge_percent bigint NOT NULL,
held bigint NOT NULL,
owed bigint NOT NULL,
disposed bigint NOT NULL,
paid bigint NOT NULL,
distributed bigint NOT NULL,
PRIMARY KEY ( period, node_id )
);
CREATE TABLE storagenode_storage_tallies (
node_id bytea NOT NULL,
interval_end_time timestamp with time zone NOT NULL,
data_total double precision NOT NULL,
PRIMARY KEY ( interval_end_time, node_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,
segments bigint,
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_gob bytea,
rate_numeric double precision 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,
user_agent bytea,
created_at timestamp with time zone NOT NULL,
project_limit integer NOT NULL DEFAULT 0,
project_bandwidth_limit bigint NOT NULL DEFAULT 0,
project_storage_limit bigint NOT NULL DEFAULT 0,
project_segment_limit bigint NOT NULL DEFAULT 0,
paid_tier boolean NOT NULL DEFAULT false,
position text,
company_name text,
company_size integer,
working_on text,
is_professional boolean NOT NULL DEFAULT false,
employee_count text,
have_sales_contact boolean NOT NULL DEFAULT false,
mfa_enabled boolean NOT NULL DEFAULT false,
mfa_secret_key text,
mfa_recovery_codes text,
signup_promo_code text,
last_verification_reminder timestamp with time zone,
PRIMARY KEY ( id )
);
CREATE TABLE value_attributions (
project_id bytea NOT NULL,
bucket_name bytea NOT NULL,
partner_id bytea NOT NULL,
user_agent bytea,
last_updated timestamp with time zone 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,
user_agent 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,
user_agent 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,
placement integer,
PRIMARY KEY ( id ),
UNIQUE ( project_id, name )
);
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 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 accounting_rollups_start_time_index ON accounting_rollups ( start_time ) ;
CREATE INDEX bucket_bandwidth_rollups_project_id_action_interval_index ON bucket_bandwidth_rollups ( project_id, action, interval_start ) ;
CREATE INDEX bucket_bandwidth_rollups_action_interval_project_id_index ON bucket_bandwidth_rollups ( action, interval_start, project_id ) ;
CREATE INDEX bucket_bandwidth_rollups_archive_project_id_action_interval_index ON bucket_bandwidth_rollup_archives ( project_id, action, interval_start ) ;
CREATE INDEX bucket_bandwidth_rollups_archive_action_interval_project_id_index ON bucket_bandwidth_rollup_archives ( action, interval_start, project_id ) ;
CREATE INDEX bucket_storage_tallies_project_id_interval_start_index ON bucket_storage_tallies ( project_id, interval_start ) ;
CREATE INDEX graceful_exit_segment_transfer_nid_dr_qa_fa_lfa_index ON graceful_exit_segment_transfer_queue ( node_id, durability_ratio, queued_at, finished_at, last_failed_at ) ;
CREATE INDEX node_last_ip ON nodes ( last_net ) ;
CREATE INDEX nodes_dis_unk_off_exit_fin_last_success_index ON nodes ( disqualified, unknown_audit_suspended, offline_suspended, exit_finished_at, last_contact_success ) ;
CREATE INDEX nodes_type_last_cont_success_free_disk_ma_mi_patch_vetted_partial_index ON nodes ( type, last_contact_success, free_disk, major, minor, patch, vetted_at ) WHERE nodes.disqualified is NULL AND nodes.unknown_audit_suspended is NULL AND nodes.exit_initiated_at is NULL AND nodes.release = true AND nodes.last_net != '' ;
CREATE INDEX nodes_dis_unk_aud_exit_init_rel_type_last_cont_success_stored_index ON nodes ( disqualified, unknown_audit_suspended, exit_initiated_at, release, type, last_contact_success ) WHERE nodes.disqualified is NULL AND nodes.unknown_audit_suspended is NULL AND nodes.exit_initiated_at is NULL AND nodes.release = true ;
CREATE INDEX oauth_clients_user_id_index ON oauth_clients ( user_id ) ;
CREATE INDEX oauth_codes_user_id_index ON oauth_codes ( user_id ) ;
CREATE INDEX oauth_codes_client_id_index ON oauth_codes ( client_id ) ;
CREATE INDEX oauth_tokens_user_id_index ON oauth_tokens ( user_id ) ;
CREATE INDEX oauth_tokens_client_id_index ON oauth_tokens ( client_id ) ;
CREATE INDEX repair_queue_updated_at_index ON repair_queue ( updated_at ) ;
CREATE INDEX repair_queue_num_healthy_pieces_attempted_at_index ON repair_queue ( segment_health, attempted_at ) ;
CREATE INDEX storagenode_bandwidth_rollups_interval_start_index ON storagenode_bandwidth_rollups ( interval_start ) ;
CREATE INDEX storagenode_bandwidth_rollup_archives_interval_start_index ON storagenode_bandwidth_rollup_archives ( interval_start ) ;
CREATE INDEX storagenode_payments_node_id_period_index ON storagenode_payments ( node_id, period ) ;
CREATE INDEX storagenode_paystubs_node_id_index ON storagenode_paystubs ( node_id ) ;
CREATE INDEX storagenode_storage_tallies_node_id_index ON storagenode_storage_tallies ( node_id ) ;
CREATE UNIQUE INDEX credits_earned_user_id_offer_id ON user_credits ( id, offer_id ) ;
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);
-- MAIN DATA --
INSERT INTO "accounting_rollups"("node_id", "start_time", "put_total", "get_total", "get_audit_total", "get_repair_total", "put_repair_total", "at_rest_total") VALUES (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', 3000, 6000, 9000, 12000, 0, 15000);
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_disk", "piece_count", "major", "minor", "patch", "hash", "timestamp", "release","latency_90", "created_at", "updated_at", "last_contact_success", "last_contact_failure", "disqualified", "disqualification_reason", "suspended", "exit_success") VALUES (E'\\153\\313\\233\\074\\327\\177\\136\\070\\346\\001', '127.0.0.1:55516', '', 0, 4, '', '', -1, 0, 0, 1, 0, '', 'epoch', false, 0, '2019-02-14 08:07:31.028103+00', '2019-02-14 08:07:31.108963+00', 'epoch', 'epoch', NULL, NULL, NULL, false);
INSERT INTO "nodes"("id", "address", "last_net", "protocol", "type", "email", "wallet", "free_disk", "piece_count", "major", "minor", "patch", "hash", "timestamp", "release","latency_90","created_at", "updated_at", "last_contact_success", "last_contact_failure", "disqualified", "disqualification_reason", "suspended","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, 0, 0, 1, 0, '', 'epoch', false, 0, '2019-02-14 08:07:31.028103+00', '2019-02-14 08:07:31.108963+00', 'epoch', 'epoch', NULL, NULL, NULL, false);
INSERT INTO "nodes"("id", "address", "last_net", "protocol", "type", "email", "wallet", "free_disk", "piece_count", "major", "minor", "patch", "hash", "timestamp", "release","latency_90","created_at", "updated_at", "last_contact_success", "last_contact_failure", "disqualified", "disqualification_reason", "suspended","exit_success") VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\014', '127.0.0.1:55517', '', 0, 4, '', '', -1, 0, 0, 1, 0, '', 'epoch', false, 0, '2019-02-14 08:07:31.028103+00', '2019-02-14 08:07:31.108963+00', 'epoch', 'epoch', NULL, NULL, NULL,false);
INSERT INTO "nodes"("id", "address", "last_net", "protocol", "type", "email", "wallet", "free_disk", "piece_count", "major", "minor", "patch", "hash", "timestamp", "release","latency_90","created_at", "updated_at", "last_contact_success", "last_contact_failure", "disqualified", "disqualification_reason", "suspended","exit_success") VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\015', '127.0.0.1:55519', '', 0, 4, '', '', -1, 0, 0, 1, 0, '', 'epoch', false, 0, '2019-02-14 08:07:31.028103+00', '2019-02-14 08:07:31.108963+00', 'epoch', 'epoch', NULL, NULL, NULL,false);
INSERT INTO "nodes"("id", "address", "last_net", "protocol", "type", "email", "wallet", "free_disk", "piece_count", "major", "minor", "patch", "hash", "timestamp", "release","latency_90","created_at", "updated_at", "last_contact_success", "last_contact_failure", "disqualified", "disqualification_reason", "suspended","exit_success", "vetted_at") VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\016', '127.0.0.1:55520', '', 0, 4, '', '', -1, 0, 0, 1, 0, '', 'epoch', false, 0, '2019-02-14 08:07:31.028103+00', '2019-02-14 08:07:31.108963+00', 'epoch', 'epoch', NULL, NULL, NULL, false, '2020-03-18 12:00:00.000000+00');
INSERT INTO "nodes"("id", "address", "last_net", "protocol", "type", "email", "wallet", "free_disk", "piece_count", "major", "minor", "patch", "hash", "timestamp", "release","latency_90","created_at", "updated_at", "last_contact_success", "last_contact_failure", "disqualified", "disqualification_reason", "suspended","exit_success") VALUES (E'\\154\\313\\233\\074\\327\\177\\136\\070\\346\\001', '127.0.0.1:55516', '', 0, 4, '', '', -1, 0, 0, 1, 0, '', 'epoch', false, 0, '2019-02-14 08:07:31.028103+00', '2019-02-14 08:07:31.108963+00', 'epoch', 'epoch', NULL, NULL, NULL, false);
INSERT INTO "nodes"("id", "address", "last_net", "last_ip_port", "protocol", "type", "email", "wallet", "free_disk", "piece_count", "major", "minor", "patch", "hash", "timestamp", "release","latency_90", "created_at", "updated_at", "last_contact_success", "last_contact_failure", "disqualified", "disqualification_reason", "suspended", "exit_success") VALUES (E'\\154\\313\\233\\074\\327\\177\\136\\070\\346\\002', '127.0.0.1:55516', '127.0.0.0', '127.0.0.1:55516', 0, 4, '', '', -1, 0, 0, 1, 0, '', 'epoch', false, 0, '2019-02-14 08:07:31.028103+00', '2019-02-14 08:07:31.108963+00', 'epoch', 'epoch', NULL, NUll, NULL, false);
INSERT INTO "nodes"("id", "address", "last_net", "protocol", "type", "email", "wallet", "free_disk", "piece_count", "major", "minor", "patch", "hash", "timestamp", "release","latency_90","created_at", "updated_at", "last_contact_success", "last_contact_failure", "disqualified", "disqualification_reason", "suspended", "exit_success") VALUES (E'\\363\\341\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\016', '127.0.0.1:55516', '', 0, 4, '', '', -1, 0, 0, 1, 0, '', 'epoch', false, 0, '2019-02-14 08:07:31.028103+00', '2019-02-14 08:07:31.108963+00', 'epoch', 'epoch', NULL, NULL, NULL, false);
INSERT INTO "nodes"("id", "address", "last_net", "protocol", "type", "email", "wallet", "wallet_features", "free_disk", "piece_count", "major", "minor", "patch", "hash", "timestamp", "release","latency_90","created_at", "updated_at", "last_contact_success", "last_contact_failure", "disqualified", "disqualification_reason", "suspended", "exit_success") VALUES (E'\\362\\341\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\016', '127.0.0.1:55516', '', 0, 4, '', '', '', -1, 0, 0, 1, 0, '', 'epoch', false, 0, '2019-02-14 08:07:31.028103+00', '2019-02-14 08:07:31.108963+00', 'epoch', 'epoch', NULL, NULL, NULL, false);
INSERT INTO "users"("id", "full_name", "short_name", "email", "normalized_email", "password_hash", "status", "partner_id", "created_at", "is_professional", "project_limit", "project_bandwidth_limit", "project_storage_limit", "paid_tier", "project_segment_limit") 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', false, 10, 50000000000, 50000000000, false, 150000);
INSERT INTO "users"("id", "full_name", "short_name", "email", "normalized_email", "password_hash", "status", "partner_id", "created_at", "position", "company_name", "working_on", "company_size", "is_professional", "employee_count", "project_limit", "project_bandwidth_limit", "project_storage_limit", "have_sales_contact", "project_segment_limit") VALUES (E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\304\\313\\206\\311",'::bytea, 'Ian', 'Pires', '3email3@mail.test', '3EMAIL3@MAIL.TEST', E'some_readable_hash'::bytea, 2, NULL, '2020-03-18 10:28:24.614594+00', 'engineer', 'storj', 'data storage', 51, true, '1-50', 10, 50000000000, 50000000000, true, 150000);
INSERT INTO "users"("id", "full_name", "short_name", "email", "normalized_email", "password_hash", "status", "partner_id", "created_at", "position", "company_name", "working_on", "company_size", "is_professional", "employee_count", "project_limit", "project_bandwidth_limit", "project_storage_limit", "project_segment_limit") VALUES (E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\205\\312",'::bytea, 'Campbell', 'Wright', '4email4@mail.test', '4EMAIL4@MAIL.TEST', E'some_readable_hash'::bytea, 2, NULL, '2020-07-17 10:28:24.614594+00', 'engineer', 'storj', 'data storage', 82, true, '1-50', 10, 50000000000, 50000000000, 150000);
INSERT INTO "users"("id", "full_name", "short_name", "email", "normalized_email", "password_hash", "status", "partner_id", "created_at", "position", "company_name", "working_on", "company_size", "is_professional", "project_limit", "project_bandwidth_limit", "project_storage_limit", "paid_tier", "mfa_enabled", "mfa_secret_key", "mfa_recovery_codes", "project_segment_limit") VALUES (E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\205\\311",'::bytea, 'Thierry', 'Berg', '2email2@mail.test', '2EMAIL2@MAIL.TEST', E'some_readable_hash'::bytea, 2, NULL, '2020-05-16 10:28:24.614594+00', 'engineer', 'storj', 'data storage', 55, true, 10, 50000000000, 50000000000, false, false, NULL, NULL, 150000);
INSERT INTO "projects"("id", "name", "description", "usage_limit", "bandwidth_limit", "max_buckets", "partner_id", "owner_id", "created_at", "segment_limit") VALUES (E'\\022\\217/\\014\\376!K\\023\\276\\031\\311}m\\236\\205\\300'::bytea, 'ProjectName', 'projects description', 5e11, 5e11, NULL, NULL, E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, '2019-02-14 08:28:24.254934+00', 150000);
INSERT INTO "projects"("id", "name", "description", "usage_limit", "bandwidth_limit", "max_buckets", "partner_id", "owner_id", "created_at", "segment_limit") VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\014'::bytea, 'projName1', 'Test project 1', 5e11, 5e11, NULL, NULL, E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, '2019-02-14 08:28:24.636949+00', 150000);
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 "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 "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' AT TIME ZONE current_setting('TIMEZONE'), 3600, 1, 1024, 2024);
INSERT INTO "storagenode_storage_tallies" VALUES (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' AT TIME ZONE current_setting('TIMEZONE'), 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' AT TIME ZONE current_setting('TIMEZONE'), 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' AT TIME ZONE current_setting('TIMEZONE'), 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' AT TIME ZONE current_setting('TIMEZONE'), 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 "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 "value_attributions" ("project_id", "bucket_name", "partner_id", "user_agent", "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, NULL, '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 "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+00');
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 "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 "stripecoinpayments_tx_conversion_rates" ("tx_id", "rate_numeric", "created_at") VALUES ('tx_id', '1.929883831', '2019-06-01 08:28:24.267934+00');
INSERT INTO "coinpayments_transactions" ("id", "user_id", "address", "amount_numeric", "received_numeric", "status", "key", "timeout", "created_at") VALUES ('tx_id', E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, 'address', 1411112222, 1311112222, 1, 'key', 60, '2019-06-01 08:28:24.267934+00');
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' AT TIME ZONE current_setting('TIMEZONE'), 3600, 1, 2024);
INSERT INTO "coupons" ("id", "user_id", "amount", "description", "type", "status", "duration", "billing_periods", "created_at") VALUES (E'\\362\\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, 2, '2019-06-01 08:28:24.267934+00');
INSERT INTO "coupons" ("id", "user_id", "amount", "description", "type", "status", "duration", "billing_periods", "created_at") VALUES (E'\\362\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\012'::bytea, E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, 50, 'description', 0, 0, 2, 2, '2019-06-01 08:28:24.267934+00');
INSERT INTO "coupons" ("id", "user_id", "amount", "description", "type", "status", "duration", "billing_periods", "created_at") VALUES (E'\\362\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\015'::bytea, E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, 50, 'description', 0, 0, 2, 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 "coupon_codes" ("id", "name", "amount", "description", "type", "billing_periods", "created_at") VALUES (E'\\362\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\014'::bytea, 'STORJ50', 50, '$50 for your first 5 months', 0, NULL, '2019-06-01 08:28:24.267934+00');
INSERT INTO "coupon_codes" ("id", "name", "amount", "description", "type", "billing_periods", "created_at") VALUES (E'\\362\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\015'::bytea, 'STORJ75', 75, '$75 for your first 5 months', 0, 2, '2019-06-01 08:28:24.267934+00');
INSERT INTO "stripecoinpayments_apply_balance_intents" ("tx_id", "state", "created_at") VALUES ('tx_id', 0, '2019-06-01 08:28:24.267934+00');
INSERT INTO "projects"("id", "name", "description", "usage_limit", "bandwidth_limit", "max_buckets", "rate_limit", "partner_id", "owner_id", "created_at", "segment_limit") VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\347'::bytea, 'projName1', 'Test project 1', 5e11, 5e11, NULL, 2000000, NULL, E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, '2020-01-15 08:28:24.636949+00', 150000);
INSERT INTO "project_bandwidth_rollups"("project_id", "interval_month", egress_allocated) VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\347'::bytea, '2020-04-01', 10000);
INSERT INTO "project_bandwidth_daily_rollups"("project_id", "interval_day", egress_allocated, egress_settled, egress_dead) VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\347'::bytea, '2021-04-22', 10000, 5000, 0);
INSERT INTO "projects"("id", "name", "description", "usage_limit", "bandwidth_limit", "max_buckets","rate_limit", "partner_id", "owner_id", "created_at", "segment_limit") VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\345'::bytea, 'egress101', 'High Bandwidth Project', 5e11, 5e11, NULL, 2000000, NULL, E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, '2020-05-15 08:46:24.000000+00', 150000);
INSERT INTO "storagenode_paystubs"("period", "node_id", "created_at", "codes", "usage_at_rest", "usage_get", "usage_put", "usage_get_repair", "usage_put_repair", "usage_get_audit", "comp_at_rest", "comp_get", "comp_put", "comp_get_repair", "comp_put_repair", "comp_get_audit", "surge_percent", "held", "owed", "disposed", "paid", "distributed") VALUES ('2020-01', '\xf2a3b4c4dfdf7221310382fd5db5aa73e1d227d6df09734ec4e5305000000000', '2020-04-07T20:14:21.479141Z', '', 1327959864508416, 294054066688, 159031363328, 226751, 0, 836608, 2861984, 5881081, 0, 226751, 0, 8, 300, 0, 26909472, 0, 26909472, 0);
INSERT INTO "nodes"("id", "address", "last_net", "protocol", "type", "email", "wallet", "free_disk", "piece_count", "major", "minor", "patch", "hash", "timestamp", "release","latency_90","created_at", "updated_at", "last_contact_success", "last_contact_failure", "disqualified", "disqualification_reason", "suspended", "exit_success", "unknown_audit_suspended", "offline_suspended", "under_review") VALUES (E'\\153\\313\\233\\074\\327\\255\\136\\070\\346\\001', '127.0.0.1:55516', '', 0, 4, '', '', -1, 0, 0, 1, 0, '', 'epoch', false, 0, '2019-02-14 08:07:31.028103+00', '2019-02-14 08:07:31.108963+00', 'epoch', 'epoch', NULL, NULL, NULL, false, '2019-02-14 08:07:31.108963+00', '2019-02-14 08:07:31.108963+00', '2019-02-14 08:07:31.108963+00');
INSERT INTO "node_api_versions"("id", "api_version", "created_at", "updated_at") VALUES (E'\\153\\313\\233\\074\\327\\177\\136\\070\\346\\001', 1, '2019-02-14 08:07:31.028103+00', '2019-02-14 08:07:31.108963+00');
INSERT INTO "node_api_versions"("id", "api_version", "created_at", "updated_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', 2, '2019-02-14 08:07:31.028103+00', '2019-02-14 08:07:31.108963+00');
INSERT INTO "node_api_versions"("id", "api_version", "created_at", "updated_at") VALUES (E'\\363\\342\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\014', 3, '2019-02-14 08:07:31.028103+00', '2019-02-14 08:07:31.108963+00');
INSERT INTO "projects"("id", "name", "description", "usage_limit", "bandwidth_limit", "rate_limit", "partner_id", "owner_id", "created_at", "max_buckets", "segment_limit") VALUES (E'300\\273|\\342N\\347\\347\\363\\342\\363\\371>+F\\256\\263'::bytea, 'egress102', 'High Bandwidth Project 2', 5e11, 5e11, 2000000, NULL, E'265\\343U\\303\\312\\312\\363\\311\\033w\\222\\303Ci",'::bytea, '2020-05-15 08:46:24.000000+00', 1000, 150000);
INSERT INTO "projects"("id", "name", "description", "usage_limit", "bandwidth_limit", "rate_limit", "partner_id", "owner_id", "created_at", "max_buckets", "segment_limit") VALUES (E'300\\273|\\342N\\347\\347\\363\\342\\363\\371>+F\\255\\244'::bytea, 'egress103', 'High Bandwidth Project 3', 5e11, 5e11, 2000000, NULL, E'265\\343U\\303\\312\\312\\363\\311\\033w\\222\\303Ci",'::bytea, '2020-05-15 08:46:24.000000+00', 1000, 150000);
INSERT INTO "projects"("id", "name", "description", "usage_limit", "bandwidth_limit", "rate_limit", "partner_id", "owner_id", "created_at", "max_buckets", "segment_limit") VALUES (E'300\\273|\\342N\\347\\347\\363\\342\\363\\371>+F\\253\\231'::bytea, 'Limit Test 1', 'This project is above the default', 50000000001, 50000000001, 2000000, NULL, E'265\\343U\\303\\312\\312\\363\\311\\033w\\222\\303Ci",'::bytea, '2020-10-14 10:10:10.000000+00', 101, 150000);
INSERT INTO "projects"("id", "name", "description", "usage_limit", "bandwidth_limit", "rate_limit", "partner_id", "owner_id", "created_at", "max_buckets", "segment_limit") VALUES (E'300\\273|\\342N\\347\\347\\363\\342\\363\\371>+F\\252\\230'::bytea, 'Limit Test 2', 'This project is below the default', 5e11, 5e11, 2000000, NULL, E'265\\343U\\303\\312\\312\\363\\311\\033w\\222\\303Ci",'::bytea, '2020-10-14 10:10:11.000000+00', NULL, 150000);
INSERT INTO "storagenode_bandwidth_rollups_phase2" ("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' AT TIME ZONE current_setting('TIMEZONE'), 3600, 1, 1024, 2024);
INSERT INTO "storagenode_bandwidth_rollup_archives" ("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' AT TIME ZONE current_setting('TIMEZONE'), 3600, 1, 1024, 2024);
INSERT INTO "bucket_bandwidth_rollup_archives" ("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' AT TIME ZONE current_setting('TIMEZONE'), 3600, 1, 1024, 2024, 3024);
INSERT INTO "storagenode_paystubs"("period", "node_id", "created_at", "codes", "usage_at_rest", "usage_get", "usage_put", "usage_get_repair", "usage_put_repair", "usage_get_audit", "comp_at_rest", "comp_get", "comp_put", "comp_get_repair", "comp_put_repair", "comp_get_audit", "surge_percent", "held", "owed", "disposed", "paid", "distributed") VALUES ('2020-12', '\x1111111111111111111111111111111111111111111111111111111111111111', '2020-04-07T20:14:21.479141Z', '', 101, 102, 103, 104, 105, 106, 107, 108, 109, 110, 111, 112, 113, 114, 115, 116, 117, 117);
INSERT INTO "storagenode_payments"("id", "created_at", "period", "node_id", "amount") VALUES (1, '2020-04-07T20:14:21.479141Z', '2020-12', '\x1111111111111111111111111111111111111111111111111111111111111111', 117);
INSERT INTO "reputations"("id", "audit_success_count", "total_audit_count", "created_at", "updated_at", "disqualified", "suspended", "audit_reputation_alpha", "audit_reputation_beta", "unknown_audit_reputation_alpha", "unknown_audit_reputation_beta", "online_score", "audit_history") VALUES (E'\\153\\313\\233\\074\\327\\177\\136\\070\\346\\001', 0, 5, '2019-02-14 08:07:31.028103+00', '2019-02-14 08:07:31.108963+00', NULL, NULL, 50, 0, 1, 0, 1, '\x0a23736f2f6d616e792f69636f6e69632f70617468732f746f2f63686f6f73652f66726f6d120a0102030405060708090a');
INSERT INTO "graceful_exit_segment_transfer_queue" ("node_id", "stream_id", "position", "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'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, 10 , 8, 1.0, '2019-09-12 10:07:31.028103+00', '2019-09-12 10:07:32.028103+00', null, null, 0, '2019-09-12 10:07:33.028103+00', 0);
INSERT INTO "segment_pending_audits" ("node_id", "piece_id", "stripe_index", "share_size", "expected_share_hash", "reverify_count", "stream_id", position) 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, '\x010101', 1);
INSERT INTO "users"("id", "full_name", "short_name", "email", "normalized_email", "password_hash", "status", "partner_id", "created_at", "is_professional", "project_limit", "project_bandwidth_limit", "project_storage_limit", "paid_tier", "project_segment_limit") VALUES (E'\\363\\311\\033w\\222\\303Ci\\266\\342U\\303\\312\\204",'::bytea, 'Noahson', 'William', '100email1@mail.test', '100EMAIL1@MAIL.TEST', E'some_readable_hash'::bytea, 1, NULL, '2019-02-14 08:28:24.614594+00', false, 10, 100000000000000, 25000000000000, true, 100000000);
INSERT INTO "repair_queue" ("stream_id", "position", "attempted_at", "segment_health", "updated_at", "inserted_at") VALUES ('\x01', 1, null, 1, '2020-09-01 00:00:00.000000+00', '2021-09-01 00:00:00.000000+00');
INSERT INTO "users"("id", "full_name", "email", "normalized_email", "password_hash", "status", "created_at", "mfa_enabled", "mfa_secret_key", "mfa_recovery_codes", "project_limit", "project_bandwidth_limit", "project_storage_limit", "project_segment_limit") VALUES (E'\\363\\311\\033w\\222\\303Ci\\266\\344U\\303\\312\\204",'::bytea, 'Noahson William', '101email1@mail.test', '101EMAIL1@MAIL.TEST', E'some_readable_hash'::bytea, 1, '2019-02-14 08:28:24.614594+00', true, 'mfa secret key', '["1a2b3c4d","e5f6g7h8"]', 3, 50000000000, 50000000000, 150000);
INSERT INTO "projects"("id", "name", "description", "usage_limit", "bandwidth_limit", "rate_limit", "burst_limit", "partner_id", "owner_id", "created_at", "max_buckets", "segment_limit") VALUES (E'300\\273|\\342N\\347\\347\\363\\342\\363\\371>+F\\251\\247'::bytea, 'Limit Test 2', 'This project is below the default', 5e11, 5e11, 2000000, 4000000, NULL, E'265\\343U\\303\\312\\312\\363\\311\\033w\\222\\303Ci",'::bytea, '2020-10-14 10:10:11.000000+00', NULL, 150000);
INSERT INTO "users"("id", "full_name", "email", "normalized_email", "password_hash", "status", "created_at", "mfa_enabled", "mfa_secret_key", "mfa_recovery_codes", "signup_promo_code", "project_limit", "project_bandwidth_limit", "project_storage_limit", "project_segment_limit") VALUES (E'\\363\\311\\033w\\222\\303Ci\\266\\344U\\303\\312\\205",'::bytea, 'Felicia Smith', '99email1@mail.test', '99EMAIL1@MAIL.TEST', E'some_readable_hash'::bytea, 1, '2021-08-14 09:13:44.614594+00', true, 'mfa secret key', '["1a2b3c4d","e5f6d7h8"]', 'promo123', 3, 50000000000, 50000000000, 150000);
INSERT INTO "stripecoinpayments_invoice_project_records"("id", "project_id", "storage", "egress", "objects", "segments", "period_start", "period_end", "state", "created_at") VALUES (E'\\300\\217/\\014\\376!K\\023\\276\\031\\311}m\\236\\205\\300'::bytea, E'\\300\\217/\\014\\376!K\\023\\276\\031\\311}m\\236\\205\\300'::bytea, 0, 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 "nodes"("id", "address", "last_net", "protocol", "type", "email", "wallet", "free_disk", "piece_count", "major", "minor", "patch", "hash", "timestamp", "release","latency_90", "created_at", "updated_at", "last_contact_success", "last_contact_failure", "disqualified", "disqualification_reason", "suspended", "exit_success", "country_code") VALUES (E'\\153\\313\\233\\074\\327\\177\\136\\070\\346\\002', '127.0.0.1:55517', '', 0, 4, '', '', -1, 0, 0, 1, 0, '', 'epoch', false, 0, '2021-02-14 08:07:31.028103+00', '2021-02-14 08:07:31.108963+00', 'epoch', 'epoch', NULL, NULL, NULL, false, 'DE');
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", "placement") VALUES (E'\\144/\\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'testbucketotheruniquename'::bytea, NULL, '2019-06-14 08:28:24.677953+00', 1, 65536, 1, 8192, 1, 4096, 4, 6, 8, 10, 1);
INSERT INTO "nodes"("id", "address", "last_net", "protocol", "type", "email", "wallet", "wallet_features", "free_disk", "piece_count", "major", "minor", "patch", "hash", "timestamp", "release","latency_90","created_at", "updated_at", "last_contact_success", "last_contact_failure", "disqualified", "disqualification_reason", "suspended", "exit_success", "country_code") VALUES (E'\\362\\341\\363\\371>+F\\256\\263\\300\\273|\\342N\\347\\017', '127.0.0.1:55517', '', 0, 4, '', '', '', -1, 0, 0, 1, 0, '', 'epoch', false, 0, '2020-02-14 08:07:31.028103+00', '2021-10-13 08:07:31.108963+00', 'epoch', 'epoch', '2021-10-13 08:07:31.108963+00', 0, NULL, false, NULL);
INSERT INTO "users"("id", "full_name", "email", "normalized_email", "password_hash", "status", "created_at", "mfa_enabled", "mfa_secret_key", "mfa_recovery_codes", "signup_promo_code", "project_limit", "project_bandwidth_limit", "project_storage_limit", "project_segment_limit") VALUES (E'\\363\\311\\033w\\222\\303Ci\\267\\342U\\303\\312\\203",'::bytea, 'Jessica Thompson', '143email1@mail.test', '143EMAIL1@MAIL.TEST', E'some_readable_hash'::bytea, 1, '2021-11-04 08:27:56.614594+00', true, 'mfa secret key', '["2b3c4d5e","f6a7e8e9"]', 'promo123', 3, '150000000000', '150000000000', 150000);
INSERT INTO "users"("id", "full_name", "email", "normalized_email", "password_hash", "status", "created_at", "mfa_enabled", "mfa_secret_key", "mfa_recovery_codes", "signup_promo_code", "project_limit", "project_bandwidth_limit", "project_storage_limit", "project_segment_limit") VALUES (E'\\363\\311\\033w\\222\\303Ci\\265\\342U\\303\\312\\202",'::bytea, 'Heather Jackson', '762email@mail.test', '762EMAIL1@MAIL.TEST', E'some_readable_hash'::bytea, 1, '2021-11-05 03:22:39.614594+00', true, 'mfa secret key', '["5e4d3c2b","e9e8a7f6"]', 'promo123', 3, '100000000000000', '25000000000000', 150000);
INSERT INTO "users"("id", "full_name", "email", "normalized_email", "password_hash", "status", "created_at", "mfa_enabled", "mfa_secret_key", "mfa_recovery_codes", "signup_promo_code", "project_limit", "project_bandwidth_limit", "project_storage_limit", "last_verification_reminder", "project_segment_limit") VALUES (E'\\364\\312\\033w\\222\\303Ci\\265\\342U\\303\\312\\202",'::bytea, 'Michael Mint', '333email2@mail.test', '333EMAIL2@MAIL.TEST', E'some_readable_hash'::bytea, 1, '2021-10-05 03:22:39.614594+00', true, 'mfa secret key', '["5e4d3c2c","e9e8a7f7"]', 'promo123', 3, '100000000000000', '25000000000000', '2021-12-05 03:22:39.614594+00', 150000);
INSERT INTO "oauth_clients"("id", "encrypted_secret", "redirect_url", "user_id", "app_name", "app_logo_url") VALUES (E'FD6209C0-7A17-4FC3-895C-E57A6C7CBBE1'::bytea, E'610B723B-E1FF-4B1D-B372-521250690C6E'::bytea, 'https://example.test/callback/storj', E'\\364\\312\\033w\\222\\303Ci\\265\\342U\\303\\312\\202",'::bytea, 'Example App', 'https://example.test/logo.png');
INSERT INTO "oauth_codes"("client_id", "user_id", "scope", "redirect_url", "challenge", "challenge_method", "code", "created_at", "expires_at", "claimed_at") VALUES (E'FD6209C0-7A17-4FC3-895C-E57A6C7CBBE1'::bytea, E'\\364\\312\\033w\\222\\303Ci\\265\\342U\\303\\312\\202",'::bytea, 'scope', 'http://localhost:12345/callback', 'challenge', 'challenge method', 'plaintext code', '2021-12-05 03:22:39.614594+00', '2021-12-05 03:22:39.614594+00', '2021-12-05 03:22:39.614594+00');
INSERT INTO "oauth_tokens"("client_id", "user_id", "scope", "kind", "token", "created_at", "expires_at") VALUES (E'FD6209C0-7A17-4FC3-895C-E57A6C7CBBE1'::bytea, E'\\364\\312\\033w\\222\\303Ci\\265\\342U\\303\\312\\202",'::bytea, 'scope', 1, E'B9C93D5F-CBD7-4615-9184-E714CFE14365'::bytea, '2021-12-05 03:22:39.614594+00', '2021-12-05 03:22:39.614594+00');
INSERT INTO "coinpayments_transactions" ("id", "user_id", "address", "amount_numeric", "received_numeric", "status", "key", "timeout", "created_at") VALUES ('different_tx_id_from_before', E'\\363\\311\\033w\\222\\303Ci\\265\\343U\\303\\312\\204",'::bytea, 'address', 125419938429, 1, 1, 'key', 60, '2021-07-28 20:24:11.932313-05');
INSERT INTO "stripecoinpayments_tx_conversion_rates" ("tx_id", "rate_numeric", "created_at") VALUES ('different_tx_id_from_before', 3.14159265359, '2021-07-28 20:24:11.932313-05');
-- NEW DATA --