From 5cc64fe016fedce1153c32fd3a0f1f45a295a68e Mon Sep 17 00:00:00 2001 From: John-Alan Simmons Date: Wed, 2 Feb 2022 00:05:50 -0500 Subject: [PATCH] tools: Created benchmark suite (#160) * Added bench readme * Started fixture framework to generate doc data * basic bench runner script. * started collection benchmark outline * Updated DB Collection client interface API * Added db cleanup routines * Switched from ResetTimer, to Start/Stop so it can be composable * Updated collection client API to include new context arg * updated gitignore to ignore benchmark artifacts * Updated test suite structure * Renamed bench file, added sync/async benchmarks * Reorganized benchmark folder to match db/tests structure. Removed old db-init files for benchmark * Reorganized collection tests into smaller units. Added async read tests. WIP: async backfill writes * Updated fixture data * Renamed get bench tests to read * Added Create/Write tests, updated utils structure, implemented async backfilling for benchmarks * Added CreateMany tests * Fixed issue where discarded transactions we're causing side-effect mutation on the document object * Fixed async transaction conflict issue * Started underlying storage engine benchmarks * Started put tests on storage, added more get tests * Updated collection benchmark utilities, extracted into dedicated package * Added new FieldDescription Meta data entry to indicate 'synthetic' internal IDs for managed related types * Added more query benchmarks, added query generation to fixtures package * Fixed tests from BREAKING change resulted from IPLD delta payload restructure * Removed auto query generation for fixtures, manual for now * Restructured benchmarks to further match db test structure. Removed opcount * Added filter query benchmarks * Added offset/limit + sort benchmarks. Removed old untracked deleted benchmark files * Updated DB and planner to expose make plan functions as public * Added benchmarks for planner/parser * Added query formatting with dockey replacement from inserted set * Added benchmarks for single and multi point lookups by dockey(s) * Fixed missing func args resulting from rebase * Added makefile * Added some comments to the async implementations. Also added a deterministic seed for the RNG system to produce consistent results * Updated CIDs from tests based on breaking change of IPLD format * Added put and put many benchmarks to storage * updated dependencies required via go mod tidy (CI issue) * Removing deadcode or useless comments * Replaced WithSchema with ForSchema * Renamed fixture.Context to fixture.Generator * Updated benchmark runner utils to take a context instead of creating a new one * Updated all references to benchmark runners for new context arg * Refactored StartTimer() calls and numType arg * removed dead/commented code * Updated public API for query planner * Added StopTimer() calls to add accuracy to the benchmarks * Updated benchmark loop to use ResetTimer for more accuracy * Moved valueSize array out * Replaced custom interface for TempDir with proper testing.TB interface * Minor comments and stuff * Moved the testutil NewDB related funcs from /tests to /bench * Removed rando comment * fixed linter errors BREAKING CHANGE: Changes internal IPLD format and resulting delta CIDs. --- .gitignore | 2 + bench/Makefile | 13 + bench/README.md | 26 ++ bench/bench_util.go | 227 ++++++++++++++++++ bench/collection/simple_create_many_test.go | 24 ++ bench/collection/simple_create_test.go | 72 ++++++ bench/collection/simple_read_test.go | 136 +++++++++++ bench/collection/utils.go | 218 +++++++++++++++++ bench/fixtures/data.go | 18 ++ bench/fixtures/fixtures.go | 101 ++++++++ bench/query/planner/simple_test.go | 38 +++ bench/query/planner/utils.go | 61 +++++ bench/query/simple/simple_test.go | 54 +++++ bench/query/simple/utils.go | 95 ++++++++ bench/query/simple/with_filter_test.go | 54 +++++ bench/query/simple/with_limit_offset_test.go | 54 +++++ bench/query/simple/with_multi_lookup_test.go | 47 ++++ bench/query/simple/with_single_lookup_test.go | 55 +++++ bench/query/simple/with_sort_test.go | 54 +++++ bench/run.sh | 0 bench/storage/get_test.go | 86 +++++++ bench/storage/put_many_test.go | 86 +++++++ bench/storage/put_test.go | 86 +++++++ bench/storage/utils.go | 149 ++++++++++++ client/core.go | 2 + core/crdt/lwwreg.go | 7 +- db/base/descriptions.go | 20 +- db/collection.go | 15 +- db/db.go | 9 +- db/tests/query/all_commits/simple_test.go | 10 +- db/tests/query/all_commits/with_count_test.go | 2 +- db/tests/query/commit/simple_test.go | 4 +- db/tests/query/latest_commits/simple_test.go | 6 +- db/tests/query/simple/with_version_test.go | 6 +- db/tests/utils.go | 67 ++++-- db/txn.go | 59 +++++ document/document.go | 12 + go.mod | 71 +++++- go.sum | 56 +---- query/graphql/planner/dagscan.go | 21 ++ query/graphql/planner/datasource.go | 3 +- query/graphql/planner/executor.go | 9 +- query/graphql/planner/planner.go | 5 +- query/graphql/planner/scan.go | 4 +- query/graphql/schema/descriptions.go | 41 ++++ query/graphql/schema/descriptions_test.go | 3 + query/graphql/schema/generate.go | 3 +- 47 files changed, 2077 insertions(+), 114 deletions(-) create mode 100644 bench/Makefile create mode 100644 bench/README.md create mode 100644 bench/bench_util.go create mode 100644 bench/collection/simple_create_many_test.go create mode 100644 bench/collection/simple_create_test.go create mode 100644 bench/collection/simple_read_test.go create mode 100644 bench/collection/utils.go create mode 100644 bench/fixtures/data.go create mode 100644 bench/fixtures/fixtures.go create mode 100644 bench/query/planner/simple_test.go create mode 100644 bench/query/planner/utils.go create mode 100644 bench/query/simple/simple_test.go create mode 100644 bench/query/simple/utils.go create mode 100644 bench/query/simple/with_filter_test.go create mode 100644 bench/query/simple/with_limit_offset_test.go create mode 100644 bench/query/simple/with_multi_lookup_test.go create mode 100644 bench/query/simple/with_single_lookup_test.go create mode 100644 bench/query/simple/with_sort_test.go create mode 100644 bench/run.sh create mode 100644 bench/storage/get_test.go create mode 100644 bench/storage/put_many_test.go create mode 100644 bench/storage/put_test.go create mode 100644 bench/storage/utils.go diff --git a/.gitignore b/.gitignore index 8030a7149c..ca76b542df 100644 --- a/.gitignore +++ b/.gitignore @@ -4,3 +4,5 @@ build/defradb* cover.out coverage-full.txt coverage-quick.txt +bench/*.log +bench/*.svg diff --git a/bench/Makefile b/bench/Makefile new file mode 100644 index 0000000000..2720a85f22 --- /dev/null +++ b/bench/Makefile @@ -0,0 +1,13 @@ +suite := all + +.PHONY: bench +bench: + $(info Running benchmark suite: $(suite)) + +ifneq (,$(findstring all,$(suite))) # Check if default ALL value is set + $(eval override suite = ...) +else ifneq ($(wildcard $(suite)/.),) # Check to make sure the requested suite exists + $(eval override suite = $(suite)/...) +endif + + go test -bench=. ./$(suite) \ No newline at end of file diff --git a/bench/README.md b/bench/README.md new file mode 100644 index 0000000000..940a040f14 --- /dev/null +++ b/bench/README.md @@ -0,0 +1,26 @@ +# DefraDB Benchmark Suite +This folder contains the DefraDB Benchmark Suite, its related code, sub packages, utilities, and data generators. + +The goal of this suite is to provide an insight to DefraDBs performance, and to provide a quantitative approach to performance analysis and comparison. As such, the benchmark results should be used soley as a relative basis, and not concrete absolute values. + +> Database benchmarking is a notorious complex issue to provide fair evaluations, that are void of contrived examples aimed to put the database "best foot forward". + +## Workflow +The main benchmark suite should ideally be run on every PR as a "check" and alerts/warnings should be posted with relative performance deviates too much from some given "baseline". + +There may be an additional `Nightly Suite` of benchmarks which would be run every night at XX:00 UTC which runs a more exhaustive benchmark, along with any tracing/analysis that would otherwise take too long to run within a PR check. + +## Planned Tests +Here's a breakdown of the various benchmarks to run. This suite needs to cover *some* unit level benchmarks (testing specific components in isolation), to allow more fine-grained insights into the performance analysis. However, the majority of the suite should cover full integration benchmarks, primarily through the Query interface. + +Some unit level benchmarks should be "cacheable" to avoid unnecessary overhead, we will try to identify which benchmarks *should* be cacheable. + +### Workloads + +#### Unit Benchmark + - Underlying KV Engine `cacheable` + - Read + - Write + - Mixed + +#### Integration Benchmark \ No newline at end of file diff --git a/bench/bench_util.go b/bench/bench_util.go new file mode 100644 index 0000000000..f98f5c3371 --- /dev/null +++ b/bench/bench_util.go @@ -0,0 +1,227 @@ +package bench + +import ( + "context" + "fmt" + "hash/fnv" + "math" + "math/rand" + "os" + "sync" + "testing" + + "github.com/dgraph-io/badger/v3" + ds "github.com/ipfs/go-datastore" + + "github.com/sourcenetwork/defradb/bench/fixtures" + "github.com/sourcenetwork/defradb/client" + "github.com/sourcenetwork/defradb/db" + defradb "github.com/sourcenetwork/defradb/db" + testutils "github.com/sourcenetwork/defradb/db/tests" + "github.com/sourcenetwork/defradb/document" + "github.com/sourcenetwork/defradb/document/key" +) + +const ( + writeBatchGroup = 100 + storageEnvName = "DEFRA_BENCH_STORAGE" +) + +var ( + storage string = "memory" +) + +func init() { + // create a consistent seed value for the random package + // so we dont have random fluctuations between runs + // (specifically thinking about the fixture generation stuff) + seed := hashToInt64("https://xkcd.com/221/") + rand.Seed(seed) + + // assign if not empty + if s := os.Getenv(storageEnvName); s != "" { + storage = s + } +} + +// hashToInt64 uses the FNV-1 hash to int +// algorithm +func hashToInt64(s string) int64 { + h := fnv.New64a() + h.Write([]byte(s)) + return int64(h.Sum64()) +} + +func SetupCollections(b *testing.B, ctx context.Context, db *defradb.DB, fixture fixtures.Generator) ([]client.Collection, error) { + numTypes := len(fixture.Types()) + collections := make([]client.Collection, numTypes) + var schema string + + // loop to get the schemas + for i := 0; i < numTypes; i++ { + gql, err := fixtures.ExtractGQLFromType(fixture.Types()[i]) + if err != nil { + return nil, fmt.Errorf("failed generating GQL: %w", err) + } + + schema += gql + schema += "\n\n" + } + + // b.Logf("Loading schema: \n%s", schema) + + if err := db.AddSchema(ctx, schema); err != nil { + return nil, fmt.Errorf("Couldn't load schema: %w", err) + } + + // loop to get collections + for i := 0; i < numTypes; i++ { + col, err := db.GetCollection(ctx, fixture.TypeName(i)) + if err != nil { + return nil, fmt.Errorf("Couldn't get the collection %v: %w", fixture.TypeName(i), err) + } + // b.Logf("Collection Name: %s", col.Name()) + collections[i] = col + } + + return collections, nil +} + +func SetupDBAndCollections(b *testing.B, ctx context.Context, fixture fixtures.Generator) (*defradb.DB, []client.Collection, error) { + db, err := NewTestDB(b) + if err != nil { + return nil, nil, err + } + + // create collections + collections, err := SetupCollections(b, ctx, db, fixture) + if err != nil { + return nil, nil, err + } + + return db, collections, nil + +} + +// Loads the given test database using the provided fixture context. +// It loads docCount number of documents asyncronously in batches of *upto* +// writeBatchGroup. +func BackfillBenchmarkDB(b *testing.B, ctx context.Context, cols []client.Collection, fixture fixtures.Generator, docCount, opCount int, doSync bool) ([][]key.DocKey, error) { + numTypes := len(fixture.Types()) + + // load fixtures + var wg sync.WaitGroup + wg.Add(docCount) + errCh := make(chan error) + waitCh := make(chan struct{}) + dockeys := make([][]key.DocKey, docCount) + + go func() { + // cut up the job from into writeBatchGroup size grouped jobs. + // Note weird math cus the last batch will likely be smaller then + // writeBatchGroup ~cus math~. + for bid := 0; float64(bid) < math.Ceil(float64(docCount)/writeBatchGroup); bid++ { + currentBatchSize := int(math.Min(float64((docCount - (bid * writeBatchGroup))), writeBatchGroup)) + var batchWg sync.WaitGroup + batchWg.Add(currentBatchSize) + + // spin up a goroutine for each doc in the current batch. + // wait for the entire batch to finish before moving on to + // the next batch + for i := 0; i < currentBatchSize; i++ { + go func(index int) { + docs, err := fixture.GenerateDocs() + if err != nil { + errCh <- fmt.Errorf("Failed to generate document payload from fixtures: %w", err) + return + } + + // fmt.Println(docs) + + // create the documents + keys := make([]key.DocKey, numTypes) + for j := 0; j < numTypes; j++ { + + doc, err := document.NewFromJSON([]byte(docs[j])) + if err != nil { + errCh <- fmt.Errorf("Failed to create document from fixture: %w", err) + return + } + + // loop forever untill commited. + // This was necessary when debugging and was left + // in place. The error check could prob use a wrap system + // but its fine :). + for { + if err := cols[j].Create(ctx, doc); err != nil && err.Error() == badger.ErrConflict.Error() { + fmt.Printf("failed to commit TX for doc %s, retrying...\n", doc.Key()) + continue + } else if err != nil { + errCh <- fmt.Errorf("Failed to create document: %w", err) + } + keys[j] = doc.Key() + break + } + } + dockeys[index] = keys + + wg.Done() + batchWg.Done() + }((bid * writeBatchGroup) + i) + } + + batchWg.Wait() + // fmt.Printf(".") + } + + // wait for our group and signal by closing waitCh + wg.Wait() + close(waitCh) + }() + + // finish or err + select { + case <-waitCh: + return dockeys, nil + case err := <-errCh: + return nil, err + } + +} + +type dbInfo interface { + Rootstore() ds.Batching + DB() *db.DB +} + +func NewTestDB(t testing.TB) (*db.DB, error) { + //nolint + dbi, err := newBenchStoreInfo(t) + return dbi.DB(), err +} + +func NewTestStorage(t testing.TB) (ds.Batching, error) { + dbi, err := newBenchStoreInfo(t) + return dbi.Rootstore(), err +} + +func newBenchStoreInfo(t testing.TB) (dbInfo, error) { + var dbi dbInfo + var err error + + switch storage { + case "memory": + dbi, err = testutils.NewBadgerMemoryDB() + case "badger": + dbi, err = testutils.NewBadgerFileDB(t) + case "memorymap": + dbi, err = testutils.NewMapDB() + default: + return nil, fmt.Errorf("invalid storage engine backend: %s", storage) + } + + if err != nil { + return nil, fmt.Errorf("Failed to create storage backend: %w", err) + } + return dbi, err +} diff --git a/bench/collection/simple_create_many_test.go b/bench/collection/simple_create_many_test.go new file mode 100644 index 0000000000..d51a069535 --- /dev/null +++ b/bench/collection/simple_create_many_test.go @@ -0,0 +1,24 @@ +package collection + +import ( + "context" + "testing" + + "github.com/sourcenetwork/defradb/bench/fixtures" +) + +func Benchmark_Collection_UserSimple_CreateMany_Sync_0_10(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchCreateMany(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 0, 10, true) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_CreateMany_Sync_0_100(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchCreateMany(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 0, 100, true) + if err != nil { + b.Fatal(err) + } +} diff --git a/bench/collection/simple_create_test.go b/bench/collection/simple_create_test.go new file mode 100644 index 0000000000..3ea48b1a01 --- /dev/null +++ b/bench/collection/simple_create_test.go @@ -0,0 +1,72 @@ +package collection + +import ( + "context" + "testing" + + "github.com/sourcenetwork/defradb/bench/fixtures" +) + +func Benchmark_Collection_UserSimple_Create_Sync_0_1(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchCreate(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 0, 1, true) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Create_Sync_0_10(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchCreate(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 0, 10, true) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Create_Sync_0_100(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchCreate(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 0, 100, true) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Create_Sync_0_1000(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchCreate(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 0, 1000, true) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Create_Async_0_1(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchCreate(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 0, 1, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Create_Async_0_100(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchCreate(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 0, 100, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Create_Async_0_1000(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchCreate(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 0, 1000, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Create_Async_0_100000(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchCreate(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 0, 100000, false) + if err != nil { + b.Fatal(err) + } +} diff --git a/bench/collection/simple_read_test.go b/bench/collection/simple_read_test.go new file mode 100644 index 0000000000..1172dc4866 --- /dev/null +++ b/bench/collection/simple_read_test.go @@ -0,0 +1,136 @@ +package collection + +import ( + "context" + "testing" + + "github.com/sourcenetwork/defradb/bench/fixtures" +) + +func Benchmark_Collection_UserSimple_Read_Sync_1_1(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1, 1, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Read_Sync_10_10(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 10, 10, true) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Read_Sync_100_100(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 100, 100, true) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Read_Sync_1000_1000(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1000, 1000, true) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Read_Sync_1000_1(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1000, 1, true) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Read_Sync_10000_1(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 10000, 1, true) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Read_Sync_100000_1(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 100000, 1, true) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Read_Sync_1000_10(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1000, 10, true) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Read_Sync_1000_100(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1000, 100, true) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Read_Async_1_1(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1, 1, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Read_Async_10_10(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 10, 10, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Read_Async_100_100(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 100, 100, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Read_Async_1000_1000(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1000, 1000, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Read_Async_1000_1(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1000, 1, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Read_Async_1000_10(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1000, 10, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Collection_UserSimple_Read_Async_1000_100(b *testing.B) { + ctx := context.Background() + err := runCollectionBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1000, 100, false) + if err != nil { + b.Fatal(err) + } +} diff --git a/bench/collection/utils.go b/bench/collection/utils.go new file mode 100644 index 0000000000..36179990e9 --- /dev/null +++ b/bench/collection/utils.go @@ -0,0 +1,218 @@ +package collection + +import ( + "context" + "fmt" + "math" + "sync" + "testing" + + benchutils "github.com/sourcenetwork/defradb/bench" + "github.com/sourcenetwork/defradb/bench/fixtures" + "github.com/sourcenetwork/defradb/client" + "github.com/sourcenetwork/defradb/document" + "github.com/sourcenetwork/defradb/document/key" +) + +const ( + writeBatchGroup = 100 +) + +func runCollectionBenchGet(b *testing.B, ctx context.Context, fixture fixtures.Generator, docCount, opCount int, doSync bool) error { + db, collections, err := benchutils.SetupDBAndCollections(b, ctx, fixture) + if err != nil { + return err + } + defer db.Close() + + dockeys, err := benchutils.BackfillBenchmarkDB(b, ctx, collections, fixture, docCount, opCount, doSync) + if err != nil { + return err + } + + // fmt.Println("Finished backfill...") + + // run benchmark + if doSync { + return runCollectionBenchGetSync(b, ctx, collections, fixture, docCount, opCount, dockeys) + } + return runCollectionBenchGetAsync(b, ctx, collections, fixture, docCount, opCount, dockeys) +} + +func runCollectionBenchGetSync(b *testing.B, + ctx context.Context, + collections []client.Collection, + fixture fixtures.Generator, + docCount, opCount int, + dockeys [][]key.DocKey, +) error { + numTypes := len(fixture.Types()) + b.ResetTimer() + for i := 0; i < b.N; i++ { // outer benchmark loop + for j := 0; j < opCount/numTypes; j++ { // number of Get operations we want to execute + for k := 0; k < numTypes; k++ { // apply op to all the related types + collections[k].Get(ctx, dockeys[j][k]) //nolint + } + } + } + b.StopTimer() + + return nil +} + +// pretty basic async loop, one goroutine for +// each operation we need to do +func runCollectionBenchGetAsync(b *testing.B, + ctx context.Context, + collections []client.Collection, + fixture fixtures.Generator, + docCount, opCount int, + dockeys [][]key.DocKey, +) error { + var wg sync.WaitGroup + numTypes := len(fixture.Types()) + b.ResetTimer() + for i := 0; i < b.N; i++ { // outer benchmark loop + for j := 0; j < opCount/numTypes; j++ { // number of Get operations we want to execute + for k := 0; k < numTypes; k++ { // apply op to all the related types + wg.Add(1) + go func(ctx context.Context, col client.Collection, dockey key.DocKey) { + col.Get(ctx, dockey) //nolint + wg.Done() + }(ctx, collections[k], dockeys[j][k]) + } + } + + wg.Wait() + } + b.StopTimer() + + return nil +} + +func runCollectionBenchCreate(b *testing.B, ctx context.Context, fixture fixtures.Generator, docCount, opCount int, doSync bool) error { + b.StopTimer() + db, collections, err := benchutils.SetupDBAndCollections(b, ctx, fixture) + if err != nil { + return err + } + defer db.Close() + + _, err = benchutils.BackfillBenchmarkDB(b, ctx, collections, fixture, docCount, opCount, doSync) + if err != nil { + return err + } + + // run benchmark + b.StartTimer() + if doSync { + return runCollectionBenchCreateSync(b, ctx, collections, fixture, docCount, opCount) + } + return runCollectionBenchCreateAsync(b, ctx, collections, fixture, docCount, opCount) +} + +func runCollectionBenchCreateMany(b *testing.B, ctx context.Context, fixture fixtures.Generator, docCount, opCount int, doSync bool) error { + db, collections, err := benchutils.SetupDBAndCollections(b, ctx, fixture) + if err != nil { + return err + } + defer db.Close() + + _, err = benchutils.BackfillBenchmarkDB(b, ctx, collections, fixture, docCount, opCount, doSync) + if err != nil { + return err + } + + numTypes := len(fixture.Types()) + // CreateMany make sure numTypes == 1 since we only support that for now + // @todo: Add support for numTypes > 1 later + if numTypes != 1 { + return fmt.Errorf("Invalid number of types for create many, have %v but max is 1", numTypes) + } + + // run benchmark + + b.ResetTimer() + for i := 0; i < b.N; i++ { + docs := make([]*document.Document, opCount) + for j := 0; j < opCount; j++ { + d, _ := fixture.GenerateDocs() + docs[j], _ = document.NewFromJSON([]byte(d[0])) + } + + collections[0].CreateMany(ctx, docs) //nolint + } + b.StopTimer() + + return nil +} + +func runCollectionBenchCreateSync(b *testing.B, + ctx context.Context, + collections []client.Collection, + fixture fixtures.Generator, + docCount, opCount int, +) error { + numTypes := len(fixture.Types()) + b.ResetTimer() + runs := opCount / numTypes + for i := 0; i < b.N; i++ { + for j := 0; j < runs; j++ { + docs, _ := fixture.GenerateDocs() + for k := 0; k < numTypes; k++ { + doc, _ := document.NewFromJSON([]byte(docs[k])) + collections[k].Create(ctx, doc) //nolint + } + } + } + b.StopTimer() + + return nil +} + +// batching +// uses an async method similar to the BackFill implementaion +// cuts the total task up into batchs up to writeBatchGroup size +// and wait for it all to finish. +func runCollectionBenchCreateAsync(b *testing.B, + ctx context.Context, + collections []client.Collection, + fixture fixtures.Generator, + docCount, opCount int, +) error { + numTypes := len(fixture.Types()) + b.StartTimer() + + for bi := 0; bi < b.N; bi++ { + var wg sync.WaitGroup + wg.Add(opCount) + + for bid := 0; float64(bid) < math.Ceil(float64(opCount)/writeBatchGroup); bid++ { + currentBatchSize := int(math.Min(float64((opCount - (bid * writeBatchGroup))), writeBatchGroup)) + var batchWg sync.WaitGroup + batchWg.Add(currentBatchSize) + + for i := 0; i < currentBatchSize; i++ { + go func(index int) { + docs, _ := fixture.GenerateDocs() + // create the documents + for j := 0; j < numTypes; j++ { + doc, _ := document.NewFromJSON([]byte(docs[j])) + collections[j].Create(ctx, doc) //nolint + } + + wg.Done() + batchWg.Done() + }((bid * writeBatchGroup) + i) + } + + batchWg.Wait() + // fmt.Printf(".") + } + + // finish or err + wg.Wait() + } + + return nil +} diff --git a/bench/fixtures/data.go b/bench/fixtures/data.go new file mode 100644 index 0000000000..c13473648c --- /dev/null +++ b/bench/fixtures/data.go @@ -0,0 +1,18 @@ +package fixtures + +var ( + gTypeToGQLType = map[string]string{ + "int": "Int", + "string": "String", + "float64": "Float", + "float32": "Float", + "bool": "Boolean", + } +) + +type User struct { + Name string `faker:"name"` + Age int + Points float32 `faker:"amount"` + Verified bool +} diff --git a/bench/fixtures/fixtures.go b/bench/fixtures/fixtures.go new file mode 100644 index 0000000000..b172b5b002 --- /dev/null +++ b/bench/fixtures/fixtures.go @@ -0,0 +1,101 @@ +package fixtures + +import ( + "bytes" + "context" + "encoding/json" + "errors" + "fmt" + "reflect" + + "github.com/bxcodec/faker" +) + +var ( + registeredFixtures = map[string][]interface{}{ + "user_simple": {User{}}, + } +) + +type Generator struct { + ctx context.Context + + schema string + types []interface{} +} + +func ForSchema(ctx context.Context, schemaName string) Generator { + return Generator{ + ctx: ctx, + schema: schemaName, + types: registeredFixtures[schemaName], + } +} + +// Types returns the defined types for this fixture set +func (g Generator) Types() []interface{} { + return g.types +} + +// Type returns type at the given index in the fixture set +func (g Generator) Type(index int) interface{} { + return g.types[index] +} + +// TypeName returns the name of the type at the given index +// in the fixture set +func (g Generator) TypeName(index int) string { + return reflect.TypeOf(g.types[index]).Name() +} + +// GenerateFixtureDocs uses the faker fixture system to +// randomly generate a new set of documents matching the defined +// struct types within the context. +func (g Generator) GenerateDocs() ([]string, error) { + results := make([]string, len(g.types)) + for i, t := range g.types { + val := reflect.New(reflect.TypeOf(t)).Interface() + + // generate our new random struct and + // write it to our reflected variable + if err := faker.FakeData(val); err != nil { + return nil, err + } + + buf, err := json.Marshal(val) + if err != nil { + return nil, err + } + results[i] = string(buf) + } + + return results, nil +} + +// extractGQLFromType extracts a GraphQL SDL definition as a string +// from a given type struct +func ExtractGQLFromType(t interface{}) (string, error) { + var buf bytes.Buffer + + if reflect.TypeOf(t).Kind() != reflect.Struct { + return "", errors.New("given type is not a struct") + } + + // get name + tt := reflect.TypeOf(t) + name := tt.Name() + + // write the GQL SDL object to the buffer, field by field + fmt.Fprintf(&buf, "type %s {\n", name) + for i := 0; i < tt.NumField(); i++ { + // @todo: Handle non-scalar types + f := tt.Field(i) + fname := f.Name + ftype := f.Type.Name() + gqlType := gTypeToGQLType[ftype] + fmt.Fprintf(&buf, "\t%s: %s\n", fname, gqlType) + } + fmt.Fprint(&buf, "}") + + return buf.String(), nil +} diff --git a/bench/query/planner/simple_test.go b/bench/query/planner/simple_test.go new file mode 100644 index 0000000000..e3c0bd7600 --- /dev/null +++ b/bench/query/planner/simple_test.go @@ -0,0 +1,38 @@ +package planner + +import ( + "context" + "testing" + + "github.com/sourcenetwork/defradb/bench/fixtures" +) + +var ( + userSimpleQuery = ` + query { + User { + _key + Name + Age + Points + Verified + } + } + ` +) + +func Benchmark_Planner_UserSimple_ParseQuery(b *testing.B) { + ctx := context.Background() + err := runQueryParserBench(b, ctx, fixtures.ForSchema(ctx, "user_simple"), userSimpleQuery) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Planner_UserSimple_MakePlan(b *testing.B) { + ctx := context.Background() + err := runMakePlanBench(b, ctx, fixtures.ForSchema(ctx, "user_simple"), userSimpleQuery) + if err != nil { + b.Fatal(err) + } +} diff --git a/bench/query/planner/utils.go b/bench/query/planner/utils.go new file mode 100644 index 0000000000..9c0ee24d6f --- /dev/null +++ b/bench/query/planner/utils.go @@ -0,0 +1,61 @@ +package planner + +import ( + "context" + "fmt" + "testing" + + benchutils "github.com/sourcenetwork/defradb/bench" + "github.com/sourcenetwork/defradb/bench/fixtures" +) + +func runQueryParserBench(b *testing.B, ctx context.Context, fixture fixtures.Generator, query string) error { + db, _, err := benchutils.SetupDBAndCollections(b, ctx, fixture) + if err != nil { + return err + } + defer db.Close() + + b.ResetTimer() + for i := 0; i < b.N; i++ { + _, err := db.Executor().ParseQueryString(query) + if err != nil { + return fmt.Errorf("Failed to parse query string: %w", err) + } + } + b.StopTimer() + + return nil +} + +func runMakePlanBench(b *testing.B, ctx context.Context, fixture fixtures.Generator, query string) error { + db, _, err := benchutils.SetupDBAndCollections(b, ctx, fixture) + if err != nil { + return err + } + defer db.Close() + + exec := db.Executor() + if exec == nil { + return fmt.Errorf("Executor can't be nil") + } + + q, err := exec.ParseQueryString(query) + if err != nil { + return fmt.Errorf("Failed to parse query string: %w", err) + } + txn, err := db.NewTxn(ctx, false) + if err != nil { + return fmt.Errorf("Failed to create txn: %w", err) + } + + b.ResetTimer() + for i := 0; i < b.N; i++ { + _, err := exec.MakePlanFromParser(ctx, db, txn, q) + if err != nil { + return fmt.Errorf("Failed to make plan: %w", err) + } + } + b.StopTimer() + return nil +} diff --git a/bench/query/simple/simple_test.go b/bench/query/simple/simple_test.go new file mode 100644 index 0000000000..e88aac69ab --- /dev/null +++ b/bench/query/simple/simple_test.go @@ -0,0 +1,54 @@ +package query + +import ( + "context" + "testing" + + "github.com/sourcenetwork/defradb/bench/fixtures" +) + +var ( + userSimpleQuery = ` + query { + User { + _key + Name + Age + Points + Verified + } + } + ` +) + +func Benchmark_Query_UserSimple_Query_Sync_1(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1, userSimpleQuery, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Query_UserSimple_Query_Sync_10(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 10, userSimpleQuery, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Query_UserSimple_Query_Sync_100(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 100, userSimpleQuery, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Query_UserSimple_Query_Sync_1000(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1000, userSimpleQuery, false) + if err != nil { + b.Fatal(err) + } +} diff --git a/bench/query/simple/utils.go b/bench/query/simple/utils.go new file mode 100644 index 0000000000..929985281b --- /dev/null +++ b/bench/query/simple/utils.go @@ -0,0 +1,95 @@ +package query + +import ( + "context" + "fmt" + "math/rand" + "strings" + "testing" + + benchutils "github.com/sourcenetwork/defradb/bench" + "github.com/sourcenetwork/defradb/bench/fixtures" + "github.com/sourcenetwork/defradb/db" + "github.com/sourcenetwork/defradb/document/key" +) + +func runQueryBenchGet(b *testing.B, ctx context.Context, fixture fixtures.Generator, docCount int, query string, doSync bool) error { + db, collections, err := benchutils.SetupDBAndCollections(b, ctx, fixture) + if err != nil { + return err + } + defer db.Close() + + dockeys, err := benchutils.BackfillBenchmarkDB(b, ctx, collections, fixture, docCount, 0, doSync) + if err != nil { + return err + } + + return runQueryBenchGetSync(b, ctx, db, docCount, dockeys, query) +} + +func runQueryBenchGetSync( + b *testing.B, + ctx context.Context, + db *db.DB, + docCount int, + dockeys [][]key.DocKey, + query string, +) error { + // fmt.Printf("Query:\n%s\n", query) + + // run any preprocessing on the query before execution (mostly just dockey insertion if needed) + query = formatQuery(b, query, dockeys) + + b.ResetTimer() + for i := 0; i < b.N; i++ { + res := db.ExecQuery(ctx, query) + if len(res.Errors) > 0 { + return fmt.Errorf("Query error: %v", res.Errors) + } + + // leave comments for debug!! + // l := len(res.Data.([]map[string]interface{})) + // if l != opCount { + // return fmt.Errorf("Invalid response, returned data doesn't match length, expected %v actual %v", docCount, l) + // } + // fmt.Println(res) + // fmt.Println("--------------------") + } + b.StopTimer() + + return nil +} + +func formatQuery(b *testing.B, query string, dockeys [][]key.DocKey) string { + numPlaceholders := strings.Count(query, "{{dockey}}") + if numPlaceholders == 0 { + return query + } + // create a copy of dockeys since we'll be mutating it + dockeysCopy := dockeys[:] + + // b.Logf("formatting query, replacing %v instances", numPlaceholders) + // b.Logf("Query before: %s", query) + + if len(dockeysCopy) < numPlaceholders { + b.Fatalf("Invalid number of query placeholders, max is %v requested is %v", len(dockeys), numPlaceholders) + } + + for i := 0; i < numPlaceholders; i++ { + // pick a random dockey, needs to be unique accross all + // loop iterations, so remove the selected one so the next + // iteration cant potentially pick it. + rIndex := rand.Intn(len(dockeysCopy)) + key := dockeysCopy[rIndex][0] + + // remove selected key + dockeysCopy = append(dockeysCopy[:rIndex], dockeysCopy[rIndex+1:]...) + + // replace + query = strings.Replace(query, "{{dockey}}", key.String(), 1) + } + + // b.Logf("Query After: %s", query) + return query +} diff --git a/bench/query/simple/with_filter_test.go b/bench/query/simple/with_filter_test.go new file mode 100644 index 0000000000..015b0fa258 --- /dev/null +++ b/bench/query/simple/with_filter_test.go @@ -0,0 +1,54 @@ +package query + +import ( + "context" + "testing" + + "github.com/sourcenetwork/defradb/bench/fixtures" +) + +var ( + userSimpleWithFilterQuery = ` + query { + User(filter: {Age: {_gt: 10}}) { + _key + Name + Age + Points + Verified + } + } + ` +) + +func Benchmark_Query_UserSimple_Query_WithFilter_Sync_1(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1, userSimpleWithFilterQuery, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Query_UserSimple_Query_WithFilter_Sync_10(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 10, userSimpleWithFilterQuery, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Query_UserSimple_Query_WithFilter_Sync_100(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 100, userSimpleWithFilterQuery, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Query_UserSimple_Query_WithFilter_Sync_1000(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1000, userSimpleWithFilterQuery, false) + if err != nil { + b.Fatal(err) + } +} diff --git a/bench/query/simple/with_limit_offset_test.go b/bench/query/simple/with_limit_offset_test.go new file mode 100644 index 0000000000..0b0f42a1ec --- /dev/null +++ b/bench/query/simple/with_limit_offset_test.go @@ -0,0 +1,54 @@ +package query + +import ( + "context" + "testing" + + "github.com/sourcenetwork/defradb/bench/fixtures" +) + +var ( + userSimpleWithLimitOffsetQuery = ` + query { + User(limit: 10, offset: 5) { + _key + Name + Age + Points + Verified + } + } + ` +) + +func Benchmark_Query_UserSimple_Query_WithLimitOffset_Sync_1(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1, userSimpleWithLimitOffsetQuery, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Query_UserSimple_Query_WithLimitOffset_Sync_10(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 10, userSimpleWithLimitOffsetQuery, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Query_UserSimple_Query_WithLimitOffset_Sync_100(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 100, userSimpleWithLimitOffsetQuery, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Query_UserSimple_Query_WithLimitOffset_Sync_1000(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1000, userSimpleWithLimitOffsetQuery, false) + if err != nil { + b.Fatal(err) + } +} diff --git a/bench/query/simple/with_multi_lookup_test.go b/bench/query/simple/with_multi_lookup_test.go new file mode 100644 index 0000000000..f81fb5e67a --- /dev/null +++ b/bench/query/simple/with_multi_lookup_test.go @@ -0,0 +1,47 @@ +package query + +import ( + "context" + "testing" + + "github.com/sourcenetwork/defradb/bench/fixtures" +) + +var ( + // 10x dockey will be replaced in the bench runner func + userSimpleWithMultiLookupQuery = ` + query { + User(dockeys: ["{{dockey}}", "{{dockey}}", "{{dockey}}", "{{dockey}}", "{{dockey}}", "{{dockey}}", "{{dockey}}", "{{dockey}}", "{{dockey}}", "{{dockey}}"]) { + _key + Name + Age + Points + Verified + } + } + ` +) + +func Benchmark_Query_UserSimple_Query_WithMultiLookup_Sync_10(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 10, userSimpleWithMultiLookupQuery, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Query_UserSimple_Query_WithMultiLookup_Sync_100(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 100, userSimpleWithMultiLookupQuery, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Query_UserSimple_Query_WithMultiLookup_Sync_1000(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1000, userSimpleWithMultiLookupQuery, false) + if err != nil { + b.Fatal(err) + } +} diff --git a/bench/query/simple/with_single_lookup_test.go b/bench/query/simple/with_single_lookup_test.go new file mode 100644 index 0000000000..30ced0e6a2 --- /dev/null +++ b/bench/query/simple/with_single_lookup_test.go @@ -0,0 +1,55 @@ +package query + +import ( + "context" + "testing" + + "github.com/sourcenetwork/defradb/bench/fixtures" +) + +var ( + // dockey will be replaced in the bench runner func + userSimpleWithSingleLookupQuery = ` + query { + User(dockey: "{{dockey}}") { + _key + Name + Age + Points + Verified + } + } + ` +) + +func Benchmark_Query_UserSimple_Query_WithSingleLookup_Sync_1(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1, userSimpleWithSingleLookupQuery, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Query_UserSimple_Query_WithSingleLookup_Sync_10(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 10, userSimpleWithSingleLookupQuery, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Query_UserSimple_Query_WithSingleLookup_Sync_100(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 100, userSimpleWithSingleLookupQuery, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Query_UserSimple_Query_WithSingleLookup_Sync_1000(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1000, userSimpleWithSingleLookupQuery, false) + if err != nil { + b.Fatal(err) + } +} diff --git a/bench/query/simple/with_sort_test.go b/bench/query/simple/with_sort_test.go new file mode 100644 index 0000000000..c0a675b240 --- /dev/null +++ b/bench/query/simple/with_sort_test.go @@ -0,0 +1,54 @@ +package query + +import ( + "context" + "testing" + + "github.com/sourcenetwork/defradb/bench/fixtures" +) + +var ( + userSimpleWithSortQuery = ` + query { + User(order: {Age: ASC}) { + _key + Name + Age + Points + Verified + } + } + ` +) + +func Benchmark_Query_UserSimple_Query_WithSort_Sync_1(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1, userSimpleWithSortQuery, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Query_UserSimple_Query_WithSort_Sync_10(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 10, userSimpleWithSortQuery, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Query_UserSimple_Query_WithSort_Sync_100(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 100, userSimpleWithSortQuery, false) + if err != nil { + b.Fatal(err) + } +} + +func Benchmark_Query_UserSimple_Query_WithSort_Sync_1000(b *testing.B) { + ctx := context.Background() + err := runQueryBenchGet(b, ctx, fixtures.ForSchema(ctx, "user_simple"), 1000, userSimpleWithSortQuery, false) + if err != nil { + b.Fatal(err) + } +} diff --git a/bench/run.sh b/bench/run.sh new file mode 100644 index 0000000000..e69de29bb2 diff --git a/bench/storage/get_test.go b/bench/storage/get_test.go new file mode 100644 index 0000000000..85324649b5 --- /dev/null +++ b/bench/storage/get_test.go @@ -0,0 +1,86 @@ +// +build test +package storage + +import ( + "context" + "fmt" + "testing" +) + +var ( + valueSize = []int{ + 64, 128, 256, 512, 1024, + } +) + +func Benchmark_Storage_Simple_Read_Sync_1_1(b *testing.B) { + for _, vsz := range valueSize { + b.Run(fmt.Sprintf("ValueSize:%04d", vsz), func(b *testing.B) { + ctx := context.Background() + err := runStorageBenchGet(b, ctx, vsz, 1, 1, true) + if err != nil { + b.Fatal(err) + } + }) + } +} + +func Benchmark_Storage_Simple_Read_Sync_1_10(b *testing.B) { + for _, vsz := range valueSize { + b.Run(fmt.Sprintf("ValueSize:%04d", vsz), func(b *testing.B) { + ctx := context.Background() + err := runStorageBenchGet(b, ctx, vsz, 1, 10, true) + if err != nil { + b.Fatal(err) + } + }) + } +} + +func Benchmark_Storage_Simple_Read_Sync_1_100(b *testing.B) { + for _, vsz := range valueSize { + b.Run(fmt.Sprintf("ValueSize:%04d", vsz), func(b *testing.B) { + ctx := context.Background() + err := runStorageBenchGet(b, ctx, vsz, 1, 100, true) + if err != nil { + b.Fatal(err) + } + }) + } +} + +func Benchmark_Storage_Simple_Read_Sync_100_1(b *testing.B) { + for _, vsz := range valueSize { + b.Run(fmt.Sprintf("ValueSize:%04d", vsz), func(b *testing.B) { + ctx := context.Background() + err := runStorageBenchGet(b, ctx, vsz, 100, 1, true) + if err != nil { + b.Fatal(err) + } + }) + } +} + +func Benchmark_Storage_Simple_Read_Sync_100_10(b *testing.B) { + for _, vsz := range valueSize { + b.Run(fmt.Sprintf("ValueSize:%04d", vsz), func(b *testing.B) { + ctx := context.Background() + err := runStorageBenchGet(b, ctx, vsz, 100, 10, true) + if err != nil { + b.Fatal(err) + } + }) + } +} + +func Benchmark_Storage_Simple_Read_Sync_100_100(b *testing.B) { + for _, vsz := range valueSize { + b.Run(fmt.Sprintf("ValueSize:%04d", vsz), func(b *testing.B) { + ctx := context.Background() + err := runStorageBenchGet(b, ctx, vsz, 100, 100, true) + if err != nil { + b.Fatal(err) + } + }) + } +} diff --git a/bench/storage/put_many_test.go b/bench/storage/put_many_test.go new file mode 100644 index 0000000000..a98c5fd5e0 --- /dev/null +++ b/bench/storage/put_many_test.go @@ -0,0 +1,86 @@ +// +build test +package storage + +import ( + "context" + "fmt" + "testing" +) + +func Benchmark_Storage_Simple_WriteMany_Sync_0_1(b *testing.B) { + + for _, vsz := range valueSize { + b.Run(fmt.Sprintf("ValueSize:%04d", vsz), func(b *testing.B) { + ctx := context.Background() + err := runStorageBenchPutMany(b, ctx, vsz, 0, 1, true) + if err != nil { + b.Fatal(err) + } + }) + } +} + +func Benchmark_Storage_Simple_WriteMany_Sync_0_10(b *testing.B) { + + for _, vsz := range valueSize { + b.Run(fmt.Sprintf("ValueSize:%04d", vsz), func(b *testing.B) { + ctx := context.Background() + err := runStorageBenchPutMany(b, ctx, vsz, 0, 10, true) + if err != nil { + b.Fatal(err) + } + }) + } +} + +func Benchmark_Storage_Simple_WriteMany_Sync_0_100(b *testing.B) { + + for _, vsz := range valueSize { + b.Run(fmt.Sprintf("ValueSize:%04d", vsz), func(b *testing.B) { + ctx := context.Background() + err := runStorageBenchPutMany(b, ctx, vsz, 0, 100, true) + if err != nil { + b.Fatal(err) + } + }) + } +} + +func Benchmark_Storage_Simple_WriteMany_Sync_100_1(b *testing.B) { + + for _, vsz := range valueSize { + b.Run(fmt.Sprintf("ValueSize:%04d", vsz), func(b *testing.B) { + ctx := context.Background() + err := runStorageBenchPutMany(b, ctx, vsz, 100, 1, true) + if err != nil { + b.Fatal(err) + } + }) + } +} + +func Benchmark_Storage_Simple_WriteMany_Sync_100_10(b *testing.B) { + + for _, vsz := range valueSize { + b.Run(fmt.Sprintf("ValueSize:%04d", vsz), func(b *testing.B) { + ctx := context.Background() + err := runStorageBenchPutMany(b, ctx, vsz, 100, 10, true) + if err != nil { + b.Fatal(err) + } + }) + } +} + +func Benchmark_Storage_Simple_WriteMany_Sync_100_100(b *testing.B) { + + for _, vsz := range valueSize { + b.Run(fmt.Sprintf("ValueSize:%04d", vsz), func(b *testing.B) { + ctx := context.Background() + err := runStorageBenchPutMany(b, ctx, vsz, 100, 100, true) + if err != nil { + b.Fatal(err) + } + }) + } +} diff --git a/bench/storage/put_test.go b/bench/storage/put_test.go new file mode 100644 index 0000000000..903fba9944 --- /dev/null +++ b/bench/storage/put_test.go @@ -0,0 +1,86 @@ +// +build test +package storage + +import ( + "context" + "fmt" + "testing" +) + +func Benchmark_Storage_Simple_Write_Sync_0_1(b *testing.B) { + + for _, vsz := range valueSize { + b.Run(fmt.Sprintf("ValueSize:%04d", vsz), func(b *testing.B) { + ctx := context.Background() + err := runStorageBenchPut(b, ctx, vsz, 0, 1, true) + if err != nil { + b.Fatal(err) + } + }) + } +} + +func Benchmark_Storage_Simple_Write_Sync_0_10(b *testing.B) { + + for _, vsz := range valueSize { + b.Run(fmt.Sprintf("ValueSize:%04d", vsz), func(b *testing.B) { + ctx := context.Background() + err := runStorageBenchPut(b, ctx, vsz, 0, 10, true) + if err != nil { + b.Fatal(err) + } + }) + } +} + +func Benchmark_Storage_Simple_Write_Sync_0_100(b *testing.B) { + + for _, vsz := range valueSize { + b.Run(fmt.Sprintf("ValueSize:%04d", vsz), func(b *testing.B) { + ctx := context.Background() + err := runStorageBenchPut(b, ctx, vsz, 0, 100, true) + if err != nil { + b.Fatal(err) + } + }) + } +} + +func Benchmark_Storage_Simple_Write_Sync_100_1(b *testing.B) { + + for _, vsz := range valueSize { + b.Run(fmt.Sprintf("ValueSize:%04d", vsz), func(b *testing.B) { + ctx := context.Background() + err := runStorageBenchPut(b, ctx, vsz, 100, 1, true) + if err != nil { + b.Fatal(err) + } + }) + } +} + +func Benchmark_Storage_Simple_Write_Sync_100_10(b *testing.B) { + + for _, vsz := range valueSize { + b.Run(fmt.Sprintf("ValueSize:%04d", vsz), func(b *testing.B) { + ctx := context.Background() + err := runStorageBenchPut(b, ctx, vsz, 100, 10, true) + if err != nil { + b.Fatal(err) + } + }) + } +} + +func Benchmark_Storage_Simple_Write_Sync_100_100(b *testing.B) { + + for _, vsz := range valueSize { + b.Run(fmt.Sprintf("ValueSize:%04d", vsz), func(b *testing.B) { + ctx := context.Background() + err := runStorageBenchPut(b, ctx, vsz, 100, 100, true) + if err != nil { + b.Fatal(err) + } + }) + } +} diff --git a/bench/storage/utils.go b/bench/storage/utils.go new file mode 100644 index 0000000000..bbe16ffbd8 --- /dev/null +++ b/bench/storage/utils.go @@ -0,0 +1,149 @@ +package storage + +import ( + "context" + "math/rand" + "testing" + + ds "github.com/ipfs/go-datastore" + + benchutils "github.com/sourcenetwork/defradb/bench" +) + +func runStorageBenchGet(b *testing.B, ctx context.Context, valueSize, objCount, opCount int, doSync bool) error { + db, err := benchutils.NewTestStorage(b) + if err != nil { + return err + } + defer db.Close() //nolint + + // backfill + keys, err := backfillBenchmarkStorageDB(ctx, db, objCount, valueSize) + if err != nil { + return err + } + + b.ResetTimer() + for i := 0; i < b.N; i++ { + for j := 0; j < opCount; j++ { + key := ds.NewKey(keys[rand.Int31n(int32(len(keys)))]) + _, err := db.Get(ctx, key) + if err != nil { + return err + } + } + } + b.StopTimer() + + return nil +} + +func runStorageBenchPut(b *testing.B, ctx context.Context, valueSize, objCount, opCount int, doSync bool) error { + db, err := benchutils.NewTestStorage(b) + if err != nil { + return err + } + defer db.Close() //nolint + + // backfill + _, err = backfillBenchmarkStorageDB(ctx, db, objCount, valueSize) + if err != nil { + return err + } + + b.ResetTimer() + for i := 0; i < b.N; i++ { + for j := 0; j < opCount; j++ { + keyBuf := make([]byte, 32) + value := make([]byte, valueSize) + if _, err := rand.Read(value); err != nil { + return err + } + if _, err := rand.Read(keyBuf); err != nil { + return err + } + key := ds.NewKey(string(keyBuf)) + + if err := db.Put(ctx, key, value); err != nil { + return err + } + } + } + b.StopTimer() + + return nil +} + +func runStorageBenchPutMany(b *testing.B, ctx context.Context, valueSize, objCount, opCount int, doSync bool) error { + db, err := benchutils.NewTestStorage(b) + if err != nil { + return err + } + defer db.Close() //nolint + + // backfill + _, err = backfillBenchmarkStorageDB(ctx, db, objCount, valueSize) + if err != nil { + return err + } + + //shuffle keys + // rand.Shuffle(len(keys), func(i, j int) { + // keys[i], keys[j] = keys[j], keys[i] + // }) + + b.ResetTimer() + for i := 0; i < b.N; i++ { + batch, err := db.Batch(ctx) + if err != nil { + return err + } + for j := 0; j < opCount; j++ { + keyBuf := make([]byte, 32) + value := make([]byte, valueSize) + if _, err := rand.Read(value); err != nil { + return err + } + if _, err := rand.Read(keyBuf); err != nil { + return err + } + key := ds.NewKey(string(keyBuf)) + + if err := batch.Put(ctx, key, value); err != nil { + return err + } + } + if err := batch.Commit(ctx); err != nil { + return err + } + } + b.StopTimer() + + return nil +} + +func backfillBenchmarkStorageDB(ctx context.Context, db ds.Batching, objCount int, valueSize int) ([]string, error) { + batch, err := db.Batch(ctx) + if err != nil { + return nil, err + } + keys := make([]string, objCount) + for i := 0; i < objCount; i++ { + keyBuf := make([]byte, 32) + value := make([]byte, valueSize) + if _, err := rand.Read(value); err != nil { + return nil, err + } + if _, err := rand.Read(keyBuf); err != nil { + return nil, err + } + key := ds.NewKey(string(keyBuf)) + keys[i] = key.String() + + if err := batch.Put(ctx, key, value); err != nil { + return nil, err + } + } + + return keys, batch.Commit(ctx) +} diff --git a/client/core.go b/client/core.go index 24ab4ea576..307bafa38a 100644 --- a/client/core.go +++ b/client/core.go @@ -68,6 +68,8 @@ type Collection interface { UpdateWithKey(context.Context, key.DocKey, interface{}, ...UpdateOpt) (*UpdateResult, error) UpdateWithKeys(context.Context, []key.DocKey, interface{}, ...UpdateOpt) (*UpdateResult, error) + Get(context.Context, key.DocKey) (*document.Document, error) + WithTxn(Txn) Collection } diff --git a/core/crdt/lwwreg.go b/core/crdt/lwwreg.go index 9b7fe9915e..7639fae402 100644 --- a/core/crdt/lwwreg.go +++ b/core/crdt/lwwreg.go @@ -38,6 +38,7 @@ var ( type LWWRegDelta struct { Priority uint64 Data []byte + DocKey []byte } // GetPriority gets the current priority for this delta @@ -59,7 +60,8 @@ func (delta *LWWRegDelta) Marshal() ([]byte, error) { err := enc.Encode(struct { Priority uint64 Data []byte - }{delta.Priority, delta.Data}) + DocKey []byte + }{delta.Priority, delta.Data, delta.DocKey}) if err != nil { return nil, err } @@ -108,7 +110,8 @@ func (reg LWWRegister) Value(ctx context.Context) ([]byte, error) { func (reg LWWRegister) Set(value []byte) *LWWRegDelta { // return NewLWWRegister(reg.id, value, reg.clock.Apply(), reg.clock) return &LWWRegDelta{ - Data: value, + Data: value, + DocKey: []byte(reg.key), } } diff --git a/db/base/descriptions.go b/db/base/descriptions.go index 2e9c87971a..a8899f89e3 100644 --- a/db/base/descriptions.go +++ b/db/base/descriptions.go @@ -125,14 +125,14 @@ const ( // Note: These values are serialized and persisted in the database, avoid modifying existing values const ( - Meta_Relation_ONE uint8 = 1 // 0b0000 0001 - Meta_Relation_MANY uint8 = 2 // 0b0000 0010 - Meta_Relation_ONEONE uint8 = 4 // 0b0000 0100 - Meta_Relation_ONEMANY uint8 = 8 // 0b0000 1000 - Meta_Relation_MANYMANY uint8 = 16 // 0b0001 0000 - _ uint8 = 32 // 0b0010 0000 - _ uint8 = 64 // 0b0100 0000 - Meta_Relation_Primary uint8 = 128 // 0b1000 0000 Primary reference entity on relation + Meta_Relation_ONE uint8 = 1 // 0b0000 0001 + Meta_Relation_MANY uint8 = 2 // 0b0000 0010 + Meta_Relation_ONEONE uint8 = 4 // 0b0000 0100 + Meta_Relation_ONEMANY uint8 = 8 // 0b0000 1000 + Meta_Relation_MANYMANY uint8 = 16 // 0b0001 0000 + _ uint8 = 32 // 0b0010 0000 + Meta_Relation_INTERNAL_ID uint8 = 64 // 0b0100 0000 + Meta_Relation_Primary uint8 = 128 // 0b1000 0000 Primary reference entity on relation ) type FieldID uint32 @@ -157,6 +157,10 @@ func (f FieldDescription) IsObject() bool { (f.Kind == FieldKind_FOREIGN_OBJECT_ARRAY) } +func (f FieldDescription) IsObjectArray() bool { + return (f.Kind == FieldKind_FOREIGN_OBJECT_ARRAY) +} + func IsSet(val, target uint8) bool { return val&target > 0 } diff --git a/db/collection.go b/db/collection.go index 04aa907de3..7db4ae8e78 100644 --- a/db/collection.go +++ b/db/collection.go @@ -412,16 +412,19 @@ func (c *Collection) save(ctx context.Context, txn *Txn, doc *document.Document) return err } if val.IsDelete() { - err := doc.SetAs(v.Name(), nil, v.Type()) - if err != nil { - log.Error("Couldn't set document as type: ", v.Type()) - } merge[k] = nil } else { merge[k] = val.Value() } - // set value as clean - val.Clean() + + // NOTE: We delay the final Clean() call till we know + // the commit on the transaction is successfull. If we didn't + // wait, and just did it here, then *if* the commit fails down + // the line, then we have no way to roll back the state + // side-effect on the documnet func called here. + txn.OnSuccess(func() { + doc.Clean() + }) links = append(links, core.DAGLink{ Name: k, diff --git a/db/db.go b/db/db.go index 416a78c916..63e9539297 100644 --- a/db/db.go +++ b/db/db.go @@ -48,7 +48,10 @@ var ( ) // make sure we match our client interface -var _ client.DB = (*DB)(nil) +var ( + _ client.DB = (*DB)(nil) + _ client.Collection = (*Collection)(nil) +) // DB is the main interface for interacting with the // DefraDB storage system. @@ -185,6 +188,10 @@ func (db *DB) PrintDump(ctx context.Context) { printStore(ctx, db.rootstore) } +func (db *DB) Executor() *planner.QueryExecutor { + return db.queryExecutor +} + // Close is called when we are shutting down the database. // This is the place for any last minute cleanup or releaseing // of resources (IE: Badger instance) diff --git a/db/tests/query/all_commits/simple_test.go b/db/tests/query/all_commits/simple_test.go index 6c52ce526d..fa8c18c279 100644 --- a/db/tests/query/all_commits/simple_test.go +++ b/db/tests/query/all_commits/simple_test.go @@ -36,14 +36,14 @@ func TestQueryAllCommitsSingleDAG(t *testing.T) { }, Results: []map[string]interface{}{ { - "cid": "bafkreiercmxn6e3qryxvuped5pplg733c5fj6gjypj5wykk63ouvcfb25m", + "cid": "bafkreie33v5gnk4vlkzedrm5tofkhhwwrb3mlnpibfsa6qzgvhpiv5qzl4", "links": []map[string]interface{}{ { - "cid": "bafybeiasnjaz6bohhhqopk77ksivqed5wgbog7575wunleaq57nar6otui", + "cid": "bafybeiftyjqxyzqtfpi65kde4hla4xm3v4dvtr7fr2p2p5ng5lfg7rrcve", "name": "Age", }, { - "cid": "bafybeifxin4fbdnc4hrn5tyimnzy53jj6oxtu5kpgohzv5y5wsrpjoih6a", + "cid": "bafybeierejzn3m6pesium3cml4flyjoe2wd2pxbmxxi5v42yqw2w4fpcxm", "name": "Name", }, }, @@ -78,11 +78,11 @@ func TestQueryAllCommitsMultipleDAG(t *testing.T) { }, Results: []map[string]interface{}{ { - "cid": "bafkreicewiwopwgdrnrdnbh4qnv45yk6vhlmdvdmeri6rue34zpbouyxsq", + "cid": "bafkreiap6dmsuhgzfbnj6zeobmhhmra37v3hokal7ef7dhaqp4qtpou2sa", "height": int64(2), }, { - "cid": "bafkreiercmxn6e3qryxvuped5pplg733c5fj6gjypj5wykk63ouvcfb25m", + "cid": "bafkreie33v5gnk4vlkzedrm5tofkhhwwrb3mlnpibfsa6qzgvhpiv5qzl4", "height": int64(1), }, }, diff --git a/db/tests/query/all_commits/with_count_test.go b/db/tests/query/all_commits/with_count_test.go index 58ea958da8..58802d5f6c 100644 --- a/db/tests/query/all_commits/with_count_test.go +++ b/db/tests/query/all_commits/with_count_test.go @@ -33,7 +33,7 @@ func TestQueryAllCommitsSingleDAGWithLinkCount(t *testing.T) { }, Results: []map[string]interface{}{ { - "cid": "bafkreiercmxn6e3qryxvuped5pplg733c5fj6gjypj5wykk63ouvcfb25m", + "cid": "bafkreie33v5gnk4vlkzedrm5tofkhhwwrb3mlnpibfsa6qzgvhpiv5qzl4", "_count": 2, }, }, diff --git a/db/tests/query/commit/simple_test.go b/db/tests/query/commit/simple_test.go index 790ba31b06..cccfd52304 100644 --- a/db/tests/query/commit/simple_test.go +++ b/db/tests/query/commit/simple_test.go @@ -19,7 +19,7 @@ func TestQueryOneCommit(t *testing.T) { test := testUtils.QueryTestCase{ Description: "query for a single block by CID", Query: `query { - commit(cid: "bafkreiercmxn6e3qryxvuped5pplg733c5fj6gjypj5wykk63ouvcfb25m") { + commit(cid: "bafkreie33v5gnk4vlkzedrm5tofkhhwwrb3mlnpibfsa6qzgvhpiv5qzl4") { cid height delta @@ -34,7 +34,7 @@ func TestQueryOneCommit(t *testing.T) { }, Results: []map[string]interface{}{ { - "cid": "bafkreiercmxn6e3qryxvuped5pplg733c5fj6gjypj5wykk63ouvcfb25m", + "cid": "bafkreie33v5gnk4vlkzedrm5tofkhhwwrb3mlnpibfsa6qzgvhpiv5qzl4", "height": int64(1), // cbor encoded delta "delta": []uint8{0xa2, 0x63, 0x41, 0x67, 0x65, 0x15, 0x64, 0x4e, 0x61, 0x6d, 0x65, 0x64, 0x4a, 0x6f, 0x68, 0x6e}, diff --git a/db/tests/query/latest_commits/simple_test.go b/db/tests/query/latest_commits/simple_test.go index eeb8cae309..d9c23d55d0 100644 --- a/db/tests/query/latest_commits/simple_test.go +++ b/db/tests/query/latest_commits/simple_test.go @@ -36,14 +36,14 @@ func TestQueryLatestCommits(t *testing.T) { }, Results: []map[string]interface{}{ { - "cid": "bafkreiercmxn6e3qryxvuped5pplg733c5fj6gjypj5wykk63ouvcfb25m", + "cid": "bafkreie33v5gnk4vlkzedrm5tofkhhwwrb3mlnpibfsa6qzgvhpiv5qzl4", "links": []map[string]interface{}{ { - "cid": "bafybeiasnjaz6bohhhqopk77ksivqed5wgbog7575wunleaq57nar6otui", + "cid": "bafybeiftyjqxyzqtfpi65kde4hla4xm3v4dvtr7fr2p2p5ng5lfg7rrcve", "name": "Age", }, { - "cid": "bafybeifxin4fbdnc4hrn5tyimnzy53jj6oxtu5kpgohzv5y5wsrpjoih6a", + "cid": "bafybeierejzn3m6pesium3cml4flyjoe2wd2pxbmxxi5v42yqw2w4fpcxm", "name": "Name", }, }, diff --git a/db/tests/query/simple/with_version_test.go b/db/tests/query/simple/with_version_test.go index 1317912d4c..d2f6f2a4a9 100644 --- a/db/tests/query/simple/with_version_test.go +++ b/db/tests/query/simple/with_version_test.go @@ -44,14 +44,14 @@ func TestQuerySimpleWithEmbeddedLatestCommit(t *testing.T) { "Age": uint64(21), "_version": []map[string]interface{}{ { - "cid": "bafkreiercmxn6e3qryxvuped5pplg733c5fj6gjypj5wykk63ouvcfb25m", + "cid": "bafkreibnfioxhb5tjfdn6ktp3gmquafhr372i6mayspppll35shk6xxvmm", "links": []map[string]interface{}{ { - "cid": "bafybeiasnjaz6bohhhqopk77ksivqed5wgbog7575wunleaq57nar6otui", + "cid": "bafybeiftyjqxyzqtfpi65kde4hla4xm3v4dvtr7fr2p2p5ng5lfg7rrcve", "name": "Age", }, { - "cid": "bafybeifxin4fbdnc4hrn5tyimnzy53jj6oxtu5kpgohzv5y5wsrpjoih6a", + "cid": "bafybeif67ysvfnusidyuoxwztrwhunuihtbrunet42422wgp22sf6ninki", "name": "Name", }, }, diff --git a/db/tests/utils.go b/db/tests/utils.go index 333501f60a..6d2ecb927d 100644 --- a/db/tests/utils.go +++ b/db/tests/utils.go @@ -7,7 +7,7 @@ // the Business Source License, use of this software will be governed // by the Apache License, Version 2.0, included in the file // licenses/APL.txt. -package tests_test +package tests import ( "context" @@ -18,19 +18,34 @@ import ( badger "github.com/dgraph-io/badger/v3" ds "github.com/ipfs/go-datastore" + "github.com/stretchr/testify/assert" + "github.com/sourcenetwork/defradb/client" badgerds "github.com/sourcenetwork/defradb/datastores/badger/v3" "github.com/sourcenetwork/defradb/db" "github.com/sourcenetwork/defradb/document" - "github.com/stretchr/testify/assert" +) + +const ( + memoryBadgerEnvName = "DEFRA_BADGER_MEMORY" + fileBadgerEnvName = "DEFRA_BADGER_FILE" + memoryMapEnvName = "DEFRA_MAP" +) + +var ( + badgerInMemory bool + badgerFile bool + mapStore bool ) type QueryTestCase struct { Description string Query string + // docs is a map from Collection Index, to a list // of docs in stringified JSON format Docs map[int][]string + // updates is a map from document index, to a list // of changes in strinigied JSON format Updates map[int][]string @@ -40,19 +55,24 @@ type QueryTestCase struct { } type databaseInfo struct { - name string - db *db.DB + name string + db *db.DB + rootstore ds.Batching } -var badgerInMemory bool -var badgerFile bool -var mapStore bool +func (dbi databaseInfo) Rootstore() ds.Batching { + return dbi.rootstore +} + +func (dbi databaseInfo) DB() *db.DB { + return dbi.db +} func init() { // We use environment variables instead of flags `go test ./...` throws for all packages that don't have the flag defined - _, badgerInMemory = os.LookupEnv("DEFRA_BADGER_MEMORY") - _, badgerFile = os.LookupEnv("DEFRA_BADGER_FILE") - _, mapStore = os.LookupEnv("DEFRA_MAP") + _, badgerInMemory = os.LookupEnv(memoryBadgerEnvName) + _, badgerFile = os.LookupEnv(fileBadgerEnvName) + _, mapStore = os.LookupEnv(memoryMapEnvName) // default is to run against all if !badgerInMemory && !badgerFile && !mapStore { @@ -63,7 +83,7 @@ func init() { } } -func newBadgerMemoryDB() (databaseInfo, error) { +func NewBadgerMemoryDB() (databaseInfo, error) { opts := badgerds.Options{Options: badger.DefaultOptions("").WithInMemory(true)} rootstore, err := badgerds.NewDatastore("", &opts) if err != nil { @@ -76,12 +96,13 @@ func newBadgerMemoryDB() (databaseInfo, error) { } return databaseInfo{ - name: "badger-in-memory", - db: db, + name: "badger-in-memory", + db: db, + rootstore: rootstore, }, nil } -func newMapDB() (databaseInfo, error) { +func NewMapDB() (databaseInfo, error) { rootstore := ds.NewMapDatastore() db, err := db.NewDB(rootstore, struct{}{}) if err != nil { @@ -89,12 +110,13 @@ func newMapDB() (databaseInfo, error) { } return databaseInfo{ - name: "ipfs-map-datastore", - db: db, + name: "ipfs-map-datastore", + db: db, + rootstore: rootstore, }, nil } -func newBadgerFileDB(t *testing.T) (databaseInfo, error) { +func NewBadgerFileDB(t testing.TB) (databaseInfo, error) { path := t.TempDir() opts := badgerds.Options{Options: badger.DefaultOptions(path)} @@ -109,8 +131,9 @@ func newBadgerFileDB(t *testing.T) (databaseInfo, error) { } return databaseInfo{ - name: "badger-file-system", - db: db, + name: "badger-file-system", + db: db, + rootstore: rootstore, }, nil } @@ -118,7 +141,7 @@ func getDatabases(t *testing.T) ([]databaseInfo, error) { databases := []databaseInfo{} if badgerInMemory { - badgerIMDatabase, err := newBadgerMemoryDB() + badgerIMDatabase, err := NewBadgerMemoryDB() if err != nil { return nil, err } @@ -126,7 +149,7 @@ func getDatabases(t *testing.T) ([]databaseInfo, error) { } if badgerFile { - badgerIMDatabase, err := newBadgerFileDB(t) + badgerIMDatabase, err := NewBadgerFileDB(t) if err != nil { return nil, err } @@ -134,7 +157,7 @@ func getDatabases(t *testing.T) ([]databaseInfo, error) { } if mapStore { - mapDatabase, err := newMapDB() + mapDatabase, err := NewMapDB() if err != nil { return nil, err } diff --git a/db/txn.go b/db/txn.go index 1946447d05..ba69943f7b 100644 --- a/db/txn.go +++ b/db/txn.go @@ -12,6 +12,7 @@ package db import ( "context" "errors" + "math/rand" "github.com/sourcenetwork/defradb/client" "github.com/sourcenetwork/defradb/core" @@ -21,6 +22,16 @@ import ( ktds "github.com/ipfs/go-datastore/keytransform" ) +var letterRunes = []rune("abcdefghijklmnopqrstuvwxyzABCDEFGHIJKLMNOPQRSTUVWXYZ") + +func RandStringRunes(n int) string { + b := make([]rune, n) + for i := range b { + b[i] = letterRunes[rand.Intn(len(letterRunes))] + } + return string(b) +} + var ( // ErrNoTxnSupport occurs when a new transaction is trying to be created from a // root datastore that doesn't support ds.TxnDatastore or ds.Batching 8885 @@ -45,6 +56,9 @@ type Txn struct { datastore core.DSReaderWriter // wrapped txn /data namespace headstore core.DSReaderWriter // wrapped txn /heads namespace dagstore core.DAGStore // wrapped txn /blocks namespace + + successFns []func() + errorFns []func() } // Txn creates a new transaction which can be set to readonly mode @@ -87,10 +101,20 @@ func (db *DB) newTxn(ctx context.Context, readonly bool) (*Txn, error) { // add the wrapped datastores using the existing KeyTransform functions from the db // @todo Check if KeyTransforms are nil beforehand shimStore := shimTxnStore{txn.Txn} + + // debug stuff... ignore + // + // txnid := RandStringRunes(5) + // txn.systemstore = ds.NewLogDatastore(ktds.Wrap(shimStore, db.ssKeyTransform), fmt.Sprintf("%s:systemstore", txnid)) + // txn.datastore = ds.NewLogDatastore(ktds.Wrap(shimStore, db.dsKeyTransform), fmt.Sprintf("%s:datastore", txnid)) + // txn.headstore = ds.NewLogDatastore(ktds.Wrap(shimStore, db.hsKeyTransform), fmt.Sprintf("%s:headstore", txnid)) + // batchstore := ds.NewLogDatastore(ktds.Wrap(shimStore, db.dagKeyTransform), fmt.Sprintf("%s:dagstore", txnid)) + txn.systemstore = ktds.Wrap(shimStore, db.ssKeyTransform) txn.datastore = ktds.Wrap(shimStore, db.dsKeyTransform) txn.headstore = ktds.Wrap(shimStore, db.hsKeyTransform) batchstore := ktds.Wrap(shimStore, db.dagKeyTransform) + txn.dagstore = store.NewDAGStore(batchstore) return txn, nil @@ -121,6 +145,41 @@ func (txn *Txn) IsBatch() bool { return ok } +func (txn *Txn) Commit(ctx context.Context) error { + if err := txn.Txn.Commit(ctx); err != nil { + txn.runErrorFns(ctx) + return err + } + txn.runSuccessFns(ctx) + return nil +} + +func (txn *Txn) OnSuccess(fn func()) { + if fn == nil { + return + } + txn.successFns = append(txn.successFns, fn) +} + +func (txn *Txn) OnError(fn func()) { + if fn == nil { + return + } + txn.errorFns = append(txn.errorFns, fn) +} + +func (txn *Txn) runErrorFns(ctx context.Context) { + for _, fn := range txn.errorFns { + fn() + } +} + +func (txn *Txn) runSuccessFns(ctx context.Context) { + for _, fn := range txn.successFns { + fn() + } +} + // Shim to make ds.Txn support ds.Datastore type shimTxnStore struct { ds.Txn diff --git a/document/document.go b/document/document.go index 7c313620d2..1e7c5429b4 100644 --- a/document/document.go +++ b/document/document.go @@ -417,6 +417,18 @@ func (doc *Document) ToMap() (map[string]interface{}, error) { return doc.toMapWithKey() } +func (doc *Document) Clean() { + for _, v := range doc.Fields() { + val, _ := doc.GetValueWithField(v) + if val.IsDirty() { + if val.IsDelete() { + doc.SetAs(v.Name(), nil, v.Type()) //nolint + } + val.Clean() + } + } +} + // converts the document into a map[string]interface{} // including any sub documents func (doc *Document) toMap() (map[string]interface{}, error) { diff --git a/go.mod b/go.mod index ee4dbf58c8..95ed06d284 100644 --- a/go.mod +++ b/go.mod @@ -1,9 +1,10 @@ module github.com/sourcenetwork/defradb -go 1.12 +go 1.17 require ( github.com/SierraSoftworks/connor v1.0.2 + github.com/bxcodec/faker v2.0.1+incompatible github.com/davecgh/go-spew v1.1.1 github.com/dgraph-io/badger/v3 v3.2103.2 github.com/fxamacker/cbor/v2 v2.2.0 @@ -19,7 +20,6 @@ require ( github.com/ipfs/go-log/v2 v2.3.0 github.com/ipfs/go-merkledag v0.5.0 github.com/jbenet/goprocess v0.1.4 - github.com/kr/text v0.2.0 // indirect github.com/mitchellh/go-homedir v1.1.0 github.com/multiformats/go-multibase v0.0.3 github.com/multiformats/go-multihash v0.0.15 @@ -32,6 +32,73 @@ require ( gopkg.in/yaml.v2 v2.4.0 ) +require ( + github.com/cespare/xxhash v1.1.0 // indirect + github.com/cespare/xxhash/v2 v2.1.1 // indirect + github.com/cpuguy83/go-md2man/v2 v2.0.0 // indirect + github.com/dgraph-io/ristretto v0.1.0 // indirect + github.com/dustin/go-humanize v1.0.0 // indirect + github.com/fsnotify/fsnotify v1.4.9 // indirect + github.com/gogo/protobuf v1.3.2 // indirect + github.com/golang/glog v0.0.0-20160126235308-23def4e6c14b // indirect + github.com/golang/groupcache v0.0.0-20200121045136-8c9f03a8e57e // indirect + github.com/golang/protobuf v1.5.0 // indirect + github.com/golang/snappy v0.0.3 // indirect + github.com/google/flatbuffers v1.12.1 // indirect + github.com/google/uuid v1.2.0 // indirect + github.com/hashicorp/golang-lru v0.5.4 // indirect + github.com/hashicorp/hcl v1.0.0 // indirect + github.com/inconshreveable/mousetrap v1.0.0 // indirect + github.com/ipfs/bbloom v0.0.4 // indirect + github.com/ipfs/go-blockservice v0.2.0 // indirect + github.com/ipfs/go-ipfs-exchange-interface v0.1.0 // indirect + github.com/ipfs/go-ipfs-util v0.0.2 // indirect + github.com/ipfs/go-ipld-cbor v0.0.5 // indirect + github.com/ipfs/go-ipld-legacy v0.1.0 // indirect + github.com/ipfs/go-metrics-interface v0.0.1 // indirect + github.com/ipfs/go-verifcid v0.0.1 // indirect + github.com/ipld/go-codec-dagpb v1.3.0 // indirect + github.com/ipld/go-ipld-prime v0.11.0 // indirect + github.com/klauspost/compress v1.12.3 // indirect + github.com/klauspost/cpuid/v2 v2.0.6 // indirect + github.com/kr/text v0.2.0 // indirect + github.com/magiconair/properties v1.8.1 // indirect + github.com/mattn/go-isatty v0.0.13 // indirect + github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1 // indirect + github.com/minio/sha256-simd v1.0.0 // indirect + github.com/mitchellh/mapstructure v1.1.2 // indirect + github.com/mr-tron/base58 v1.2.0 // indirect + github.com/multiformats/go-base32 v0.0.3 // indirect + github.com/multiformats/go-base36 v0.1.0 // indirect + github.com/multiformats/go-varint v0.0.6 // indirect + github.com/opentracing/opentracing-go v1.2.0 // indirect + github.com/pelletier/go-toml v1.2.0 // indirect + github.com/pkg/errors v0.9.1 // indirect + github.com/pmezard/go-difflib v1.0.0 // indirect + github.com/polydawn/refmt v0.0.0-20201211092308-30ac6d18308e // indirect + github.com/russross/blackfriday/v2 v2.0.1 // indirect + github.com/shurcooL/sanitized_anchor_name v1.0.0 // indirect + github.com/spf13/afero v1.1.2 // indirect + github.com/spf13/cast v1.3.0 // indirect + github.com/spf13/jwalterweatherman v1.0.0 // indirect + github.com/spf13/pflag v1.0.5 // indirect + github.com/subosito/gotenv v1.2.0 // indirect + github.com/whyrusleeping/cbor-gen v0.0.0-20200123233031-1cdf64d27158 // indirect + github.com/x448/float16 v0.8.4 // indirect + go.opencensus.io v0.23.0 // indirect + go.uber.org/atomic v1.7.0 // indirect + go.uber.org/multierr v1.7.0 // indirect + golang.org/x/crypto v0.0.0-20210513164829-c07d793c2f9a // indirect + golang.org/x/net v0.0.0-20210423184538-5f58ad60dda6 // indirect + golang.org/x/sys v0.0.0-20210514084401-e8d321eab015 // indirect + golang.org/x/text v0.3.6 // indirect + golang.org/x/xerrors v0.0.0-20200804184101-5ec99f83aff1 // indirect + google.golang.org/protobuf v1.27.1 // indirect + gopkg.in/ini.v1 v1.51.0 // indirect + gopkg.in/yaml.v3 v3.0.0-20210107192922-496545a6307b // indirect + honnef.co/go/tools v0.1.3 // indirect +) + replace ( github.com/SierraSoftworks/connor => github.com/sourcenetwork/connor v1.0.3-0.20210312091030-4823d0411a12 diff --git a/go.sum b/go.sum index bfa5e2b896..3b1f573bae 100644 --- a/go.sum +++ b/go.sum @@ -52,7 +52,6 @@ github.com/benbjohnson/clock v1.1.0 h1:Q92kusRqC1XV2MjkWETPvjJVqKetz1OzxZB7mHJLj github.com/benbjohnson/clock v1.1.0/go.mod h1:J11/hYXuz8f4ySSvYwY0FKfm+ezbsZBKZxNJlLklBHA= github.com/beorn7/perks v0.0.0-20180321164747-3a771d992973/go.mod h1:Dwedo/Wpr24TaqPxmxbtue+5NUziq4I4S80YR8gNf3Q= github.com/beorn7/perks v1.0.0/go.mod h1:KWe93zE9D1o94FZ5RNwFwVgaQK1VOXiVxmqh+CedLV8= -github.com/beorn7/perks v1.0.1 h1:VlbKKnNfV8bJzeqoa4cOKqO6bYr3WgKZxO8Z16+hsOM= github.com/beorn7/perks v1.0.1/go.mod h1:G2ZrVWU2WbWT9wwq4/hrbKbnv/1ERSJQ0ibhJ6rlkpw= github.com/bgentry/speakeasy v0.1.0/go.mod h1:+zsyZBPWlz7T6j88CTgSN5bM796AkVf0kBD4zp0CCIs= github.com/bketelsen/crypt v0.0.3-0.20200106085610-5cbc8cc4026c/go.mod h1:MKsuJmJgSg28kpZDP6UIiPt0e0Oz0kqKNGyRaWEPv84= @@ -75,6 +74,8 @@ github.com/btcsuite/snappy-go v1.0.0/go.mod h1:8woku9dyThutzjeg+3xrA5iCpBRH8XEEg github.com/btcsuite/websocket v0.0.0-20150119174127-31079b680792/go.mod h1:ghJtEyQwv5/p4Mg4C0fgbePVuGr935/5ddU9Z3TmDRY= github.com/btcsuite/winsvc v1.0.0/go.mod h1:jsenWakMcC0zFBFurPLEAyrnc/teJEM1O46fmI40EZs= github.com/buger/jsonparser v0.0.0-20181115193947-bf1c66bbce23/go.mod h1:bbYlZJ7hK1yFx9hf58LP0zeX7UjIGs20ufpu3evjr+s= +github.com/bxcodec/faker v2.0.1+incompatible h1:P0KUpUw5w6WJXwrPfv35oc91i4d8nf40Nwln+M/+faA= +github.com/bxcodec/faker v2.0.1+incompatible/go.mod h1:BNzfpVdTwnFJ6GtfYTcQu6l6rHShT+veBxNCnjCx5XM= github.com/casbin/casbin/v2 v2.1.2/go.mod h1:YcPU1XXisHhLzuxH9coDNf2FbKpjGlbCg3n9yuLkIJQ= github.com/cenkalti/backoff v2.2.1+incompatible/go.mod h1:90ReRw6GdpyfrHakVjL/QHaoyV4aDUVVkXQJJJ3NXXM= github.com/census-instrumentation/opencensus-proto v0.2.1/go.mod h1:f6KPmirojxKA12rnyqOA5BBL4O983OfeGPqjHWSTneU= @@ -82,7 +83,6 @@ github.com/cespare/xxhash v1.1.0 h1:a6HrQnmkObjyL+Gs60czilIUGqrzKutQD6XZog3p+ko= github.com/cespare/xxhash v1.1.0/go.mod h1:XrSqR1VqqWfGrhpAt58auRo0WTKS1nRRg3ghfAqPWnc= github.com/cespare/xxhash/v2 v2.1.1 h1:6MnRN8NT7+YBpUIWxHtefFZOKTAPgGjpQSxqLNn0+qY= github.com/cespare/xxhash/v2 v2.1.1/go.mod h1:VGX0DQ3Q6kWi7AoAeZDth3/j3BFtOZR5XLFGgcrjCOs= -github.com/cheekybits/genny v1.0.0 h1:uGGa4nei+j20rOSeDeP5Of12XVm7TGUd4dJA9RDitfE= github.com/cheekybits/genny v1.0.0/go.mod h1:+tQajlRqAUrPI7DOSpB0XAqZYtQakVtB7wXkRAgjxjQ= github.com/clbanning/x2j v0.0.0-20191024224557-825249438eec/go.mod h1:jMjuTZXRI4dUb/I5gc9Hdhagfvm9+RyrPryS/auMzxE= github.com/client9/misspell v0.3.4/go.mod h1:qj6jICC3Q7zFZvVWo7KLAzC3yx5G7kyvSDkc90ppPyw= @@ -94,7 +94,6 @@ github.com/coreos/etcd v3.3.10+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc github.com/coreos/etcd v3.3.13+incompatible/go.mod h1:uF7uidLiAD3TWHmW31ZFd/JWoc32PjwdhPthX9715RE= github.com/coreos/go-etcd v2.0.0+incompatible/go.mod h1:Jez6KQU2B/sWsbdaef3ED8NzMklzPG4d5KIOhIy30Tk= github.com/coreos/go-semver v0.2.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= -github.com/coreos/go-semver v0.3.0 h1:wkHLiw0WNATZnSG7epLsujiMCgPAc9xhjJ4tgnAxmfM= github.com/coreos/go-semver v0.3.0/go.mod h1:nnelYz7RCh+5ahJtPPxZlU+153eP4D4r3EedlOD2RNk= github.com/coreos/go-systemd v0.0.0-20180511133405-39ca1b05acc7/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= github.com/coreos/go-systemd v0.0.0-20181012123002-c6f51f82210d/go.mod h1:F5haX7vjVVG0kc13fIWeqUViNPyEJxv/OmvnBo0Yme4= @@ -115,7 +114,6 @@ github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSs github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= github.com/davecgh/go-spew v1.1.1/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davidlazar/go-crypto v0.0.0-20170701192655-dcfb0a7ac018/go.mod h1:rQYf4tfk5sSwFsnDg3qYaBxSjsD9S8+59vW0dKUgme4= -github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c h1:pFUpOrbxDR6AkioZ1ySsx5yxlDQZ8stG2b88gTPxgJU= github.com/davidlazar/go-crypto v0.0.0-20200604182044-b73af7476f6c/go.mod h1:6UhI8N9EjYm1c2odKpFpAYeR8dsBeM7PtzQhRgxRr9U= github.com/decred/dcrd/lru v1.0.0/go.mod h1:mxKOwFd7lFjN2GZYsiz/ecgqR6kkYAl+0pz0tEMk218= github.com/dgraph-io/badger v1.5.5-0.20190226225317-8115aed38f8f/go.mod h1:VZxzAIRPHRVNRKRo6AXrX9BJegn6il06VMTZVJYCIjQ= @@ -147,9 +145,7 @@ github.com/envoyproxy/go-control-plane v0.9.4/go.mod h1:6rpuAdCZL397s3pYoYcLgu1m github.com/envoyproxy/protoc-gen-validate v0.1.0/go.mod h1:iSmxcyjqTsJpI2R4NaDN7+kN2VEUnK/pcBlmesArF7c= github.com/fatih/color v1.7.0/go.mod h1:Zm6kSWBoL9eyXnKyktHP6abPY2pDugNf5KwzbycvMj4= github.com/flynn/go-shlex v0.0.0-20150515145356-3f9db97f8568/go.mod h1:xEzjJPgXI435gkrCt3MPfRiAkVrwSbHsst4LCFVfpJc= -github.com/flynn/noise v1.0.0 h1:DlTHqmzmvcEiKj+4RYo/imoswx/4r6iBlCMfVtrMXpQ= github.com/flynn/noise v1.0.0/go.mod h1:xbMo+0i6+IGbYdJhF31t2eR1BIU0CYc12+BNAKwUTag= -github.com/francoispqt/gojay v1.2.13 h1:d2m3sFjloqoIUQU3TsHBgj6qg/BVGlTBeHDUmyJnXKk= github.com/francoispqt/gojay v1.2.13/go.mod h1:ehT5mTG4ua4581f1++1WLG0vPdaA9HaiDsoyrBGkyDY= github.com/franela/goblin v0.0.0-20200105215937-c9ffbefa60db/go.mod h1:7dvUGVsVBjqR7JHJk0brhHOZYGmfBYOrK0ZhYMEtBr4= github.com/franela/goreq v0.0.0-20171204163338-bcd34c9993f8/go.mod h1:ZhphrRTfi2rbfLwlschooIH4+wKKDR4Pdxhh+TRoA20= @@ -196,7 +192,6 @@ github.com/golang/mock v1.1.1/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfb github.com/golang/mock v1.2.0/go.mod h1:oTYuIxOrZwtPieC+H1uAHpcLFnEyAGVDL/k47Jfbm0A= github.com/golang/mock v1.3.1/go.mod h1:sBzyDLLjw3U8JLTeZvSv8jJB+tU5PVekmnlKIyFUx0Y= github.com/golang/mock v1.4.0/go.mod h1:UOMv5ysSaYNkG+OFQykRIcU/QvvxJf3p21QfJ2Bt3cw= -github.com/golang/mock v1.4.4 h1:l75CXGRSwbaYNpl/Z2X1XIIAMSCquvXgpVZDhwEIJsc= github.com/golang/mock v1.4.4/go.mod h1:l3mdAwkq5BuhzHwde/uurv3sEJeZMXNpwsxVWU71h+4= github.com/golang/protobuf v1.2.0/go.mod h1:6lQm79b+lXiMfvg/cZm0SGofjICqVBUtrP5yJMmIC1U= github.com/golang/protobuf v1.3.0/go.mod h1:Qd/q+1AKNOZr9uGQzbzCmRO6sUih6GTPZv6a1/R87v0= @@ -255,7 +250,6 @@ github.com/gorilla/mux v1.6.2/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2z github.com/gorilla/mux v1.7.3/go.mod h1:1lud6UwP+6orDFRuTfBEV8e9/aOM/c4fVVCaMa2zaAs= github.com/gorilla/websocket v0.0.0-20170926233335-4201258b820c/go.mod h1:E7qHFY5m1UJ88s3WnNqhKjPHQ0heANvMoAMk2YaljkQ= github.com/gorilla/websocket v1.4.1/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= -github.com/gorilla/websocket v1.4.2 h1:+/TMaTYc4QFitKJxsQ7Yye35DkWvkdLcvGKqM+x0Ufc= github.com/gorilla/websocket v1.4.2/go.mod h1:YR8l580nyteQvAITg2hZ9XVh4b55+EU/adAjf1fMHhE= github.com/gregjones/httpcache v0.0.0-20180305231024-9cad4c3443a7/go.mod h1:FecbI9+v66THATjSRHfNgh1IVFe/9kFxbXtjV0ctIMA= github.com/grpc-ecosystem/go-grpc-middleware v1.0.0/go.mod h1:FiyG127CGDf3tlThmgyCl78X/SZQqEOJBCDaAfeWzPs= @@ -394,10 +388,8 @@ github.com/jackpal/go-nat-pmp v1.0.1/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+ github.com/jackpal/go-nat-pmp v1.0.2 h1:KzKSgb7qkJvOUTqYl9/Hg/me3pWgBmERKrTGD7BdWus= github.com/jackpal/go-nat-pmp v1.0.2/go.mod h1:QPH045xvCAeXUZOxsnwmrtiCoxIr9eob+4orBN1SBKc= github.com/jbenet/go-cienv v0.0.0-20150120210510-1bb1476777ec/go.mod h1:rGaEvXB4uRSZMmzKNLoXvTu1sfx+1kv/DojUlPrSZGs= -github.com/jbenet/go-cienv v0.1.0 h1:Vc/s0QbQtoxX8MwwSLWWh+xNNZvM3Lw7NsTcHrvvhMc= github.com/jbenet/go-cienv v0.1.0/go.mod h1:TqNnHUmJgXau0nCzC7kXWeotg3J9W34CUv5Djy1+FlA= github.com/jbenet/go-temp-err-catcher v0.0.0-20150120210811-aac704a3f4f2/go.mod h1:8GXXJV31xl8whumTzdZsTt3RnUIiPqzkyf7mxToRCMs= -github.com/jbenet/go-temp-err-catcher v0.1.0 h1:zpb3ZH6wIE8Shj2sKS+khgRvf7T7RABoLk/+KKHggpk= github.com/jbenet/go-temp-err-catcher v0.1.0/go.mod h1:0kJRvmDZXNMIiJirNPEYfhpPwbGVtZVWC34vc5WLsDk= github.com/jbenet/goprocess v0.0.0-20160826012719-b497e2f366b8/go.mod h1:Ly/wlsjFq/qrU3Rar62tu1gASgGw6chQbSh/XgIIXCY= github.com/jbenet/goprocess v0.1.3/go.mod h1:5yspPrukOVuOLORacaBi858NqyClJPQxYZlqdZVfqY4= @@ -453,13 +445,11 @@ github.com/libp2p/go-buffer-pool v0.0.2 h1:QNK2iAFa8gjAe1SPz6mHSMuCcjs+X1wlHzeOS github.com/libp2p/go-buffer-pool v0.0.2/go.mod h1:MvaB6xw5vOrDl8rYZGLFdKAuk/hRoRZd1Vi32+RXyFM= github.com/libp2p/go-conn-security-multistream v0.1.0/go.mod h1:aw6eD7LOsHEX7+2hJkDxw1MteijaVcI+/eP2/x3J1xc= github.com/libp2p/go-conn-security-multistream v0.2.0/go.mod h1:hZN4MjlNetKD3Rq5Jb/P5ohUnFLNzEAR4DLSzpn2QLU= -github.com/libp2p/go-conn-security-multistream v0.2.1 h1:ft6/POSK7F+vl/2qzegnHDaXFU0iWB4yVTYrioC6Zy0= github.com/libp2p/go-conn-security-multistream v0.2.1/go.mod h1:cR1d8gA0Hr59Fj6NhaTpFhJZrjSYuNmhpT2r25zYR70= github.com/libp2p/go-eventbus v0.1.0/go.mod h1:vROgu5cs5T7cv7POWlWxBaVLxfSegC5UGQf8A2eEmx4= github.com/libp2p/go-eventbus v0.2.1 h1:VanAdErQnpTioN2TowqNcOijf6YwhuODe4pPKSDpxGc= github.com/libp2p/go-eventbus v0.2.1/go.mod h1:jc2S4SoEVPP48H9Wpzm5aiGwUCBMfGhVhhBjyhhCJs8= github.com/libp2p/go-flow-metrics v0.0.1/go.mod h1:Iv1GH0sG8DtYN3SVJ2eG221wMiNpZxBdp967ls1g+k8= -github.com/libp2p/go-flow-metrics v0.0.3 h1:8tAs/hSdNvUiLgtlSy3mxwxWP4I9y/jlkPFT7epKdeM= github.com/libp2p/go-flow-metrics v0.0.3/go.mod h1:HeoSNUrOJVK1jEpDqVEiUOIXqhbnS27omG0uWU5slZs= github.com/libp2p/go-libp2p v0.6.1/go.mod h1:CTFnWXogryAHjXAKEbOf1OWY+VeAP3lDMZkfEI5sT54= github.com/libp2p/go-libp2p v0.7.0/go.mod h1:hZJf8txWeCduQRDC/WSqBGMxaTHCOYHt2xSU1ivxn0k= @@ -475,11 +465,9 @@ github.com/libp2p/go-libp2p-autonat v0.4.2 h1:YMp7StMi2dof+baaxkbxaizXjY1RPvU71C github.com/libp2p/go-libp2p-autonat v0.4.2/go.mod h1:YxaJlpr81FhdOv3W3BTconZPfhaYivRdf53g+S2wobk= github.com/libp2p/go-libp2p-blankhost v0.1.1/go.mod h1:pf2fvdLJPsC1FsVrNP3DUUvMzUts2dsLLBEpo1vW1ro= github.com/libp2p/go-libp2p-blankhost v0.1.4/go.mod h1:oJF0saYsAXQCSfDq254GMNmLNz6ZTHTOvtF4ZydUvwU= -github.com/libp2p/go-libp2p-blankhost v0.2.0 h1:3EsGAi0CBGcZ33GwRuXEYJLLPoVWyXJ1bcJzAJjINkk= github.com/libp2p/go-libp2p-blankhost v0.2.0/go.mod h1:eduNKXGTioTuQAUcZ5epXi9vMl+t4d8ugUBRQ4SqaNQ= github.com/libp2p/go-libp2p-circuit v0.1.4/go.mod h1:CY67BrEjKNDhdTk8UgBX1Y/H5c3xkAcs3gnksxY7osU= github.com/libp2p/go-libp2p-circuit v0.2.1/go.mod h1:BXPwYDN5A8z4OEY9sOfr2DUQMLQvKt/6oku45YUmjIo= -github.com/libp2p/go-libp2p-circuit v0.4.0 h1:eqQ3sEYkGTtybWgr6JLqJY6QLtPWRErvFjFDfAOO1wc= github.com/libp2p/go-libp2p-circuit v0.4.0/go.mod h1:t/ktoFIUzM6uLQ+o1G6NuBl2ANhBKN9Bc8jRIk31MoA= github.com/libp2p/go-libp2p-core v0.0.1/go.mod h1:g/VxnTZ/1ygHxH3dKok7Vno1VfpvGcGip57wjTU4fco= github.com/libp2p/go-libp2p-core v0.0.2/go.mod h1:9dAcntw/n46XycV4RnlBq3BpgrmyUi9LuoTNdPrbUco= @@ -506,7 +494,6 @@ github.com/libp2p/go-libp2p-core v0.8.5/go.mod h1:FfewUH/YpvWbEB+ZY9AQRQ4TAD8sJB github.com/libp2p/go-libp2p-crypto v0.1.0/go.mod h1:sPUokVISZiy+nNuTTH/TY+leRSxnFj/2GLjtOTW90hI= github.com/libp2p/go-libp2p-discovery v0.2.0/go.mod h1:s4VGaxYMbw4+4+tsoQTqh7wfxg97AEdo4GYBt6BadWg= github.com/libp2p/go-libp2p-discovery v0.3.0/go.mod h1:o03drFnz9BVAZdzC/QUQ+NeQOu38Fu7LJGEOK2gQltw= -github.com/libp2p/go-libp2p-discovery v0.5.0 h1:Qfl+e5+lfDgwdrXdu4YNCWyEo3fWuP+WgN9mN0iWviQ= github.com/libp2p/go-libp2p-discovery v0.5.0/go.mod h1:+srtPIU9gDaBNu//UHvcdliKBIcr4SfDcm0/PfPJLug= github.com/libp2p/go-libp2p-loggables v0.1.0 h1:h3w8QFfCt2UJl/0/NW4K829HX/0S4KD31PQ7m8UXXO8= github.com/libp2p/go-libp2p-loggables v0.1.0/go.mod h1:EyumB2Y6PrYjr55Q3/tiJ/o3xoDasoRYM7nOzEpoa90= @@ -515,14 +502,12 @@ github.com/libp2p/go-libp2p-mplex v0.2.1/go.mod h1:SC99Rxs8Vuzrf/6WhmH41kNn13TiY github.com/libp2p/go-libp2p-mplex v0.2.2/go.mod h1:74S9eum0tVQdAfFiKxAyKzNdSuLqw5oadDq7+L/FELo= github.com/libp2p/go-libp2p-mplex v0.2.3/go.mod h1:CK3p2+9qH9x+7ER/gWWDYJ3QW5ZxWDkm+dVvjfuG3ek= github.com/libp2p/go-libp2p-mplex v0.4.0/go.mod h1:yCyWJE2sc6TBTnFpjvLuEJgTSw/u+MamvzILKdX7asw= -github.com/libp2p/go-libp2p-mplex v0.4.1 h1:/pyhkP1nLwjG3OM+VuaNJkQT/Pqq73WzB3aDN3Fx1sc= github.com/libp2p/go-libp2p-mplex v0.4.1/go.mod h1:cmy+3GfqfM1PceHTLL7zQzAAYaryDu6iPSC+CIb094g= github.com/libp2p/go-libp2p-nat v0.0.5/go.mod h1:1qubaE5bTZMJE+E/uu2URroMbzdubFz1ChgiN79yKPE= github.com/libp2p/go-libp2p-nat v0.0.6 h1:wMWis3kYynCbHoyKLPBEMu4YRLltbm8Mk08HGSfvTkU= github.com/libp2p/go-libp2p-nat v0.0.6/go.mod h1:iV59LVhB3IkFvS6S6sauVTSOrNEANnINbI/fkaLimiw= github.com/libp2p/go-libp2p-netutil v0.1.0 h1:zscYDNVEcGxyUpMd0JReUZTrpMfia8PmLKcKF72EAMQ= github.com/libp2p/go-libp2p-netutil v0.1.0/go.mod h1:3Qv/aDqtMLTUyQeundkKsA+YCThNdbQD54k3TqjpbFU= -github.com/libp2p/go-libp2p-noise v0.2.0 h1:wmk5nhB9a2w2RxMOyvsoKjizgJOEaJdfAakr0jN8gds= github.com/libp2p/go-libp2p-noise v0.2.0/go.mod h1:IEbYhBBzGyvdLBoxxULL/SGbJARhUeqlO8lVSREYu2Q= github.com/libp2p/go-libp2p-peer v0.2.0/go.mod h1:RCffaCvUyW2CJmG2gAWVqwePwW7JMgxjsHm7+J5kjWY= github.com/libp2p/go-libp2p-peerstore v0.1.0/go.mod h1:2CeHkQsr8svp4fZ+Oi9ykN1HBb6u0MOvdJ7YIsmcwtY= @@ -533,9 +518,7 @@ github.com/libp2p/go-libp2p-peerstore v0.2.2/go.mod h1:NQxhNjWxf1d4w6PihR8btWIRj github.com/libp2p/go-libp2p-peerstore v0.2.6/go.mod h1:ss/TWTgHZTMpsU/oKVVPQCGuDHItOpf2W8RxAi50P2s= github.com/libp2p/go-libp2p-peerstore v0.2.7 h1:83JoLxyR9OYTnNfB5vvFqvMUv/xDNa6NoPHnENhBsGw= github.com/libp2p/go-libp2p-peerstore v0.2.7/go.mod h1:ss/TWTgHZTMpsU/oKVVPQCGuDHItOpf2W8RxAi50P2s= -github.com/libp2p/go-libp2p-pnet v0.2.0 h1:J6htxttBipJujEjz1y0a5+eYoiPcFHhSYHH6na5f0/k= github.com/libp2p/go-libp2p-pnet v0.2.0/go.mod h1:Qqvq6JH/oMZGwqs3N1Fqhv8NVhrdYcO0BW4wssv21LA= -github.com/libp2p/go-libp2p-quic-transport v0.10.0 h1:koDCbWD9CCHwcHZL3/WEvP2A+e/o5/W5L3QS/2SPMA0= github.com/libp2p/go-libp2p-quic-transport v0.10.0/go.mod h1:RfJbZ8IqXIhxBRm5hqUEJqjiiY8xmEuq3HUDS993MkA= github.com/libp2p/go-libp2p-record v0.1.0 h1:wHwBGbFzymoIl69BpgwIu0O6ta3TXGcMPvHUAcodzRc= github.com/libp2p/go-libp2p-record v0.1.0/go.mod h1:ujNc8iuE5dlKWVy6wuL6dd58t0n7xI4hAIl8pE6wu5Q= @@ -548,7 +531,6 @@ github.com/libp2p/go-libp2p-swarm v0.2.2/go.mod h1:fvmtQ0T1nErXym1/aa1uJEyN7JzaT github.com/libp2p/go-libp2p-swarm v0.2.3/go.mod h1:P2VO/EpxRyDxtChXz/VPVXyTnszHvokHKRhfkEgFKNM= github.com/libp2p/go-libp2p-swarm v0.2.8/go.mod h1:JQKMGSth4SMqonruY0a8yjlPVIkb0mdNSwckW7OYziM= github.com/libp2p/go-libp2p-swarm v0.3.0/go.mod h1:hdv95GWCTmzkgeJpP+GK/9D9puJegb7H57B5hWQR5Kk= -github.com/libp2p/go-libp2p-swarm v0.5.0 h1:HIK0z3Eqoo8ugmN8YqWAhD2RORgR+3iNXYG4U2PFd1E= github.com/libp2p/go-libp2p-swarm v0.5.0/go.mod h1:sU9i6BoHE0Ve5SKz3y9WfKrh8dUat6JknzUehFx8xW4= github.com/libp2p/go-libp2p-testing v0.0.2/go.mod h1:gvchhf3FQOtBdr+eFUABet5a4MBLK8jM3V4Zghvmi+E= github.com/libp2p/go-libp2p-testing v0.0.3/go.mod h1:gvchhf3FQOtBdr+eFUABet5a4MBLK8jM3V4Zghvmi+E= @@ -559,12 +541,10 @@ github.com/libp2p/go-libp2p-testing v0.1.2-0.20200422005655-8775583591d8/go.mod github.com/libp2p/go-libp2p-testing v0.3.0/go.mod h1:efZkql4UZ7OVsEfaxNHZPzIehtsBXMrXnCfJIgDti5g= github.com/libp2p/go-libp2p-testing v0.4.0 h1:PrwHRi0IGqOwVQWR3xzgigSlhlLfxgfXgkHxr77EghQ= github.com/libp2p/go-libp2p-testing v0.4.0/go.mod h1:Q+PFXYoiYFN5CAEG2w3gLPEzotlKsNSbKQ/lImlOWF0= -github.com/libp2p/go-libp2p-tls v0.1.3 h1:twKMhMu44jQO+HgQK9X8NHO5HkeJu2QbhLzLJpa8oNM= github.com/libp2p/go-libp2p-tls v0.1.3/go.mod h1:wZfuewxOndz5RTnCAxFliGjvYSDA40sKitV4c50uI1M= github.com/libp2p/go-libp2p-transport-upgrader v0.1.1/go.mod h1:IEtA6or8JUbsV07qPW4r01GnTenLW4oi3lOPbUMGJJA= github.com/libp2p/go-libp2p-transport-upgrader v0.2.0/go.mod h1:mQcrHj4asu6ArfSoMuyojOdjx73Q47cYD7s5+gZOlns= github.com/libp2p/go-libp2p-transport-upgrader v0.3.0/go.mod h1:i+SKzbRnvXdVbU3D1dwydnTmKRPXiAR/fyvi1dXuL4o= -github.com/libp2p/go-libp2p-transport-upgrader v0.4.2 h1:4JsnbfJzgZeRS9AWN7B9dPqn/LY/HoQTlO9gtdJTIYM= github.com/libp2p/go-libp2p-transport-upgrader v0.4.2/go.mod h1:NR8ne1VwfreD5VIWIU62Agt/J18ekORFU/j1i2y8zvk= github.com/libp2p/go-libp2p-yamux v0.2.0/go.mod h1:Db2gU+XfLpm6E4rG5uGCFX6uXA8MEXOxFcRoXUODaK8= github.com/libp2p/go-libp2p-yamux v0.2.2/go.mod h1:lIohaR0pT6mOt0AZ0L2dFze9hds9Req3OfS+B+dv4qw= @@ -573,18 +553,15 @@ github.com/libp2p/go-libp2p-yamux v0.2.7/go.mod h1:X28ENrBMU/nm4I3Nx4sZ4dgjZ6VhL github.com/libp2p/go-libp2p-yamux v0.2.8/go.mod h1:/t6tDqeuZf0INZMTgd0WxIRbtK2EzI2h7HbFm9eAKI4= github.com/libp2p/go-libp2p-yamux v0.4.0/go.mod h1:+DWDjtFMzoAwYLVkNZftoucn7PelNoy5nm3tZ3/Zw30= github.com/libp2p/go-libp2p-yamux v0.5.0/go.mod h1:AyR8k5EzyM2QN9Bbdg6X1SkVVuqLwTGf0L4DFq9g6po= -github.com/libp2p/go-libp2p-yamux v0.5.4 h1:/UOPtT/6DHPtr3TtKXBHa6g0Le0szYuI33Xc/Xpd7fQ= github.com/libp2p/go-libp2p-yamux v0.5.4/go.mod h1:tfrXbyaTqqSU654GTvK3ocnSZL3BuHoeTSqhcel1wsE= github.com/libp2p/go-maddr-filter v0.0.4/go.mod h1:6eT12kSQMA9x2pvFQa+xesMKUBlj9VImZbj3B9FBH/Q= github.com/libp2p/go-maddr-filter v0.0.5/go.mod h1:Jk+36PMfIqCJhAnaASRH83bdAvfDRp/w6ENFaC9bG+M= -github.com/libp2p/go-maddr-filter v0.1.0 h1:4ACqZKw8AqiuJfwFGq1CYDFugfXTOos+qQ3DETkhtCE= github.com/libp2p/go-maddr-filter v0.1.0/go.mod h1:VzZhTXkMucEGGEOSKddrwGiOv0tUhgnKqNEmIAz/bPU= github.com/libp2p/go-mplex v0.0.3/go.mod h1:pK5yMLmOoBR1pNCqDlA2GQrdAVTMkqFalaTWe7l4Yd0= github.com/libp2p/go-mplex v0.1.0/go.mod h1:SXgmdki2kwCUlCCbfGLEgHjC4pFqhTp0ZoV6aiKgxDU= github.com/libp2p/go-mplex v0.1.1/go.mod h1:Xgz2RDCi3co0LeZfgjm4OgUF15+sVR8SRcu3SFXI1lk= github.com/libp2p/go-mplex v0.1.2/go.mod h1:Xgz2RDCi3co0LeZfgjm4OgUF15+sVR8SRcu3SFXI1lk= github.com/libp2p/go-mplex v0.2.0/go.mod h1:0Oy/A9PQlwBytDRp4wSkFnzHYDKcpLot35JQ6msjvYQ= -github.com/libp2p/go-mplex v0.3.0 h1:U1T+vmCYJaEoDJPV1aq31N56hS+lJgb397GsylNSgrU= github.com/libp2p/go-mplex v0.3.0/go.mod h1:0Oy/A9PQlwBytDRp4wSkFnzHYDKcpLot35JQ6msjvYQ= github.com/libp2p/go-msgio v0.0.2/go.mod h1:63lBBgOTDKQL6EWazRMCwXsEeEeK9O2Cd+0+6OOuipQ= github.com/libp2p/go-msgio v0.0.4/go.mod h1:63lBBgOTDKQL6EWazRMCwXsEeEeK9O2Cd+0+6OOuipQ= @@ -605,11 +582,9 @@ github.com/libp2p/go-openssl v0.0.5/go.mod h1:unDrJpgy3oFr+rqXsarWifmJuNnJR4chtO github.com/libp2p/go-openssl v0.0.7 h1:eCAzdLejcNVBzP/iZM9vqHnQm+XyCEbSSIheIPRGNsw= github.com/libp2p/go-openssl v0.0.7/go.mod h1:unDrJpgy3oFr+rqXsarWifmJuNnJR4chtO1HmaZjggc= github.com/libp2p/go-reuseport v0.0.1/go.mod h1:jn6RmB1ufnQwl0Q1f+YxAj8isJgDCQzaaxIFYDhcYEA= -github.com/libp2p/go-reuseport v0.0.2 h1:XSG94b1FJfGA01BUrT82imejHQyTxO4jEWqheyCXYvU= github.com/libp2p/go-reuseport v0.0.2/go.mod h1:SPD+5RwGC7rcnzngoYC86GjPzjSywuQyMVAheVBD9nQ= github.com/libp2p/go-reuseport-transport v0.0.2/go.mod h1:YkbSDrvjUVDL6b8XqriyA20obEtsW9BLkuOUyQAOCbs= github.com/libp2p/go-reuseport-transport v0.0.3/go.mod h1:Spv+MPft1exxARzP2Sruj2Wb5JSyHNncjf1Oi2dEbzM= -github.com/libp2p/go-reuseport-transport v0.0.4 h1:OZGz0RB620QDGpv300n1zaOcKGGAoGVf8h9txtt/1uM= github.com/libp2p/go-reuseport-transport v0.0.4/go.mod h1:trPa7r/7TJK/d+0hdBLOCGvpQQVOU74OXbNCIMkufGw= github.com/libp2p/go-sockaddr v0.0.2/go.mod h1:syPvOmNs24S3dFVGJA1/mrqdeijPxLV2Le3BRLKd68k= github.com/libp2p/go-sockaddr v0.1.0/go.mod h1:syPvOmNs24S3dFVGJA1/mrqdeijPxLV2Le3BRLKd68k= @@ -617,16 +592,13 @@ github.com/libp2p/go-sockaddr v0.1.1 h1:yD80l2ZOdGksnOyHrhxDdTDFrf7Oy+v3FMVArIRg github.com/libp2p/go-sockaddr v0.1.1/go.mod h1:syPvOmNs24S3dFVGJA1/mrqdeijPxLV2Le3BRLKd68k= github.com/libp2p/go-stream-muxer v0.0.1/go.mod h1:bAo8x7YkSpadMTbtTaxGVHWUQsR/l5MEaHbKaliuT14= github.com/libp2p/go-stream-muxer-multistream v0.2.0/go.mod h1:j9eyPol/LLRqT+GPLSxvimPhNph4sfYfMoDPd7HkzIc= -github.com/libp2p/go-stream-muxer-multistream v0.3.0 h1:TqnSHPJEIqDEO7h1wZZ0p3DXdvDSiLHQidKKUGZtiOY= github.com/libp2p/go-stream-muxer-multistream v0.3.0/go.mod h1:yDh8abSIzmZtqtOt64gFJUXEryejzNb0lisTt+fAMJA= github.com/libp2p/go-tcp-transport v0.1.0/go.mod h1:oJ8I5VXryj493DEJ7OsBieu8fcg2nHGctwtInJVpipc= github.com/libp2p/go-tcp-transport v0.1.1/go.mod h1:3HzGvLbx6etZjnFlERyakbaYPdfjg2pWP97dFZworkY= github.com/libp2p/go-tcp-transport v0.2.0/go.mod h1:vX2U0CnWimU4h0SGSEsg++AzvBcroCGYw28kh94oLe0= -github.com/libp2p/go-tcp-transport v0.2.3 h1:Esfw8dbK3zwdZItnIJGmOOg1YGDYja4eOmJcoDx29uU= github.com/libp2p/go-tcp-transport v0.2.3/go.mod h1:9dvr03yqrPyYGIEN6Dy5UvdJZjyPFvl1S/igQ5QD1SU= github.com/libp2p/go-ws-transport v0.2.0/go.mod h1:9BHJz/4Q5A9ludYWKoGCFC5gUElzlHoKzu0yY9p/klM= github.com/libp2p/go-ws-transport v0.3.0/go.mod h1:bpgTJmRZAvVHrgHybCVyqoBmyLQ1fiZuEaBYusP5zsk= -github.com/libp2p/go-ws-transport v0.4.0 h1:9tvtQ9xbws6cA5LvqdE6Ne3vcmGB4f1z9SByggk4s0k= github.com/libp2p/go-ws-transport v0.4.0/go.mod h1:EcIEKqf/7GDjth6ksuS/6p7R49V4CBY6/E7R/iyhYUA= github.com/libp2p/go-yamux v1.2.2/go.mod h1:FGTiPvoV/3DVdgWpX+tM0OW3tsM+W5bSE3gZwqQTcow= github.com/libp2p/go-yamux v1.3.0/go.mod h1:FGTiPvoV/3DVdgWpX+tM0OW3tsM+W5bSE3gZwqQTcow= @@ -634,13 +606,10 @@ github.com/libp2p/go-yamux v1.3.3/go.mod h1:FGTiPvoV/3DVdgWpX+tM0OW3tsM+W5bSE3gZ github.com/libp2p/go-yamux v1.3.5/go.mod h1:FGTiPvoV/3DVdgWpX+tM0OW3tsM+W5bSE3gZwqQTcow= github.com/libp2p/go-yamux v1.3.7/go.mod h1:fr7aVgmdNGJK+N1g+b6DW6VxzbRCjCOejR/hkmpooHE= github.com/libp2p/go-yamux v1.4.0/go.mod h1:fr7aVgmdNGJK+N1g+b6DW6VxzbRCjCOejR/hkmpooHE= -github.com/libp2p/go-yamux v1.4.1 h1:P1Fe9vF4th5JOxxgQvfbOHkrGqIZniTLf+ddhZp8YTI= github.com/libp2p/go-yamux v1.4.1/go.mod h1:fr7aVgmdNGJK+N1g+b6DW6VxzbRCjCOejR/hkmpooHE= -github.com/libp2p/go-yamux/v2 v2.2.0 h1:RwtpYZ2/wVviZ5+3pjC8qdQ4TKnrak0/E01N1UWoAFU= github.com/libp2p/go-yamux/v2 v2.2.0/go.mod h1:3So6P6TV6r75R9jiBpiIKgU/66lOarCZjqROGxzPpPQ= github.com/lightstep/lightstep-tracer-common/golang/gogo v0.0.0-20190605223551-bc2310a04743/go.mod h1:qklhhLq1aX+mtWk9cPHPzaBjWImj5ULL6C7HFJtXQMM= github.com/lightstep/lightstep-tracer-go v0.18.1/go.mod h1:jlF1pusYV4pidLvZ+XD0UBX0ZE6WURAspgAczcDHrL4= -github.com/lucas-clemente/quic-go v0.19.3 h1:eCDQqvGBB+kCTkA0XrAFtNe81FMa0/fn4QSoeAbmiF4= github.com/lucas-clemente/quic-go v0.19.3/go.mod h1:ADXpNbTQjq1hIzCpB+y/k5iz4n4z4IwqoLb94Kh5Hu8= github.com/lunixbochs/vtclean v1.0.0/go.mod h1:pHhQNgMf3btfWnGBVipUOjRYhoOsdGqdm/+2c2E2WMI= github.com/lyft/protoc-gen-validate v0.0.13/go.mod h1:XbGvPuh87YZc5TdIa2/I4pLk0QoUACkjt2znoq26NVQ= @@ -650,11 +619,8 @@ github.com/magiconair/properties v1.8.1/go.mod h1:PppfXfuXeibc/6YijjN8zIbojt8czP github.com/mailru/easyjson v0.0.0-20180823135443-60711f1a8329/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/mailru/easyjson v0.0.0-20190312143242-1de009706dbe/go.mod h1:C1wdFJiN94OJF2b5HbByQZoLdCWB1Yqtg26g4irojpc= github.com/marten-seemann/qpack v0.2.1/go.mod h1:F7Gl5L1jIgN1D11ucXefiuJS9UMVP2opoCp2jDKb7wc= -github.com/marten-seemann/qtls v0.10.0 h1:ECsuYUKalRL240rRD4Ri33ISb7kAQ3qGDlrrl55b2pc= github.com/marten-seemann/qtls v0.10.0/go.mod h1:UvMd1oaYDACI99/oZUYLzMCkBXQVT0aGm99sJhbT8hs= -github.com/marten-seemann/qtls-go1-15 v0.1.1 h1:LIH6K34bPVttyXnUWixk0bzH6/N07VxbSabxn5A5gZQ= github.com/marten-seemann/qtls-go1-15 v0.1.1/go.mod h1:GyFwywLKkRt+6mfU99csTEY1joMZz5vmB1WNZH3P81I= -github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd h1:br0buuQ854V8u83wA0rVZ8ttrq5CpaPZdvrK0LP2lOk= github.com/marten-seemann/tcp v0.0.0-20210406111302-dfbc87cc63fd/go.mod h1:QuCEs1Nt24+FYQEqAAncTDPJIuGs+LxK1MCiFL25pMU= github.com/mattn/go-colorable v0.0.9/go.mod h1:9vuHe8Xs5qXnSaW/c/ABM9alt+Vo+STaOChaDxuIBZU= github.com/mattn/go-colorable v0.1.1/go.mod h1:FuOcm+DKB9mbwrcAfNl7/TZVBZ6rcnceauSikq3lYCQ= @@ -664,7 +630,6 @@ github.com/mattn/go-isatty v0.0.5/go.mod h1:Iq45c/XA43vh69/j3iqttzPXn0bhXyGjM0Hd github.com/mattn/go-isatty v0.0.13 h1:qdl+GuBjcsKKDco5BsxPJlId98mSWNKqYA+Co0SC1yA= github.com/mattn/go-isatty v0.0.13/go.mod h1:cbi8OIDigv2wuxKPP5vlRcQ1OAZbq2CE4Kysco4FUpU= github.com/mattn/go-runewidth v0.0.2/go.mod h1:LwmH8dsx7+W8Uxz3IHJYH5QSwggIsqBzpuz5H//U1FU= -github.com/matttproud/golang_protobuf_extensions v1.0.1 h1:4hp9jkHxhMHkqkrB3Ix0jegS5sx/RkqARlsWZ6pIwiU= github.com/matttproud/golang_protobuf_extensions v1.0.1/go.mod h1:D8He9yQNgCq6Z5Ld7szi9bcBfOoFv/3dc6xSMkL2PC0= github.com/mgutz/ansi v0.0.0-20170206155736-9520e82c474b/go.mod h1:01TrycV0kFyexm33Z7vhZRXopbI8J3TDReVlkTgMUxE= github.com/microcosm-cc/bluemonday v1.0.1/go.mod h1:hsXNsILzKxV+sX77C5b8FSuKF00vh2OMYv+xgHpAMF4= @@ -673,11 +638,8 @@ github.com/miekg/dns v1.1.12/go.mod h1:W1PPwlIAgtquWBMBEV9nkV9Cazfe8ScdGz/Lj7v3N github.com/miekg/dns v1.1.28/go.mod h1:KNUDUusw/aVsxyTYZM1oqvCicbwhgbNgztCETuNZ7xM= github.com/miekg/dns v1.1.41 h1:WMszZWJG0XmzbK9FEmzH2TVcqYzFesusSIB41b8KHxY= github.com/miekg/dns v1.1.41/go.mod h1:p6aan82bvRIyn+zDIv9xYNUpwa73JcSh9BKwknJysuI= -github.com/mikioh/tcp v0.0.0-20190314235350-803a9b46060c h1:bzE/A84HN25pxAuk9Eej1Kz9OUelF97nAc82bDquQI8= github.com/mikioh/tcp v0.0.0-20190314235350-803a9b46060c/go.mod h1:0SQS9kMwD2VsyFEB++InYyBJroV/FRmBgcydeSUcJms= -github.com/mikioh/tcpinfo v0.0.0-20190314235526-30a79bb1804b h1:z78hV3sbSMAUoyUMM0I83AUIT6Hu17AWfgjzIbtrYFc= github.com/mikioh/tcpinfo v0.0.0-20190314235526-30a79bb1804b/go.mod h1:lxPUiZwKoFL8DUUmalo2yJJUCxbPKtm8OKfqr2/FTNU= -github.com/mikioh/tcpopt v0.0.0-20190314235656-172688c1accc h1:PTfri+PuQmWDqERdnNMiD9ZejrlswWrCpBEZgWOiTrc= github.com/mikioh/tcpopt v0.0.0-20190314235656-172688c1accc/go.mod h1:cGKTAVKx4SxOuR/czcZ/E2RSJ3sfHs8FpHhQ5CWMf9s= github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1 h1:lYpkrQH5ajf0OXOcUbGjvZxxijuBwbbmlSxLiuofa+g= github.com/minio/blake2b-simd v0.0.0-20160723061019-3f5f724cb5b1/go.mod h1:pD8RvIylQ358TN4wwqatJ8rNavkEINozVn9DtGI3dfQ= @@ -833,14 +795,12 @@ github.com/prometheus/client_golang v0.9.3/go.mod h1:/TN21ttK/J9q6uSwhBd54HahCDf github.com/prometheus/client_golang v1.0.0/go.mod h1:db9x61etRT2tGnBNRi70OPL5FsnadC4Ky3P0J6CfImo= github.com/prometheus/client_golang v1.3.0/go.mod h1:hJaj2vgQTGQmVCsAACORcieXFeDPbaTKGT+JTgUa3og= github.com/prometheus/client_golang v1.7.1/go.mod h1:PY5Wy2awLA44sXw4AOSfFBetzPP4j5+D6mVACh+pe2M= -github.com/prometheus/client_golang v1.10.0 h1:/o0BDeWzLWXNZ+4q5gXltUvaMpJqckTa+jTNoB+z4cg= github.com/prometheus/client_golang v1.10.0/go.mod h1:WJM3cc3yu7XKBKa/I8WeZm+V3eltZnBwfENSU7mdogU= github.com/prometheus/client_model v0.0.0-20180712105110-5c3871d89910/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190115171406-56726106282f/go.mod h1:MbSGuTsp3dbXC40dX6PRTWyKYBIrTGTE9sqQNg2J8bo= github.com/prometheus/client_model v0.0.0-20190129233127-fd36f4220a90/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.0.0-20190812154241-14fe0d1b01d4/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/client_model v0.1.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= -github.com/prometheus/client_model v0.2.0 h1:uq5h0d+GuxiXLJLNABMgp2qUWDPiLvgCzz2dUR+/W/M= github.com/prometheus/client_model v0.2.0/go.mod h1:xMI15A0UPsDsEKsMN9yxemIoYk6Tm2C1GtYGdfGttqA= github.com/prometheus/common v0.0.0-20180801064454-c7de2306084e/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= github.com/prometheus/common v0.0.0-20181113130724-41aa239b4cce/go.mod h1:daVV7qP5qjZbuso7PdcryaAu0sAZbrN9i7WWcTMWvro= @@ -849,7 +809,6 @@ github.com/prometheus/common v0.4.0/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y8 github.com/prometheus/common v0.4.1/go.mod h1:TNfzLD0ON7rHzMJeJkieUDPYmFC7Snx/y86RQel1bk4= github.com/prometheus/common v0.7.0/go.mod h1:DjGbpBbp5NYNiECxcL/VnbXCCaQpKd3tt26CguLLsqA= github.com/prometheus/common v0.10.0/go.mod h1:Tlit/dnDKsSWFlCLTWaA1cyBgKHSMdTB80sz/V91rCo= -github.com/prometheus/common v0.18.0 h1:WCVKW7aL6LEe1uryfI9dnEc2ZqNB1Fn0ok930v0iL1Y= github.com/prometheus/common v0.18.0/go.mod h1:U+gB1OBLb1lF3O42bTCL+FK18tX9Oar16Clt/msog/s= github.com/prometheus/procfs v0.0.0-20180725123919-05ee40e3a273/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= github.com/prometheus/procfs v0.0.0-20181005140218-185b4288413d/go.mod h1:c3At6R/oaqEKCNdg8wHV1ftS6bRYblBhIjjI8uT2IGk= @@ -858,7 +817,6 @@ github.com/prometheus/procfs v0.0.0-20190507164030-5867b95ac084/go.mod h1:TjEm7z github.com/prometheus/procfs v0.0.2/go.mod h1:TjEm7ze935MbeOT/UhFTIMYKhuLP4wbCsTZCD3I8kEA= github.com/prometheus/procfs v0.0.8/go.mod h1:7Qr8sr6344vo1JqZ6HhLceV9o3AJ1Ff+GxbHq6oeK9A= github.com/prometheus/procfs v0.1.3/go.mod h1:lV6e/gmhEcM9IjHGsFOCxxuZ+z1YqCvr4OA4YeYWdaU= -github.com/prometheus/procfs v0.6.0 h1:mxy4L2jP6qMonqmq+aTtOx1ifVWUgG/TAmntgbh3xv4= github.com/prometheus/procfs v0.6.0/go.mod h1:cz+aTbrPOrUb4q7XlbU9ygM+/jj0fzG6c1xBZuNvfVA= github.com/prometheus/tsdb v0.7.1/go.mod h1:qhTCs0VvXwvX/y3TZrWD7rabWM+ijKTux40TwIPHuXU= github.com/rcrowley/go-metrics v0.0.0-20181016184325-3113b8401b8a/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4= @@ -976,7 +934,6 @@ github.com/whyrusleeping/go-logging v0.0.0-20170515211332-0457bb6b88fc/go.mod h1 github.com/whyrusleeping/go-logging v0.0.1/go.mod h1:lDPYj54zutzG1XYfHAhcc7oNXEburHQBn+Iqd4yS4vE= github.com/whyrusleeping/mafmt v1.2.8/go.mod h1:faQJFPbLSxzD9xpA02ttW/tS9vZykNvXwGvqIpk20FA= github.com/whyrusleeping/mdns v0.0.0-20190826153040-b9b60ed33aa9/go.mod h1:j4l84WPFclQPj320J9gp0XwNKBb3U0zt5CBqjPp22G4= -github.com/whyrusleeping/multiaddr-filter v0.0.0-20160516205228-e903e4adabd7 h1:E9S12nwJwEOXe2d6gT6qxdvqMnNq+VnSsKPgm2ZZNds= github.com/whyrusleeping/multiaddr-filter v0.0.0-20160516205228-e903e4adabd7/go.mod h1:X2c0RVCI1eSUFI8eLcY3c0423ykwiUdxLJtkDvruhjI= github.com/x-cray/logrus-prefixed-formatter v0.5.2/go.mod h1:2duySbKsL6M18s5GU7VPsoEPHyzalCE06qoARUCeBBE= github.com/x448/float16 v0.8.4 h1:qLwI1I70+NjRFUR3zs1JPUCgaCXSh3SW62uAKT1mSBM= @@ -1006,7 +963,6 @@ go.uber.org/atomic v1.5.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.6.0/go.mod h1:sABNBOSYdrvTF6hTgEIbc7YasKWGhgEQZyfxyTvoXHQ= go.uber.org/atomic v1.7.0 h1:ADUqmZGgLDDfbSL9ZmPxKTybcoEYHgpYfELNoN+7hsw= go.uber.org/atomic v1.7.0/go.mod h1:fEN4uk6kAWBTFdckzkM89CLk9XfWZrxpCo0nPH17wJc= -go.uber.org/goleak v1.0.0 h1:qsup4IcBdlmsnGfqyLl4Ntn3C2XCCuKAE7DwHpScyUo= go.uber.org/goleak v1.0.0/go.mod h1:8a7PlsEVH3e/a/GLqe5IIrQx6GzcnRmZEufDUTk4A7A= go.uber.org/multierr v1.1.0/go.mod h1:wR5kodmAFQ0UK8QlbwjlSNy0Z68gJhDJUG5sjR94q/0= go.uber.org/multierr v1.3.0/go.mod h1:VgVr7evmIr6uPjLBxg28wmKNXyqE9akIJ5XnfpiKl+4= @@ -1126,7 +1082,6 @@ golang.org/x/sync v0.0.0-20190911185100-cd5d95a43a6e/go.mod h1:RxMgew5VJxzue5/jJ golang.org/x/sync v0.0.0-20200317015054-43a5402ce75a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201020160332-67f06af15bc9/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sync v0.0.0-20201207232520-09787c993a3a/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= -golang.org/x/sync v0.0.0-20210220032951-036812b2e83c h1:5KslGYwFpkhGh+Q16bwMP3cOontH8FOep7tGV86Y7SQ= golang.org/x/sync v0.0.0-20210220032951-036812b2e83c/go.mod h1:RxMgew5VJxzue5/jJTE5uejpjVlOe/izrB70Jof72aM= golang.org/x/sys v0.0.0-20180823144017-11551d06cbcc/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= golang.org/x/sys v0.0.0-20180830151530-49385e6e1522/go.mod h1:STP8DvDyc/dI5b8T5hshtkjS+E42TnysNCUPdjciGhY= @@ -1172,6 +1127,7 @@ golang.org/x/sys v0.0.0-20200615200032-f1bc736245b1/go.mod h1:h1NjWce9XRLGQEsW7w golang.org/x/sys v0.0.0-20200625212154-ddb9806d33ae/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20200930185726-fdedc70b468f/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20201119102817-f84b799fce68/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= +golang.org/x/sys v0.0.0-20210119212857-b64e53b001e4/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210124154548-22da62e12c0c/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210303074136-134d130e1a04/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= golang.org/x/sys v0.0.0-20210309074719-68d13333faf2/go.mod h1:h1NjWce9XRLGQEsW7wpKNCjG9DtNlClVuFLEZdDNbEs= @@ -1224,8 +1180,9 @@ golang.org/x/tools v0.0.0-20191216052735-49a3e744a425/go.mod h1:TB2adYChydJhpapK golang.org/x/tools v0.0.0-20200103221440-774c71fcf114/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20200130002326-2f3ba24bd6e7/go.mod h1:TB2adYChydJhpapKDTa4BR/hXlZSLoq2Wpct/0txZ28= golang.org/x/tools v0.0.0-20200619180055-7c47624df98f/go.mod h1:EkVYQZoAsY45+roYkvgYkIh4xh/qjgUK9TdY2XT94GE= -golang.org/x/tools v0.0.0-20210106214847-113979e3529a h1:CB3a9Nez8M13wwlr/E2YtwoU+qYHKfC+JrDa45RXXoQ= golang.org/x/tools v0.0.0-20210106214847-113979e3529a/go.mod h1:emZCQorbCU4vsT4fOWvOPXz4eW1wZW4PmDk9uLelYpA= +golang.org/x/tools v0.1.0 h1:po9/4sTYwZU9lPhi1tOrb4hCv3qrhiQ77LZfGa2OjwY= +golang.org/x/tools v0.1.0/go.mod h1:xkSsbof2nBLbhDlRMhhhyNLN/zl3eTqcnHD5viDpcZ0= golang.org/x/xerrors v0.0.0-20190717185122-a985d3407aa7/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191011141410-1b5146add898/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= golang.org/x/xerrors v0.0.0-20191204190536-9bdfabe68543/go.mod h1:I/5z698sn9Ka8TeJc9MKroUUfqBBauWjQqLJ2OPfmY0= @@ -1327,8 +1284,9 @@ honnef.co/go/tools v0.0.0-20190102054323-c2f93a96b099/go.mod h1:rf3lG4BRIbNafJWh honnef.co/go/tools v0.0.0-20190106161140-3f1c8253044a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190418001031-e561f6794a2a/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= honnef.co/go/tools v0.0.0-20190523083050-ea95bdfd59fc/go.mod h1:rf3lG4BRIbNafJWhAfAdb/ePZxsR/4RtNHQocxwk9r4= -honnef.co/go/tools v0.0.1-2019.2.3 h1:3JgtbtFHMiCmsznwGVTUWbgGov+pVqnlf1dEJTNAXeM= honnef.co/go/tools v0.0.1-2019.2.3/go.mod h1:a3bituU0lyd329TUQxRnasdCoJDkEUEAqEt0JzvZhAg= +honnef.co/go/tools v0.1.3 h1:qTakTkI6ni6LFD5sBwwsdSO+AQqbSIxOauHTTQKZ/7o= +honnef.co/go/tools v0.1.3/go.mod h1:NgwopIslSNH47DimFoV78dnkksY2EFtX0ajyb3K/las= rsc.io/binaryregexp v0.2.0/go.mod h1:qTv7/COck+e2FymRvadv62gMdZztPaShugOCi3I+8D8= rsc.io/quote/v3 v3.1.0/go.mod h1:yEA65RcK8LyAZtP9Kv3t0HmxON59tX3rD+tICJqUlj0= rsc.io/sampler v1.3.0/go.mod h1:T1hPZKmBbMNahiBKFy5HrXp6adAjACjK9JXDnKaTXpA= diff --git a/query/graphql/planner/dagscan.go b/query/graphql/planner/dagscan.go index 016b376469..471d35e508 100644 --- a/query/graphql/planner/dagscan.go +++ b/query/graphql/planner/dagscan.go @@ -9,6 +9,27 @@ // licenses/APL.txt. package planner +// -> D1 -> E1 -> F1 +// A -> B -> C | +// -> D2 -> E2 -> F2 + +/* + +/db/blocks/QmKJHSDLFKJHSLDFKJHSFLDFDJKSDF => IPLD_BLOCK_BYTE_ARRAY +/db/blocks/QmJSDHGFKJSHGDKKSDGHJKFGHKSD => IPLD_BLOCK_BYTE_ARRAY +/db/blocks/QmHLSHDFLHJSDFLHJFSLDKSH => IPLD_BLOCK_BYTE_ARRAY => []byte("hello") +/db/blocks/QmSFHLSDHLHJSDLFHJLSD => IPLD_BLOCK_BYTE_ARRA => []byte("zgoodbye") +/db/blocks/QmSKFJHLSDHJFLSFHD => IPLD_BLOCK_BYTE_ARRAY => []byte("stupid") + +/db/data/1/0/bae-ALICE/1:v => "stupid" +/db/data/1/0/bae-ALICE/C:v => []byte... + +/db/heads/bae-ALICE/C/QmJSDHGFKJSHGDKKSDGHJKFGHKSD => [priority=1] +/db/heads/bae-ALICE/C/QmKJHSDLFKJHSLDFKJHSFLDFDJKSDF => [priority=1] +/db/heads/bae-ALICE/1/QmSKFJHLSDHJFLSFHD => [priority=2] + +*/ + import ( "container/list" "errors" diff --git a/query/graphql/planner/datasource.go b/query/graphql/planner/datasource.go index ee55c37b18..24bb477cdb 100644 --- a/query/graphql/planner/datasource.go +++ b/query/graphql/planner/datasource.go @@ -12,6 +12,7 @@ package planner import ( "encoding/json" "errors" + "fmt" "github.com/sourcenetwork/defradb/db/base" ) @@ -63,7 +64,7 @@ func (p *Planner) getCollectionDesc(name string) (base.CollectionDescription, er var desc base.CollectionDescription buf, err := p.txn.Systemstore().Get(p.ctx, key.ToDS()) if err != nil { - return desc, err + return desc, fmt.Errorf("Failed to get collection description: %w", err) } err = json.Unmarshal(buf, &desc) diff --git a/query/graphql/planner/executor.go b/query/graphql/planner/executor.go index 1090fd4339..f6489d47f7 100644 --- a/query/graphql/planner/executor.go +++ b/query/graphql/planner/executor.go @@ -67,7 +67,7 @@ func (e *QueryExecutor) MakeSelectQuery(ctx context.Context, db client.DB, txn c } func (e *QueryExecutor) ExecQuery(ctx context.Context, db client.DB, txn client.Txn, query string, args ...interface{}) ([]map[string]interface{}, error) { - q, err := e.parseQueryString(query) + q, err := e.ParseQueryString(query) if err != nil { return nil, err } @@ -76,7 +76,12 @@ func (e *QueryExecutor) ExecQuery(ctx context.Context, db client.DB, txn client. return planner.queryDocs(q) } -func (e *QueryExecutor) parseQueryString(query string) (*parser.Query, error) { +func (e *QueryExecutor) MakePlanFromParser(ctx context.Context, db client.DB, txn client.Txn, query *parser.Query) (planNode, error) { + planner := makePlanner(ctx, db, txn) + return planner.makePlan(query) +} + +func (e *QueryExecutor) ParseQueryString(query string) (*parser.Query, error) { source := source.NewSource(&source.Source{ Body: []byte(query), Name: "GraphQL request", diff --git a/query/graphql/planner/planner.go b/query/graphql/planner/planner.go index 30a7f075e0..3d235902c3 100644 --- a/query/graphql/planner/planner.go +++ b/query/graphql/planner/planner.go @@ -141,6 +141,7 @@ func (p *Planner) newObjectMutationPlan(stmt *parser.Mutation) (planNode, error) default: return nil, fmt.Errorf("unknown mutation action %T", stmt.Type) } + } func (p *Planner) makePlan(stmt parser.Statement) (planNode, error) { @@ -386,7 +387,7 @@ func (p *Planner) walkAndFindPlanType(plan, target planNode) planNode { } func (p *Planner) queryDocs(query *parser.Query) ([]map[string]interface{}, error) { - plan, err := p.query(query) + plan, err := p.makePlan(query) if err != nil { return nil, err } @@ -430,7 +431,7 @@ func (p *Planner) queryDocs(query *parser.Query) ([]map[string]interface{}, erro return docs, err } -func (p *Planner) query(query *parser.Query) (planNode, error) { +func (p *Planner) MakePlan(query *parser.Query) (planNode, error) { return p.makePlan(query) } diff --git a/query/graphql/planner/scan.go b/query/graphql/planner/scan.go index 3e29094f01..62636ef6d6 100644 --- a/query/graphql/planner/scan.go +++ b/query/graphql/planner/scan.go @@ -10,8 +10,6 @@ package planner import ( - "fmt" - "github.com/sourcenetwork/defradb/core" "github.com/sourcenetwork/defradb/db/base" "github.com/sourcenetwork/defradb/db/fetcher" @@ -69,7 +67,7 @@ func (n *scanNode) initScan() error { n.spans = append(n.spans, core.NewSpan(start, start.PrefixEnd())) } - fmt.Println("Initializing scan with the following spans:", n.spans) + // fmt.Println("Initializing scan with the following spans:", n.spans) err := n.fetcher.Start(n.p.ctx, n.p.txn, n.spans) if err != nil { return err diff --git a/query/graphql/schema/descriptions.go b/query/graphql/schema/descriptions.go index 01a7d976c9..4c60a8ec5f 100644 --- a/query/graphql/schema/descriptions.go +++ b/query/graphql/schema/descriptions.go @@ -13,6 +13,7 @@ import ( "errors" "fmt" "sort" + "strings" "github.com/sourcenetwork/defradb/core" "github.com/sourcenetwork/defradb/db/base" @@ -128,6 +129,21 @@ func (g *Generator) CreateDescriptions(types []*gql.Object) ([]base.CollectionDe continue } + // check if we already have a defined field + // with the same name. + // NOTE: This will happen for the virtual ID + // field associated with a related type, as + // its defined down below in the IsObject block. + if _, exists := desc.GetField(fname); exists { + // lets make sure its an _id field, otherwise + // we might have an error here + if strings.HasSuffix(fname, "_id") { + continue + } else { + return nil, fmt.Errorf("Error: found a duplicate field '%s' for type %s", fname, t.Name()) + } + } + fd := base.FieldDescription{ Name: fname, Kind: gqlTypeToFieldKind(field.Type), @@ -156,6 +172,31 @@ func (g *Generator) CreateDescriptions(types []*gql.Object) ([]base.CollectionDe } fd.Meta = rel.Kind() | fieldRelationType + + // handle object id field, defined as {{object_name}}_id + // with type gql.ID + // If it exists we need to delete and redefine + // if it doesn't exist we simply define, and make sure we + // skip later + + if !fd.IsObjectArray() { + for i, sf := range desc.Schema.Fields { + if sf.Name == fmt.Sprintf("%s_id", fname) { + // delete element matching + desc.Schema.Fields = append(desc.Schema.Fields[:i], desc.Schema.Fields[i+1:]...) + break + } + } + + // create field + fdRelated := base.FieldDescription{ + Name: fmt.Sprintf("%s_id", fname), + Kind: gqlTypeToFieldKind(gql.ID), + Meta: base.Meta_Relation_INTERNAL_ID, + } + fdRelated.Typ = defaultCRDTForFieldKind[fdRelated.Kind] + desc.Schema.Fields = append(desc.Schema.Fields, fdRelated) + } } desc.Schema.Fields = append(desc.Schema.Fields, fd) diff --git a/query/graphql/schema/descriptions_test.go b/query/graphql/schema/descriptions_test.go index b897ab5b72..44aa36f70d 100644 --- a/query/graphql/schema/descriptions_test.go +++ b/query/graphql/schema/descriptions_test.go @@ -184,6 +184,7 @@ func TestSingleSimpleType(t *testing.T) { Name: "author_id", Kind: base.FieldKind_DocKey, Typ: core.LWW_REGISTER, + Meta: base.Meta_Relation_INTERNAL_ID, }, { Name: "name", @@ -231,6 +232,7 @@ func TestSingleSimpleType(t *testing.T) { Name: "published_id", Kind: base.FieldKind_DocKey, Typ: core.LWW_REGISTER, + Meta: base.Meta_Relation_INTERNAL_ID, }, }, }, @@ -276,6 +278,7 @@ func TestSingleSimpleType(t *testing.T) { Name: "author_id", Kind: base.FieldKind_DocKey, Typ: core.LWW_REGISTER, + Meta: base.Meta_Relation_INTERNAL_ID, }, { Name: "name", diff --git a/query/graphql/schema/generate.go b/query/graphql/schema/generate.go index 1c6d26eb34..738d1faa9e 100644 --- a/query/graphql/schema/generate.go +++ b/query/graphql/schema/generate.go @@ -13,7 +13,6 @@ import ( "errors" "fmt" "log" - "strings" "github.com/sourcenetwork/defradb/db/base" "github.com/sourcenetwork/defradb/query/graphql/parser" @@ -868,7 +867,7 @@ type queryInputTypeConfig struct { } func (g *Generator) genTypeQueryableFieldList(obj *gql.Object, config queryInputTypeConfig) *gql.Field { - name := strings.ToLower(obj.Name()) + name := obj.Name() // add the generated types to the type map if err := g.manager.schema.AppendType(config.filter); err != nil {