From 46eaef47db8db7fe237c4a5f2bf670cd8f988cb1 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Fri, 13 Jun 2025 03:21:49 +0530 Subject: [PATCH 01/52] perf(vector): Improve hnsw by sharding vectors --- posting/index.go | 201 ++++++++++++++++++++++++++++++++- tok/hnsw/persistent_factory.go | 21 ++-- tok/hnsw/persistent_hnsw.go | 41 +++++++ tok/index/index.go | 7 +- tok/index_factory.go | 14 ++- worker/task.go | 40 ++++--- 6 files changed, 293 insertions(+), 31 deletions(-) diff --git a/posting/index.go b/posting/index.go index 87b1bcfd015..abb98ff4233 100644 --- a/posting/index.go +++ b/posting/index.go @@ -15,6 +15,7 @@ import ( "math" "os" "strings" + "sync" "sync/atomic" "time" "unsafe" @@ -33,8 +34,11 @@ import ( "github.com/hypermodeinc/dgraph/v25/schema" "github.com/hypermodeinc/dgraph/v25/tok" "github.com/hypermodeinc/dgraph/v25/tok/hnsw" + "github.com/hypermodeinc/dgraph/v25/tok/index" "github.com/hypermodeinc/dgraph/v25/types" "github.com/hypermodeinc/dgraph/v25/x" + + "github.com/viterin/vek/vek32" ) var emptyCountParams countParams @@ -162,7 +166,7 @@ func (txn *Txn) addIndexMutations(ctx context.Context, info *indexMutationInfo) // retrieve vector from inUuid save as inVec inVec := types.BytesAsFloatArray(data[0].Value.([]byte)) tc := hnsw.NewTxnCache(NewViTxn(txn), txn.StartTs) - indexer, err := info.factorySpecs[0].CreateIndex(attr) + indexer, err := info.factorySpecs[0].CreateIndex(attr, 0) if err != nil { return []*pb.DirectedEdge{}, err } @@ -1361,6 +1365,198 @@ func (rb *indexRebuildInfo) prefixesForTokIndexes() ([][]byte, error) { return prefixes, nil } +type vectorCentroids struct { + dimension int + numCenters int + + centroids [][]float32 + counts []int64 + weights [][]float32 + mutexs []*sync.Mutex +} + +func (vc *vectorCentroids) findCentroid(input []float32) int { + minIdx := 0 + minDist := math.MaxFloat32 + for i, centroid := range vc.centroids { + dist := vek32.Distance(centroid, input) + if float64(dist) < minDist { + minDist = float64(dist) + minIdx = i + } + } + return minIdx +} + +func (vc *vectorCentroids) addVector(vec []float32) { + idx := vc.findCentroid(vec) + vc.mutexs[idx].Lock() + defer vc.mutexs[idx].Unlock() + for i := 0; i < vc.dimension; i++ { + vc.weights[idx][i] += vec[i] + } + vc.counts[idx]++ +} + +func (vc *vectorCentroids) updateCentroids() { + for i := 0; i < vc.numCenters; i++ { + for j := 0; j < vc.dimension; j++ { + vc.centroids[i][j] = vc.weights[i][j] / float32(vc.counts[i]) + vc.weights[i][j] = 0 + } + fmt.Printf("%d, ", vc.counts[i]) + vc.counts[i] = 0 + } + fmt.Println() +} + +func (vc *vectorCentroids) randomInit() { + vc.dimension = len(vc.centroids[0]) + vc.numCenters = len(vc.centroids) + vc.centroids = make([][]float32, vc.numCenters) + vc.counts = make([]int64, vc.numCenters) + vc.weights = make([][]float32, vc.numCenters) + vc.mutexs = make([]*sync.Mutex, vc.numCenters) + for i := 0; i < vc.numCenters; i++ { + vc.weights[i] = make([]float32, vc.dimension) + vc.counts[i] = 0 + vc.mutexs[i] = &sync.Mutex{} + } +} + +func (vc *vectorCentroids) addSeedCentroid(vec []float32) { + vc.centroids = append(vc.centroids, vec) +} + +const numCentroids = 1000 + +func rebuildVectorIndex(ctx context.Context, factorySpecs []*tok.FactoryCreateSpec, rb *IndexRebuild) error { + pk := x.ParsedKey{Attr: rb.Attr} + vc := &vectorCentroids{} + + MemLayerInstance.IterateDisk(ctx, IterateDiskArgs{ + Prefix: pk.DataPrefix(), + ReadTs: rb.StartTs, + AllVersions: false, + Reverse: false, + CheckInclusion: func(uid uint64) error { + return nil + }, + Function: func(l *List, pk x.ParsedKey) error { + val, err := l.Value(rb.StartTs) + if err != nil { + return err + } + inVec := types.BytesAsFloatArray(val.Value.([]byte)) + vc.addSeedCentroid(inVec) + if len(vc.centroids) == numCentroids { + return ErrStopIteration + } + return nil + }, + StartKey: x.DataKey(rb.Attr, 0), + }) + + vc.randomInit() + + fmt.Println("Clustering Vectors") + for range 5 { + builder := rebuilder{attr: rb.Attr, prefix: pk.DataPrefix(), startTs: rb.StartTs} + builder.fn = func(uid uint64, pl *List, txn *Txn) ([]*pb.DirectedEdge, error) { + edges := []*pb.DirectedEdge{} + val, err := pl.Value(txn.StartTs) + if err != nil { + return []*pb.DirectedEdge{}, err + } + + inVec := types.BytesAsFloatArray(val.Value.([]byte)) + vc.addVector(inVec) + return edges, nil + } + + err := builder.RunWithoutTemp(ctx) + if err != nil { + return err + } + + vc.updateCentroids() + } + + tcs := make([]*hnsw.TxnCache, vc.numCenters) + txns := make([]*Txn, vc.numCenters) + indexers := make([]index.VectorIndex[float32], vc.numCenters) + for i := 0; i < vc.numCenters; i++ { + txns[i] = NewTxn(rb.StartTs) + tcs[i] = hnsw.NewTxnCache(NewViTxn(txns[i]), rb.StartTs) + indexers_i, err := factorySpecs[0].CreateIndex(pk.Attr, i) + if err != nil { + return err + } + vc.mutexs[i] = &sync.Mutex{} + indexers[i] = indexers_i + } + + var edgesCreated atomic.Int64 + + numPasses := vc.numCenters / 100 + for pass_idx := range numPasses { + builder := rebuilder{attr: rb.Attr, prefix: pk.DataPrefix(), startTs: rb.StartTs} + builder.fn = func(uid uint64, pl *List, txn *Txn) ([]*pb.DirectedEdge, error) { + val, err := pl.Value(txn.StartTs) + if err != nil { + return []*pb.DirectedEdge{}, err + } + + inVec := types.BytesAsFloatArray(val.Value.([]byte)) + idx := vc.findCentroid(inVec) + if idx%numPasses != pass_idx { + return []*pb.DirectedEdge{}, nil + } + vc.mutexs[idx].Lock() + defer vc.mutexs[idx].Unlock() + _, err = indexers[idx].Insert(ctx, tcs[idx], uid, inVec) + if err != nil { + return []*pb.DirectedEdge{}, err + } + + edgesCreated.Add(int64(1)) + return nil, nil + } + + err := builder.RunWithoutTemp(ctx) + if err != nil { + return err + } + + for idx := range vc.counts { + if idx%numPasses != pass_idx { + continue + } + txns[idx].Update() + writer := NewTxnWriter(pstore) + + x.ExponentialRetry(int(x.Config.MaxRetries), + 20*time.Millisecond, func() error { + err := txns[idx].CommitToDisk(writer, rb.StartTs) + if err == badger.ErrBannedKey { + glog.Errorf("Error while writing to banned namespace.") + return nil + } + return err + }) + + txns[idx].cache.plists = nil + txns[idx] = nil + tcs[idx] = nil + indexers[idx] = nil + } + + fmt.Printf("Created %d edges in pass %d out of %d\n", edgesCreated.Load(), pass_idx, numPasses) + } + + return nil +} + // rebuildTokIndex rebuilds index for a given attribute. // We commit mutations with startTs and ignore the errors. func rebuildTokIndex(ctx context.Context, rb *IndexRebuild) error { @@ -1392,6 +1588,9 @@ func rebuildTokIndex(ctx context.Context, rb *IndexRebuild) error { } runForVectors := (len(factorySpecs) != 0) + if runForVectors { + return rebuildVectorIndex(ctx, factorySpecs, rb) + } pk := x.ParsedKey{Attr: rb.Attr} builder := rebuilder{attr: rb.Attr, prefix: pk.DataPrefix(), startTs: rb.StartTs} diff --git a/tok/hnsw/persistent_factory.go b/tok/hnsw/persistent_factory.go index ff4c622f218..76c9eaa6f5f 100644 --- a/tok/hnsw/persistent_factory.go +++ b/tok/hnsw/persistent_factory.go @@ -87,25 +87,27 @@ func (hf *persistentIndexFactory[T]) AllowedOptions() opt.AllowedOptions { func (hf *persistentIndexFactory[T]) Create( name string, o opt.Options, - floatBits int) (index.VectorIndex[T], error) { + floatBits int, + split int) (index.VectorIndex[T], error) { hf.mu.Lock() defer hf.mu.Unlock() - return hf.createWithLock(name, o, floatBits) + return hf.createWithLock(name, o, floatBits, split) } func (hf *persistentIndexFactory[T]) createWithLock( name string, o opt.Options, - floatBits int) (index.VectorIndex[T], error) { - if !hf.isNameAvailableWithLock(name) { + floatBits int, + split int) (index.VectorIndex[T], error) { + if !hf.isNameAvailableWithLock(fmt.Sprintf("%s-%d", name, split)) { err := errors.New("index with name " + name + " already exists") return nil, err } retVal := &persistentHNSW[T]{ pred: name, - vecEntryKey: ConcatStrings(name, VecEntry), - vecKey: ConcatStrings(name, VecKeyword), - vecDead: ConcatStrings(name, VecDead), + vecEntryKey: ConcatStrings(name, VecEntry, fmt.Sprintf("_%d", split)), + vecKey: ConcatStrings(name, VecKeyword, fmt.Sprintf("_%d", split)), + vecDead: ConcatStrings(name, VecDead, fmt.Sprintf("_%d", split)), floatBits: floatBits, nodeAllEdges: map[uint64][][]uint64{}, } @@ -152,7 +154,8 @@ func (hf *persistentIndexFactory[T]) removeWithLock(name string) error { func (hf *persistentIndexFactory[T]) CreateOrReplace( name string, o opt.Options, - floatBits int) (index.VectorIndex[T], error) { + floatBits int, + split int) (index.VectorIndex[T], error) { hf.mu.Lock() defer hf.mu.Unlock() vi, err := hf.findWithLock(name) @@ -165,5 +168,5 @@ func (hf *persistentIndexFactory[T]) CreateOrReplace( return nil, err } } - return hf.createWithLock(name, o, floatBits) + return hf.createWithLock(name, o, floatBits, split) } diff --git a/tok/hnsw/persistent_hnsw.go b/tok/hnsw/persistent_hnsw.go index e13ddddaf89..4b0d3101cc4 100644 --- a/tok/hnsw/persistent_hnsw.go +++ b/tok/hnsw/persistent_hnsw.go @@ -8,6 +8,7 @@ package hnsw import ( "context" "fmt" + "sort" "strings" "time" @@ -254,6 +255,46 @@ func (ph *persistentHNSW[T]) Search(ctx context.Context, c index.CacheType, quer return r.Neighbors, err } +type resultRow[T c.Float] struct { + uid uint64 + dist T +} + +func (ph *persistentHNSW[T]) MergeResults(ctx context.Context, c index.CacheType, list []uint64, query []T, maxResults int, filter index.SearchFilter[T]) ([]uint64, error) { + var result []resultRow[T] + + for i := range list { + var vec []T + err := ph.getVecFromUid(list[i], c, &vec) + if err != nil { + return nil, err + } + + dist, err := ph.simType.distanceScore(vec, query, ph.floatBits) + if err != nil { + return nil, err + } + result = append(result, resultRow[T]{ + uid: list[i], + dist: dist, + }) + } + + sort.Slice(result, func(i, j int) bool { + return result[i].dist < result[j].dist + }) + + uids := []uint64{} + for i := range maxResults { + if i > len(result) { + break + } + uids = append(uids, result[i].uid) + } + + return uids, nil +} + // SearchWithUid searches the hnsw graph for the nearest neighbors of the query uid // and returns the traversal path and the nearest neighbors func (ph *persistentHNSW[T]) SearchWithUid(_ context.Context, c index.CacheType, queryUid uint64, diff --git a/tok/index/index.go b/tok/index/index.go index e0a62255ce1..e00fb440932 100644 --- a/tok/index/index.go +++ b/tok/index/index.go @@ -39,7 +39,7 @@ type IndexFactory[T c.Float] interface { // same object. // The set of vectors to use in the index process is defined by // source. - Create(name string, o opts.Options, floatBits int) (VectorIndex[T], error) + Create(name string, o opts.Options, floatBits int, split int) (VectorIndex[T], error) // Find is expected to retrieve the VectorIndex corresponding with the // name. If it attempts to find a name that does not exist, the VectorIndex @@ -56,7 +56,7 @@ type IndexFactory[T c.Float] interface { // CreateOrReplace will create a new index -- as defined by the Create // function -- if it does not yet exist, otherwise, it will replace any // index with the given name. - CreateOrReplace(name string, o opts.Options, floatBits int) (VectorIndex[T], error) + CreateOrReplace(name string, o opts.Options, floatBits int, split int) (VectorIndex[T], error) } // SearchFilter defines a predicate function that we will use to determine @@ -93,6 +93,9 @@ type OptionalIndexSupport[T c.Float] interface { type VectorIndex[T c.Float] interface { OptionalIndexSupport[T] + MergeResults(ctx context.Context, c CacheType, list []uint64, query []T, maxResults int, + filter SearchFilter[T]) ([]uint64, error) + // Search will find the uids for a given set of vectors based on the // input query, limiting to the specified maximum number of results. // The filter parameter indicates that we might discard certain parameters diff --git a/tok/index_factory.go b/tok/index_factory.go index abef317b952..d67610bce1d 100644 --- a/tok/index_factory.go +++ b/tok/index_factory.go @@ -45,7 +45,7 @@ func (fcs *FactoryCreateSpec) Name() string { return fcs.factory.Name() + fcs.factory.GetOptions(fcs.opts) } -func (fcs *FactoryCreateSpec) CreateIndex(name string) (index.VectorIndex[float32], error) { +func (fcs *FactoryCreateSpec) CreateIndex(name string, split int) (index.VectorIndex[float32], error) { if fcs == nil || fcs.factory == nil { return nil, errors.Errorf( @@ -61,7 +61,7 @@ func (fcs *FactoryCreateSpec) CreateIndex(name string) (index.VectorIndex[float3 // has the downside of not allowing us to reuse the pre-existing // index. // nil VectorSource at the moment. - return fcs.factory.CreateOrReplace(name, fcs.opts, 32) + return fcs.factory.CreateOrReplace(name, fcs.opts, 32, split) } func createIndexFactory(f index.IndexFactory[float32]) IndexFactory { @@ -79,8 +79,9 @@ func (f *indexFactory) AllowedOptions() opts.AllowedOptions { func (f *indexFactory) Create( name string, o opts.Options, - floatBits int) (index.VectorIndex[float32], error) { - return f.delegate.Create(name, o, floatBits) + floatBits int, + split int) (index.VectorIndex[float32], error) { + return f.delegate.Create(name, o, floatBits, split) } func (f *indexFactory) Find(name string) (index.VectorIndex[float32], error) { return f.delegate.Find(name) @@ -91,8 +92,9 @@ func (f *indexFactory) Remove(name string) error { func (f *indexFactory) CreateOrReplace( name string, o opts.Options, - floatBits int) (index.VectorIndex[float32], error) { - return f.delegate.CreateOrReplace(name, o, floatBits) + floatBits int, + split int) (index.VectorIndex[float32], error) { + return f.delegate.CreateOrReplace(name, o, floatBits, split) } func (f *indexFactory) GetOptions(o opts.Options) string { diff --git a/worker/task.go b/worker/task.go index 92c1d02350f..0ecd371d842 100644 --- a/worker/task.go +++ b/worker/task.go @@ -12,6 +12,7 @@ import ( "sort" "strconv" "strings" + "sync" "time" "github.com/golang/glog" @@ -360,20 +361,33 @@ func (qs *queryState) handleValuePostings(ctx context.Context, args funcArgs) er posting.NewViLocalCache(qs.cache), args.q.ReadTs, ) - indexer, err := cspec.CreateIndex(args.q.Attr) - if err != nil { - return err - } - var nnUids []uint64 - if srcFn.vectorInfo != nil { - nnUids, err = indexer.Search(ctx, qc, srcFn.vectorInfo, - int(numNeighbors), index.AcceptAll[float32]) - } else { - nnUids, err = indexer.SearchWithUid(ctx, qc, srcFn.vectorUid, - int(numNeighbors), index.AcceptAll[float32]) - } - if err != nil && !strings.Contains(err.Error(), hnsw.EmptyHNSWTreeError+": "+badger.ErrKeyNotFound.Error()) { + var nnUids []uint64 + var wg sync.WaitGroup + wg.Add(1000) + var mutex sync.Mutex + for i := range 1000 { + go func(idx int) { + nnuids := make([]uint64, 0) + indexer, _ := cspec.CreateIndex(args.q.Attr, i) + if srcFn.vectorInfo != nil { + nnuids, _ = indexer.Search(ctx, qc, srcFn.vectorInfo, + int(numNeighbors), index.AcceptAll[float32]) + } else { + nnuids, _ = indexer.SearchWithUid(ctx, qc, srcFn.vectorUid, + int(numNeighbors), index.AcceptAll[float32]) + } + mutex.Lock() + nnUids = append(nnUids, nnuids...) + mutex.Unlock() + wg.Done() + }(i) + } + wg.Wait() + indexer, _ := cspec.CreateIndex(args.q.Attr, 0) + nnUids, err = indexer.MergeResults(ctx, qc, nnUids, srcFn.vectorInfo, + int(numNeighbors), index.AcceptAll[float32]) + if err != nil { return err } sort.Slice(nnUids, func(i, j int) bool { return nnUids[i] < nnUids[j] }) From 7e3aa8e493a98bae4da113cd1b2e82d2c5c33674 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Wed, 25 Jun 2025 02:32:21 +0530 Subject: [PATCH 02/52] added changes --- posting/index.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/posting/index.go b/posting/index.go index abb98ff4233..e8e70bcf24f 100644 --- a/posting/index.go +++ b/posting/index.go @@ -1413,7 +1413,6 @@ func (vc *vectorCentroids) updateCentroids() { func (vc *vectorCentroids) randomInit() { vc.dimension = len(vc.centroids[0]) vc.numCenters = len(vc.centroids) - vc.centroids = make([][]float32, vc.numCenters) vc.counts = make([]int64, vc.numCenters) vc.weights = make([][]float32, vc.numCenters) vc.mutexs = make([]*sync.Mutex, vc.numCenters) @@ -1433,6 +1432,7 @@ const numCentroids = 1000 func rebuildVectorIndex(ctx context.Context, factorySpecs []*tok.FactoryCreateSpec, rb *IndexRebuild) error { pk := x.ParsedKey{Attr: rb.Attr} vc := &vectorCentroids{} + vc.centroids = make([][]float32, 0) MemLayerInstance.IterateDisk(ctx, IterateDiskArgs{ Prefix: pk.DataPrefix(), From d651ad9b577d33099ce47fefe850a8cd8afcfa87 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Wed, 25 Jun 2025 22:38:40 +0530 Subject: [PATCH 03/52] added changes --- posting/index.go | 307 +++++++++++--------- tok/hnsw/helper.go | 4 + tok/hnsw/persistent_factory.go | 32 +- tok/hnsw/persistent_hnsw.go | 36 +++ tok/index/index.go | 24 +- tok/index_factory.go | 14 +- tok/kmeans/kmeans.go | 135 +++++++++ tok/partitioned_hnsw/partitioned_factory.go | 160 ++++++++++ tok/partitioned_hnsw/partitioned_hnsw.go | 193 ++++++++++++ worker/task.go | 32 +- 10 files changed, 759 insertions(+), 178 deletions(-) create mode 100644 tok/kmeans/kmeans.go create mode 100644 tok/partitioned_hnsw/partitioned_factory.go create mode 100644 tok/partitioned_hnsw/partitioned_hnsw.go diff --git a/posting/index.go b/posting/index.go index e8e70bcf24f..667915ae000 100644 --- a/posting/index.go +++ b/posting/index.go @@ -15,7 +15,6 @@ import ( "math" "os" "strings" - "sync" "sync/atomic" "time" "unsafe" @@ -34,11 +33,10 @@ import ( "github.com/hypermodeinc/dgraph/v25/schema" "github.com/hypermodeinc/dgraph/v25/tok" "github.com/hypermodeinc/dgraph/v25/tok/hnsw" - "github.com/hypermodeinc/dgraph/v25/tok/index" + tokIndex "github.com/hypermodeinc/dgraph/v25/tok/index" + "github.com/hypermodeinc/dgraph/v25/types" "github.com/hypermodeinc/dgraph/v25/x" - - "github.com/viterin/vek/vek32" ) var emptyCountParams countParams @@ -166,7 +164,7 @@ func (txn *Txn) addIndexMutations(ctx context.Context, info *indexMutationInfo) // retrieve vector from inUuid save as inVec inVec := types.BytesAsFloatArray(data[0].Value.([]byte)) tc := hnsw.NewTxnCache(NewViTxn(txn), txn.StartTs) - indexer, err := info.factorySpecs[0].CreateIndex(attr, 0) + indexer, err := info.factorySpecs[0].CreateIndex(attr) if err != nil { return []*pb.DirectedEdge{}, err } @@ -1365,112 +1363,67 @@ func (rb *indexRebuildInfo) prefixesForTokIndexes() ([][]byte, error) { return prefixes, nil } -type vectorCentroids struct { - dimension int - numCenters int +const numCentroids = 1000 - centroids [][]float32 - counts []int64 - weights [][]float32 - mutexs []*sync.Mutex -} +func rebuildVectorIndex(ctx context.Context, factorySpecs []*tok.FactoryCreateSpec, rb *IndexRebuild) error { + pk := x.ParsedKey{Attr: rb.Attr} -func (vc *vectorCentroids) findCentroid(input []float32) int { - minIdx := 0 - minDist := math.MaxFloat32 - for i, centroid := range vc.centroids { - dist := vek32.Distance(centroid, input) - if float64(dist) < minDist { - minDist = float64(dist) - minIdx = i - } + indexer, err := factorySpecs[0].CreateIndex(pk.Attr) + if err != nil { + return err } - return minIdx -} -func (vc *vectorCentroids) addVector(vec []float32) { - idx := vc.findCentroid(vec) - vc.mutexs[idx].Lock() - defer vc.mutexs[idx].Unlock() - for i := 0; i < vc.dimension; i++ { - vc.weights[idx][i] += vec[i] + if indexer.NumSeedVectors() > 0 { + count := 0 + MemLayerInstance.IterateDisk(ctx, IterateDiskArgs{ + Prefix: pk.DataPrefix(), + ReadTs: rb.StartTs, + AllVersions: false, + Reverse: false, + CheckInclusion: func(uid uint64) error { + return nil + }, + Function: func(l *List, pk x.ParsedKey) error { + val, err := l.Value(rb.StartTs) + if err != nil { + return err + } + inVec := types.BytesAsFloatArray(val.Value.([]byte)) + count += 1 + indexer.AddSeedVector(inVec) + if count == indexer.NumSeedVectors() { + return ErrStopIteration + } + return nil + }, + StartKey: x.DataKey(rb.Attr, 0), + }) } - vc.counts[idx]++ -} -func (vc *vectorCentroids) updateCentroids() { - for i := 0; i < vc.numCenters; i++ { - for j := 0; j < vc.dimension; j++ { - vc.centroids[i][j] = vc.weights[i][j] / float32(vc.counts[i]) - vc.weights[i][j] = 0 - } - fmt.Printf("%d, ", vc.counts[i]) - vc.counts[i] = 0 + txns := make([]*Txn, indexer.NumThreads()) + for i := range txns { + txns[i] = NewTxn(rb.StartTs) } - fmt.Println() -} - -func (vc *vectorCentroids) randomInit() { - vc.dimension = len(vc.centroids[0]) - vc.numCenters = len(vc.centroids) - vc.counts = make([]int64, vc.numCenters) - vc.weights = make([][]float32, vc.numCenters) - vc.mutexs = make([]*sync.Mutex, vc.numCenters) - for i := 0; i < vc.numCenters; i++ { - vc.weights[i] = make([]float32, vc.dimension) - vc.counts[i] = 0 - vc.mutexs[i] = &sync.Mutex{} + caches := make([]tokIndex.CacheType, indexer.NumThreads()) + for i := range caches { + caches[i] = hnsw.NewTxnCache(NewViTxn(txns[i]), rb.StartTs) } -} -func (vc *vectorCentroids) addSeedCentroid(vec []float32) { - vc.centroids = append(vc.centroids, vec) -} + for pass_idx := range indexer.NumBuildPasses() { + fmt.Println("Building pass", pass_idx) -const numCentroids = 1000 + indexer.StartBuild(caches) -func rebuildVectorIndex(ctx context.Context, factorySpecs []*tok.FactoryCreateSpec, rb *IndexRebuild) error { - pk := x.ParsedKey{Attr: rb.Attr} - vc := &vectorCentroids{} - vc.centroids = make([][]float32, 0) - - MemLayerInstance.IterateDisk(ctx, IterateDiskArgs{ - Prefix: pk.DataPrefix(), - ReadTs: rb.StartTs, - AllVersions: false, - Reverse: false, - CheckInclusion: func(uid uint64) error { - return nil - }, - Function: func(l *List, pk x.ParsedKey) error { - val, err := l.Value(rb.StartTs) - if err != nil { - return err - } - inVec := types.BytesAsFloatArray(val.Value.([]byte)) - vc.addSeedCentroid(inVec) - if len(vc.centroids) == numCentroids { - return ErrStopIteration - } - return nil - }, - StartKey: x.DataKey(rb.Attr, 0), - }) - - vc.randomInit() - - fmt.Println("Clustering Vectors") - for range 5 { builder := rebuilder{attr: rb.Attr, prefix: pk.DataPrefix(), startTs: rb.StartTs} builder.fn = func(uid uint64, pl *List, txn *Txn) ([]*pb.DirectedEdge, error) { edges := []*pb.DirectedEdge{} - val, err := pl.Value(txn.StartTs) + val, err := pl.Value(rb.StartTs) if err != nil { return []*pb.DirectedEdge{}, err } inVec := types.BytesAsFloatArray(val.Value.([]byte)) - vc.addVector(inVec) + indexer.BuildInsert(ctx, uid, inVec) return edges, nil } @@ -1479,48 +1432,25 @@ func rebuildVectorIndex(ctx context.Context, factorySpecs []*tok.FactoryCreateSp return err } - vc.updateCentroids() + indexer.EndBuild() } - tcs := make([]*hnsw.TxnCache, vc.numCenters) - txns := make([]*Txn, vc.numCenters) - indexers := make([]index.VectorIndex[float32], vc.numCenters) - for i := 0; i < vc.numCenters; i++ { - txns[i] = NewTxn(rb.StartTs) - tcs[i] = hnsw.NewTxnCache(NewViTxn(txns[i]), rb.StartTs) - indexers_i, err := factorySpecs[0].CreateIndex(pk.Attr, i) - if err != nil { - return err - } - vc.mutexs[i] = &sync.Mutex{} - indexers[i] = indexers_i - } + for pass_idx := range indexer.NumIndexPasses() { + fmt.Println("Indexing pass", pass_idx) - var edgesCreated atomic.Int64 + indexer.StartBuild(caches) - numPasses := vc.numCenters / 100 - for pass_idx := range numPasses { builder := rebuilder{attr: rb.Attr, prefix: pk.DataPrefix(), startTs: rb.StartTs} builder.fn = func(uid uint64, pl *List, txn *Txn) ([]*pb.DirectedEdge, error) { - val, err := pl.Value(txn.StartTs) + edges := []*pb.DirectedEdge{} + val, err := pl.Value(rb.StartTs) if err != nil { return []*pb.DirectedEdge{}, err } inVec := types.BytesAsFloatArray(val.Value.([]byte)) - idx := vc.findCentroid(inVec) - if idx%numPasses != pass_idx { - return []*pb.DirectedEdge{}, nil - } - vc.mutexs[idx].Lock() - defer vc.mutexs[idx].Unlock() - _, err = indexers[idx].Insert(ctx, tcs[idx], uid, inVec) - if err != nil { - return []*pb.DirectedEdge{}, err - } - - edgesCreated.Add(int64(1)) - return nil, nil + indexer.BuildInsert(ctx, uid, inVec) + return edges, nil } err := builder.RunWithoutTemp(ctx) @@ -1528,10 +1458,7 @@ func rebuildVectorIndex(ctx context.Context, factorySpecs []*tok.FactoryCreateSp return err } - for idx := range vc.counts { - if idx%numPasses != pass_idx { - continue - } + for _, idx := range indexer.EndBuild() { txns[idx].Update() writer := NewTxnWriter(pstore) @@ -1547,14 +1474,132 @@ func rebuildVectorIndex(ctx context.Context, factorySpecs []*tok.FactoryCreateSp txns[idx].cache.plists = nil txns[idx] = nil - tcs[idx] = nil - indexers[idx] = nil } - - fmt.Printf("Created %d edges in pass %d out of %d\n", edgesCreated.Load(), pass_idx, numPasses) } return nil + + // MemLayerInstance.IterateDisk(ctx, IterateDiskArgs{ + // Prefix: pk.DataPrefix(), + // ReadTs: rb.StartTs, + // AllVersions: false, + // Reverse: false, + // CheckInclusion: func(uid uint64) error { + // return nil + // }, + // Function: func(l *List, pk x.ParsedKey) error { + // val, err := l.Value(rb.StartTs) + // if err != nil { + // return err + // } + // inVec := types.BytesAsFloatArray(val.Value.([]byte)) + // vc.addSeedCentroid(inVec) + // if len(vc.centroids) == numCentroids { + // return ErrStopIteration + // } + // return nil + // }, + // StartKey: x.DataKey(rb.Attr, 0), + // }) + + // vc.randomInit() + + // fmt.Println("Clustering Vectors") + // for range 5 { + // builder := rebuilder{attr: rb.Attr, prefix: pk.DataPrefix(), startTs: rb.StartTs} + // builder.fn = func(uid uint64, pl *List, txn *Txn) ([]*pb.DirectedEdge, error) { + // edges := []*pb.DirectedEdge{} + // val, err := pl.Value(txn.StartTs) + // if err != nil { + // return []*pb.DirectedEdge{}, err + // } + + // inVec := types.BytesAsFloatArray(val.Value.([]byte)) + // vc.addVector(inVec) + // return edges, nil + // } + + // err := builder.RunWithoutTemp(ctx) + // if err != nil { + // return err + // } + + // vc.updateCentroids() + // } + + // tcs := make([]*hnsw.TxnCache, vc.numCenters) + // txns := make([]*Txn, vc.numCenters) + // indexers := make([]index.VectorIndex[float32], vc.numCenters) + // for i := 0; i < vc.numCenters; i++ { + // txns[i] = NewTxn(rb.StartTs) + // tcs[i] = hnsw.NewTxnCache(NewViTxn(txns[i]), rb.StartTs) + // indexers_i, err := factorySpecs[0].CreateIndex(pk.Attr, i) + // if err != nil { + // return err + // } + // vc.mutexs[i] = &sync.Mutex{} + // indexers[i] = indexers_i + // } + + // var edgesCreated atomic.Int64 + + // numPasses := vc.numCenters / 100 + // for pass_idx := range numPasses { + // builder := rebuilder{attr: rb.Attr, prefix: pk.DataPrefix(), startTs: rb.StartTs} + // builder.fn = func(uid uint64, pl *List, txn *Txn) ([]*pb.DirectedEdge, error) { + // val, err := pl.Value(txn.StartTs) + // if err != nil { + // return []*pb.DirectedEdge{}, err + // } + + // inVec := types.BytesAsFloatArray(val.Value.([]byte)) + // idx := vc.findCentroid(inVec) + // if idx%numPasses != pass_idx { + // return []*pb.DirectedEdge{}, nil + // } + // vc.mutexs[idx].Lock() + // defer vc.mutexs[idx].Unlock() + // _, err = indexers[idx].Insert(ctx, tcs[idx], uid, inVec) + // if err != nil { + // return []*pb.DirectedEdge{}, err + // } + + // edgesCreated.Add(int64(1)) + // return nil, nil + // } + + // err := builder.RunWithoutTemp(ctx) + // if err != nil { + // return err + // } + + // for idx := range vc.counts { + // if idx%numPasses != pass_idx { + // continue + // } + // txns[idx].Update() + // writer := NewTxnWriter(pstore) + + // x.ExponentialRetry(int(x.Config.MaxRetries), + // 20*time.Millisecond, func() error { + // err := txns[idx].CommitToDisk(writer, rb.StartTs) + // if err == badger.ErrBannedKey { + // glog.Errorf("Error while writing to banned namespace.") + // return nil + // } + // return err + // }) + + // txns[idx].cache.plists = nil + // txns[idx] = nil + // tcs[idx] = nil + // indexers[idx] = nil + // } + + // fmt.Printf("Created %d edges in pass %d out of %d\n", edgesCreated.Load(), pass_idx, numPasses) + // } + + // return nil } // rebuildTokIndex rebuilds index for a given attribute. diff --git a/tok/hnsw/helper.go b/tok/hnsw/helper.go index 477f5bc9b27..c3b78c7a488 100644 --- a/tok/hnsw/helper.go +++ b/tok/hnsw/helper.go @@ -114,6 +114,10 @@ func euclideanDistanceSq[T c.Float](a, b []T, floatBits int) (T, error) { return applyDistanceFunction(a, b, floatBits, "euclidean distance", vek32.Distance, vek.Distance) } +func EuclideanDistanceSq[T c.Float](a, b []T, floatBits int) (T, error) { + return applyDistanceFunction(a, b, floatBits, "euclidean distance", vek32.Distance, vek.Distance) +} + // Used for distance, since shorter distance is better func insortPersistentHeapAscending[T c.Float]( slice []minPersistentHeapElement[T], diff --git a/tok/hnsw/persistent_factory.go b/tok/hnsw/persistent_factory.go index 76c9eaa6f5f..4bc13b48ea6 100644 --- a/tok/hnsw/persistent_factory.go +++ b/tok/hnsw/persistent_factory.go @@ -78,6 +78,17 @@ func (hf *persistentIndexFactory[T]) AllowedOptions() opt.AllowedOptions { return retVal } +func UpdateIndexSplit[T c.Float](vi index.VectorIndex[T], split int) error { + hnsw, ok := vi.(*persistentHNSW[T]) + if !ok { + return errors.New("index is not a persistent HNSW index") + } + hnsw.vecEntryKey = ConcatStrings(hnsw.pred, fmt.Sprintf("%s_%d", VecEntry, split)) + hnsw.vecKey = ConcatStrings(hnsw.pred, fmt.Sprintf("%s_%d", VecKeyword, split)) + hnsw.vecDead = ConcatStrings(hnsw.pred, fmt.Sprintf("%s_%d", VecDead, split)) + return nil +} + // Create is an implementation of the IndexFactory interface function, invoked by an HNSWIndexFactory // instance. It takes in a string name and a VectorSource implementation, and returns a VectorIndex and error // flag. It creates an HNSW instance using the index name and populates other parts of the HNSW struct such as @@ -87,27 +98,25 @@ func (hf *persistentIndexFactory[T]) AllowedOptions() opt.AllowedOptions { func (hf *persistentIndexFactory[T]) Create( name string, o opt.Options, - floatBits int, - split int) (index.VectorIndex[T], error) { + floatBits int) (index.VectorIndex[T], error) { hf.mu.Lock() defer hf.mu.Unlock() - return hf.createWithLock(name, o, floatBits, split) + return hf.createWithLock(name, o, floatBits) } func (hf *persistentIndexFactory[T]) createWithLock( name string, o opt.Options, - floatBits int, - split int) (index.VectorIndex[T], error) { - if !hf.isNameAvailableWithLock(fmt.Sprintf("%s-%d", name, split)) { + floatBits int) (index.VectorIndex[T], error) { + if !hf.isNameAvailableWithLock(name) { err := errors.New("index with name " + name + " already exists") return nil, err } retVal := &persistentHNSW[T]{ pred: name, - vecEntryKey: ConcatStrings(name, VecEntry, fmt.Sprintf("_%d", split)), - vecKey: ConcatStrings(name, VecKeyword, fmt.Sprintf("_%d", split)), - vecDead: ConcatStrings(name, VecDead, fmt.Sprintf("_%d", split)), + vecEntryKey: ConcatStrings(name, VecEntry), + vecKey: ConcatStrings(name, VecKeyword), + vecDead: ConcatStrings(name, VecDead), floatBits: floatBits, nodeAllEdges: map[uint64][][]uint64{}, } @@ -154,8 +163,7 @@ func (hf *persistentIndexFactory[T]) removeWithLock(name string) error { func (hf *persistentIndexFactory[T]) CreateOrReplace( name string, o opt.Options, - floatBits int, - split int) (index.VectorIndex[T], error) { + floatBits int) (index.VectorIndex[T], error) { hf.mu.Lock() defer hf.mu.Unlock() vi, err := hf.findWithLock(name) @@ -168,5 +176,5 @@ func (hf *persistentIndexFactory[T]) CreateOrReplace( return nil, err } } - return hf.createWithLock(name, o, floatBits, split) + return hf.createWithLock(name, o, floatBits) } diff --git a/tok/hnsw/persistent_hnsw.go b/tok/hnsw/persistent_hnsw.go index 4b0d3101cc4..f1b0e4f3d84 100644 --- a/tok/hnsw/persistent_hnsw.go +++ b/tok/hnsw/persistent_hnsw.go @@ -33,6 +33,7 @@ type persistentHNSW[T c.Float] struct { // layer for uuid 65443. The result will be a neighboring uuid. nodeAllEdges map[uint64][][]uint64 deadNodes map[uint64]struct{} + cache index.CacheType } func GetPersistantOptions[T c.Float](o opt.Options) string { @@ -112,6 +113,41 @@ func (ph *persistentHNSW[T]) applyOptions(o opt.Options) error { return nil } +func (ph *persistentHNSW[T]) NumBuildPasses() int { + return 0 +} + +func (ph *persistentHNSW[T]) NumIndexPasses() int { + return 1 +} + +func (ph *persistentHNSW[T]) NumSeedVectors() int { + return 0 +} + +func (ph *persistentHNSW[T]) StartBuild(caches []index.CacheType) { + ph.nodeAllEdges = make(map[uint64][][]uint64) + ph.cache = caches[0] +} + +func (ph *persistentHNSW[T]) EndBuild() []int { + ph.nodeAllEdges = nil + ph.cache = nil + return []int{0} +} + +func (ph *persistentHNSW[T]) NumThreads() int { + return 1 +} + +func (ph *persistentHNSW[T]) BuildInsert(ctx context.Context, uid uint64, vec []T) error { + _, err := ph.Insert(ctx, ph.cache, uid, vec) + return err +} + +func (ph *persistentHNSW[T]) AddSeedVector(vec []T) { +} + func (ph *persistentHNSW[T]) emptyFinalResultWithError(e error) ( *index.SearchPathResult, error) { return index.NewSearchPathResult(), e diff --git a/tok/index/index.go b/tok/index/index.go index e00fb440932..503c2b66faa 100644 --- a/tok/index/index.go +++ b/tok/index/index.go @@ -39,7 +39,7 @@ type IndexFactory[T c.Float] interface { // same object. // The set of vectors to use in the index process is defined by // source. - Create(name string, o opts.Options, floatBits int, split int) (VectorIndex[T], error) + Create(name string, o opts.Options, floatBits int) (VectorIndex[T], error) // Find is expected to retrieve the VectorIndex corresponding with the // name. If it attempts to find a name that does not exist, the VectorIndex @@ -56,7 +56,7 @@ type IndexFactory[T c.Float] interface { // CreateOrReplace will create a new index -- as defined by the Create // function -- if it does not yet exist, otherwise, it will replace any // index with the given name. - CreateOrReplace(name string, o opts.Options, floatBits int, split int) (VectorIndex[T], error) + CreateOrReplace(name string, o opts.Options, floatBits int) (VectorIndex[T], error) } // SearchFilter defines a predicate function that we will use to determine @@ -89,6 +89,17 @@ type OptionalIndexSupport[T c.Float] interface { filter SearchFilter[T]) (*SearchPathResult, error) } +type VectorPartitionStrat[T c.Float] interface { + FindIndexForSearch(vec []T) ([]int, error) + FindIndexForInsert(vec []T) (int, error) + NumPasses() int + NumSeedVectors() int + StartBuildPass() + EndBuildPass() + AddSeedVector(vec []T) + AddVector(vec []T) error +} + // A VectorIndex can be used to Search for vectors and add vectors to an index. type VectorIndex[T c.Float] interface { OptionalIndexSupport[T] @@ -119,6 +130,15 @@ type VectorIndex[T c.Float] interface { // Insert will add a vector and uuid into the existing VectorIndex. If // uuid already exists, it should throw an error to not insert duplicate uuids Insert(ctx context.Context, c CacheType, uuid uint64, vec []T) ([]*KeyValue, error) + + BuildInsert(ctx context.Context, uuid uint64, vec []T) error + AddSeedVector(vec []T) + NumBuildPasses() int + NumIndexPasses() int + NumSeedVectors() int + StartBuild(caches []CacheType) + EndBuild() []int + NumThreads() int } // A Txn is an interface representation of a persistent storage transaction, diff --git a/tok/index_factory.go b/tok/index_factory.go index d67610bce1d..abef317b952 100644 --- a/tok/index_factory.go +++ b/tok/index_factory.go @@ -45,7 +45,7 @@ func (fcs *FactoryCreateSpec) Name() string { return fcs.factory.Name() + fcs.factory.GetOptions(fcs.opts) } -func (fcs *FactoryCreateSpec) CreateIndex(name string, split int) (index.VectorIndex[float32], error) { +func (fcs *FactoryCreateSpec) CreateIndex(name string) (index.VectorIndex[float32], error) { if fcs == nil || fcs.factory == nil { return nil, errors.Errorf( @@ -61,7 +61,7 @@ func (fcs *FactoryCreateSpec) CreateIndex(name string, split int) (index.VectorI // has the downside of not allowing us to reuse the pre-existing // index. // nil VectorSource at the moment. - return fcs.factory.CreateOrReplace(name, fcs.opts, 32, split) + return fcs.factory.CreateOrReplace(name, fcs.opts, 32) } func createIndexFactory(f index.IndexFactory[float32]) IndexFactory { @@ -79,9 +79,8 @@ func (f *indexFactory) AllowedOptions() opts.AllowedOptions { func (f *indexFactory) Create( name string, o opts.Options, - floatBits int, - split int) (index.VectorIndex[float32], error) { - return f.delegate.Create(name, o, floatBits, split) + floatBits int) (index.VectorIndex[float32], error) { + return f.delegate.Create(name, o, floatBits) } func (f *indexFactory) Find(name string) (index.VectorIndex[float32], error) { return f.delegate.Find(name) @@ -92,9 +91,8 @@ func (f *indexFactory) Remove(name string) error { func (f *indexFactory) CreateOrReplace( name string, o opts.Options, - floatBits int, - split int) (index.VectorIndex[float32], error) { - return f.delegate.CreateOrReplace(name, o, floatBits, split) + floatBits int) (index.VectorIndex[float32], error) { + return f.delegate.CreateOrReplace(name, o, floatBits) } func (f *indexFactory) GetOptions(o opts.Options) string { diff --git a/tok/kmeans/kmeans.go b/tok/kmeans/kmeans.go new file mode 100644 index 00000000000..00768b1f006 --- /dev/null +++ b/tok/kmeans/kmeans.go @@ -0,0 +1,135 @@ +package kmeans + +import ( + "fmt" + "math" + "sync" + + c "github.com/hypermodeinc/dgraph/v25/tok/constraints" + "github.com/hypermodeinc/dgraph/v25/tok/index" +) + +type Kmeans[T c.Float] struct { + floatBits int + centroids *vectorCentroids[T] +} + +func CreateKMeans[T c.Float](floatBits int, distFunc func(a, b []T, floatBits int) (T, error)) index.VectorPartitionStrat[T] { + return &Kmeans[T]{ + floatBits: floatBits, + centroids: &vectorCentroids[T]{ + distFunc: distFunc, + floatBits: floatBits, + }, + } +} + +func (km *Kmeans[T]) AddSeedVector(vec []T) { + km.centroids.addSeedCentroid(vec) +} + +func (km *Kmeans[T]) AddVector(vec []T) error { + return km.centroids.addVector(vec) +} + +func (km *Kmeans[T]) FindIndexForSearch(vec []T) ([]int, error) { + res := make([]int, len(km.centroids.centroids)) + for i := range res { + res[i] = i + } + return res, nil +} + +func (km *Kmeans[T]) FindIndexForInsert(vec []T) (int, error) { + return km.centroids.findCentroid(vec) +} + +func (km *Kmeans[T]) NumPasses() int { + return 5 +} + +func (km *Kmeans[T]) NumSeedVectors() int { + return 1000 +} + +func (km *Kmeans[T]) StartBuildPass() { + if km.centroids.weights == nil { + km.centroids.randomInit() + } +} + +func (km *Kmeans[T]) EndBuildPass() { + km.centroids.updateCentroids() +} + +type vectorCentroids[T c.Float] struct { + dimension int + numCenters int + + distFunc func(a, b []T, floatBits int) (T, error) + + centroids [][]T + counts []int64 + weights [][]T + mutexs []*sync.Mutex + floatBits int +} + +func (vc *vectorCentroids[T]) findCentroid(input []T) (int, error) { + minIdx := 0 + minDist := math.MaxFloat32 + for i, centroid := range vc.centroids { + dist, err := vc.distFunc(centroid, input, vc.floatBits) + if err != nil { + return 0, err + } + if float64(dist) < minDist { + minDist = float64(dist) + minIdx = i + } + } + return minIdx, nil +} + +func (vc *vectorCentroids[T]) addVector(vec []T) error { + idx, err := vc.findCentroid(vec) + if err != nil { + return err + } + vc.mutexs[idx].Lock() + defer vc.mutexs[idx].Unlock() + for i := 0; i < vc.dimension; i++ { + vc.weights[idx][i] += vec[i] + } + vc.counts[idx]++ + return nil +} + +func (vc *vectorCentroids[T]) updateCentroids() { + for i := 0; i < vc.numCenters; i++ { + for j := 0; j < vc.dimension; j++ { + vc.centroids[i][j] = vc.weights[i][j] / T(vc.counts[i]) + vc.weights[i][j] = 0 + } + fmt.Printf("%d, ", vc.counts[i]) + vc.counts[i] = 0 + } + fmt.Println() +} + +func (vc *vectorCentroids[T]) randomInit() { + vc.dimension = len(vc.centroids[0]) + vc.numCenters = len(vc.centroids) + vc.counts = make([]int64, vc.numCenters) + vc.weights = make([][]T, vc.numCenters) + vc.mutexs = make([]*sync.Mutex, vc.numCenters) + for i := 0; i < vc.numCenters; i++ { + vc.weights[i] = make([]T, vc.dimension) + vc.counts[i] = 0 + vc.mutexs[i] = &sync.Mutex{} + } +} + +func (vc *vectorCentroids[T]) addSeedCentroid(vec []T) { + vc.centroids = append(vc.centroids, vec) +} diff --git a/tok/partitioned_hnsw/partitioned_factory.go b/tok/partitioned_hnsw/partitioned_factory.go new file mode 100644 index 00000000000..28dc608a362 --- /dev/null +++ b/tok/partitioned_hnsw/partitioned_factory.go @@ -0,0 +1,160 @@ +/* + * SPDX-FileCopyrightText: © Hypermode Inc. + * SPDX-License-Identifier: Apache-2.0 + */ + +package partitioned_hnsw + +import ( + "errors" + "fmt" + "sync" + + c "github.com/hypermodeinc/dgraph/v25/tok/constraints" + "github.com/hypermodeinc/dgraph/v25/tok/hnsw" + "github.com/hypermodeinc/dgraph/v25/tok/index" + opt "github.com/hypermodeinc/dgraph/v25/tok/options" +) + +const ( + NumClustersOpt string = "numClusters" + PartitionStratOpt string = "partitionStratOpt" + PartitionedHNSW string = "partionedHNSW" +) + +type partitionedHNSWIndexFactory[T c.Float] struct { + indexMap map[string]index.VectorIndex[T] + floatBits int + mu sync.RWMutex +} + +// CreateFactory creates an instance of the private struct persistentIndexFactory. +// NOTE: if T and floatBits do not match in # of bits, there will be consequences. +func CreateFactory[T c.Float](floatBits int) index.IndexFactory[T] { + return &partitionedHNSWIndexFactory[T]{ + indexMap: map[string]index.VectorIndex[T]{}, + floatBits: floatBits, + } +} + +// Implements NamedFactory interface for use as a plugin. +func (hf *partitionedHNSWIndexFactory[T]) Name() string { return PartitionedHNSW } + +func (hf *partitionedHNSWIndexFactory[T]) GetOptions(o opt.Options) string { + return hnsw.GetPersistantOptions[T](o) +} + +func (hf *partitionedHNSWIndexFactory[T]) isNameAvailableWithLock(name string) bool { + _, nameUsed := hf.indexMap[name] + return !nameUsed +} + +// hf.AllowedOptions() allows persistentIndexFactory to implement the +// IndexFactory interface (see vector-indexer/index/index.go for details). +// We define here options for exponent, maxLevels, efSearch, efConstruction, +// and metric. +func (hf *partitionedHNSWIndexFactory[T]) AllowedOptions() opt.AllowedOptions { + retVal := opt.NewAllowedOptions() + retVal.AddIntOption(hnsw.ExponentOpt). + AddIntOption(hnsw.MaxLevelsOpt). + AddIntOption(hnsw.EfConstructionOpt). + AddIntOption(hnsw.EfSearchOpt). + AddIntOption(NumClustersOpt). + AddStringOption(PartitionStratOpt) + getSimFunc := func(optValue string) (any, error) { + if optValue != hnsw.Euclidean && optValue != hnsw.Cosine && optValue != hnsw.DotProd { + return nil, errors.New(fmt.Sprintf("Can't create a vector index for %s", optValue)) + } + return hnsw.GetSimType[T](optValue, hf.floatBits), nil + } + + retVal.AddCustomOption(hnsw.MetricOpt, getSimFunc) + return retVal +} + +// Create is an implementation of the IndexFactory interface function, invoked by an HNSWIndexFactory +// instance. It takes in a string name and a VectorSource implementation, and returns a VectorIndex and error +// flag. It creates an HNSW instance using the index name and populates other parts of the HNSW struct such as +// multFactor, maxLevels, efConstruction, maxNeighbors, and efSearch using struct parameters. +// It then populates the HNSW graphs using the InsertChunk function until there are no more items to populate. +// Finally, the function adds the name and hnsw object to the in memory map and returns the object. +func (hf *partitionedHNSWIndexFactory[T]) Create( + name string, + o opt.Options, + floatBits int) (index.VectorIndex[T], error) { + hf.mu.Lock() + defer hf.mu.Unlock() + return hf.createWithLock(name, o, floatBits) +} + +func (hf *partitionedHNSWIndexFactory[T]) createWithLock( + name string, + o opt.Options, + floatBits int) (index.VectorIndex[T], error) { + if !hf.isNameAvailableWithLock(name) { + err := errors.New("index with name " + name + " already exists") + return nil, err + } + retVal := &partitionedHNSW[T]{ + pred: name, + floatBits: floatBits, + clusterMap: map[int]index.VectorIndex[T]{}, + } + err := retVal.applyOptions(o) + if err != nil { + return nil, err + } + hf.indexMap[name] = retVal + return retVal, nil +} + +// Find is an implementation of the IndexFactory interface function, invoked by an persistentIndexFactory +// instance. It returns the VectorIndex corresponding with a string name using the in memory map. +func (hf *partitionedHNSWIndexFactory[T]) Find(name string) (index.VectorIndex[T], error) { + hf.mu.RLock() + defer hf.mu.RUnlock() + return hf.findWithLock(name) +} + +func (hf *partitionedHNSWIndexFactory[T]) findWithLock(name string) (index.VectorIndex[T], error) { + vecInd := hf.indexMap[name] + return vecInd, nil +} + +// Remove is an implementation of the IndexFactory interface function, invoked by an persistentIndexFactory +// instance. It removes the VectorIndex corresponding with a string name using the in memory map. +func (hf *partitionedHNSWIndexFactory[T]) Remove(name string) error { + hf.mu.Lock() + defer hf.mu.Unlock() + return hf.removeWithLock(name) +} + +func (hf *partitionedHNSWIndexFactory[T]) removeWithLock(name string) error { + delete(hf.indexMap, name) + return nil +} + +// CreateOrReplace is an implementation of the IndexFactory interface funciton, +// invoked by an persistentIndexFactory. It checks if a VectorIndex +// correpsonding with name exists. If it does, it removes it, and replaces it +// via the Create function using the passed VectorSource. If the VectorIndex +// does not exist, it creates that VectorIndex corresponding with the name using +// the VectorSource. +func (hf *partitionedHNSWIndexFactory[T]) CreateOrReplace( + name string, + o opt.Options, + floatBits int) (index.VectorIndex[T], error) { + hf.mu.Lock() + defer hf.mu.Unlock() + vi, err := hf.findWithLock(name) + if err != nil { + return nil, err + } + if vi != nil { + err = hf.removeWithLock(name) + if err != nil { + return nil, err + } + } + return hf.createWithLock(name, o, floatBits) +} diff --git a/tok/partitioned_hnsw/partitioned_hnsw.go b/tok/partitioned_hnsw/partitioned_hnsw.go new file mode 100644 index 00000000000..5dcc6dd564f --- /dev/null +++ b/tok/partitioned_hnsw/partitioned_hnsw.go @@ -0,0 +1,193 @@ +// CreateFactory creates an instance of the private struct persistentIndexFactory. +// NOTE: if T and floatBits do not match in # of bits, there will be consequences. + +package partitioned_hnsw + +import ( + "context" + "errors" + "sync" + + c "github.com/hypermodeinc/dgraph/v25/tok/constraints" + hnsw "github.com/hypermodeinc/dgraph/v25/tok/hnsw" + "github.com/hypermodeinc/dgraph/v25/tok/index" + "github.com/hypermodeinc/dgraph/v25/tok/kmeans" + opt "github.com/hypermodeinc/dgraph/v25/tok/options" +) + +type partitionedHNSW[T c.Float] struct { + floatBits int + pred string + + clusterMap map[int]index.VectorIndex[T] + numClusters int + factory index.IndexFactory[T] + partition index.VectorPartitionStrat[T] + + hnswOptions opt.Options + partitionStrat string + + caches []index.CacheType + buildPass int +} + +func (ph *partitionedHNSW[T]) applyOptions(o opt.Options) error { + if o.Specifies(NumClustersOpt) { + ph.numClusters, _, _ = opt.GetOpt(o, NumClustersOpt, 1000) + } + + if o.Specifies(PartitionStratOpt) { + ph.partitionStrat, _, _ = opt.GetOpt(o, PartitionStratOpt, "kmeans") + } + + if ph.partitionStrat != "kmeans" && ph.partitionStrat != "query" { + return errors.New("partition strategy must be kmeans or query") + } + + if ph.partitionStrat == "kmeans" { + ph.partition = kmeans.CreateKMeans(ph.floatBits, hnsw.EuclideanDistanceSq[T]) + } + + ph.buildPass = 0 + ph.hnswOptions = o + ph.factory = hnsw.CreateFactory[T](ph.floatBits) + for i := range ph.numClusters { + vi, err := ph.factory.Create(ph.pred, ph.hnswOptions, ph.floatBits) + if err != nil { + return err + } + err = hnsw.UpdateIndexSplit(vi, i) + if err != nil { + return err + } + ph.clusterMap[i] = vi + } + return nil +} + +func (ph *partitionedHNSW[T]) AddSeedVector(vec []T) { + ph.partition.AddSeedVector(vec) +} + +func (ph *partitionedHNSW[T]) BuildInsert(ctx context.Context, uuid uint64, vec []T) error { + passIdx := ph.buildPass - ph.partition.NumPasses() + if passIdx < 0 { + return ph.partition.AddVector(vec) + } + index, err := ph.partition.FindIndexForInsert(vec) + if err != nil { + return err + } + if index%NUM_PASSES != passIdx { + return nil + } + return ph.clusterMap[index].BuildInsert(ctx, uuid, vec) +} + +const NUM_PASSES = 10 + +func (ph *partitionedHNSW[T]) NumBuildPasses() int { + return ph.partition.NumPasses() +} + +func (ph *partitionedHNSW[T]) NumIndexPasses() int { + return NUM_PASSES +} + +func (ph *partitionedHNSW[T]) NumThreads() int { + return NUM_PASSES +} + +func (ph *partitionedHNSW[T]) NumSeedVectors() int { + return ph.partition.NumSeedVectors() +} + +func (ph *partitionedHNSW[T]) StartBuild(caches []index.CacheType) { + ph.caches = caches + if ph.buildPass <= ph.partition.NumPasses() { + ph.partition.StartBuildPass() + return + } + + for i := range ph.clusterMap { + if i%NUM_PASSES != (ph.buildPass - ph.partition.NumPasses()) { + continue + } + ph.clusterMap[i].StartBuild([]index.CacheType{ph.caches[i]}) + } +} + +func (ph *partitionedHNSW[T]) EndBuild() []int { + res := []int{} + + if ph.buildPass > ph.partition.NumPasses() { + for i := range ph.clusterMap { + if i%NUM_PASSES != (ph.buildPass - ph.partition.NumPasses()) { + continue + } + ph.clusterMap[i].EndBuild() + res = append(res, i) + } + } + + ph.buildPass += 1 + + if len(res) > 0 { + return res + } + + if ph.buildPass <= ph.partition.NumPasses() { + ph.partition.EndBuildPass() + } + return []int{} +} + +func (ph *partitionedHNSW[T]) Insert(ctx context.Context, txn index.CacheType, uid uint64, vec []T) ([]*index.KeyValue, error) { + index, err := ph.partition.FindIndexForInsert(vec) + if err != nil { + return nil, err + } + return ph.clusterMap[index].Insert(ctx, txn, uid, vec) +} + +func (ph *partitionedHNSW[T]) Search(ctx context.Context, txn index.CacheType, query []T, maxResults int, filter index.SearchFilter[T]) ([]uint64, error) { + indexes, err := ph.partition.FindIndexForSearch(query) + if err != nil { + return nil, err + } + res := []uint64{} + mutex := &sync.Mutex{} + var wg sync.WaitGroup + for _, index := range indexes { + wg.Add(1) + go func(i int) { + defer wg.Done() + ids, err := ph.clusterMap[i].Search(ctx, txn, query, maxResults, filter) + if err != nil { + return + } + mutex.Lock() + res = append(res, ids...) + mutex.Unlock() + }(index) + } + wg.Wait() + return ph.clusterMap[0].MergeResults(ctx, txn, res, query, maxResults, filter) +} + +func (ph *partitionedHNSW[T]) SearchWithPath(ctx context.Context, txn index.CacheType, query []T, maxResults int, filter index.SearchFilter[T]) (*index.SearchPathResult, error) { + indexes, err := ph.partition.FindIndexForSearch(query) + if err != nil { + return nil, err + } + return ph.clusterMap[indexes[0]].SearchWithPath(ctx, txn, query, maxResults, filter) +} + +func (ph *partitionedHNSW[T]) SearchWithUid(ctx context.Context, txn index.CacheType, uid uint64, maxResults int, filter index.SearchFilter[T]) ([]uint64, error) { + // #TODO + return ph.clusterMap[0].SearchWithUid(ctx, txn, uid, maxResults, filter) +} + +func (ph *partitionedHNSW[T]) MergeResults(ctx context.Context, txn index.CacheType, list []uint64, query []T, maxResults int, filter index.SearchFilter[T]) ([]uint64, error) { + return ph.clusterMap[0].MergeResults(ctx, txn, list, query, maxResults, filter) +} diff --git a/worker/task.go b/worker/task.go index 0ecd371d842..1b97bc98702 100644 --- a/worker/task.go +++ b/worker/task.go @@ -12,7 +12,6 @@ import ( "sort" "strconv" "strings" - "sync" "time" "github.com/golang/glog" @@ -362,34 +361,17 @@ func (qs *queryState) handleValuePostings(ctx context.Context, args funcArgs) er args.q.ReadTs, ) - var nnUids []uint64 - var wg sync.WaitGroup - wg.Add(1000) - var mutex sync.Mutex - for i := range 1000 { - go func(idx int) { - nnuids := make([]uint64, 0) - indexer, _ := cspec.CreateIndex(args.q.Attr, i) - if srcFn.vectorInfo != nil { - nnuids, _ = indexer.Search(ctx, qc, srcFn.vectorInfo, - int(numNeighbors), index.AcceptAll[float32]) - } else { - nnuids, _ = indexer.SearchWithUid(ctx, qc, srcFn.vectorUid, - int(numNeighbors), index.AcceptAll[float32]) - } - mutex.Lock() - nnUids = append(nnUids, nnuids...) - mutex.Unlock() - wg.Done() - }(i) - } - wg.Wait() - indexer, _ := cspec.CreateIndex(args.q.Attr, 0) - nnUids, err = indexer.MergeResults(ctx, qc, nnUids, srcFn.vectorInfo, + indexer, err := cspec.CreateIndex(args.q.Attr) + if err != nil { + return err + } + + nnUids, err := indexer.Search(ctx, qc, srcFn.vectorInfo, int(numNeighbors), index.AcceptAll[float32]) if err != nil { return err } + sort.Slice(nnUids, func(i, j int) bool { return nnUids[i] < nnUids[j] }) args.out.UidMatrix = append(args.out.UidMatrix, &pb.List{Uids: nnUids}) return nil From bbaa7fbed2b7f241aafd4e0b405daa1d5a93e05a Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Wed, 25 Jun 2025 22:44:58 +0530 Subject: [PATCH 04/52] added changes --- tok/tok.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tok/tok.go b/tok/tok.go index c74c7a9d10b..e20a647c43b 100644 --- a/tok/tok.go +++ b/tok/tok.go @@ -21,6 +21,7 @@ import ( "github.com/hypermodeinc/dgraph/v25/protos/pb" "github.com/hypermodeinc/dgraph/v25/tok/hnsw" opts "github.com/hypermodeinc/dgraph/v25/tok/options" + "github.com/hypermodeinc/dgraph/v25/tok/partitioned_hnsw" "github.com/hypermodeinc/dgraph/v25/types" "github.com/hypermodeinc/dgraph/v25/x" ) @@ -85,6 +86,7 @@ var indexFactories = make(map[string]IndexFactory) func init() { registerTokenizer(BigFloatTokenizer{}) registerIndexFactory(createIndexFactory(hnsw.CreateFactory[float32](32))) + registerIndexFactory(createIndexFactory(partitioned_hnsw.CreateFactory[float32](32))) registerTokenizer(GeoTokenizer{}) registerTokenizer(IntTokenizer{}) registerTokenizer(FloatTokenizer{}) From c8951dc9402bfd897a75cfab17ad72be573a3038 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Wed, 25 Jun 2025 22:54:58 +0530 Subject: [PATCH 05/52] added changes --- schema/parse.go | 2 +- tok/tok.go | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/schema/parse.go b/schema/parse.go index 776596d292b..1fd4d64128d 100644 --- a/schema/parse.go +++ b/schema/parse.go @@ -306,7 +306,7 @@ func parseTokenOrVectorIndexSpec( tokenizer, has := tok.GetTokenizer(tokenOrFactoryName) if !has { return tokenOrFactoryName, nil, false, - next.Errorf("Invalid tokenizer %s", next.Val) + next.Errorf("Invalid tokenizer 1 %s", next.Val) } tokenizerType, ok := types.TypeForName(tokenizer.Type()) x.AssertTrue(ok) // Type is validated during tokenizer loading. diff --git a/tok/tok.go b/tok/tok.go index e20a647c43b..b7f6d3976c0 100644 --- a/tok/tok.go +++ b/tok/tok.go @@ -7,6 +7,7 @@ package tok import ( "encoding/binary" + "fmt" "math/big" "plugin" "strings" @@ -156,6 +157,7 @@ func GetTokenizer(name string) (Tokenizer, bool) { // GetIndexFactory returns IndexFactory given name. func GetIndexFactory(name string) (IndexFactory, bool) { + fmt.Println("HERE GET INDEX FACTORY", indexFactories) f, found := indexFactories[name] return f, found } From a87231a20709f33a9ee946f7387025f20884be7b Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Wed, 25 Jun 2025 22:55:56 +0530 Subject: [PATCH 06/52] added changes --- tok/tok.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tok/tok.go b/tok/tok.go index b7f6d3976c0..3931a27972d 100644 --- a/tok/tok.go +++ b/tok/tok.go @@ -157,7 +157,7 @@ func GetTokenizer(name string) (Tokenizer, bool) { // GetIndexFactory returns IndexFactory given name. func GetIndexFactory(name string) (IndexFactory, bool) { - fmt.Println("HERE GET INDEX FACTORY", indexFactories) + fmt.Println("HERE GET INDEX FACTORY", indexFactories, name) f, found := indexFactories[name] return f, found } From 874638aca22d4afe236cc533fef14483847f020d Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Wed, 25 Jun 2025 22:57:03 +0530 Subject: [PATCH 07/52] added changes --- tok/partitioned_hnsw/partitioned_factory.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tok/partitioned_hnsw/partitioned_factory.go b/tok/partitioned_hnsw/partitioned_factory.go index 28dc608a362..3d803a3fa6c 100644 --- a/tok/partitioned_hnsw/partitioned_factory.go +++ b/tok/partitioned_hnsw/partitioned_factory.go @@ -19,7 +19,7 @@ import ( const ( NumClustersOpt string = "numClusters" PartitionStratOpt string = "partitionStratOpt" - PartitionedHNSW string = "partionedHNSW" + PartitionedHNSW string = "partionedhnsw" ) type partitionedHNSWIndexFactory[T c.Float] struct { From 09f4093f049385c35e5b0b1912cb972235a3436f Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Wed, 25 Jun 2025 23:00:13 +0530 Subject: [PATCH 08/52] added changes --- tok/partitioned_hnsw/partitioned_hnsw.go | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/tok/partitioned_hnsw/partitioned_hnsw.go b/tok/partitioned_hnsw/partitioned_hnsw.go index 5dcc6dd564f..49a6d296c50 100644 --- a/tok/partitioned_hnsw/partitioned_hnsw.go +++ b/tok/partitioned_hnsw/partitioned_hnsw.go @@ -32,13 +32,8 @@ type partitionedHNSW[T c.Float] struct { } func (ph *partitionedHNSW[T]) applyOptions(o opt.Options) error { - if o.Specifies(NumClustersOpt) { - ph.numClusters, _, _ = opt.GetOpt(o, NumClustersOpt, 1000) - } - - if o.Specifies(PartitionStratOpt) { - ph.partitionStrat, _, _ = opt.GetOpt(o, PartitionStratOpt, "kmeans") - } + ph.numClusters, _, _ = opt.GetOpt(o, NumClustersOpt, 1000) + ph.partitionStrat, _, _ = opt.GetOpt(o, PartitionStratOpt, "kmeans") if ph.partitionStrat != "kmeans" && ph.partitionStrat != "query" { return errors.New("partition strategy must be kmeans or query") From 7b14cac9514a345b0e03315b8abf70a1511c57f1 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Thu, 26 Jun 2025 06:45:41 +0530 Subject: [PATCH 09/52] added changes --- tok/hnsw/persistent_hnsw.go | 15 ++++++++++++++- 1 file changed, 14 insertions(+), 1 deletion(-) diff --git a/tok/hnsw/persistent_hnsw.go b/tok/hnsw/persistent_hnsw.go index f1b0e4f3d84..729a8ee1b03 100644 --- a/tok/hnsw/persistent_hnsw.go +++ b/tok/hnsw/persistent_hnsw.go @@ -141,7 +141,20 @@ func (ph *persistentHNSW[T]) NumThreads() int { } func (ph *persistentHNSW[T]) BuildInsert(ctx context.Context, uid uint64, vec []T) error { - _, err := ph.Insert(ctx, ph.cache, uid, vec) + newPh := &persistentHNSW[T]{ + maxLevels: ph.maxLevels, + efConstruction: ph.efConstruction, + efSearch: ph.efSearch, + pred: ph.pred, + vecEntryKey: ph.vecEntryKey, + vecKey: ph.vecKey, + vecDead: ph.vecDead, + simType: ph.simType, + floatBits: ph.floatBits, + nodeAllEdges: make(map[uint64][][]uint64), + cache: ph.cache, + } + _, err := newPh.Insert(ctx, ph.cache, uid, vec) return err } From 1cf52d89024c8b4fc6ac95673c26029e2a86a5e1 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Thu, 26 Jun 2025 07:03:01 +0530 Subject: [PATCH 10/52] added changes --- tok/partitioned_hnsw/partitioned_hnsw.go | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/tok/partitioned_hnsw/partitioned_hnsw.go b/tok/partitioned_hnsw/partitioned_hnsw.go index 49a6d296c50..a42a44e7ff4 100644 --- a/tok/partitioned_hnsw/partitioned_hnsw.go +++ b/tok/partitioned_hnsw/partitioned_hnsw.go @@ -21,7 +21,6 @@ type partitionedHNSW[T c.Float] struct { clusterMap map[int]index.VectorIndex[T] numClusters int - factory index.IndexFactory[T] partition index.VectorPartitionStrat[T] hnswOptions opt.Options @@ -45,9 +44,9 @@ func (ph *partitionedHNSW[T]) applyOptions(o opt.Options) error { ph.buildPass = 0 ph.hnswOptions = o - ph.factory = hnsw.CreateFactory[T](ph.floatBits) for i := range ph.numClusters { - vi, err := ph.factory.Create(ph.pred, ph.hnswOptions, ph.floatBits) + factory := hnsw.CreateFactory[T](ph.floatBits) + vi, err := factory.Create(ph.pred, ph.hnswOptions, ph.floatBits) if err != nil { return err } From b66cfc6d2d3838d9f98646d8615fa5e3be2c6cee Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Thu, 26 Jun 2025 07:05:17 +0530 Subject: [PATCH 11/52] added changes --- tok/partitioned_hnsw/partitioned_hnsw.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tok/partitioned_hnsw/partitioned_hnsw.go b/tok/partitioned_hnsw/partitioned_hnsw.go index a42a44e7ff4..d94e3d976da 100644 --- a/tok/partitioned_hnsw/partitioned_hnsw.go +++ b/tok/partitioned_hnsw/partitioned_hnsw.go @@ -89,7 +89,7 @@ func (ph *partitionedHNSW[T]) NumIndexPasses() int { } func (ph *partitionedHNSW[T]) NumThreads() int { - return NUM_PASSES + return ph.numClusters } func (ph *partitionedHNSW[T]) NumSeedVectors() int { From 8a2dad2544fe02648797612f18df44e84307a35c Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Thu, 26 Jun 2025 07:17:16 +0530 Subject: [PATCH 12/52] added changes --- tok/partitioned_hnsw/partitioned_hnsw.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tok/partitioned_hnsw/partitioned_hnsw.go b/tok/partitioned_hnsw/partitioned_hnsw.go index d94e3d976da..cada5ff5b7b 100644 --- a/tok/partitioned_hnsw/partitioned_hnsw.go +++ b/tok/partitioned_hnsw/partitioned_hnsw.go @@ -6,6 +6,7 @@ package partitioned_hnsw import ( "context" "errors" + "fmt" "sync" c "github.com/hypermodeinc/dgraph/v25/tok/constraints" @@ -158,6 +159,7 @@ func (ph *partitionedHNSW[T]) Search(ctx context.Context, txn index.CacheType, q defer wg.Done() ids, err := ph.clusterMap[i].Search(ctx, txn, query, maxResults, filter) if err != nil { + fmt.Println("Error", err) return } mutex.Lock() From 9c5d1756b781162376ee531c59f53878cf5cf240 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Thu, 26 Jun 2025 07:19:02 +0530 Subject: [PATCH 13/52] added changes --- tok/partitioned_hnsw/partitioned_hnsw.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tok/partitioned_hnsw/partitioned_hnsw.go b/tok/partitioned_hnsw/partitioned_hnsw.go index cada5ff5b7b..bcb214c76ce 100644 --- a/tok/partitioned_hnsw/partitioned_hnsw.go +++ b/tok/partitioned_hnsw/partitioned_hnsw.go @@ -163,11 +163,13 @@ func (ph *partitionedHNSW[T]) Search(ctx context.Context, txn index.CacheType, q return } mutex.Lock() + fmt.Println("Addign result:", ids) res = append(res, ids...) mutex.Unlock() }(index) } wg.Wait() + fmt.Println("Result:", res) return ph.clusterMap[0].MergeResults(ctx, txn, res, query, maxResults, filter) } From 7b15cac1ee0f92befb16301fd67d799aafd2f637 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Thu, 26 Jun 2025 07:21:04 +0530 Subject: [PATCH 14/52] added changes --- tok/kmeans/kmeans.go | 2 +- tok/partitioned_hnsw/partitioned_hnsw.go | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/tok/kmeans/kmeans.go b/tok/kmeans/kmeans.go index 00768b1f006..6f6693d9300 100644 --- a/tok/kmeans/kmeans.go +++ b/tok/kmeans/kmeans.go @@ -33,7 +33,7 @@ func (km *Kmeans[T]) AddVector(vec []T) error { } func (km *Kmeans[T]) FindIndexForSearch(vec []T) ([]int, error) { - res := make([]int, len(km.centroids.centroids)) + res := make([]int, km.NumSeedVectors()) for i := range res { res[i] = i } diff --git a/tok/partitioned_hnsw/partitioned_hnsw.go b/tok/partitioned_hnsw/partitioned_hnsw.go index bcb214c76ce..31e2ed7d149 100644 --- a/tok/partitioned_hnsw/partitioned_hnsw.go +++ b/tok/partitioned_hnsw/partitioned_hnsw.go @@ -169,7 +169,7 @@ func (ph *partitionedHNSW[T]) Search(ctx context.Context, txn index.CacheType, q }(index) } wg.Wait() - fmt.Println("Result:", res) + fmt.Println("Result:", res, indexes) return ph.clusterMap[0].MergeResults(ctx, txn, res, query, maxResults, filter) } From 9afdb8c4e0a5129f2d299e7aaa133aeb14fc3837 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Thu, 26 Jun 2025 07:23:07 +0530 Subject: [PATCH 15/52] added changes --- tok/partitioned_hnsw/partitioned_hnsw.go | 4 ---- 1 file changed, 4 deletions(-) diff --git a/tok/partitioned_hnsw/partitioned_hnsw.go b/tok/partitioned_hnsw/partitioned_hnsw.go index 31e2ed7d149..d94e3d976da 100644 --- a/tok/partitioned_hnsw/partitioned_hnsw.go +++ b/tok/partitioned_hnsw/partitioned_hnsw.go @@ -6,7 +6,6 @@ package partitioned_hnsw import ( "context" "errors" - "fmt" "sync" c "github.com/hypermodeinc/dgraph/v25/tok/constraints" @@ -159,17 +158,14 @@ func (ph *partitionedHNSW[T]) Search(ctx context.Context, txn index.CacheType, q defer wg.Done() ids, err := ph.clusterMap[i].Search(ctx, txn, query, maxResults, filter) if err != nil { - fmt.Println("Error", err) return } mutex.Lock() - fmt.Println("Addign result:", ids) res = append(res, ids...) mutex.Unlock() }(index) } wg.Wait() - fmt.Println("Result:", res, indexes) return ph.clusterMap[0].MergeResults(ctx, txn, res, query, maxResults, filter) } From b2700d8bbf5755a2c99ccc5f53c3163a6e651488 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Thu, 26 Jun 2025 07:40:54 +0530 Subject: [PATCH 16/52] added changes --- tok/partitioned_hnsw/partitioned_hnsw.go | 11 ++++++++--- tok/tok.go | 2 -- 2 files changed, 8 insertions(+), 5 deletions(-) diff --git a/tok/partitioned_hnsw/partitioned_hnsw.go b/tok/partitioned_hnsw/partitioned_hnsw.go index d94e3d976da..cc7f22f20fc 100644 --- a/tok/partitioned_hnsw/partitioned_hnsw.go +++ b/tok/partitioned_hnsw/partitioned_hnsw.go @@ -26,8 +26,9 @@ type partitionedHNSW[T c.Float] struct { hnswOptions opt.Options partitionStrat string - caches []index.CacheType - buildPass int + caches []index.CacheType + buildPass int + buildSyncMaps map[int]*sync.Mutex } func (ph *partitionedHNSW[T]) applyOptions(o opt.Options) error { @@ -75,7 +76,10 @@ func (ph *partitionedHNSW[T]) BuildInsert(ctx context.Context, uuid uint64, vec if index%NUM_PASSES != passIdx { return nil } - return ph.clusterMap[index].BuildInsert(ctx, uuid, vec) + ph.buildSyncMaps[index].Lock() + defer ph.buildSyncMaps[index].Unlock() + _, err = ph.clusterMap[index].Insert(ctx, ph.caches[index], uuid, vec) + return err } const NUM_PASSES = 10 @@ -107,6 +111,7 @@ func (ph *partitionedHNSW[T]) StartBuild(caches []index.CacheType) { if i%NUM_PASSES != (ph.buildPass - ph.partition.NumPasses()) { continue } + ph.buildSyncMaps[i] = &sync.Mutex{} ph.clusterMap[i].StartBuild([]index.CacheType{ph.caches[i]}) } } diff --git a/tok/tok.go b/tok/tok.go index 3931a27972d..e20a647c43b 100644 --- a/tok/tok.go +++ b/tok/tok.go @@ -7,7 +7,6 @@ package tok import ( "encoding/binary" - "fmt" "math/big" "plugin" "strings" @@ -157,7 +156,6 @@ func GetTokenizer(name string) (Tokenizer, bool) { // GetIndexFactory returns IndexFactory given name. func GetIndexFactory(name string) (IndexFactory, bool) { - fmt.Println("HERE GET INDEX FACTORY", indexFactories, name) f, found := indexFactories[name] return f, found } From 347d86fe06199b1d3901c3d0cc1b9f4c5cb6d575 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Thu, 26 Jun 2025 07:42:23 +0530 Subject: [PATCH 17/52] added changes --- tok/partitioned_hnsw/partitioned_factory.go | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/tok/partitioned_hnsw/partitioned_factory.go b/tok/partitioned_hnsw/partitioned_factory.go index 3d803a3fa6c..8c925ca003a 100644 --- a/tok/partitioned_hnsw/partitioned_factory.go +++ b/tok/partitioned_hnsw/partitioned_factory.go @@ -96,9 +96,10 @@ func (hf *partitionedHNSWIndexFactory[T]) createWithLock( return nil, err } retVal := &partitionedHNSW[T]{ - pred: name, - floatBits: floatBits, - clusterMap: map[int]index.VectorIndex[T]{}, + pred: name, + floatBits: floatBits, + clusterMap: map[int]index.VectorIndex[T]{}, + buildSyncMaps: map[int]*sync.Mutex{}, } err := retVal.applyOptions(o) if err != nil { From 9c24bd1cb1fb4bb41d42207b6210ce79818f113e Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Thu, 26 Jun 2025 07:44:48 +0530 Subject: [PATCH 18/52] added changes --- tok/partitioned_hnsw/partitioned_hnsw.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tok/partitioned_hnsw/partitioned_hnsw.go b/tok/partitioned_hnsw/partitioned_hnsw.go index cc7f22f20fc..1897679d2f7 100644 --- a/tok/partitioned_hnsw/partitioned_hnsw.go +++ b/tok/partitioned_hnsw/partitioned_hnsw.go @@ -108,10 +108,10 @@ func (ph *partitionedHNSW[T]) StartBuild(caches []index.CacheType) { } for i := range ph.clusterMap { + ph.buildSyncMaps[i] = &sync.Mutex{} if i%NUM_PASSES != (ph.buildPass - ph.partition.NumPasses()) { continue } - ph.buildSyncMaps[i] = &sync.Mutex{} ph.clusterMap[i].StartBuild([]index.CacheType{ph.caches[i]}) } } From 91b443b0d0646fa6a337925373909f6d48f114cc Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Thu, 26 Jun 2025 07:59:03 +0530 Subject: [PATCH 19/52] added changes --- tok/partitioned_hnsw/partitioned_hnsw.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/tok/partitioned_hnsw/partitioned_hnsw.go b/tok/partitioned_hnsw/partitioned_hnsw.go index 1897679d2f7..b36258beaff 100644 --- a/tok/partitioned_hnsw/partitioned_hnsw.go +++ b/tok/partitioned_hnsw/partitioned_hnsw.go @@ -102,7 +102,7 @@ func (ph *partitionedHNSW[T]) NumSeedVectors() int { func (ph *partitionedHNSW[T]) StartBuild(caches []index.CacheType) { ph.caches = caches - if ph.buildPass <= ph.partition.NumPasses() { + if ph.buildPass < ph.partition.NumPasses() { ph.partition.StartBuildPass() return } @@ -119,7 +119,7 @@ func (ph *partitionedHNSW[T]) StartBuild(caches []index.CacheType) { func (ph *partitionedHNSW[T]) EndBuild() []int { res := []int{} - if ph.buildPass > ph.partition.NumPasses() { + if ph.buildPass >= ph.partition.NumPasses() { for i := range ph.clusterMap { if i%NUM_PASSES != (ph.buildPass - ph.partition.NumPasses()) { continue @@ -135,7 +135,7 @@ func (ph *partitionedHNSW[T]) EndBuild() []int { return res } - if ph.buildPass <= ph.partition.NumPasses() { + if ph.buildPass < ph.partition.NumPasses() { ph.partition.EndBuildPass() } return []int{} From 7793f6fa8a27352f4677a10b9117019cafed873f Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Fri, 27 Jun 2025 21:56:44 +0530 Subject: [PATCH 20/52] added vt --- posting/index.go | 165 ++++++++++++++- schema/schema.go | 5 + tok/hnsw/helper.go | 245 ++++++----------------- tok/hnsw/persistent_factory.go | 11 +- tok/hnsw/persistent_hnsw.go | 111 ++++------ tok/index/index.go | 20 +- tok/partitioned_hnsw/partitioned_hnsw.go | 7 +- worker/backup.go | 5 + worker/task.go | 9 +- 9 files changed, 293 insertions(+), 285 deletions(-) diff --git a/posting/index.go b/posting/index.go index 667915ae000..b350ddfdf1a 100644 --- a/posting/index.go +++ b/posting/index.go @@ -163,21 +163,18 @@ func (txn *Txn) addIndexMutations(ctx context.Context, info *indexMutationInfo) len(info.factorySpecs) > 0 { // retrieve vector from inUuid save as inVec inVec := types.BytesAsFloatArray(data[0].Value.([]byte)) - tc := hnsw.NewTxnCache(NewViTxn(txn), txn.StartTs) + vt := &VectorTransaction{ + txn: txn, + startTs: txn.StartTs, + } indexer, err := info.factorySpecs[0].CreateIndex(attr) if err != nil { return []*pb.DirectedEdge{}, err } - edges, err := indexer.Insert(ctx, tc, uid, inVec) - if err != nil { + if err := indexer.Insert(ctx, vt, uid, inVec); err != nil { return []*pb.DirectedEdge{}, err } - pbEdges := []*pb.DirectedEdge{} - for _, e := range edges { - pbe := indexEdgeToPbEdge(e) - pbEdges = append(pbEdges, pbe) - } - return pbEdges, nil + return []*pb.DirectedEdge{}, nil } } @@ -1365,6 +1362,147 @@ func (rb *indexRebuildInfo) prefixesForTokIndexes() ([][]byte, error) { const numCentroids = 1000 +type VectorTransaction struct { + txn *Txn + startTs uint64 + vecPred string + edgePred string + vector map[uint64]*[]byte + edges map[uint64]*[]byte + others map[string]*[]byte +} + +func (vt *VectorTransaction) NewVT(startTs uint64) { + vt.txn = &Txn{ + StartTs: startTs, + cache: NewLocalCache(startTs), + } + vt.startTs = startTs + vt.vector = make(map[uint64]*[]byte) + vt.edges = make(map[uint64]*[]byte) + vt.others = make(map[string]*[]byte) +} + +func (vt *VectorTransaction) Find(prefix []byte, filter func(val []byte) bool) (uint64, error) { + return vt.txn.cache.Find(prefix, filter) +} + +func (vt *VectorTransaction) SetVector(uid uint64, vec *[]byte) { + vt.vector[uid] = vec +} + +func (vt *VectorTransaction) SetEdge(uid uint64, edge *[]byte) { + vt.edges[uid] = edge +} + +func (vt *VectorTransaction) SetOther(key string, val *[]byte) { + vt.others[key] = val +} + +func (vt *VectorTransaction) GetVector(uid uint64) *[]byte { + val, ok := vt.vector[uid] + if ok { + return val + } + pl, err := vt.txn.GetScalarList(x.DataKey(vt.vecPred, uid)) + if err != nil { + return nil + } + rval, err := pl.Value(vt.startTs) + if err != nil { + return nil + } + value := rval.Value.([]byte) + vt.vector[uid] = &value + return &value +} + +func (vt *VectorTransaction) GetEdge(uid uint64) *[]byte { + val, ok := vt.edges[uid] + if ok { + return val + } + pl, err := vt.txn.GetScalarList(x.DataKey(vt.edgePred, uid)) + if err != nil { + return nil + } + rval, err := pl.Value(vt.startTs) + if err != nil { + return nil + } + value := rval.Value.([]byte) + vt.edges[uid] = &value + return &value +} + +func (vt *VectorTransaction) GetOther(key string) *[]byte { + val, ok := vt.others[key] + if ok { + return val + } + pl, err := vt.txn.GetScalarList(x.DataKey(key, 1)) + if err != nil { + return nil + } + rval, err := pl.Value(vt.startTs) + if err != nil { + return nil + } + value := rval.Value.([]byte) + vt.others[key] = &value + return &value +} + +func (vt *VectorTransaction) Update() { + for uid, edges := range vt.edges { + posting := &pb.Posting{ + Uid: uid, + Op: uint32(pb.DirectedEdge_SET), + Value: *edges, + ValType: pb.Posting_BINARY, + } + pl := &pb.PostingList{ + Postings: []*pb.Posting{posting}, + } + data, err := proto.Marshal(pl) + if err != nil { + return + } + vt.txn.cache.deltas[string(x.DataKey(vt.edgePred, uid))] = data + } + + for str, edges := range vt.others { + posting := &pb.Posting{ + Uid: 1, + Op: uint32(pb.DirectedEdge_SET), + Value: *edges, + ValType: pb.Posting_BINARY, + } + pl := &pb.PostingList{ + Postings: []*pb.Posting{posting}, + } + data, err := proto.Marshal(pl) + if err != nil { + return + } + vt.txn.cache.deltas[string(x.DataKey(str, 1))] = data + } + + vt.vector = nil + vt.edges = nil + vt.others = nil +} + +func (vt *VectorTransaction) LockKey(key []byte) { +} + +func (vt *VectorTransaction) UnlockKey(key []byte) { +} + +func (vt *VectorTransaction) Ts() uint64 { + return vt.startTs +} + func rebuildVectorIndex(ctx context.Context, factorySpecs []*tok.FactoryCreateSpec, rb *IndexRebuild) error { pk := x.ParsedKey{Attr: rb.Attr} @@ -1406,7 +1544,13 @@ func rebuildVectorIndex(ctx context.Context, factorySpecs []*tok.FactoryCreateSp } caches := make([]tokIndex.CacheType, indexer.NumThreads()) for i := range caches { - caches[i] = hnsw.NewTxnCache(NewViTxn(txns[i]), rb.StartTs) + caches[i] = &VectorTransaction{ + txn: txns[i], + startTs: rb.StartTs, + vector: make(map[uint64]*[]byte), + edges: make(map[uint64]*[]byte), + others: make(map[string]*[]byte), + } } for pass_idx := range indexer.NumBuildPasses() { @@ -1460,6 +1604,7 @@ func rebuildVectorIndex(ctx context.Context, factorySpecs []*tok.FactoryCreateSp for _, idx := range indexer.EndBuild() { txns[idx].Update() + caches[idx].(*VectorTransaction).Update() writer := NewTxnWriter(pstore) x.ExponentialRetry(int(x.Config.MaxRetries), diff --git a/schema/schema.go b/schema/schema.go index 0069f728c2a..6d5b12f977b 100644 --- a/schema/schema.go +++ b/schema/schema.go @@ -473,6 +473,11 @@ func (s *state) PredicatesToDelete(pred string) []string { preds = append(preds, pred+hnsw.VecEntry) preds = append(preds, pred+hnsw.VecKeyword) preds = append(preds, pred+hnsw.VecDead) + for i := range 1000 { + preds = append(preds, fmt.Sprintf("%s%s_%d", pred, hnsw.VecEntry, i)) + preds = append(preds, fmt.Sprintf("%s%s_%d", pred, hnsw.VecKeyword, i)) + preds = append(preds, fmt.Sprintf("%s%s_%d", pred, hnsw.VecDead, i)) + } } } return preds diff --git a/tok/hnsw/helper.go b/tok/hnsw/helper.go index c3b78c7a488..183834a82bd 100644 --- a/tok/hnsw/helper.go +++ b/tok/hnsw/helper.go @@ -233,67 +233,6 @@ func GetSimType[T c.Float](indexType string, floatBits int) SimilarityType[T] { } } -// TxnCache implements CacheType interface -type TxnCache struct { - txn index.Txn - startTs uint64 -} - -func (tc *TxnCache) Get(key []byte) (rval []byte, rerr error) { - return tc.txn.Get(key) -} - -func (tc *TxnCache) Ts() uint64 { - return tc.startTs -} - -func (tc *TxnCache) Find(prefix []byte, filter func([]byte) bool) (uint64, error) { - return tc.txn.Find(prefix, filter) -} - -func NewTxnCache(txn index.Txn, startTs uint64) *TxnCache { - return &TxnCache{ - txn: txn, - startTs: startTs, - } -} - -// QueryCache implements index.CacheType interface -type QueryCache struct { - cache index.LocalCache - readTs uint64 -} - -func (qc *QueryCache) Find(prefix []byte, filter func([]byte) bool) (uint64, error) { - return qc.cache.Find(prefix, filter) -} - -func (qc *QueryCache) Get(key []byte) (rval []byte, rerr error) { - return qc.cache.Get(key) -} - -func (qc *QueryCache) Ts() uint64 { - return qc.readTs -} - -func NewQueryCache(cache index.LocalCache, readTs uint64) *QueryCache { - return &QueryCache{ - cache: cache, - readTs: readTs, - } -} - -// getDataFromKeyWithCacheType(keyString, uid, c) looks up data in c -// associated with keyString and uid. -func getDataFromKeyWithCacheType(keyString string, uid uint64, c index.CacheType) ([]byte, error) { - key := DataKey(keyString, uid) - data, err := c.Get(key) - if err != nil { - return nil, fmt.Errorf("%w: %w; %s", err, errFetchingPostingList, keyString+" with uid "+strconv.FormatUint(uid, 10)) - } - return data, nil -} - // populateEdgeDataFromStore(keyString, uid, c, edgeData) // will fill edgeData with the contents of the neighboring edges for // a given DataKey by looking into the given cache (which may result @@ -306,34 +245,22 @@ func populateEdgeDataFromKeyWithCacheType( uid uint64, c index.CacheType, edgeData *[][]uint64) (bool, error) { - data, err := getDataFromKeyWithCacheType(keyString, uid, c) - // Note that posting list fetching errors are treated as just not having - // found the data -- no harm, no foul, as it is probably a - // dead reference that we can ignore. - if err != nil && !errors.Is(err, errFetchingPostingList) { - return false, err - } + data := c.GetEdge(uid) if data == nil { return false, nil } - err = decodeUint64MatrixUnsafe(data, edgeData) + err := decodeUint64MatrixUnsafe(*data, edgeData) return true, err } // entryUuidInsert adds the entry uuid to the given key func entryUuidInsert( ctx context.Context, - key []byte, - txn index.Txn, + c index.CacheType, predEntryKey string, - entryUuid []byte) (*index.KeyValue, error) { - edge := &index.KeyValue{ - Entity: 1, - Attr: predEntryKey, - Value: entryUuid, - } - err := txn.AddMutationWithLockHeld(ctx, key, edge) - return edge, err + entryUuid []byte) error { + c.SetOther(predEntryKey, &entryUuid) + return nil } func ConcatStrings(strs ...string) string { @@ -363,22 +290,14 @@ var emptyVec = []byte{} // adds the data corresponding to a uid to the given vec variable in the form of []T // this does not allocate memory for vec, so it must be allocated before calling this function func (ph *persistentHNSW[T]) getVecFromUid(uid uint64, c index.CacheType, vec *[]T) error { - data, err := getDataFromKeyWithCacheType(ph.pred, uid, c) - if err != nil { - if errors.Is(err, errFetchingPostingList) { - // no vector. Return empty array of floats - index.BytesAsFloatArray(emptyVec, vec, ph.floatBits) - return fmt.Errorf("%w; %w", errNilVector, err) - } - return err - } - if data != nil { - index.BytesAsFloatArray(data, vec, ph.floatBits) - return nil - } else { + data := c.GetVector(uid) + if data == nil { + // no vector. Return empty array of floats index.BytesAsFloatArray(emptyVec, vec, ph.floatBits) - return errNilVector + return fmt.Errorf("%w; %w", errNilVector, errFetchingPostingList) } + index.BytesAsFloatArray(*data, vec, ph.floatBits) + return nil } // chooses whether to create the entry and start nodes based on if it already @@ -386,24 +305,19 @@ func (ph *persistentHNSW[T]) getVecFromUid(uid uint64, c index.CacheType, vec *[ // levels. func (ph *persistentHNSW[T]) createEntryAndStartNodes( ctx context.Context, - c *TxnCache, + c index.CacheType, inUuid uint64, - vec *[]T) (uint64, []*index.KeyValue, error) { - txn := c.txn - edges := []*index.KeyValue{} - entryKey := DataKey(ph.vecEntryKey, 1) // 0-profile_vector_entry - txn.LockKey(entryKey) - defer txn.UnlockKey(entryKey) - data, _ := txn.GetWithLockHeld(entryKey) - - create_edges := func(inUuid uint64) (uint64, []*index.KeyValue, error) { - startEdges, err := ph.addStartNodeToAllLevels(ctx, entryKey, txn, inUuid) + vec *[]T) (uint64, error) { + + data := c.GetOther(ph.vecEntryKey) + + create_edges := func(inUuid uint64) (uint64, error) { + err := ph.addStartNodeToAllLevels(ctx, c, inUuid) if err != nil { - return 0, []*index.KeyValue{}, err + return 0, err } // return entry node at all levels - edges = append(edges, startEdges...) - return 0, edges, nil + return 0, nil } if data == nil { @@ -411,7 +325,7 @@ func (ph *persistentHNSW[T]) createEntryAndStartNodes( return create_edges(inUuid) } - entry := BytesToUint64(data) // convert entry Uuid returned from Get to uint64 + entry := BytesToUint64(*data) // convert entry Uuid returned from Get to uint64 err := ph.getVecFromUid(entry, c, vec) if err != nil || len(*vec) == 0 { // The entry vector has been deleted. We have to create a new entry vector. @@ -423,7 +337,7 @@ func (ph *persistentHNSW[T]) createEntryAndStartNodes( return create_edges(entry) } - return entry, edges, nil + return entry, nil } // Converts the matrix into linear array that looks like @@ -493,47 +407,33 @@ func decodeUint64MatrixUnsafe(data []byte, matrix *[][]uint64) error { // adds empty layers to all levels func (ph *persistentHNSW[T]) addStartNodeToAllLevels( ctx context.Context, - entryKey []byte, - txn index.Txn, - inUuid uint64) ([]*index.KeyValue, error) { - edges := []*index.KeyValue{} - key := DataKey(ph.vecKey, inUuid) + c index.CacheType, + inUuid uint64) error { emptyEdgesBytes := encodeUint64MatrixUnsafe(make([][]uint64, ph.maxLevels)) // creates empty at all levels only for entry node - edge, err := ph.newPersistentEdgeKeyValueEntry(ctx, key, txn, inUuid, emptyEdgesBytes) + err := ph.newPersistentEdgeKeyValueEntry(ctx, c, inUuid, emptyEdgesBytes) if err != nil { - return []*index.KeyValue{}, err + return err } - edges = append(edges, edge) inUuidByte := Uint64ToBytes(inUuid) // add inUuid as entry for this structure from now on - edge, err = entryUuidInsert(ctx, entryKey, txn, ph.vecEntryKey, inUuidByte) + err = entryUuidInsert(ctx, c, ph.vecEntryKey, inUuidByte) if err != nil { - return []*index.KeyValue{}, err + return err } - edges = append(edges, edge) - return edges, nil + return nil } // creates a new edge with the given uuid and edges. Lock must be held before calling this function -func (ph *persistentHNSW[T]) newPersistentEdgeKeyValueEntry(ctx context.Context, key []byte, - txn index.Txn, uuid uint64, edges []byte) (*index.KeyValue, error) { - txn.LockKey(key) - defer txn.UnlockKey(key) - edge := &index.KeyValue{ - Entity: uuid, - Attr: ph.vecKey, - Value: edges, - } - if err := txn.AddMutationWithLockHeld(ctx, key, edge); err != nil { - return nil, err - } - return edge, nil +func (ph *persistentHNSW[T]) newPersistentEdgeKeyValueEntry(ctx context.Context, + c index.CacheType, uuid uint64, edges []byte) error { + c.SetEdge(uuid, &edges) + return nil } -func (ph *persistentHNSW[T]) distance_betw(ctx context.Context, tc *TxnCache, inUuid, outUuid uint64, inVec, +func (ph *persistentHNSW[T]) distance_betw(ctx context.Context, c index.CacheType, inUuid, outUuid uint64, inVec, outVec *[]T) T { - err := ph.getVecFromUid(outUuid, tc, outVec) + err := ph.getVecFromUid(outUuid, c, outVec) if err != nil { log.Printf("[ERROR] While getting vector %s", err) return -1 @@ -583,48 +483,39 @@ func (h *HeapDataHolder) Pop() interface{} { // addNeighbors adds the neighbors of the given uuid to the given level. // It returns the edge created and the error if any. -func (ph *persistentHNSW[T]) addNeighbors(ctx context.Context, tc *TxnCache, - uuid uint64, allLayerNeighbors [][]uint64) (*index.KeyValue, error) { - - txn := tc.txn - keyPred := ph.vecKey - key := DataKey(keyPred, uuid) - txn.LockKey(key) - defer txn.UnlockKey(key) - var nnEdgesErr error +func (ph *persistentHNSW[T]) addNeighbors(ctx context.Context, c index.CacheType, + uuid uint64, allLayerNeighbors [][]uint64) error { + + // Lock the vector key + edges_data := c.GetEdge(uuid) var allLayerEdges [][]uint64 - var ok bool - allLayerEdges, ok = ph.nodeAllEdges[uuid] - if !ok { - data, _ := txn.GetWithLockHeld(key) - if data == nil { - allLayerEdges = allLayerNeighbors - } else { - // all edges of nearest neighbor - err := decodeUint64MatrixUnsafe(data, &allLayerEdges) - if err != nil { - return nil, err - } - } + if edges_data != nil { + decodeUint64MatrixUnsafe(*edges_data, &allLayerEdges) + } else { + encodedData := encodeUint64MatrixUnsafe(allLayerNeighbors) + c.SetEdge(uuid, &encodedData) + return nil } + var inVec, outVec []T for level := range ph.maxLevels { - allLayerEdges[level], nnEdgesErr = ph.removeDeadNodes(allLayerEdges[level], tc) - if nnEdgesErr != nil { - return nil, nnEdgesErr + var err error + allLayerEdges[level], err = ph.removeDeadNodes(allLayerEdges[level], c) + if err != nil { + return err } // This adds at most efConstruction number of edges for each layer for this node allLayerEdges[level] = append(allLayerEdges[level], allLayerNeighbors[level]...) if len(allLayerEdges[level]) > ph.efConstruction { - err := ph.getVecFromUid(uuid, tc, &inVec) + err := ph.getVecFromUid(uuid, c, &inVec) if err != nil { log.Printf("[ERROR] While getting vector %s", err) } else { h := &HeapDataHolder{ data: allLayerEdges[level], compare: func(i, j uint64) bool { - return ph.distance_betw(ctx, tc, uuid, i, &inVec, &outVec) > - ph.distance_betw(ctx, tc, uuid, j, &inVec, &outVec) + return ph.distance_betw(ctx, c, uuid, i, &inVec, &outVec) > + ph.distance_betw(ctx, c, uuid, j, &inVec, &outVec) }} for _, e := range allLayerNeighbors[level] { @@ -636,34 +527,24 @@ func (ph *persistentHNSW[T]) addNeighbors(ctx context.Context, tc *TxnCache, } } - // on every modification of the layer edges, add it to in mem map so you dont have to always be reading - // from persistent storage - ph.nodeAllEdges[uuid] = allLayerEdges inboundEdgesBytes := encodeUint64MatrixUnsafe(allLayerEdges) - - edge := &index.KeyValue{ - Entity: uuid, - Attr: ph.vecKey, - Value: inboundEdgesBytes, - } - if err := txn.AddMutationWithLockHeld(ctx, key, edge); err != nil { - return nil, err - } - return edge, nil + c.SetEdge(uuid, &inboundEdgesBytes) + return nil } // removeDeadNodes(nnEdges, tc) removes dead nodes from nnEdges and returns the new nnEdges -func (ph *persistentHNSW[T]) removeDeadNodes(nnEdges []uint64, tc *TxnCache) ([]uint64, error) { +func (ph *persistentHNSW[T]) removeDeadNodes(nnEdges []uint64, c index.CacheType) ([]uint64, error) { // TODO add a path to delete deadNodes if ph.deadNodes == nil { - data, err := getDataFromKeyWithCacheType(ph.vecDead, 1, tc) - if err != nil && !errors.Is(err, errFetchingPostingList) { - return []uint64{}, err + data := c.GetOther(ph.vecDead) + if data == nil { + return []uint64{}, nil } var deadNodes []uint64 if data != nil { // if dead nodes exist, convert to []uint64 - deadNodes, err = ParseEdges(string(data)) + var err error + deadNodes, err = ParseEdges(string(*data)) if err != nil { return []uint64{}, err } diff --git a/tok/hnsw/persistent_factory.go b/tok/hnsw/persistent_factory.go index 4bc13b48ea6..5704a8e511b 100644 --- a/tok/hnsw/persistent_factory.go +++ b/tok/hnsw/persistent_factory.go @@ -113,12 +113,11 @@ func (hf *persistentIndexFactory[T]) createWithLock( return nil, err } retVal := &persistentHNSW[T]{ - pred: name, - vecEntryKey: ConcatStrings(name, VecEntry), - vecKey: ConcatStrings(name, VecKeyword), - vecDead: ConcatStrings(name, VecDead), - floatBits: floatBits, - nodeAllEdges: map[uint64][][]uint64{}, + pred: name, + vecEntryKey: ConcatStrings(name, VecEntry), + vecKey: ConcatStrings(name, VecKeyword), + vecDead: ConcatStrings(name, VecDead), + floatBits: floatBits, } err := retVal.applyOptions(o) if err != nil { diff --git a/tok/hnsw/persistent_hnsw.go b/tok/hnsw/persistent_hnsw.go index 729a8ee1b03..631163eb492 100644 --- a/tok/hnsw/persistent_hnsw.go +++ b/tok/hnsw/persistent_hnsw.go @@ -10,6 +10,7 @@ import ( "fmt" "sort" "strings" + "sync/atomic" "time" "github.com/golang/glog" @@ -31,9 +32,11 @@ type persistentHNSW[T c.Float] struct { floatBits int // nodeAllEdges[65443][1][3] indicates the 3rd neighbor in the first // layer for uuid 65443. The result will be a neighboring uuid. - nodeAllEdges map[uint64][][]uint64 - deadNodes map[uint64]struct{} - cache index.CacheType + deadNodes map[uint64]struct{} + cache index.CacheType + + entryVec []T + isEntryVecSet atomic.Bool } func GetPersistantOptions[T c.Float](o opt.Options) string { @@ -126,12 +129,10 @@ func (ph *persistentHNSW[T]) NumSeedVectors() int { } func (ph *persistentHNSW[T]) StartBuild(caches []index.CacheType) { - ph.nodeAllEdges = make(map[uint64][][]uint64) ph.cache = caches[0] } func (ph *persistentHNSW[T]) EndBuild() []int { - ph.nodeAllEdges = nil ph.cache = nil return []int{0} } @@ -151,11 +152,9 @@ func (ph *persistentHNSW[T]) BuildInsert(ctx context.Context, uid uint64, vec [] vecDead: ph.vecDead, simType: ph.simType, floatBits: ph.floatBits, - nodeAllEdges: make(map[uint64][][]uint64), cache: ph.cache, } - _, err := newPh.Insert(ctx, ph.cache, uid, vec) - return err + return newPh.Insert(ctx, ph.cache, uid, vec) } func (ph *persistentHNSW[T]) AddSeedVector(vec []T) { @@ -176,22 +175,13 @@ func (ph *persistentHNSW[T]) emptySearchResultWithError(e error) (*searchLayerRe // in persistent store) and false otherwise. // (Of course, it may also return an error if a problem was encountered). func (ph *persistentHNSW[T]) fillNeighborEdges(uuid uint64, c index.CacheType, edges *[][]uint64) (bool, error) { - var ok bool - *edges, ok = ph.nodeAllEdges[uuid] - if ok { - return true, nil + edge := ph.cache.GetEdge(uuid) + if edge == nil { + return false, nil } - - ok, err := populateEdgeDataFromKeyWithCacheType(ph.vecKey, uuid, c, edges) - if err != nil { + if err := decodeUint64MatrixUnsafe(*edge, edges); err != nil { return false, err } - if !ok { - return false, nil - } - - // add this to in mem storage of uid -> edges - ph.nodeAllEdges[uuid] = *edges return true, nil } @@ -405,24 +395,20 @@ func (ph *persistentHNSW[T]) PickStartNode( c index.CacheType, startVec *[]T) (uint64, error) { - data, err := getDataFromKeyWithCacheType(ph.vecEntryKey, 1, c) - if err != nil { - if errors.Is(err, errFetchingPostingList) { - // The index might be empty - return ph.calculateNewEntryVec(ctx, c, startVec) - } - return 0, err + data := c.GetOther(ph.vecEntryKey) + if data == nil { + return ph.calculateNewEntryVec(ctx, c, startVec) } - entry := BytesToUint64(data) - if err = ph.getVecFromUid(entry, c, startVec); err != nil && !errors.Is(err, errNilVector) { + entry := BytesToUint64(*data) + if err := ph.getVecFromUid(entry, c, startVec); err != nil && !errors.Is(err, errNilVector) { return 0, err } if len(*startVec) == 0 { return ph.calculateNewEntryVec(ctx, c, startVec) } - return entry, err + return entry, nil } // SearchWithPath allows persistentHNSW to implement index.OptionalIndexSupport. @@ -482,25 +468,21 @@ func (ph *persistentHNSW[T]) SearchWithPath( // InsertToPersistentStorage inserts a node into the hnsw graph and returns the // traversal path and the edges created func (ph *persistentHNSW[T]) Insert(ctx context.Context, c index.CacheType, - inUuid uint64, inVec []T) ([]*index.KeyValue, error) { - tc, ok := c.(*TxnCache) - if !ok { - return []*index.KeyValue{}, nil - } - _, edges, err := ph.insertHelper(ctx, tc, inUuid, inVec) - return edges, err + inUuid uint64, inVec []T) error { + _, err := ph.insertHelper(ctx, c, inUuid, inVec) + return err } // InsertToPersistentStorage inserts a node into the hnsw graph and returns the // traversal path and the edges created -func (ph *persistentHNSW[T]) insertHelper(ctx context.Context, tc *TxnCache, - inUuid uint64, inVec []T) ([]minPersistentHeapElement[T], []*index.KeyValue, error) { +func (ph *persistentHNSW[T]) insertHelper(ctx context.Context, c index.CacheType, + inUuid uint64, inVec []T) ([]minPersistentHeapElement[T], error) { // return all the new edges created at all HNSW levels var startVec []T - entry, edges, err := ph.createEntryAndStartNodes(ctx, tc, inUuid, &startVec) - if err != nil || len(edges) > 0 { - return []minPersistentHeapElement[T]{}, edges, err + entry, err := ph.createEntryAndStartNodes(ctx, c, inUuid, &startVec) + if err != nil { + return []minPersistentHeapElement[T]{}, err } if entry == inUuid { @@ -508,7 +490,7 @@ func (ph *persistentHNSW[T]) insertHelper(ctx context.Context, tc *TxnCache, // it'll just overwrite w the same info // only situation where you can add duplicate nodes is if your // mutation adds the same node as entry - return []minPersistentHeapElement[T]{}, []*index.KeyValue{}, nil + return []minPersistentHeapElement[T]{}, nil } // startVecs: vectors used to calc where to start up until inLevel, @@ -517,40 +499,33 @@ func (ph *persistentHNSW[T]) insertHelper(ctx context.Context, tc *TxnCache, // var nns []minPersistentHeapElement[T] visited := []minPersistentHeapElement[T]{} inLevel := getInsertLayer(ph.maxLevels) // calculate layer to insert node at (randomized every time) - var layerErr error for level := range inLevel { // perform insertion for layers [level, max_level) only, when level < inLevel just find better start - err := ph.getVecFromUid(entry, tc, &startVec) + err := ph.getVecFromUid(entry, c, &startVec) if err != nil { - return []minPersistentHeapElement[T]{}, []*index.KeyValue{}, err + return []minPersistentHeapElement[T]{}, err } - layerResult, err := ph.searchPersistentLayer(tc, level, entry, startVec, + layerResult, err := ph.searchPersistentLayer(c, level, entry, startVec, inVec, false, ph.efSearch, index.AcceptAll[T]) if err != nil { - return []minPersistentHeapElement[T]{}, []*index.KeyValue{}, err + return []minPersistentHeapElement[T]{}, err } entry = layerResult.bestNeighbor().index } - emptyEdges := make([][]uint64, ph.maxLevels) - _, err = ph.addNeighbors(ctx, tc, inUuid, emptyEdges) - if err != nil { - return []minPersistentHeapElement[T]{}, []*index.KeyValue{}, err - } - var outboundEdgesAllLayers = make([][]uint64, ph.maxLevels) var inboundEdgesAllLayersMap = make(map[uint64][][]uint64) nnUidArray := []uint64{} for level := inLevel; level < ph.maxLevels; level++ { - err := ph.getVecFromUid(entry, tc, &startVec) + err := ph.getVecFromUid(entry, c, &startVec) if err != nil { - return []minPersistentHeapElement[T]{}, []*index.KeyValue{}, err + return []minPersistentHeapElement[T]{}, err } - layerResult, err := ph.searchPersistentLayer(tc, level, entry, startVec, + layerResult, err := ph.searchPersistentLayer(c, level, entry, startVec, inVec, false, ph.efConstruction, index.AcceptAll[T]) if err != nil { - return []minPersistentHeapElement[T]{}, []*index.KeyValue{}, layerErr + return []minPersistentHeapElement[T]{}, err } entry = layerResult.bestNeighbor().index @@ -569,19 +544,17 @@ func (ph *persistentHNSW[T]) insertHelper(ctx context.Context, tc *TxnCache, append(outboundEdgesAllLayers[level], nns[i].index) } } - edge, err := ph.addNeighbors(ctx, tc, inUuid, outboundEdgesAllLayers) + err = ph.addNeighbors(ctx, c, inUuid, outboundEdgesAllLayers) + if err != nil { + return []minPersistentHeapElement[T]{}, err + } for i := range nnUidArray { - edge, err := ph.addNeighbors( - ctx, tc, nnUidArray[i], inboundEdgesAllLayersMap[nnUidArray[i]]) + err = ph.addNeighbors( + ctx, c, nnUidArray[i], inboundEdgesAllLayersMap[nnUidArray[i]]) if err != nil { - return []minPersistentHeapElement[T]{}, []*index.KeyValue{}, err + return []minPersistentHeapElement[T]{}, err } - edges = append(edges, edge) - } - if err != nil { - return []minPersistentHeapElement[T]{}, []*index.KeyValue{}, err } - edges = append(edges, edge) - return visited, edges, nil + return visited, nil } diff --git a/tok/index/index.go b/tok/index/index.go index 503c2b66faa..24f709900e6 100644 --- a/tok/index/index.go +++ b/tok/index/index.go @@ -129,7 +129,7 @@ type VectorIndex[T c.Float] interface { // Insert will add a vector and uuid into the existing VectorIndex. If // uuid already exists, it should throw an error to not insert duplicate uuids - Insert(ctx context.Context, c CacheType, uuid uint64, vec []T) ([]*KeyValue, error) + Insert(ctx context.Context, c CacheType, uuid uint64, vec []T) error BuildInsert(ctx context.Context, uuid uint64, vec []T) error AddSeedVector(vec []T) @@ -144,18 +144,16 @@ type VectorIndex[T c.Float] interface { // A Txn is an interface representation of a persistent storage transaction, // where multiple operations are performed on a database type Txn interface { - // StartTs gets the exact time that the transaction started, returned in uint64 format - StartTs() uint64 // Get uses a []byte key to return the Value corresponding to the key - Get(key []byte) (rval []byte, rerr error) + // Get(key []byte) (rval []byte, rerr error) // GetWithLockHeld uses a []byte key to return the Value corresponding to the key with a mutex lock held - GetWithLockHeld(key []byte) (rval []byte, rerr error) + // GetWithLockHeld(key []byte) (rval []byte, rerr error) Find(prefix []byte, filter func(val []byte) bool) (uint64, error) // Adds a mutation operation on a index.Txn interface, where the mutation // is represented in the form of an index.DirectedEdge - AddMutation(ctx context.Context, key []byte, t *KeyValue) error + // AddMutation(ctx context.Context, key []byte, t *KeyValue) error // Same as AddMutation but with a mutex lock held - AddMutationWithLockHeld(ctx context.Context, key []byte, t *KeyValue) error + // AddMutationWithLockHeld(ctx context.Context, key []byte, t *KeyValue) error // mutex lock LockKey(key []byte) // mutex unlock @@ -173,7 +171,13 @@ type LocalCache interface { // CacheType is an interface representation of the cache of a persistent storage system type CacheType interface { - Get(key []byte) (rval []byte, rerr error) + // Get(key []byte) (rval []byte, rerr error) Ts() uint64 Find(prefix []byte, filter func(val []byte) bool) (uint64, error) + SetVector(uid uint64, vec *[]byte) + SetEdge(uid uint64, edge *[]byte) + SetOther(key string, val *[]byte) + GetVector(uid uint64) *[]byte + GetEdge(uid uint64) *[]byte + GetOther(key string) *[]byte } diff --git a/tok/partitioned_hnsw/partitioned_hnsw.go b/tok/partitioned_hnsw/partitioned_hnsw.go index b36258beaff..b4bb94df8e9 100644 --- a/tok/partitioned_hnsw/partitioned_hnsw.go +++ b/tok/partitioned_hnsw/partitioned_hnsw.go @@ -78,8 +78,7 @@ func (ph *partitionedHNSW[T]) BuildInsert(ctx context.Context, uuid uint64, vec } ph.buildSyncMaps[index].Lock() defer ph.buildSyncMaps[index].Unlock() - _, err = ph.clusterMap[index].Insert(ctx, ph.caches[index], uuid, vec) - return err + return ph.clusterMap[index].BuildInsert(ctx, uuid, vec) } const NUM_PASSES = 10 @@ -141,10 +140,10 @@ func (ph *partitionedHNSW[T]) EndBuild() []int { return []int{} } -func (ph *partitionedHNSW[T]) Insert(ctx context.Context, txn index.CacheType, uid uint64, vec []T) ([]*index.KeyValue, error) { +func (ph *partitionedHNSW[T]) Insert(ctx context.Context, txn index.CacheType, uid uint64, vec []T) error { index, err := ph.partition.FindIndexForInsert(vec) if err != nil { - return nil, err + return err } return ph.clusterMap[index].Insert(ctx, txn, uid, vec) } diff --git a/worker/backup.go b/worker/backup.go index 0d803f47c06..1a58f6081e7 100644 --- a/worker/backup.go +++ b/worker/backup.go @@ -300,6 +300,11 @@ func ProcessBackupRequest(ctx context.Context, req *pb.BackupRequest) error { if pred.Type == "float32vector" && len(pred.IndexSpecs) != 0 { vecPredMap[gid] = append(predMap[gid], pred.Predicate+hnsw.VecEntry, pred.Predicate+hnsw.VecKeyword, pred.Predicate+hnsw.VecDead) + for i := range 1000 { + vecPredMap[gid] = append(vecPredMap[gid], fmt.Sprintf("%s%s_%d", pred.Predicate, hnsw.VecEntry, i)) + vecPredMap[gid] = append(vecPredMap[gid], fmt.Sprintf("%s%s_%d", pred.Predicate, hnsw.VecKeyword, i)) + vecPredMap[gid] = append(vecPredMap[gid], fmt.Sprintf("%s%s_%d", pred.Predicate, hnsw.VecDead, i)) + } } } } diff --git a/worker/task.go b/worker/task.go index 1b97bc98702..840a8314857 100644 --- a/worker/task.go +++ b/worker/task.go @@ -33,7 +33,6 @@ import ( "github.com/hypermodeinc/dgraph/v25/schema" ctask "github.com/hypermodeinc/dgraph/v25/task" "github.com/hypermodeinc/dgraph/v25/tok" - "github.com/hypermodeinc/dgraph/v25/tok/hnsw" "github.com/hypermodeinc/dgraph/v25/tok/index" "github.com/hypermodeinc/dgraph/v25/types" "github.com/hypermodeinc/dgraph/v25/types/facets" @@ -356,17 +355,15 @@ func (qs *queryState) handleValuePostings(ctx context.Context, args funcArgs) er return err } //TODO: generate maxLevels from schema, filter, etc. - qc := hnsw.NewQueryCache( - posting.NewViLocalCache(qs.cache), - args.q.ReadTs, - ) + vt := &posting.VectorTransaction{} + vt.NewVT(q.ReadTs) indexer, err := cspec.CreateIndex(args.q.Attr) if err != nil { return err } - nnUids, err := indexer.Search(ctx, qc, srcFn.vectorInfo, + nnUids, err := indexer.Search(ctx, vt, srcFn.vectorInfo, int(numNeighbors), index.AcceptAll[float32]) if err != nil { return err From 992f9e8caa14cf433822165db974d59f5ac68c44 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Sat, 28 Jun 2025 00:16:22 +0530 Subject: [PATCH 21/52] added changes --- posting/index.go | 11 ++++++++--- 1 file changed, 8 insertions(+), 3 deletions(-) diff --git a/posting/index.go b/posting/index.go index b350ddfdf1a..0a8eebe1975 100644 --- a/posting/index.go +++ b/posting/index.go @@ -1551,6 +1551,10 @@ func rebuildVectorIndex(ctx context.Context, factorySpecs []*tok.FactoryCreateSp edges: make(map[uint64]*[]byte), others: make(map[string]*[]byte), } + if len(caches) > 1 { + caches[i].(*VectorTransaction).vecPred = rb.Attr + caches[i].(*VectorTransaction).edgePred = fmt.Sprintf("%s_%d", rb.Attr, i) + } } for pass_idx := range indexer.NumBuildPasses() { @@ -2041,10 +2045,11 @@ func prefixesToDropVectorIndexEdges(ctx context.Context, rb *IndexRebuild) [][]b prefixes = append(prefixes, x.PredicatePrefix(hnsw.ConcatStrings(rb.Attr, hnsw.VecDead))) prefixes = append(prefixes, x.PredicatePrefix(hnsw.ConcatStrings(rb.Attr, hnsw.VecKeyword))) - for i := range hnsw.VectorIndexMaxLevels { - prefixes = append(prefixes, x.PredicatePrefix(hnsw.ConcatStrings(rb.Attr, hnsw.VecKeyword, fmt.Sprint(i)))) + for i := range 1000 { + prefixes = append(prefixes, x.PredicatePrefix(hnsw.ConcatStrings(rb.Attr, hnsw.VecKeyword, fmt.Sprintf("_%d", i)))) + prefixes = append(prefixes, x.PredicatePrefix(hnsw.ConcatStrings(rb.Attr, hnsw.VecDead, fmt.Sprintf("_%d", i)))) + prefixes = append(prefixes, x.PredicatePrefix(hnsw.ConcatStrings(rb.Attr, hnsw.VecEntry, fmt.Sprintf("_%d", i)))) } - return prefixes } From 997d35590f2376ef5f6e47906b4b2e7a6074a2ac Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Sat, 28 Jun 2025 01:18:28 +0530 Subject: [PATCH 22/52] added changes --- posting/index.go | 54 +++++++++++++++--------- tok/hnsw/persistent_hnsw.go | 17 +++++--- tok/index/index.go | 7 +-- tok/partitioned_hnsw/partitioned_hnsw.go | 20 ++++++--- worker/task.go | 12 ++++-- 5 files changed, 69 insertions(+), 41 deletions(-) diff --git a/posting/index.go b/posting/index.go index 0a8eebe1975..dab0413d5c7 100644 --- a/posting/index.go +++ b/posting/index.go @@ -163,15 +163,16 @@ func (txn *Txn) addIndexMutations(ctx context.Context, info *indexMutationInfo) len(info.factorySpecs) > 0 { // retrieve vector from inUuid save as inVec inVec := types.BytesAsFloatArray(data[0].Value.([]byte)) - vt := &VectorTransaction{ - txn: txn, - startTs: txn.StartTs, - } indexer, err := info.factorySpecs[0].CreateIndex(attr) if err != nil { return []*pb.DirectedEdge{}, err } - if err := indexer.Insert(ctx, vt, uid, inVec); err != nil { + caches := GetVectorTransactions(indexer.NumThreads(), txn.StartTs, attr) + for i := range caches { + caches[i].(*VectorTransaction).txn = txn + } + indexer.SetCaches(caches) + if err := indexer.Insert(ctx, uid, inVec); err != nil { return []*pb.DirectedEdge{}, err } return []*pb.DirectedEdge{}, nil @@ -1362,6 +1363,18 @@ func (rb *indexRebuildInfo) prefixesForTokIndexes() ([][]byte, error) { const numCentroids = 1000 +func GetVectorTransactions(n int, startTs uint64, pred string) []tokIndex.CacheType { + retVal := make([]tokIndex.CacheType, n) + for i := 0; i < n; i++ { + retVal[i] = &VectorTransaction{} + retVal[i].(*VectorTransaction).NewVT(startTs) + if n > 1 { + retVal[i].(*VectorTransaction).UpdateSplit(i, pred) + } + } + return retVal +} + type VectorTransaction struct { txn *Txn startTs uint64 @@ -1372,6 +1385,16 @@ type VectorTransaction struct { others map[string]*[]byte } +func (vt *VectorTransaction) SetCache(cache *LocalCache) { + vt.txn = NewTxn(vt.startTs) + vt.txn.cache = cache +} + +func (vt *VectorTransaction) UpdateSplit(i int, pred string) { + vt.vecPred = pred + vt.edgePred = fmt.Sprintf("%s_%d", pred, i) +} + func (vt *VectorTransaction) NewVT(startTs uint64) { vt.txn = &Txn{ StartTs: startTs, @@ -1542,25 +1565,16 @@ func rebuildVectorIndex(ctx context.Context, factorySpecs []*tok.FactoryCreateSp for i := range txns { txns[i] = NewTxn(rb.StartTs) } - caches := make([]tokIndex.CacheType, indexer.NumThreads()) - for i := range caches { - caches[i] = &VectorTransaction{ - txn: txns[i], - startTs: rb.StartTs, - vector: make(map[uint64]*[]byte), - edges: make(map[uint64]*[]byte), - others: make(map[string]*[]byte), - } - if len(caches) > 1 { - caches[i].(*VectorTransaction).vecPred = rb.Attr - caches[i].(*VectorTransaction).edgePred = fmt.Sprintf("%s_%d", rb.Attr, i) - } + caches := GetVectorTransactions(indexer.NumThreads(), rb.StartTs, rb.Attr) + for i := range len(caches) { + caches[i].(*VectorTransaction).txn = txns[i] } + indexer.SetCaches(caches) for pass_idx := range indexer.NumBuildPasses() { fmt.Println("Building pass", pass_idx) - indexer.StartBuild(caches) + indexer.StartBuild() builder := rebuilder{attr: rb.Attr, prefix: pk.DataPrefix(), startTs: rb.StartTs} builder.fn = func(uid uint64, pl *List, txn *Txn) ([]*pb.DirectedEdge, error) { @@ -1586,7 +1600,7 @@ func rebuildVectorIndex(ctx context.Context, factorySpecs []*tok.FactoryCreateSp for pass_idx := range indexer.NumIndexPasses() { fmt.Println("Indexing pass", pass_idx) - indexer.StartBuild(caches) + indexer.StartBuild() builder := rebuilder{attr: rb.Attr, prefix: pk.DataPrefix(), startTs: rb.StartTs} builder.fn = func(uid uint64, pl *List, txn *Txn) ([]*pb.DirectedEdge, error) { diff --git a/tok/hnsw/persistent_hnsw.go b/tok/hnsw/persistent_hnsw.go index 631163eb492..70105f61f85 100644 --- a/tok/hnsw/persistent_hnsw.go +++ b/tok/hnsw/persistent_hnsw.go @@ -128,8 +128,11 @@ func (ph *persistentHNSW[T]) NumSeedVectors() int { return 0 } -func (ph *persistentHNSW[T]) StartBuild(caches []index.CacheType) { - ph.cache = caches[0] +func (ph *persistentHNSW[T]) StartBuild() { +} + +func (ph *persistentHNSW[T]) SetCaches(c []index.CacheType) { + ph.cache = c[0] } func (ph *persistentHNSW[T]) EndBuild() []int { @@ -154,7 +157,7 @@ func (ph *persistentHNSW[T]) BuildInsert(ctx context.Context, uid uint64, vec [] floatBits: ph.floatBits, cache: ph.cache, } - return newPh.Insert(ctx, ph.cache, uid, vec) + return newPh.Insert(ctx, uid, vec) } func (ph *persistentHNSW[T]) AddSeedVector(vec []T) { @@ -288,9 +291,9 @@ func (ph *persistentHNSW[T]) searchPersistentLayer( // Search searches the hnsw graph for the nearest neighbors of the query vector // and returns the traversal path and the nearest neighbors -func (ph *persistentHNSW[T]) Search(ctx context.Context, c index.CacheType, query []T, +func (ph *persistentHNSW[T]) Search(ctx context.Context, query []T, maxResults int, filter index.SearchFilter[T]) (nnUids []uint64, err error) { - r, err := ph.SearchWithPath(ctx, c, query, maxResults, filter) + r, err := ph.SearchWithPath(ctx, ph.cache, query, maxResults, filter) return r.Neighbors, err } @@ -467,9 +470,9 @@ func (ph *persistentHNSW[T]) SearchWithPath( // InsertToPersistentStorage inserts a node into the hnsw graph and returns the // traversal path and the edges created -func (ph *persistentHNSW[T]) Insert(ctx context.Context, c index.CacheType, +func (ph *persistentHNSW[T]) Insert(ctx context.Context, inUuid uint64, inVec []T) error { - _, err := ph.insertHelper(ctx, c, inUuid, inVec) + _, err := ph.insertHelper(ctx, ph.cache, inUuid, inVec) return err } diff --git a/tok/index/index.go b/tok/index/index.go index 24f709900e6..c4836094a2f 100644 --- a/tok/index/index.go +++ b/tok/index/index.go @@ -113,7 +113,7 @@ type VectorIndex[T c.Float] interface { // based on some input criteria. The maxResults count is counted *after* // being filtered. In other words, we only count those results that had not // been filtered out. - Search(ctx context.Context, c CacheType, query []T, + Search(ctx context.Context, query []T, maxResults int, filter SearchFilter[T]) ([]uint64, error) @@ -129,16 +129,17 @@ type VectorIndex[T c.Float] interface { // Insert will add a vector and uuid into the existing VectorIndex. If // uuid already exists, it should throw an error to not insert duplicate uuids - Insert(ctx context.Context, c CacheType, uuid uint64, vec []T) error + Insert(ctx context.Context, uuid uint64, vec []T) error BuildInsert(ctx context.Context, uuid uint64, vec []T) error AddSeedVector(vec []T) NumBuildPasses() int NumIndexPasses() int NumSeedVectors() int - StartBuild(caches []CacheType) + StartBuild() EndBuild() []int NumThreads() int + SetCaches(caches []CacheType) } // A Txn is an interface representation of a persistent storage transaction, diff --git a/tok/partitioned_hnsw/partitioned_hnsw.go b/tok/partitioned_hnsw/partitioned_hnsw.go index b4bb94df8e9..71d876a9a27 100644 --- a/tok/partitioned_hnsw/partitioned_hnsw.go +++ b/tok/partitioned_hnsw/partitioned_hnsw.go @@ -99,8 +99,14 @@ func (ph *partitionedHNSW[T]) NumSeedVectors() int { return ph.partition.NumSeedVectors() } -func (ph *partitionedHNSW[T]) StartBuild(caches []index.CacheType) { +func (ph *partitionedHNSW[T]) SetCaches(caches []index.CacheType) { ph.caches = caches + for i := range ph.clusterMap { + ph.clusterMap[i].SetCaches([]index.CacheType{ph.caches[i]}) + } +} + +func (ph *partitionedHNSW[T]) StartBuild() { if ph.buildPass < ph.partition.NumPasses() { ph.partition.StartBuildPass() return @@ -111,7 +117,7 @@ func (ph *partitionedHNSW[T]) StartBuild(caches []index.CacheType) { if i%NUM_PASSES != (ph.buildPass - ph.partition.NumPasses()) { continue } - ph.clusterMap[i].StartBuild([]index.CacheType{ph.caches[i]}) + ph.clusterMap[i].StartBuild() } } @@ -140,15 +146,15 @@ func (ph *partitionedHNSW[T]) EndBuild() []int { return []int{} } -func (ph *partitionedHNSW[T]) Insert(ctx context.Context, txn index.CacheType, uid uint64, vec []T) error { +func (ph *partitionedHNSW[T]) Insert(ctx context.Context, uid uint64, vec []T) error { index, err := ph.partition.FindIndexForInsert(vec) if err != nil { return err } - return ph.clusterMap[index].Insert(ctx, txn, uid, vec) + return ph.clusterMap[index].Insert(ctx, uid, vec) } -func (ph *partitionedHNSW[T]) Search(ctx context.Context, txn index.CacheType, query []T, maxResults int, filter index.SearchFilter[T]) ([]uint64, error) { +func (ph *partitionedHNSW[T]) Search(ctx context.Context, query []T, maxResults int, filter index.SearchFilter[T]) ([]uint64, error) { indexes, err := ph.partition.FindIndexForSearch(query) if err != nil { return nil, err @@ -160,7 +166,7 @@ func (ph *partitionedHNSW[T]) Search(ctx context.Context, txn index.CacheType, q wg.Add(1) go func(i int) { defer wg.Done() - ids, err := ph.clusterMap[i].Search(ctx, txn, query, maxResults, filter) + ids, err := ph.clusterMap[i].Search(ctx, query, maxResults, filter) if err != nil { return } @@ -170,7 +176,7 @@ func (ph *partitionedHNSW[T]) Search(ctx context.Context, txn index.CacheType, q }(index) } wg.Wait() - return ph.clusterMap[0].MergeResults(ctx, txn, res, query, maxResults, filter) + return ph.clusterMap[0].MergeResults(ctx, ph.caches[0], res, query, maxResults, filter) } func (ph *partitionedHNSW[T]) SearchWithPath(ctx context.Context, txn index.CacheType, query []T, maxResults int, filter index.SearchFilter[T]) (*index.SearchPathResult, error) { diff --git a/worker/task.go b/worker/task.go index 840a8314857..92c399b9468 100644 --- a/worker/task.go +++ b/worker/task.go @@ -354,16 +354,20 @@ func (qs *queryState) handleValuePostings(ctx context.Context, args funcArgs) er if err != nil { return err } - //TODO: generate maxLevels from schema, filter, etc. - vt := &posting.VectorTransaction{} - vt.NewVT(q.ReadTs) + //TODO: generate maxLevels from schema, filter, etc. indexer, err := cspec.CreateIndex(args.q.Attr) if err != nil { return err } - nnUids, err := indexer.Search(ctx, vt, srcFn.vectorInfo, + caches := posting.GetVectorTransactions(indexer.NumThreads(), q.ReadTs, args.q.Attr) + for i := range caches { + caches[i].(*posting.VectorTransaction).SetCache(qs.cache) + } + indexer.SetCaches(caches) + + nnUids, err := indexer.Search(ctx, srcFn.vectorInfo, int(numNeighbors), index.AcceptAll[float32]) if err != nil { return err From b7a9896928ed6c3d4a2f38f0f76c8f752c84e1f1 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Sat, 28 Jun 2025 01:23:16 +0530 Subject: [PATCH 23/52] added changes --- posting/index.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/posting/index.go b/posting/index.go index dab0413d5c7..6eab7da35c7 100644 --- a/posting/index.go +++ b/posting/index.go @@ -1511,6 +1511,8 @@ func (vt *VectorTransaction) Update() { vt.txn.cache.deltas[string(x.DataKey(str, 1))] = data } + fmt.Println("AFTER UPDATE", len(vt.txn.cache.deltas)) + vt.vector = nil vt.edges = nil vt.others = nil From deda3459a1a6481580a63a667a2035a0fe76d856 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Sat, 28 Jun 2025 02:22:28 +0530 Subject: [PATCH 24/52] added changes --- posting/index.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/posting/index.go b/posting/index.go index 6eab7da35c7..f6423a6ab67 100644 --- a/posting/index.go +++ b/posting/index.go @@ -1436,6 +1436,7 @@ func (vt *VectorTransaction) GetVector(uid uint64) *[]byte { return nil } value := rval.Value.([]byte) + fmt.Println("GET VECTOR: ", uid, value, vt.vecPred) vt.vector[uid] = &value return &value } @@ -1454,6 +1455,7 @@ func (vt *VectorTransaction) GetEdge(uid uint64) *[]byte { return nil } value := rval.Value.([]byte) + fmt.Println("GET EDGE: ", uid, value, vt.edgePred) vt.edges[uid] = &value return &value } From eff837c5b27abed0d863c22ed6366d3a5e57b081 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Sat, 28 Jun 2025 02:25:07 +0530 Subject: [PATCH 25/52] added changes --- posting/index.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/posting/index.go b/posting/index.go index f6423a6ab67..e80a7e6aebd 100644 --- a/posting/index.go +++ b/posting/index.go @@ -1436,7 +1436,7 @@ func (vt *VectorTransaction) GetVector(uid uint64) *[]byte { return nil } value := rval.Value.([]byte) - fmt.Println("GET VECTOR: ", uid, value, vt.vecPred) + //fmt.Println("GET VECTOR: ", uid, value, vt.vecPred) vt.vector[uid] = &value return &value } From eb2b87e9c1f4454cf262cd4e90286ddff4593e92 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Sat, 28 Jun 2025 02:28:38 +0530 Subject: [PATCH 26/52] added changes --- posting/index.go | 2 +- posting/mvcc.go | 3 +++ 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/posting/index.go b/posting/index.go index e80a7e6aebd..8cf2c6d314d 100644 --- a/posting/index.go +++ b/posting/index.go @@ -1447,6 +1447,7 @@ func (vt *VectorTransaction) GetEdge(uid uint64) *[]byte { return val } pl, err := vt.txn.GetScalarList(x.DataKey(vt.edgePred, uid)) + fmt.Println("GET EDGE: ", uid, pl, err, vt.edgePred) if err != nil { return nil } @@ -1455,7 +1456,6 @@ func (vt *VectorTransaction) GetEdge(uid uint64) *[]byte { return nil } value := rval.Value.([]byte) - fmt.Println("GET EDGE: ", uid, value, vt.edgePred) vt.edges[uid] = &value return &value } diff --git a/posting/mvcc.go b/posting/mvcc.go index 85021c67854..06e9b79dcf6 100644 --- a/posting/mvcc.go +++ b/posting/mvcc.go @@ -9,6 +9,7 @@ import ( "bytes" "context" "encoding/hex" + "fmt" "math" "strconv" "sync" @@ -297,6 +298,8 @@ func (txn *Txn) CommitToDisk(writer *TxnWriter, commitTs uint64) error { for ; idx < len(keys); idx++ { key := keys[idx] data := cache.deltas[key] + pk, _ := x.Parse([]byte(key)) + fmt.Println(pk, data) if len(data) == 0 { continue } From 0539ce860095143a7ce7d87e07cc95c26b212924 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Sat, 28 Jun 2025 02:41:02 +0530 Subject: [PATCH 27/52] added changes --- posting/index.go | 1 + 1 file changed, 1 insertion(+) diff --git a/posting/index.go b/posting/index.go index 8cf2c6d314d..586451eba6a 100644 --- a/posting/index.go +++ b/posting/index.go @@ -1466,6 +1466,7 @@ func (vt *VectorTransaction) GetOther(key string) *[]byte { return val } pl, err := vt.txn.GetScalarList(x.DataKey(key, 1)) + fmt.Println("GET OTHER: ", key, pl, err) if err != nil { return nil } From 5d330e7a0fa44d3688506ddeb44f8d057e537116 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Sat, 28 Jun 2025 02:44:54 +0530 Subject: [PATCH 28/52] added changes --- posting/index.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/posting/index.go b/posting/index.go index 586451eba6a..79be059de45 100644 --- a/posting/index.go +++ b/posting/index.go @@ -1447,11 +1447,11 @@ func (vt *VectorTransaction) GetEdge(uid uint64) *[]byte { return val } pl, err := vt.txn.GetScalarList(x.DataKey(vt.edgePred, uid)) - fmt.Println("GET EDGE: ", uid, pl, err, vt.edgePred) if err != nil { return nil } rval, err := pl.Value(vt.startTs) + fmt.Println("GET EDGE: ", uid, pl, err, vt.edgePred, rval) if err != nil { return nil } From 49608a50613f775983e7487f69e8a56c2ed8cd69 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Sat, 28 Jun 2025 02:46:56 +0530 Subject: [PATCH 29/52] added changes --- posting/index.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/posting/index.go b/posting/index.go index 79be059de45..7353d7d2f9b 100644 --- a/posting/index.go +++ b/posting/index.go @@ -1466,11 +1466,11 @@ func (vt *VectorTransaction) GetOther(key string) *[]byte { return val } pl, err := vt.txn.GetScalarList(x.DataKey(key, 1)) - fmt.Println("GET OTHER: ", key, pl, err) if err != nil { return nil } rval, err := pl.Value(vt.startTs) + fmt.Println("GET OTHER: ", key, pl, rval, err) if err != nil { return nil } From 793a249e04ec7896972c4ada125a9d5699edbf62 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Sat, 28 Jun 2025 02:48:30 +0530 Subject: [PATCH 30/52] added changes --- posting/index.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/posting/index.go b/posting/index.go index 7353d7d2f9b..a0741a9cf83 100644 --- a/posting/index.go +++ b/posting/index.go @@ -1427,7 +1427,7 @@ func (vt *VectorTransaction) GetVector(uid uint64) *[]byte { if ok { return val } - pl, err := vt.txn.GetScalarList(x.DataKey(vt.vecPred, uid)) + pl, err := vt.txn.Get(x.DataKey(vt.vecPred, uid)) if err != nil { return nil } @@ -1446,7 +1446,7 @@ func (vt *VectorTransaction) GetEdge(uid uint64) *[]byte { if ok { return val } - pl, err := vt.txn.GetScalarList(x.DataKey(vt.edgePred, uid)) + pl, err := vt.txn.Get(x.DataKey(vt.edgePred, uid)) if err != nil { return nil } @@ -1465,7 +1465,7 @@ func (vt *VectorTransaction) GetOther(key string) *[]byte { if ok { return val } - pl, err := vt.txn.GetScalarList(x.DataKey(key, 1)) + pl, err := vt.txn.Get(x.DataKey(key, 1)) if err != nil { return nil } From eb6f6792faa612a30dfb9df2dd5fd75f2d21ed15 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Sat, 28 Jun 2025 02:50:28 +0530 Subject: [PATCH 31/52] added changes --- posting/mvcc.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/posting/mvcc.go b/posting/mvcc.go index 06e9b79dcf6..fa081e1c02d 100644 --- a/posting/mvcc.go +++ b/posting/mvcc.go @@ -637,6 +637,7 @@ func ReadPostingList(key []byte, it *badger.Iterator) (*List, error) { // lists ended up being rolled-up multiple times. This issue was caught by the // uid-set Jepsen test. pk, err := x.Parse(key) + fmt.Println("READING ", pk) if err != nil { return nil, errors.Wrapf(err, "while reading posting list with key [%v]", key) } @@ -710,6 +711,7 @@ func ReadPostingList(key []byte, it *badger.Iterator) (*List, error) { if err := proto.Unmarshal(val, pl); err != nil { return err } + fmt.Println("HERE", pl) pl.CommitTs = item.Version() if l.mutationMap == nil { l.mutationMap = newMutableLayer() From 73ec4d38cb4f099ab8ebe1121fead4eb4cc1f662 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Sat, 28 Jun 2025 02:52:27 +0530 Subject: [PATCH 32/52] added changes --- posting/mvcc.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/posting/mvcc.go b/posting/mvcc.go index fa081e1c02d..01ccf25bcf9 100644 --- a/posting/mvcc.go +++ b/posting/mvcc.go @@ -711,12 +711,12 @@ func ReadPostingList(key []byte, it *badger.Iterator) (*List, error) { if err := proto.Unmarshal(val, pl); err != nil { return err } - fmt.Println("HERE", pl) pl.CommitTs = item.Version() if l.mutationMap == nil { l.mutationMap = newMutableLayer() } l.mutationMap.insertCommittedPostings(pl) + fmt.Println("HERE", pk, pl, l.mutationMap) return nil }) if err != nil { From 6e463c1cd92bcd4332c5084529af9a21af3f2cf2 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Sat, 28 Jun 2025 03:06:12 +0530 Subject: [PATCH 33/52] added changes --- posting/index.go | 8 ++++---- posting/list.go | 8 ++++++++ 2 files changed, 12 insertions(+), 4 deletions(-) diff --git a/posting/index.go b/posting/index.go index a0741a9cf83..815575ee2b4 100644 --- a/posting/index.go +++ b/posting/index.go @@ -1482,10 +1482,10 @@ func (vt *VectorTransaction) GetOther(key string) *[]byte { func (vt *VectorTransaction) Update() { for uid, edges := range vt.edges { posting := &pb.Posting{ - Uid: uid, - Op: uint32(pb.DirectedEdge_SET), + Op: Set, Value: *edges, ValType: pb.Posting_BINARY, + Uid: math.MaxUint64, } pl := &pb.PostingList{ Postings: []*pb.Posting{posting}, @@ -1499,10 +1499,10 @@ func (vt *VectorTransaction) Update() { for str, edges := range vt.others { posting := &pb.Posting{ - Uid: 1, - Op: uint32(pb.DirectedEdge_SET), + Op: Set, Value: *edges, ValType: pb.Posting_BINARY, + Uid: math.MaxUint64, } pl := &pb.PostingList{ Postings: []*pb.Posting{posting}, diff --git a/posting/list.go b/posting/list.go index 754a093a141..1471bf03af9 100644 --- a/posting/list.go +++ b/posting/list.go @@ -711,6 +711,14 @@ type ListOptions struct { First int } +func NewVectorPosting(uid uint64, vec *[]byte) *pb.Posting { + return &pb.Posting{ + Value: *vec, + ValType: pb.Posting_BINARY, + Op: Set, + } +} + // NewPosting takes the given edge and returns its equivalent representation as a posting. func NewPosting(t *pb.DirectedEdge) *pb.Posting { var op uint32 From 5ec29d4db777ef28f7b7338f5daf9f078282271e Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Sat, 28 Jun 2025 03:10:52 +0530 Subject: [PATCH 34/52] added changes --- posting/index.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/posting/index.go b/posting/index.go index 815575ee2b4..ec98e4a3132 100644 --- a/posting/index.go +++ b/posting/index.go @@ -1451,7 +1451,7 @@ func (vt *VectorTransaction) GetEdge(uid uint64) *[]byte { return nil } rval, err := pl.Value(vt.startTs) - fmt.Println("GET EDGE: ", uid, pl, err, vt.edgePred, rval) + //fmt.Println("GET EDGE: ", uid, pl, err, vt.edgePred, rval) if err != nil { return nil } @@ -1470,7 +1470,7 @@ func (vt *VectorTransaction) GetOther(key string) *[]byte { return nil } rval, err := pl.Value(vt.startTs) - fmt.Println("GET OTHER: ", key, pl, rval, err) + //fmt.Println("GET OTHER: ", key, pl, rval, err) if err != nil { return nil } From 6a07b13c483e5619cff0943aedd2b8e870b68a51 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Sat, 28 Jun 2025 03:12:32 +0530 Subject: [PATCH 35/52] added changes --- posting/mvcc.go | 9 ++++----- 1 file changed, 4 insertions(+), 5 deletions(-) diff --git a/posting/mvcc.go b/posting/mvcc.go index 01ccf25bcf9..4db4afc36a3 100644 --- a/posting/mvcc.go +++ b/posting/mvcc.go @@ -9,7 +9,6 @@ import ( "bytes" "context" "encoding/hex" - "fmt" "math" "strconv" "sync" @@ -298,8 +297,8 @@ func (txn *Txn) CommitToDisk(writer *TxnWriter, commitTs uint64) error { for ; idx < len(keys); idx++ { key := keys[idx] data := cache.deltas[key] - pk, _ := x.Parse([]byte(key)) - fmt.Println(pk, data) + //pk, _ := x.Parse([]byte(key)) + //fmt.Println(pk, data) if len(data) == 0 { continue } @@ -637,7 +636,7 @@ func ReadPostingList(key []byte, it *badger.Iterator) (*List, error) { // lists ended up being rolled-up multiple times. This issue was caught by the // uid-set Jepsen test. pk, err := x.Parse(key) - fmt.Println("READING ", pk) + //fmt.Println("READING ", pk) if err != nil { return nil, errors.Wrapf(err, "while reading posting list with key [%v]", key) } @@ -716,7 +715,7 @@ func ReadPostingList(key []byte, it *badger.Iterator) (*List, error) { l.mutationMap = newMutableLayer() } l.mutationMap.insertCommittedPostings(pl) - fmt.Println("HERE", pk, pl, l.mutationMap) + //fmt.Println("HERE", pk, pl, l.mutationMap) return nil }) if err != nil { From c312eda182aef485b8ffcbbd6878c3652dbb199a Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Sat, 28 Jun 2025 03:18:20 +0530 Subject: [PATCH 36/52] added changes --- posting/index.go | 1 + tok/hnsw/persistent_hnsw.go | 1 + 2 files changed, 2 insertions(+) diff --git a/posting/index.go b/posting/index.go index ec98e4a3132..91e76965042 100644 --- a/posting/index.go +++ b/posting/index.go @@ -1475,6 +1475,7 @@ func (vt *VectorTransaction) GetOther(key string) *[]byte { return nil } value := rval.Value.([]byte) + fmt.Println("GET OTHER: ", key, value) vt.others[key] = &value return &value } diff --git a/tok/hnsw/persistent_hnsw.go b/tok/hnsw/persistent_hnsw.go index 70105f61f85..11e641c6a06 100644 --- a/tok/hnsw/persistent_hnsw.go +++ b/tok/hnsw/persistent_hnsw.go @@ -403,6 +403,7 @@ func (ph *persistentHNSW[T]) PickStartNode( return ph.calculateNewEntryVec(ctx, c, startVec) } + fmt.Println("Start Node", data) entry := BytesToUint64(*data) if err := ph.getVecFromUid(entry, c, startVec); err != nil && !errors.Is(err, errNilVector) { return 0, err From 47ad085199244d7ffed1235028aa82694ef0867e Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Sat, 28 Jun 2025 03:20:59 +0530 Subject: [PATCH 37/52] added changes --- posting/index.go | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/posting/index.go b/posting/index.go index 91e76965042..5bdc4ff0faf 100644 --- a/posting/index.go +++ b/posting/index.go @@ -1456,6 +1456,9 @@ func (vt *VectorTransaction) GetEdge(uid uint64) *[]byte { return nil } value := rval.Value.([]byte) + if len(value) == 0 { + return nil + } vt.edges[uid] = &value return &value } @@ -1475,7 +1478,10 @@ func (vt *VectorTransaction) GetOther(key string) *[]byte { return nil } value := rval.Value.([]byte) - fmt.Println("GET OTHER: ", key, value) + //fmt.Println("GET OTHER: ", key, value) + if len(value) == 0 { + return nil + } vt.others[key] = &value return &value } From 706e1bb3ba010257ffb574b65f87ec82fea66bff Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Sat, 28 Jun 2025 03:24:51 +0530 Subject: [PATCH 38/52] added changes --- tok/hnsw/persistent_hnsw.go | 1 - 1 file changed, 1 deletion(-) diff --git a/tok/hnsw/persistent_hnsw.go b/tok/hnsw/persistent_hnsw.go index 11e641c6a06..70105f61f85 100644 --- a/tok/hnsw/persistent_hnsw.go +++ b/tok/hnsw/persistent_hnsw.go @@ -403,7 +403,6 @@ func (ph *persistentHNSW[T]) PickStartNode( return ph.calculateNewEntryVec(ctx, c, startVec) } - fmt.Println("Start Node", data) entry := BytesToUint64(*data) if err := ph.getVecFromUid(entry, c, startVec); err != nil && !errors.Is(err, errNilVector) { return 0, err From e779d3484c95ab5765dd4f591bbd9289bb142cd1 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Mon, 30 Jun 2025 21:06:29 +0530 Subject: [PATCH 39/52] added changes --- posting/index.go | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/posting/index.go b/posting/index.go index 5bdc4ff0faf..4931ed17f64 100644 --- a/posting/index.go +++ b/posting/index.go @@ -1383,6 +1383,8 @@ type VectorTransaction struct { vector map[uint64]*[]byte edges map[uint64]*[]byte others map[string]*[]byte + + readDisk bool } func (vt *VectorTransaction) SetCache(cache *LocalCache) { @@ -1404,6 +1406,7 @@ func (vt *VectorTransaction) NewVT(startTs uint64) { vt.vector = make(map[uint64]*[]byte) vt.edges = make(map[uint64]*[]byte) vt.others = make(map[string]*[]byte) + vt.readDisk = true } func (vt *VectorTransaction) Find(prefix []byte, filter func(val []byte) bool) (uint64, error) { @@ -1427,6 +1430,9 @@ func (vt *VectorTransaction) GetVector(uid uint64) *[]byte { if ok { return val } + if !vt.readDisk { + return nil + } pl, err := vt.txn.Get(x.DataKey(vt.vecPred, uid)) if err != nil { return nil @@ -1446,6 +1452,9 @@ func (vt *VectorTransaction) GetEdge(uid uint64) *[]byte { if ok { return val } + if !vt.readDisk { + return nil + } pl, err := vt.txn.Get(x.DataKey(vt.edgePred, uid)) if err != nil { return nil @@ -1468,6 +1477,9 @@ func (vt *VectorTransaction) GetOther(key string) *[]byte { if ok { return val } + if !vt.readDisk { + return nil + } pl, err := vt.txn.Get(x.DataKey(key, 1)) if err != nil { return nil @@ -1580,6 +1592,7 @@ func rebuildVectorIndex(ctx context.Context, factorySpecs []*tok.FactoryCreateSp caches := GetVectorTransactions(indexer.NumThreads(), rb.StartTs, rb.Attr) for i := range len(caches) { caches[i].(*VectorTransaction).txn = txns[i] + caches[i].(*VectorTransaction).readDisk = false } indexer.SetCaches(caches) From 450b40dca2ed6001d756ce7b06d8738b8239ccd4 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Mon, 30 Jun 2025 21:17:41 +0530 Subject: [PATCH 40/52] added changes --- posting/index.go | 1 + 1 file changed, 1 insertion(+) diff --git a/posting/index.go b/posting/index.go index 4931ed17f64..81f13bbb6c6 100644 --- a/posting/index.go +++ b/posting/index.go @@ -1418,6 +1418,7 @@ func (vt *VectorTransaction) SetVector(uid uint64, vec *[]byte) { } func (vt *VectorTransaction) SetEdge(uid uint64, edge *[]byte) { + fmt.Println("SETTING EDGE", uid, &edge) vt.edges[uid] = edge } From 9f0b41fb0c6a761feba6728be2a71dbb51d1adc2 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Mon, 30 Jun 2025 23:55:29 +0530 Subject: [PATCH 41/52] added changes --- posting/index.go | 2 +- tok/hnsw/persistent_hnsw.go | 5 +++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/posting/index.go b/posting/index.go index 81f13bbb6c6..e9e905bf85a 100644 --- a/posting/index.go +++ b/posting/index.go @@ -1418,7 +1418,7 @@ func (vt *VectorTransaction) SetVector(uid uint64, vec *[]byte) { } func (vt *VectorTransaction) SetEdge(uid uint64, edge *[]byte) { - fmt.Println("SETTING EDGE", uid, &edge) + fmt.Println("SETTING EDGE", uid, edge) vt.edges[uid] = edge } diff --git a/tok/hnsw/persistent_hnsw.go b/tok/hnsw/persistent_hnsw.go index 70105f61f85..af9f17068ec 100644 --- a/tok/hnsw/persistent_hnsw.go +++ b/tok/hnsw/persistent_hnsw.go @@ -12,11 +12,13 @@ import ( "strings" "sync/atomic" "time" + "unsafe" "github.com/golang/glog" c "github.com/hypermodeinc/dgraph/v25/tok/constraints" "github.com/hypermodeinc/dgraph/v25/tok/index" opt "github.com/hypermodeinc/dgraph/v25/tok/options" + "github.com/hypermodeinc/dgraph/v25/types" "github.com/pkg/errors" ) @@ -145,6 +147,9 @@ func (ph *persistentHNSW[T]) NumThreads() int { } func (ph *persistentHNSW[T]) BuildInsert(ctx context.Context, uid uint64, vec []T) error { + floatVec := *(*[]float32)(unsafe.Pointer(&vec)) + vecBytes := types.FloatArrayAsBytes(floatVec) + ph.cache.SetVector(uid, &vecBytes) newPh := &persistentHNSW[T]{ maxLevels: ph.maxLevels, efConstruction: ph.efConstruction, From 5660a29ff96dd37d5d65f281ce2980c9b5a41473 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Mon, 30 Jun 2025 23:59:43 +0530 Subject: [PATCH 42/52] added changes --- posting/index.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/posting/index.go b/posting/index.go index e9e905bf85a..1eaad6d711d 100644 --- a/posting/index.go +++ b/posting/index.go @@ -1418,7 +1418,7 @@ func (vt *VectorTransaction) SetVector(uid uint64, vec *[]byte) { } func (vt *VectorTransaction) SetEdge(uid uint64, edge *[]byte) { - fmt.Println("SETTING EDGE", uid, edge) + //fmt.Println("SETTING EDGE", uid, edge) vt.edges[uid] = edge } From dda6f14bbbe60508ef3ff932c7d8ad552218e23c Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Tue, 1 Jul 2025 00:27:50 +0530 Subject: [PATCH 43/52] added changes --- posting/index.go | 50 ++++++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 50 insertions(+) diff --git a/posting/index.go b/posting/index.go index 1eaad6d711d..c0df053d782 100644 --- a/posting/index.go +++ b/posting/index.go @@ -794,6 +794,54 @@ func (r *rebuilder) RunWithoutTemp(ctx context.Context) error { }) } +func printTreeStatsDeltas(txn *Txn) { + txn.cache.Lock() + + numLevels := 20 + numNodes := make([]int, numLevels) + numConnections := make([]int, numLevels) + + var temp [][]uint64 + for key, plMarshalled := range txn.cache.deltas { + pk, _ := x.Parse([]byte(key)) + var data pb.PostingList + proto.Unmarshal(plMarshalled, &data) + if strings.HasSuffix(pk.Attr, "__vector_") { + err := decodeUint64MatrixUnsafe(data.Postings[0].Value, &temp) + if err != nil { + fmt.Println("Error while decoding", err) + } + + for i := range temp { + if len(temp[i]) > 0 { + numNodes[i] += 1 + } + numConnections[i] += len(temp[i]) + } + + } + } + + for i := range numLevels { + fmt.Printf("%d, ", numNodes[i]) + } + fmt.Println("") + for i := range numLevels { + fmt.Printf("%d, ", numConnections[i]) + } + fmt.Println("") + for i := range numLevels { + if numNodes[i] == 0 { + fmt.Printf("0, ") + continue + } + fmt.Printf("%d, ", numConnections[i]/numNodes[i]) + } + fmt.Println("") + + txn.cache.Unlock() +} + func printTreeStats(txn *Txn) { txn.cache.Lock() @@ -1661,6 +1709,8 @@ func rebuildVectorIndex(ctx context.Context, factorySpecs []*tok.FactoryCreateSp return err }) + printTreeStatsDeltas(txns[idx]) + txns[idx].cache.deltas = nil txns[idx].cache.plists = nil txns[idx] = nil } From 6df6e9fa1fc3e3efe275e028f8bc405d815a17b4 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Tue, 1 Jul 2025 00:29:44 +0530 Subject: [PATCH 44/52] added changes --- posting/index.go | 1 + 1 file changed, 1 insertion(+) diff --git a/posting/index.go b/posting/index.go index c0df053d782..b16c0b173cf 100644 --- a/posting/index.go +++ b/posting/index.go @@ -806,6 +806,7 @@ func printTreeStatsDeltas(txn *Txn) { pk, _ := x.Parse([]byte(key)) var data pb.PostingList proto.Unmarshal(plMarshalled, &data) + fmt.Println("TREE STATS:", pk) if strings.HasSuffix(pk.Attr, "__vector_") { err := decodeUint64MatrixUnsafe(data.Postings[0].Value, &temp) if err != nil { From a6ee3b9660f8d738156eedad2c71cbc37991c88e Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Tue, 1 Jul 2025 00:32:59 +0530 Subject: [PATCH 45/52] added changes --- posting/index.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/posting/index.go b/posting/index.go index b16c0b173cf..070883e884e 100644 --- a/posting/index.go +++ b/posting/index.go @@ -1443,7 +1443,7 @@ func (vt *VectorTransaction) SetCache(cache *LocalCache) { func (vt *VectorTransaction) UpdateSplit(i int, pred string) { vt.vecPred = pred - vt.edgePred = fmt.Sprintf("%s_%d", pred, i) + vt.edgePred = fmt.Sprintf("%s%s_%d", pred, hnsw.VecKeyword, i) } func (vt *VectorTransaction) NewVT(startTs uint64) { From 4d3bce91dd61624c38aacf72a76382f3d36367f2 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Tue, 1 Jul 2025 00:36:49 +0530 Subject: [PATCH 46/52] added changes --- posting/index.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/posting/index.go b/posting/index.go index 070883e884e..11a349c014e 100644 --- a/posting/index.go +++ b/posting/index.go @@ -813,6 +813,8 @@ func printTreeStatsDeltas(txn *Txn) { fmt.Println("Error while decoding", err) } + fmt.Println(pk, temp) + for i := range temp { if len(temp[i]) > 0 { numNodes[i] += 1 From a0daad0647e6018ba0a299645a9487886d5fc6f8 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Tue, 1 Jul 2025 00:39:24 +0530 Subject: [PATCH 47/52] added changes --- posting/index.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/posting/index.go b/posting/index.go index 11a349c014e..9721e890d93 100644 --- a/posting/index.go +++ b/posting/index.go @@ -807,7 +807,7 @@ func printTreeStatsDeltas(txn *Txn) { var data pb.PostingList proto.Unmarshal(plMarshalled, &data) fmt.Println("TREE STATS:", pk) - if strings.HasSuffix(pk.Attr, "__vector_") { + if strings.Contains(pk.Attr, "__vector_") && !strings.Contains(pk.Attr, hnsw.VecEntry) { err := decodeUint64MatrixUnsafe(data.Postings[0].Value, &temp) if err != nil { fmt.Println("Error while decoding", err) From 21c923530b632315cff7ccf00a083c7e61ecf469 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Tue, 1 Jul 2025 00:53:08 +0530 Subject: [PATCH 48/52] added changes --- tok/hnsw/helper.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tok/hnsw/helper.go b/tok/hnsw/helper.go index 183834a82bd..5133ae35430 100644 --- a/tok/hnsw/helper.go +++ b/tok/hnsw/helper.go @@ -527,6 +527,8 @@ func (ph *persistentHNSW[T]) addNeighbors(ctx context.Context, c index.CacheType } } + fmt.Println("allLayerEdges", allLayerEdges, allLayerNeighbors) + inboundEdgesBytes := encodeUint64MatrixUnsafe(allLayerEdges) c.SetEdge(uuid, &inboundEdgesBytes) return nil From c05821c04e3c89b64b2a3017c22ab2a6f5acf567 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Tue, 1 Jul 2025 00:59:45 +0530 Subject: [PATCH 49/52] added changes --- tok/hnsw/helper.go | 2 ++ 1 file changed, 2 insertions(+) diff --git a/tok/hnsw/helper.go b/tok/hnsw/helper.go index 5133ae35430..42633810bce 100644 --- a/tok/hnsw/helper.go +++ b/tok/hnsw/helper.go @@ -497,6 +497,8 @@ func (ph *persistentHNSW[T]) addNeighbors(ctx context.Context, c index.CacheType return nil } + fmt.Println("allLayerEdges", uuid, allLayerEdges) + var inVec, outVec []T for level := range ph.maxLevels { var err error From d31ae7e7dd81da4b5cc9a62471e3df82b589be8e Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Tue, 1 Jul 2025 01:05:16 +0530 Subject: [PATCH 50/52] added changes --- tok/hnsw/helper.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tok/hnsw/helper.go b/tok/hnsw/helper.go index 42633810bce..e458fbc945f 100644 --- a/tok/hnsw/helper.go +++ b/tok/hnsw/helper.go @@ -542,7 +542,7 @@ func (ph *persistentHNSW[T]) removeDeadNodes(nnEdges []uint64, c index.CacheType if ph.deadNodes == nil { data := c.GetOther(ph.vecDead) if data == nil { - return []uint64{}, nil + return nnEdges, nil } var deadNodes []uint64 From c238c08ae8506166d69450e48f1a0edf5e94ebc5 Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Tue, 1 Jul 2025 01:12:56 +0530 Subject: [PATCH 51/52] added changes --- posting/index.go | 3 --- tok/hnsw/helper.go | 4 ---- 2 files changed, 7 deletions(-) diff --git a/posting/index.go b/posting/index.go index 9721e890d93..ba975069b6a 100644 --- a/posting/index.go +++ b/posting/index.go @@ -806,15 +806,12 @@ func printTreeStatsDeltas(txn *Txn) { pk, _ := x.Parse([]byte(key)) var data pb.PostingList proto.Unmarshal(plMarshalled, &data) - fmt.Println("TREE STATS:", pk) if strings.Contains(pk.Attr, "__vector_") && !strings.Contains(pk.Attr, hnsw.VecEntry) { err := decodeUint64MatrixUnsafe(data.Postings[0].Value, &temp) if err != nil { fmt.Println("Error while decoding", err) } - fmt.Println(pk, temp) - for i := range temp { if len(temp[i]) > 0 { numNodes[i] += 1 diff --git a/tok/hnsw/helper.go b/tok/hnsw/helper.go index e458fbc945f..4a608380a52 100644 --- a/tok/hnsw/helper.go +++ b/tok/hnsw/helper.go @@ -497,8 +497,6 @@ func (ph *persistentHNSW[T]) addNeighbors(ctx context.Context, c index.CacheType return nil } - fmt.Println("allLayerEdges", uuid, allLayerEdges) - var inVec, outVec []T for level := range ph.maxLevels { var err error @@ -529,8 +527,6 @@ func (ph *persistentHNSW[T]) addNeighbors(ctx context.Context, c index.CacheType } } - fmt.Println("allLayerEdges", allLayerEdges, allLayerNeighbors) - inboundEdgesBytes := encodeUint64MatrixUnsafe(allLayerEdges) c.SetEdge(uuid, &inboundEdgesBytes) return nil From 511d74e2bdc34a7c0b6557cf920ca9c408e1760c Mon Sep 17 00:00:00 2001 From: Harshil Goel Date: Tue, 1 Jul 2025 05:23:30 +0530 Subject: [PATCH 52/52] added changes --- tok/hnsw/helper.go | 1 - tok/partitioned_hnsw/partitioned_hnsw.go | 2 +- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/tok/hnsw/helper.go b/tok/hnsw/helper.go index 4a608380a52..d57c8b1b5b7 100644 --- a/tok/hnsw/helper.go +++ b/tok/hnsw/helper.go @@ -520,7 +520,6 @@ func (ph *persistentHNSW[T]) addNeighbors(ctx context.Context, c index.CacheType for _, e := range allLayerNeighbors[level] { heap.Push(h, e) - heap.Pop(h) } } allLayerEdges[level] = allLayerEdges[level][:ph.efConstruction] diff --git a/tok/partitioned_hnsw/partitioned_hnsw.go b/tok/partitioned_hnsw/partitioned_hnsw.go index 71d876a9a27..fa430dedfd3 100644 --- a/tok/partitioned_hnsw/partitioned_hnsw.go +++ b/tok/partitioned_hnsw/partitioned_hnsw.go @@ -81,7 +81,7 @@ func (ph *partitionedHNSW[T]) BuildInsert(ctx context.Context, uuid uint64, vec return ph.clusterMap[index].BuildInsert(ctx, uuid, vec) } -const NUM_PASSES = 10 +const NUM_PASSES = 5 func (ph *partitionedHNSW[T]) NumBuildPasses() int { return ph.partition.NumPasses()