From b751aa32c4ed121e5aa493ea469bb5c0a09c10c5 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Mon, 4 May 2026 21:48:59 -0400 Subject: [PATCH 01/17] feat(mutations): add per-predicate mutation pipeline MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Adds an alternative path through `applyMutations` that fans the edges of a single mutation out by predicate and processes each predicate's batch in its own goroutine, exposing parallelism the legacy serial-by-edge path cannot. The per-predicate runners drive the full mutation lifecycle (scalar/list, reverse, count, tokenized index, vector index) through new helpers in posting/index.go: `MutationPipeline.Process` → `ProcessPredicate` → `ProcessSingle` / `ProcessList` / `ProcessVectorIndex`, with `InsertTokenizerIndexes`, `ProcessReverse`, and `ProcessCount` shared across the predicate-shaped paths. Supporting infrastructure: - `posting.Deltas` — per-txn delta store split into a sharded raw-bytes map and a per-predicate `indexMap` for batched index writes. - `types.LockedShardedMap` — generic sharded RWMutex map used by Deltas and the in-memory index aggregation. - New mutation-time helpers on `posting.Txn` (`AddDelta`, `GetScalarList`, `addConflictKey`, `addConflictKeyWithUid`). - Test scaffolding in `worker/mutation_unit_test.go`, `worker/sort_test.go`, and `worker/draft_test.go`. This commit squashes the original WIP series authored by Harshil Goel (commits 978a0d41d…41d6445ce on the abandoned branch) plus the merge into current `main` and the mechanical clean-ups required to compile against it: import paths `hypermodeinc/dgraph` → `dgraph-io/dgraph`, license headers, a `posting/mvcc_test.go::TestRegression9597` fix-up for the `LocalCache.deltas` map → `*Deltas` refactor, and a stray doc reference in TESTING.md. The pipeline ships gated and disabled; the on/off knob and subsequent correctness fixes follow in later commits in this series. Co-Authored-By: Matthew McNeely Co-Authored-By: Claude Opus 4.7 (1M context) --- TESTING.md | 2 +- graphql/admin/admin.go | 4 + posting/index.go | 1098 +++++++++++++++++++++++++++++++++- posting/index_test.go | 27 +- posting/list.go | 107 +++- posting/list_test.go | 1 - posting/lists.go | 198 +++++- posting/mvcc.go | 56 +- posting/mvcc_test.go | 10 +- posting/oracle.go | 107 +++- query/upgrade_test.go | 2 +- schema/schema.go | 53 +- types/locked_sharded_map.go | 187 ++++++ worker/draft.go | 26 +- worker/draft_test.go | 101 ++++ worker/mutation.go | 10 + worker/mutation_unit_test.go | 12 +- worker/sort_test.go | 355 ++++++++++- worker/task.go | 4 + 19 files changed, 2214 insertions(+), 146 deletions(-) create mode 100644 types/locked_sharded_map.go diff --git a/TESTING.md b/TESTING.md index e0118f47b9b..384f84f9e44 100644 --- a/TESTING.md +++ b/TESTING.md @@ -87,7 +87,7 @@ programmatic control over local Dgraph clusters. Most newer integration2 and upg ## Module Structure -The main module is `github.com/hypermodeinc/dgraph` +The main module is `github.com/dgraph-io/dgraph` The codebase is organized into several key packages: diff --git a/graphql/admin/admin.go b/graphql/admin/admin.go index da5c14546c1..5544aff50ac 100644 --- a/graphql/admin/admin.go +++ b/graphql/admin/admin.go @@ -655,6 +655,10 @@ func newAdminResolver( return } + if len(pl.Postings) == 2 && string(pl.Postings[0].Value) == "_STAR_ALL" { + pl.Postings = pl.Postings[1:] + } + // There should be only one posting. if len(pl.Postings) != 1 { glog.Errorf("Only one posting is expected in the graphql schema posting list but got %d", diff --git a/posting/index.go b/posting/index.go index ae6c3352a44..8731aca1b69 100644 --- a/posting/index.go +++ b/posting/index.go @@ -15,14 +15,17 @@ import ( "math" "os" "strings" + "sync" "sync/atomic" "time" "unsafe" + "github.com/dgryski/go-farm" "github.com/golang/glog" "github.com/pkg/errors" ostats "go.opencensus.io/stats" "go.opentelemetry.io/otel/trace" + "golang.org/x/sync/errgroup" "google.golang.org/protobuf/proto" "github.com/dgraph-io/badger/v4" @@ -63,23 +66,1097 @@ func indexTokens(ctx context.Context, info *indexMutationInfo) ([]string, error) } sv, err := types.Convert(info.val, schemaType) if err != nil { - return nil, err + return nil, errors.Wrap(err, "Cannot convert value to scalar type") } var tokens []string for _, it := range info.tokenizers { toks, err := tok.BuildTokens(sv.Value, tok.GetTokenizerForLang(it, lang)) if err != nil { - return tokens, err + return tokens, errors.Wrapf(err, "Cannot build tokens for attribute %s", attr) } tokens = append(tokens, toks...) } return tokens, nil } -// addIndexMutations adds mutation(s) for a single term, to maintain the index, -// but only for the given tokenizers. -// TODO - See if we need to pass op as argument as t should already have Op. +type MutationPipeline struct { + txn *Txn +} + +func NewMutationPipeline(txn *Txn) *MutationPipeline { + return &MutationPipeline{txn: txn} +} + +type PredicatePipeline struct { + attr string + edges chan *pb.DirectedEdge + wg *sync.WaitGroup + errCh chan error +} + +func (pp *PredicatePipeline) close() { + pp.wg.Done() +} + +func (mp *MutationPipeline) ProcessVectorIndex(ctx context.Context, pipeline *PredicatePipeline, info predicateInfo) error { + var wg errgroup.Group + numThreads := 10 + + for i := 0; i < numThreads; i++ { + wg.Go(func() error { + for edge := range pipeline.edges { + uid := edge.Entity + + key := x.DataKey(pipeline.attr, uid) + pl, err := mp.txn.Get(key) + if err != nil { + return err + } + if err := pl.AddMutationWithIndex(ctx, edge, mp.txn); err != nil { + return err + } + } + return nil + }) + } + + if err := wg.Wait(); err != nil { + return err + } + + return nil +} + +func (mp *MutationPipeline) InsertTokenizerIndexes(ctx context.Context, pipeline *PredicatePipeline, postings *map[uint64]*pb.PostingList, info predicateInfo) error { + startTime := time.Now() + defer func() { + fmt.Println("Inserting tokenizer indexes for predicate", pipeline.attr, "took", time.Since(startTime)) + }() + + tokenizers := schema.State().Tokenizer(ctx, pipeline.attr) + if len(tokenizers) == 0 { + return nil + } + + indexesGenInMutation := types.NewLockedShardedMap[string, *MutableLayer]() + wg := &sync.WaitGroup{} + + syncMap := sync.Map{} + + chanFn := func(uids chan uint64, estimatedSize int) { + defer wg.Done() + indexGenInThread := make(map[string]*pb.PostingList, estimatedSize) + tokenizers := schema.State().Tokenizer(ctx, pipeline.attr) + + factorySpecs, err := schema.State().FactoryCreateSpec(ctx, pipeline.attr) + if err != nil { + pipeline.errCh <- err + return + } + + indexEdge := &pb.DirectedEdge{ + Attr: pipeline.attr, + } + + for uid := range uids { + postingList := (*postings)[uid] + newList := &pb.PostingList{} + if info.isSingleEdge && len(postingList.Postings) == 2 { + newList.Postings = append(newList.Postings, postingList.Postings[1]) + newList.Postings = append(newList.Postings, postingList.Postings[0]) + } else { + newList = postingList + } + for _, posting := range newList.Postings { + info := &indexMutationInfo{ + tokenizers: tokenizers, + factorySpecs: factorySpecs, + op: pb.DirectedEdge_SET, + val: types.Val{ + Tid: types.TypeID(posting.ValType), + Value: posting.Value, + }, + } + + info.edge = &pb.DirectedEdge{ + Attr: pipeline.attr, + Op: pb.DirectedEdge_SET, + Lang: string(posting.LangTag), + Value: posting.Value, + } + + key := fmt.Sprintf("%s,%s", posting.LangTag, posting.Value) + tokens, loaded := syncMap.Load(key) + + if !loaded { + tokens, err = indexTokens(ctx, info) + if err != nil { + fmt.Println("ERRORRRING", err) + x.Panic(err) + } + syncMap.Store(key, tokens) + } + + indexEdge.Op = GetPostingOp(posting.Op) + indexEdge.ValueId = uid + mpost := makePostingFromEdge(mp.txn.StartTs, indexEdge) + + for _, token := range tokens.([]string) { + key := x.IndexKey(pipeline.attr, token) + pk, _ := x.Parse([]byte(key)) + fmt.Println("TOKENS", key, pk) + val, ok := indexGenInThread[string(key)] + if !ok { + val = &pb.PostingList{} + } + val.Postings = append(val.Postings, mpost) + indexGenInThread[string(key)] = val + } + } + } + + for key, value := range indexGenInThread { + pk, _ := x.Parse([]byte(key)) + fmt.Println("LOCAL MAP", pk, value) + indexesGenInMutation.Update(key, func(val *MutableLayer, ok bool) *MutableLayer { + if !ok { + val = newMutableLayer() + val.currentEntries = &pb.PostingList{} + } + for _, posting := range value.Postings { + val.insertPosting(posting, false) + } + return val + }) + } + } + + numGo := 10 + wg.Add(numGo) + chMap := make(map[int]chan uint64) + + for i := 0; i < numGo; i++ { + uidCh := make(chan uint64, numGo) + chMap[i] = uidCh + go chanFn(uidCh, len(*postings)/numGo) + } + + for uid := range *postings { + chMap[int(uid)%numGo] <- uid + } + + for i := 0; i < numGo; i++ { + close(chMap[i]) + } + + wg.Wait() + + mp.txn.cache.Lock() + defer mp.txn.cache.Unlock() + + indexGenInTxn := mp.txn.cache.deltas.GetIndexMapForPredicate(pipeline.attr) + if indexGenInTxn == nil { + indexGenInTxn = types.NewLockedShardedMap[string, *pb.PostingList]() + mp.txn.cache.deltas.indexMap[pipeline.attr] = indexGenInTxn + } + + fmt.Println("INSERTING INDEX", pipeline.attr, *postings) + updateFn := func(key string, value *MutableLayer) { + pk, _ := x.Parse([]byte(key)) + fmt.Println("UPDATE INDEX", pk, value) + indexGenInTxn.Update(key, func(val *pb.PostingList, ok bool) *pb.PostingList { + if !ok { + val = &pb.PostingList{} + } + val.Postings = append(val.Postings, value.currentEntries.Postings...) + return val + }) + } + + if info.hasUpsert { + err := indexesGenInMutation.Iterate(func(key string, value *MutableLayer) error { + updateFn(key, value) + mp.txn.addConflictKey(farm.Fingerprint64([]byte(key))) + return nil + }) + if err != nil { + return err + } + } else { + err := indexesGenInMutation.Iterate(func(key string, value *MutableLayer) error { + updateFn(key, value) + mp.txn.addConflictKeyWithUid([]byte(key), value.currentEntries, info.hasUpsert, info.noConflict) + return nil + }) + if err != nil { + return err + } + } + + return nil +} + +// func (mp *MutationPipeline) InsertTokenizerIndexes(ctx context.Context, pipeline *PredicatePipeline, postings *map[uint64]*pb.PostingList, info predicateInfo) error { +// startTime := time.Now() +// defer func() { +// fmt.Println("Inserting tokenizer indexes for predicate", pipeline.attr, "took", time.Since(startTime)) +// }() + +// tokenizers := schema.State().Tokenizer(ctx, pipeline.attr) +// if len(tokenizers) == 0 { +// return nil +// } + +// values := make(map[string]*pb.PostingList, len(tokenizers)*len(*postings)) +// valPost := make(map[string]*pb.Posting) + +// indexEdge1 := &pb.DirectedEdge{ +// Attr: pipeline.attr, +// } + +// for uid, postingList := range *postings { +// fmt.Println("POSTING", uid, postingList) +// for _, posting := range postingList.Postings { +// key := fmt.Sprintf("%s,%s", posting.LangTag, posting.Value) +// valPl, ok := values[key] +// if !ok { +// valPl = &pb.PostingList{} +// } + +// indexEdge1.Op = GetPostingOp(posting.Op) +// indexEdge1.ValueId = uid + +// mpost := makePostingFromEdge(mp.txn.StartTs, indexEdge1) +// valPl.Postings = append(valPl.Postings, mpost) +// values[key] = valPl + +// newPosting := new(pb.Posting) +// newPosting.ValType = posting.ValType +// newPosting.Value = posting.Value +// newPosting.LangTag = posting.LangTag +// valPost[key] = newPosting +// } +// } + +// keysCreated := make([]string, 0, len(values)) +// for i := range values { +// keysCreated = append(keysCreated, i) +// } + +// //fmt.Println("START") + +// f := func(numGo int) *types.LockedShardedMap[string, *MutableLayer] { +// wg := &sync.WaitGroup{} + +// globalMap := types.NewLockedShardedMap[string, *MutableLayer]() +// process := func(start int) { +// factorySpecs, err := schema.State().FactoryCreateSpec(ctx, pipeline.attr) +// if err != nil { +// pipeline.errCh <- err +// return +// } + +// defer wg.Done() +// localMap := make(map[string]*pb.PostingList, len(values)/numGo) +// for i := start; i < len(values); i += numGo { +// key := keysCreated[i] +// valPl := values[key] +// if len(valPl.Postings) == 0 { +// continue +// } + +// posting := valPost[key] +// // Build info per iteration without indexEdge. +// info := &indexMutationInfo{ +// tokenizers: tokenizers, +// factorySpecs: factorySpecs, +// op: pb.DirectedEdge_SET, +// val: types.Val{ +// Tid: types.TypeID(posting.ValType), +// Value: posting.Value, +// }, +// } + +// info.edge = &pb.DirectedEdge{ +// Attr: pipeline.attr, +// Op: pb.DirectedEdge_SET, +// Lang: string(posting.LangTag), +// Value: posting.Value, +// } + +// tokens, erri := indexTokens(ctx, info) +// if erri != nil { +// fmt.Println("ERRORRRING", erri) +// x.Panic(erri) +// } + +// for _, token := range tokens { +// key := x.IndexKey(pipeline.attr, token) +// pk, _ := x.Parse([]byte(key)) +// fmt.Println("TOKENS", key, i, numGo, pk) +// val, ok := localMap[string(key)] +// if !ok { +// val = &pb.PostingList{} +// } +// val.Postings = append(val.Postings, valPl.Postings...) +// localMap[string(key)] = val +// } +// } + +// for key, value := range localMap { +// pk, _ := x.Parse([]byte(key)) +// fmt.Println("LOCAL MAP", pk, numGo, value) +// globalMap.Update(key, func(val *MutableLayer, ok bool) *MutableLayer { +// if !ok { +// val = newMutableLayer() +// val.currentEntries = &pb.PostingList{} +// } +// for _, posting := range value.Postings { +// val.insertPosting(posting, false) +// } +// return val +// }) +// } +// } + +// for i := range numGo { +// wg.Add(1) +// go process(i) +// } + +// wg.Wait() + +// return globalMap +// } + +// globalMapI := f(1) + +// mp.txn.cache.Lock() +// defer mp.txn.cache.Unlock() + +// globalMap := mp.txn.cache.deltas.GetIndexMapForPredicate(pipeline.attr) +// if globalMap == nil { +// globalMap = types.NewLockedShardedMap[string, *pb.PostingList]() +// mp.txn.cache.deltas.indexMap[pipeline.attr] = globalMap +// } + +// updateFn := func(key string, value *MutableLayer) { +// globalMap.Update(key, func(val *pb.PostingList, ok bool) *pb.PostingList { +// if !ok { +// val = &pb.PostingList{} +// } +// val.Postings = append(val.Postings, value.currentEntries.Postings...) +// return val +// }) +// } + +// if info.hasUpsert { +// err := globalMapI.Iterate(func(key string, value *MutableLayer) error { +// updateFn(key, value) +// mp.txn.addConflictKey(farm.Fingerprint64([]byte(key))) +// return nil +// }) +// if err != nil { +// return err +// } +// } else { +// err := globalMapI.Iterate(func(key string, value *MutableLayer) error { +// updateFn(key, value) +// mp.txn.addConflictKeyWithUid([]byte(key), value.currentEntries, info.hasUpsert, info.noConflict) +// return nil +// }) +// if err != nil { +// return err +// } +// } + +// return nil +// } + +type predicateInfo struct { + isList bool + index bool + reverse bool + count bool + noConflict bool + hasUpsert bool + isUid bool + + isSingleEdge bool +} + +func (mp *MutationPipeline) ProcessList(ctx context.Context, pipeline *PredicatePipeline, info predicateInfo) error { + su, schemaExists := schema.State().Get(ctx, pipeline.attr) + + mutations := make(map[uint64]*MutableLayer, 1000) + + for edge := range pipeline.edges { + if edge.Op != pb.DirectedEdge_DEL && !schemaExists { + return errors.Errorf("runMutation: Unable to find schema for %s", edge.Attr) + } + + if err := ValidateAndConvert(edge, &su); err != nil { + return err + } + + uid := edge.Entity + pl, exists := mutations[uid] + if !exists { + pl = newMutableLayer() + pl.currentEntries = &pb.PostingList{} + } + + mpost := NewPosting(edge) + mpost.StartTs = mp.txn.StartTs + if mpost.PostingType != pb.Posting_REF { + edge.ValueId = FingerprintEdge(edge) + mpost.Uid = edge.ValueId + } + + pl.insertPosting(mpost, false) + mutations[uid] = pl + } + + postings := make(map[uint64]*pb.PostingList, 1000) + for uid, pl := range mutations { + postings[uid] = pl.currentEntries + } + + if info.reverse { + if err := mp.ProcessReverse(ctx, pipeline, &postings, info); err != nil { + return err + } + } + + if info.index { + if err := mp.InsertTokenizerIndexes(ctx, pipeline, &postings, info); err != nil { + return err + } + } + + if info.count { + return mp.ProcessCount(ctx, pipeline, &postings, info, true, false) + } + + dataKey := x.DataKey(pipeline.attr, 0) + baseKey := string(dataKey[:len(dataKey)-8]) // Avoid repeated conversion + + for uid, pl := range postings { + if len(pl.Postings) == 0 { + continue + } + + binary.BigEndian.PutUint64(dataKey[len(dataKey)-8:], uid) + if newPl, err := mp.txn.AddDelta(baseKey+string(dataKey[len(dataKey)-8:]), pl, info.isUid, true); err != nil { + return err + } else { + if !info.noConflict { + mp.txn.addConflictKeyWithUid(dataKey, newPl, info.hasUpsert, info.noConflict) + } + } + } + + return nil +} + +func findSingleValueInPostingList(pb *pb.PostingList) *pb.Posting { + if pb == nil { + return nil + } + for _, p := range pb.Postings { + if p.Op == Set { + return p + } + } + return nil +} + +func (mp *MutationPipeline) ProcessReverse(ctx context.Context, pipeline *PredicatePipeline, postings *map[uint64]*pb.PostingList, info predicateInfo) error { + key := x.ReverseKey(pipeline.attr, 0) + edge := &pb.DirectedEdge{ + Attr: pipeline.attr, + } + reverseredMap := make(map[uint64]*pb.PostingList, 1000) + for uid, postingList := range *postings { + for _, posting := range postingList.Postings { + postingList, ok := reverseredMap[posting.Uid] + if !ok { + postingList = &pb.PostingList{} + } + edge.Entity = posting.Uid + edge.ValueId = uid + edge.ValueType = posting.ValType + edge.Op = GetPostingOp(posting.Op) + edge.Facets = posting.Facets + + postingList.Postings = append(postingList.Postings, makePostingFromEdge(mp.txn.StartTs, edge)) + reverseredMap[posting.Uid] = postingList + } + } + + if info.count { + newInfo := predicateInfo{ + isList: true, + index: info.index, + reverse: info.reverse, + count: info.count, + noConflict: info.noConflict, + hasUpsert: info.hasUpsert, + } + return mp.ProcessCount(ctx, pipeline, &reverseredMap, newInfo, true, true) + } + + for uid, pl := range reverseredMap { + if len(pl.Postings) == 0 { + continue + } + binary.BigEndian.PutUint64(key[len(key)-8:], uid) + if newPl, err := mp.txn.AddDelta(string(key), pl, true, true); err != nil { + return err + } else { + mp.txn.addConflictKeyWithUid(key, newPl, info.hasUpsert, info.noConflict) + } + } + + return nil +} + +func makePostingFromEdge(startTs uint64, edge *pb.DirectedEdge) *pb.Posting { + mpost := NewPosting(edge) + mpost.StartTs = startTs + if mpost.PostingType != pb.Posting_REF { + edge.ValueId = FingerprintEdge(edge) + mpost.Uid = edge.ValueId + } + return mpost +} + +func (mp *MutationPipeline) handleOldDeleteForSingle(pipeline *PredicatePipeline, postings map[uint64]*pb.PostingList) error { + edge := &pb.DirectedEdge{ + Attr: pipeline.attr, + } + + dataKey := x.DataKey(pipeline.attr, 0) + + for uid, postingList := range postings { + currValue := findSingleValueInPostingList(postingList) + if currValue == nil { + continue + } + + binary.BigEndian.PutUint64(dataKey[len(dataKey)-8:], uid) + list, err := mp.txn.GetScalarList(dataKey) + if err != nil { + return err + } + + oldValList, err := list.StaticValue(mp.txn.StartTs) + if err != nil { + return err + } + + oldVal := findSingleValueInPostingList(oldValList) + + if oldVal == nil { + continue + } + + if string(oldVal.Value) == string(currValue.Value) { + postings[uid] = &pb.PostingList{} + continue + } + + edge.Op = pb.DirectedEdge_DEL + edge.Value = oldVal.Value + edge.ValueType = oldVal.ValType + edge.ValueId = oldVal.Uid + + mpost := makePostingFromEdge(mp.txn.StartTs, edge) + postingList.Postings = append(postingList.Postings, mpost) + postings[uid] = postingList + } + + return nil +} + +func (txn *Txn) addConflictKeyWithUid(key []byte, pl *pb.PostingList, hasUpsert bool, hasNoConflict bool) { + if hasNoConflict { + return + } + txn.Lock() + defer txn.Unlock() + if txn.conflicts == nil { + txn.conflicts = make(map[uint64]struct{}) + } + keyHash := farm.Fingerprint64(key) + if hasUpsert { + txn.conflicts[keyHash] = struct{}{} + return + } + for _, post := range pl.Postings { + txn.conflicts[keyHash^post.Uid] = struct{}{} + } +} + +func (mp *MutationPipeline) ProcessCount(ctx context.Context, pipeline *PredicatePipeline, postings *map[uint64]*pb.PostingList, info predicateInfo, isListEdge bool, isReverseEdge bool) error { + dataKey := x.DataKey(pipeline.attr, 0) + if isReverseEdge { + dataKey = x.ReverseKey(pipeline.attr, 0) + } + edge := pb.DirectedEdge{ + Attr: pipeline.attr, + } + + countMap := make(map[int]*pb.PostingList, 2*len(*postings)) + + insertEdgeCount := func(count int) { + c, ok := countMap[count] + if !ok { + c = &pb.PostingList{} + countMap[count] = c + } + c.Postings = append(c.Postings, makePostingFromEdge(mp.txn.StartTs, &edge)) + countMap[count] = c + } + + for uid, postingList := range *postings { + binary.BigEndian.PutUint64(dataKey[len(dataKey)-8:], uid) + list, err := mp.txn.Get(dataKey) + if err != nil { + return err + } + + list.Lock() + prevCount := list.GetLength(mp.txn.StartTs) + + for _, post := range postingList.Postings { + found, _, _ := list.findPosting(post.StartTs, post.Uid) + if found { + if post.Op == Set && isListEdge { + post.Op = Ovr + } + } else { + if post.Op == Del { + continue + } + } + + if err := list.updateMutationLayer(post, !isListEdge, true); err != nil { + return err + } + } + + newCount := list.GetLength(mp.txn.StartTs) + updated := list.mutationMap.currentEntries != nil + list.Unlock() + + if updated { + if !isListEdge { + if !info.noConflict { + mp.txn.addConflictKey(farm.Fingerprint64(dataKey)) + } + } else { + mp.txn.addConflictKeyWithUid(dataKey, postingList, info.hasUpsert, info.noConflict) + } + } + + if newCount == prevCount { + continue + } + + //fmt.Println("COUNT STATS", uid, prevCount, newCount, postingList, list.Print()) + + edge.ValueId = uid + edge.Op = pb.DirectedEdge_DEL + if prevCount > 0 { + insertEdgeCount(prevCount) + } + edge.Op = pb.DirectedEdge_SET + if newCount > 0 { + insertEdgeCount(newCount) + } + } + + for c, pl := range countMap { + //fmt.Println("COUNT", c, pl) + ck := x.CountKey(pipeline.attr, uint32(c), isReverseEdge) + if newPl, err := mp.txn.AddDelta(string(ck), pl, true, true); err != nil { + return err + } else { + mp.txn.addConflictKeyWithUid(ck, newPl, info.hasUpsert, info.noConflict) + } + } + + return nil +} + +func (mp *MutationPipeline) ProcessSingle(ctx context.Context, pipeline *PredicatePipeline, info predicateInfo) error { + su, schemaExists := schema.State().Get(ctx, pipeline.attr) + + postings := make(map[uint64]*pb.PostingList, 1000) + + dataKey := x.DataKey(pipeline.attr, 0) + insertDeleteAllEdge := !(info.index || info.reverse || info.count) // nolint + + var oldVal *pb.Posting + for edge := range pipeline.edges { + // fmt.Println("SINGLE EDGE", edge) + if edge.Op != pb.DirectedEdge_DEL && !schemaExists { + return errors.Errorf("runMutation: Unable to find schema for %s", edge.Attr) + } + + if err := ValidateAndConvert(edge, &su); err != nil { + return err + } + + uid := edge.Entity + pl, exists := postings[uid] + + setPosting := func() { + mpost := makePostingFromEdge(mp.txn.StartTs, edge) + if len(pl.Postings) == 0 { + if insertDeleteAllEdge { + pl = &pb.PostingList{ + Postings: []*pb.Posting{createDeleteAllPosting(), mpost}, + } + } else { + pl = &pb.PostingList{ + Postings: []*pb.Posting{mpost}, + } + } + } else { + if pl.Postings[len(pl.Postings)-1].Op == Set { + pl.Postings[len(pl.Postings)-1] = mpost + } else { + pl.Postings = append(pl.Postings, mpost) + } + } + postings[uid] = pl + } + + if exists { + if edge.Op == pb.DirectedEdge_DEL { + oldVal = findSingleValueInPostingList(pl) + if string(edge.Value) == string(oldVal.Value) { + setPosting() + } + } else { + setPosting() + } + continue + } + + pl = &pb.PostingList{} + postings[uid] = pl + + if edge.Op == pb.DirectedEdge_DEL { + binary.BigEndian.PutUint64(dataKey[len(dataKey)-8:], uid) + list, err := mp.txn.GetScalarList(dataKey) + if err != nil { + return err + } + if list != nil { + l, err := list.StaticValue(mp.txn.StartTs) + if err != nil { + return err + } + oldVal = findSingleValueInPostingList(l) + } + if oldVal != nil { + if string(oldVal.Value) == string(edge.Value) { + setPosting() + } + } + } else { + setPosting() + } + } + + if info.index || info.reverse || info.count { + if err := mp.handleOldDeleteForSingle(pipeline, postings); err != nil { + return err + } + } + + if info.index { + if err := mp.InsertTokenizerIndexes(ctx, pipeline, &postings, info); err != nil { + return err + } + } + + if info.reverse { + if err := mp.ProcessReverse(ctx, pipeline, &postings, info); err != nil { + return err + } + } + + if info.count { + // Count should take care of updating the posting list + return mp.ProcessCount(ctx, pipeline, &postings, info, false, false) + } + + baseKey := string(dataKey[:len(dataKey)-8]) // Avoid repeated conversion + + for uid, pl := range postings { + //fmt.Println("ADDING DELTA", uid, pipeline.attr, pl) + binary.BigEndian.PutUint64(dataKey[len(dataKey)-8:], uid) + key := baseKey + string(dataKey[len(dataKey)-8:]) + + if !info.noConflict { + mp.txn.addConflictKey(farm.Fingerprint64([]byte(key))) + } + + if _, err := mp.txn.AddDelta(key, pl, false, false); err != nil { + return err + } + } + + return nil +} + +func runMutation(ctx context.Context, edge *pb.DirectedEdge, txn *Txn) error { + ctx = schema.GetWriteContext(ctx) + + // We shouldn't check whether this Alpha serves this predicate or not. Membership information + // isn't consistent across the entire cluster. We should just apply whatever is given to us. + su, ok := schema.State().Get(ctx, edge.Attr) + if edge.Op != pb.DirectedEdge_DEL { + if !ok { + return errors.Errorf("runMutation: Unable to find schema for %s", edge.Attr) + } + } + + key := x.DataKey(edge.Attr, edge.Entity) + // The following is a performance optimization which allows us to not read a posting list from + // disk. We calculate this based on how AddMutationWithIndex works. The general idea is that if + // we're not using the read posting list, we don't need to retrieve it. We need the posting list + // if we're doing count index or delete operation. For scalar predicates, we just get the last item merged. + // In other cases, we can just create a posting list facade in memory and use it to store the delta in Badger. + // Later, the rollup operation would consolidate all these deltas into a posting list. + isList := su.GetList() + var getFn func(key []byte) (*List, error) + switch { + case len(edge.Lang) == 0 && !isList: + // Scalar Predicates, without lang + getFn = txn.GetScalarList + case len(edge.Lang) > 0 || su.GetCount(): + // Language or Count Index + getFn = txn.Get + case edge.Op == pb.DirectedEdge_DEL: + // Covers various delete cases to keep things simple. + getFn = txn.Get + default: + // Only count index needs to be read. For other indexes on list, we don't need to read any data. + // For indexes on scalar prediactes, only the last element needs to be left. + // Delete cases covered above. + getFn = txn.GetFromDelta + } + + plist, err := getFn(key) + if err != nil { + return err + } + return plist.AddMutationWithIndex(ctx, edge, txn) +} + +func (mp *MutationPipeline) ProcessPredicate(ctx context.Context, pipeline *PredicatePipeline) error { + defer pipeline.close() + ctx = schema.GetWriteContext(ctx) + + // We shouldn't check whether this Alpha serves this predicate or not. Membership information + // isn't consistent across the entire cluster. We should just apply whatever is given to us. + su, ok := schema.State().Get(ctx, pipeline.attr) + info := predicateInfo{} + runForVectorIndex := false + + if ok { + info.index = schema.State().IsIndexed(ctx, pipeline.attr) + info.count = schema.State().HasCount(ctx, pipeline.attr) + info.reverse = schema.State().IsReversed(ctx, pipeline.attr) + info.noConflict = schema.State().HasNoConflict(pipeline.attr) + info.hasUpsert = schema.State().HasUpsert(pipeline.attr) + info.isList = schema.State().IsList(pipeline.attr) + info.isUid = su.ValueType == pb.Posting_UID + factorySpecs, err := schema.State().FactoryCreateSpec(ctx, pipeline.attr) + if err != nil { + return err + } + if len(factorySpecs) > 0 { + runForVectorIndex = true + } + } + + if runForVectorIndex { + return mp.ProcessVectorIndex(ctx, pipeline, info) + } + + runListFn := false + + if ok { + if info.isList || su.Lang { + runListFn = true + } + } + + info.isSingleEdge = !runListFn + + if runListFn { + if err := mp.ProcessList(ctx, pipeline, info); err != nil { + return err + } + } + + if ok && !runListFn { + if err := mp.ProcessSingle(ctx, pipeline, info); err != nil { + return err + } + } + + for edge := range pipeline.edges { + if err := runMutation(ctx, edge, mp.txn); err != nil { + return err + } + } + + return nil +} + +func isStarAll(v []byte) bool { + return bytes.Equal(v, []byte(x.Star)) +} + +func ValidateAndConvert(edge *pb.DirectedEdge, su *pb.SchemaUpdate) error { + if types.TypeID(edge.ValueType) == types.DefaultID && isStarAll(edge.Value) { + return nil + } + + storageType := TypeID(edge) + schemaType := types.TypeID(su.ValueType) + + // type checks + switch { + case edge.Lang != "" && !su.GetLang(): + return errors.Errorf("Attr: [%v] should have @lang directive in schema to mutate edge: [%v]", + x.ParseAttr(edge.Attr), edge) + + case !schemaType.IsScalar() && !storageType.IsScalar(): + return nil + + case !schemaType.IsScalar() && storageType.IsScalar(): + return errors.Errorf("Input for predicate %q of type uid is scalar. Edge: %v", + x.ParseAttr(edge.Attr), edge) + + case schemaType.IsScalar() && !storageType.IsScalar(): + return errors.Errorf("Input for predicate %q of type scalar is uid. Edge: %v", + x.ParseAttr(edge.Attr), edge) + + case schemaType == types.TypeID(pb.Posting_VFLOAT): + if !(storageType == types.TypeID(pb.Posting_VFLOAT) || storageType == types.TypeID(pb.Posting_STRING) || //nolint + storageType == types.TypeID(pb.Posting_DEFAULT)) { + return errors.Errorf("Input for predicate %q of type vector is not vector."+ + " Did you forget to add quotes before []?. Edge: %v", x.ParseAttr(edge.Attr), edge) + } + + // The suggested storage type matches the schema, OK! (Nothing to do ...) + case storageType == schemaType && schemaType != types.DefaultID: + return nil + + // We accept the storage type iff we don't have a schema type and a storage type is specified. + case schemaType == types.DefaultID: + schemaType = storageType + } + + var ( + dst types.Val + err error + ) + + src := types.Val{Tid: types.TypeID(edge.ValueType), Value: edge.Value} + // check compatibility of schema type and storage type + // The goal is to convert value on edge to value type defined by schema. + if dst, err = types.Convert(src, schemaType); err != nil { + return err + } + + // convert to schema type + b := types.ValueForType(types.BinaryID) + if err = types.Marshal(dst, &b); err != nil { + return err + } + + if x.WorkerConfig.AclEnabled && x.ParseAttr(edge.GetAttr()) == "dgraph.rule.permission" { + perm, ok := dst.Value.(int64) + if !ok { + return errors.Errorf("Value for predicate should be of type int") + } + if perm < 0 || perm > 7 { + return errors.Errorf("Can't set to %d, Value for this"+ + " predicate should be between 0 and 7", perm) + } + } + + // TODO: Figure out why this is Enum. It really seems like an odd choice -- rather than + // specifying it as the same type as presented in su. + edge.ValueType = schemaType.Enum() + var ok bool + edge.Value, ok = b.Value.([]byte) + if !ok { + return errors.Errorf("failure to convert edge type: '%+v' to schema type: '%+v'", + storageType, schemaType) + } + + return nil +} + +func (mp *MutationPipeline) Process(ctx context.Context, edges []*pb.DirectedEdge) error { + predicates := map[string]*PredicatePipeline{} + var wg sync.WaitGroup + numWg := 0 + eg, egCtx := errgroup.WithContext(ctx) + for _, edge := range edges { + //fmt.Println("PROCESSING EDGE", edge) + if edge.Op == pb.DirectedEdge_DEL && string(edge.Value) == x.Star { + l, err := mp.txn.Get(x.DataKey(edge.Attr, edge.Entity)) + if err != nil { + return err + } + if err = l.handleDeleteAll(ctx, edge, mp.txn); err != nil { + return err + } + continue + } + pred, ok := predicates[edge.Attr] + if !ok { + pred = &PredicatePipeline{ + attr: edge.Attr, + edges: make(chan *pb.DirectedEdge, 1000), + wg: &wg, + } + predicates[edge.Attr] = pred + wg.Add(1) + numWg += 1 + // Capture pred by passing it as a parameter to the closure + eg.Go(func(p *PredicatePipeline) func() error { + return func() error { + return mp.ProcessPredicate(egCtx, p) + } + }(pred)) + } + pred.edges <- edge + } + for _, pred := range predicates { + close(pred.edges) + } + if numWg == 0 { + return nil + } + // Wait for all predicate processors; returns first error (and cancels others via context). + if err := eg.Wait(); err != nil { + return err + } + return nil +} func (txn *Txn) addIndexMutations(ctx context.Context, info *indexMutationInfo) ([]*pb.DirectedEdge, error) { if info.tokenizers == nil { @@ -951,7 +2028,7 @@ func (r *rebuilder) Run(ctx context.Context) error { // txn.cache.Lock() is not required because we are the only one making changes to txn. kvs := make([]*bpb.KV, 0) - for key, data := range streamTxn.cache.deltas { + if err := streamTxn.cache.deltas.IterateBytes(func(key string, data []byte) error { version := atomic.AddUint64(&counter, 1) kv := bpb.KV{ Key: []byte(key), @@ -960,7 +2037,11 @@ func (r *rebuilder) Run(ctx context.Context) error { Version: version, } kvs = append(kvs, &kv) + return nil + }); err != nil { + return nil, err } + txns[threadId] = NewTxn(r.startTs) return &bpb.KVList{Kv: kvs}, nil } @@ -1009,7 +2090,7 @@ func (r *rebuilder) Run(ctx context.Context) error { // Convert data into deltas. streamTxn.Update() // txn.cache.Lock() is not required because we are the only one making changes to txn. - for key, data := range streamTxn.cache.deltas { + if err := streamTxn.cache.deltas.IterateBytes(func(key string, data []byte) error { version := atomic.AddUint64(&counter, 1) kv := bpb.KV{ Key: []byte(key), @@ -1018,6 +2099,9 @@ func (r *rebuilder) Run(ctx context.Context) error { Version: version, } kvs = append(kvs, &kv) + return nil + }); err != nil { + return nil, err } txns[threadId] = NewTxn(r.startTs) diff --git a/posting/index_test.go b/posting/index_test.go index 3f75c26fb10..d8dbab7c769 100644 --- a/posting/index_test.go +++ b/posting/index_test.go @@ -139,19 +139,28 @@ func addMutation(t *testing.T, l *List, edge *pb.DirectedEdge, op uint32, } txn := Oracle().RegisterStartTs(startTs) txn.cache.SetIfAbsent(string(l.key), l) - if index { - require.NoError(t, l.AddMutationWithIndex(context.Background(), edge, txn)) - } else { - require.NoError(t, l.addMutation(context.Background(), txn, edge)) - } + mp := NewMutationPipeline(txn) + err := mp.Process(context.Background(), []*pb.DirectedEdge{edge}) + require.NoError(t, err) txn.Update() txn.UpdateCachedKeys(commitTs) writer := NewTxnWriter(pstore) require.NoError(t, txn.CommitToDisk(writer, commitTs)) require.NoError(t, writer.Flush()) + newTxn := Oracle().RegisterStartTs(commitTs + 1) + l1, err := newTxn.Get(l.key) + require.NoError(t, err) + *l = *l1 //nolint } +const schemaPreVal = ` + name: string . + name2: string . + dob: dateTime . + friend: [uid] . +` + const schemaVal = ` name: string @index(term) . name2: string @index(term) . @@ -263,6 +272,9 @@ func addEdgeToUID(t *testing.T, attr string, src uint64, func TestCountReverseIndexWithData(t *testing.T) { require.NoError(t, pstore.DropAll()) MemLayerInstance.clear() + preIndex := "testcount: [uid] ." + require.NoError(t, schema.ParseBytes([]byte(preIndex), 1)) + indexNameCountVal := "testcount: [uid] @count @reverse ." attr := x.AttrInRootNamespace("testcount") @@ -298,6 +310,9 @@ func TestCountReverseIndexWithData(t *testing.T) { func TestCountReverseIndexEmptyPosting(t *testing.T) { require.NoError(t, pstore.DropAll()) MemLayerInstance.clear() + preIndex := "testcount: [uid] ." + require.NoError(t, schema.ParseBytes([]byte(preIndex), 1)) + indexNameCountVal := "testcount: [uid] @count @reverse ." attr := x.AttrInRootNamespace("testcount") @@ -330,6 +345,8 @@ func TestCountReverseIndexEmptyPosting(t *testing.T) { } func TestRebuildTokIndex(t *testing.T) { + require.NoError(t, schema.ParseBytes([]byte(schemaPreVal), 1)) + addEdgeToValue(t, x.AttrInRootNamespace("name2"), 91, "Michonne", uint64(1), uint64(2)) addEdgeToValue(t, x.AttrInRootNamespace("name2"), 92, "David", uint64(3), uint64(4)) diff --git a/posting/list.go b/posting/list.go index 1c0c7a0fc55..cbb2542d0a6 100644 --- a/posting/list.go +++ b/posting/list.go @@ -335,6 +335,7 @@ func (mm *MutableLayer) insertCommittedPostings(pl *pb.PostingList) { // We insert old postings in reverse order. So we only need to read the first update to an UID. if _, ok := mm.committedUids[mpost.Uid]; !ok { mm.committedUids[mpost.Uid] = mpost + fmt.Println("Buidlding committed uids", mm.committedUids, mpost) } } } @@ -404,13 +405,18 @@ func (mm *MutableLayer) print() string { if mm == nil { return "" } - return fmt.Sprintf("Committed List: %+v Proposed list: %+v Delete all marker: %d \n", + return fmt.Sprintf("Committed List: %+v Proposed list: %+v Delete all marker: %d. Count: %d \n", mm.committedEntries, mm.currentEntries, - mm.deleteAllMarker) + mm.deleteAllMarker, + mm.length) } func (l *List) Print() string { + if l.plist.Pack != nil { + uids := codec.Decode(l.plist.Pack, 0) + return fmt.Sprintf("minTs: %d, committed uids: %+v, mutationMap: %s", l.minTs, uids, l.mutationMap.print()) + } return fmt.Sprintf("minTs: %d, plist: %+v, mutationMap: %s", l.minTs, l.plist, l.mutationMap.print()) } @@ -712,6 +718,53 @@ type ListOptions struct { First int } +func NewPostingExisting(p *pb.Posting, t *pb.DirectedEdge) { + var op uint32 + switch t.Op { + case pb.DirectedEdge_SET: + op = Set + case pb.DirectedEdge_OVR: + op = Ovr + case pb.DirectedEdge_DEL: + op = Del + default: + x.Fatalf("Unhandled operation: %+v", t) + } + + var postingType pb.Posting_PostingType + switch { + case len(t.Lang) > 0: + postingType = pb.Posting_VALUE_LANG + case t.ValueId == 0: + postingType = pb.Posting_VALUE + default: + postingType = pb.Posting_REF + } + + p.Uid = t.ValueId + p.Value = t.Value + p.ValType = t.ValueType + p.PostingType = postingType + p.LangTag = []byte(t.Lang) + p.Op = op + p.Facets = t.Facets +} + +func GetPostingOp(top uint32) pb.DirectedEdge_Op { + var op pb.DirectedEdge_Op + switch top { + case Set: + op = pb.DirectedEdge_SET + case Del: + op = pb.DirectedEdge_DEL + case Ovr: + op = pb.DirectedEdge_OVR + default: + x.Fatalf("Unhandled operation: %+v", top) + } + return op +} + // NewPosting takes the given edge and returns its equivalent representation as a posting. func NewPosting(t *pb.DirectedEdge) *pb.Posting { var op uint32 @@ -789,12 +842,12 @@ func (l *List) updateMutationLayer(mpost *pb.Posting, singleUidUpdate, hasCountI // The current value should be deleted in favor of this value. This needs to // be done because the fingerprint for the value is not math.MaxUint64 as is // the case with the rest of the scalar predicates. - newPlist := &pb.PostingList{} + newPlist := &pb.PostingList{ + Postings: []*pb.Posting{createDeleteAllPosting()}, + } if mpost.Op != Del { - // If we are setting a new value then we can just delete all the older values. - newPlist.Postings = append(newPlist.Postings, createDeleteAllPosting()) + newPlist.Postings = append(newPlist.Postings, mpost) } - newPlist.Postings = append(newPlist.Postings, mpost) l.mutationMap.setCurrentEntries(mpost.StartTs, newPlist) return nil } @@ -833,6 +886,10 @@ func fingerprintEdge(t *pb.DirectedEdge) uint64 { return id } +func FingerprintEdge(t *pb.DirectedEdge) uint64 { + return fingerprintEdge(t) +} + func (l *List) addMutation(ctx context.Context, txn *Txn, t *pb.DirectedEdge) error { l.Lock() defer l.Unlock() @@ -1043,6 +1100,12 @@ func (l *List) setMutationAfterCommit(startTs, commitTs uint64, pl *pb.PostingLi func (l *List) setMutation(startTs uint64, data []byte) { pl := new(pb.PostingList) x.Check(proto.Unmarshal(data, pl)) + l.setMutationWithPosting(startTs, pl) +} + +func (l *List) setMutationWithPosting(startTs uint64, pl *pb.PostingList) { + // pk, _ := x.Parse(l.key) + // fmt.Println("Setting mutation for ", l.key, pk, pl) l.Lock() if l.mutationMap == nil { @@ -1110,6 +1173,13 @@ func (l *List) pickPostings(readTs uint64) (uint64, []*pb.Posting) { } return pi.Uid < pj.Uid }) + + if len(posts) > 0 { + if hasDeleteAll(posts[0]) { + posts = posts[1:] + } + } + return deleteAllMarker, posts } @@ -1258,6 +1328,11 @@ func (l *List) GetLength(readTs uint64) int { length += immutLen } + // pureLength := l.length(readTs, 0) + // if pureLength != length { + // panic(fmt.Sprintf("pure length != length %d %d %s", pureLength, length, l.Print())) + // } + return length } @@ -1451,11 +1526,17 @@ func (l *List) Rollup(alloc *z.Allocator, readTs uint64) ([]*bpb.KV, error) { return bytes.Compare(kvs[i].Key, kvs[j].Key) <= 0 }) - x.PrintRollup(out.plist, out.parts, l.key, kv.Version) + PrintRollup(out.plist, out.parts, l.key, kv.Version) x.VerifyPostingSplits(kvs, out.plist, out.parts, l.key) return kvs, nil } +func PrintRollup(plist *pb.PostingList, parts map[uint64]*pb.PostingList, baseKey []byte, ts uint64) { + k, _ := x.Parse(baseKey) + uids := codec.Decode(plist.Pack, 0) + fmt.Printf("[TXNLOG] DOING ROLLUP for key: %+v at timestamp: %v, uids: %+v\n", k, ts, uids) +} + // ToBackupPostingList uses rollup to generate a single list with no splits. // It's used during backup so that each backed up posting list is stored in a single key. func (l *List) ToBackupPostingList( @@ -2007,6 +2088,18 @@ func (l *List) findStaticValue(readTs uint64) *pb.PostingList { if l.plist != nil && len(l.plist.Postings) > 0 { return l.plist } + if l.plist != nil && l.plist.Pack != nil { + uids := codec.Decode(l.plist.Pack, 0) + return &pb.PostingList{ + Postings: []*pb.Posting{ + { + Uid: uids[0], + ValType: pb.Posting_UID, + Op: Set, + }, + }, + } + } return nil } diff --git a/posting/list_test.go b/posting/list_test.go index 79bce4a1879..4511bddda7c 100644 --- a/posting/list_test.go +++ b/posting/list_test.go @@ -124,7 +124,6 @@ func TestGetSinglePosting(t *testing.T) { res, err := l.StaticValue(1) require.NoError(t, err) - fmt.Println(res, res == nil) require.Equal(t, res == nil, true) l.plist = create_pl(1, 1) diff --git a/posting/lists.go b/posting/lists.go index a4bc4fb355b..239d13786d9 100644 --- a/posting/lists.go +++ b/posting/lists.go @@ -19,6 +19,7 @@ import ( "github.com/dgraph-io/badger/v4" "github.com/dgraph-io/dgo/v250/protos/api" "github.com/dgraph-io/dgraph/v25/protos/pb" + "github.com/dgraph-io/dgraph/v25/types" "github.com/dgraph-io/dgraph/v25/x" "github.com/dgraph-io/ristretto/v2/z" ) @@ -66,10 +67,7 @@ type LocalCache struct { startTs uint64 commitTs uint64 - // The keys for these maps is a string representation of the Badger key for the posting list. - // deltas keep track of the updates made by txn. These must be kept around until written to disk - // during commit. - deltas map[string][]byte + deltas *Deltas // max committed timestamp of the read posting lists. maxVersions map[string]uint64 @@ -78,6 +76,143 @@ type LocalCache struct { plists map[string]*List } +// The keys for these maps is a string representation of the Badger key for the posting list. +// deltas keep track of the updates made by txn. These must be kept around until written to disk +// during commit. +type Deltas struct { + deltas *types.LockedShardedMap[string, []byte] + + // We genereate indexes for the posting lists all at once. Moving them from this map to deltas + // map is uneccessary. More data can be stored per predicate later on. + indexMap map[string]*types.LockedShardedMap[string, *pb.PostingList] +} + +func NewDeltas() *Deltas { + return &Deltas{ + deltas: types.NewLockedShardedMap[string, []byte](), + indexMap: map[string]*types.LockedShardedMap[string, *pb.PostingList]{}, + } +} + +// Call this function after taking a lock on the cache. +func (d *Deltas) GetIndexMapForPredicate(pred string) *types.LockedShardedMap[string, *pb.PostingList] { + val, ok := d.indexMap[pred] + if !ok { + d.indexMap[pred] = types.NewLockedShardedMap[string, *pb.PostingList]() + return d.indexMap[pred] + } + return val +} + +func (d *Deltas) Get(key string) (*pb.PostingList, bool) { + if d == nil { + return nil, false + } + pk, err := x.Parse([]byte(key)) + if err != nil { + return nil, false + } + + res := &pb.PostingList{} + + val, ok := d.deltas.Get(key) + if ok { + if err := proto.Unmarshal(val, res); err != nil { + return nil, false + } + } + + if indexMap, ok := d.indexMap[pk.Attr]; ok { + if value, ok1 := indexMap.Get(key); ok1 { + res.Postings = append(res.Postings, value.Postings...) + } + } + + // fmt.Println("GETTING KEY FROM DELTAS", pk, "res", res, "val", val, "d.deltas", d.deltas, "d.indexMap[pk.Attr]", d.indexMap[pk.Attr], "ok", ok) + + return res, len(res.Postings) > 0 +} + +func (d *Deltas) GetBytes(key string) ([]byte, bool) { + if len(d.indexMap) == 0 { + return d.deltas.Get(key) + } + + pk, err := x.Parse([]byte(key)) + if err != nil { + return nil, false + } + + delta, deltaFound := d.deltas.Get(key) + + if indexMap, ok := d.indexMap[pk.Attr]; ok { + if value, ok1 := indexMap.Get(key); ok1 && deltaFound && len(value.Postings) > 0 { + res := &pb.PostingList{} + if err := proto.Unmarshal(delta, res); err != nil { + return nil, false + } + res.Postings = append(res.Postings, value.Postings...) + data, err := proto.Marshal(res) + if err != nil { + return nil, false + } + return data, true + } else if ok1 && len(value.Postings) > 0 { + data, err := proto.Marshal(value) + if err != nil { + return nil, false + } + return data, true + } + } + + return delta, deltaFound +} + +func (d *Deltas) AddToDeltas(key string, delta []byte) { + d.deltas.Set(key, delta) +} + +func (d *Deltas) IterateKeys(fn func(key string) error) error { + for _, v := range d.indexMap { + if err := v.Iterate(func(key string, value *pb.PostingList) error { + return fn(key) + }); err != nil { + return err + } + } + if err := d.deltas.Iterate(func(key string, value []byte) error { + return fn(key) + }); err != nil { + return err + } + return nil +} + +func (d *Deltas) IteratePostings(fn func(key string, value *pb.PostingList) error) error { + return d.IterateKeys(func(key string) error { + val, ok := d.Get(key) + if !ok { + return nil + } + return fn(key, val) + }) +} + +func (d *Deltas) IterateBytes(fn func(key string, value []byte) error) error { + return d.IterateKeys(func(key string) error { + val, ok := d.Get(key) + if !ok { + return nil + } + data, err := proto.Marshal(val) + if err != nil { + return err + } + return fn(key, data) + }) +} + // struct to implement LocalCache interface from vector-indexer // acts as wrapper for dgraph *LocalCache type viLocalCache struct { @@ -132,7 +267,7 @@ func NewViLocalCache(delegate *LocalCache) *viLocalCache { func NewLocalCache(startTs uint64) *LocalCache { return &LocalCache{ startTs: startTs, - deltas: make(map[string][]byte), + deltas: NewDeltas(), plists: make(map[string]*List), maxVersions: make(map[string]uint64), } @@ -261,9 +396,20 @@ func (lc *LocalCache) getInternal(key []byte, readFromDisk, readUids bool) (*Lis lc.RLock() defer lc.RUnlock() if lc.plists == nil { - return getNew(key, pstore, lc.startTs, readUids) + l, err := getNew(key, pstore, lc.startTs, readUids) + if err != nil { + return nil, err + } + pk, _ := x.Parse(key) + fmt.Println("READING NEW PLIST", pk, l.Print()) + return l, nil } if l, ok := lc.plists[skey]; ok { + if delta, ok := lc.deltas.Get(skey); ok && delta != nil { + l.setMutationWithPosting(lc.startTs, delta) + } + pk, _ := x.Parse(key) + fmt.Println("READING PLIST", pk, l.Print()) return l, nil } return nil, nil @@ -291,10 +437,13 @@ func (lc *LocalCache) getInternal(key []byte, readFromDisk, readUids bool) (*Lis // If we just brought this posting list into memory and we already have a delta for it, let's // apply it before returning the list. lc.RLock() - if delta, ok := lc.deltas[skey]; ok && len(delta) > 0 { - pl.setMutation(lc.startTs, delta) + if delta, ok := lc.deltas.Get(skey); ok && delta != nil { + pl.setMutationWithPosting(lc.startTs, delta) } lc.RUnlock() + + pk, _ := x.Parse(key) + fmt.Println("READING ", pk, pl.Print()) return lc.SetIfAbsent(skey, pl), nil } @@ -331,21 +480,26 @@ func (lc *LocalCache) readPostingListAt(key []byte) (*pb.PostingList, error) { // given key. This is used for retrieving the value of a scalar predicats. func (lc *LocalCache) GetSinglePosting(key []byte) (*pb.PostingList, error) { // This would return an error if there is some data in the local cache, but we couldn't read it. + + pk, _ := x.Parse(key) + fmt.Println("READING SINGLE ", pk) + getListFromLocalCache := func() (*pb.PostingList, error) { lc.RLock() - pl := &pb.PostingList{} - if delta, ok := lc.deltas[string(key)]; ok && len(delta) > 0 { - err := proto.Unmarshal(delta, pl) + if delta, ok := lc.deltas.Get(string(key)); ok && delta != nil { lc.RUnlock() - return pl, err + fmt.Println("READING SINGLE FROM DELTA", pk, delta) + return delta, nil } l := lc.plists[string(key)] lc.RUnlock() if l != nil { - return l.StaticValue(lc.startTs) + res, err := l.StaticValue(lc.startTs) + fmt.Println("READING SINGLE FROM PLISTS", pk, res, err, l.Print()) + return res, err } return nil, nil @@ -363,25 +517,28 @@ func (lc *LocalCache) GetSinglePosting(key []byte) (*pb.PostingList, error) { } pl, err := getPostings() + + fmt.Println("READING SINGLE ", pk, "pl:", pl) + if err == badger.ErrKeyNotFound { + fmt.Println("READING ", pk, nil) return nil, nil } if err != nil { + fmt.Println("READING ", pk, err) return nil, err } // Filter and remove STAR_ALL and OP_DELETE Postings idx := 0 for _, postings := range pl.Postings { - if hasDeleteAll(postings) { - return nil, nil - } if postings.Op != Del { pl.Postings[idx] = postings idx++ } } pl.Postings = pl.Postings[:idx] + // fmt.Println("READING ", pk, pl) return pl, nil } @@ -412,7 +569,7 @@ func (lc *LocalCache) UpdateDeltasAndDiscardLists() { for key, pl := range lc.plists { data := pl.getMutation(lc.startTs) if len(data) > 0 { - lc.deltas[key] = data + lc.deltas.AddToDeltas(key, data) } lc.maxVersions[key] = pl.maxVersion() // We can't run pl.release() here because LocalCache is still being used by other callers @@ -425,16 +582,19 @@ func (lc *LocalCache) UpdateDeltasAndDiscardLists() { func (lc *LocalCache) fillPreds(ctx *api.TxnContext, gid uint32) { lc.RLock() defer lc.RUnlock() - for key := range lc.deltas { + if err := lc.deltas.IterateKeys(func(key string) error { pk, err := x.Parse([]byte(key)) x.Check(err) if len(pk.Attr) == 0 { - continue + return nil } // Also send the group id that the predicate was being served by. This is useful when // checking if Zero should allow a commit during a predicate move. predKey := fmt.Sprintf("%d-%s", gid, pk.Attr) ctx.Preds = append(ctx.Preds, predKey) + return nil + }); err != nil { + x.Check(err) } ctx.Preds = x.Unique(ctx.Preds) } diff --git a/posting/mvcc.go b/posting/mvcc.go index 81c5e375553..c917671631e 100644 --- a/posting/mvcc.go +++ b/posting/mvcc.go @@ -9,6 +9,7 @@ import ( "bytes" "context" "encoding/hex" + "fmt" "math" "strconv" "sync" @@ -273,8 +274,11 @@ func (txn *Txn) CommitToDisk(writer *TxnWriter, commitTs uint64) error { defer cache.Unlock() var keys []string - for key := range cache.deltas { + if err := cache.deltas.IterateKeys(func(key string) error { keys = append(keys, key) + return nil + }); err != nil { + return err } defer func() { @@ -293,10 +297,19 @@ func (txn *Txn) CommitToDisk(writer *TxnWriter, commitTs uint64) error { err := writer.update(commitTs, func(btxn *badger.Txn) error { for ; idx < len(keys); idx++ { key := keys[idx] - data := cache.deltas[key] - if len(data) == 0 { + data, ok := cache.deltas.GetBytes(key) + if !ok || data == nil { + continue + } + pl := &pb.PostingList{} + if err := proto.Unmarshal(data, pl); err != nil { + return err + } + if len(pl.Postings) == 0 { continue } + pk, _ := x.Parse([]byte(key)) + fmt.Println("COMMITTING", pk, pl) if ts := cache.maxVersions[key]; ts >= commitTs { // Skip write because we already have a write at a higher ts. // Logging here can cause a lot of output when doing Raft log replay. So, let's @@ -486,7 +499,8 @@ func (ml *MemoryLayer) IterateDisk(ctx context.Context, f IterateDiskArgs) error if err != nil { return err } - empty, err := l.IsEmpty(f.ReadTs, 0) + empty, err := false, nil + //empty, err := l.IsEmpty(f.ReadTs, 0) switch { case err != nil: return err @@ -575,7 +589,7 @@ func (ml *MemoryLayer) wait() { ml.cache.wait() } -func (ml *MemoryLayer) updateItemInCache(key string, delta []byte, startTs, commitTs uint64) { +func (ml *MemoryLayer) updateItemInCache(key string, delta *pb.PostingList, startTs, commitTs uint64) { if commitTs == 0 { return } @@ -587,24 +601,19 @@ func (ml *MemoryLayer) updateItemInCache(key string, delta []byte, startTs, comm } val, ok := ml.cache.get([]byte(key)) - if !ok { - return - } - val.lastUpdate = commitTs + if ok && val.list != nil && val.list.minTs <= commitTs { + val.lastUpdate = commitTs - if val.list != nil { - p := new(pb.PostingList) - x.Check(proto.Unmarshal(delta, p)) - - if p.Pack == nil { - val.list.setMutationAfterCommit(startTs, commitTs, p, true) - checkForRollup([]byte(key), val.list) - } else { - // Data was rolled up. TODO figure out how is UpdateCachedKeys getting delta which is pack) - ml.del([]byte(key)) + if val.list != nil { + if delta.Pack == nil { + val.list.setMutationAfterCommit(startTs, commitTs, delta, true) + checkForRollup([]byte(key), val.list) + } else { + // Data was rolled up. TODO figure out how is UpdateCachedKeys getting delta which is pack) + ml.del([]byte(key)) + } } - } } @@ -615,8 +624,11 @@ func (txn *Txn) UpdateCachedKeys(commitTs uint64) { } MemLayerInstance.wait() - for key, delta := range txn.cache.deltas { - MemLayerInstance.updateItemInCache(key, delta, txn.StartTs, commitTs) + if err := txn.cache.deltas.IteratePostings(func(key string, value *pb.PostingList) error { + MemLayerInstance.updateItemInCache(key, value, txn.StartTs, commitTs) + return nil + }); err != nil { + glog.Errorf("UpdateCachedKeys: error while iterating deltas: %v", err) } } diff --git a/posting/mvcc_test.go b/posting/mvcc_test.go index e519e359d6f..b7856195af0 100644 --- a/posting/mvcc_test.go +++ b/posting/mvcc_test.go @@ -73,7 +73,7 @@ func TestCacheAfterDeltaUpdateRecieved(t *testing.T) { // Write delta to disk and call update txn := Oracle().RegisterStartTs(5) - txn.cache.deltas[string(key)] = delta + txn.cache.deltas.AddToDeltas(string(key), delta) writer := NewTxnWriter(pstore) require.NoError(t, txn.CommitToDisk(writer, 15)) @@ -145,6 +145,8 @@ func BenchmarkTestCache(b *testing.B) { } func TestRollupTimestamp(t *testing.T) { + require.NoError(t, schema.ParseBytes([]byte("rollup: [uid] ."), 1)) + attr := x.AttrInRootNamespace("rollup") key := x.DataKey(attr, 1) // 3 Delta commits. @@ -212,7 +214,7 @@ func TestCacheStaleWhenMaxTsLessThanReadTs(t *testing.T) { require.NoError(t, err) txn1 := Oracle().RegisterStartTs(5) - txn1.cache.deltas[string(key)] = delta1 + txn1.cache.deltas.AddToDeltas(string(key), delta1) writer1 := NewTxnWriter(pstore) require.NoError(t, txn1.CommitToDisk(writer1, 10)) @@ -245,7 +247,7 @@ func TestCacheStaleWhenMaxTsLessThanReadTs(t *testing.T) { require.NoError(t, err) txn2 := Oracle().RegisterStartTs(15) - txn2.cache.deltas[string(key)] = delta2 + txn2.cache.deltas.AddToDeltas(string(key), delta2) writer2 := NewTxnWriter(pstore) require.NoError(t, txn2.CommitToDisk(writer2, 20)) @@ -274,6 +276,8 @@ func TestCacheStaleWhenMaxTsLessThanReadTs(t *testing.T) { } func TestPostingListRead(t *testing.T) { + require.NoError(t, schema.ParseBytes([]byte("emptypl: [uid] ."), 1)) + attr := x.AttrInRootNamespace("emptypl") key := x.DataKey(attr, 1) diff --git a/posting/oracle.go b/posting/oracle.go index d7c3837b4b2..63526608146 100644 --- a/posting/oracle.go +++ b/posting/oracle.go @@ -9,6 +9,7 @@ import ( "context" "encoding/hex" "math" + "sort" "sync" "sync/atomic" "time" @@ -16,6 +17,7 @@ import ( "github.com/dgraph-io/badger/v4" "github.com/golang/glog" ostats "go.opencensus.io/stats" + "google.golang.org/protobuf/proto" "github.com/dgraph-io/dgraph/v25/protos/pb" "github.com/dgraph-io/dgraph/v25/tok/index" @@ -54,6 +56,84 @@ type Txn struct { lastUpdate time.Time cache *LocalCache // This pointer does not get modified. + + pointers [](*[]byte) +} + +func (txn *Txn) AddPointer(p *[]byte) { + if txn.pointers == nil { + txn.pointers = make([](*[]byte), 1) + txn.pointers[0] = p + } + txn.pointers = append(txn.pointers, p) +} + +func (txn *Txn) GetPointers() [](*[]byte) { + return txn.pointers +} + +func SortAndDedupPostings(postings []*pb.Posting) []*pb.Posting { + // Sort postings by UID + sort.Slice(postings, func(i, j int) bool { + return postings[i].Uid < postings[j].Uid + }) + + //In-place filtering: keep only the last occurrence for each UID + n := 0 // write index + for i := 0; i < len(postings); { + j := i + 1 + // Skip all postings with same UID + for j < len(postings) && postings[j].Uid == postings[i].Uid { + j++ + } + // Keep only the last posting for this UID + postings[n] = postings[j-1] + n++ + i = j + } + return postings[:n] +} + +func (txn *Txn) AddDelta(key string, input *pb.PostingList, doSortAndDedup bool, addToList bool) (*pb.PostingList, error) { + txn.cache.Lock() + defer txn.cache.Unlock() + + pl := new(pb.PostingList) + + if addToList { + prevDelta, ok := txn.cache.deltas.Get(key) + if ok { + pl.Postings = append(pl.Postings, prevDelta.Postings...) + } + } + + pl.Postings = append(pl.Postings, input.Postings...) + + if doSortAndDedup { + pl.Postings = SortAndDedupPostings(pl.Postings) + } + + newPl, err := proto.Marshal(pl) + if err != nil { + glog.Errorf("Error marshalling posting list: %v", err) + return nil, err + } + + txn.cache.deltas.AddToDeltas(key, newPl) + + list, listOk := txn.cache.plists[key] + if listOk { + list.setMutation(txn.StartTs, newPl) + } + return pl, nil +} + +func (txn *Txn) LockCache() { + txn.cache.Lock() +} + +func (txn *Txn) UnlockCache() { + txn.cache.Unlock() } // struct to implement Txn interface from vector-indexer @@ -324,8 +404,11 @@ func (o *oracle) ProcessDelta(delta *pb.OracleDelta) { for _, status := range delta.Txns { txn := o.pendingTxns[status.StartTs] if txn != nil && status.CommitTs > 0 { - for k := range txn.cache.deltas { - IncrRollup.addKeyToBatch([]byte(k), 0) + if err := txn.cache.deltas.IterateBytes(func(key string, value []byte) error { + IncrRollup.addKeyToBatch([]byte(key), 0) + return nil + }); err != nil { + glog.Errorf("ProcessDelta: error while iterating deltas for txn %d: %v", status.StartTs, err) } } delete(o.pendingTxns, status.StartTs) @@ -379,17 +462,6 @@ func (o *oracle) GetTxn(startTs uint64) *Txn { return o.pendingTxns[startTs] } -func (txn *Txn) matchesDelta(ok func(key []byte) bool) bool { - txn.Lock() - defer txn.Unlock() - for key := range txn.cache.deltas { - if ok([]byte(key)) { - return true - } - } - return false -} - // IterateTxns returns a list of start timestamps for currently pending transactions, which match // the provided function. func (o *oracle) IterateTxns(ok func(key []byte) bool) []uint64 { @@ -397,8 +469,13 @@ func (o *oracle) IterateTxns(ok func(key []byte) bool) []uint64 { defer o.RUnlock() var timestamps []uint64 for startTs, txn := range o.pendingTxns { - if txn.matchesDelta(ok) { - timestamps = append(timestamps, startTs) + if err := txn.cache.deltas.IterateBytes(func(key string, value []byte) error { + if ok([]byte(key)) { + timestamps = append(timestamps, startTs) + } + return nil + }); err != nil { + glog.Errorf("IterateTxns: error while iterating deltas for txn %d: %v", startTs, err) } } return timestamps diff --git a/query/upgrade_test.go b/query/upgrade_test.go index 6e2774fe912..a7761042a67 100644 --- a/query/upgrade_test.go +++ b/query/upgrade_test.go @@ -50,7 +50,7 @@ func TestMain(m *testing.M) { WithReplicas(1).WithACL(time.Hour).WithVersion(uc.Before) c, err := dgraphtest.NewLocalCluster(conf) x.Panic(err) - defer func() { c.Cleanup(code != 0) }() + defer func() { c.Cleanup(true) }() x.Panic(c.Start()) hc, err := c.HTTPClient() diff --git a/schema/schema.go b/schema/schema.go index 3ce0da8ea74..4281e5d23ad 100644 --- a/schema/schema.go +++ b/schema/schema.go @@ -666,8 +666,7 @@ func initialTypesInternal(namespace uint64, all bool) []*pb.TypeUpdate { ValueType: pb.Posting_STRING, }, }, - }, - &pb.TypeUpdate{ + }, &pb.TypeUpdate{ TypeName: "dgraph.graphql.persisted_query", Fields: []*pb.SchemaUpdate{ { @@ -697,24 +696,23 @@ func initialTypesInternal(namespace uint64, all bool) []*pb.TypeUpdate { if all || x.WorkerConfig.AclEnabled { // These type definitions are required for deleteUser and deleteGroup GraphQL API to work // properly. - initialTypes = append(initialTypes, - &pb.TypeUpdate{ - TypeName: "dgraph.type.User", - Fields: []*pb.SchemaUpdate{ - { - Predicate: "dgraph.xid", - ValueType: pb.Posting_STRING, - }, - { - Predicate: "dgraph.password", - ValueType: pb.Posting_PASSWORD, - }, - { - Predicate: "dgraph.user.group", - ValueType: pb.Posting_UID, - }, + initialTypes = append(initialTypes, &pb.TypeUpdate{ + TypeName: "dgraph.type.User", + Fields: []*pb.SchemaUpdate{ + { + Predicate: "dgraph.xid", + ValueType: pb.Posting_STRING, + }, + { + Predicate: "dgraph.password", + ValueType: pb.Posting_PASSWORD, + }, + { + Predicate: "dgraph.user.group", + ValueType: pb.Posting_UID, }, }, + }, &pb.TypeUpdate{ TypeName: "dgraph.type.Group", Fields: []*pb.SchemaUpdate{ @@ -771,36 +769,31 @@ func CompleteInitialSchema(namespace uint64) []*pb.SchemaUpdate { func initialSchemaInternal(namespace uint64, all bool) []*pb.SchemaUpdate { var initialSchema []*pb.SchemaUpdate - initialSchema = append(initialSchema, []*pb.SchemaUpdate{ - { + initialSchema = append(initialSchema, + &pb.SchemaUpdate{ Predicate: "dgraph.type", ValueType: pb.Posting_STRING, Directive: pb.SchemaUpdate_INDEX, Tokenizer: []string{"exact"}, List: true, - }, - { + }, &pb.SchemaUpdate{ Predicate: "dgraph.drop.op", ValueType: pb.Posting_STRING, - }, - { + }, &pb.SchemaUpdate{ Predicate: "dgraph.graphql.schema", ValueType: pb.Posting_STRING, - }, - { + }, &pb.SchemaUpdate{ Predicate: "dgraph.graphql.xid", ValueType: pb.Posting_STRING, Directive: pb.SchemaUpdate_INDEX, Tokenizer: []string{"exact"}, Upsert: true, - }, - { + }, &pb.SchemaUpdate{ Predicate: "dgraph.graphql.p_query", ValueType: pb.Posting_STRING, Directive: pb.SchemaUpdate_INDEX, Tokenizer: []string{"sha256"}, - }, - }...) + }) if namespace == x.RootNamespace { initialSchema = append(initialSchema, []*pb.SchemaUpdate{ diff --git a/types/locked_sharded_map.go b/types/locked_sharded_map.go new file mode 100644 index 00000000000..19db5ff8cea --- /dev/null +++ b/types/locked_sharded_map.go @@ -0,0 +1,187 @@ +/* + * SPDX-FileCopyrightText: © 2017-2025 Istari Digital, Inc. + * SPDX-License-Identifier: Apache-2.0 + */ + +// Package types contains some very common utilities used by Dgraph. These utilities +// are of "miscellaneous" nature, e.g., error checking. +package types + +import ( + "sync" + + "github.com/dgryski/go-farm" +) + +// LockedShardedMap is a thread-safe, sharded map with generic key-value types. +type LockedShardedMap[K comparable, V any] struct { + shards []map[K]V + locks []sync.RWMutex +} + +// NewLockedShardedMap creates a new LockedShardedMap. +func NewLockedShardedMap[K comparable, V any]() *LockedShardedMap[K, V] { + shards := make([]map[K]V, NumShards) + locks := make([]sync.RWMutex, NumShards) + for i := range shards { + shards[i] = make(map[K]V) + } + return &LockedShardedMap[K, V]{shards: shards, locks: locks} +} + +func (s *LockedShardedMap[K, V]) getShardIndex(key K) int { + // Only works for integer-like keys (uint64 etc). For generic types, + // a better hash function is needed. We'll assume uint64 for now. + switch k := any(key).(type) { + case uint64: + return int(k % uint64(NumShards)) + case string: + return int(farm.Fingerprint64([]byte(k)) % uint64(NumShards)) + default: + panic("LockedShardedMap only supports uint64 and string keys for now") + } +} + +func (s *LockedShardedMap[K, V]) Set(key K, value V) { + if s == nil { + return + } + index := s.getShardIndex(key) + s.locks[index].Lock() + defer s.locks[index].Unlock() + s.shards[index][key] = value +} + +func (s *LockedShardedMap[K, V]) Get(key K) (V, bool) { + var zero V + if s == nil { + return zero, false + } + index := s.getShardIndex(key) + s.locks[index].RLock() + defer s.locks[index].RUnlock() + val, ok := s.shards[index][key] + return val, ok +} + +func (s *LockedShardedMap[K, V]) Update(key K, update func(V, bool) V) { + if s == nil { + return + } + index := s.getShardIndex(key) + s.locks[index].Lock() + defer s.locks[index].Unlock() + val, ok := s.shards[index][key] + s.shards[index][key] = update(val, ok) +} + +func (s *LockedShardedMap[K, V]) Merge(other *LockedShardedMap[K, V], ag func(a, b V) V) { + var wg sync.WaitGroup + for i := range s.shards { + wg.Add(1) + go func(i int) { + defer wg.Done() + otherShard := other.shards[i] + for k, v := range otherShard { + s.locks[i].Lock() + if existing, ok := s.shards[i][k]; ok { + s.shards[i][k] = ag(existing, v) + } else { + s.shards[i][k] = v + } + s.locks[i].Unlock() + } + }(i) + } + wg.Wait() +} + +func (s *LockedShardedMap[K, V]) Len() int { + if s == nil { + return 0 + } + var count int + for i := range s.shards { + s.locks[i].RLock() + count += len(s.shards[i]) + s.locks[i].RUnlock() + } + return count +} + +func (s *LockedShardedMap[K, V]) ParallelIterate(f func(K, V) error) error { + if s == nil { + return nil + } + + var ( + wg sync.WaitGroup + errCh = make(chan error, 1) + once sync.Once + ) + + for i := range s.shards { + wg.Add(1) + go func(i int) { + defer wg.Done() + + s.locks[i].RLock() + defer s.locks[i].RUnlock() + + for k, v := range s.shards[i] { + if err := f(k, v); err != nil { + once.Do(func() { + errCh <- err + }) + return + } + } + }(i) + } + + // Wait in a separate goroutine so we can still select on errCh. + done := make(chan struct{}) + go func() { + wg.Wait() + close(done) + }() + + select { + case err := <-errCh: + return err + case <-done: + return nil + } +} + +func (s *LockedShardedMap[K, V]) Iterate(f func(K, V) error) error { + if s == nil { + return nil + } + for i := range s.shards { + s.locks[i].RLock() + for k, v := range s.shards[i] { + if err := f(k, v); err != nil { + s.locks[i].RUnlock() + return err + } + } + s.locks[i].RUnlock() + } + return nil +} + +func (s *LockedShardedMap[K, V]) IsEmpty() bool { + if s == nil { + return true + } + for i := range s.shards { + s.locks[i].RLock() + if len(s.shards[i]) > 0 { + s.locks[i].RUnlock() + return false + } + s.locks[i].RUnlock() + } + return true +} diff --git a/worker/draft.go b/worker/draft.go index c2cb9947519..d4e2d9fdee6 100644 --- a/worker/draft.go +++ b/worker/draft.go @@ -516,6 +516,22 @@ func (n *node) applyMutations(ctx context.Context, proposal *pb.Proposal) (rerr m := proposal.Mutations + txn := posting.Oracle().RegisterStartTs(m.StartTs) + if txn.ShouldAbort() { + span.AddEvent("Txn should abort.", trace.WithAttributes( + attribute.Int64("start_ts", int64(m.StartTs)), + )) + return x.ErrConflict + } + // Discard the posting lists from cache to release memory at the end. + defer txn.Update() + + featureFlag := true + if featureFlag { + mp := posting.NewMutationPipeline(txn) + return mp.Process(ctx, m.Edges) + } + // It is possible that the user gives us multiple versions of the same edge, one with no facets // and another with facets. In that case, use stable sort to maintain the ordering given to us // by the user. @@ -528,16 +544,6 @@ func (n *node) applyMutations(ctx context.Context, proposal *pb.Proposal) (rerr return ei.GetEntity() < ej.GetEntity() }) - txn := posting.Oracle().RegisterStartTs(m.StartTs) - if txn.ShouldAbort() { - span.AddEvent("Txn should abort.", trace.WithAttributes( - attribute.Int64("start_ts", int64(m.StartTs)), - )) - return x.ErrConflict - } - // Discard the posting lists from cache to release memory at the end. - defer txn.Update() - process := func(edges []*pb.DirectedEdge) error { var retries int for _, edge := range edges { diff --git a/worker/draft_test.go b/worker/draft_test.go index 945e1ce7971..057c975c0a4 100644 --- a/worker/draft_test.go +++ b/worker/draft_test.go @@ -38,6 +38,107 @@ func getEntryForCommit(index, startTs, commitTs uint64) raftpb.Entry { return raftpb.Entry{Index: index, Term: 1, Type: raftpb.EntryNormal, Data: data} } +// func BenchmarkProcessListIndex(b *testing.B) { +// dir, err := os.MkdirTemp("", "storetest_") +// x.Check(err) +// defer os.RemoveAll(dir) + +// opt := badger.DefaultOptions(dir) +// ps, err := badger.OpenManaged(opt) +// x.Check(err) +// pstore = ps +// // Not using posting list cache +// posting.Init(ps, 0, false) +// Init(ps) +// err = schema.ParseBytes([]byte("testAttr: [string] @index(exact) ."), 1) +// require.NoError(b, err) + +// ctx := context.Background() +// pipeline := &PredicatePipeline{ +// attr: "0-testAttr", +// edges: make(chan *pb.DirectedEdge, 1000), +// wg: &sync.WaitGroup{}, +// errCh: make(chan error, 1), +// } + +// txn := posting.Oracle().RegisterStartTs(5) +// mp := &MutationPipeline{txn: txn} + +// // Generate 1000 edges +// populatePipeline := func() { +// pipeline = &PredicatePipeline{ +// attr: "0-testAttr", +// edges: make(chan *pb.DirectedEdge, 1000), +// wg: &sync.WaitGroup{}, +// errCh: make(chan error, 1), +// } + +// txn = posting.Oracle().RegisterStartTs(5) +// mp = &MutationPipeline{txn: txn} + +// for i := 0; i < 1000; i++ { +// edge := &pb.DirectedEdge{ +// Entity: uint64(i + 1), +// Attr: "0-testAttr", +// Value: []byte(fmt.Sprintf("value%d", rand.Intn(1000))), +// ValueType: pb.Posting_STRING, +// Op: pb.DirectedEdge_SET, +// } +// pipeline.edges <- edge +// } +// } + +// b.ResetTimer() + +// b.Run("Baseline", func(b *testing.B) { +// for i := 0; i < b.N; i++ { +// populatePipeline() +// } +// }) + +// b.Run("DefaultPipeline", func(b *testing.B) { +// for i := 0; i < b.N; i++ { +// populatePipeline() +// var wg sync.WaitGroup +// wg.Add(1) +// go func() { +// mp.DefaultPipeline(ctx, pipeline) +// wg.Done() +// }() +// close(pipeline.edges) +// wg.Wait() +// } +// }) + +// b.Run("ProcessListWithoutIndex", func(b *testing.B) { +// for i := 0; i < b.N; i++ { +// populatePipeline() +// var wg sync.WaitGroup +// wg.Add(1) +// go func() { +// mp.ProcessListWithoutIndex(ctx, pipeline) +// wg.Done() +// }() +// close(pipeline.edges) +// wg.Wait() +// } +// }) + +// b.Run("ProcessListIndex", func(b *testing.B) { +// for i := 0; i < b.N; i++ { +// populatePipeline() +// var wg sync.WaitGroup +// wg.Add(1) +// go func() { +// mp.ProcessListIndex(ctx, pipeline) +// wg.Done() +// }() +// close(pipeline.edges) +// wg.Wait() +// } +// }) +// } + func TestCalculateSnapshot(t *testing.T) { dir := t.TempDir() ds := raftwal.Init(dir) diff --git a/worker/mutation.go b/worker/mutation.go index fdac2a41c1b..53673cdb485 100644 --- a/worker/mutation.go +++ b/worker/mutation.go @@ -55,6 +55,16 @@ func isDeletePredicateEdge(edge *pb.DirectedEdge) bool { return edge.Entity == 0 && isStarAll(edge.Value) } +func newRunMutations(ctx context.Context, edges []*pb.DirectedEdge, txn *posting.Txn) error { + mp := posting.NewMutationPipeline(txn) + return mp.Process(ctx, edges) +} + +func newRunMutation(ctx context.Context, edge *pb.DirectedEdge, txn *posting.Txn) error { + mp := posting.NewMutationPipeline(txn) + return mp.Process(ctx, []*pb.DirectedEdge{edge}) +} + // runMutation goes through all the edges and applies them. func runMutation(ctx context.Context, edge *pb.DirectedEdge, txn *posting.Txn) error { ctx = schema.GetWriteContext(ctx) diff --git a/worker/mutation_unit_test.go b/worker/mutation_unit_test.go index c95034b1275..a8e313c44b1 100644 --- a/worker/mutation_unit_test.go +++ b/worker/mutation_unit_test.go @@ -47,8 +47,8 @@ func TestReverseEdge(t *testing.T) { Op: pb.DirectedEdge_DEL, } - x.Check(runMutation(ctx, edge, txn)) - x.Check(runMutation(ctx, edge, txn)) + x.Check(newRunMutation(ctx, edge, txn)) + x.Check(newRunMutation(ctx, edge, txn)) pl, err := txn.Get(x.DataKey(attr, 1)) require.NoError(t, err) @@ -105,10 +105,10 @@ func TestReverseEdgeSetDel(t *testing.T) { Op: pb.DirectedEdge_SET, } - x.Check(runMutation(ctx, edgeSet1, txn)) - x.Check(runMutation(ctx, edgeSet2, txn)) - x.Check(runMutation(ctx, edgeSet3, txn)) - x.Check(runMutation(ctx, edgeDel, txn)) + x.Check(newRunMutation(ctx, edgeSet1, txn)) + x.Check(newRunMutation(ctx, edgeSet2, txn)) + x.Check(newRunMutation(ctx, edgeSet3, txn)) + x.Check(newRunMutation(ctx, edgeDel, txn)) pl, err := txn.Get(x.ReverseKey(attr, 2)) require.NoError(t, err) diff --git a/worker/sort_test.go b/worker/sort_test.go index 4c7e1f39542..1dfee6e460e 100644 --- a/worker/sort_test.go +++ b/worker/sort_test.go @@ -8,8 +8,10 @@ package worker import ( "context" "fmt" + "math" "math/rand" "os" + "sync" "testing" "github.com/dgraph-io/badger/v4" @@ -17,7 +19,10 @@ import ( "github.com/dgraph-io/dgraph/v25/posting" "github.com/dgraph-io/dgraph/v25/protos/pb" "github.com/dgraph-io/dgraph/v25/schema" + "github.com/dgraph-io/dgraph/v25/tok" + "github.com/dgraph-io/dgraph/v25/types" "github.com/dgraph-io/dgraph/v25/x" + "github.com/pkg/errors" "github.com/stretchr/testify/require" ) @@ -90,6 +95,319 @@ func TestEmptyTypeSchema(t *testing.T) { x.ParseNamespaceAttr(types[0].TypeName) } +func TestDatetime(t *testing.T) { + // Setup temporary directory for Badger DB + dir, err := os.MkdirTemp("", "storetest_") + require.NoError(t, err) + defer os.RemoveAll(dir) + + opt := badger.DefaultOptions(dir) + ps, err := badger.OpenManaged(opt) + require.NoError(t, err) + posting.Init(ps, 0, false) + Init(ps) + + // Set schema + schemaTxt := ` + t: datetime @index(year) . + ` + err = schema.ParseBytes([]byte(schemaTxt), 1) + require.NoError(t, err) + + ctx := context.Background() + newRunMutation := func(startTs, commitTs uint64, edges []*pb.DirectedEdge) { + txn := posting.Oracle().RegisterStartTs(startTs) + for _, edge := range edges { + require.NoError(t, newRunMutation(ctx, edge, txn)) + } + txn.Update() + writer := posting.NewTxnWriter(ps) + require.NoError(t, txn.CommitToDisk(writer, commitTs)) + require.NoError(t, writer.Flush()) + txn.UpdateCachedKeys(commitTs) + } + + newRunMutation(1, 3, []*pb.DirectedEdge{ + { + Entity: 1, + Attr: x.AttrInRootNamespace("t"), + Value: []byte("2020-01-01T00:00:00Z"), + ValueType: pb.Posting_DEFAULT, + Op: pb.DirectedEdge_SET, + }, + }) + +} + +type indexMutationInfo struct { + tokenizers []tok.Tokenizer + factorySpecs []*tok.FactoryCreateSpec + edge *pb.DirectedEdge // Represents the original uid -> value edge. + val types.Val + op pb.DirectedEdge_Op +} + +func indexTokens(ctx context.Context, info *indexMutationInfo) ([]string, error) { + attr := info.edge.Attr + lang := info.edge.GetLang() + + schemaType, err := schema.State().TypeOf(attr) + if err != nil || !schemaType.IsScalar() { + return nil, errors.Errorf("Cannot index attribute %s of type object.", attr) + } + + if !schema.State().IsIndexed(ctx, attr) { + return nil, errors.Errorf("Attribute %s is not indexed.", attr) + } + sv, err := types.Convert(info.val, schemaType) + if err != nil { + return nil, errors.Wrap(err, "Cannot convert value to scalar type") + } + + var tokens []string + for _, it := range info.tokenizers { + toks, err := tok.BuildTokens(sv.Value, tok.GetTokenizerForLang(it, lang)) + if err != nil { + return tokens, errors.Wrapf(err, "Cannot build tokens for attribute %s", attr) + } + tokens = append(tokens, toks...) + } + return tokens, nil +} + +func TestStringIndexWithLang(t *testing.T) { + // Setup temporary directory for Badger DB + dir, err := os.MkdirTemp("", "storetest_") + require.NoError(t, err) + defer os.RemoveAll(dir) + + opt := badger.DefaultOptions(dir) + ps, err := badger.OpenManaged(opt) + require.NoError(t, err) + posting.Init(ps, 0, false) + Init(ps) + + // Set schema + schemaTxt := ` + name: string @index(fulltext, trigram, term, exact) @lang . + ` + + err = schema.ParseBytes([]byte(schemaTxt), 1) + require.NoError(t, err) + + ctx := context.Background() + newRunMutation := func(startTs, commitTs uint64, edges []*pb.DirectedEdge) { + txn := posting.Oracle().RegisterStartTs(startTs) + require.NoError(t, newRunMutations(ctx, edges, txn)) + txn.Update() + writer := posting.NewTxnWriter(ps) + require.NoError(t, txn.CommitToDisk(writer, commitTs)) + require.NoError(t, writer.Flush()) + txn.UpdateCachedKeys(commitTs) + } + + attr := x.AttrInRootNamespace("name") + + // Prepare 400 mutations across 4 threads, 100 per thread (kept modest for stability). + const ( + threads = 10 + perThread = 20000 + total = threads * perThread + baseStartTs = uint64(10) + ) + + // uid -> value map + values := make(map[uint64]string, total) + for i := 0; i < total; i++ { + uid := uint64(i + 1) + // Simple deterministic values with shared tokens and unique numbers. + values[uid] = fmt.Sprintf("title %d", i+1) + } + + // Build expected token -> set of uids + tokenizers := schema.State().Tokenizer(ctx, attr) + expected := make(map[string]map[uint64]struct{}, total) + for uid, val := range values { + info := &indexMutationInfo{ + tokenizers: tokenizers, + op: pb.DirectedEdge_SET, + val: types.Val{Tid: types.StringID, Value: []byte(val)}, + edge: &pb.DirectedEdge{ + Attr: attr, + Value: []byte(val), + Lang: "en", + Op: pb.DirectedEdge_SET, + }, + } + toks, err := indexTokens(ctx, info) + require.NoError(t, err) + for _, tk := range toks { + if expected[tk] == nil { + expected[tk] = make(map[uint64]struct{}) + } + expected[tk][uid] = struct{}{} + } + } + + // Run 4 threads; each thread writes 100 edges with distinct ts + var wg sync.WaitGroup + wg.Add(threads) + for th := 0; th < threads; th++ { + th := th + go func() { + defer wg.Done() + start := th*perThread + 1 + end := start + perThread + edges := make([]*pb.DirectedEdge, 0, perThread) + for i := start; i < end; i++ { + uid := uint64(i) + edges = append(edges, &pb.DirectedEdge{ + Entity: uid, + Attr: attr, + Value: []byte(values[uid]), + ValueType: pb.Posting_DEFAULT, + Lang: "en", + Op: pb.DirectedEdge_SET, + }) + } + sTs := baseStartTs + uint64(th*10) + cTs := sTs + 2 + newRunMutation(sTs, cTs, edges) + }() + } + wg.Wait() + + // Verify all tokens have the expected UIDs. + readTs := baseStartTs + uint64(threads*10) + 10 + for tk, uidset := range expected { + key := x.IndexKey(attr, tk) + txn := posting.Oracle().RegisterStartTs(readTs) + pl, err := txn.Get(key) + require.NoError(t, err) + lst, err := pl.Uids(posting.ListOptions{ReadTs: readTs}) + require.NoError(t, err) + got := make(map[uint64]struct{}, len(lst.Uids)) + for _, u := range lst.Uids { + got[u] = struct{}{} + } + // Compare sets + require.Equal(t, len(uidset), len(got), "mismatch uid count for token %q", tk) + for u := range uidset { + if _, ok := got[u]; !ok { + t.Fatalf("missing uid %d for token %q", u, tk) + } + } + } +} + +func TestCount(t *testing.T) { + t.Skip() + // Setup temporary directory for Badger DB + dir, err := os.MkdirTemp("", "storetest_") + require.NoError(t, err) + defer os.RemoveAll(dir) + + opt := badger.DefaultOptions(dir) + ps, err := badger.OpenManaged(opt) + require.NoError(t, err) + posting.Init(ps, 0, false) + Init(ps) + + // Set schema + schemaTxt := ` + friends: [uid] @count . + ` + + err = schema.ParseBytes([]byte(schemaTxt), 1) + require.NoError(t, err) + ctx := context.Background() + newRunMutation := func(startTs, commitTs uint64, edges []*pb.DirectedEdge) { + txn := posting.Oracle().RegisterStartTs(startTs) + require.NoError(t, newRunMutations(ctx, edges, txn)) + txn.Update() + writer := posting.NewTxnWriter(ps) + require.NoError(t, txn.CommitToDisk(writer, commitTs)) + require.NoError(t, writer.Flush()) + txn.UpdateCachedKeys(commitTs) + } + + pred := x.AttrInRootNamespace("friends") + + // Prepare mutations such that each subject gets multiple uid edges, and + // each edge is added from a different thread. We also send multiple + // batches per thread. + const ( + subjects = 10 // total number of subjects/entities + edgesPer = 5 // number of edges per subject + threads = 2 // one thread per edge ordinal, touching all subjects + baseStartTs = uint64(10) + total = subjects * edgesPer + ) + + // 1) Pre-generate all mutations into one big slice + edgesAll := make([]*pb.DirectedEdge, 0, total) + for subj := 1; subj <= subjects; subj++ { + uid := uint64(subj) + for e := 0; e < edgesPer; e++ { + // Unique object per (subject, edge-ordinal) pair to avoid duplicates. + // Ensures exactly 'edgesPer' distinct UIDs per subject. + obj := uint64(1_000_000 + subj*100 + e) + edgesAll = append(edgesAll, &pb.DirectedEdge{ + Entity: uid, + Attr: pred, + ValueId: obj, + ValueType: pb.Posting_UID, + Op: pb.DirectedEdge_SET, + }) + } + } + + // Shuffle the edges to simulate randomness (determinism depends on rand.Seed in package scope) + for i := range edgesAll { + j := rand.Intn(i + 1) + edgesAll[i], edgesAll[j] = edgesAll[j], edgesAll[i] + } + + // 2) Dispatch pre-generated mutations into threads, in multiple batches per thread + var wg sync.WaitGroup + wg.Add(threads) + for th := 0; th < threads; th++ { + th := th + go func() { + defer wg.Done() + // Split each thread's disjoint chunk into multiple batches/transactions + const batches = 5 + chunk := total / threads + chunkStart := th * chunk + chunkEnd := chunkStart + chunk + perBatch := chunk / batches + for b := 0; b < batches; b++ { + batchStart := chunkStart + b*perBatch + batchEnd := batchStart + perBatch + if b == batches-1 { + batchEnd = chunkEnd + } + batch := edgesAll[batchStart:batchEnd] + // Space out start/commit timestamps per thread and per batch to avoid collisions + sTs := baseStartTs + uint64(th*100) + uint64(b*2) + cTs := sTs + 1 + newRunMutation(sTs, cTs, batch) + } + }() + } + wg.Wait() + + // Verify the @count index for the exact number of edges per subject. + countKey := x.CountKey(pred, edgesPer, false) + txn := posting.Oracle().RegisterStartTs(math.MaxUint64) + pl, err := txn.Get(countKey) + require.NoError(t, err) + uids, err := pl.Uids(posting.ListOptions{ReadTs: math.MaxUint64}) + require.NoError(t, err) + fmt.Println(uids.Uids) + require.Equal(t, subjects, len(uids.Uids)) +} + func TestDeleteSetWithVarEdgeCorruptsData(t *testing.T) { // Setup temporary directory for Badger DB dir, err := os.MkdirTemp("", "storetest_") @@ -119,10 +437,10 @@ func TestDeleteSetWithVarEdgeCorruptsData(t *testing.T) { uidRoom := uint64(1) uidJohn := uint64(2) - runMutation := func(startTs, commitTs uint64, edges []*pb.DirectedEdge) { + newRunMutation := func(startTs, commitTs uint64, edges []*pb.DirectedEdge) { txn := posting.Oracle().RegisterStartTs(startTs) for _, edge := range edges { - require.NoError(t, runMutation(ctx, edge, txn)) + require.NoError(t, newRunMutation(ctx, edge, txn)) } txn.Update() writer := posting.NewTxnWriter(ps) @@ -132,7 +450,7 @@ func TestDeleteSetWithVarEdgeCorruptsData(t *testing.T) { } // Initial mutation: Set John → Leopard - runMutation(1, 3, []*pb.DirectedEdge{ + newRunMutation(1, 3, []*pb.DirectedEdge{ { Entity: uidJohn, Attr: attrPerson, @@ -162,7 +480,7 @@ func TestDeleteSetWithVarEdgeCorruptsData(t *testing.T) { // Second mutation: Remove John from Leopard, assign Amanda uidAmanda := uint64(3) - runMutation(6, 8, []*pb.DirectedEdge{ + newRunMutation(6, 8, []*pb.DirectedEdge{ { Entity: uidJohn, Attr: attrOffice, @@ -225,7 +543,7 @@ func TestGetScalarList(t *testing.T) { runM := func(startTs, commitTs uint64, edges []*pb.DirectedEdge) { txn := posting.Oracle().RegisterStartTs(startTs) for _, edge := range edges { - x.Check(runMutation(context.Background(), edge, txn)) + x.Check(newRunMutation(context.Background(), edge, txn)) } txn.Update() writer := posting.NewTxnWriter(pstore) @@ -283,7 +601,7 @@ func TestMultipleTxnListCount(t *testing.T) { runM := func(startTs, commitTs uint64, edges []*pb.DirectedEdge) { txn := posting.Oracle().RegisterStartTs(startTs) for _, edge := range edges { - x.Check(runMutation(ctx, edge, txn)) + x.Check(newRunMutation(ctx, edge, txn)) } txn.Update() writer := posting.NewTxnWriter(pstore) @@ -342,7 +660,7 @@ func TestScalarPredicateRevCount(t *testing.T) { runM := func(startTs, commitTs uint64, edges []*pb.DirectedEdge) { txn := posting.Oracle().RegisterStartTs(startTs) for _, edge := range edges { - x.Check(runMutation(ctx, edge, txn)) + x.Check(newRunMutation(ctx, edge, txn)) } txn.Update() writer := posting.NewTxnWriter(pstore) @@ -423,7 +741,7 @@ func TestScalarPredicateIntCount(t *testing.T) { runM := func(startTs, commitTs uint64, edge *pb.DirectedEdge) { txn := posting.Oracle().RegisterStartTs(startTs) - x.Check(runMutation(ctx, edge, txn)) + x.Check(newRunMutation(ctx, edge, txn)) txn.Update() writer := posting.NewTxnWriter(pstore) require.NoError(t, txn.CommitToDisk(writer, commitTs)) @@ -477,7 +795,7 @@ func TestScalarPredicateCount(t *testing.T) { runM := func(startTs, commitTs uint64, edge *pb.DirectedEdge) { txn := posting.Oracle().RegisterStartTs(startTs) - x.Check(runMutation(ctx, edge, txn)) + x.Check(newRunMutation(ctx, edge, txn)) txn.Update() writer := posting.NewTxnWriter(pstore) require.NoError(t, txn.CommitToDisk(writer, commitTs)) @@ -531,7 +849,7 @@ func TestSingleUidReplacement(t *testing.T) { attr := x.AttrInRootNamespace("singleUidReplaceTest") // Txn 1. Set 1 -> 2 - x.Check(runMutation(ctx, &pb.DirectedEdge{ + x.Check(newRunMutation(ctx, &pb.DirectedEdge{ ValueId: 2, Attr: attr, Entity: 1, @@ -547,7 +865,7 @@ func TestSingleUidReplacement(t *testing.T) { // Txn 2. Set 1 -> 3 txn = posting.Oracle().RegisterStartTs(9) - x.Check(runMutation(ctx, &pb.DirectedEdge{ + x.Check(newRunMutation(ctx, &pb.DirectedEdge{ ValueId: 3, Attr: attr, Entity: 1, @@ -591,14 +909,14 @@ func TestSingleString(t *testing.T) { attr := x.AttrInRootNamespace("singleUidTest") // Txn 1. Set 1 -> david 2 -> blush - x.Check(runMutation(ctx, &pb.DirectedEdge{ + x.Check(newRunMutation(ctx, &pb.DirectedEdge{ Value: []byte("david"), Attr: attr, Entity: 1, Op: pb.DirectedEdge_SET, }, txn)) - x.Check(runMutation(ctx, &pb.DirectedEdge{ + x.Check(newRunMutation(ctx, &pb.DirectedEdge{ Value: []byte("blush"), Attr: attr, Entity: 2, @@ -614,14 +932,14 @@ func TestSingleString(t *testing.T) { // Txn 2. Set 2 -> david 1 -> blush txn = posting.Oracle().RegisterStartTs(9) - x.Check(runMutation(ctx, &pb.DirectedEdge{ + x.Check(newRunMutation(ctx, &pb.DirectedEdge{ Value: []byte("david"), Attr: attr, Entity: 2, Op: pb.DirectedEdge_SET, }, txn)) - x.Check(runMutation(ctx, &pb.DirectedEdge{ + x.Check(newRunMutation(ctx, &pb.DirectedEdge{ Value: []byte("blush"), Attr: attr, Entity: 1, @@ -693,7 +1011,7 @@ func TestLangExact(t *testing.T) { Lang: "en", } - x.Check(runMutation(ctx, edge, txn)) + x.Check(newRunMutation(ctx, edge, txn)) edge = &pb.DirectedEdge{ Value: []byte("hindi"), @@ -703,7 +1021,7 @@ func TestLangExact(t *testing.T) { Lang: "hi", } - x.Check(runMutation(ctx, edge, txn)) + x.Check(newRunMutation(ctx, edge, txn)) txn.Update() writer := posting.NewTxnWriter(pstore) @@ -745,7 +1063,6 @@ func BenchmarkAddMutationWithIndex(b *testing.B) { posting.Init(ps, 0, false) Init(ps) err = schema.ParseBytes([]byte("benchmarkadd: string @index(term) ."), 1) - fmt.Println(err) if err != nil { panic(err) } @@ -767,7 +1084,7 @@ func BenchmarkAddMutationWithIndex(b *testing.B) { Op: pb.DirectedEdge_SET, } - x.Check(runMutation(ctx, edge, txn)) + x.Check(newRunMutation(ctx, edge, txn)) } } diff --git a/worker/task.go b/worker/task.go index 409ec3f0fc4..eb7443ea28d 100644 --- a/worker/task.go +++ b/worker/task.go @@ -487,6 +487,8 @@ func (qs *queryState) handleValuePostings(ctx context.Context, args funcArgs) er Value: p.Value, } } + pk, _ := x.Parse(key) + fmt.Println("READING SINGLE", pk, vals, pl) } else { pl, err := qs.cache.Get(key) if err != nil { @@ -506,6 +508,8 @@ func (qs *queryState) handleValuePostings(ctx context.Context, args funcArgs) er } vals, fcs, err = retrieveValuesAndFacets(args, pl, facetsTree, listType) + pk, _ := x.Parse(key) + fmt.Println("READING", pk, vals, fcs, pl.Print()) switch { case err == posting.ErrNoValue || (err == nil && len(vals) == 0): From 0a7c1f2f8aa2949568f51722fca76f2548e2e78c Mon Sep 17 00:00:00 2001 From: Matthew McNeely Date: Fri, 1 May 2026 13:08:56 -0400 Subject: [PATCH 02/17] chore(posting): strip debug fmt.Println from hot paths MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Removes the unconditional debug prints scattered through posting/ during the original WIP work — they fired on every read, mutation, rollup, and commit. None of them were guarded by a verbosity flag, so under load they would have produced megabytes of stdout noise per second. Sites stripped: - posting/lists.go: READING / READING SINGLE / GETTING KEY FROM DELTAS - posting/index.go: TOKENS, LOCAL MAP, INSERTING INDEX, UPDATE INDEX, ERRORRRING, "Inserting tokenizer indexes ... took" - posting/mvcc.go: COMMITTING (and unused fmt import) - posting/list.go: "Buidlding committed uids", "Setting mutation", PrintRollup helper (called once internally, never elsewhere) Left in place: printTreeStats() in index.go, which is already gated by the DEBUG_SHOW_HNSW_TREE env var and is an intentional opt-in HNSW debug helper. No behavior change. Co-Authored-By: Claude Opus 4.7 (1M context) --- posting/index.go | 13 ------------- posting/list.go | 11 ----------- posting/lists.go | 29 ++--------------------------- posting/mvcc.go | 3 --- 4 files changed, 2 insertions(+), 54 deletions(-) diff --git a/posting/index.go b/posting/index.go index 8731aca1b69..0dfe9bff30e 100644 --- a/posting/index.go +++ b/posting/index.go @@ -129,11 +129,6 @@ func (mp *MutationPipeline) ProcessVectorIndex(ctx context.Context, pipeline *Pr } func (mp *MutationPipeline) InsertTokenizerIndexes(ctx context.Context, pipeline *PredicatePipeline, postings *map[uint64]*pb.PostingList, info predicateInfo) error { - startTime := time.Now() - defer func() { - fmt.Println("Inserting tokenizer indexes for predicate", pipeline.attr, "took", time.Since(startTime)) - }() - tokenizers := schema.State().Tokenizer(ctx, pipeline.attr) if len(tokenizers) == 0 { return nil @@ -192,7 +187,6 @@ func (mp *MutationPipeline) InsertTokenizerIndexes(ctx context.Context, pipeline if !loaded { tokens, err = indexTokens(ctx, info) if err != nil { - fmt.Println("ERRORRRING", err) x.Panic(err) } syncMap.Store(key, tokens) @@ -204,8 +198,6 @@ func (mp *MutationPipeline) InsertTokenizerIndexes(ctx context.Context, pipeline for _, token := range tokens.([]string) { key := x.IndexKey(pipeline.attr, token) - pk, _ := x.Parse([]byte(key)) - fmt.Println("TOKENS", key, pk) val, ok := indexGenInThread[string(key)] if !ok { val = &pb.PostingList{} @@ -217,8 +209,6 @@ func (mp *MutationPipeline) InsertTokenizerIndexes(ctx context.Context, pipeline } for key, value := range indexGenInThread { - pk, _ := x.Parse([]byte(key)) - fmt.Println("LOCAL MAP", pk, value) indexesGenInMutation.Update(key, func(val *MutableLayer, ok bool) *MutableLayer { if !ok { val = newMutableLayer() @@ -261,10 +251,7 @@ func (mp *MutationPipeline) InsertTokenizerIndexes(ctx context.Context, pipeline mp.txn.cache.deltas.indexMap[pipeline.attr] = indexGenInTxn } - fmt.Println("INSERTING INDEX", pipeline.attr, *postings) updateFn := func(key string, value *MutableLayer) { - pk, _ := x.Parse([]byte(key)) - fmt.Println("UPDATE INDEX", pk, value) indexGenInTxn.Update(key, func(val *pb.PostingList, ok bool) *pb.PostingList { if !ok { val = &pb.PostingList{} diff --git a/posting/list.go b/posting/list.go index cbb2542d0a6..39b8cab742c 100644 --- a/posting/list.go +++ b/posting/list.go @@ -335,7 +335,6 @@ func (mm *MutableLayer) insertCommittedPostings(pl *pb.PostingList) { // We insert old postings in reverse order. So we only need to read the first update to an UID. if _, ok := mm.committedUids[mpost.Uid]; !ok { mm.committedUids[mpost.Uid] = mpost - fmt.Println("Buidlding committed uids", mm.committedUids, mpost) } } } @@ -1104,9 +1103,6 @@ func (l *List) setMutation(startTs uint64, data []byte) { } func (l *List) setMutationWithPosting(startTs uint64, pl *pb.PostingList) { - // pk, _ := x.Parse(l.key) - // fmt.Println("Setting mutation for ", l.key, pk, pl) - l.Lock() if l.mutationMap == nil { l.mutationMap = newMutableLayer() @@ -1526,17 +1522,10 @@ func (l *List) Rollup(alloc *z.Allocator, readTs uint64) ([]*bpb.KV, error) { return bytes.Compare(kvs[i].Key, kvs[j].Key) <= 0 }) - PrintRollup(out.plist, out.parts, l.key, kv.Version) x.VerifyPostingSplits(kvs, out.plist, out.parts, l.key) return kvs, nil } -func PrintRollup(plist *pb.PostingList, parts map[uint64]*pb.PostingList, baseKey []byte, ts uint64) { - k, _ := x.Parse(baseKey) - uids := codec.Decode(plist.Pack, 0) - fmt.Printf("[TXNLOG] DOING ROLLUP for key: %+v at timestamp: %v, uids: %+v\n", k, ts, uids) -} - // ToBackupPostingList uses rollup to generate a single list with no splits. // It's used during backup so that each backed up posting list is stored in a single key. func (l *List) ToBackupPostingList( diff --git a/posting/lists.go b/posting/lists.go index 239d13786d9..9a50361f984 100644 --- a/posting/lists.go +++ b/posting/lists.go @@ -128,8 +128,6 @@ func (d *Deltas) Get(key string) (*pb.PostingList, bool) { } } - // fmt.Println("GETTING KEY FROM DELTAS", pk, "res", res, "val", val, "d.deltas", d.deltas, "d.indexMap[pk.Attr]", d.indexMap[pk.Attr], "ok", ok) - return res, len(res.Postings) > 0 } @@ -396,20 +394,12 @@ func (lc *LocalCache) getInternal(key []byte, readFromDisk, readUids bool) (*Lis lc.RLock() defer lc.RUnlock() if lc.plists == nil { - l, err := getNew(key, pstore, lc.startTs, readUids) - if err != nil { - return nil, err - } - pk, _ := x.Parse(key) - fmt.Println("READING NEW PLIST", pk, l.Print()) - return l, nil + return getNew(key, pstore, lc.startTs, readUids) } if l, ok := lc.plists[skey]; ok { if delta, ok := lc.deltas.Get(skey); ok && delta != nil { l.setMutationWithPosting(lc.startTs, delta) } - pk, _ := x.Parse(key) - fmt.Println("READING PLIST", pk, l.Print()) return l, nil } return nil, nil @@ -442,8 +432,6 @@ func (lc *LocalCache) getInternal(key []byte, readFromDisk, readUids bool) (*Lis } lc.RUnlock() - pk, _ := x.Parse(key) - fmt.Println("READING ", pk, pl.Print()) return lc.SetIfAbsent(skey, pl), nil } @@ -480,16 +468,11 @@ func (lc *LocalCache) readPostingListAt(key []byte) (*pb.PostingList, error) { // given key. This is used for retrieving the value of a scalar predicats. func (lc *LocalCache) GetSinglePosting(key []byte) (*pb.PostingList, error) { // This would return an error if there is some data in the local cache, but we couldn't read it. - - pk, _ := x.Parse(key) - fmt.Println("READING SINGLE ", pk) - getListFromLocalCache := func() (*pb.PostingList, error) { lc.RLock() if delta, ok := lc.deltas.Get(string(key)); ok && delta != nil { lc.RUnlock() - fmt.Println("READING SINGLE FROM DELTA", pk, delta) return delta, nil } @@ -497,9 +480,7 @@ func (lc *LocalCache) GetSinglePosting(key []byte) (*pb.PostingList, error) { lc.RUnlock() if l != nil { - res, err := l.StaticValue(lc.startTs) - fmt.Println("READING SINGLE FROM PLISTS", pk, res, err, l.Print()) - return res, err + return l.StaticValue(lc.startTs) } return nil, nil @@ -517,15 +498,10 @@ func (lc *LocalCache) GetSinglePosting(key []byte) (*pb.PostingList, error) { } pl, err := getPostings() - - fmt.Println("READING SINGLE ", pk, "pl:", pl) - if err == badger.ErrKeyNotFound { - fmt.Println("READING ", pk, nil) return nil, nil } if err != nil { - fmt.Println("READING ", pk, err) return nil, err } @@ -538,7 +514,6 @@ func (lc *LocalCache) GetSinglePosting(key []byte) (*pb.PostingList, error) { } } pl.Postings = pl.Postings[:idx] - // fmt.Println("READING ", pk, pl) return pl, nil } diff --git a/posting/mvcc.go b/posting/mvcc.go index c917671631e..e3bb2900bd1 100644 --- a/posting/mvcc.go +++ b/posting/mvcc.go @@ -9,7 +9,6 @@ import ( "bytes" "context" "encoding/hex" - "fmt" "math" "strconv" "sync" @@ -308,8 +307,6 @@ func (txn *Txn) CommitToDisk(writer *TxnWriter, commitTs uint64) error { if len(pl.Postings) == 0 { continue } - pk, _ := x.Parse([]byte(key)) - fmt.Println("COMMITTING", pk, pl) if ts := cache.maxVersions[key]; ts >= commitTs { // Skip write because we already have a write at a higher ts. // Logging here can cause a lot of output when doing Raft log replay. So, let's From db61e4b6ccea587827bbf37e00472c7dd2e04404 Mon Sep 17 00:00:00 2001 From: Matthew McNeely Date: Fri, 1 May 2026 13:11:54 -0400 Subject: [PATCH 03/17] chore(posting): remove dead commented-out code Two large dead-code blocks left over from the original WIP: - posting/index.go: ~175 lines of an alternate InsertTokenizerIndexes implementation, fully commented out. The live implementation directly above it supersedes it; keeping the commented variant just made the file harder to follow. Also drop the scattered "//fmt.Println(...)" leftovers next to live code. - worker/draft_test.go: BenchmarkProcessListIndex was added entirely commented out and references methods (DefaultPipeline, ProcessListWithoutIndex, ProcessListIndex) that don't exist on MutationPipeline. If we want a benchmark for the pipeline, we should write one against the real API. No behavior change. Co-Authored-By: Claude Opus 4.7 (1M context) --- posting/index.go | 183 ------------------------------------------- worker/draft_test.go | 101 ------------------------ 2 files changed, 284 deletions(-) diff --git a/posting/index.go b/posting/index.go index 0dfe9bff30e..50349ef41b8 100644 --- a/posting/index.go +++ b/posting/index.go @@ -284,183 +284,6 @@ func (mp *MutationPipeline) InsertTokenizerIndexes(ctx context.Context, pipeline return nil } -// func (mp *MutationPipeline) InsertTokenizerIndexes(ctx context.Context, pipeline *PredicatePipeline, postings *map[uint64]*pb.PostingList, info predicateInfo) error { -// startTime := time.Now() -// defer func() { -// fmt.Println("Inserting tokenizer indexes for predicate", pipeline.attr, "took", time.Since(startTime)) -// }() - -// tokenizers := schema.State().Tokenizer(ctx, pipeline.attr) -// if len(tokenizers) == 0 { -// return nil -// } - -// values := make(map[string]*pb.PostingList, len(tokenizers)*len(*postings)) -// valPost := make(map[string]*pb.Posting) - -// indexEdge1 := &pb.DirectedEdge{ -// Attr: pipeline.attr, -// } - -// for uid, postingList := range *postings { -// fmt.Println("POSTING", uid, postingList) -// for _, posting := range postingList.Postings { -// key := fmt.Sprintf("%s,%s", posting.LangTag, posting.Value) -// valPl, ok := values[key] -// if !ok { -// valPl = &pb.PostingList{} -// } - -// indexEdge1.Op = GetPostingOp(posting.Op) -// indexEdge1.ValueId = uid - -// mpost := makePostingFromEdge(mp.txn.StartTs, indexEdge1) -// valPl.Postings = append(valPl.Postings, mpost) -// values[key] = valPl - -// newPosting := new(pb.Posting) -// newPosting.ValType = posting.ValType -// newPosting.Value = posting.Value -// newPosting.LangTag = posting.LangTag -// valPost[key] = newPosting -// } -// } - -// keysCreated := make([]string, 0, len(values)) -// for i := range values { -// keysCreated = append(keysCreated, i) -// } - -// //fmt.Println("START") - -// f := func(numGo int) *types.LockedShardedMap[string, *MutableLayer] { -// wg := &sync.WaitGroup{} - -// globalMap := types.NewLockedShardedMap[string, *MutableLayer]() -// process := func(start int) { -// factorySpecs, err := schema.State().FactoryCreateSpec(ctx, pipeline.attr) -// if err != nil { -// pipeline.errCh <- err -// return -// } - -// defer wg.Done() -// localMap := make(map[string]*pb.PostingList, len(values)/numGo) -// for i := start; i < len(values); i += numGo { -// key := keysCreated[i] -// valPl := values[key] -// if len(valPl.Postings) == 0 { -// continue -// } - -// posting := valPost[key] -// // Build info per iteration without indexEdge. -// info := &indexMutationInfo{ -// tokenizers: tokenizers, -// factorySpecs: factorySpecs, -// op: pb.DirectedEdge_SET, -// val: types.Val{ -// Tid: types.TypeID(posting.ValType), -// Value: posting.Value, -// }, -// } - -// info.edge = &pb.DirectedEdge{ -// Attr: pipeline.attr, -// Op: pb.DirectedEdge_SET, -// Lang: string(posting.LangTag), -// Value: posting.Value, -// } - -// tokens, erri := indexTokens(ctx, info) -// if erri != nil { -// fmt.Println("ERRORRRING", erri) -// x.Panic(erri) -// } - -// for _, token := range tokens { -// key := x.IndexKey(pipeline.attr, token) -// pk, _ := x.Parse([]byte(key)) -// fmt.Println("TOKENS", key, i, numGo, pk) -// val, ok := localMap[string(key)] -// if !ok { -// val = &pb.PostingList{} -// } -// val.Postings = append(val.Postings, valPl.Postings...) -// localMap[string(key)] = val -// } -// } - -// for key, value := range localMap { -// pk, _ := x.Parse([]byte(key)) -// fmt.Println("LOCAL MAP", pk, numGo, value) -// globalMap.Update(key, func(val *MutableLayer, ok bool) *MutableLayer { -// if !ok { -// val = newMutableLayer() -// val.currentEntries = &pb.PostingList{} -// } -// for _, posting := range value.Postings { -// val.insertPosting(posting, false) -// } -// return val -// }) -// } -// } - -// for i := range numGo { -// wg.Add(1) -// go process(i) -// } - -// wg.Wait() - -// return globalMap -// } - -// globalMapI := f(1) - -// mp.txn.cache.Lock() -// defer mp.txn.cache.Unlock() - -// globalMap := mp.txn.cache.deltas.GetIndexMapForPredicate(pipeline.attr) -// if globalMap == nil { -// globalMap = types.NewLockedShardedMap[string, *pb.PostingList]() -// mp.txn.cache.deltas.indexMap[pipeline.attr] = globalMap -// } - -// updateFn := func(key string, value *MutableLayer) { -// globalMap.Update(key, func(val *pb.PostingList, ok bool) *pb.PostingList { -// if !ok { -// val = &pb.PostingList{} -// } -// val.Postings = append(val.Postings, value.currentEntries.Postings...) -// return val -// }) -// } - -// if info.hasUpsert { -// err := globalMapI.Iterate(func(key string, value *MutableLayer) error { -// updateFn(key, value) -// mp.txn.addConflictKey(farm.Fingerprint64([]byte(key))) -// return nil -// }) -// if err != nil { -// return err -// } -// } else { -// err := globalMapI.Iterate(func(key string, value *MutableLayer) error { -// updateFn(key, value) -// mp.txn.addConflictKeyWithUid([]byte(key), value.currentEntries, info.hasUpsert, info.noConflict) -// return nil -// }) -// if err != nil { -// return err -// } -// } - -// return nil -// } - type predicateInfo struct { isList bool index bool @@ -752,8 +575,6 @@ func (mp *MutationPipeline) ProcessCount(ctx context.Context, pipeline *Predicat continue } - //fmt.Println("COUNT STATS", uid, prevCount, newCount, postingList, list.Print()) - edge.ValueId = uid edge.Op = pb.DirectedEdge_DEL if prevCount > 0 { @@ -766,7 +587,6 @@ func (mp *MutationPipeline) ProcessCount(ctx context.Context, pipeline *Predicat } for c, pl := range countMap { - //fmt.Println("COUNT", c, pl) ck := x.CountKey(pipeline.attr, uint32(c), isReverseEdge) if newPl, err := mp.txn.AddDelta(string(ck), pl, true, true); err != nil { return err @@ -788,7 +608,6 @@ func (mp *MutationPipeline) ProcessSingle(ctx context.Context, pipeline *Predica var oldVal *pb.Posting for edge := range pipeline.edges { - // fmt.Println("SINGLE EDGE", edge) if edge.Op != pb.DirectedEdge_DEL && !schemaExists { return errors.Errorf("runMutation: Unable to find schema for %s", edge.Attr) } @@ -886,7 +705,6 @@ func (mp *MutationPipeline) ProcessSingle(ctx context.Context, pipeline *Predica baseKey := string(dataKey[:len(dataKey)-8]) // Avoid repeated conversion for uid, pl := range postings { - //fmt.Println("ADDING DELTA", uid, pipeline.attr, pl) binary.BigEndian.PutUint64(dataKey[len(dataKey)-8:], uid) key := baseKey + string(dataKey[len(dataKey)-8:]) @@ -1102,7 +920,6 @@ func (mp *MutationPipeline) Process(ctx context.Context, edges []*pb.DirectedEdg numWg := 0 eg, egCtx := errgroup.WithContext(ctx) for _, edge := range edges { - //fmt.Println("PROCESSING EDGE", edge) if edge.Op == pb.DirectedEdge_DEL && string(edge.Value) == x.Star { l, err := mp.txn.Get(x.DataKey(edge.Attr, edge.Entity)) if err != nil { diff --git a/worker/draft_test.go b/worker/draft_test.go index 057c975c0a4..945e1ce7971 100644 --- a/worker/draft_test.go +++ b/worker/draft_test.go @@ -38,107 +38,6 @@ func getEntryForCommit(index, startTs, commitTs uint64) raftpb.Entry { return raftpb.Entry{Index: index, Term: 1, Type: raftpb.EntryNormal, Data: data} } -// func BenchmarkProcessListIndex(b *testing.B) { -// dir, err := os.MkdirTemp("", "storetest_") -// x.Check(err) -// defer os.RemoveAll(dir) - -// opt := badger.DefaultOptions(dir) -// ps, err := badger.OpenManaged(opt) -// x.Check(err) -// pstore = ps -// // Not using posting list cache -// posting.Init(ps, 0, false) -// Init(ps) -// err = schema.ParseBytes([]byte("testAttr: [string] @index(exact) ."), 1) -// require.NoError(b, err) - -// ctx := context.Background() -// pipeline := &PredicatePipeline{ -// attr: "0-testAttr", -// edges: make(chan *pb.DirectedEdge, 1000), -// wg: &sync.WaitGroup{}, -// errCh: make(chan error, 1), -// } - -// txn := posting.Oracle().RegisterStartTs(5) -// mp := &MutationPipeline{txn: txn} - -// // Generate 1000 edges -// populatePipeline := func() { -// pipeline = &PredicatePipeline{ -// attr: "0-testAttr", -// edges: make(chan *pb.DirectedEdge, 1000), -// wg: &sync.WaitGroup{}, -// errCh: make(chan error, 1), -// } - -// txn = posting.Oracle().RegisterStartTs(5) -// mp = &MutationPipeline{txn: txn} - -// for i := 0; i < 1000; i++ { -// edge := &pb.DirectedEdge{ -// Entity: uint64(i + 1), -// Attr: "0-testAttr", -// Value: []byte(fmt.Sprintf("value%d", rand.Intn(1000))), -// ValueType: pb.Posting_STRING, -// Op: pb.DirectedEdge_SET, -// } -// pipeline.edges <- edge -// } -// } - -// b.ResetTimer() - -// b.Run("Baseline", func(b *testing.B) { -// for i := 0; i < b.N; i++ { -// populatePipeline() -// } -// }) - -// b.Run("DefaultPipeline", func(b *testing.B) { -// for i := 0; i < b.N; i++ { -// populatePipeline() -// var wg sync.WaitGroup -// wg.Add(1) -// go func() { -// mp.DefaultPipeline(ctx, pipeline) -// wg.Done() -// }() -// close(pipeline.edges) -// wg.Wait() -// } -// }) - -// b.Run("ProcessListWithoutIndex", func(b *testing.B) { -// for i := 0; i < b.N; i++ { -// populatePipeline() -// var wg sync.WaitGroup -// wg.Add(1) -// go func() { -// mp.ProcessListWithoutIndex(ctx, pipeline) -// wg.Done() -// }() -// close(pipeline.edges) -// wg.Wait() -// } -// }) - -// b.Run("ProcessListIndex", func(b *testing.B) { -// for i := 0; i < b.N; i++ { -// populatePipeline() -// var wg sync.WaitGroup -// wg.Add(1) -// go func() { -// mp.ProcessListIndex(ctx, pipeline) -// wg.Done() -// }() -// close(pipeline.edges) -// wg.Wait() -// } -// }) -// } - func TestCalculateSnapshot(t *testing.T) { dir := t.TempDir() ds := raftwal.Init(dir) From c23489eef76384be32c37f372e1f286a3c2de1bd Mon Sep 17 00:00:00 2001 From: Matthew McNeely Date: Fri, 1 May 2026 13:14:24 -0400 Subject: [PATCH 04/17] feat(mutations): gate per-predicate pipeline behind a feature flag Replace the hardcoded \`featureFlag := true\` in applyMutations with a real superflag knob, defaulted off: - Add WorkerOptions.MutationsUsePipeline (bool) in x/config.go. - Extend the feature-flags superflag with mutations-use-pipeline=false and wire alpha to populate WorkerConfig.MutationsUsePipeline from it. - worker/draft.go applyMutations now branches on x.WorkerConfig.MutationsUsePipeline; default false routes mutations through the legacy path, preserving current behavior. Tests can opt into the new pipeline by setting x.WorkerConfig.MutationsUsePipeline = true. CLI usage: dgraph alpha --feature-flags="mutations-use-pipeline=true" Co-Authored-By: Claude Opus 4.7 (1M context) --- dgraph/cmd/alpha/run.go | 1 + worker/draft.go | 3 +-- worker/server_state.go | 3 ++- x/config.go | 4 ++++ 4 files changed, 8 insertions(+), 3 deletions(-) diff --git a/dgraph/cmd/alpha/run.go b/dgraph/cmd/alpha/run.go index 901d7b0a808..9f827403ebb 100644 --- a/dgraph/cmd/alpha/run.go +++ b/dgraph/cmd/alpha/run.go @@ -796,6 +796,7 @@ func run() { x.Config.NormalizeCompatibilityMode = featureFlagsConf.GetString("normalize-compatibility-mode") enableDetailedMetrics := featureFlagsConf.GetBool("enable-detailed-metrics") x.WorkerConfig.SlowQueryLogThreshold = featureFlagsConf.GetDuration("log-slow-query-threshold") + x.WorkerConfig.MutationsUsePipeline = featureFlagsConf.GetBool("mutations-use-pipeline") x.PrintVersion() glog.Infof("x.Config: %+v", x.Config) diff --git a/worker/draft.go b/worker/draft.go index d4e2d9fdee6..ad43e2f55f0 100644 --- a/worker/draft.go +++ b/worker/draft.go @@ -526,8 +526,7 @@ func (n *node) applyMutations(ctx context.Context, proposal *pb.Proposal) (rerr // Discard the posting lists from cache to release memory at the end. defer txn.Update() - featureFlag := true - if featureFlag { + if x.WorkerConfig.MutationsUsePipeline { mp := posting.NewMutationPipeline(txn) return mp.Process(ctx, m.Edges) } diff --git a/worker/server_state.go b/worker/server_state.go index 0591ccb4b5b..6824b0adc26 100644 --- a/worker/server_state.go +++ b/worker/server_state.go @@ -42,7 +42,8 @@ const ( GraphQLDefaults = `introspection=true; debug=false; extensions=true; poll-interval=1s; ` + `lambda-url=;` CacheDefaults = `size-mb=4096; percentage=40,40,20; remove-on-update=false` - FeatureFlagsDefaults = `normalize-compatibility-mode=; enable-detailed-metrics=false; log-slow-query-threshold=0` + FeatureFlagsDefaults = `normalize-compatibility-mode=; enable-detailed-metrics=false; ` + + `log-slow-query-threshold=0; mutations-use-pipeline=false` ) // ServerState holds the state of the Dgraph server. diff --git a/x/config.go b/x/config.go index 37081a3df5a..7550172037b 100644 --- a/x/config.go +++ b/x/config.go @@ -138,6 +138,10 @@ type WorkerOptions struct { HardSync bool // Audit contains the audit flags that enables the audit. Audit bool + // MutationsUsePipeline enables the per-predicate mutation pipeline in + // applyMutations. When false (default), mutations follow the legacy + // serial path. The flag is plumbed via the "feature-flags" superflag. + MutationsUsePipeline bool } // WorkerConfig stores the global instance of the worker package's options. From 30adee46727b4ea5cd44cc05fd6ec08ce1cfdfc6 Mon Sep 17 00:00:00 2001 From: Matthew McNeely Date: Fri, 1 May 2026 13:31:23 -0400 Subject: [PATCH 05/17] test(worker): document why TestCount is skipped MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit TestCount is t.Skip()'d on the branch, but the reason wasn't recorded. Investigation: the test launches concurrent transactions sharing entity uids and bypasses the Oracle's conflict-checking commit path — it just calls CommitToDisk() directly with disjoint commit timestamps. Both the legacy AddMutationWithIndex path and the new mutation pipeline fail it identically: with two threads adding edges to the same subject's [uid] @count predicate, neither path can serialize @count updates without real txn conflicts, so the count index ends up inconsistent and many subjects are missing from count(N). This is expected without conflict checking — the unit harness can't exercise the safety the Oracle provides. Re-enable when we wire either: (a) Oracle.WaitForTs/IsAborted into the harness, or (b) this test through worker.applyMutations() (which does invoke the Oracle conflict path). Single-thread TestCount passes, so the per-predicate pipeline's own count logic is correct in the absence of contention. The existing TestStringIndexWithLang covers the multithreaded happy path with disjoint uids. Co-Authored-By: Claude Opus 4.7 (1M context) --- worker/sort_test.go | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/worker/sort_test.go b/worker/sort_test.go index 1dfee6e460e..2ec4826c0ea 100644 --- a/worker/sort_test.go +++ b/worker/sort_test.go @@ -301,7 +301,8 @@ func TestStringIndexWithLang(t *testing.T) { } func TestCount(t *testing.T) { - t.Skip() + t.Skip("Inherently racy: bypasses the Oracle conflict-checking commit path. " + + "Legacy and new pipeline both fail. Re-enable when the harness uses real txn conflicts.") // Setup temporary directory for Badger DB dir, err := os.MkdirTemp("", "storetest_") require.NoError(t, err) From 365140fb84c54d15f53a75618f5ff83092bd42e8 Mon Sep 17 00:00:00 2001 From: Matthew McNeely Date: Fri, 1 May 2026 14:18:43 -0400 Subject: [PATCH 06/17] fix(pipeline): scalar Del-of-old-value must not wipe new Set in ProcessCount MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Bug: scalar @count writes were nondeterministically losing data under concurrent transactions. Roughly half the deltas committed by the new mutation pipeline contained only a [DeleteAll] posting and no Set, so reads at maxTs returned an empty value list. Root cause: in ProcessSingle, handleOldDeleteForSingle appends a synthetic Del-of-old-value to postings[uid] alongside the user's Set, so InsertTokenizerIndexes / ProcessReverse / count diffing can see the prior value. ProcessCount then iterates the postings and calls list.updateMutationLayer(post, singleUidUpdate=true, ...) on each. For non-Lang scalar predicates fingerprintEdge returns math.MaxUint64, so the synthetic Del and the user Set both have Uid == math.MaxUint64. The first iteration (Set new) leaves mutationMap.currentEntries = [DeleteAll, Set new]; the second iteration (Del old) finds the Set we just inserted via findPosting and applies updateMutationLayer in singleUidUpdate mode, which unconditionally rewrites currentEntries to [DeleteAll] (the Del branch never appends mpost) — wiping the new value. Fix: in ProcessCount, when iterating a !isListEdge predicate's postings, if the list contains a Set/Ovr posting, treat any Del as synthetic and skip it for the data-list update. Standalone user Dels (no accompanying Set) are still applied. Index/reverse/count diffing already happen before ProcessCount runs and aren't affected. Repro: TestPipelineCountIndexConcurrent in worker/sort_test.go is a new conflict-aware in-process harness that mirrors the systest TestCountIndexConcurrentSetDelScalarPredicate. It runs 200 contending transactions setting <0x1> "name" against a "string @index(exact) @count" schema with a fakeOracle that implements the same hasConflict algorithm as dgraph/cmd/zero/oracle.go. Pre-fix the test fails roughly 50% of runs with an empty data list and the wrong count buckets; post-fix it is stable across 20+ -count iterations and under -race. Existing tests (TestScalarPredicateIntCount, *RevCount, *Count, TestSingleUidReplacement, TestDeleteSetWithVarEdgeCorruptsData, TestStringIndexWithLang, TestMultipleTxnListCount, TestGetScalarList, TestDatetime) all pass. Co-Authored-By: Claude Opus 4.7 (1M context) --- posting/index.go | 24 ++++++ worker/sort_test.go | 176 ++++++++++++++++++++++++++++++++++++++++++++ 2 files changed, 200 insertions(+) diff --git a/posting/index.go b/posting/index.go index 50349ef41b8..e78b112ba1a 100644 --- a/posting/index.go +++ b/posting/index.go @@ -540,7 +540,31 @@ func (mp *MutationPipeline) ProcessCount(ctx context.Context, pipeline *Predicat list.Lock() prevCount := list.GetLength(mp.txn.StartTs) + // For scalar (non-list) predicates, handleOldDeleteForSingle may have + // appended a synthetic Del-of-old-value alongside the user's Set, so + // that InsertTokenizerIndexes / ProcessReverse / count diffing can see + // the prior value. The synthetic Del must NOT be applied to the data + // list: scalar value postings all share Uid == math.MaxUint64 + // (fingerprintEdge returns MaxUint64 for non-Lang scalars), and + // updateMutationLayer in singleUidUpdate mode would overwrite the + // just-inserted Set with [DeleteAll] and drop the new value entirely. + // A user-initiated Del (no accompanying Set) must still be applied. + skipSyntheticDel := false + if !isListEdge { + hasSet := false + for _, post := range postingList.Postings { + if post.Op == Set || post.Op == Ovr { + hasSet = true + break + } + } + skipSyntheticDel = hasSet + } + for _, post := range postingList.Postings { + if skipSyntheticDel && post.Op == Del { + continue + } found, _, _ := list.findPosting(post.StartTs, post.Uid) if found { if post.Op == Set && isListEdge { diff --git a/worker/sort_test.go b/worker/sort_test.go index 2ec4826c0ea..2ae82190440 100644 --- a/worker/sort_test.go +++ b/worker/sort_test.go @@ -11,11 +11,14 @@ import ( "math" "math/rand" "os" + "strconv" "sync" + "sync/atomic" "testing" "github.com/dgraph-io/badger/v4" bpb "github.com/dgraph-io/badger/v4/pb" + "github.com/dgraph-io/dgo/v250/protos/api" "github.com/dgraph-io/dgraph/v25/posting" "github.com/dgraph-io/dgraph/v25/protos/pb" "github.com/dgraph-io/dgraph/v25/schema" @@ -409,6 +412,179 @@ func TestCount(t *testing.T) { require.Equal(t, subjects, len(uids.Uids)) } +// fakeOracle is an in-memory stand-in for the zero Oracle. It hands out +// monotonically increasing timestamps and rejects commits whose conflict +// keys overlap a higher commitTs — same algorithm as +// dgraph/cmd/zero/oracle.go's hasConflict. +type fakeOracle struct { + mu sync.Mutex + nextTs uint64 + keyCommit map[uint64]uint64 // conflict-key fingerprint -> commitTs + committed atomic.Int64 + aborted atomic.Int64 +} + +func newFakeOracle(initialTs uint64) *fakeOracle { + return &fakeOracle{nextTs: initialTs, keyCommit: map[uint64]uint64{}} +} + +func (o *fakeOracle) reserveStartTs() uint64 { + o.mu.Lock() + defer o.mu.Unlock() + o.nextTs++ + return o.nextTs +} + +// tryCommit mirrors zero/oracle.go: for each conflict key, if a later +// commitTs already exists, abort. Else stamp all keys with a fresh +// commitTs and return it. +func (o *fakeOracle) tryCommit(startTs uint64, conflictKeys []uint64) (uint64, bool) { + o.mu.Lock() + defer o.mu.Unlock() + for _, k := range conflictKeys { + if last, ok := o.keyCommit[k]; ok && last > startTs { + o.aborted.Add(1) + return 0, false + } + } + o.nextTs++ + commitTs := o.nextTs + for _, k := range conflictKeys { + o.keyCommit[k] = commitTs + } + o.committed.Add(1) + return commitTs, true +} + +// runPipelineTxn drives a single mutation through the new pipeline with +// real conflict-aware commit semantics. Returns (committed, error). +func runPipelineTxn(t *testing.T, ps *badger.DB, oracle *fakeOracle, + edges []*pb.DirectedEdge) bool { + t.Helper() + startTs := oracle.reserveStartTs() + txn := posting.Oracle().RegisterStartTs(startTs) + + if err := newRunMutations(context.Background(), edges, txn); err != nil { + t.Fatalf("pipeline failed at startTs=%d: %v", startTs, err) + } + + // FillContext bridges plists -> deltas (via Update) and emits the + // txn's conflict keys as base-36 strings on ctx.Keys. + ctxApi := &api.TxnContext{} + txn.FillContext(ctxApi, 1, false) + + keys := make([]uint64, 0, len(ctxApi.Keys)) + for _, k := range ctxApi.Keys { + ki, err := strconv.ParseUint(k, 36, 64) + require.NoError(t, err) + keys = append(keys, ki) + } + + commitTs, ok := oracle.tryCommit(startTs, keys) + if !ok { + return false + } + writer := posting.NewTxnWriter(ps) + require.NoError(t, txn.CommitToDisk(writer, commitTs)) + require.NoError(t, writer.Flush()) + txn.UpdateCachedKeys(commitTs) + return true +} + +// TestPipelineCountIndexConcurrent mirrors the systest's +// TestCountIndexConcurrentSetDelScalarPredicate at unit-test scope: many +// concurrent transactions setting <0x1> "name" against a +// scalar string predicate with @index(exact) @count, with real +// conflict-checking commit semantics. After everything settles, the data +// list for 0x1 should hold exactly one value, the count(1) bucket should +// reference exactly 0x1, and no other count bucket should reference 0x1. +func TestPipelineCountIndexConcurrent(t *testing.T) { + dir, err := os.MkdirTemp("", "storetest_") + require.NoError(t, err) + defer os.RemoveAll(dir) + + ps, err := badger.OpenManaged(badger.DefaultOptions(dir)) + require.NoError(t, err) + defer ps.Close() + posting.Init(ps, 0, false) + Init(ps) + posting.Oracle().ResetTxns() + + require.NoError(t, schema.ParseBytes( + []byte(`name: string @index(exact) @count .`), 1)) + + pred := x.AttrInRootNamespace("name") + const target uint64 = 1 + + oracle := newFakeOracle(10) + + const ( + numRoutines = 10 + txnsPerRoute = 20 + ) + + var wg sync.WaitGroup + wg.Add(numRoutines) + for r := 0; r < numRoutines; r++ { + go func(seed int) { + defer wg.Done() + rnd := rand.New(rand.NewSource(int64(seed))) + for i := 0; i < txnsPerRoute; i++ { + value := []byte(fmt.Sprintf("name%d", rnd.Intn(10000))) + // Retry on conflict — same as a client doing dg.NewTxn().Mutate(). + // Each attempt uses a fresh edge: makePostingFromEdge mutates + // edge.ValueId during processing, and reusing the object across + // attempts would make ValidateAndConvert see it as a uid edge. + // Real production gets a freshly-deserialized edge per Raft apply. + for attempt := 0; attempt < 100; attempt++ { + edge := &pb.DirectedEdge{ + Entity: target, + Attr: pred, + Value: value, + ValueType: pb.Posting_STRING, + Op: pb.DirectedEdge_SET, + } + if runPipelineTxn(t, ps, oracle, []*pb.DirectedEdge{edge}) { + break + } + } + } + }(r) + } + wg.Wait() + + t.Logf("committed=%d aborted=%d", oracle.committed.Load(), oracle.aborted.Load()) + + // Verify final state: exactly one value on 0x1, that uid in count(1) only. + readTxn := posting.Oracle().RegisterStartTs(math.MaxUint64) + + dataKey := x.DataKey(pred, target) + dpl, err := readTxn.Get(dataKey) + require.NoError(t, err) + // Scalar string predicate: AllValues returns the live posting list values + // (one entry for a non-list scalar with a current value). + vals, err := dpl.AllValues(math.MaxUint64) + require.NoError(t, err) + require.Equal(t, 1, len(vals), + "scalar predicate should retain exactly one value, got %v", vals) + + for c := 0; c <= 5; c++ { + ck := x.CountKey(pred, uint32(c), false) + cpl, err := readTxn.Get(ck) + require.NoError(t, err) + cuids, err := cpl.Uids(posting.ListOptions{ReadTs: math.MaxUint64}) + require.NoError(t, err) + switch c { + case 1: + require.Equal(t, []uint64{target}, cuids.Uids, + "count(1) bucket must contain exactly the target uid") + default: + require.NotContains(t, cuids.Uids, target, + "count(%d) bucket must not contain the target uid", c) + } + } +} + func TestDeleteSetWithVarEdgeCorruptsData(t *testing.T) { // Setup temporary directory for Badger DB dir, err := os.MkdirTemp("", "storetest_") From c9349c7e025c1651427d1495e17cb487bf7c0446 Mon Sep 17 00:00:00 2001 From: Matthew McNeely Date: Fri, 1 May 2026 15:45:36 -0400 Subject: [PATCH 07/17] fix(pipeline): InsertTokenizerIndexes deadlocks on uids >= 2^63 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit When loading via dgraph live (or any mutation source whose uids span the full uint64 range, including xidmap-assigned uids), the per-predicate pipeline hung indefinitely on the very first batch with zero forward progress. A goroutine dump showed the dispatcher goroutine wedged on \`chan send (nil chan)\` at the line: chMap[int(uid)%numGo] <- uid uid is uint64. Casting directly to int produces a negative value for uid >= 2^63, so int(uid)%10 can be in [-9, -1]. chMap[-3] returns the zero value for a chan uint64, which is a nil channel; sending on a nil channel blocks forever. The 10 worker goroutines (also created here) were idle on \`for uid := range uids\` since no uids ever reached them, so the parent \`wg.Wait()\` and the surrounding errgroup never returned. applyMutations therefore never released the txn, the alpha's old-txn abort loop kept retrying every minute, and live-load showed "Txns: 0 N-Quads: 0" indefinitely. Fix: hash unsigned, then cast: \`chMap[int(uid%uint64(numGo))]\`. Verified end-to-end with the live loader against the 1million.rdf.gz benchmark dataset (1,041,684 n-quads, schema mixes [uid] @reverse @count, [uid] @count, datetime @index(year), string @index(...) @lang, geo @index(geo), string @index(exact) @upsert): legacy : 13.85s / 14.74s (avg ~14.3s, ~77k n-quads/s) pipeline : 9.65s / 9.36s (avg ~9.5s, ~116k n-quads/s) That is ~1.50x faster on a realistic multi-predicate, multi-index workload — i.e. the case the per-predicate runner pipeline is built for. Also adds worker/pipeline_bench_test.go: in-process Go benchmarks comparing legacy runMutation vs newRunMutations across a matrix of (predicates, edges-per-predicate, indexed/non-indexed) shapes. They show the pipeline loses ~2x on tiny mutations (1-10 edges) and wins 1.2x-1.55x on bulk (10 preds x 100+ edges, indexed or not), which is why the feature flag stays default-off and the live-loader speedup above is the right place to evaluate this work. Co-Authored-By: Claude Opus 4.7 (1M context) --- posting/index.go | 6 +- worker/pipeline_bench_test.go | 210 ++++++++++++++++++++++++++++++++++ 2 files changed, 215 insertions(+), 1 deletion(-) create mode 100644 worker/pipeline_bench_test.go diff --git a/posting/index.go b/posting/index.go index e78b112ba1a..57d996cba5a 100644 --- a/posting/index.go +++ b/posting/index.go @@ -233,7 +233,11 @@ func (mp *MutationPipeline) InsertTokenizerIndexes(ctx context.Context, pipeline } for uid := range *postings { - chMap[int(uid)%numGo] <- uid + // uid is uint64; converting directly to int can produce a negative + // value for uid >= 2^63, which would index outside chMap and resolve + // to a nil channel (deadlocks the dispatcher). Hash unsigned, then + // cast. + chMap[int(uid%uint64(numGo))] <- uid } for i := 0; i < numGo; i++ { diff --git a/worker/pipeline_bench_test.go b/worker/pipeline_bench_test.go new file mode 100644 index 00000000000..e2d68032d47 --- /dev/null +++ b/worker/pipeline_bench_test.go @@ -0,0 +1,210 @@ +/* + * SPDX-FileCopyrightText: © 2017-2025 Istari Digital, Inc. + * SPDX-License-Identifier: Apache-2.0 + */ + +package worker + +import ( + "context" + "fmt" + "os" + "testing" + + "github.com/dgraph-io/badger/v4" + "github.com/dgraph-io/dgraph/v25/posting" + "github.com/dgraph-io/dgraph/v25/protos/pb" + "github.com/dgraph-io/dgraph/v25/schema" + "github.com/dgraph-io/dgraph/v25/x" +) + +// Benchmarks comparing the legacy serial mutation path (runMutation per edge) +// with the new per-predicate mutation pipeline (newRunMutations). +// +// What the pipeline ought to win on: +// - many predicates per transaction → one goroutine per predicate +// - many indexed edges per predicate → 10-way intra-predicate +// parallelism on tokenization +// +// What it shouldn't help (and may regret): +// - tiny mutations (1-2 edges, 1 predicate) where goroutine spin-up cost +// dominates the mutation work +// +// Each iteration is a single transaction: build a fresh batch of edges, +// run mutations, txn.Update(), CommitToDisk. We do NOT include the b.ResetTimer() +// before edge construction because edge construction is part of the +// per-transaction cost the pipeline is supposed to amortize. + +func benchSetup(b *testing.B, schemaTxt string) *badger.DB { + b.Helper() + dir, err := os.MkdirTemp("", "pipeline_bench_") + if err != nil { + b.Fatal(err) + } + b.Cleanup(func() { _ = os.RemoveAll(dir) }) + + ps, err := badger.OpenManaged(badger.DefaultOptions(dir).WithLoggingLevel(badger.ERROR)) + if err != nil { + b.Fatal(err) + } + b.Cleanup(func() { _ = ps.Close() }) + + posting.Init(ps, 0, false) + Init(ps) + posting.Oracle().ResetTxns() + if err := schema.ParseBytes([]byte(schemaTxt), 1); err != nil { + b.Fatal(err) + } + return ps +} + +// buildEdges constructs numPreds*edgesPerPred edges across distinct predicates, +// indexed-string-valued. The same generator drives both legacy and pipeline +// runs so the input is identical. +func buildEdges(numPreds, edgesPerPred int, baseUid uint64) []*pb.DirectedEdge { + edges := make([]*pb.DirectedEdge, 0, numPreds*edgesPerPred) + for p := 0; p < numPreds; p++ { + attr := x.AttrInRootNamespace(fmt.Sprintf("p%d", p)) + for e := 0; e < edgesPerPred; e++ { + edges = append(edges, &pb.DirectedEdge{ + Entity: baseUid + uint64(e), + Attr: attr, + Value: []byte(fmt.Sprintf("v%d_%d", p, e)), + ValueType: pb.Posting_STRING, + Op: pb.DirectedEdge_SET, + }) + } + } + return edges +} + +// schemaForPreds emits "p0: string @index(exact) ., p1: ..., ..." (or no +// index, depending on indexed). Each predicate is a distinct list-or-scalar. +func schemaForPreds(numPreds int, indexed bool, list bool) string { + var b []byte + for p := 0; p < numPreds; p++ { + ty := "string" + if list { + ty = "[string]" + } + idx := "" + if indexed { + idx = " @index(exact)" + } + b = append(b, []byte(fmt.Sprintf("p%d: %s%s .\n", p, ty, idx))...) + } + return string(b) +} + +// runOne executes one transaction's mutations through the chosen path. +// startTs/commitTs must be unique per call. +func runOnePipeline(b *testing.B, ps *badger.DB, edges []*pb.DirectedEdge, startTs, commitTs uint64) { + b.Helper() + txn := posting.Oracle().RegisterStartTs(startTs) + if err := newRunMutations(context.Background(), edges, txn); err != nil { + b.Fatal(err) + } + txn.Update() + w := posting.NewTxnWriter(ps) + if err := txn.CommitToDisk(w, commitTs); err != nil { + b.Fatal(err) + } + if err := w.Flush(); err != nil { + b.Fatal(err) + } + txn.UpdateCachedKeys(commitTs) +} + +func runOneLegacy(b *testing.B, ps *badger.DB, edges []*pb.DirectedEdge, startTs, commitTs uint64) { + b.Helper() + txn := posting.Oracle().RegisterStartTs(startTs) + for _, e := range edges { + if err := runMutation(context.Background(), e, txn); err != nil { + b.Fatal(err) + } + } + txn.Update() + w := posting.NewTxnWriter(ps) + if err := txn.CommitToDisk(w, commitTs); err != nil { + b.Fatal(err) + } + if err := w.Flush(); err != nil { + b.Fatal(err) + } + txn.UpdateCachedKeys(commitTs) +} + +// runBench runs sub-benchmarks (legacy vs pipeline) for a single +// (numPreds, edgesPerPred, indexed, list) configuration. +func runBench(b *testing.B, numPreds, edgesPerPred int, indexed, list bool) { + for _, mode := range []struct { + name string + fn func(*testing.B, *badger.DB, []*pb.DirectedEdge, uint64, uint64) + }{ + {"legacy", runOneLegacy}, + {"pipeline", runOnePipeline}, + } { + b.Run(mode.name, func(b *testing.B) { + ps := benchSetup(b, schemaForPreds(numPreds, indexed, list)) + b.ReportAllocs() + b.ResetTimer() + ts := uint64(10) + for i := 0; i < b.N; i++ { + edges := buildEdges(numPreds, edgesPerPred, uint64(i)*1_000_000+1) + mode.fn(b, ps, edges, ts, ts+1) + ts += 2 + } + }) + } +} + +// 1 predicate, 1 edge — smallest possible mutation. Pipeline overhead +// is most visible here. +func BenchmarkMutate_1pred_1edge_indexed(b *testing.B) { + runBench(b, 1, 1, true, false) +} + +// 1 predicate, 100 indexed edges — exercises intra-predicate +// tokenization parallelism. +func BenchmarkMutate_1pred_100edges_indexed(b *testing.B) { + runBench(b, 1, 100, true, false) +} + +// 10 predicates, 1 edge each — per-predicate parallelism with light work +// per predicate. +func BenchmarkMutate_10preds_1edge_indexed(b *testing.B) { + runBench(b, 10, 1, true, false) +} + +// 10 predicates, 100 edges each — full benefit case: per-predicate AND +// intra-predicate parallelism on indexed work. +func BenchmarkMutate_10preds_100edges_indexed(b *testing.B) { + runBench(b, 10, 100, true, false) +} + +// 1 predicate, 1000 indexed edges — heavy intra-predicate. +func BenchmarkMutate_1pred_1000edges_indexed(b *testing.B) { + runBench(b, 1, 1000, true, false) +} + +// 10 predicates, 1000 edges each — large mutation, indexed. +func BenchmarkMutate_10preds_1000edges_indexed(b *testing.B) { + runBench(b, 10, 1000, true, false) +} + +// Non-indexed counterparts isolate per-predicate parallelism from the +// tokenization parallelism. +func BenchmarkMutate_10preds_1000edges_noindex(b *testing.B) { + runBench(b, 10, 1000, false, false) +} + +// Very large indexed mutation: 50 predicates × 1000 edges each = 50k edges. +// Where the pipeline should shine most. +func BenchmarkMutate_50preds_1000edges_indexed(b *testing.B) { + runBench(b, 50, 1000, true, false) +} + +// 50 predicates, 100 edges each (5k edges) — typical-ish bulk write shape. +func BenchmarkMutate_50preds_100edges_indexed(b *testing.B) { + runBench(b, 50, 100, true, false) +} From 732512e115f0ab185273d2c208763169d30f2e6a Mon Sep 17 00:00:00 2001 From: Matthew McNeely Date: Fri, 1 May 2026 15:54:53 -0400 Subject: [PATCH 08/17] feat(mutations): replace pipeline on/off flag with edge-count threshold MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The benchmark matrix in worker/pipeline_bench_test.go showed the pipeline loses ~2x on small mutations (≤10 edges total) and wins ~1.5x on bulk (live-loader sized: 1000 edges per txn across many predicates). A binary on/off flag forces an all-or-nothing choice, penalising whichever side of that crossover the workload spends most time on. Replace MutationsUsePipeline (bool) with MutationsPipelineThreshold (int): threshold = 0 -> never use the pipeline (default; legacy behavior) threshold = 1 -> always use the pipeline (any txn with ≥1 edge) threshold = N -> use the pipeline only when len(m.Edges) >= N The threshold compares against total edges in the proposal. From the benches the crossover is around 100; the live-loader 1M dataset uses ~1000 edges per txn, so anything from 100-1000 will engage the pipeline only on bulk-shaped mutations and leave small interactive mutations on the legacy serial path. Wiring: - x.WorkerConfig.MutationsPipelineThreshold (int) replaces the bool field. - feature-flags superflag: "mutations-pipeline-threshold=0". - alpha/run.go reads it via featureFlagsConf.GetInt64. - worker/draft.go applyMutations branches on `t > 0 && len(m.Edges) >= t`. Verified end-to-end against the live-loader benchmark (1million.rdf.gz, official 1M schema): threshold=0 : 13.56s, 80,129 N-Quads/s (legacy, matches baseline) threshold=1 : 9.92s, 115,742 N-Quads/s (always-on, matches prior) CLI usage: dgraph alpha --feature-flags="mutations-pipeline-threshold=200" Co-Authored-By: Claude Opus 4.7 (1M context) --- dgraph/cmd/alpha/run.go | 2 +- worker/draft.go | 2 +- worker/server_state.go | 2 +- x/config.go | 16 ++++++++++++---- 4 files changed, 15 insertions(+), 7 deletions(-) diff --git a/dgraph/cmd/alpha/run.go b/dgraph/cmd/alpha/run.go index 9f827403ebb..d26a5186180 100644 --- a/dgraph/cmd/alpha/run.go +++ b/dgraph/cmd/alpha/run.go @@ -796,7 +796,7 @@ func run() { x.Config.NormalizeCompatibilityMode = featureFlagsConf.GetString("normalize-compatibility-mode") enableDetailedMetrics := featureFlagsConf.GetBool("enable-detailed-metrics") x.WorkerConfig.SlowQueryLogThreshold = featureFlagsConf.GetDuration("log-slow-query-threshold") - x.WorkerConfig.MutationsUsePipeline = featureFlagsConf.GetBool("mutations-use-pipeline") + x.WorkerConfig.MutationsPipelineThreshold = int(featureFlagsConf.GetInt64("mutations-pipeline-threshold")) x.PrintVersion() glog.Infof("x.Config: %+v", x.Config) diff --git a/worker/draft.go b/worker/draft.go index ad43e2f55f0..b0bcbeac205 100644 --- a/worker/draft.go +++ b/worker/draft.go @@ -526,7 +526,7 @@ func (n *node) applyMutations(ctx context.Context, proposal *pb.Proposal) (rerr // Discard the posting lists from cache to release memory at the end. defer txn.Update() - if x.WorkerConfig.MutationsUsePipeline { + if t := x.WorkerConfig.MutationsPipelineThreshold; t > 0 && len(m.Edges) >= t { mp := posting.NewMutationPipeline(txn) return mp.Process(ctx, m.Edges) } diff --git a/worker/server_state.go b/worker/server_state.go index 6824b0adc26..1df10e69c09 100644 --- a/worker/server_state.go +++ b/worker/server_state.go @@ -43,7 +43,7 @@ const ( `lambda-url=;` CacheDefaults = `size-mb=4096; percentage=40,40,20; remove-on-update=false` FeatureFlagsDefaults = `normalize-compatibility-mode=; enable-detailed-metrics=false; ` + - `log-slow-query-threshold=0; mutations-use-pipeline=false` + `log-slow-query-threshold=0; mutations-pipeline-threshold=0` ) // ServerState holds the state of the Dgraph server. diff --git a/x/config.go b/x/config.go index 7550172037b..2a756b0f916 100644 --- a/x/config.go +++ b/x/config.go @@ -138,10 +138,18 @@ type WorkerOptions struct { HardSync bool // Audit contains the audit flags that enables the audit. Audit bool - // MutationsUsePipeline enables the per-predicate mutation pipeline in - // applyMutations. When false (default), mutations follow the legacy - // serial path. The flag is plumbed via the "feature-flags" superflag. - MutationsUsePipeline bool + // MutationsPipelineThreshold gates the per-predicate mutation pipeline + // in applyMutations. A mutation runs through the pipeline only when + // MutationsPipelineThreshold > 0 and len(m.Edges) >= the threshold; + // otherwise it falls back to the legacy serial path. Set to 0 (default) + // to disable the pipeline entirely. Set to 1 to always use the pipeline. + // The pipeline pays goroutine spin-up cost per predicate, so small + // mutations are slower on it; bulk multi-predicate mutations are + // faster — pick a value above the per-mutation edge count where the + // crossover happens for your workload (~100 in benchmarks here). + // Plumbed via the "feature-flags" superflag as + // "mutations-pipeline-threshold". + MutationsPipelineThreshold int } // WorkerConfig stores the global instance of the worker package's options. From 8b326777e3f03c1f7a7abe34f110ac34359474e3 Mon Sep 17 00:00:00 2001 From: Matthew McNeely Date: Fri, 1 May 2026 15:58:23 -0400 Subject: [PATCH 09/17] feat(mutations): flip default mutations-pipeline-threshold from 0 to 1 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Engages the per-predicate mutation pipeline by default so that systest runs (and any other test suites that don't override feature-flags) exercise the pipeline path on every mutation, not the legacy serial path. Threshold of 1 means "any mutation with ≥1 edge takes the pipeline" — i.e. always on. This is a deliberate ramp toward shipping the pipeline. Operators who want to opt small interactive mutations out of the pipeline (where benches showed ~2x slowdown for ≤10-edge txns) can set a higher threshold: dgraph alpha --feature-flags="mutations-pipeline-threshold=200" To turn the pipeline fully off, set 0. Co-Authored-By: Claude Opus 4.7 (1M context) --- worker/server_state.go | 2 +- x/config.go | 11 ++++++----- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/worker/server_state.go b/worker/server_state.go index 1df10e69c09..66bdb3f8e4d 100644 --- a/worker/server_state.go +++ b/worker/server_state.go @@ -43,7 +43,7 @@ const ( `lambda-url=;` CacheDefaults = `size-mb=4096; percentage=40,40,20; remove-on-update=false` FeatureFlagsDefaults = `normalize-compatibility-mode=; enable-detailed-metrics=false; ` + - `log-slow-query-threshold=0; mutations-pipeline-threshold=0` + `log-slow-query-threshold=0; mutations-pipeline-threshold=1` ) // ServerState holds the state of the Dgraph server. diff --git a/x/config.go b/x/config.go index 2a756b0f916..4613db0841f 100644 --- a/x/config.go +++ b/x/config.go @@ -141,12 +141,13 @@ type WorkerOptions struct { // MutationsPipelineThreshold gates the per-predicate mutation pipeline // in applyMutations. A mutation runs through the pipeline only when // MutationsPipelineThreshold > 0 and len(m.Edges) >= the threshold; - // otherwise it falls back to the legacy serial path. Set to 0 (default) - // to disable the pipeline entirely. Set to 1 to always use the pipeline. - // The pipeline pays goroutine spin-up cost per predicate, so small + // otherwise it falls back to the legacy serial path. Set to 0 to + // disable the pipeline entirely. Set to 1 (default) to always use it. + // The pipeline pays goroutine spin-up cost per predicate, so tiny // mutations are slower on it; bulk multi-predicate mutations are - // faster — pick a value above the per-mutation edge count where the - // crossover happens for your workload (~100 in benchmarks here). + // faster — set to a value above the per-mutation edge count where the + // crossover happens for your workload (~100 in benchmarks here) if + // you want only large mutations to take the pipeline path. // Plumbed via the "feature-flags" superflag as // "mutations-pipeline-threshold". MutationsPipelineThreshold int From eb83b8cf3f635d7dfd0559dfc0ae6a98e3e23929 Mon Sep 17 00:00:00 2001 From: Matthew McNeely Date: Fri, 1 May 2026 16:45:25 -0400 Subject: [PATCH 10/17] fix(posting): re-enable IsEmpty check in IterateDisk MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The original branch commit 41d6445ce1 ("fixed some bug") replaced the IsEmpty(readTs) call in IterateDisk with a hardcoded `false`, forcing every key found by the iterator to be reported as non-empty. That broke has() for any uid whose value had been removed via star-deletion ( *): the data list still exists in badger with a DeleteAll marker on top, but the live posting list is empty at readTs — IsEmpty returns true and the uid should be skipped. Surfaced by systest TestSystestSuite/TestHasDeletedEdge in systest/mutations-and-queries: 3 nodes are created with "", one is star-deleted, follow-up has(end) is expected to return 2 uids. With IsEmpty stubbed to false it returned 3. No comment was left on the original change. Restoring the call. The mutations-and-queries package is fully green with this in place (66/66 tests pass including TestHasDeletedEdge); if a real underlying issue motivated the original disable we'll chase it with a real diagnosis instead of silently dropping a safety check. Co-Authored-By: Claude Opus 4.7 (1M context) --- posting/mvcc.go | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/posting/mvcc.go b/posting/mvcc.go index e3bb2900bd1..e8e6732c95c 100644 --- a/posting/mvcc.go +++ b/posting/mvcc.go @@ -496,8 +496,7 @@ func (ml *MemoryLayer) IterateDisk(ctx context.Context, f IterateDiskArgs) error if err != nil { return err } - empty, err := false, nil - //empty, err := l.IsEmpty(f.ReadTs, 0) + empty, err := l.IsEmpty(f.ReadTs, 0) switch { case err != nil: return err From ef3eb6a6eaf0afc3f1250c851df6efd2d9a7b76d Mon Sep 17 00:00:00 2001 From: Matthew McNeely Date: Fri, 1 May 2026 21:07:35 -0400 Subject: [PATCH 11/17] fix(pipeline): nil-deref in ProcessSingle on multi-Del-per-uid + strip stale READING debug prints MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Two bugs surfaced by graphql/e2e/auth's TestOrderAndOffset (cleanup mutation \`mutation DelTask { deleteTask(filter: {}) }\` crashed the alpha mid-request, causing the test client to see EOF on POST): 1. posting/index.go ProcessSingle SIGSEGV at line 675. GraphQL deleteTask cleanup expands into multiple Del edges per entity (one per predicate the entity has — uid, type, list edges, etc.). When two Del edges to the same uid land in one transaction's batch, the second iteration through ProcessSingle's per-edge loop does: pl, exists := postings[uid] if exists { if edge.Op == DEL { oldVal = findSingleValueInPostingList(pl) if string(edge.Value) == string(oldVal.Value) { ... } ^^^^^^ nil deref } } findSingleValueInPostingList only returns Set postings; if the accumulated list holds only Dels (from the prior iteration), it returns nil and we panic dereferencing oldVal.Value. Two fixes here: - Guard the deref: \`if oldVal != nil && string(...) == ...\`. - Move \`var oldVal *pb.Posting\` inside the loop. It was declared at function scope, so a stale value from one edge could bleed into the nil-guarded branch for a different uid on a later iteration. Per-edge scope makes the intent explicit. 2. worker/task.go: two leftover \`fmt.Println("READING SINGLE", ...)\` and \`fmt.Println("READING", ...)\` calls in the value-postings read path. Same class of debug spew Phase 1B stripped from posting/, missed because that sweep didn't include worker/. Removed both. Safe — they were unconditional prints on every query value read. The graphql/e2e/auth and graphql/e2e/auth/debug_off packages now both pass in 30s. \`./posting/\` and \`./worker/\` unit tests still green. Co-Authored-By: Claude Opus 4.7 (1M context) --- posting/index.go | 11 +++++++++-- worker/task.go | 4 ---- 2 files changed, 9 insertions(+), 6 deletions(-) diff --git a/posting/index.go b/posting/index.go index 57d996cba5a..779ed1775b9 100644 --- a/posting/index.go +++ b/posting/index.go @@ -634,7 +634,6 @@ func (mp *MutationPipeline) ProcessSingle(ctx context.Context, pipeline *Predica dataKey := x.DataKey(pipeline.attr, 0) insertDeleteAllEdge := !(info.index || info.reverse || info.count) // nolint - var oldVal *pb.Posting for edge := range pipeline.edges { if edge.Op != pb.DirectedEdge_DEL && !schemaExists { return errors.Errorf("runMutation: Unable to find schema for %s", edge.Attr) @@ -644,6 +643,9 @@ func (mp *MutationPipeline) ProcessSingle(ctx context.Context, pipeline *Predica return err } + // oldVal is reset per edge so a stale value from a previous iteration + // can't bleed into the nil-guarded branch below. + var oldVal *pb.Posting uid := edge.Entity pl, exists := postings[uid] @@ -671,8 +673,13 @@ func (mp *MutationPipeline) ProcessSingle(ctx context.Context, pipeline *Predica if exists { if edge.Op == pb.DirectedEdge_DEL { + // findSingleValueInPostingList returns nil when the + // accumulated postings for this uid hold only Del entries + // (no Set), which happens when the same uid receives + // multiple Del edges in one batch (e.g. GraphQL + // deleteTask cleanup deleting many predicates per entity). oldVal = findSingleValueInPostingList(pl) - if string(edge.Value) == string(oldVal.Value) { + if oldVal != nil && string(edge.Value) == string(oldVal.Value) { setPosting() } } else { diff --git a/worker/task.go b/worker/task.go index eb7443ea28d..409ec3f0fc4 100644 --- a/worker/task.go +++ b/worker/task.go @@ -487,8 +487,6 @@ func (qs *queryState) handleValuePostings(ctx context.Context, args funcArgs) er Value: p.Value, } } - pk, _ := x.Parse(key) - fmt.Println("READING SINGLE", pk, vals, pl) } else { pl, err := qs.cache.Get(key) if err != nil { @@ -508,8 +506,6 @@ func (qs *queryState) handleValuePostings(ctx context.Context, args funcArgs) er } vals, fcs, err = retrieveValuesAndFacets(args, pl, facetsTree, listType) - pk, _ := x.Parse(key) - fmt.Println("READING", pk, vals, fcs, pl.Print()) switch { case err == posting.ErrNoValue || (err == nil && len(vals) == 0): From 6ec42f722418dc2613437546c57be880e4403ee3 Mon Sep 17 00:00:00 2001 From: Matthew McNeely Date: Sat, 2 May 2026 16:10:54 -0400 Subject: [PATCH 12/17] fix(posting): defensive-copy reused key buffers in ReadPostingList + addKeyToBatch MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit The 21million live-load systest failed with `~genre @filter(uid(F))` returning wrong films for some genres — corruption pattern was that one genre's reverse list contained another genre's films verbatim, while a third genre's reverse list was missing thousands of entries. Different genres were affected on each load, but always: legacy runMutation produced correct reverse counts; the per-predicate pipeline produced corrupt ones. Root cause: the pipeline's ProcessCount per-uid loop allocates one key buffer per ProcessCount call and mutates its trailing 8 bytes each iteration via `binary.BigEndian.PutUint64(dataKey[len-8:], uid)`. Two distinct sites then captured the slice header rather than the bytes: 1. ReadPostingList's `l.key = key` aliased the caller's buffer. saveInCache stores a copyList of the freshly-read list, and copyList sets `key: l.key` — also an alias. The cached list's key field therefore points at a buffer that the pipeline keeps mutating; by the time the async rollup path retrieves the cached list and runs `kv.Key = alloc.Copy(l.key)` to build the rolled-up KV, the bytes are whatever the LAST iteration left behind. Rollup then writes a BitCompletePosting with WithDiscard() to the wrong key, overwriting an unrelated reverse list with this list's (rolled-up) contents. 2. ReadPostingList's defer `IncrRollup.addKeyToBatch(key, ...)` appended the slice header verbatim to the rollup queue. Every queued entry from one ProcessCount goroutine ended up pointing at the same shared dataKey buffer; by the time the rollup goroutine processed the batch the bytes had collapsed to the final iteration's uid, redirecting many distinct rollup targets to the same key. Both sites fixed by taking ownership of the bytes (`append nil` / explicit `make+copy`). Legacy runMutation hits ReadPostingList too but allocates a fresh key per call, so it never aliased anything; the bug is only visible when a caller deliberately reuses one buffer across many uids the way ProcessCount does. Verified end-to-end on the systest/21million/live load with mutations-pipeline-threshold=1: a fresh load + sweep across all 764 Genre entities now reports `count(~genre) == count(forward edges)` for every genre. Pre-fix the same sweep showed 4 mismatched genres with thousands of stale or missing reverse entries; on a different load it showed Documentary missing 23,325 of its 31,370 reverse entries while Children's/Family had 1,435 stale Crime-Thriller entries. Unit tests in posting/ and worker/ still pass. Co-Authored-By: Claude Opus 4.7 (1M context) --- posting/mvcc.go | 24 ++++++++++++++++++++++-- 1 file changed, 22 insertions(+), 2 deletions(-) diff --git a/posting/mvcc.go b/posting/mvcc.go index e8e6732c95c..59ed5c26f0a 100644 --- a/posting/mvcc.go +++ b/posting/mvcc.go @@ -137,9 +137,20 @@ func (ir *incrRollupi) rollUpKey(writer *TxnWriter, key []byte) error { // TODO: When the opRollup is not running the keys from keysPool of ir are dropped. Figure out some // way to handle that. func (ir *incrRollupi) addKeyToBatch(key []byte, priority int) { + // Defensive copy. Callers (notably ReadPostingList's defer and the + // per-predicate mutation pipeline's ProcessCount loop) reuse a single + // key buffer across iterations and mutate its last 8 bytes per uid; + // keeping a slice header that aliases that buffer means every queued + // rollup target collapses to whatever uid was processed last. The + // async rollup then writes a BitCompletePosting to the wrong key, + // overwriting an unrelated posting list with WithDiscard. Take a copy + // here so the rollup queue owns its bytes. + keyCopy := make([]byte, len(key)) + copy(keyCopy, key) + rki := ir.priorityKeys[priority] batch := rki.keysPool.Get().(*[][]byte) - *batch = append(*batch, key) + *batch = append(*batch, keyCopy) if len(*batch) < 16 { rki.keysPool.Put(batch) return @@ -678,7 +689,16 @@ func ReadPostingList(key []byte, it *badger.Iterator) (*List, error) { } l := new(List) - l.key = key + // Copy the key bytes. The caller (mutation pipeline ProcessCount and + // similar paths) reuses a single key buffer across loop iterations and + // mutates its trailing 8 bytes to scan many uids; without copying here + // the list's l.key field aliases that buffer, so by the time the list + // is read back from the in-memory cache (saveInCache stashes a + // copyList that shares the key alias) the bytes have changed. The + // async rollup path then takes alloc.Copy(l.key) to build the rolled- + // up KV's key and ends up writing a BitCompletePosting (with + // WithDiscard) to a different list's key — corrupting that list. + l.key = append([]byte(nil), key...) l.plist = new(pb.PostingList) l.mutationMap = newMutableLayer() l.minTs = 0 From bed2ea59e5b70b2b774d9e296777d97ad800f64f Mon Sep 17 00:00:00 2001 From: Matthew McNeely Date: Sat, 2 May 2026 16:11:13 -0400 Subject: [PATCH 13/17] test(worker): four [uid] @reverse @count regression harnesses MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Added in pursuit of the 21million live-load failure that turned out to be the dataKey buffer-aliasing bug fixed in 0a9ffc174. None of these tests reproduce that specific corruption — it needs the async rollup path that only triggers on a real running cluster — but they pin down what the per-predicate pipeline does correctly for the non-rollup cases: - TestPipelineReverseListCount: one transaction, multiple subjects pointing at multiple objects on a [uid] @reverse @count predicate. Verifies forward and reverse lists are both complete. - TestPipelineReverseListCountMultiBatch: 50 subjects x 20 objects spread across 143 sequential transactions in batches of 7. - TestPipelineReverseListCountMultiPred: 30 subjects x 12 objects across 3 distinct list-uid + reverse + count predicates, with shuffled edges and small batches so the per-predicate pipeline goroutines for each predicate run in parallel inside Process(). - TestPipelineReverseListCountConcurrent: same shape as the multi-batch case but with 10 worker goroutines submitting batches in parallel through the fakeOracle conflict-checking harness. All four pass cleanly. They guard against regressions in the in-memory mutation path's reverse-list bookkeeping; the rollup path that the 0a9ffc174 fix actually targets is exercised by the systest/21million/live integration test. Co-Authored-By: Claude Opus 4.7 (1M context) --- worker/sort_test.go | 459 ++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 459 insertions(+) diff --git a/worker/sort_test.go b/worker/sort_test.go index 2ae82190440..ca60a53f3fa 100644 --- a/worker/sort_test.go +++ b/worker/sort_test.go @@ -585,6 +585,465 @@ func TestPipelineCountIndexConcurrent(t *testing.T) { } } +// TestPipelineReverseListCount mirrors the [uid] @reverse @count shape from +// the 21million live-load test (genre predicate). One subject points at +// multiple objects in a single transaction; we then verify that BOTH the +// forward data list and the reverse data list are complete. +// +// Background: systest/21million/live's TestQueries/Run_queries/query-017 +// fails consistently with one specific film's `genre = Animation` edge +// missing, while other genre edges from the same film are intact. This +// is the smallest in-process repro of the same forward/reverse fanout +// pattern that the live loader hits. +func TestPipelineReverseListCount(t *testing.T) { + dir, err := os.MkdirTemp("", "pipelinerevcount_") + require.NoError(t, err) + defer os.RemoveAll(dir) + + ps, err := badger.OpenManaged(badger.DefaultOptions(dir)) + require.NoError(t, err) + defer ps.Close() + posting.Init(ps, 0, false) + Init(ps) + posting.Oracle().ResetTxns() + + require.NoError(t, schema.ParseBytes( + []byte(`genre: [uid] @reverse @count .`), 1)) + + pred := x.AttrInRootNamespace("genre") + + // Subject -> list of object uids. Mirrors a film with multiple genres, + // and several films sharing some genres. + const ( + madagascar = uint64(100) + brotherly = uint64(101) + animation = uint64(200) + shortFilm = uint64(201) + comedy = uint64(202) + ) + wantForward := map[uint64][]uint64{ + madagascar: {animation, shortFilm}, + brotherly: {animation, shortFilm, comedy}, + } + + // Single transaction containing all edges (the simplest case — no + // concurrency, no batching, no Oracle conflict path). If this fails, + // the bug is purely in the forward+reverse fanout within ProcessList. + edges := []*pb.DirectedEdge{} + for subj, objs := range wantForward { + for _, obj := range objs { + edges = append(edges, &pb.DirectedEdge{ + Entity: subj, + Attr: pred, + ValueId: obj, + ValueType: pb.Posting_UID, + Op: pb.DirectedEdge_SET, + }) + } + } + + txn := posting.Oracle().RegisterStartTs(10) + require.NoError(t, newRunMutations(context.Background(), edges, txn)) + txn.Update() + w := posting.NewTxnWriter(ps) + require.NoError(t, txn.CommitToDisk(w, 11)) + require.NoError(t, w.Flush()) + txn.UpdateCachedKeys(11) + + readTxn := posting.Oracle().RegisterStartTs(math.MaxUint64) + + // Forward: each subject must have all its expected objects. + for subj, wantObjs := range wantForward { + key := x.DataKey(pred, subj) + pl, err := readTxn.Get(key) + require.NoError(t, err) + got, err := pl.Uids(posting.ListOptions{ReadTs: math.MaxUint64}) + require.NoError(t, err) + require.ElementsMatch(t, wantObjs, got.Uids, + "forward list for subject %d", subj) + } + + // Reverse: each object must have all the subjects that point at it. + wantReverse := map[uint64][]uint64{} + for subj, objs := range wantForward { + for _, obj := range objs { + wantReverse[obj] = append(wantReverse[obj], subj) + } + } + for obj, wantSubjs := range wantReverse { + key := x.ReverseKey(pred, obj) + pl, err := readTxn.Get(key) + require.NoError(t, err) + got, err := pl.Uids(posting.ListOptions{ReadTs: math.MaxUint64}) + require.NoError(t, err) + require.ElementsMatch(t, wantSubjs, got.Uids, + "reverse list for object %d", obj) + } +} + +// TestPipelineReverseListCountMultiBatch escalates TestPipelineReverseListCount +// by spreading edges across many sequential transactions, similar to how the +// live loader chunks edges into many batches. Each batch is one transaction. +func TestPipelineReverseListCountMultiBatch(t *testing.T) { + dir, err := os.MkdirTemp("", "pipelinerevcount2_") + require.NoError(t, err) + defer os.RemoveAll(dir) + + ps, err := badger.OpenManaged(badger.DefaultOptions(dir)) + require.NoError(t, err) + defer ps.Close() + posting.Init(ps, 0, false) + Init(ps) + posting.Oracle().ResetTxns() + + require.NoError(t, schema.ParseBytes( + []byte(`genre: [uid] @reverse @count .`), 1)) + + pred := x.AttrInRootNamespace("genre") + + // Many subjects, many objects, every (subject, object) combination — a + // dense forward fanout that pushes every reverse list to grow as well. + const ( + nSubjects = 50 + nObjects = 20 + ) + subjBase := uint64(10000) + objBase := uint64(20000) + + // Build every edge. + allEdges := make([]*pb.DirectedEdge, 0, nSubjects*nObjects) + for s := 0; s < nSubjects; s++ { + for o := 0; o < nObjects; o++ { + allEdges = append(allEdges, &pb.DirectedEdge{ + Entity: subjBase + uint64(s), + Attr: pred, + ValueId: objBase + uint64(o), + ValueType: pb.Posting_UID, + Op: pb.DirectedEdge_SET, + }) + } + } + + // Split into batches of 7 — a value chosen to make each batch carry a + // non-trivial mix of subjects and objects per ProcessList run. + const batchSize = 7 + ts := uint64(10) + for start := 0; start < len(allEdges); start += batchSize { + end := start + batchSize + if end > len(allEdges) { + end = len(allEdges) + } + batch := allEdges[start:end] + + txn := posting.Oracle().RegisterStartTs(ts) + require.NoError(t, newRunMutations(context.Background(), batch, txn)) + txn.Update() + w := posting.NewTxnWriter(ps) + require.NoError(t, txn.CommitToDisk(w, ts+1)) + require.NoError(t, w.Flush()) + txn.UpdateCachedKeys(ts + 1) + ts += 2 + } + + readTxn := posting.Oracle().RegisterStartTs(math.MaxUint64) + + // Forward: each subject must hold exactly all nObjects objects. + wantForwardObjs := make([]uint64, nObjects) + for o := 0; o < nObjects; o++ { + wantForwardObjs[o] = objBase + uint64(o) + } + for s := 0; s < nSubjects; s++ { + subj := subjBase + uint64(s) + key := x.DataKey(pred, subj) + pl, err := readTxn.Get(key) + require.NoError(t, err) + got, err := pl.Uids(posting.ListOptions{ReadTs: math.MaxUint64}) + require.NoError(t, err) + require.ElementsMatch(t, wantForwardObjs, got.Uids, + "forward list for subject %d (s=%d): missing some objects", subj, s) + } + + // Reverse: each object must hold exactly all nSubjects subjects. + wantReverseSubjs := make([]uint64, nSubjects) + for s := 0; s < nSubjects; s++ { + wantReverseSubjs[s] = subjBase + uint64(s) + } + for o := 0; o < nObjects; o++ { + obj := objBase + uint64(o) + key := x.ReverseKey(pred, obj) + pl, err := readTxn.Get(key) + require.NoError(t, err) + got, err := pl.Uids(posting.ListOptions{ReadTs: math.MaxUint64}) + require.NoError(t, err) + require.ElementsMatch(t, wantReverseSubjs, got.Uids, + "reverse list for object %d (o=%d): missing some subjects", obj, o) + } +} + +// TestPipelineReverseListCountMultiPred escalates further: multiple +// list-uid predicates with @reverse @count are mutated together in each +// batch, so the per-predicate pipeline goroutines for each predicate run +// in parallel inside Process(). This is the live-loader's actual shape +// (genre + director.film + starring + rated all live in the same payload). +func TestPipelineReverseListCountMultiPred(t *testing.T) { + dir, err := os.MkdirTemp("", "pipelinerevcount3_") + require.NoError(t, err) + defer os.RemoveAll(dir) + + ps, err := badger.OpenManaged(badger.DefaultOptions(dir)) + require.NoError(t, err) + defer ps.Close() + posting.Init(ps, 0, false) + Init(ps) + posting.Oracle().ResetTxns() + + require.NoError(t, schema.ParseBytes([]byte(` + genre: [uid] @reverse @count . + starring: [uid] @reverse @count . + director_film: [uid] @reverse @count . + `), 1)) + + preds := []string{ + x.AttrInRootNamespace("genre"), + x.AttrInRootNamespace("starring"), + x.AttrInRootNamespace("director_film"), + } + + const ( + nSubjects = 30 + nObjects = 12 + ) + subjBase := uint64(10000) + objBase := uint64(20000) + + allEdges := make([]*pb.DirectedEdge, 0, len(preds)*nSubjects*nObjects) + for _, pred := range preds { + for s := 0; s < nSubjects; s++ { + for o := 0; o < nObjects; o++ { + allEdges = append(allEdges, &pb.DirectedEdge{ + Entity: subjBase + uint64(s), + Attr: pred, + ValueId: objBase + uint64(o), + ValueType: pb.Posting_UID, + Op: pb.DirectedEdge_SET, + }) + } + } + } + // Shuffle so each batch carries an interleaved mix of predicates. + rnd := rand.New(rand.NewSource(1)) + rnd.Shuffle(len(allEdges), func(i, j int) { + allEdges[i], allEdges[j] = allEdges[j], allEdges[i] + }) + + const batchSize = 23 + ts := uint64(10) + for start := 0; start < len(allEdges); start += batchSize { + end := start + batchSize + if end > len(allEdges) { + end = len(allEdges) + } + batch := allEdges[start:end] + + txn := posting.Oracle().RegisterStartTs(ts) + require.NoError(t, newRunMutations(context.Background(), batch, txn)) + txn.Update() + w := posting.NewTxnWriter(ps) + require.NoError(t, txn.CommitToDisk(w, ts+1)) + require.NoError(t, w.Flush()) + txn.UpdateCachedKeys(ts + 1) + ts += 2 + } + + readTxn := posting.Oracle().RegisterStartTs(math.MaxUint64) + + wantForwardObjs := make([]uint64, nObjects) + for o := 0; o < nObjects; o++ { + wantForwardObjs[o] = objBase + uint64(o) + } + wantReverseSubjs := make([]uint64, nSubjects) + for s := 0; s < nSubjects; s++ { + wantReverseSubjs[s] = subjBase + uint64(s) + } + + for _, pred := range preds { + for s := 0; s < nSubjects; s++ { + subj := subjBase + uint64(s) + pl, err := readTxn.Get(x.DataKey(pred, subj)) + require.NoError(t, err) + got, err := pl.Uids(posting.ListOptions{ReadTs: math.MaxUint64}) + require.NoError(t, err) + require.ElementsMatch(t, wantForwardObjs, got.Uids, + "forward list for pred=%q subj=%d", pred, subj) + } + for o := 0; o < nObjects; o++ { + obj := objBase + uint64(o) + pl, err := readTxn.Get(x.ReverseKey(pred, obj)) + require.NoError(t, err) + got, err := pl.Uids(posting.ListOptions{ReadTs: math.MaxUint64}) + require.NoError(t, err) + require.ElementsMatch(t, wantReverseSubjs, got.Uids, + "reverse list for pred=%q obj=%d", pred, obj) + } + } +} + +// TestPipelineReverseListCountConcurrent mirrors the live-loader's actual +// concurrency: many goroutines submitting batched transactions in parallel, +// going through a real conflict-checking commit (the fakeOracle harness +// also used by TestPipelineCountIndexConcurrent), against a [uid] @reverse +// @count predicate. This is the closest in-process reproduction of the +// systest/21million/live shape that we have without standing up a real +// cluster. +func TestPipelineReverseListCountConcurrent(t *testing.T) { + dir, err := os.MkdirTemp("", "pipelinerevcount4_") + require.NoError(t, err) + defer os.RemoveAll(dir) + + ps, err := badger.OpenManaged(badger.DefaultOptions(dir)) + require.NoError(t, err) + defer ps.Close() + posting.Init(ps, 0, false) + Init(ps) + posting.Oracle().ResetTxns() + + require.NoError(t, schema.ParseBytes( + []byte(`genre: [uid] @reverse @count .`), 1)) + + pred := x.AttrInRootNamespace("genre") + + const ( + nSubjects = 60 // films + nObjects = 25 // genres + ) + subjBase := uint64(10000) + objBase := uint64(20000) + + allEdges := make([]*pb.DirectedEdge, 0, nSubjects*nObjects) + for s := 0; s < nSubjects; s++ { + for o := 0; o < nObjects; o++ { + allEdges = append(allEdges, &pb.DirectedEdge{ + Entity: subjBase + uint64(s), + Attr: pred, + ValueId: objBase + uint64(o), + ValueType: pb.Posting_UID, + Op: pb.DirectedEdge_SET, + }) + } + } + rnd := rand.New(rand.NewSource(7)) + rnd.Shuffle(len(allEdges), func(i, j int) { + allEdges[i], allEdges[j] = allEdges[j], allEdges[i] + }) + + // Chunk into many small batches so multiple goroutines compete on the + // same predicate, mimicking `dgraph live -c 10` behavior. + const batchSize = 17 + type batch []*pb.DirectedEdge + batches := []batch{} + for start := 0; start < len(allEdges); start += batchSize { + end := start + batchSize + if end > len(allEdges) { + end = len(allEdges) + } + batches = append(batches, allEdges[start:end]) + } + + oracle := newFakeOracle(10) + const concurrency = 10 + + jobs := make(chan batch, len(batches)) + for _, b := range batches { + jobs <- b + } + close(jobs) + + var wg sync.WaitGroup + wg.Add(concurrency) + for w := 0; w < concurrency; w++ { + go func() { + defer wg.Done() + for b := range jobs { + // Retry until commit, with a fresh edge clone each attempt + // (the pipeline mutates edge.ValueId during processing). + for attempt := 0; attempt < 200; attempt++ { + clones := make([]*pb.DirectedEdge, len(b)) + for i, e := range b { + clones[i] = &pb.DirectedEdge{ + Entity: e.Entity, Attr: e.Attr, + ValueId: e.ValueId, ValueType: e.ValueType, Op: e.Op, + } + } + if runPipelineTxn(t, ps, oracle, clones) { + break + } + } + } + }() + } + wg.Wait() + + t.Logf("committed=%d aborted=%d", oracle.committed.Load(), oracle.aborted.Load()) + + readTxn := posting.Oracle().RegisterStartTs(math.MaxUint64) + + wantForwardObjs := make([]uint64, nObjects) + for o := 0; o < nObjects; o++ { + wantForwardObjs[o] = objBase + uint64(o) + } + wantReverseSubjs := make([]uint64, nSubjects) + for s := 0; s < nSubjects; s++ { + wantReverseSubjs[s] = subjBase + uint64(s) + } + + missingForward := []string{} + missingReverse := []string{} + for s := 0; s < nSubjects; s++ { + subj := subjBase + uint64(s) + pl, err := readTxn.Get(x.DataKey(pred, subj)) + require.NoError(t, err) + got, err := pl.Uids(posting.ListOptions{ReadTs: math.MaxUint64}) + require.NoError(t, err) + if len(got.Uids) != nObjects { + missingForward = append(missingForward, + fmt.Sprintf("subj=%d has %d/%d", subj, len(got.Uids), nObjects)) + } + } + for o := 0; o < nObjects; o++ { + obj := objBase + uint64(o) + pl, err := readTxn.Get(x.ReverseKey(pred, obj)) + require.NoError(t, err) + got, err := pl.Uids(posting.ListOptions{ReadTs: math.MaxUint64}) + require.NoError(t, err) + if len(got.Uids) != nSubjects { + missingReverse = append(missingReverse, + fmt.Sprintf("obj=%d has %d/%d", obj, len(got.Uids), nSubjects)) + } + } + require.Empty(t, missingForward, "forward lists missing entries") + require.Empty(t, missingReverse, "reverse lists missing entries") + + // And the strict version: each list must match exactly. + for s := 0; s < nSubjects; s++ { + subj := subjBase + uint64(s) + pl, err := readTxn.Get(x.DataKey(pred, subj)) + require.NoError(t, err) + got, err := pl.Uids(posting.ListOptions{ReadTs: math.MaxUint64}) + require.NoError(t, err) + require.ElementsMatch(t, wantForwardObjs, got.Uids, + "forward list for subj %d", subj) + } + for o := 0; o < nObjects; o++ { + obj := objBase + uint64(o) + pl, err := readTxn.Get(x.ReverseKey(pred, obj)) + require.NoError(t, err) + got, err := pl.Uids(posting.ListOptions{ReadTs: math.MaxUint64}) + require.NoError(t, err) + require.ElementsMatch(t, wantReverseSubjs, got.Uids, + "reverse list for obj %d", obj) + } +} + func TestDeleteSetWithVarEdgeCorruptsData(t *testing.T) { // Setup temporary directory for Badger DB dir, err := os.MkdirTemp("", "storetest_") From 6ce7bed6df4e06251e1d1483d76becaec81aec68 Mon Sep 17 00:00:00 2001 From: Matthew McNeely Date: Sat, 2 May 2026 16:16:58 -0400 Subject: [PATCH 14/17] test(systest): add query-073 covering reverse counts for all 592 non-empty genres MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit A comprehensive `count(~genre)` check across every Genre entity in the 21million dataset. The query asks for each genre's name and the size of its reverse posting list, sorted by name; the fixture pins the expected count for all 592 genres that have at least one film. Motivation: the per-predicate mutation pipeline had a buffer-aliasing bug (fixed in 0a9ffc174) that produced wrong reverse-list contents on a different small subset of genres each load — sometimes Documentary lost most of its 31,370 entries, sometimes Children's/Family gained ~1,500 spurious Crime-Thriller entries, sometimes Backstage Musical and Indie film were the affected pair. The existing query-017 (Taraji-films-by-genre) only catches it when that specific actor's films happen to land on a corrupted genre, and query-016 / query-044 only test genres above 30,000 films. This new query exercises every genre's reverse list and pins the exact expected count, so any regression that mis-routes reverse edges for any genre will fail it directly. Co-Authored-By: Claude Opus 4.7 (1M context) --- systest/21million/queries/query-073 | 2378 +++++++++++++++++++++++++++ 1 file changed, 2378 insertions(+) create mode 100644 systest/21million/queries/query-073 diff --git a/systest/21million/queries/query-073 b/systest/21million/queries/query-073 new file mode 100644 index 00000000000..670cb00f07d --- /dev/null +++ b/systest/21million/queries/query-073 @@ -0,0 +1,2378 @@ +{ + q(func: type(Genre), orderasc: name@en) @filter(gt(count(~genre), 0)) { + name@en + n_films: count(~genre) + } +} +--- +{ + "q": [ + { + "name@en": "/m/04rlf", + "n_films": 2 + }, + { + "name@en": "3D film", + "n_films": 4 + }, + { + "name@en": "Abstract animation", + "n_films": 6 + }, + { + "name@en": "Absurdism", + "n_films": 74 + }, + { + "name@en": "Acid Western", + "n_films": 8 + }, + { + "name@en": "Action", + "n_films": 48 + }, + { + "name@en": "Action", + "n_films": 1 + }, + { + "name@en": "Action Comedy", + "n_films": 149 + }, + { + "name@en": "Action Film", + "n_films": 17719 + }, + { + "name@en": "Action Thriller", + "n_films": 588 + }, + { + "name@en": "Action/Adventure", + "n_films": 5731 + }, + { + "name@en": "Adevnture", + "n_films": 1 + }, + { + "name@en": "Adult", + "n_films": 5 + }, + { + "name@en": "Adventure", + "n_films": 20 + }, + { + "name@en": "Adventure Comedy", + "n_films": 123 + }, + { + "name@en": "Adventure Film", + "n_films": 9367 + }, + { + "name@en": "Adventure game", + "n_films": 57 + }, + { + "name@en": "Adventure Travel", + "n_films": 8 + }, + { + "name@en": "African-American Comedies", + "n_films": 1 + }, + { + "name@en": "Ages 0-2", + "n_films": 2 + }, + { + "name@en": "Ages 2-4", + "n_films": 1 + }, + { + "name@en": "Ages 5-7", + "n_films": 1 + }, + { + "name@en": "Airplanes and airports", + "n_films": 30 + }, + { + "name@en": "Albinism in popular culture", + "n_films": 11 + }, + { + "name@en": "Alien Film", + "n_films": 97 + }, + { + "name@en": "Alien invasion", + "n_films": 4 + }, + { + "name@en": "Alien Sci-Fi", + "n_films": 2 + }, + { + "name@en": "Alternate history", + "n_films": 4 + }, + { + "name@en": "Alternative Health", + "n_films": 4 + }, + { + "name@en": "Amateur film", + "n_films": 1 + }, + { + "name@en": "Ambiance", + "n_films": 11 + }, + { + "name@en": "Animal", + "n_films": 54 + }, + { + "name@en": "Animal Picture", + "n_films": 193 + }, + { + "name@en": "Animated cartoon", + "n_films": 132 + }, + { + "name@en": "Animated documentary", + "n_films": 9 + }, + { + "name@en": "Animated Musical", + "n_films": 42 + }, + { + "name@en": "Animation", + "n_films": 12799 + }, + { + "name@en": "Anime", + "n_films": 1106 + }, + { + "name@en": "Anime Sci-Fi", + "n_films": 4 + }, + { + "name@en": "Anthology", + "n_films": 14 + }, + { + "name@en": "Anthology film", + "n_films": 2 + }, + { + "name@en": "Anti-McCarthyist", + "n_films": 1 + }, + { + "name@en": "Anti-war film", + "n_films": 43 + }, + { + "name@en": "Anti-war movement", + "n_films": 2 + }, + { + "name@en": "Arab cinema", + "n_films": 1 + }, + { + "name@en": "Archaeology", + "n_films": 8 + }, + { + "name@en": "Architecture & Design", + "n_films": 11 + }, + { + "name@en": "Archives and records", + "n_films": 5 + }, + { + "name@en": "Armchair Travel", + "n_films": 25 + }, + { + "name@en": "Art history", + "n_films": 18 + }, + { + "name@en": "Artistic gymnastics", + "n_films": 2 + }, + { + "name@en": "Arts and Culture", + "n_films": 395 + }, + { + "name@en": "Assamese cinema", + "n_films": 10 + }, + { + "name@en": "Auto racing", + "n_films": 210 + }, + { + "name@en": "Avant-garde", + "n_films": 152 + }, + { + "name@en": "B movie", + "n_films": 1059 + }, + { + "name@en": "B-boying", + "n_films": 2 + }, + { + "name@en": "B-Western", + "n_films": 86 + }, + { + "name@en": "Backstage", + "n_films": 1 + }, + { + "name@en": "Backstage Musical", + "n_films": 7246 + }, + { + "name@en": "Ballet", + "n_films": 10 + }, + { + "name@en": "BDSM", + "n_films": 3 + }, + { + "name@en": "Beach Film", + "n_films": 21 + }, + { + "name@en": "Beach party film", + "n_films": 3 + }, + { + "name@en": "Beauty & Fashion", + "n_films": 15 + }, + { + "name@en": "Bedroom farce", + "n_films": 1 + }, + { + "name@en": "Bengali Cinema", + "n_films": 854 + }, + { + "name@en": "Best of compilation", + "n_films": 1 + }, + { + "name@en": "Bhojpuri cinema", + "n_films": 4 + }, + { + "name@en": "Biker Film", + "n_films": 38 + }, + { + "name@en": "Biographical Documentaries", + "n_films": 27 + }, + { + "name@en": "Biographical film", + "n_films": 5968 + }, + { + "name@en": "Biography", + "n_films": 67 + }, + { + "name@en": "Biological Sciences", + "n_films": 15 + }, + { + "name@en": "Bisexual pornography", + "n_films": 12 + }, + { + "name@en": "Black comedy", + "n_films": 1313 + }, + { + "name@en": "Black-and-white", + "n_films": 7780 + }, + { + "name@en": "Blaxploitation film", + "n_films": 121 + }, + { + "name@en": "Bloopers & Candid Camera", + "n_films": 7 + }, + { + "name@en": "Bollywood", + "n_films": 5119 + }, + { + "name@en": "British Cinema", + "n_films": 10 + }, + { + "name@en": "British Empire Film", + "n_films": 21 + }, + { + "name@en": "British New Wave", + "n_films": 12 + }, + { + "name@en": "Bruceploitation", + "n_films": 4 + }, + { + "name@en": "Buddy cop film", + "n_films": 11 + }, + { + "name@en": "Buddy film", + "n_films": 357 + }, + { + "name@en": "Business", + "n_films": 2 + }, + { + "name@en": "Business", + "n_films": 17 + }, + { + "name@en": "C-Movie", + "n_films": 1 + }, + { + "name@en": "Camp", + "n_films": 2 + }, + { + "name@en": "Cannibal film", + "n_films": 1 + }, + { + "name@en": "Caper story", + "n_films": 96 + }, + { + "name@en": "Careers", + "n_films": 4 + }, + { + "name@en": "Cartoon", + "n_films": 374 + }, + { + "name@en": "Cavalry Film", + "n_films": 19 + }, + { + "name@en": "Chase Movie", + "n_films": 89 + }, + { + "name@en": "Chick flick", + "n_films": 2 + }, + { + "name@en": "Childhood Drama", + "n_films": 132 + }, + { + "name@en": "Children's Fantasy", + "n_films": 229 + }, + { + "name@en": "Children's Issues", + "n_films": 14 + }, + { + "name@en": "Children's television series", + "n_films": 1 + }, + { + "name@en": "Children's/Family", + "n_films": 884 + }, + { + "name@en": "Chinese Movies", + "n_films": 2446 + }, + { + "name@en": "Chivalric romance", + "n_films": 2 + }, + { + "name@en": "Christian film", + "n_films": 122 + }, + { + "name@en": "Christian film industry", + "n_films": 3 + }, + { + "name@en": "Christianity", + "n_films": 5 + }, + { + "name@en": "Christmas", + "n_films": 1 + }, + { + "name@en": "Christmas movie", + "n_films": 94 + }, + { + "name@en": "Christmas special", + "n_films": 3 + }, + { + "name@en": "Cin\u00e9ma v\u00e9rit\u00e9", + "n_films": 7 + }, + { + "name@en": "Clay animation", + "n_films": 7 + }, + { + "name@en": "Closeted", + "n_films": 1 + }, + { + "name@en": "Collage", + "n_films": 2 + }, + { + "name@en": "College life", + "n_films": 1 + }, + { + "name@en": "Color motion picture film", + "n_films": 2 + }, + { + "name@en": "Combat Films", + "n_films": 93 + }, + { + "name@en": "Comedy", + "n_films": 41070 + }, + { + "name@en": "Comedy of Errors", + "n_films": 182 + }, + { + "name@en": "Comedy of manners", + "n_films": 218 + }, + { + "name@en": "Comedy Thriller", + "n_films": 93 + }, + { + "name@en": "Comedy Western", + "n_films": 241 + }, + { + "name@en": "Comedy-drama", + "n_films": 2173 + }, + { + "name@en": "Coming of age", + "n_films": 781 + }, + { + "name@en": "Coming out", + "n_films": 13 + }, + { + "name@en": "Coming-of-age story", + "n_films": 277 + }, + { + "name@en": "Commedia all'italiana", + "n_films": 2 + }, + { + "name@en": "Commedia sexy all'italiana", + "n_films": 2 + }, + { + "name@en": "Compilation film", + "n_films": 15 + }, + { + "name@en": "Computer Animation", + "n_films": 197 + }, + { + "name@en": "Computers", + "n_films": 13 + }, + { + "name@en": "Concert", + "n_films": 4 + }, + { + "name@en": "Concert film", + "n_films": 3050 + }, + { + "name@en": "Conspiracy fiction", + "n_films": 1 + }, + { + "name@en": "Cooking & Food", + "n_films": 6 + }, + { + "name@en": "Costume Adventure", + "n_films": 80 + }, + { + "name@en": "Costume drama", + "n_films": 359 + }, + { + "name@en": "Costume Horror", + "n_films": 42 + }, + { + "name@en": "Courtroom Comedy", + "n_films": 6 + }, + { + "name@en": "Creature Film", + "n_films": 306 + }, + { + "name@en": "Crime", + "n_films": 224 + }, + { + "name@en": "Crime Comedy", + "n_films": 272 + }, + { + "name@en": "Crime Drama", + "n_films": 479 + }, + { + "name@en": "Crime Fiction", + "n_films": 13019 + }, + { + "name@en": "Crime film", + "n_films": 42 + }, + { + "name@en": "Crime Thriller", + "n_films": 2541 + }, + { + "name@en": "Cult film", + "n_films": 937 + }, + { + "name@en": "Culture & Society", + "n_films": 895 + }, + { + "name@en": "Current affairs", + "n_films": 1 + }, + { + "name@en": "Cyberpunk", + "n_films": 9 + }, + { + "name@en": "Cycling", + "n_films": 1 + }, + { + "name@en": "Czechoslovak New Wave", + "n_films": 6 + }, + { + "name@en": "Damsel in distress", + "n_films": 1 + }, + { + "name@en": "Dance film", + "n_films": 986 + }, + { + "name@en": "Demonic child", + "n_films": 5 + }, + { + "name@en": "Detective fiction", + "n_films": 520 + }, + { + "name@en": "Diet & Nutrition", + "n_films": 8 + }, + { + "name@en": "Direct-to-video", + "n_films": 51 + }, + { + "name@en": "Disaster Film", + "n_films": 318 + }, + { + "name@en": "Docmentary", + "n_films": 1 + }, + { + "name@en": "Docudrama", + "n_films": 477 + }, + { + "name@en": "Docufiction", + "n_films": 28 + }, + { + "name@en": "Documentary film", + "n_films": 31370 + }, + { + "name@en": "Documentation", + "n_films": 1 + }, + { + "name@en": "Dogme 95", + "n_films": 77 + }, + { + "name@en": "Dokumentarfilm", + "n_films": 1 + }, + { + "name@en": "Domaci", + "n_films": 1 + }, + { + "name@en": "Domestic Comedy", + "n_films": 179 + }, + { + "name@en": "Doomsday film", + "n_films": 79 + }, + { + "name@en": "Drama", + "n_films": 238 + }, + { + "name@en": "Drama", + "n_films": 76252 + }, + { + "name@en": "Dystopia", + "n_films": 62 + }, + { + "name@en": "Early Black Cinema", + "n_films": 7 + }, + { + "name@en": "East Asian cinema", + "n_films": 3747 + }, + { + "name@en": "Education", + "n_films": 26 + }, + { + "name@en": "Educational & Guidance", + "n_films": 1 + }, + { + "name@en": "Educational entertainment", + "n_films": 1 + }, + { + "name@en": "Educational film", + "n_films": 3852 + }, + { + "name@en": "Edupunk", + "n_films": 1 + }, + { + "name@en": "Ensemble Film", + "n_films": 328 + }, + { + "name@en": "Environment", + "n_films": 11 + }, + { + "name@en": "Environmental Science", + "n_films": 36 + }, + { + "name@en": "Epic film", + "n_films": 318 + }, + { + "name@en": "Epic Western", + "n_films": 16 + }, + { + "name@en": "Equestrianism", + "n_films": 1 + }, + { + "name@en": "Erotic Drama", + "n_films": 132 + }, + { + "name@en": "Erotic Science Fiction", + "n_films": 1 + }, + { + "name@en": "Erotic thriller", + "n_films": 471 + }, + { + "name@en": "Erotica", + "n_films": 719 + }, + { + "name@en": "Escape Film", + "n_films": 45 + }, + { + "name@en": "Essay Film", + "n_films": 25 + }, + { + "name@en": "Estonian animation", + "n_films": 3 + }, + { + "name@en": "Ethnofiction", + "n_films": 2 + }, + { + "name@en": "Ethnographic film", + "n_films": 4 + }, + { + "name@en": "Eurospy film", + "n_films": 1 + }, + { + "name@en": "Eurowestern", + "n_films": 5 + }, + { + "name@en": "Evil clown", + "n_films": 3 + }, + { + "name@en": "Existentialism", + "n_films": 16 + }, + { + "name@en": "Experimental film", + "n_films": 1563 + }, + { + "name@en": "Exploitation film", + "n_films": 43 + }, + { + "name@en": "Expressionism", + "n_films": 3 + }, + { + "name@en": "Extraterrestrial life", + "n_films": 1 + }, + { + "name@en": "Extreme Sports", + "n_films": 174 + }, + { + "name@en": "Fairy tale", + "n_films": 80 + }, + { + "name@en": "Family", + "n_films": 11267 + }, + { + "name@en": "Family", + "n_films": 11 + }, + { + "name@en": "Family Comedies", + "n_films": 3 + }, + { + "name@en": "Family Drama", + "n_films": 1024 + }, + { + "name@en": "Family-Oriented Adventure", + "n_films": 237 + }, + { + "name@en": "Fan film", + "n_films": 43 + }, + { + "name@en": "Fantasy", + "n_films": 466 + }, + { + "name@en": "Fantasy", + "n_films": 1 + }, + { + "name@en": "Fantasy", + "n_films": 5854 + }, + { + "name@en": "Fantasy", + "n_films": 1 + }, + { + "name@en": "Fantasy Adventure", + "n_films": 162 + }, + { + "name@en": "Fantasy Comedy", + "n_films": 171 + }, + { + "name@en": "Fantasy Drama", + "n_films": 15 + }, + { + "name@en": "Farce", + "n_films": 293 + }, + { + "name@en": "Fashion", + "n_films": 1 + }, + { + "name@en": "Feature film", + "n_films": 26 + }, + { + "name@en": "Female buddy film", + "n_films": 4 + }, + { + "name@en": "Feminist Film", + "n_films": 80 + }, + { + "name@en": "Fiction", + "n_films": 71 + }, + { + "name@en": "Fiction", + "n_films": 1 + }, + { + "name@en": "Fictional film", + "n_films": 43 + }, + { + "name@en": "Filipino Movies", + "n_films": 18 + }, + { + "name@en": "Film & Television History", + "n_films": 428 + }, + { + "name@en": "Film \u00e0 clef", + "n_films": 29 + }, + { + "name@en": "Film adaptation", + "n_films": 1448 + }, + { + "name@en": "Film noir", + "n_films": 757 + }, + { + "name@en": "Film-Opera", + "n_films": 5 + }, + { + "name@en": "Film-poem", + "n_films": 1 + }, + { + "name@en": "Filmed Play", + "n_films": 18 + }, + { + "name@en": "Films with live action and animation", + "n_films": 8 + }, + { + "name@en": "Finance & Investing", + "n_films": 7 + }, + { + "name@en": "Flash animation", + "n_films": 1 + }, + { + "name@en": "Foreign legion", + "n_films": 8 + }, + { + "name@en": "Found footage", + "n_films": 10 + }, + { + "name@en": "French New Wave", + "n_films": 1 + }, + { + "name@en": "Future history", + "n_films": 1 + }, + { + "name@en": "Future noir", + "n_films": 18 + }, + { + "name@en": "Game show", + "n_films": 4 + }, + { + "name@en": "Gangster Film", + "n_films": 426 + }, + { + "name@en": "Gay", + "n_films": 306 + }, + { + "name@en": "Gay & Lesbian Films", + "n_films": 3 + }, + { + "name@en": "Gay Interest", + "n_films": 300 + }, + { + "name@en": "Gay pornography", + "n_films": 10970 + }, + { + "name@en": "Gay Themed", + "n_films": 467 + }, + { + "name@en": "Gender Issues", + "n_films": 107 + }, + { + "name@en": "Giallo", + "n_films": 20 + }, + { + "name@en": "Glamorized Spy Film", + "n_films": 47 + }, + { + "name@en": "Goat gland film", + "n_films": 3 + }, + { + "name@en": "Gothic Film", + "n_films": 52 + }, + { + "name@en": "Graphic & Applied Arts", + "n_films": 32 + }, + { + "name@en": "Gross out", + "n_films": 45 + }, + { + "name@en": "Gujarati cinema", + "n_films": 1 + }, + { + "name@en": "Gulf War", + "n_films": 4 + }, + { + "name@en": "Hagiography", + "n_films": 26 + }, + { + "name@en": "Hard rock", + "n_films": 1 + }, + { + "name@en": "Hardcore pornography", + "n_films": 278 + }, + { + "name@en": "Hardcore punk", + "n_films": 1 + }, + { + "name@en": "Haunted House Film", + "n_films": 66 + }, + { + "name@en": "Health & Fitness", + "n_films": 218 + }, + { + "name@en": "Heaven-Can-Wait Fantasies", + "n_films": 24 + }, + { + "name@en": "Heavenly Comedy", + "n_films": 30 + }, + { + "name@en": "Heavenly Drama", + "n_films": 2 + }, + { + "name@en": "Heavy metal", + "n_films": 1 + }, + { + "name@en": "Heist film", + "n_films": 203 + }, + { + "name@en": "Hentai", + "n_films": 35 + }, + { + "name@en": "Hip hop film", + "n_films": 770 + }, + { + "name@en": "Historical drama", + "n_films": 1038 + }, + { + "name@en": "Historical Epic", + "n_films": 64 + }, + { + "name@en": "Historical fiction", + "n_films": 2095 + }, + { + "name@en": "Historical period drama", + "n_films": 1404 + }, + { + "name@en": "History", + "n_films": 2272 + }, + { + "name@en": "Hobbies and interests", + "n_films": 167 + }, + { + "name@en": "Holiday", + "n_films": 4 + }, + { + "name@en": "Holiday Film", + "n_films": 97 + }, + { + "name@en": "Home movies", + "n_films": 2 + }, + { + "name@en": "Horor", + "n_films": 1 + }, + { + "name@en": "Horror", + "n_films": 10511 + }, + { + "name@en": "Horror comedy", + "n_films": 794 + }, + { + "name@en": "Human sexuality", + "n_films": 5 + }, + { + "name@en": "Hybrid Western", + "n_films": 41 + }, + { + "name@en": "Illnesses & Disabilities", + "n_films": 65 + }, + { + "name@en": "Incest pornography", + "n_films": 2 + }, + { + "name@en": "Independent Dramas", + "n_films": 2 + }, + { + "name@en": "Indian Movies", + "n_films": 8 + }, + { + "name@en": "Indian Western", + "n_films": 26 + }, + { + "name@en": "Indie film", + "n_films": 8644 + }, + { + "name@en": "Informational", + "n_films": 6 + }, + { + "name@en": "Inspirational Drama", + "n_films": 39 + }, + { + "name@en": "inspirational film", + "n_films": 1 + }, + { + "name@en": "Instrumental Music", + "n_films": 56 + }, + { + "name@en": "International", + "n_films": 1 + }, + { + "name@en": "Interpersonal Relationships", + "n_films": 55 + }, + { + "name@en": "Inventions & Innovations", + "n_films": 16 + }, + { + "name@en": "Japanese", + "n_films": 1 + }, + { + "name@en": "Japanese horror", + "n_films": 3 + }, + { + "name@en": "Japanese Movies", + "n_films": 3257 + }, + { + "name@en": "Jazz & Easy Listening", + "n_films": 1 + }, + { + "name@en": "Jazz & Modern Dance", + "n_films": 16 + }, + { + "name@en": "Jidaigeki", + "n_films": 9 + }, + { + "name@en": "Journalism", + "n_films": 9 + }, + { + "name@en": "Jukebox musical", + "n_films": 6 + }, + { + "name@en": "Jungle Film", + "n_films": 66 + }, + { + "name@en": "Juvenile Delinquency Film", + "n_films": 49 + }, + { + "name@en": "Kafkaesque", + "n_films": 3 + }, + { + "name@en": "Kaiju", + "n_films": 7 + }, + { + "name@en": "Kannada", + "n_films": 10 + }, + { + "name@en": "Kids' TV", + "n_films": 3 + }, + { + "name@en": "Kitchen sink realism", + "n_films": 15 + }, + { + "name@en": "Korean drama", + "n_films": 109 + }, + { + "name@en": "Korean horror", + "n_films": 3 + }, + { + "name@en": "Language & Literature", + "n_films": 48 + }, + { + "name@en": "Language Instruction", + "n_films": 1 + }, + { + "name@en": "Latin American cinema", + "n_films": 13 + }, + { + "name@en": "Latino", + "n_films": 2 + }, + { + "name@en": "Law & Crime", + "n_films": 84 + }, + { + "name@en": "Legal drama", + "n_films": 9 + }, + { + "name@en": "Leisure Arts", + "n_films": 15 + }, + { + "name@en": "LGBT", + "n_films": 3605 + }, + { + "name@en": "Libraries and librarians", + "n_films": 5 + }, + { + "name@en": "Lifestyle", + "n_films": 1 + }, + { + "name@en": "Linguistics", + "n_films": 5 + }, + { + "name@en": "Literary Studies", + "n_films": 33 + }, + { + "name@en": "Live action", + "n_films": 15 + }, + { + "name@en": "Lost film", + "n_films": 17 + }, + { + "name@en": "Mafia", + "n_films": 1 + }, + { + "name@en": "Mafia comedy", + "n_films": 3 + }, + { + "name@en": "Magic realism", + "n_films": 4 + }, + { + "name@en": "Malayalam Cinema", + "n_films": 2155 + }, + { + "name@en": "Male bonding", + "n_films": 2 + }, + { + "name@en": "Marathi cinema", + "n_films": 286 + }, + { + "name@en": "Marriage & Commitment", + "n_films": 10 + }, + { + "name@en": "Marriage Drama", + "n_films": 160 + }, + { + "name@en": "Martial Arts Film", + "n_films": 1944 + }, + { + "name@en": "Masala film", + "n_films": 6 + }, + { + "name@en": "Mashup", + "n_films": 1 + }, + { + "name@en": "Mecha", + "n_films": 21 + }, + { + "name@en": "Mecha anime and manga", + "n_films": 3 + }, + { + "name@en": "Media Satire", + "n_films": 71 + }, + { + "name@en": "Media studies", + "n_films": 45 + }, + { + "name@en": "Medical fiction", + "n_films": 74 + }, + { + "name@en": "Medieval fantasy", + "n_films": 1 + }, + { + "name@en": "Melodrama", + "n_films": 1206 + }, + { + "name@en": "Metalcore", + "n_films": 2 + }, + { + "name@en": "Miniseries", + "n_films": 1 + }, + { + "name@en": "Mistery", + "n_films": 1 + }, + { + "name@en": "Mockbuster", + "n_films": 13 + }, + { + "name@en": "Mockumentary", + "n_films": 280 + }, + { + "name@en": "Mondo film", + "n_films": 11 + }, + { + "name@en": "Monster", + "n_films": 104 + }, + { + "name@en": "Monster movie", + "n_films": 252 + }, + { + "name@en": "Montage", + "n_films": 1 + }, + { + "name@en": "Motion capture", + "n_films": 1 + }, + { + "name@en": "Motorsport", + "n_films": 15 + }, + { + "name@en": "Movies About Gladiators", + "n_films": 2 + }, + { + "name@en": "Multimedia", + "n_films": 2 + }, + { + "name@en": "Mumblecore", + "n_films": 8 + }, + { + "name@en": "Murder-mystery", + "n_films": 5 + }, + { + "name@en": "Museums and art galleries", + "n_films": 1 + }, + { + "name@en": "Music", + "n_films": 8705 + }, + { + "name@en": "Music video", + "n_films": 183 + }, + { + "name@en": "Musical comedy", + "n_films": 2681 + }, + { + "name@en": "Musical Drama", + "n_films": 2841 + }, + { + "name@en": "Musical theatre", + "n_films": 2 + }, + { + "name@en": "Mystery", + "n_films": 6520 + }, + { + "name@en": "Mystery film", + "n_films": 23 + }, + { + "name@en": "Mythic fiction", + "n_films": 5 + }, + { + "name@en": "Mythological Fantasy", + "n_films": 23 + }, + { + "name@en": "Natural horror film", + "n_films": 153 + }, + { + "name@en": "Nature", + "n_films": 73 + }, + { + "name@en": "Nazi exploitation", + "n_films": 4 + }, + { + "name@en": "Neo-noir", + "n_films": 58 + }, + { + "name@en": "New Queer Cinema", + "n_films": 1 + }, + { + "name@en": "News", + "n_films": 76 + }, + { + "name@en": "Newsreel", + "n_films": 7 + }, + { + "name@en": "Ninja movie", + "n_films": 1 + }, + { + "name@en": "No Wave Cinema", + "n_films": 1 + }, + { + "name@en": "Non-fiction", + "n_films": 40 + }, + { + "name@en": "Northern", + "n_films": 48 + }, + { + "name@en": "Nuclear warfare", + "n_films": 1 + }, + { + "name@en": "Nudie film", + "n_films": 1 + }, + { + "name@en": "Nunsploitation", + "n_films": 14 + }, + { + "name@en": "Opera", + "n_films": 51 + }, + { + "name@en": "Operetta", + "n_films": 6 + }, + { + "name@en": "Operetta film", + "n_films": 5 + }, + { + "name@en": "Ostern", + "n_films": 5 + }, + { + "name@en": "Outdoor & Mountain Sports", + "n_films": 8 + }, + { + "name@en": "Outlaw", + "n_films": 2 + }, + { + "name@en": "Paracinema", + "n_films": 1 + }, + { + "name@en": "Paranormal", + "n_films": 6 + }, + { + "name@en": "Parody", + "n_films": 1229 + }, + { + "name@en": "Parody film", + "n_films": 19 + }, + { + "name@en": "Part-talkie", + "n_films": 5 + }, + { + "name@en": "Patriotic film", + "n_films": 3 + }, + { + "name@en": "Period Horror", + "n_films": 1 + }, + { + "name@en": "Pink film", + "n_films": 53 + }, + { + "name@en": "Point of view shot", + "n_films": 2 + }, + { + "name@en": "Police", + "n_films": 1 + }, + { + "name@en": "Police Comedy", + "n_films": 26 + }, + { + "name@en": "Police procedural", + "n_films": 2 + }, + { + "name@en": "Political cinema", + "n_films": 855 + }, + { + "name@en": "Political Documetary", + "n_films": 2 + }, + { + "name@en": "Political drama", + "n_films": 1386 + }, + { + "name@en": "Political fiction", + "n_films": 4 + }, + { + "name@en": "Political satire", + "n_films": 73 + }, + { + "name@en": "Political thriller", + "n_films": 282 + }, + { + "name@en": "Politics", + "n_films": 1 + }, + { + "name@en": "Poliziotteschi", + "n_films": 42 + }, + { + "name@en": "Pop rock", + "n_films": 1 + }, + { + "name@en": "Pornochanchada", + "n_films": 1 + }, + { + "name@en": "Pornographic film", + "n_films": 6516 + }, + { + "name@en": "Pornography", + "n_films": 2 + }, + { + "name@en": "Post-Apocalyptic fiction", + "n_films": 30 + }, + { + "name@en": "Prison escape", + "n_films": 2 + }, + { + "name@en": "Prison film", + "n_films": 143 + }, + { + "name@en": "Propaganda film", + "n_films": 221 + }, + { + "name@en": "Psycho film", + "n_films": 1 + }, + { + "name@en": "Psycho-biddy", + "n_films": 13 + }, + { + "name@en": "Psychological Drama", + "n_films": 47 + }, + { + "name@en": "Psychological horror", + "n_films": 21 + }, + { + "name@en": "Psychological thriller", + "n_films": 1746 + }, + { + "name@en": "Public information film", + "n_films": 14 + }, + { + "name@en": "Punk rock", + "n_films": 395 + }, + { + "name@en": "Puppetoon", + "n_films": 1 + }, + { + "name@en": "Race film", + "n_films": 13 + }, + { + "name@en": "Rape and revenge film", + "n_films": 2 + }, + { + "name@en": "Real Robot", + "n_films": 2 + }, + { + "name@en": "Reality film", + "n_films": 4 + }, + { + "name@en": "Reality television", + "n_films": 30 + }, + { + "name@en": "Reboot", + "n_films": 5 + }, + { + "name@en": "Religion", + "n_films": 21 + }, + { + "name@en": "Religious Film", + "n_films": 170 + }, + { + "name@en": "Remake", + "n_films": 53 + }, + { + "name@en": "Revenge", + "n_films": 5 + }, + { + "name@en": "Revisionist Fairy Tale", + "n_films": 1 + }, + { + "name@en": "Revisionist Western", + "n_films": 68 + }, + { + "name@en": "Road movie", + "n_films": 399 + }, + { + "name@en": "Road-Horror", + "n_films": 16 + }, + { + "name@en": "Roadshow theatrical release", + "n_films": 24 + }, + { + "name@en": "Roadshow/Carny", + "n_films": 4 + }, + { + "name@en": "Rock & Pop", + "n_films": 1 + }, + { + "name@en": "Rockumentary", + "n_films": 1077 + }, + { + "name@en": "Romance", + "n_films": 4 + }, + { + "name@en": "Romance", + "n_films": 39 + }, + { + "name@en": "Romance Film", + "n_films": 21094 + }, + { + "name@en": "Romantic comedy", + "n_films": 3602 + }, + { + "name@en": "Romantic Drama", + "n_films": 1 + }, + { + "name@en": "Romantic fantasy", + "n_films": 67 + }, + { + "name@en": "Romantic Movies", + "n_films": 4 + }, + { + "name@en": "Romantic thriller", + "n_films": 4 + }, + { + "name@en": "Rubble film", + "n_films": 1 + }, + { + "name@en": "Samurai cinema", + "n_films": 39 + }, + { + "name@en": "Satire", + "n_films": 713 + }, + { + "name@en": "School story", + "n_films": 5 + }, + { + "name@en": "Sci-Fi", + "n_films": 3 + }, + { + "name@en": "Sci-Fi Adventure", + "n_films": 31 + }, + { + "name@en": "Sci-Fi Horror", + "n_films": 157 + }, + { + "name@en": "Science", + "n_films": 1 + }, + { + "name@en": "Science and Nature Documentaries", + "n_films": 1 + }, + { + "name@en": "Science fantasy", + "n_films": 2 + }, + { + "name@en": "Science Fiction", + "n_films": 5851 + }, + { + "name@en": "Science fiction film", + "n_films": 20 + }, + { + "name@en": "Science fiction Western", + "n_films": 9 + }, + { + "name@en": "Screwball comedy", + "n_films": 345 + }, + { + "name@en": "Self-help", + "n_films": 13 + }, + { + "name@en": "Semidocumentary", + "n_films": 1 + }, + { + "name@en": "Sepia tone", + "n_films": 1 + }, + { + "name@en": "Serial film", + "n_films": 19 + }, + { + "name@en": "Sermons", + "n_films": 4 + }, + { + "name@en": "Sex comedy", + "n_films": 373 + }, + { + "name@en": "Sex Horror", + "n_films": 8 + }, + { + "name@en": "Sexploitation film", + "n_films": 142 + }, + { + "name@en": "Sh\u014djo manga", + "n_films": 1 + }, + { + "name@en": "Short Film", + "n_films": 46476 + }, + { + "name@en": "Short story", + "n_films": 1 + }, + { + "name@en": "Showbiz Dramas", + "n_films": 1 + }, + { + "name@en": "Silent comedy", + "n_films": 7 + }, + { + "name@en": "Silent film", + "n_films": 6955 + }, + { + "name@en": "Singing cowboy", + "n_films": 5 + }, + { + "name@en": "Sketch comedy", + "n_films": 1 + }, + { + "name@en": "Slapstick", + "n_films": 614 + }, + { + "name@en": "Slasher", + "n_films": 1198 + }, + { + "name@en": "Slice of life", + "n_films": 96 + }, + { + "name@en": "Snuff film", + "n_films": 1 + }, + { + "name@en": "Social & Cultural Documentaries", + "n_films": 5 + }, + { + "name@en": "Social commentary", + "n_films": 1 + }, + { + "name@en": "Social film", + "n_films": 5 + }, + { + "name@en": "Social guidance film", + "n_films": 16 + }, + { + "name@en": "Social issues", + "n_films": 3 + }, + { + "name@en": "Social problem film", + "n_films": 155 + }, + { + "name@en": "Softcore pornography", + "n_films": 246 + }, + { + "name@en": "South Asian cinema", + "n_films": 358 + }, + { + "name@en": "Southeast Asian cinema", + "n_films": 7 + }, + { + "name@en": "Southern Gothic", + "n_films": 1 + }, + { + "name@en": "Space opera", + "n_films": 4 + }, + { + "name@en": "Space Western", + "n_films": 12 + }, + { + "name@en": "Spaghetti Western", + "n_films": 153 + }, + { + "name@en": "Special Interest", + "n_films": 41 + }, + { + "name@en": "Spiritual film", + "n_films": 297 + }, + { + "name@en": "Spirituality", + "n_films": 1 + }, + { + "name@en": "Splatter film", + "n_films": 35 + }, + { + "name@en": "Sponsored film", + "n_films": 9 + }, + { + "name@en": "Sports", + "n_films": 111 + }, + { + "name@en": "Sports films", + "n_films": 6114 + }, + { + "name@en": "Spy fiction", + "n_films": 1 + }, + { + "name@en": "Spy film", + "n_films": 554 + }, + { + "name@en": "Stand-up comedy", + "n_films": 686 + }, + { + "name@en": "Star vehicle", + "n_films": 1 + }, + { + "name@en": "Steampunk", + "n_films": 19 + }, + { + "name@en": "Stoner film", + "n_films": 46 + }, + { + "name@en": "Stop motion", + "n_films": 128 + }, + { + "name@en": "Structural film", + "n_films": 1 + }, + { + "name@en": "Superhero movie", + "n_films": 306 + }, + { + "name@en": "Supermarionation", + "n_films": 2 + }, + { + "name@en": "Supernatural", + "n_films": 633 + }, + { + "name@en": "Supernatural Horror", + "n_films": 4 + }, + { + "name@en": "Surf film", + "n_films": 22 + }, + { + "name@en": "Surrealism", + "n_films": 97 + }, + { + "name@en": "Suspense", + "n_films": 993 + }, + { + "name@en": "Swashbuckler film", + "n_films": 137 + }, + { + "name@en": "Sword and Sandal film", + "n_films": 53 + }, + { + "name@en": "Sword and sorcery", + "n_films": 61 + }, + { + "name@en": "Talk show", + "n_films": 8 + }, + { + "name@en": "Talkie", + "n_films": 2 + }, + { + "name@en": "Tamil cinema", + "n_films": 2817 + }, + { + "name@en": "Tearjerkers", + "n_films": 1 + }, + { + "name@en": "Technicolor", + "n_films": 6 + }, + { + "name@en": "Techno-thriller", + "n_films": 1 + }, + { + "name@en": "Teen drama", + "n_films": 4 + }, + { + "name@en": "Teen film", + "n_films": 1047 + }, + { + "name@en": "Teen pornography", + "n_films": 3 + }, + { + "name@en": "Teen Screams", + "n_films": 1 + }, + { + "name@en": "Teleplay", + "n_films": 2 + }, + { + "name@en": "Television advertisement", + "n_films": 4 + }, + { + "name@en": "Television documentary", + "n_films": 93 + }, + { + "name@en": "Television film", + "n_films": 1414 + }, + { + "name@en": "Television Film", + "n_films": 1 + }, + { + "name@en": "Television pilot", + "n_films": 6 + }, + { + "name@en": "Terrorism", + "n_films": 1 + }, + { + "name@en": "Theatrical Performances", + "n_films": 1 + }, + { + "name@en": "Therimin music", + "n_films": 3 + }, + { + "name@en": "Thrash metal", + "n_films": 1 + }, + { + "name@en": "Thriller", + "n_films": 17716 + }, + { + "name@en": "Time travel", + "n_films": 67 + }, + { + "name@en": "Tokusatsu", + "n_films": 4 + }, + { + "name@en": "Tollywood", + "n_films": 1868 + }, + { + "name@en": "Tragicomedy", + "n_films": 76 + }, + { + "name@en": "Training film", + "n_films": 7 + }, + { + "name@en": "Transforming robots", + "n_films": 1 + }, + { + "name@en": "Transsexualism", + "n_films": 2 + }, + { + "name@en": "Travel", + "n_films": 683 + }, + { + "name@en": "Travel & Adventure Documentaries", + "n_films": 1 + }, + { + "name@en": "Travel documentary", + "n_films": 5 + }, + { + "name@en": "Trial drama", + "n_films": 300 + }, + { + "name@en": "Tribute", + "n_films": 1 + }, + { + "name@en": "Triller", + "n_films": 1 + }, + { + "name@en": "True crime", + "n_films": 60 + }, + { + "name@en": "Tutorial", + "n_films": 1 + }, + { + "name@en": "TV Documentaries", + "n_films": 1 + }, + { + "name@en": "Underground film", + "n_films": 6 + }, + { + "name@en": "Vampire film", + "n_films": 18 + }, + { + "name@en": "Variety show", + "n_films": 1 + }, + { + "name@en": "Vigilante film", + "n_films": 1 + }, + { + "name@en": "Viral video", + "n_films": 1 + }, + { + "name@en": "War", + "n_films": 20 + }, + { + "name@en": "War effort", + "n_films": 1 + }, + { + "name@en": "War film", + "n_films": 5288 + }, + { + "name@en": "Web documentary", + "n_films": 2 + }, + { + "name@en": "Webisode", + "n_films": 2 + }, + { + "name@en": "Werewolf fiction", + "n_films": 4 + }, + { + "name@en": "Western", + "n_films": 4099 + }, + { + "name@en": "Whodunit", + "n_films": 87 + }, + { + "name@en": "Women in prison film", + "n_films": 13 + }, + { + "name@en": "Workplace Comedy", + "n_films": 86 + }, + { + "name@en": "World cinema", + "n_films": 21668 + }, + { + "name@en": "World history", + "n_films": 27 + }, + { + "name@en": "Wrestling", + "n_films": 1 + }, + { + "name@en": "Wuxia", + "n_films": 110 + }, + { + "name@en": "Yakuza film", + "n_films": 22 + }, + { + "name@en": "Youth", + "n_films": 1 + }, + { + "name@en": "Yuri", + "n_films": 2 + }, + { + "name@en": "Z movie", + "n_films": 2 + }, + { + "name@en": "Zapata Western", + "n_films": 1 + }, + { + "name@en": "Zombie", + "n_films": 4 + }, + { + "name@en": "Zombie", + "n_films": 8 + }, + { + "name@en": "Zombie comedy", + "n_films": 3 + }, + { + "name@en": "Zombie Film", + "n_films": 306 + }, + { + "name@en": "Zombie Movies", + "n_films": 2 + }, + { + "n_films": 2 + } + ] +} From 9916d7b62ab77f6ea2bfc8f741e7de14cb3c8494 Mon Sep 17 00:00:00 2001 From: Matthew McNeely Date: Mon, 4 May 2026 15:33:01 -0400 Subject: [PATCH 15/17] fix(pipeline): bg-indexed star-delete + scalar Set data-list corruption Two related bugs in the per-predicate mutation pipeline that surface as stale index entries under TestStringIndex (systest/bgindex): 1. Process() (the star-delete fast path) was passing the caller's bare context to handleDeleteAll. schema.State().IsIndexed / IsReversed / HasCount only consult the pending mutSchema (the new schema with the index being built) when the context carries the isWrite flag. The legacy runMutation path calls schema.GetWriteContext at the top; the pipeline didn't. During a background index build, every star-delete saw isIndexed=false and skipped addIndexMutations(DEL) for the prior value, leaving stale uids in the index permanently. Lifting GetWriteContext into Process covers both the star-delete path and the later predicate-pipeline goroutines. 2. ProcessSingle's data-list write was passing the unfiltered postings list to AddDelta. handleOldDeleteForSingle appends a synthetic Del(oldVal) alongside the user's Set(newVal) so InsertTokenizerIndexes and ProcessReverse can emit Del-of-old entries. For scalar non-list non-lang predicates both postings share Uid == math.MaxUint64, and at read time pickPostings' equal-ts tie-break falls back to Go's unstable sort.Slice while setMutationAfterCommit overwrites committedUids[mpost.Uid] in append order. Either way Del can clobber the new Set and the data list reads as "no value." ProcessCount already strips the synthetic Del via skipSyntheticDel; mirror that filter in ProcessSingle's main data-list path. systest/bgindex now passes (TestStringIndex, TestReverseIndex, TestCountIndex, TestParallelIndexing). Co-Authored-By: Claude Opus 4.7 (1M context) --- posting/index.go | 42 +++++++++++++++++++++++++++++++++++++++++- 1 file changed, 41 insertions(+), 1 deletion(-) diff --git a/posting/index.go b/posting/index.go index 779ed1775b9..7719d87462c 100644 --- a/posting/index.go +++ b/posting/index.go @@ -738,6 +738,17 @@ func (mp *MutationPipeline) ProcessSingle(ctx context.Context, pipeline *Predica } baseKey := string(dataKey[:len(dataKey)-8]) // Avoid repeated conversion + // handleOldDeleteForSingle may have appended a synthetic Del-of-old-value + // alongside the user's Set so InsertTokenizerIndexes / ProcessReverse can + // emit index/reverse Del entries for the prior value. The synthetic Del + // must NOT be written to the scalar data list: scalar value postings all + // share Uid == math.MaxUint64, and at read time pickPostings sorts the two + // equal-ts postings non-deterministically (Go's sort.Slice is unstable), + // while setMutationAfterCommit's committedUids[mpost.Uid] = mpost + // overwrites in append order. Either way Del can clobber the new Set, so + // the data list reads as "no value." Strip the synthetic Del here before + // committing the data delta — same logic ProcessCount uses internally. + stripSyntheticDel := info.index || info.reverse for uid, pl := range postings { binary.BigEndian.PutUint64(dataKey[len(dataKey)-8:], uid) @@ -747,7 +758,28 @@ func (mp *MutationPipeline) ProcessSingle(ctx context.Context, pipeline *Predica mp.txn.addConflictKey(farm.Fingerprint64([]byte(key))) } - if _, err := mp.txn.AddDelta(key, pl, false, false); err != nil { + writePl := pl + if stripSyntheticDel && len(pl.Postings) > 1 { + hasSet := false + for _, post := range pl.Postings { + if post.Op == Set || post.Op == Ovr { + hasSet = true + break + } + } + if hasSet { + filtered := &pb.PostingList{} + for _, post := range pl.Postings { + if post.Op == Del { + continue + } + filtered.Postings = append(filtered.Postings, post) + } + writePl = filtered + } + } + + if _, err := mp.txn.AddDelta(key, writePl, false, false); err != nil { return err } } @@ -950,6 +982,14 @@ func ValidateAndConvert(edge *pb.DirectedEdge, su *pb.SchemaUpdate) error { } func (mp *MutationPipeline) Process(ctx context.Context, edges []*pb.DirectedEdge) error { + // handleDeleteAll calls schema.State().IsIndexed/IsReversed/HasCount, all of + // which only consult the pending mutSchema (the new schema with the index + // being built) when the context carries the isWrite flag. The legacy path + // in runMutation already calls schema.GetWriteContext at the top; the + // pipeline must do the same here, otherwise during a background index + // build a star-delete won't generate the index Del entries for the prior + // value, leaving stale uids in the index permanently. + ctx = schema.GetWriteContext(ctx) predicates := map[string]*PredicatePipeline{} var wg sync.WaitGroup numWg := 0 From f999230c884358941ad0a87e823e095579fb8b6e Mon Sep 17 00:00:00 2001 From: Matthew McNeely Date: Mon, 4 May 2026 19:53:13 -0400 Subject: [PATCH 16/17] fix(pipeline): serialize ProcessVectorIndex to prevent HNSW corruption MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit HNSW Insert performs multi-key read-modify-write across the entry-pointer key, the per-level edge lists, and back-edges into the neighbors of the new node. Per-list locks make each key's update atomic, but the cross-key sequences ("read entry, lock entry, read neighbor, modify neighbor, append back-edge to neighbor's list…") are not. Running 10 worker goroutines concurrently against the same txn cache lets updates stomp on each other, leaving nodes that have a data-list entry but are unreachable from the entry point. Surface symptom: similar_to(k=N) returns fewer than N hits even though all N vectors committed to the data list. TestVectorTwoTxnWithoutCommit reliably reproduced this on Linux CI (and not on the macOS dev box, likely a scheduling artifact). Legacy applyMutations arrives at single-threaded vector handling indirectly via x.DivideAndRule: for any num < 256 the helper rounds numGo down to 1, so a 5-edge vector mutation runs serially on main and the bug never surfaces there. Mirror that here by dropping numThreads to 1 — correctness over within-txn parallelism for vector predicates. Cross-txn parallelism is unaffected. Co-Authored-By: Claude Opus 4.7 (1M context) --- posting/index.go | 46 ++++++++++++++++++++++------------------------ 1 file changed, 22 insertions(+), 24 deletions(-) diff --git a/posting/index.go b/posting/index.go index 7719d87462c..d17e09bb680 100644 --- a/posting/index.go +++ b/posting/index.go @@ -100,31 +100,29 @@ func (pp *PredicatePipeline) close() { } func (mp *MutationPipeline) ProcessVectorIndex(ctx context.Context, pipeline *PredicatePipeline, info predicateInfo) error { - var wg errgroup.Group - numThreads := 10 - - for i := 0; i < numThreads; i++ { - wg.Go(func() error { - for edge := range pipeline.edges { - uid := edge.Entity - - key := x.DataKey(pipeline.attr, uid) - pl, err := mp.txn.Get(key) - if err != nil { - return err - } - if err := pl.AddMutationWithIndex(ctx, edge, mp.txn); err != nil { - return err - } - } - return nil - }) - } - - if err := wg.Wait(); err != nil { - return err + // Vector inserts must run serially within a txn. HNSW Insert does + // multi-key read-modify-write across the entry-pointer key, level edge + // lists, and neighbor back-edges; per-list locks make individual key + // updates atomic but the cross-key sequences are not. Concurrent + // inserters on the same txn cache lose updates and leave nodes + // unreachable from the entry point — visible as similar_to(k=N) returning + // fewer than N hits even though all N vectors are in the data list. The + // legacy applyMutations path arrives at the same result indirectly: + // x.DivideAndRule rounds num<256 down to numGo=1, so any vector mutation + // short of a few hundred edges runs single-threaded on main. Match that + // here. (TestVectorTwoTxnWithoutCommit reliably reproduced the corruption + // on Linux CI when this used 10 workers.) + for edge := range pipeline.edges { + uid := edge.Entity + key := x.DataKey(pipeline.attr, uid) + pl, err := mp.txn.Get(key) + if err != nil { + return err + } + if err := pl.AddMutationWithIndex(ctx, edge, mp.txn); err != nil { + return err + } } - return nil } From 97609b3a9b971954cb9e9276d9210cf3c1123927 Mon Sep 17 00:00:00 2001 From: Matthew McNeely Date: Mon, 4 May 2026 21:01:02 -0400 Subject: [PATCH 17/17] chore: add description of pipeline feature flag --- dgraph/cmd/alpha/run.go | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/dgraph/cmd/alpha/run.go b/dgraph/cmd/alpha/run.go index d26a5186180..0a20084e303 100644 --- a/dgraph/cmd/alpha/run.go +++ b/dgraph/cmd/alpha/run.go @@ -274,6 +274,16 @@ they form a Raft group and provide synchronous replication. "with structured fields including trace ID for correlation with distributed traces. "+ "Disabled by default (0). Note: enabling this logs query text which may contain "+ "sensitive data; do not enable in deployments with strict data privacy requirements."). + Flag("mutations-pipeline-threshold", + "Per-mutation edge-count threshold for routing through the per-predicate "+ + "mutation pipeline, which parallelizes work across distinct predicates "+ + "in a single mutation. 0 disables the pipeline entirely (all mutations "+ + "take the legacy path). 1 always uses the pipeline. A value N>1 uses "+ + "the pipeline only when len(edges) >= N, leaving small interactive "+ + "mutations on the legacy path — the pipeline pays per-predicate "+ + "goroutine spin-up cost, so tiny mutations are slightly slower on it; "+ + "bulk multi-predicate mutations are faster (crossover ~100 edges in "+ + "benchmarks here)."). String()) }