diff --git a/go/store/prolly/proximity_map.go b/go/store/prolly/proximity_map.go new file mode 100644 index 0000000000..dc9545e783 --- /dev/null +++ b/go/store/prolly/proximity_map.go @@ -0,0 +1,113 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package prolly + +import ( + "context" + "github.com/dolthub/dolt/go/store/hash" + "github.com/dolthub/dolt/go/store/prolly/message" + "github.com/dolthub/dolt/go/store/prolly/tree" + "github.com/dolthub/dolt/go/store/val" + "github.com/dolthub/go-mysql-server/sql" + "github.com/dolthub/go-mysql-server/sql/expression" +) + +// ProximityMap wraps a tree.ProximityMap but operates on typed Tuples instead of raw bytestrings. +type ProximityMap struct { + tuples tree.ProximityMap[val.Tuple, val.Tuple, val.TupleDesc] + keyDesc val.TupleDesc + valDesc val.TupleDesc + ctx context.Context +} + +// NewProximityMap creates an empty prolly Tree Map +func NewProximityMap(ctx context.Context, node tree.Node, ns tree.NodeStore, keyDesc val.TupleDesc, valDesc val.TupleDesc) ProximityMap { + tuples := tree.ProximityMap[val.Tuple, val.Tuple, val.TupleDesc]{ + Root: node, + NodeStore: ns, + Order: keyDesc, + DistanceType: expression.DistanceL2Squared{}, + Convert: func(bytes []byte) []float64 { + h, _ := keyDesc.GetJSONAddr(0, bytes) + doc := tree.NewJSONDoc(h, ns) + jsonWrapper, err := doc.ToIndexedJSONDocument(ctx) + if err != nil { + panic(err) + } + floats, err := sql.ConvertToVector(jsonWrapper) + if err != nil { + panic(err) + } + return floats + }, + } + return ProximityMap{ + tuples: tuples, + keyDesc: keyDesc, + valDesc: valDesc, + } +} + +type VectorIter interface { + Next(ctx context.Context) (k interface{}, v val.Tuple) +} + +func NewProximityMapFromTupleIter(ctx context.Context, ns tree.NodeStore, distanceType expression.DistanceType, keyDesc val.TupleDesc, valDesc val.TupleDesc, keys []val.Tuple, values []val.Tuple, logChunkSize uint8) (ProximityMap, error) { + serializer := message.NewVectorIndexSerializer(ns.Pool()) + ch, err := tree.NewChunkerWithDeterministicSplitter(ctx, nil, 0, ns, serializer, logChunkSize) + + if err != nil { + return ProximityMap{}, err + } + + for i := 0; i < len(keys); i++ { + if err = ch.AddPair(ctx, tree.Item(keys[i]), tree.Item(values[i])); err != nil { + return ProximityMap{}, err + } + } + + root, err := ch.Done(ctx) + if err != nil { + return ProximityMap{}, err + } + + // We now have a map where each node is at the right level, but now we need to sort it. + + getHash := func(tuple []byte) hash.Hash { + h, _ := keyDesc.GetJSONAddr(0, tuple) + return h + } + newRoot, err := tree.FixupProximityMap[val.Tuple, val.TupleDesc](ctx, ns, distanceType, root, getHash, keyDesc) + if err != nil { + return ProximityMap{}, err + } + + return NewProximityMap(ctx, newRoot, ns, keyDesc, valDesc), nil +} + +// Count returns the number of key-value pairs in the Map. +func (m ProximityMap) Count() (int, error) { + return m.tuples.Count() +} + +// Get searches for the key-value pair keyed by |key| and passes the results to the callback. +// If |key| is not present in the map, a nil key-value pair are passed. +func (m ProximityMap) Get(ctx context.Context, query interface{}, cb tree.KeyValueFn[val.Tuple, val.Tuple]) (err error) { + return m.tuples.GetExact(ctx, query, cb) +} + +func (m ProximityMap) GetClosest(ctx context.Context, query interface{}, cb tree.KeyValueDistanceFn[val.Tuple, val.Tuple], limit int) (err error) { + return m.tuples.GetClosest(ctx, query, cb, limit) +} diff --git a/go/store/prolly/proximity_map_test.go b/go/store/prolly/proximity_map_test.go new file mode 100644 index 0000000000..bc34384302 --- /dev/null +++ b/go/store/prolly/proximity_map_test.go @@ -0,0 +1,192 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package prolly + +import ( + "context" + "github.com/dolthub/dolt/go/store/hash" + "github.com/dolthub/dolt/go/store/pool" + "github.com/dolthub/dolt/go/store/prolly/tree" + "github.com/dolthub/dolt/go/store/val" + "github.com/dolthub/go-mysql-server/sql" + "github.com/dolthub/go-mysql-server/sql/expression" + "github.com/dolthub/go-mysql-server/sql/types" + "github.com/stretchr/testify/require" + "testing" +) + +func newJsonValue(t *testing.T, v interface{}) sql.JSONWrapper { + doc, _, err := types.JSON.Convert(v) + require.NoError(t, err) + return doc.(sql.JSONWrapper) +} + +// newJsonDocument creates a JSON value from a provided value. +func newJsonDocument(t *testing.T, ctx context.Context, ns tree.NodeStore, v interface{}) hash.Hash { + doc := newJsonValue(t, v) + root, err := tree.SerializeJsonToAddr(ctx, ns, doc) + require.NoError(t, err) + return root.HashOf() +} + +func createProximityMap(t *testing.T, ctx context.Context, ns tree.NodeStore, vectors []interface{}, pks []int64, logChunkSize uint8) (ProximityMap, []val.Tuple, []val.Tuple) { + bp := pool.NewBuffPool() + + count := len(vectors) + require.Equal(t, count, len(pks)) + + kd := val.NewTupleDescriptor( + val.Type{Enc: val.JSONAddrEnc, Nullable: true}, + ) + + vd := val.NewTupleDescriptor( + val.Type{Enc: val.Int64Enc, Nullable: true}, + ) + + distanceType := expression.DistanceL2Squared{} + + keys := make([]val.Tuple, count) + keyBuilder := val.NewTupleBuilder(kd) + for i, vector := range vectors { + keyBuilder.PutJSONAddr(0, newJsonDocument(t, ctx, ns, vector)) + keys[i] = keyBuilder.Build(bp) + } + + valueBuilder := val.NewTupleBuilder(vd) + values := make([]val.Tuple, count) + for i, pk := range pks { + valueBuilder.PutInt64(0, pk) + values[i] = valueBuilder.Build(bp) + } + + m, err := NewProximityMapFromTupleIter(ctx, ns, distanceType, kd, vd, keys, values, logChunkSize) + require.NoError(t, err) + mapCount, err := m.Count() + require.NoError(t, err) + require.Equal(t, count, mapCount) + + return m, keys, values +} + +func TestEmptyProximityMap(t *testing.T) { + ctx := context.Background() + ns := tree.NewTestNodeStore() + createProximityMap(t, ctx, ns, nil, nil, 10) +} + +func TestSingleEntryProximityMap(t *testing.T) { + ctx := context.Background() + ns := tree.NewTestNodeStore() + m, keys, values := createProximityMap(t, ctx, ns, []interface{}{"[1.0]"}, []int64{1}, 10) + matches := 0 + vectorHash, _ := m.keyDesc.GetJSONAddr(0, keys[0]) + vectorDoc, err := tree.NewJSONDoc(vectorHash, ns).ToIndexedJSONDocument(ctx) + require.NoError(t, err) + err = m.Get(ctx, vectorDoc, func(foundKey val.Tuple, foundValue val.Tuple) error { + require.Equal(t, keys[0], foundKey) + require.Equal(t, values[0], foundValue) + matches++ + return nil + }) + require.NoError(t, err) + require.Equal(t, matches, 1) +} + +func TestDoubleEntryProximityMapGetExact(t *testing.T) { + ctx := context.Background() + ns := tree.NewTestNodeStore() + m, keys, values := createProximityMap(t, ctx, ns, []interface{}{"[0.0, 6.0]", "[3.0, 4.0]"}, []int64{1, 2}, 10) + matches := 0 + for i, key := range keys { + vectorHash, _ := m.keyDesc.GetJSONAddr(0, key) + vectorDoc, err := tree.NewJSONDoc(vectorHash, ns).ToIndexedJSONDocument(ctx) + err = m.Get(ctx, vectorDoc, func(foundKey val.Tuple, foundValue val.Tuple) error { + require.Equal(t, key, foundKey) + require.Equal(t, values[i], foundValue) + matches++ + return nil + }) + require.NoError(t, err) + } + require.Equal(t, matches, len(keys)) +} + +func TestDoubleEntryProximityMapGetClosest(t *testing.T) { + ctx := context.Background() + ns := tree.NewTestNodeStore() + m, keys, values := createProximityMap(t, ctx, ns, []interface{}{"[0.0, 6.0]", "[3.0, 4.0]"}, []int64{1, 2}, 10) + matches := 0 + + cb := func(foundKey val.Tuple, foundValue val.Tuple, distance float64) error { + require.Equal(t, keys[1], foundKey) + require.Equal(t, values[1], foundValue) + require.InDelta(t, distance, 25.0, 0.1) + matches++ + return nil + } + + err := m.GetClosest(ctx, newJsonValue(t, "[0.0, 0.0]"), cb, 1) + require.NoError(t, err) + require.Equal(t, matches, 1) +} + +func TestMultilevelProximityMap(t *testing.T) { + ctx := context.Background() + ns := tree.NewTestNodeStore() + keyStrings := []interface{}{ + "[0.0, 1.0]", + "[3.0, 4.0]", + "[5.0, 6.0]", + "[7.0, 8.0]", + } + valueStrings := []int64{1, 2, 3, 4} + m, keys, values := createProximityMap(t, ctx, ns, keyStrings, valueStrings, 1) + matches := 0 + for i, key := range keys { + vectorHash, _ := m.keyDesc.GetJSONAddr(0, key) + vectorDoc, err := tree.NewJSONDoc(vectorHash, ns).ToIndexedJSONDocument(ctx) + require.NoError(t, err) + err = m.Get(ctx, vectorDoc, func(foundKey val.Tuple, foundValue val.Tuple) error { + require.Equal(t, key, foundKey) + require.Equal(t, values[i], foundValue) + matches++ + return nil + }) + require.NoError(t, err) + } + require.Equal(t, matches, len(keys)) +} + +func TestInsertOrderIndependence(t *testing.T) { + ctx := context.Background() + ns := tree.NewTestNodeStore() + keyStrings1 := []interface{}{ + "[0.0, 1.0]", + "[3.0, 4.0]", + "[5.0, 6.0]", + "[7.0, 8.0]", + } + valueStrings1 := []int64{1, 2, 3, 4} + keyStrings2 := []interface{}{ + "[7.0, 8.0]", + "[5.0, 6.0]", + "[3.0, 4.0]", + "[0.0, 1.0]", + } + valueStrings2 := []int64{4, 3, 2, 1} + m1, _, _ := createProximityMap(t, ctx, ns, keyStrings1, valueStrings1, 1) + m2, _, _ := createProximityMap(t, ctx, ns, keyStrings2, valueStrings2, 1) + require.Equal(t, m1.tuples.Root.HashOf(), m2.tuples.Root.HashOf()) +} diff --git a/go/store/prolly/tree/proximity_map.go b/go/store/prolly/tree/proximity_map.go new file mode 100644 index 0000000000..d0f5c7ac9a --- /dev/null +++ b/go/store/prolly/tree/proximity_map.go @@ -0,0 +1,401 @@ +// Copyright 2024 Dolthub, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package tree + +import ( + "bytes" + "context" + "fmt" + "github.com/dolthub/dolt/go/store/hash" + "github.com/dolthub/dolt/go/store/prolly/message" + "github.com/dolthub/go-mysql-server/sql" + "github.com/dolthub/go-mysql-server/sql/expression" + "math" + "sort" +) + +type KeyValueDistanceFn[K, V ~[]byte] func(key K, value V, distance float64) error + +// ProximityMap is a static Prolly Tree where the position of a key in the tree is based on proximity, as opposed to a traditional ordering. +// O provides the ordering only within a node. +type ProximityMap[K, V ~[]byte, O Ordering[K]] struct { + Root Node + NodeStore NodeStore + DistanceType expression.DistanceType + Convert func([]byte) []float64 + Order O +} + +func (t ProximityMap[K, V, O]) Count() (int, error) { + return t.Root.TreeCount() +} + +func (t ProximityMap[K, V, O]) Height() int { + return t.Root.Level() + 1 +} + +func (t ProximityMap[K, V, O]) HashOf() hash.Hash { + return t.Root.HashOf() +} + +func (t ProximityMap[K, V, O]) WalkAddresses(ctx context.Context, cb AddressCb) error { + return WalkAddresses(ctx, t.Root, t.NodeStore, cb) +} + +func (t ProximityMap[K, V, O]) WalkNodes(ctx context.Context, cb NodeCb) error { + return WalkNodes(ctx, t.Root, t.NodeStore, cb) +} + +// GetExact searches for an exact vector in the index, calling |cb| with the matching key-value pairs. +func (t ProximityMap[K, V, O]) GetExact(ctx context.Context, query interface{}, cb KeyValueFn[K, V]) (err error) { + nd := t.Root + + queryVector, err := sql.ConvertToVector(query) + if err != nil { + return err + } + + // Find the child with the minimum distance. + + for { + var closestKey K + var closestIdx int + distance := math.Inf(1) + + for i := 0; i < int(nd.count); i++ { + k := nd.GetKey(i) + newDistance, err := t.DistanceType.Eval(t.Convert(k), queryVector) + if err != nil { + return err + } + if newDistance < distance { + closestIdx = i + distance = newDistance + closestKey = []byte(k) + } + } + + if nd.IsLeaf() { + return cb(closestKey, []byte(nd.GetValue(closestIdx))) + } + + nd, err = fetchChild(ctx, t.NodeStore, nd.getAddress(closestIdx)) + if err != nil { + return err + } + } +} + +func (t ProximityMap[K, V, O]) Has(ctx context.Context, query K) (ok bool, err error) { + err = t.GetExact(ctx, query, func(_ K, _ V) error { + ok = true + return nil + }) + return ok, err +} + +// GetClosest performs an approximate nearest neighbors search. It finds |limit| vectors that are close to the query vector, +// and calls |cb| with the matching key-value pairs. +func (t ProximityMap[K, V, O]) GetClosest(ctx context.Context, query interface{}, cb KeyValueDistanceFn[K, V], limit int) (err error) { + if limit != 1 { + return fmt.Errorf("currently only limit = 1 (find single closest vector) is supported for ProximityMap") + } + + queryVector, err := sql.ConvertToVector(query) + if err != nil { + return err + } + + nd := t.Root + + var closestKey K + var closestIdx int + distance := math.Inf(1) + + for { + for i := 0; i < int(nd.count); i++ { + k := nd.GetKey(i) + newDistance, err := t.DistanceType.Eval(t.Convert(k), queryVector) + if err != nil { + return err + } + if newDistance < distance { + closestIdx = i + distance = newDistance + closestKey = []byte(k) + } + } + + if nd.IsLeaf() { + return cb(closestKey, []byte(nd.GetValue(closestIdx)), distance) + } + + nd, err = fetchChild(ctx, t.NodeStore, nd.getAddress(closestIdx)) + if err != nil { + return err + } + } +} + +func (t ProximityMap[K, V, O]) IterAll(ctx context.Context) (*OrderedTreeIter[K, V], error) { + c, err := newCursorAtStart(ctx, t.NodeStore, t.Root) + if err != nil { + return nil, err + } + + s, err := newCursorPastEnd(ctx, t.NodeStore, t.Root) + if err != nil { + return nil, err + } + + stop := func(curr *cursor) bool { + return curr.compare(s) >= 0 + } + + if stop(c) { + // empty range + return &OrderedTreeIter[K, V]{curr: nil}, nil + } + + return &OrderedTreeIter[K, V]{curr: c, stop: stop, step: c.advance}, nil +} + +func getJsonValueFromHash(ctx context.Context, ns NodeStore, h hash.Hash) (interface{}, error) { + return NewJSONDoc(h, ns).ToIndexedJSONDocument(ctx) +} + +func getVectorFromHash(ctx context.Context, ns NodeStore, h hash.Hash) ([]float64, error) { + otherValue, err := getJsonValueFromHash(ctx, ns, h) + if err != nil { + return nil, err + } + return sql.ConvertToVector(otherValue) +} + +// Building/inserting into a ProximityMap requires a Fixup step, which reorganizes part (in the worst case, all) of the +// tree such that each node is a child of the closest node in the previous row. +// Currently, this is a brute force approach that visits the entire affected region of the tree in level-order, builds +// a new tree structure in memory, and then serializes the new tree to disk. There is room to improvement here. + +// An in-memory representation of a Vector Index node. +// It stores a list of (vectorHash, key, value OR address) tuples. +// The first element is always the same vector used as the parent key. +// The remaining elements are sorted prior to serialization. +type memoryNode[K ~[]byte, O Ordering[K]] struct { + vectorHashes []hash.Hash + keys [][]byte + addresses []memoryNode[K, O] + values [][]byte + order O +} + +var _ sort.Interface = (*memoryNode[[]byte, Ordering[[]byte]])(nil) + +func (m *memoryNode[K, O]) Len() int { + keys := m.keys[1:] + return len(keys) +} + +func (m *memoryNode[K, O]) Less(i, j int) bool { + keys := m.keys[1:] + return m.order.Compare(keys[i], keys[j]) < 0 +} + +func (m *memoryNode[K, O]) Swap(i, j int) { + vectorHashes := m.vectorHashes[1:] + vectorHashes[i], vectorHashes[j] = vectorHashes[j], vectorHashes[i] + keys := m.keys[1:] + keys[i], keys[j] = keys[j], keys[i] + if m.addresses != nil { + addresses := m.addresses[1:] + addresses[i], addresses[j] = addresses[j], addresses[i] + } + if m.values != nil { + values := m.values[1:] + values[i], values[j] = values[j], values[i] + } +} + +var _ sort.Interface = (*memoryNode[[]byte, Ordering[[]byte]])(nil) + +func serializeAndWriteNode(ctx context.Context, ns NodeStore, s message.Serializer, level int, subtrees []uint64, keys [][]byte, values [][]byte) (node Node, err error) { + msg := s.Serialize(keys, values, subtrees, level) + node, err = NodeFromBytes(msg) + if err != nil { + return Node{}, err + } + _, err = ns.Write(ctx, node) + return node, err +} + +func (m *memoryNode[K, O]) serialize(ctx context.Context, ns NodeStore, s message.Serializer, level int) (node Node, err error) { + sort.Sort(m) + if level == 0 { + return serializeAndWriteNode(ctx, ns, s, 0, nil, m.keys, m.values) + } + values := make([][]byte, 0, len(m.addresses)) + subTrees := make([]uint64, 0, len(m.addresses)) + for _, address := range m.addresses { + child, err := address.serialize(ctx, ns, s, level-1) + if err != nil { + return Node{}, err + } + childHash := child.HashOf() + values = append(values, childHash[:]) + childCount, err := message.GetTreeCount(child.msg) + if err != nil { + return Node{}, err + } + subTrees = append(subTrees, uint64(childCount)) + } + return serializeAndWriteNode(ctx, ns, s, level, subTrees, m.keys, values) +} + +func (m *memoryNode[K, O]) insert(ctx context.Context, ns NodeStore, distanceType expression.DistanceType, vectorHash hash.Hash, key Item, value Item, vector []float64, level int, isLeaf bool, order O) error { + if level == 0 { + if isLeaf { + if bytes.Equal(m.keys[0], key) { + m.values[0] = value + } else { + m.vectorHashes = append(m.vectorHashes, vectorHash) + m.keys = append(m.keys, key) + m.values = append(m.values, value) + } + return nil + } + // We're inserting into the row that's currently the bottom of the in-memory representation, + // but this isn't the leaf row of the final tree: more rows will be added afterward. + if bytes.Equal(m.keys[0], key) { + m.addresses[0] = memoryNode[K, O]{ + order: order, + vectorHashes: []hash.Hash{vectorHash}, + keys: [][]byte{key}, + addresses: []memoryNode[K, O]{{}}, + values: [][]byte{nil}, + } + } else { + m.vectorHashes = append(m.vectorHashes, vectorHash) + m.keys = append(m.keys, key) + m.addresses = append(m.addresses, memoryNode[K, O]{ + order: order, + vectorHashes: []hash.Hash{vectorHash}, + keys: [][]byte{key}, + addresses: []memoryNode[K, O]{{}}, + values: [][]byte{nil}, + }) + } + return nil + } + closestIdx := 0 + otherVector, err := getVectorFromHash(ctx, ns, m.vectorHashes[0]) + if err != nil { + return err + } + distance, err := distanceType.Eval(vector, otherVector) + if err != nil { + return err + } + for i := 1; i < len(m.keys); i++ { + candidateVector, err := getVectorFromHash(ctx, ns, m.vectorHashes[i]) + if err != nil { + return err + } + candidateDistance, err := distanceType.Eval(vector, candidateVector) + if err != nil { + return err + } + if candidateDistance < distance { + distance = candidateDistance + closestIdx = i + } + } + return m.addresses[closestIdx].insert(ctx, ns, distanceType, vectorHash, key, value, vector, level-1, isLeaf, order) +} + +func levelTraversal(ctx context.Context, nd Node, ns NodeStore, level int, cb func(nd Node) error) error { + if level == 0 { + return cb(nd) + } + for i := 0; i < int(nd.count); i++ { + child, err := ns.Read(ctx, nd.getAddress(i)) + if err != nil { + return err + } + err = levelTraversal(ctx, child, ns, level-1, cb) + if err != nil { + return err + } + } + return nil +} + +// FixupProximityMap takes the root not of a vector index which may not be in the correct order, and moves and reorders +// nodes to make it correct. It ensures the following invariants: +// - In any node except the root node, the first key is the same as the key in the edge pointing to that node. +// (This is the node's "defining key") +// - All other keys within a node are sorted. +// - Each non-root node contains only the keys (including transitively) that are closer to that node's defining key than +// any other key in that node's parent. +func FixupProximityMap[K ~[]byte, O Ordering[K]](ctx context.Context, ns NodeStore, distanceType expression.DistanceType, n Node, getHash func([]byte) hash.Hash, order O) (Node, error) { + if n.Level() == 0 { + return n, nil + } + // Iterate over the keys, starting at the level 1 nodes (with root as level 0) + result := memoryNode[K, O]{ + vectorHashes: make([]hash.Hash, n.Count()), + keys: make([][]byte, n.Count()), + addresses: make([]memoryNode[K, O], n.Count()), + order: order, + } + for i := 0; i < n.Count(); i++ { + keyItem := n.GetKey(i) + result.keys[i] = keyItem + vectorHash := getHash(keyItem) + result.vectorHashes[i] = vectorHash + result.addresses[i] = memoryNode[K, O]{ + order: order, + vectorHashes: []hash.Hash{vectorHash}, + keys: [][]byte{keyItem}, + addresses: []memoryNode[K, O]{{}}, + values: [][]byte{nil}, + } + } + + for level := 1; level <= n.Level(); level++ { + // Insert each key into the appropriate place in the result. + err := levelTraversal(ctx, n, ns, level, func(nd Node) error { + for i := 0; i < nd.Count(); i++ { + key := nd.GetKey(i) + vecHash := getHash(key) + vector, err := getVectorFromHash(ctx, ns, vecHash) + if err != nil { + return err + } + isLeaf := level == n.Level() + err = result.insert(ctx, ns, distanceType, vecHash, key, nd.GetValue(i), vector, level, isLeaf, order) + if err != nil { + return err + } + } + return nil + }) + if err != nil { + return Node{}, err + } + } + // Convert the in-memory representation back into a Node. + serializer := message.NewVectorIndexSerializer(ns.Pool()) + return result.serialize(ctx, ns, serializer, n.Level()) +}