storj/satellite/orders/orders_test.go
Jeff Wendling 78c6d5bb32 satellite/satellitedb: reported_serials table for processing orders
this commit introduces the reported_serials table. its purpose is
to allow for blind writes into it as nodes report in so that we have
minimal contention. in order to continue to accurately account for
used bandwidth, though, we cannot immediately add the settled amount.
if we did, we would have to give up on blind writes.

the table's primary key is structured precisely so that we can quickly
find expired orders and so that we maximally benefit from rocksdb
path prefix compression. we do this by rounding the expires at time
forward to the next day, effectively giving us storagenode petnames
for free. and since there's no secondary index or foreign key
constraints, this design should use significantly less space than
the current used_serials table while also reducing contention.

after inserting the orders into the table, we have a chore that
periodically consumes all of the expired orders in it and inserts
them into the existing rollups tables. this is as if we changed
the nodes to report as the order expired rather than as soon as
possible, so the belief in correctness of the refactor is higher.

since we are able to process large batches of orders (typically
a day's worth), we can use the code to maximally batch inserts into
the rollup tables to make inserts as friendly as possible to
cockroach.

Change-Id: I25d609ca2679b8331979184f16c6d46d4f74c1a6
2020-01-15 19:21:21 -07:00

284 lines
8.9 KiB
Go

// Copyright (C) 2019 Storj Labs, Inc.
// See LICENSE for copying information.
package orders_test
import (
"context"
"strconv"
"testing"
"time"
"github.com/skyrings/skyring-common/tools/uuid"
"github.com/stretchr/testify/assert"
"github.com/stretchr/testify/require"
"storj.io/common/memory"
"storj.io/common/pb"
"storj.io/common/storj"
"storj.io/common/testcontext"
"storj.io/common/testrand"
"storj.io/storj/private/testplanet"
"storj.io/storj/satellite"
"storj.io/storj/satellite/orders"
"storj.io/storj/satellite/satellitedb/satellitedbtest"
)
func TestSendingReceivingOrders(t *testing.T) {
// test happy path
testplanet.Run(t, testplanet.Config{
SatelliteCount: 1, StorageNodeCount: 6, UplinkCount: 1,
}, func(t *testing.T, ctx *testcontext.Context, planet *testplanet.Planet) {
planet.Satellites[0].Audit.Worker.Loop.Pause()
for _, storageNode := range planet.StorageNodes {
storageNode.Storage2.Orders.Sender.Pause()
}
expectedData := testrand.Bytes(50 * memory.KiB)
redundancy := noLongTailRedundancy(planet)
err := planet.Uplinks[0].UploadWithConfig(ctx, planet.Satellites[0], &redundancy, "testbucket", "test/path", expectedData)
require.NoError(t, err)
sumBeforeSend := 0
for _, storageNode := range planet.StorageNodes {
infos, err := storageNode.DB.Orders().ListUnsent(ctx, 10)
require.NoError(t, err)
sumBeforeSend += len(infos)
}
require.NotZero(t, sumBeforeSend)
sumUnsent := 0
sumArchived := 0
for _, storageNode := range planet.StorageNodes {
storageNode.Storage2.Orders.Sender.TriggerWait()
infos, err := storageNode.DB.Orders().ListUnsent(ctx, 10)
require.NoError(t, err)
sumUnsent += len(infos)
archivedInfos, err := storageNode.DB.Orders().ListArchived(ctx, sumBeforeSend)
require.NoError(t, err)
sumArchived += len(archivedInfos)
}
require.Zero(t, sumUnsent)
require.Equal(t, sumBeforeSend, sumArchived)
})
}
func TestUnableToSendOrders(t *testing.T) {
// test sending when satellite is unavailable
testplanet.Run(t, testplanet.Config{
SatelliteCount: 1, StorageNodeCount: 6, UplinkCount: 1,
}, func(t *testing.T, ctx *testcontext.Context, planet *testplanet.Planet) {
planet.Satellites[0].Audit.Worker.Loop.Pause()
for _, storageNode := range planet.StorageNodes {
storageNode.Storage2.Orders.Sender.Pause()
}
expectedData := testrand.Bytes(50 * memory.KiB)
redundancy := noLongTailRedundancy(planet)
err := planet.Uplinks[0].UploadWithConfig(ctx, planet.Satellites[0], &redundancy, "testbucket", "test/path", expectedData)
require.NoError(t, err)
sumBeforeSend := 0
for _, storageNode := range planet.StorageNodes {
infos, err := storageNode.DB.Orders().ListUnsent(ctx, 10)
require.NoError(t, err)
sumBeforeSend += len(infos)
}
require.NotZero(t, sumBeforeSend)
err = planet.StopPeer(planet.Satellites[0])
require.NoError(t, err)
sumUnsent := 0
sumArchived := 0
for _, storageNode := range planet.StorageNodes {
storageNode.Storage2.Orders.Sender.TriggerWait()
infos, err := storageNode.DB.Orders().ListUnsent(ctx, 10)
require.NoError(t, err)
sumUnsent += len(infos)
archivedInfos, err := storageNode.DB.Orders().ListArchived(ctx, sumBeforeSend)
require.NoError(t, err)
sumArchived += len(archivedInfos)
}
require.Zero(t, sumArchived)
require.Equal(t, sumBeforeSend, sumUnsent)
})
}
func TestUploadDownloadBandwidth(t *testing.T) {
testplanet.Run(t, testplanet.Config{
SatelliteCount: 1, StorageNodeCount: 6, UplinkCount: 1,
}, func(t *testing.T, ctx *testcontext.Context, planet *testplanet.Planet) {
wayInTheFuture := time.Now().UTC().Add(1000 * time.Hour)
hourBeforeTheFuture := wayInTheFuture.Add(-time.Hour)
planet.Satellites[0].Audit.Worker.Loop.Pause()
for _, storageNode := range planet.StorageNodes {
storageNode.Storage2.Orders.Sender.Pause()
}
expectedData := testrand.Bytes(50 * memory.KiB)
redundancy := noLongTailRedundancy(planet)
err := planet.Uplinks[0].UploadWithConfig(ctx, planet.Satellites[0], &redundancy, "testbucket", "test/path", expectedData)
require.NoError(t, err)
data, err := planet.Uplinks[0].Download(ctx, planet.Satellites[0], "testbucket", "test/path")
require.NoError(t, err)
require.Equal(t, expectedData, data)
//HACKFIX: We need enough time to pass after the download ends for storagenodes to save orders
time.Sleep(200 * time.Millisecond)
var expectedBucketBandwidth int64
expectedStorageBandwidth := make(map[storj.NodeID]int64)
for _, storageNode := range planet.StorageNodes {
infos, err := storageNode.DB.Orders().ListUnsent(ctx, 10)
require.NoError(t, err)
if len(infos) > 0 {
for _, info := range infos {
expectedBucketBandwidth += info.Order.Amount
expectedStorageBandwidth[storageNode.ID()] += info.Order.Amount
}
}
}
for _, storageNode := range planet.StorageNodes {
storageNode.Storage2.Orders.Sender.TriggerWait()
}
// Run the chore as if we were far in the future so that the orders are expired.
reportedRollupChore := planet.Satellites[0].Core.Accounting.ReportedRollupChore
require.NoError(t, reportedRollupChore.RunOnce(ctx, wayInTheFuture))
projects, err := planet.Satellites[0].DB.Console().Projects().GetAll(ctx)
require.NoError(t, err)
ordersDB := planet.Satellites[0].DB.Orders()
bucketBandwidth, err := ordersDB.GetBucketBandwidth(ctx, projects[0].ID, []byte("testbucket"), hourBeforeTheFuture, wayInTheFuture)
require.NoError(t, err)
require.Equal(t, expectedBucketBandwidth, bucketBandwidth)
for _, storageNode := range planet.StorageNodes {
nodeBandwidth, err := ordersDB.GetStorageNodeBandwidth(ctx, storageNode.ID(), hourBeforeTheFuture, wayInTheFuture)
require.NoError(t, err)
require.Equal(t, expectedStorageBandwidth[storageNode.ID()], nodeBandwidth)
}
})
}
func noLongTailRedundancy(planet *testplanet.Planet) storj.RedundancyScheme {
redundancy := planet.Uplinks[0].GetConfig(planet.Satellites[0]).GetRedundancyScheme()
redundancy.OptimalShares = redundancy.TotalShares
return redundancy
}
func TestSplitBucketIDInvalid(t *testing.T) {
var testCases = []struct {
name string
bucketID []byte
}{
{"invalid, not valid UUID", []byte("not UUID string/bucket1")},
{"invalid, not valid UUID, no bucket", []byte("not UUID string")},
{"invalid, no project, no bucket", []byte("")},
}
for _, tt := range testCases {
tt := tt // avoid scopelint error, ref: https://github.com/golangci/golangci-lint/issues/281
t.Run(tt.name, func(t *testing.T) {
_, _, err := orders.SplitBucketID(tt.bucketID)
assert.NotNil(t, err)
assert.Error(t, err)
})
}
}
func TestSplitBucketIDValid(t *testing.T) {
var testCases = []struct {
name string
project string
bucketName string
expectedBucketName string
}{
{"valid, no bucket, no objects", "bb6218e3-4b4a-4819-abbb-fa68538e33c0", "", ""},
{"valid, with bucket", "bb6218e3-4b4a-4819-abbb-fa68538e33c0", "testbucket", "testbucket"},
{"valid, with object", "bb6218e3-4b4a-4819-abbb-fa68538e33c0", "testbucket/foo/bar.txt", "testbucket"},
}
for _, tt := range testCases {
tt := tt // avoid scopelint error, ref: https://github.com/golangci/golangci-lint/issues/281
t.Run(tt.name, func(t *testing.T) {
expectedProjectID, err := uuid.Parse(tt.project)
assert.NoError(t, err)
bucketID := expectedProjectID.String() + "/" + tt.bucketName
actualProjectID, actualBucketName, err := orders.SplitBucketID([]byte(bucketID))
assert.NoError(t, err)
assert.Equal(t, actualProjectID, expectedProjectID)
assert.Equal(t, actualBucketName, []byte(tt.expectedBucketName))
})
}
}
func BenchmarkOrders(b *testing.B) {
ctx := testcontext.New(b)
defer ctx.Cleanup()
counts := []int{50, 100, 250, 500, 1000}
for _, c := range counts {
c := c
satellitedbtest.Bench(b, func(b *testing.B, db satellite.DB) {
snID := testrand.NodeID()
projectID, _ := uuid.New()
bucketID := []byte(projectID.String() + "/b")
b.Run("Benchmark Order Processing:"+strconv.Itoa(c), func(b *testing.B) {
ctx := context.Background()
for i := 0; i < b.N; i++ {
requests := buildBenchmarkData(ctx, b, db, snID, bucketID, c)
_, err := db.Orders().ProcessOrders(ctx, requests)
assert.NoError(b, err)
}
})
})
}
}
func buildBenchmarkData(ctx context.Context, b *testing.B, db satellite.DB, storageNodeID storj.NodeID, bucketID []byte, orderCount int) (_ []*orders.ProcessOrderRequest) {
requests := make([]*orders.ProcessOrderRequest, 0, orderCount)
for i := 0; i < orderCount; i++ {
snUUID, _ := uuid.New()
sn, err := storj.SerialNumberFromBytes(snUUID[:])
require.NoError(b, err)
err = db.Orders().CreateSerialInfo(ctx, sn, bucketID, time.Now().Add(time.Hour*24))
require.NoError(b, err)
order := &pb.Order{
SerialNumber: sn,
Amount: 1,
}
orderLimit := &pb.OrderLimit{
SerialNumber: sn,
StorageNodeId: storageNodeID,
Action: 2,
}
requests = append(requests, &orders.ProcessOrderRequest{Order: order,
OrderLimit: orderLimit})
}
return requests
}