78c6d5bb32
this commit introduces the reported_serials table. its purpose is to allow for blind writes into it as nodes report in so that we have minimal contention. in order to continue to accurately account for used bandwidth, though, we cannot immediately add the settled amount. if we did, we would have to give up on blind writes. the table's primary key is structured precisely so that we can quickly find expired orders and so that we maximally benefit from rocksdb path prefix compression. we do this by rounding the expires at time forward to the next day, effectively giving us storagenode petnames for free. and since there's no secondary index or foreign key constraints, this design should use significantly less space than the current used_serials table while also reducing contention. after inserting the orders into the table, we have a chore that periodically consumes all of the expired orders in it and inserts them into the existing rollups tables. this is as if we changed the nodes to report as the order expired rather than as soon as possible, so the belief in correctness of the refactor is higher. since we are able to process large batches of orders (typically a day's worth), we can use the code to maximally batch inserts into the rollup tables to make inserts as friendly as possible to cockroach. Change-Id: I25d609ca2679b8331979184f16c6d46d4f74c1a6
280 lines
7.9 KiB
Go
280 lines
7.9 KiB
Go
// Copyright (C) 2019 Storj Labs, Inc.
|
|
// See LICENSE for copying information.
|
|
|
|
package orders_test
|
|
|
|
import (
|
|
"context"
|
|
"fmt"
|
|
"testing"
|
|
"time"
|
|
|
|
"github.com/skyrings/skyring-common/tools/uuid"
|
|
"github.com/stretchr/testify/assert"
|
|
"github.com/stretchr/testify/require"
|
|
"go.uber.org/zap/zaptest"
|
|
|
|
"storj.io/common/memory"
|
|
"storj.io/common/pb"
|
|
"storj.io/common/testcontext"
|
|
"storj.io/common/testrand"
|
|
"storj.io/storj/private/testplanet"
|
|
"storj.io/storj/satellite"
|
|
"storj.io/storj/satellite/accounting"
|
|
"storj.io/storj/satellite/orders"
|
|
"storj.io/storj/satellite/satellitedb/satellitedbtest"
|
|
)
|
|
|
|
// unfortunately, in GB is apparently the only way we can get this data, so we need to
|
|
// arrange for the test to produce a total value that won't lose too much precision
|
|
// in the conversion to GB.
|
|
func getTotalBandwidthInGB(ctx context.Context, accountingDB accounting.ProjectAccounting, projectID uuid.UUID, since time.Time) (int64, error) {
|
|
total, err := accountingDB.GetAllocatedBandwidthTotal(ctx, projectID, since.Add(-time.Hour))
|
|
if err != nil {
|
|
return 0, err
|
|
}
|
|
return total, nil
|
|
}
|
|
|
|
// TestOrdersWriteCacheBatchLimitReached makes sure bandwidth rollup values are not written to the
|
|
// db until the batch size is reached.
|
|
func TestOrdersWriteCacheBatchLimitReached(t *testing.T) {
|
|
satellitedbtest.Run(t, func(t *testing.T, db satellite.DB) {
|
|
ctx := testcontext.New(t)
|
|
defer ctx.Cleanup()
|
|
|
|
useBatchSize := 10
|
|
amount := (memory.MB * 500).Int64()
|
|
projectID := testrand.UUID()
|
|
startTime := time.Now().UTC()
|
|
|
|
owc := orders.NewRollupsWriteCache(zaptest.NewLogger(t), db.Orders(), useBatchSize)
|
|
|
|
accountingDB := db.ProjectAccounting()
|
|
|
|
// use different bucketName for each write, so they don't get aggregated yet
|
|
for i := 0; i < useBatchSize-1; i++ {
|
|
bucketName := fmt.Sprintf("my_files_%d", i)
|
|
err := owc.UpdateBucketBandwidthAllocation(ctx, projectID, []byte(bucketName), pb.PieceAction_GET, amount, startTime)
|
|
require.NoError(t, err)
|
|
|
|
// check that nothing was actually written since it should just be stored
|
|
total, err := getTotalBandwidthInGB(ctx, accountingDB, projectID, startTime)
|
|
require.NoError(t, err)
|
|
require.Equal(t, int64(0), total)
|
|
}
|
|
|
|
whenDone := owc.OnNextFlush()
|
|
// write one more rollup record to hit the threshold
|
|
err := owc.UpdateBucketBandwidthAllocation(ctx, projectID, []byte("my_files_last"), pb.PieceAction_GET, amount, startTime)
|
|
require.NoError(t, err)
|
|
|
|
// make sure flushing is done
|
|
select {
|
|
case <-whenDone:
|
|
break
|
|
case <-ctx.Done():
|
|
t.Fatal(ctx.Err())
|
|
}
|
|
|
|
total, err := getTotalBandwidthInGB(ctx, accountingDB, projectID, startTime)
|
|
require.NoError(t, err)
|
|
require.Equal(t, amount*int64(useBatchSize), total)
|
|
})
|
|
}
|
|
|
|
// TestOrdersWriteCacheBatchChore makes sure bandwidth rollup values are not written to the
|
|
// db until the chore flushes the DB (assuming the batch size is not reached).
|
|
func TestOrdersWriteCacheBatchChore(t *testing.T) {
|
|
testplanet.Run(t, testplanet.Config{
|
|
SatelliteCount: 1,
|
|
},
|
|
func(t *testing.T, ctx *testcontext.Context, planet *testplanet.Planet) {
|
|
useBatchSize := 10
|
|
amount := (memory.MB * 500).Int64()
|
|
projectID := testrand.UUID()
|
|
startTime := time.Now().UTC()
|
|
|
|
planet.Satellites[0].Orders.Chore.Loop.Pause()
|
|
|
|
accountingDB := planet.Satellites[0].DB.ProjectAccounting()
|
|
ordersDB := planet.Satellites[0].Orders.DB
|
|
|
|
// use different pieceAction for each write, so they don't get aggregated yet
|
|
for i := 0; i < useBatchSize-1; i++ {
|
|
bucketName := fmt.Sprintf("my_files_%d", i)
|
|
err := ordersDB.UpdateBucketBandwidthAllocation(ctx, projectID, []byte(bucketName), pb.PieceAction_GET, amount, startTime)
|
|
require.NoError(t, err)
|
|
|
|
// check that nothing was actually written
|
|
total, err := getTotalBandwidthInGB(ctx, accountingDB, projectID, startTime)
|
|
require.NoError(t, err)
|
|
require.Equal(t, int64(0), total)
|
|
}
|
|
|
|
owc := ordersDB.(*orders.RollupsWriteCache)
|
|
whenDone := owc.OnNextFlush()
|
|
// wait for Loop to complete
|
|
planet.Satellites[0].Orders.Chore.Loop.Restart()
|
|
planet.Satellites[0].Orders.Chore.Loop.TriggerWait()
|
|
|
|
// make sure flushing is done
|
|
select {
|
|
case <-whenDone:
|
|
break
|
|
case <-ctx.Done():
|
|
t.Fatal(ctx.Err())
|
|
}
|
|
|
|
total, err := getTotalBandwidthInGB(ctx, accountingDB, projectID, startTime)
|
|
require.NoError(t, err)
|
|
require.Equal(t, amount*int64(useBatchSize-1), total)
|
|
},
|
|
)
|
|
}
|
|
|
|
func TestUpdateBucketBandwidthAllocation(t *testing.T) {
|
|
testplanet.Run(t, testplanet.Config{
|
|
SatelliteCount: 1,
|
|
},
|
|
func(t *testing.T, ctx *testcontext.Context, planet *testplanet.Planet) {
|
|
// don't let the loop flush our cache while we're checking it
|
|
planet.Satellites[0].Orders.Chore.Loop.Pause()
|
|
ordersDB := planet.Satellites[0].Orders.DB
|
|
|
|
// setup: check there is nothing in the cache to start
|
|
cache, ok := ordersDB.(*orders.RollupsWriteCache)
|
|
require.True(t, ok)
|
|
size := cache.CurrentSize()
|
|
require.Equal(t, size, 0)
|
|
|
|
// setup: add one item to the cache
|
|
projectID := testrand.UUID()
|
|
bucketName := []byte("testbucketname")
|
|
amount := (memory.MB * 500).Int64()
|
|
err := ordersDB.UpdateBucketBandwidthAllocation(ctx, projectID, bucketName, pb.PieceAction_GET, amount, time.Now().UTC())
|
|
require.NoError(t, err)
|
|
|
|
// test: confirm there is one item in the cache now
|
|
size = cache.CurrentSize()
|
|
require.Equal(t, size, 1)
|
|
projectMap := cache.CurrentData()
|
|
expectedKey := orders.CacheKey{
|
|
ProjectID: projectID,
|
|
BucketName: string(bucketName),
|
|
Action: pb.PieceAction_GET,
|
|
}
|
|
expected := orders.RollupData{
|
|
expectedKey: {
|
|
Inline: 0,
|
|
Allocated: amount,
|
|
},
|
|
}
|
|
require.Equal(t, projectMap, expected)
|
|
|
|
// setup: add another item to the cache but with a different projectID
|
|
projectID2 := testrand.UUID()
|
|
amount2 := (memory.MB * 10).Int64()
|
|
err = ordersDB.UpdateBucketBandwidthAllocation(ctx, projectID2, bucketName, pb.PieceAction_GET, amount2, time.Now().UTC())
|
|
require.NoError(t, err)
|
|
size = cache.CurrentSize()
|
|
require.Equal(t, size, 2)
|
|
projectMap2 := cache.CurrentData()
|
|
// test: confirm there are two items in the cache now for different projectIDs
|
|
|
|
expectedKey = orders.CacheKey{
|
|
ProjectID: projectID2,
|
|
BucketName: string(bucketName),
|
|
Action: pb.PieceAction_GET,
|
|
}
|
|
expected[expectedKey] = orders.CacheData{
|
|
Inline: 0,
|
|
Allocated: amount2,
|
|
}
|
|
require.Equal(t, projectMap2, expected)
|
|
},
|
|
)
|
|
}
|
|
|
|
func TestSortRollups(t *testing.T) {
|
|
rollups := []orders.BucketBandwidthRollup{
|
|
{
|
|
ProjectID: uuid.UUID{1},
|
|
BucketName: "a",
|
|
Action: pb.PieceAction_GET, // GET is 2
|
|
Inline: 1,
|
|
Allocated: 2,
|
|
},
|
|
{
|
|
ProjectID: uuid.UUID{2},
|
|
BucketName: "a",
|
|
Action: pb.PieceAction_GET,
|
|
Inline: 1,
|
|
Allocated: 2,
|
|
},
|
|
{
|
|
ProjectID: uuid.UUID{1},
|
|
BucketName: "b",
|
|
Action: pb.PieceAction_GET,
|
|
Inline: 1,
|
|
Allocated: 2,
|
|
},
|
|
{
|
|
ProjectID: uuid.UUID{1},
|
|
BucketName: "a",
|
|
Action: pb.PieceAction_GET_AUDIT,
|
|
Inline: 1,
|
|
Allocated: 2,
|
|
},
|
|
{
|
|
ProjectID: uuid.UUID{1},
|
|
BucketName: "a",
|
|
Action: pb.PieceAction_GET,
|
|
Inline: 1,
|
|
Allocated: 2,
|
|
},
|
|
}
|
|
|
|
expRollups := []orders.BucketBandwidthRollup{
|
|
{
|
|
ProjectID: uuid.UUID{1},
|
|
BucketName: "a",
|
|
Action: pb.PieceAction_GET, // GET is 2
|
|
Inline: 1,
|
|
Allocated: 2,
|
|
},
|
|
{
|
|
ProjectID: uuid.UUID{1},
|
|
BucketName: "a",
|
|
Action: pb.PieceAction_GET,
|
|
Inline: 1,
|
|
Allocated: 2,
|
|
},
|
|
{
|
|
ProjectID: uuid.UUID{1},
|
|
BucketName: "a",
|
|
Action: pb.PieceAction_GET_AUDIT,
|
|
Inline: 1,
|
|
Allocated: 2,
|
|
},
|
|
{
|
|
ProjectID: uuid.UUID{1},
|
|
BucketName: "b",
|
|
Action: pb.PieceAction_GET,
|
|
Inline: 1,
|
|
Allocated: 2,
|
|
},
|
|
{
|
|
ProjectID: uuid.UUID{2},
|
|
BucketName: "a",
|
|
Action: pb.PieceAction_GET,
|
|
Inline: 1,
|
|
Allocated: 2,
|
|
},
|
|
}
|
|
|
|
assert.NotEqual(t, expRollups, rollups)
|
|
orders.SortBucketBandwidthRollups(rollups)
|
|
assert.Equal(t, expRollups, rollups)
|
|
}
|