From 99b59a0e2688a18bf7d352c9aec1bfec5ee8aa33 Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Thu, 10 Oct 2024 17:06:53 +0530 Subject: [PATCH 01/28] first draft --- build.go | 9 +- faiss_vector_posting.go | 4 +- new.go | 2 + section.go | 1 + section_synonym.go | 746 ++++++++++++++++++++++++++++++++++++++++ segment.go | 111 +++++- synonym_cache.go | 122 +++++++ synonym_posting.go | 190 ++++++++++ synonym_test.go | 15 + thesaurus.go | 114 ++++++ thesaurus_test.go | 15 + 11 files changed, 1321 insertions(+), 8 deletions(-) create mode 100644 section_synonym.go create mode 100644 synonym_cache.go create mode 100644 synonym_posting.go create mode 100644 synonym_test.go create mode 100644 thesaurus.go create mode 100644 thesaurus_test.go diff --git a/build.go b/build.go index 53fd34d1..acbacd0a 100644 --- a/build.go +++ b/build.go @@ -170,11 +170,14 @@ func InitSegmentBase(mem []byte, memCRC uint32, chunkMode uint32, numDocs uint64 fieldDvReaders: make([]map[uint16]*docValueReader, len(segmentSections)), docValueOffset: 0, // docValueOffsets identified automatically by the section fieldFSTs: make(map[uint16]*vellum.FST), + thesaurusFSTs: make(map[uint16]*vellum.FST), vecIndexCache: newVectorIndexCache(), + synIndexCache: newSynonymIndexCache(), // following fields gets populated by loadFieldsNew - fieldsMap: make(map[string]uint16), - dictLocs: make([]uint64, 0), - fieldsInv: make([]string, 0), + fieldsMap: make(map[string]uint16), + thesaurusMap: make(map[string]uint16), + dictLocs: make([]uint64, 0), + fieldsInv: make([]string, 0), } sb.updateSize() diff --git a/faiss_vector_posting.go b/faiss_vector_posting.go index 9c53c074..46dafbbe 100644 --- a/faiss_vector_posting.go +++ b/faiss_vector_posting.go @@ -87,7 +87,9 @@ var emptyVecPostingsIterator = &VecPostingsIterator{} var emptyVecPostingsList = &VecPostingsList{} func (vpl *VecPostingsList) Iterator(prealloc segment.VecPostingsIterator) segment.VecPostingsIterator { - + if vpl.postings == nil { + return emptyVecPostingsIterator + } // tbd: do we check the cardinality of postings and scores? var preallocPI *VecPostingsIterator pi, ok := prealloc.(*VecPostingsIterator) diff --git a/new.go b/new.go index f0d37c43..806a261b 100644 --- a/new.go +++ b/new.go @@ -102,6 +102,8 @@ type interim struct { // name -> field id + 1 FieldsMap map[string]uint16 + ThesaurusMap map[string]uint16 + // FieldsInv is the inverse of FieldsMap // field id -> name FieldsInv []string diff --git a/section.go b/section.go index 1ace25e3..13d8f57b 100644 --- a/section.go +++ b/section.go @@ -58,6 +58,7 @@ type resetable interface { const ( SectionInvertedTextIndex = iota SectionFaissVectorIndex + SectionSynonymIndex ) // ----------------------------------------------------------------------------- diff --git a/section_synonym.go b/section_synonym.go new file mode 100644 index 00000000..405a8119 --- /dev/null +++ b/section_synonym.go @@ -0,0 +1,746 @@ +// Copyright (c) 2024 Couchbase, 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 zap + +import ( + "bytes" + "encoding/binary" + "fmt" + "io" + "math" + "sort" + + "github.com/RoaringBitmap/roaring" + "github.com/RoaringBitmap/roaring/roaring64" + index "github.com/blevesearch/bleve_index_api" + seg "github.com/blevesearch/scorch_segment_api/v2" + "github.com/blevesearch/vellum" +) + +func init() { + registerSegmentSection(SectionSynonymIndex, &synonymIndexSection{}) +} + +// ----------------------------------------------------------------------------- + +type synonymIndexOpaque struct { + results []index.Document + + // indicates whether the following structs are initialized + init bool + + // ThesaurusMap adds 1 to thesaurus id to avoid zero value issues + // name -> thesaurus id + 1 + ThesaurusMap map[string]uint16 + + // ThesaurusMapInv is the inverse of ThesaurusMap + // thesaurus id + 1 -> name + ThesaurusInv []string + + // Thesaurus for each thesaurus ID + // thesaurus id -> LHS term -> synonym postings list id + 1 + Thesauri []map[string]uint64 + + // LHS Terms for each thesaurus ID, where terms are sorted ascending + // thesaurus id -> []term + ThesaurusKeys [][]string + + // FieldIDtoThesaurusID maps the field id to the thesaurus id + // field id -> thesaurus id + FieldIDtoThesaurusID map[uint16]int + + // SynonymIDtoTerm maps synonym id to term for each thesaurus + // thesaurus id -> synonym id -> term + SynonymTermToID []map[string]uint32 + + // SynonymTermToID maps term to synonym id for each thesaurus + // thesaurus id -> term -> synonym id + // this is the inverse of SynonymIDtoTerm for each thesaurus + SynonymIDtoTerm []map[uint32]string + + // synonym postings list -> synonym bitmap + Synonyms []*roaring64.Bitmap + + builder *vellum.Builder + builderBuf bytes.Buffer + + tmp0 []byte + + thesaurusAddrs map[int]int +} + +func (so *synonymIndexOpaque) Set(key string, value interface{}) { + switch key { + case "results": + so.results = value.([]index.Document) + } +} + +func (so *synonymIndexOpaque) Reset() (err error) { + // cleanup stuff over here + so.results = nil + so.init = false + so.ThesaurusMap = nil + so.ThesaurusInv = nil + for i := range so.Thesauri { + so.Thesauri[i] = nil + } + so.Thesauri = so.Thesauri[:0] + for i := range so.ThesaurusKeys { + so.ThesaurusKeys[i] = so.ThesaurusKeys[i][:0] + } + so.ThesaurusKeys = so.ThesaurusKeys[:0] + for _, idn := range so.Synonyms { + idn.Clear() + } + so.Synonyms = so.Synonyms[:0] + so.builderBuf.Reset() + if so.builder != nil { + err = so.builder.Reset(&so.builderBuf) + } + + so.tmp0 = so.tmp0[:0] + return err +} + +func (so *synonymIndexOpaque) process(field index.SynonymField, fieldID uint16, docNum uint32) { + if !so.init && so.results != nil { + so.realloc(fieldID) + so.init = true + } + + tid := so.FieldIDtoThesaurusID[fieldID] + + thesaurus := so.Thesauri[tid] + + termSynMap := so.SynonymTermToID[tid] + + synDefs := field.SynonymDefinitions() + + for _, def := range synDefs { + pid := thesaurus[def.Term()] - 1 + + bs := so.Synonyms[pid] + + for _, syn := range def.Synonyms() { + code := encodeSynonym(termSynMap[syn], docNum) + bs.Add(code) + } + } +} + +func (so *synonymIndexOpaque) realloc(fieldID uint16) { + var pidNext int + var sidNext uint32 + so.ThesaurusMap = map[string]uint16{} + + for _, result := range so.results { + if synDoc, ok := result.(index.SynonymDocument); ok { + synDoc.VisitSynonymField(func(synField index.SynonymField) { + so.getOrDefineThesaurus(fieldID, synField.Name()) + }) + } + } + + visitSynonymField := func(synField index.SynonymField) { + thesaurusID := uint16(so.getOrDefineThesaurus(fieldID, synField.Name())) + + thesaurus := so.Thesauri[thesaurusID] + thesaurusKeys := so.ThesaurusKeys[thesaurusID] + + synTermMap := so.SynonymIDtoTerm[thesaurusID] + + termSynMap := so.SynonymTermToID[thesaurusID] + + synDefs := synField.SynonymDefinitions() + for _, synDef := range synDefs { + term := synDef.Term() + _, exists := thesaurus[term] + if !exists { + pidNext++ + pidPlus1 := uint64(pidNext) + + thesaurus[term] = pidPlus1 + thesaurusKeys = append(thesaurusKeys, term) + } + syns := synDef.Synonyms() + for _, syn := range syns { + _, exists := termSynMap[syn] + if !exists { + sidNext++ + sidPlus1 := sidNext + termSynMap[syn] = sidPlus1 + synTermMap[sidPlus1] = syn + } + } + } + so.ThesaurusKeys[thesaurusID] = thesaurusKeys + } + + for _, result := range so.results { + if synDoc, ok := result.(index.SynonymDocument); ok { + synDoc.VisitSynonymField(visitSynonymField) + } + } + + numSynonymsLists := pidNext + + if cap(so.Synonyms) >= numSynonymsLists { + so.Synonyms = so.Synonyms[:numSynonymsLists] + } else { + synonyms := make([]*roaring64.Bitmap, numSynonymsLists) + copy(synonyms, so.Synonyms[:cap(so.Synonyms)]) + for i := 0; i < numSynonymsLists; i++ { + if synonyms[i] == nil { + synonyms[i] = roaring64.New() + } + } + so.Synonyms = synonyms + } + + for _, thes := range so.ThesaurusKeys { + sort.Strings(thes) + } +} + +func (so *synonymIndexOpaque) getOrDefineThesaurus(fieldID uint16, thesaurusName string) int { + thesaurusIDPlus1, exists := so.ThesaurusMap[thesaurusName] + if !exists { + thesaurusIDPlus1 = uint16(len(so.ThesaurusInv) + 1) + so.ThesaurusMap[thesaurusName] = thesaurusIDPlus1 + so.ThesaurusInv = append(so.ThesaurusInv, thesaurusName) + + so.Thesauri = append(so.Thesauri, make(map[string]uint64)) + + so.SynonymIDtoTerm = append(so.SynonymIDtoTerm, make(map[uint32]string)) + + so.SynonymTermToID = append(so.SynonymTermToID, make(map[string]uint32)) + + so.FieldIDtoThesaurusID[fieldID] = int(thesaurusIDPlus1 - 1) + + n := len(so.ThesaurusKeys) + if n < cap(so.ThesaurusKeys) { + so.ThesaurusKeys = so.ThesaurusKeys[:n+1] + so.ThesaurusKeys[n] = so.ThesaurusKeys[n][:0] + } else { + so.ThesaurusKeys = append(so.ThesaurusKeys, []string(nil)) + } + } + + return int(thesaurusIDPlus1 - 1) +} + +func (so *synonymIndexOpaque) grabBuf(size int) []byte { + buf := so.tmp0 + if cap(buf) < size { + buf = make([]byte, size) + so.tmp0 = buf + } + return buf[:size] +} + +func (so *synonymIndexOpaque) writeThesauri(w *CountHashWriter) (thesOffsets []uint64, err error) { + + if so.results == nil || len(so.results) == 0 { + return nil, nil + } + + thesOffsets = make([]uint64, len(so.ThesaurusInv)) + + buf := so.grabBuf(binary.MaxVarintLen64) + + if so.builder == nil { + so.builder, err = vellum.New(&so.builderBuf, nil) + if err != nil { + return nil, err + } + } + + for thesaurusID, terms := range so.ThesaurusKeys { + thes := so.Thesauri[thesaurusID] + for _, term := range terms { // terms are already sorted + pid := thes[term] - 1 + postingsBS := so.Synonyms[pid] + postingsOffset, err := writeSynonyms(postingsBS, w, buf) + if err != nil { + return nil, err + } + + if postingsOffset > uint64(0) { + err = so.builder.Insert([]byte(term), postingsOffset) + if err != nil { + return nil, err + } + } + } + + err = so.builder.Close() + if err != nil { + return nil, err + } + + // record where this dictionary starts + thesOffsets[thesaurusID] = uint64(w.Count()) + + vellumData := so.builderBuf.Bytes() + + // write out the length of the vellum data + n := binary.PutUvarint(buf, uint64(len(vellumData))) + _, err = w.Write(buf[:n]) + if err != nil { + return nil, err + } + + // write this vellum to disk + _, err = w.Write(vellumData) + if err != nil { + return nil, err + } + + // reset vellum for reuse + so.builderBuf.Reset() + + err = so.builder.Reset(&so.builderBuf) + if err != nil { + return nil, err + } + + // write out the synTermMap for this thesaurus + _, err := writeSynTermMap(so.SynonymIDtoTerm[thesaurusID], w, buf) + if err != nil { + return nil, err + } + + thesaurusStart := w.Count() + + n = binary.PutUvarint(buf, fieldNotUninverted) + _, err = w.Write(buf[:n]) + if err != nil { + return nil, err + } + + n = binary.PutUvarint(buf, fieldNotUninverted) + _, err = w.Write(buf[:n]) + if err != nil { + return nil, err + } + + n = binary.PutUvarint(buf, thesOffsets[thesaurusID]) + _, err = w.Write(buf[:n]) + if err != nil { + return nil, err + } + so.thesaurusAddrs[thesaurusID] = thesaurusStart + } + return thesOffsets, nil +} + +// ----------------------------------------------------------------------------- + +type synonymIndexSection struct { +} + +func (s *synonymIndexSection) getSynonymIndexOpaque(opaque map[int]resetable) *synonymIndexOpaque { + if _, ok := opaque[SectionSynonymIndex]; !ok { + opaque[SectionSynonymIndex] = s.InitOpaque(nil) + } + return opaque[SectionSynonymIndex].(*synonymIndexOpaque) +} + +func (s *synonymIndexSection) InitOpaque(args map[string]interface{}) resetable { + rv := &synonymIndexOpaque{ + thesaurusAddrs: map[int]int{}, + } + for k, v := range args { + rv.Set(k, v) + } + + return rv +} + +func (s *synonymIndexSection) Process(opaque map[int]resetable, docNum uint32, field index.Field, fieldID uint16) { + if fieldID == math.MaxUint16 { + return + } + if sf, ok := field.(index.SynonymField); ok { + // at this point we have a synonym document being processed + // and this document is expected to have a single field + // which is a synonym field. + // we consider the + // fieldName as the thesaurusName and + // fieldID as the thesaurusID. + so := s.getSynonymIndexOpaque(opaque) + so.process(sf, fieldID, docNum) + } +} + +func (s *synonymIndexSection) Persist(opaque map[int]resetable, w *CountHashWriter) (n int64, err error) { + synIndexOpaque := s.getSynonymIndexOpaque(opaque) + _, err = synIndexOpaque.writeThesauri(w) + return 0, err +} + +func (s *synonymIndexSection) AddrForField(opaque map[int]resetable, fieldID int) int { + synIndexOpaque := s.getSynonymIndexOpaque(opaque) + tid, exists := synIndexOpaque.FieldIDtoThesaurusID[uint16(fieldID)] + if !exists { + return 0 + } + return synIndexOpaque.thesaurusAddrs[tid] +} + +func (s *synonymIndexSection) Merge(opaque map[int]resetable, segments []*SegmentBase, + drops []*roaring.Bitmap, fieldsInv []string, newDocNumsIn [][]uint64, + w *CountHashWriter, closeCh chan struct{}) error { + so := s.getSynonymIndexOpaque(opaque) + thesaurusAddrs, fieldIDtoThesaurusID, err := mergeAndPersistSynonymSection(segments, drops, fieldsInv, newDocNumsIn, w, closeCh) + if err != nil { + return err + } + + so.thesaurusAddrs = thesaurusAddrs + so.FieldIDtoThesaurusID = fieldIDtoThesaurusID + return nil +} + +// ----------------------------------------------------------------------------- + +func encodeSynonym(synonymID uint32, docID uint32) uint64 { + return uint64(synonymID)<<32 | uint64(docID) +} + +func writeSynonyms(postings *roaring64.Bitmap, w *CountHashWriter, bufMaxVarintLen64 []byte) ( + offset uint64, err error) { + termCardinality := postings.GetCardinality() + if termCardinality <= 0 { + return 0, nil + } + + postingsOffset := uint64(w.Count()) + + _, err = writeRoaringSynonymWithLen(postings, w, bufMaxVarintLen64) + if err != nil { + return 0, err + } + + return postingsOffset, nil +} + +func writeSynTermMap(synTermMap map[uint32]string, w *CountHashWriter, bufMaxVarintLen64 []byte) ( + offset uint64, err error) { + + if len(synTermMap) == 0 { + return 0, nil + } + + synTermMapOffset := uint64(w.Count()) + n := binary.PutUvarint(bufMaxVarintLen64, uint64(len(synTermMap))) + _, err = w.Write(bufMaxVarintLen64[:n]) + if err != nil { + return 0, err + } + + for sid, term := range synTermMap { + n = binary.PutUvarint(bufMaxVarintLen64, uint64(sid)) + _, err = w.Write(bufMaxVarintLen64[:n]) + if err != nil { + return 0, err + } + + n = binary.PutUvarint(bufMaxVarintLen64, uint64(len(term))) + _, err = w.Write(bufMaxVarintLen64[:n]) + if err != nil { + return 0, err + } + + _, err = w.Write([]byte(term)) + if err != nil { + return 0, err + } + } + + return synTermMapOffset, nil +} + +// writes out the length of the roaring bitmap in bytes as varint +// then writes out the roaring bitmap itself +func writeRoaringSynonymWithLen(r *roaring64.Bitmap, w io.Writer, + reuseBufVarint []byte) (int, error) { + buf, err := r.ToBytes() + if err != nil { + return 0, err + } + + var tw int + + // write out the length + n := binary.PutUvarint(reuseBufVarint, uint64(len(buf))) + nw, err := w.Write(reuseBufVarint[:n]) + tw += nw + if err != nil { + return tw, err + } + + // write out the roaring bytes + nw, err = w.Write(buf) + tw += nw + if err != nil { + return tw, err + } + + return tw, nil +} + +func mergeAndPersistSynonymSection(segments []*SegmentBase, dropsIn []*roaring.Bitmap, + fieldsInv []string, newDocNumsIn [][]uint64, w *CountHashWriter, + closeCh chan struct{}) (map[int]int, map[uint16]int, error) { + + var bufMaxVarintLen64 []byte = make([]byte, binary.MaxVarintLen64) + + var synonyms *SynonymsList + var synItr *SynonymsIterator + + thesaurusAddrs := make(map[int]int) + + var vellumBuf bytes.Buffer + newVellum, err := vellum.New(&vellumBuf, nil) + if err != nil { + return nil, nil, err + } + + newRoaring := roaring64.NewBitmap() + + newDocNums := make([][]uint64, 0, len(segments)) + + drops := make([]*roaring.Bitmap, 0, len(segments)) + + thesauri := make([]*Thesaurus, 0, len(segments)) + + itrs := make([]vellum.Iterator, 0, len(segments)) + + fieldIDtoThesaurusID := make(map[uint16]int) + + synTermMap := make(map[uint32]string) + termSynMap := make(map[string]uint32) + var thesaurusID int + var newSynonymID uint32 + + // for each field + for fieldID, fieldName := range fieldsInv { + // collect FST iterators from all active segments for this field + newDocNums = newDocNums[:0] + drops = drops[:0] + thesauri = thesauri[:0] + itrs = itrs[:0] + newSynonymID = 0 + for syn := range synTermMap { + delete(synTermMap, syn) + } + for syn := range termSynMap { + delete(termSynMap, syn) + } + + for segmentI, segment := range segments { + // check for the closure in meantime + if isClosed(closeCh) { + return nil, nil, seg.ErrClosed + } + + thes, err2 := segment.thesaurus(fieldName) + if err2 != nil { + return nil, nil, err2 + } + if thes != nil && thes.fst != nil { + itr, err2 := thes.fst.Iterator(nil, nil) + if err2 != nil && err2 != vellum.ErrIteratorDone { + return nil, nil, err2 + } + if itr != nil { + newDocNums = append(newDocNums, newDocNumsIn[segmentI]) + if dropsIn[segmentI] != nil && !dropsIn[segmentI].IsEmpty() { + drops = append(drops, dropsIn[segmentI]) + } else { + drops = append(drops, nil) + } + thesauri = append(thesauri, thes) + itrs = append(itrs, itr) + } + } + } + + // if no iterators, skip this field + if len(itrs) == 0 { + continue + } + + var prevTerm []byte + + newRoaring.Clear() + + finishTerm := func(term []byte) error { + postingsOffset, err := writeSynonyms(newRoaring, w, bufMaxVarintLen64) + if err != nil { + return err + } + if postingsOffset > 0 { + err = newVellum.Insert(term, postingsOffset) + if err != nil { + return err + } + } + newRoaring.Clear() + return nil + } + + enumerator, err := newEnumerator(itrs) + + for err == nil { + term, itrI, postingsOffset := enumerator.Current() + + if prevTerm != nil && !bytes.Equal(prevTerm, term) { + // check for the closure in meantime + if isClosed(closeCh) { + return nil, nil, seg.ErrClosed + } + + // if the term changed, write out the info collected + // for the previous term + err = finishTerm(prevTerm) + if err != nil { + return nil, nil, err + } + } + + synonyms, err = thesauri[itrI].synonymsListFromOffset( + postingsOffset, drops[itrI], synonyms) + if err != nil { + return nil, nil, err + } + synItr = synonyms.iterator(synItr) + + newSynonymID, err = mergeSynonyms(synItr, newDocNums[itrI], newRoaring, synTermMap, termSynMap, newSynonymID) + if err != nil { + return nil, nil, err + } + + prevTerm = prevTerm[:0] // copy to prevTerm in case Next() reuses term mem + prevTerm = append(prevTerm, term...) + err = enumerator.Next() + } + if err != vellum.ErrIteratorDone { + return nil, nil, err + } + // close the enumerator to free the underlying iterators + err = enumerator.Close() + if err != nil { + return nil, nil, err + } + + if prevTerm != nil { + err = finishTerm(prevTerm) + if err != nil { + return nil, nil, err + } + } + + err = newVellum.Close() + if err != nil { + return nil, nil, err + } + vellumData := vellumBuf.Bytes() + + thesOffset := uint64(w.Count()) + + // write out the length of the vellum data + n := binary.PutUvarint(bufMaxVarintLen64, uint64(len(vellumData))) + _, err = w.Write(bufMaxVarintLen64[:n]) + if err != nil { + return nil, nil, err + } + + // write this vellum to disk + _, err = w.Write(vellumData) + if err != nil { + return nil, nil, err + } + + // reset vellum buffer and vellum builder + vellumBuf.Reset() + err = newVellum.Reset(&vellumBuf) + if err != nil { + return nil, nil, err + } + + // write out the synTermMap for this thesaurus + _, err = writeSynTermMap(synTermMap, w, bufMaxVarintLen64) + if err != nil { + return nil, nil, err + } + + thesStart := w.Count() + + n = binary.PutUvarint(bufMaxVarintLen64, fieldNotUninverted) + _, err = w.Write(bufMaxVarintLen64[:n]) + if err != nil { + return nil, nil, err + } + + n = binary.PutUvarint(bufMaxVarintLen64, fieldNotUninverted) + _, err = w.Write(bufMaxVarintLen64[:n]) + if err != nil { + return nil, nil, err + } + + n = binary.PutUvarint(bufMaxVarintLen64, thesOffset) + _, err = w.Write(bufMaxVarintLen64[:n]) + if err != nil { + return nil, nil, err + } + + // if we have a new thesaurus, add it to the thesaurus map + fieldIDtoThesaurusID[uint16(fieldID)] = thesaurusID + thesaurusAddrs[thesaurusID] = thesStart + thesaurusID++ + } + + return thesaurusAddrs, fieldIDtoThesaurusID, nil +} + +func mergeSynonyms(synItr *SynonymsIterator, newDocNums []uint64, newRoaring *roaring64.Bitmap, + synTermMap map[uint32]string, termSynMap map[string]uint32, newSynonymID uint32) (uint32, error) { + next, err := synItr.Next() + for next != nil && err == nil { + synNewDocNum := newDocNums[next.DocNum()] + if synNewDocNum == docDropped { + return 0, fmt.Errorf("see hit with dropped docNum") + } + nextTerm := next.Term() + var synNewID uint32 + if synID, ok := termSynMap[nextTerm]; ok { + synNewID = synID + } else { + synNewID = newSynonymID + termSynMap[nextTerm] = newSynonymID + synTermMap[newSynonymID] = nextTerm + newSynonymID++ + } + synNewCode := encodeSynonym(synNewID, uint32(synNewDocNum)) + newRoaring.Add(synNewCode) + next, err = synItr.Next() + } + return newSynonymID, nil +} + +// ----------------------------------------------------------------------------- diff --git a/segment.go b/segment.go index 8780ead1..dfd0c912 100644 --- a/segment.go +++ b/segment.go @@ -54,8 +54,11 @@ func (*ZapPlugin) Open(path string) (segment.Segment, error) { rv := &Segment{ SegmentBase: SegmentBase{ fieldsMap: make(map[string]uint16), + thesaurusMap: make(map[string]uint16), fieldFSTs: make(map[uint16]*vellum.FST), + thesaurusFSTs: make(map[uint16]*vellum.FST), vecIndexCache: newVectorIndexCache(), + synIndexCache: newSynonymIndexCache(), fieldDvReaders: make([]map[uint16]*docValueReader, len(segmentSections)), }, f: f, @@ -92,6 +95,7 @@ type SegmentBase struct { memCRC uint32 chunkMode uint32 fieldsMap map[string]uint16 // fieldName -> fieldID+1 + thesaurusMap map[string]uint16 // thesaurusName -> thesaurusID+1 fieldsInv []string // fieldID -> fieldName fieldsSectionsMap []map[uint16]uint64 // fieldID -> section -> address numDocs uint64 @@ -100,6 +104,7 @@ type SegmentBase struct { sectionsIndexOffset uint64 docValueOffset uint64 dictLocs []uint64 + thesaurusLocs []uint64 fieldDvReaders []map[uint16]*docValueReader // naive chunk cache per field; section->field->reader fieldDvNames []string // field names cached in fieldDvReaders size uint64 @@ -108,11 +113,13 @@ type SegmentBase struct { bytesRead uint64 bytesWritten uint64 - m sync.Mutex - fieldFSTs map[uint16]*vellum.FST + m sync.Mutex + fieldFSTs map[uint16]*vellum.FST + thesaurusFSTs map[uint16]*vellum.FST // this cache comes into play when vectors are supported in builds. vecIndexCache *vectorIndexCache + synIndexCache *synonymIndexCache } func (sb *SegmentBase) Size() int { @@ -128,11 +135,17 @@ func (sb *SegmentBase) updateSize() { sizeInBytes += (len(k) + SizeOfString) + SizeOfUint16 } - // fieldsInv, dictLocs + // thesaurusMap + for k := range sb.thesaurusMap { + sizeInBytes += (len(k) + SizeOfString) + SizeOfUint16 + } + + // fieldsInv, dictLocs, thesaurusLocs for _, entry := range sb.fieldsInv { sizeInBytes += len(entry) + SizeOfString } sizeInBytes += len(sb.dictLocs) * SizeOfUint64 + sizeInBytes += len(sb.thesaurusLocs) * SizeOfUint64 // fieldDvReaders for _, secDvReaders := range sb.fieldDvReaders { @@ -149,7 +162,11 @@ func (sb *SegmentBase) updateSize() { func (sb *SegmentBase) AddRef() {} func (sb *SegmentBase) DecRef() (err error) { return nil } -func (sb *SegmentBase) Close() (err error) { sb.vecIndexCache.Clear(); return nil } +func (sb *SegmentBase) Close() (err error) { + sb.vecIndexCache.Clear() + sb.synIndexCache.Clear() + return nil +} // Segment implements a persisted segment.Segment interface, by // embedding an mmap()'ed SegmentBase. @@ -298,6 +315,7 @@ func (s *SegmentBase) loadFields() error { dictLoc, read := binary.Uvarint(s.mem[addr:fieldsIndexEnd]) n := uint64(read) s.dictLocs = append(s.dictLocs, dictLoc) + // do not load thesaurusLocs as they are not present in the older file formats var nameLen uint64 nameLen, read = binary.Uvarint(s.mem[addr+n : fieldsIndexEnd]) @@ -308,6 +326,7 @@ func (s *SegmentBase) loadFields() error { s.incrementBytesRead(n + nameLen) s.fieldsInv = append(s.fieldsInv, name) s.fieldsMap[name] = uint16(fieldID + 1) + // do not load thesaurusMap as they are not present in the older file formats fieldID++ } @@ -413,6 +432,26 @@ func (s *SegmentBase) loadFieldNew(fieldID uint16, pos uint64, // account the bytes read while parsing the field's inverted index section s.incrementBytesRead(uint64(read + n)) s.dictLocs = append(s.dictLocs, dictLoc) + } else if fieldSectionType == SectionSynonymIndex { + // this field is a thesaurus field and we need to read the thesaurus + // data from the file. + if fieldSectionAddr == 0 { + s.thesaurusLocs = append(s.thesaurusLocs, 0) + continue + } + read := 0 + // skip the doc values + _, n := binary.Uvarint(s.mem[fieldSectionAddr : fieldSectionAddr+binary.MaxVarintLen64]) + fieldSectionAddr += uint64(n) + read += n + _, n = binary.Uvarint(s.mem[fieldSectionAddr : fieldSectionAddr+binary.MaxVarintLen64]) + fieldSectionAddr += uint64(n) + read += n + thesaurusLoc, n := binary.Uvarint(s.mem[fieldSectionAddr : fieldSectionAddr+binary.MaxVarintLen64]) + // account the bytes read while parsing the field's synonym section + s.incrementBytesRead(uint64(read + n)) + s.thesaurusLocs = append(s.thesaurusLocs, thesaurusLoc) + s.thesaurusMap[fieldName] = uint16(fieldID + 1) } } @@ -472,6 +511,58 @@ func (sb *SegmentBase) dictionary(field string) (rv *Dictionary, err error) { return rv, nil } +// Thesaurus returns the term thesaurus for the specified field +func (s *SegmentBase) Thesaurus(name string) (segment.Thesaurus, error) { + thesaurus, err := s.thesaurus(name) + if err == nil && thesaurus == nil { + return emptyThesaurus, nil + } + return thesaurus, err +} + +func (sb *SegmentBase) thesaurus(name string) (rv *Thesaurus, err error) { + thesaurusIDPlus1 := sb.thesaurusMap[name] + if thesaurusIDPlus1 > 0 { + rv = &Thesaurus{ + sb: sb, + name: name, + thesaurusID: thesaurusIDPlus1 - 1, + } + thesaurusStart := sb.thesaurusLocs[rv.thesaurusID] + if thesaurusStart > 0 { + var ok bool + sb.m.Lock() + if rv.fst, ok = sb.thesaurusFSTs[rv.thesaurusID]; !ok { + // read the length of the vellum data + vellumLen, read := binary.Uvarint(sb.mem[thesaurusStart : thesaurusStart+binary.MaxVarintLen64]) + if vellumLen == 0 { + sb.m.Unlock() + return nil, fmt.Errorf("empty thesaurus for name: %v", name) + } + fstBytes := sb.mem[thesaurusStart+uint64(read) : thesaurusStart+uint64(read)+vellumLen] + rv.fst, err = vellum.Load(fstBytes) + if err != nil { + sb.m.Unlock() + return nil, fmt.Errorf("thesaurus name %s vellum err: %v", name, err) + } + rv.synIDTermMap = sb.synIndexCache.loadOrCreate(rv.thesaurusID, sb.mem[thesaurusStart+uint64(read)+vellumLen:]) + sb.thesaurusFSTs[rv.thesaurusID] = rv.fst + } else { + if rv.synIDTermMap, ok = sb.synIndexCache.load(rv.thesaurusID); !ok { + sb.m.Unlock() + return nil, fmt.Errorf("thesaurus name %s synIDTermMap not found", name) + } + } + sb.m.Unlock() + rv.fstReader, err = rv.fst.Reader() + if err != nil { + return nil, fmt.Errorf("thesaurus name %s vellum reader err: %v", name, err) + } + } + } + return rv, nil +} + // visitDocumentCtx holds data structures that are reusable across // multiple VisitDocument() calls to avoid memory allocations type visitDocumentCtx struct { @@ -650,6 +741,7 @@ func (s *Segment) Close() (err error) { func (s *Segment) closeActual() (err error) { // clear contents from the vector index cache before un-mmapping s.vecIndexCache.Clear() + s.synIndexCache.Clear() if s.mm != nil { err = s.mm.Unmap() @@ -719,6 +811,17 @@ func (s *Segment) DictAddr(field string) (uint64, error) { return s.dictLocs[fieldIDPlus1-1], nil } +// ThesaurusAddr is a helper function to compute the file offset where the +// thesaurus is stored for the specified thesaurus name. +func (s *Segment) ThesaurusAddr(name string) (uint64, error) { + thesaurusIDPlus1, ok := s.thesaurusMap[name] + if !ok { + return 0, fmt.Errorf("no such thesaurus '%s'", name) + } + + return s.thesaurusLocs[thesaurusIDPlus1-1], nil +} + func (s *Segment) getSectionDvOffsets(fieldID int, secID uint16) (uint64, uint64, uint64, error) { // Version is gonna be 16 var fieldLocStart uint64 = fieldNotUninverted diff --git a/synonym_cache.go b/synonym_cache.go new file mode 100644 index 00000000..635c3bac --- /dev/null +++ b/synonym_cache.go @@ -0,0 +1,122 @@ +// Copyright (c) 2024 Couchbase, 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 zap + +import ( + "encoding/binary" + "sync" +) + +func newSynonymIndexCache() *synonymIndexCache { + return &synonymIndexCache{ + cache: make(map[uint16]*synonymCacheEntry), + closeCh: make(chan struct{}), + } +} + +type synonymIndexCache struct { + closeCh chan struct{} + m sync.RWMutex + + cache map[uint16]*synonymCacheEntry +} + +func (sc *synonymIndexCache) Clear() { + sc.m.Lock() + sc.cache = nil + sc.m.Unlock() +} + +func (sc *synonymIndexCache) loadOrCreate(thesaurusID uint16, mem []byte) map[uint32][]byte { + + sc.m.RLock() + + entry, ok := sc.cache[thesaurusID] + if ok { + sc.m.RUnlock() + return entry.load() + } + + sc.m.RUnlock() + + sc.m.Lock() + defer sc.m.Unlock() + + entry, ok = sc.cache[thesaurusID] + if ok { + return entry.load() + } + + return sc.createAndCacheLOCKED(thesaurusID, mem) +} + +func (sc *synonymIndexCache) load(thesaurusID uint16) (map[uint32][]byte, bool) { + sc.m.RLock() + defer sc.m.RUnlock() + + entry, ok := sc.cache[thesaurusID] + if !ok { + return nil, false + } + + return entry.load(), true +} + +func (sc *synonymIndexCache) createAndCacheLOCKED(thesaurusID uint16, mem []byte) map[uint32][]byte { + synTermMap := make(map[uint32][]byte) + pos := 0 + numSyns, n := binary.Uvarint(mem[pos : pos+binary.MaxVarintLen64]) + pos += n + for i := 0; i < int(numSyns); i++ { + synID, n := binary.Uvarint(mem[pos : pos+binary.MaxVarintLen64]) + pos += n + + termLen, n := binary.Uvarint(mem[pos : pos+binary.MaxVarintLen64]) + pos += n + + term := mem[pos : pos+int(termLen)] + + synTermMap[uint32(synID)] = term + } + sc.insertLOCKED(thesaurusID, synTermMap) + return synTermMap +} + +func (sc *synonymIndexCache) insertLOCKED(thesaurusID uint16, synTermMap map[uint32][]byte) { + _, ok := sc.cache[thesaurusID] + if !ok { + // initializing the alpha with 0.4 essentially means that we are favoring + // the history a little bit more relative to the current sample value. + // this makes the average to be kept above the threshold value for a + // longer time and thereby the index to be resident in the cache + // for longer time. + sc.cache[thesaurusID] = createSynonymCacheEntry(synTermMap) + } +} + +func createSynonymCacheEntry(synTermMap map[uint32][]byte) *synonymCacheEntry { + ce := &synonymCacheEntry{ + synTermMap: synTermMap, + } + return ce +} + +type synonymCacheEntry struct { + synTermMap map[uint32][]byte +} + +func (ce *synonymCacheEntry) load() map[uint32][]byte { + return ce.synTermMap +} diff --git a/synonym_posting.go b/synonym_posting.go new file mode 100644 index 00000000..77ad5762 --- /dev/null +++ b/synonym_posting.go @@ -0,0 +1,190 @@ +// Copyright (c) 2024 Couchbase, 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 zap + +import ( + "bytes" + "encoding/binary" + "fmt" + + "github.com/RoaringBitmap/roaring" + "github.com/RoaringBitmap/roaring/roaring64" + segment "github.com/blevesearch/scorch_segment_api/v2" +) + +type SynonymsList struct { + sb *SegmentBase + synonymsOffset uint64 + synonyms *roaring64.Bitmap + except *roaring.Bitmap + + synIDTermMap map[uint32][]byte + + buffer *bytes.Reader +} + +// represents an immutable, empty synonyms list +var emptySynonymsList = &SynonymsList{} + +// Iterator returns an iterator for this postings list +func (s *SynonymsList) Iterator(prealloc segment.SynonymsIterator) segment.SynonymsIterator { + if s.synonyms == nil { + return emptySynonymsIterator + } + + var preallocSI *SynonymsIterator + pi, ok := prealloc.(*SynonymsIterator) + if ok && pi != nil { + preallocSI = pi + } + if preallocSI == emptySynonymsIterator { + preallocSI = nil + } + + return s.iterator(preallocSI) +} + +func (s *SynonymsList) iterator(rv *SynonymsIterator) *SynonymsIterator { + if rv == nil { + rv = &SynonymsIterator{} + } else { + *rv = SynonymsIterator{} // clear the struct + } + rv.synonyms = s + rv.except = s.except + rv.Actual = s.synonyms.Iterator() + rv.ActualBM = s.synonyms + + rv.synIDTermMap = s.synIDTermMap + + return rv +} + +func (rv *SynonymsList) read(synonymsOffset uint64, t *Thesaurus) error { + rv.synonymsOffset = synonymsOffset + + var n uint64 + var read int + + var synonymsLen uint64 + synonymsLen, read = binary.Uvarint(t.sb.mem[synonymsOffset+n : synonymsOffset+n+binary.MaxVarintLen64]) + n += uint64(read) + + roaringBytes := t.sb.mem[synonymsOffset+n : synonymsOffset+n+synonymsLen] + + if rv.synonyms == nil { + rv.synonyms = roaring64.NewBitmap() + } + + rv.buffer.Reset(roaringBytes) + + _, err := rv.synonyms.ReadFrom(rv.buffer) + if err != nil { + return fmt.Errorf("error loading roaring bitmap: %v", err) + } + + return nil +} + +// ----------------------------------------------------------------------------- +// SynonymsIterator provides a way to iterate through the synonyms list +type SynonymsIterator struct { + synonyms *SynonymsList + except *roaring.Bitmap + + Actual roaring64.IntPeekable64 + ActualBM *roaring64.Bitmap + + synIDTermMap map[uint32][]byte + nextSyn Synonym +} + +var emptySynonymsIterator = &SynonymsIterator{} + +func (i *SynonymsIterator) Next() (segment.Synonym, error) { + return i.next() +} + +func (i *SynonymsIterator) next() (segment.Synonym, error) { + synID, docNum, exists, err := i.nextSynonym() + if err != nil || !exists { + return nil, err + } + + if i.synIDTermMap == nil { + return nil, fmt.Errorf("synIDTermMap is nil") + } + + // If the synonymID is not found in the map, return an error + term, exists := i.synIDTermMap[synID] + if !exists { + return nil, fmt.Errorf("synonymID %d not found in map", synID) + } + + i.nextSyn = Synonym{} // clear the struct + rv := &i.nextSyn + rv.term = string(term) + rv.synID = synID + rv.docNum = docNum + + return rv, nil +} + +func (i *SynonymsIterator) nextSynonym() (uint32, uint32, bool, error) { + // If no synonyms are available, return early + if i.Actual == nil || i.synonyms == nil || i.synonyms == emptySynonymsList { + return 0, 0, false, nil + } + + var code uint64 + var docNum uint32 + var synID uint32 + + // Loop to find the next valid docNum, checking against the except + for i.Actual.HasNext() { + code = i.Actual.Next() + synID, docNum = decodeSynonym(code) + + // If docNum is not in the 'except' set, it's a valid result + if i.except == nil || !i.except.Contains(docNum) { + return synID, docNum, true, nil + } + } + + // If no valid docNum is found, return false + return 0, 0, false, nil +} + +type Synonym struct { + term string + synID uint32 + docNum uint32 +} + +func (s *Synonym) Term() string { + return s.term +} + +func (s *Synonym) SynonymID() uint32 { + return s.synID +} + +func (s *Synonym) DocNum() uint32 { + return s.docNum +} + +func decodeSynonym(synonymCode uint64) (synonymID uint32, docID uint32) { + return uint32(synonymCode >> 32), uint32(synonymCode) +} diff --git a/synonym_test.go b/synonym_test.go new file mode 100644 index 00000000..fc710c55 --- /dev/null +++ b/synonym_test.go @@ -0,0 +1,15 @@ +// Copyright (c) 2024 Couchbase, 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 zap diff --git a/thesaurus.go b/thesaurus.go new file mode 100644 index 00000000..399e77e7 --- /dev/null +++ b/thesaurus.go @@ -0,0 +1,114 @@ +// Copyright (c) 2024 Couchbase, 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 zap + +import ( + "bytes" + "fmt" + + "github.com/RoaringBitmap/roaring" + segment "github.com/blevesearch/scorch_segment_api/v2" + "github.com/blevesearch/vellum" +) + +// Thesaurus is the zap representation of a Thesaurus +type Thesaurus struct { + sb *SegmentBase + name string + thesaurusID uint16 + synIDTermMap map[uint32][]byte + fst *vellum.FST + + fstReader *vellum.Reader +} + +// // represents an immutable, empty Thesaurus +var emptyThesaurus = &Thesaurus{} + +// SynonymsList returns the synonyms list for the specified term +func (t *Thesaurus) SynonymsList(term []byte, except *roaring.Bitmap, prealloc segment.SynonymsList) (segment.SynonymsList, error) { + var preallocSL *SynonymsList + sl, ok := prealloc.(*SynonymsList) + if ok && sl != nil { + preallocSL = sl + } + return t.synonymsList(term, except, preallocSL) +} + +func (t *Thesaurus) synonymsList(term []byte, except *roaring.Bitmap, rv *SynonymsList) (*SynonymsList, error) { + if t.fstReader == nil { + if rv == nil || rv == emptySynonymsList { + return emptySynonymsList, nil + } + return t.synonymsListInit(rv, except), nil + } + + synonymsOffset, exists, err := t.fstReader.Get(term) + + if err != nil { + return nil, fmt.Errorf("vellum err: %v", err) + } + if !exists { + if rv == nil || rv == emptySynonymsList { + return emptySynonymsList, nil + } + return t.synonymsListInit(rv, except), nil + } + + return t.synonymsListFromOffset(synonymsOffset, except, rv) +} + +func (t *Thesaurus) synonymsListFromOffset(synonymsOffset uint64, except *roaring.Bitmap, rv *SynonymsList) (*SynonymsList, error) { + rv = t.synonymsListInit(rv, except) + + err := rv.read(synonymsOffset, t) + if err != nil { + return nil, err + } + + return rv, nil +} + +func (t *Thesaurus) synonymsListInit(rv *SynonymsList, except *roaring.Bitmap) *SynonymsList { + if rv == nil || rv == emptySynonymsList { + rv = &SynonymsList{} + rv.buffer = bytes.NewReader(nil) + } else { + synonyms := rv.synonyms + buf := rv.buffer + if synonyms != nil { + synonyms.Clear() + } + if buf != nil { + buf.Reset(nil) + } + + *rv = SynonymsList{} // clear the struct + + rv.synonyms = synonyms + rv.buffer = buf + } + rv.sb = t.sb + rv.except = except + rv.synIDTermMap = t.synIDTermMap + return rv +} + +func (t *Thesaurus) Contains(key []byte) (bool, error) { + if t.fst != nil { + return t.fst.Contains(key) + } + return false, nil +} diff --git a/thesaurus_test.go b/thesaurus_test.go new file mode 100644 index 00000000..fc710c55 --- /dev/null +++ b/thesaurus_test.go @@ -0,0 +1,15 @@ +// Copyright (c) 2024 Couchbase, 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 zap From 2bde50a37243f08442dcb82a0aa52c0bc71d6366 Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Thu, 10 Oct 2024 19:00:37 +0530 Subject: [PATCH 02/28] second draft --- build.go | 8 ++-- new.go | 2 - segment.go | 108 ++++++++++++----------------------------------- synonym_cache.go | 84 +++++++++++++++++------------------- thesaurus.go | 2 +- 5 files changed, 70 insertions(+), 134 deletions(-) diff --git a/build.go b/build.go index acbacd0a..cbbd2abc 100644 --- a/build.go +++ b/build.go @@ -170,14 +170,12 @@ func InitSegmentBase(mem []byte, memCRC uint32, chunkMode uint32, numDocs uint64 fieldDvReaders: make([]map[uint16]*docValueReader, len(segmentSections)), docValueOffset: 0, // docValueOffsets identified automatically by the section fieldFSTs: make(map[uint16]*vellum.FST), - thesaurusFSTs: make(map[uint16]*vellum.FST), vecIndexCache: newVectorIndexCache(), synIndexCache: newSynonymIndexCache(), // following fields gets populated by loadFieldsNew - fieldsMap: make(map[string]uint16), - thesaurusMap: make(map[string]uint16), - dictLocs: make([]uint64, 0), - fieldsInv: make([]string, 0), + fieldsMap: make(map[string]uint16), + dictLocs: make([]uint64, 0), + fieldsInv: make([]string, 0), } sb.updateSize() diff --git a/new.go b/new.go index 806a261b..f0d37c43 100644 --- a/new.go +++ b/new.go @@ -102,8 +102,6 @@ type interim struct { // name -> field id + 1 FieldsMap map[string]uint16 - ThesaurusMap map[string]uint16 - // FieldsInv is the inverse of FieldsMap // field id -> name FieldsInv []string diff --git a/segment.go b/segment.go index dfd0c912..c868798d 100644 --- a/segment.go +++ b/segment.go @@ -54,9 +54,7 @@ func (*ZapPlugin) Open(path string) (segment.Segment, error) { rv := &Segment{ SegmentBase: SegmentBase{ fieldsMap: make(map[string]uint16), - thesaurusMap: make(map[string]uint16), fieldFSTs: make(map[uint16]*vellum.FST), - thesaurusFSTs: make(map[uint16]*vellum.FST), vecIndexCache: newVectorIndexCache(), synIndexCache: newSynonymIndexCache(), fieldDvReaders: make([]map[uint16]*docValueReader, len(segmentSections)), @@ -95,7 +93,6 @@ type SegmentBase struct { memCRC uint32 chunkMode uint32 fieldsMap map[string]uint16 // fieldName -> fieldID+1 - thesaurusMap map[string]uint16 // thesaurusName -> thesaurusID+1 fieldsInv []string // fieldID -> fieldName fieldsSectionsMap []map[uint16]uint64 // fieldID -> section -> address numDocs uint64 @@ -104,7 +101,6 @@ type SegmentBase struct { sectionsIndexOffset uint64 docValueOffset uint64 dictLocs []uint64 - thesaurusLocs []uint64 fieldDvReaders []map[uint16]*docValueReader // naive chunk cache per field; section->field->reader fieldDvNames []string // field names cached in fieldDvReaders size uint64 @@ -113,9 +109,8 @@ type SegmentBase struct { bytesRead uint64 bytesWritten uint64 - m sync.Mutex - fieldFSTs map[uint16]*vellum.FST - thesaurusFSTs map[uint16]*vellum.FST + m sync.Mutex + fieldFSTs map[uint16]*vellum.FST // this cache comes into play when vectors are supported in builds. vecIndexCache *vectorIndexCache @@ -135,17 +130,11 @@ func (sb *SegmentBase) updateSize() { sizeInBytes += (len(k) + SizeOfString) + SizeOfUint16 } - // thesaurusMap - for k := range sb.thesaurusMap { - sizeInBytes += (len(k) + SizeOfString) + SizeOfUint16 - } - // fieldsInv, dictLocs, thesaurusLocs for _, entry := range sb.fieldsInv { sizeInBytes += len(entry) + SizeOfString } sizeInBytes += len(sb.dictLocs) * SizeOfUint64 - sizeInBytes += len(sb.thesaurusLocs) * SizeOfUint64 // fieldDvReaders for _, secDvReaders := range sb.fieldDvReaders { @@ -315,7 +304,6 @@ func (s *SegmentBase) loadFields() error { dictLoc, read := binary.Uvarint(s.mem[addr:fieldsIndexEnd]) n := uint64(read) s.dictLocs = append(s.dictLocs, dictLoc) - // do not load thesaurusLocs as they are not present in the older file formats var nameLen uint64 nameLen, read = binary.Uvarint(s.mem[addr+n : fieldsIndexEnd]) @@ -326,7 +314,6 @@ func (s *SegmentBase) loadFields() error { s.incrementBytesRead(n + nameLen) s.fieldsInv = append(s.fieldsInv, name) s.fieldsMap[name] = uint16(fieldID + 1) - // do not load thesaurusMap as they are not present in the older file formats fieldID++ } @@ -432,26 +419,6 @@ func (s *SegmentBase) loadFieldNew(fieldID uint16, pos uint64, // account the bytes read while parsing the field's inverted index section s.incrementBytesRead(uint64(read + n)) s.dictLocs = append(s.dictLocs, dictLoc) - } else if fieldSectionType == SectionSynonymIndex { - // this field is a thesaurus field and we need to read the thesaurus - // data from the file. - if fieldSectionAddr == 0 { - s.thesaurusLocs = append(s.thesaurusLocs, 0) - continue - } - read := 0 - // skip the doc values - _, n := binary.Uvarint(s.mem[fieldSectionAddr : fieldSectionAddr+binary.MaxVarintLen64]) - fieldSectionAddr += uint64(n) - read += n - _, n = binary.Uvarint(s.mem[fieldSectionAddr : fieldSectionAddr+binary.MaxVarintLen64]) - fieldSectionAddr += uint64(n) - read += n - thesaurusLoc, n := binary.Uvarint(s.mem[fieldSectionAddr : fieldSectionAddr+binary.MaxVarintLen64]) - // account the bytes read while parsing the field's synonym section - s.incrementBytesRead(uint64(read + n)) - s.thesaurusLocs = append(s.thesaurusLocs, thesaurusLoc) - s.thesaurusMap[fieldName] = uint16(fieldID + 1) } } @@ -521,43 +488,33 @@ func (s *SegmentBase) Thesaurus(name string) (segment.Thesaurus, error) { } func (sb *SegmentBase) thesaurus(name string) (rv *Thesaurus, err error) { - thesaurusIDPlus1 := sb.thesaurusMap[name] - if thesaurusIDPlus1 > 0 { + fieldIDPlus1 := sb.fieldsMap[name] + if fieldIDPlus1 == 0 { + return nil, nil + } + thesaurusStart := sb.fieldsSectionsMap[fieldIDPlus1-1][SectionSynonymIndex] + if thesaurusStart > 0 { rv = &Thesaurus{ - sb: sb, - name: name, - thesaurusID: thesaurusIDPlus1 - 1, + sb: sb, + name: name, + fieldID: fieldIDPlus1 - 1, } - thesaurusStart := sb.thesaurusLocs[rv.thesaurusID] - if thesaurusStart > 0 { - var ok bool - sb.m.Lock() - if rv.fst, ok = sb.thesaurusFSTs[rv.thesaurusID]; !ok { - // read the length of the vellum data - vellumLen, read := binary.Uvarint(sb.mem[thesaurusStart : thesaurusStart+binary.MaxVarintLen64]) - if vellumLen == 0 { - sb.m.Unlock() - return nil, fmt.Errorf("empty thesaurus for name: %v", name) - } - fstBytes := sb.mem[thesaurusStart+uint64(read) : thesaurusStart+uint64(read)+vellumLen] - rv.fst, err = vellum.Load(fstBytes) - if err != nil { - sb.m.Unlock() - return nil, fmt.Errorf("thesaurus name %s vellum err: %v", name, err) - } - rv.synIDTermMap = sb.synIndexCache.loadOrCreate(rv.thesaurusID, sb.mem[thesaurusStart+uint64(read)+vellumLen:]) - sb.thesaurusFSTs[rv.thesaurusID] = rv.fst - } else { - if rv.synIDTermMap, ok = sb.synIndexCache.load(rv.thesaurusID); !ok { - sb.m.Unlock() - return nil, fmt.Errorf("thesaurus name %s synIDTermMap not found", name) - } - } - sb.m.Unlock() - rv.fstReader, err = rv.fst.Reader() - if err != nil { - return nil, fmt.Errorf("thesaurus name %s vellum reader err: %v", name, err) - } + // the below loop loads the following: + // 1. doc values(first 2 iterations) - adhering to the sections format. never + // valid values for synonym section. + for i := 0; i < 2; i++ { + _, n := binary.Uvarint(sb.mem[thesaurusStart : thesaurusStart+binary.MaxVarintLen64]) + thesaurusStart += uint64(n) + } + fst, synTermMap, err := sb.synIndexCache.loadOrCreate(rv.fieldID, sb.mem[thesaurusStart:]) + if err != nil { + return nil, fmt.Errorf("thesaurus name %s err: %v", name, err) + } + rv.fst = fst + rv.synIDTermMap = synTermMap + rv.fstReader, err = rv.fst.Reader() + if err != nil { + return nil, fmt.Errorf("thesaurus name %s vellum reader err: %v", name, err) } } return rv, nil @@ -811,17 +768,6 @@ func (s *Segment) DictAddr(field string) (uint64, error) { return s.dictLocs[fieldIDPlus1-1], nil } -// ThesaurusAddr is a helper function to compute the file offset where the -// thesaurus is stored for the specified thesaurus name. -func (s *Segment) ThesaurusAddr(name string) (uint64, error) { - thesaurusIDPlus1, ok := s.thesaurusMap[name] - if !ok { - return 0, fmt.Errorf("no such thesaurus '%s'", name) - } - - return s.thesaurusLocs[thesaurusIDPlus1-1], nil -} - func (s *Segment) getSectionDvOffsets(fieldID int, secID uint16) (uint64, uint64, uint64, error) { // Version is gonna be 16 var fieldLocStart uint64 = fieldNotUninverted diff --git a/synonym_cache.go b/synonym_cache.go index 635c3bac..1636b466 100644 --- a/synonym_cache.go +++ b/synonym_cache.go @@ -16,7 +16,10 @@ package zap import ( "encoding/binary" + "fmt" "sync" + + "github.com/blevesearch/vellum" ) func newSynonymIndexCache() *synonymIndexCache { @@ -39,11 +42,9 @@ func (sc *synonymIndexCache) Clear() { sc.m.Unlock() } -func (sc *synonymIndexCache) loadOrCreate(thesaurusID uint16, mem []byte) map[uint32][]byte { - +func (sc *synonymIndexCache) loadOrCreate(fieldID uint16, mem []byte) (*vellum.FST, map[uint32][]byte, error) { sc.m.RLock() - - entry, ok := sc.cache[thesaurusID] + entry, ok := sc.cache[fieldID] if ok { sc.m.RUnlock() return entry.load() @@ -54,69 +55,62 @@ func (sc *synonymIndexCache) loadOrCreate(thesaurusID uint16, mem []byte) map[ui sc.m.Lock() defer sc.m.Unlock() - entry, ok = sc.cache[thesaurusID] + entry, ok = sc.cache[fieldID] if ok { return entry.load() } - return sc.createAndCacheLOCKED(thesaurusID, mem) + return sc.createAndCacheLOCKED(fieldID, mem) } -func (sc *synonymIndexCache) load(thesaurusID uint16) (map[uint32][]byte, bool) { - sc.m.RLock() - defer sc.m.RUnlock() - - entry, ok := sc.cache[thesaurusID] - if !ok { - return nil, false +func (sc *synonymIndexCache) createAndCacheLOCKED(fieldID uint16, mem []byte) (*vellum.FST, map[uint32][]byte, error) { + var pos uint64 + // read the length of the vellum data + vellumLen, read := binary.Uvarint(mem[pos : pos+binary.MaxVarintLen64]) + if vellumLen == 0 || read <= 0 { + return nil, nil, fmt.Errorf("vellum length is 0") + } + fstBytes := mem[pos+uint64(read) : pos+uint64(read)+vellumLen] + fst, err := vellum.Load(fstBytes) + if err != nil { + return nil, nil, fmt.Errorf("vellum err: %v", err) } - - return entry.load(), true -} - -func (sc *synonymIndexCache) createAndCacheLOCKED(thesaurusID uint16, mem []byte) map[uint32][]byte { synTermMap := make(map[uint32][]byte) - pos := 0 numSyns, n := binary.Uvarint(mem[pos : pos+binary.MaxVarintLen64]) - pos += n + pos += uint64(n) + if numSyns == 0 { + return nil, nil, fmt.Errorf("no synonyms found") + } for i := 0; i < int(numSyns); i++ { synID, n := binary.Uvarint(mem[pos : pos+binary.MaxVarintLen64]) - pos += n - + pos += uint64(n) termLen, n := binary.Uvarint(mem[pos : pos+binary.MaxVarintLen64]) - pos += n - - term := mem[pos : pos+int(termLen)] - + pos += uint64(n) + if termLen == 0 { + return nil, nil, fmt.Errorf("term length is 0") + } + term := mem[pos : pos+uint64(termLen)] synTermMap[uint32(synID)] = term } - sc.insertLOCKED(thesaurusID, synTermMap) - return synTermMap + sc.insertLOCKED(fieldID, fst, synTermMap) + return fst, synTermMap, nil } -func (sc *synonymIndexCache) insertLOCKED(thesaurusID uint16, synTermMap map[uint32][]byte) { - _, ok := sc.cache[thesaurusID] +func (sc *synonymIndexCache) insertLOCKED(fieldID uint16, fst *vellum.FST, synTermMap map[uint32][]byte) { + _, ok := sc.cache[fieldID] if !ok { - // initializing the alpha with 0.4 essentially means that we are favoring - // the history a little bit more relative to the current sample value. - // this makes the average to be kept above the threshold value for a - // longer time and thereby the index to be resident in the cache - // for longer time. - sc.cache[thesaurusID] = createSynonymCacheEntry(synTermMap) - } -} - -func createSynonymCacheEntry(synTermMap map[uint32][]byte) *synonymCacheEntry { - ce := &synonymCacheEntry{ - synTermMap: synTermMap, + sc.cache[fieldID] = &synonymCacheEntry{ + fst: fst, + synTermMap: synTermMap, + } } - return ce } type synonymCacheEntry struct { + fst *vellum.FST synTermMap map[uint32][]byte } -func (ce *synonymCacheEntry) load() map[uint32][]byte { - return ce.synTermMap +func (ce *synonymCacheEntry) load() (*vellum.FST, map[uint32][]byte, error) { + return ce.fst, ce.synTermMap, nil } diff --git a/thesaurus.go b/thesaurus.go index 399e77e7..28515f9d 100644 --- a/thesaurus.go +++ b/thesaurus.go @@ -27,7 +27,7 @@ import ( type Thesaurus struct { sb *SegmentBase name string - thesaurusID uint16 + fieldID uint16 synIDTermMap map[uint32][]byte fst *vellum.FST From 9f7957fe20ba2ba0db693dedd608c6cb78fd6673 Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Mon, 14 Oct 2024 18:59:53 +0530 Subject: [PATCH 03/28] third draft --- doc_test.go | 146 ++++++++++++++++++++++ section_faiss_vector_index.go | 4 +- section_inverted_text_index.go | 36 ++++-- section_synonym.go | 121 +++++++++---------- segment.go | 7 +- synonym_cache.go | 5 +- thesaurus_test.go | 214 +++++++++++++++++++++++++++++++++ 7 files changed, 455 insertions(+), 78 deletions(-) diff --git a/doc_test.go b/doc_test.go index 85e227b2..0b0a3a03 100644 --- a/doc_test.go +++ b/doc_test.go @@ -176,3 +176,149 @@ func (s *stubField) NumPlainTextBytes() uint64 { func (s *stubField) Compose(field string, length int, freq index.TokenFrequencies) { } + +// ----------------------------------------------------------------------------- +type stubSynonymDefinition struct { + term string + synonyms []string +} + +func (s *stubSynonymDefinition) Term() string { + return s.term +} + +func (s *stubSynonymDefinition) Synonyms() []string { + return s.synonyms +} + +func newStubSynonymDefinition(term string, synonyms []string) index.SynonymDefinition { + return &stubSynonymDefinition{ + term: term, + synonyms: synonyms, + } +} + +// ----------------------------------------------------------------------------- +type stubSynonymField struct { + name string + synonymDefs []index.SynonymDefinition +} + +func (s *stubSynonymField) Name() string { + return s.name +} + +func (s *stubSynonymField) Value() []byte { + return nil +} + +func (s *stubSynonymField) ArrayPositions() []uint64 { + return nil +} + +func (s *stubSynonymField) EncodedFieldType() byte { + return 0 +} + +func (s *stubSynonymField) Analyze() { + +} + +func (s *stubSynonymField) Options() index.FieldIndexingOptions { + return 0 +} + +func (s *stubSynonymField) AnalyzedLength() int { + return 0 +} + +func (s *stubSynonymField) AnalyzedTokenFrequencies() index.TokenFrequencies { + return nil +} + +func (s *stubSynonymField) NumPlainTextBytes() uint64 { + return 0 +} + +func (sf *stubSynonymField) VisitSynonymDefinitions(visitor func(index.SynonymDefinition)) { + for _, def := range sf.synonymDefs { + visitor(def) + } +} + +func analyzeStubTerm(term string) string { + lowerCaseTerm := strings.ToLower(term) + return lowerCaseTerm +} + +func newStubSynonymField(name, analyzer string, defs []index.SynonymDefinition) index.SynonymField { + analyzedSynonymDefs := make([]index.SynonymDefinition, 0, len(defs)) + for _, def := range defs { + analyzedTerm := analyzeStubTerm(def.Term()) + analyzedSynonyms := make([]string, 0, len(def.Synonyms())) + for _, syn := range def.Synonyms() { + analyzedSynonyms = append(analyzedSynonyms, analyzeStubTerm(syn)) + } + analyzedSynonymDefs = append(analyzedSynonymDefs, newStubSynonymDefinition(analyzedTerm, analyzedSynonyms)) + } + return &stubSynonymField{ + name: name, + synonymDefs: analyzedSynonymDefs, + } +} + +// ----------------------------------------------------------------------------- +type stubSynonymDocument struct { + id string + fields []index.Field +} + +func (s *stubSynonymDocument) ID() string { + return s.id +} + +func (s *stubSynonymDocument) Size() int { + return 0 +} + +func (s *stubSynonymDocument) VisitFields(visitor index.FieldVisitor) { + for _, f := range s.fields { + visitor(f) + } +} + +func (s *stubSynonymDocument) HasComposite() bool { + return false +} + +func (s *stubSynonymDocument) VisitComposite(visitor index.CompositeFieldVisitor) { +} + +func (s *stubSynonymDocument) NumPlainTextBytes() uint64 { + return 0 +} +func (s *stubSynonymDocument) StoredFieldsBytes() uint64 { + return 0 +} + +func (s *stubSynonymDocument) AddIDField() { + s.fields = append(s.fields, newStubFieldSplitString("_id", nil, s.id, true, false, false)) +} + +func (s *stubSynonymDocument) VisitSynonymField(visitor index.SynonymFieldVisitor) { + for _, f := range s.fields { + if sf, ok := f.(index.SynonymField); ok { + visitor(sf) + } + } +} + +func newStubSynonymDocument(id string, synonymField index.SynonymField) index.SynonymDocument { + rv := &stubSynonymDocument{ + id: id, + fields: []index.Field{synonymField}, + } + return rv +} + +// ----------------------------------------------------------------------------- diff --git a/section_faiss_vector_index.go b/section_faiss_vector_index.go index 1c9f91a0..ef712317 100644 --- a/section_faiss_vector_index.go +++ b/section_faiss_vector_index.go @@ -36,10 +36,10 @@ const defaultFaissOMPThreads = 1 func init() { rand.Seed(time.Now().UTC().UnixNano()) registerSegmentSection(SectionFaissVectorIndex, &faissVectorIndexSection{}) - isFieldNotApplicableToInvertedTextSection = func(field index.Field) bool { + invertedIndexExclusionChecks = append(invertedIndexExclusionChecks, func(field index.Field) bool { _, ok := field.(index.VectorField) return ok - } + }) faiss.SetOMPThreads(defaultFaissOMPThreads) } diff --git a/section_inverted_text_index.go b/section_inverted_text_index.go index ea8722e4..51702721 100644 --- a/section_inverted_text_index.go +++ b/section_inverted_text_index.go @@ -34,16 +34,27 @@ func init() { type invertedTextIndexSection struct { } -// this function is something that tells the inverted index section whether to -// process a particular field or not - since it might be processed by another -// section this function helps in avoiding unnecessary work. -// (only used by faiss vector section currently, will need a separate API for every -// section we introduce in the future or a better way forward - TODO) -var isFieldNotApplicableToInvertedTextSection func(field index.Field) bool +// This function checks whether the inverted index section should process +// a particular field, avoiding unnecessary work if another section will handle it. +var isFieldExcludedFromInvertedIndex = func(field index.Field) bool { + for _, excludeField := range invertedIndexExclusionChecks { + if excludeField(field) { + // atleast one section has agreed to exclude this field + // from inverted index processing and has agreed to process it + // independently + return true + } + } + // no section has excluded this field from inverted index processing + // so it should be processed by the inverted index section + return false +} + +// List of checks to determine if a field is excluded from the inverted index section +var invertedIndexExclusionChecks = make([]func(field index.Field) bool, 0) func (i *invertedTextIndexSection) Process(opaque map[int]resetable, docNum uint32, field index.Field, fieldID uint16) { - if isFieldNotApplicableToInvertedTextSection == nil || - !isFieldNotApplicableToInvertedTextSection(field) { + if !isFieldExcludedFromInvertedIndex(field) { invIndexOpaque := i.getInvertedIndexOpaque(opaque) invIndexOpaque.process(field, fieldID, docNum) } @@ -439,6 +450,13 @@ func (io *invertedIndexOpaque) writeDicts(w *CountHashWriter) (dictOffsets []uin } for fieldID, terms := range io.DictKeys { + dict := io.Dicts[fieldID] + // dict is nil if the field is excluded from inverted index + // processing, so skip it + if len(dict) == 0 { + continue + } + if cap(docTermMap) < len(io.results) { docTermMap = make([][]byte, len(io.results)) } else { @@ -448,8 +466,6 @@ func (io *invertedIndexOpaque) writeDicts(w *CountHashWriter) (dictOffsets []uin } } - dict := io.Dicts[fieldID] - for _, term := range terms { // terms are already sorted pid := dict[term] - 1 diff --git a/section_synonym.go b/section_synonym.go index 405a8119..b4604191 100644 --- a/section_synonym.go +++ b/section_synonym.go @@ -31,6 +31,10 @@ import ( func init() { registerSegmentSection(SectionSynonymIndex, &synonymIndexSection{}) + invertedIndexExclusionChecks = append(invertedIndexExclusionChecks, func(field index.Field) bool { + _, ok := field.(index.SynonymField) + return ok + }) } // ----------------------------------------------------------------------------- @@ -110,6 +114,9 @@ func (so *synonymIndexOpaque) Reset() (err error) { if so.builder != nil { err = so.builder.Reset(&so.builderBuf) } + so.FieldIDtoThesaurusID = nil + so.SynonymTermToID = nil + so.SynonymIDtoTerm = nil so.tmp0 = so.tmp0[:0] return err @@ -127,9 +134,7 @@ func (so *synonymIndexOpaque) process(field index.SynonymField, fieldID uint16, termSynMap := so.SynonymTermToID[tid] - synDefs := field.SynonymDefinitions() - - for _, def := range synDefs { + field.VisitSynonymDefinitions(func(def index.SynonymDefinition) { pid := thesaurus[def.Term()] - 1 bs := so.Synonyms[pid] @@ -138,13 +143,14 @@ func (so *synonymIndexOpaque) process(field index.SynonymField, fieldID uint16, code := encodeSynonym(termSynMap[syn], docNum) bs.Add(code) } - } + }) } func (so *synonymIndexOpaque) realloc(fieldID uint16) { var pidNext int var sidNext uint32 so.ThesaurusMap = map[string]uint16{} + so.FieldIDtoThesaurusID = map[uint16]int{} for _, result := range so.results { if synDoc, ok := result.(index.SynonymDocument); ok { @@ -154,44 +160,42 @@ func (so *synonymIndexOpaque) realloc(fieldID uint16) { } } - visitSynonymField := func(synField index.SynonymField) { - thesaurusID := uint16(so.getOrDefineThesaurus(fieldID, synField.Name())) + for _, result := range so.results { + if synDoc, ok := result.(index.SynonymDocument); ok { + synDoc.VisitSynonymField(func(synField index.SynonymField) { - thesaurus := so.Thesauri[thesaurusID] - thesaurusKeys := so.ThesaurusKeys[thesaurusID] + thesaurusID := uint16(so.getOrDefineThesaurus(fieldID, synField.Name())) - synTermMap := so.SynonymIDtoTerm[thesaurusID] + thesaurus := so.Thesauri[thesaurusID] + thesaurusKeys := so.ThesaurusKeys[thesaurusID] - termSynMap := so.SynonymTermToID[thesaurusID] + synTermMap := so.SynonymIDtoTerm[thesaurusID] - synDefs := synField.SynonymDefinitions() - for _, synDef := range synDefs { - term := synDef.Term() - _, exists := thesaurus[term] - if !exists { - pidNext++ - pidPlus1 := uint64(pidNext) + termSynMap := so.SynonymTermToID[thesaurusID] - thesaurus[term] = pidPlus1 - thesaurusKeys = append(thesaurusKeys, term) - } - syns := synDef.Synonyms() - for _, syn := range syns { - _, exists := termSynMap[syn] - if !exists { - sidNext++ - sidPlus1 := sidNext - termSynMap[syn] = sidPlus1 - synTermMap[sidPlus1] = syn - } - } - } - so.ThesaurusKeys[thesaurusID] = thesaurusKeys - } + synField.VisitSynonymDefinitions(func(synDef index.SynonymDefinition) { + term := synDef.Term() + _, exists := thesaurus[term] + if !exists { + pidNext++ + pidPlus1 := uint64(pidNext) - for _, result := range so.results { - if synDoc, ok := result.(index.SynonymDocument); ok { - synDoc.VisitSynonymField(visitSynonymField) + thesaurus[term] = pidPlus1 + thesaurusKeys = append(thesaurusKeys, term) + } + syns := synDef.Synonyms() + for _, syn := range syns { + _, exists := termSynMap[syn] + if !exists { + sidNext++ + sidPlus1 := sidNext + termSynMap[syn] = sidPlus1 + synTermMap[sidPlus1] = syn + } + } + }) + so.ThesaurusKeys[thesaurusID] = thesaurusKeys + }) } } @@ -291,7 +295,6 @@ func (so *synonymIndexOpaque) writeThesauri(w *CountHashWriter) (thesOffsets []u return nil, err } - // record where this dictionary starts thesOffsets[thesaurusID] = uint64(w.Count()) vellumData := so.builderBuf.Bytes() @@ -318,7 +321,7 @@ func (so *synonymIndexOpaque) writeThesauri(w *CountHashWriter) (thesOffsets []u } // write out the synTermMap for this thesaurus - _, err := writeSynTermMap(so.SynonymIDtoTerm[thesaurusID], w, buf) + err := writeSynTermMap(so.SynonymIDtoTerm[thesaurusID], w, buf) if err != nil { return nil, err } @@ -430,7 +433,7 @@ func writeSynonyms(postings *roaring64.Bitmap, w *CountHashWriter, bufMaxVarintL postingsOffset := uint64(w.Count()) - _, err = writeRoaringSynonymWithLen(postings, w, bufMaxVarintLen64) + err = writeRoaringSynonymWithLen(postings, w, bufMaxVarintLen64) if err != nil { return 0, err } @@ -438,69 +441,61 @@ func writeSynonyms(postings *roaring64.Bitmap, w *CountHashWriter, bufMaxVarintL return postingsOffset, nil } -func writeSynTermMap(synTermMap map[uint32]string, w *CountHashWriter, bufMaxVarintLen64 []byte) ( - offset uint64, err error) { - +func writeSynTermMap(synTermMap map[uint32]string, w *CountHashWriter, bufMaxVarintLen64 []byte) error { if len(synTermMap) == 0 { - return 0, nil + return nil } - - synTermMapOffset := uint64(w.Count()) n := binary.PutUvarint(bufMaxVarintLen64, uint64(len(synTermMap))) - _, err = w.Write(bufMaxVarintLen64[:n]) + _, err := w.Write(bufMaxVarintLen64[:n]) if err != nil { - return 0, err + return err } for sid, term := range synTermMap { n = binary.PutUvarint(bufMaxVarintLen64, uint64(sid)) _, err = w.Write(bufMaxVarintLen64[:n]) if err != nil { - return 0, err + return err } n = binary.PutUvarint(bufMaxVarintLen64, uint64(len(term))) _, err = w.Write(bufMaxVarintLen64[:n]) if err != nil { - return 0, err + return err } _, err = w.Write([]byte(term)) if err != nil { - return 0, err + return err } } - return synTermMapOffset, nil + return nil } // writes out the length of the roaring bitmap in bytes as varint // then writes out the roaring bitmap itself func writeRoaringSynonymWithLen(r *roaring64.Bitmap, w io.Writer, - reuseBufVarint []byte) (int, error) { + reuseBufVarint []byte) error { buf, err := r.ToBytes() if err != nil { - return 0, err + return err } - var tw int - // write out the length n := binary.PutUvarint(reuseBufVarint, uint64(len(buf))) - nw, err := w.Write(reuseBufVarint[:n]) - tw += nw + _, err = w.Write(reuseBufVarint[:n]) if err != nil { - return tw, err + return err } // write out the roaring bytes - nw, err = w.Write(buf) - tw += nw + _, err = w.Write(buf) if err != nil { - return tw, err + return err } - return tw, nil + return nil } func mergeAndPersistSynonymSection(segments []*SegmentBase, dropsIn []*roaring.Bitmap, @@ -684,7 +679,7 @@ func mergeAndPersistSynonymSection(segments []*SegmentBase, dropsIn []*roaring.B } // write out the synTermMap for this thesaurus - _, err = writeSynTermMap(synTermMap, w, bufMaxVarintLen64) + err = writeSynTermMap(synTermMap, w, bufMaxVarintLen64) if err != nil { return nil, nil, err } diff --git a/segment.go b/segment.go index c868798d..971418d7 100644 --- a/segment.go +++ b/segment.go @@ -130,7 +130,7 @@ func (sb *SegmentBase) updateSize() { sizeInBytes += (len(k) + SizeOfString) + SizeOfUint16 } - // fieldsInv, dictLocs, thesaurusLocs + // fieldsInv, dictLocs for _, entry := range sb.fieldsInv { sizeInBytes += len(entry) + SizeOfString } @@ -506,7 +506,10 @@ func (sb *SegmentBase) thesaurus(name string) (rv *Thesaurus, err error) { _, n := binary.Uvarint(sb.mem[thesaurusStart : thesaurusStart+binary.MaxVarintLen64]) thesaurusStart += uint64(n) } - fst, synTermMap, err := sb.synIndexCache.loadOrCreate(rv.fieldID, sb.mem[thesaurusStart:]) + // 2. thesaurus location - valid value for synonym section. + thesLoc, n := binary.Uvarint(sb.mem[thesaurusStart : thesaurusStart+binary.MaxVarintLen64]) + thesaurusStart += uint64(n) + fst, synTermMap, err := sb.synIndexCache.loadOrCreate(rv.fieldID, sb.mem[thesLoc:]) if err != nil { return nil, fmt.Errorf("thesaurus name %s err: %v", name, err) } diff --git a/synonym_cache.go b/synonym_cache.go index 1636b466..060cd875 100644 --- a/synonym_cache.go +++ b/synonym_cache.go @@ -70,11 +70,13 @@ func (sc *synonymIndexCache) createAndCacheLOCKED(fieldID uint16, mem []byte) (* if vellumLen == 0 || read <= 0 { return nil, nil, fmt.Errorf("vellum length is 0") } - fstBytes := mem[pos+uint64(read) : pos+uint64(read)+vellumLen] + pos += uint64(read) + fstBytes := mem[pos : pos+vellumLen] fst, err := vellum.Load(fstBytes) if err != nil { return nil, nil, fmt.Errorf("vellum err: %v", err) } + pos += vellumLen synTermMap := make(map[uint32][]byte) numSyns, n := binary.Uvarint(mem[pos : pos+binary.MaxVarintLen64]) pos += uint64(n) @@ -90,6 +92,7 @@ func (sc *synonymIndexCache) createAndCacheLOCKED(fieldID uint16, mem []byte) (* return nil, nil, fmt.Errorf("term length is 0") } term := mem[pos : pos+uint64(termLen)] + pos += uint64(termLen) synTermMap[uint32(synID)] = term } sc.insertLOCKED(fieldID, fst, synTermMap) diff --git a/thesaurus_test.go b/thesaurus_test.go index fc710c55..67849ecd 100644 --- a/thesaurus_test.go +++ b/thesaurus_test.go @@ -13,3 +13,217 @@ // limitations under the License. package zap + +import ( + "os" + "sort" + "strconv" + "testing" + + "errors" + + index "github.com/blevesearch/bleve_index_api" + segment "github.com/blevesearch/scorch_segment_api/v2" +) + +func createEquivalentSynonymMap(input []string, resultMap map[string][]string) map[string][]string { + if resultMap == nil { + resultMap = make(map[string][]string) + } + for _, elem := range input { + for _, otherElem := range input { + if elem != otherElem { + resultMap[elem] = append(resultMap[elem], otherElem) + } + } + } + return resultMap +} + +func buildTestSynonymDocument(id string, collection string, terms []string, synonyms []string) index.Document { + synDefs := make(map[string][]string) + if terms == nil { + synDefs = createEquivalentSynonymMap(synonyms, synDefs) + } else { + for _, term := range terms { + synDefs[term] = synonyms + } + } + synonymDefs := make([]index.SynonymDefinition, 0, len(synDefs)) + for term, synonyms := range synDefs { + synonymDefs = append(synonymDefs, newStubSynonymDefinition(term, synonyms)) + } + synDoc := newStubSynonymDocument(id, newStubSynonymField(collection, "standard", synonymDefs)) + synDoc.AddIDField() + return synDoc +} + +func buildTestSegmentForThesaurus(results []index.Document) (*SegmentBase, error) { + seg, _, err := zapPlugin.newWithChunkMode(results, 1024) + return seg.(*SegmentBase), err +} + +func extractSynonymsForTermFromThesaurus(thes segment.Thesaurus, term string) ([]string, error) { + list, err := thes.SynonymsList([]byte(term), nil, nil) + if err != nil { + return nil, err + } + if list == nil { + return nil, errors.New("expected synonyms list") + } + listItr := list.Iterator(nil) + if listItr == nil { + return nil, errors.New("expected non-nil iterator") + } + var synonyms []string + for { + next, err := listItr.Next() + if err != nil { + return nil, err + } + if next == nil { + break + } + synonyms = append(synonyms, next.Term()) + } + return synonyms, nil +} + +func testSegmentSynonymAccuracy(collectionName string, testSynonymMap map[string][]string, seg segment.Segment) error { + dict, err := seg.Dictionary(collectionName) + if err != nil { + return err + } + if dict != emptyDictionary { + pl, err := dict.PostingsList([]byte{'a'}, nil, nil) + if err != nil { + return err + } + if pl != emptyPostingsList { + return errors.New("expected empty postings list") + } + } + synSeg, ok := seg.(segment.SynonymSegment) + if !ok { + return errors.New("expected synonym segment") + } + thes, err := synSeg.Thesaurus(collectionName) + if err != nil { + return err + } + if thes == emptyThesaurus { + return errors.New("expected a thesaurus") + } + for term, expectedSynonyms := range testSynonymMap { + synonyms, err := extractSynonymsForTermFromThesaurus(thes, term) + if err != nil { + return err + } + if len(synonyms) != len(expectedSynonyms) { + return errors.New("unexpected number of synonyms") + } + sort.Strings(synonyms) + sort.Strings(expectedSynonyms) + for i, synonym := range synonyms { + if synonym != expectedSynonyms[i] { + return errors.New("unexpected synonym") + } + } + } + return nil +} + +type testSynonymDefinition struct { + terms []string + synonyms []string +} + +func createExpectedSynonymMap(input []testSynonymDefinition) map[string][]string { + rv := make(map[string][]string) + for _, testSynonymDefinition := range input { + if testSynonymDefinition.terms == nil { + rv = createEquivalentSynonymMap(testSynonymDefinition.synonyms, rv) + } else { + for _, term := range testSynonymDefinition.terms { + rv[term] = append(rv[term], testSynonymDefinition.synonyms...) + } + } + } + return rv +} + +func TestThesaurusSingleSegment(t *testing.T) { + err := os.RemoveAll("/tmp/scorch.zap") + if err != nil { + t.Fatalf("error removing directory: %v", err) + } + collectionName := "coll1" + testSynonymDefinitions := []testSynonymDefinition{ + { + terms: nil, + synonyms: []string{ + "adeptness", + "aptitude", + "facility", + "faculty", + "capacity", + "power", + "knack", + "proficiency", + "ability", + }, + }, + { + terms: []string{"afflict"}, + synonyms: []string{ + "affect", + "bother", + "distress", + "oppress", + "trouble", + "torment", + }, + }, + { + terms: []string{"capacity"}, + synonyms: []string{ + "volume", + "content", + "size", + "dimensions", + "measure", + }, + }, + } + var testSynonymDocuments []index.Document + for i, testSynonymDefinition := range testSynonymDefinitions { + testSynonymDocuments = append(testSynonymDocuments, buildTestSynonymDocument( + strconv.Itoa(i), + collectionName, + testSynonymDefinition.terms, + testSynonymDefinition.synonyms, + )) + } + sb, err := buildTestSegmentForThesaurus(testSynonymDocuments) + if err != nil { + t.Fatalf("error building test seg: %v", err) + } + err = PersistSegmentBase(sb, "/tmp/scorch.zap") + if err != nil { + t.Fatalf("error persisting seg: %v", err) + } + seg, err := zapPlugin.Open("/tmp/scorch.zap") + if err != nil { + t.Fatalf("error opening seg: %v", err) + } + defer func() { + cerr := seg.Close() + if cerr != nil { + t.Fatalf("error closing seg: %v", err) + } + }() + err = testSegmentSynonymAccuracy(collectionName, createExpectedSynonymMap(testSynonymDefinitions), seg) + if err != nil { + t.Fatalf("error testing segment: %v", err) + } +} From 093bee13e4ba9f28228f73782f674b2d58e42450 Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Tue, 15 Oct 2024 14:39:31 +0530 Subject: [PATCH 04/28] fourth draft --- new.go | 36 +++++---- section_synonym.go | 26 ++++--- thesaurus_test.go | 188 ++++++++++++++++++++++++++++++--------------- 3 files changed, 160 insertions(+), 90 deletions(-) diff --git a/new.go b/new.go index f0d37c43..c99b933d 100644 --- a/new.go +++ b/new.go @@ -174,23 +174,6 @@ func (s *interim) convert() (uint64, uint64, error) { s.FieldsMap = map[string]uint16{} } - args := map[string]interface{}{ - "results": s.results, - "chunkMode": s.chunkMode, - } - if s.opaque == nil { - s.opaque = map[int]resetable{} - for i, x := range segmentSections { - s.opaque[int(i)] = x.InitOpaque(args) - } - } else { - for k, v := range args { - for _, op := range s.opaque { - op.Set(k, v) - } - } - } - s.getOrDefineField("_id") // _id field is fieldID 0 for _, result := range s.results { @@ -208,6 +191,25 @@ func (s *interim) convert() (uint64, uint64, error) { s.FieldsMap[fieldName] = uint16(fieldID + 1) } + args := map[string]interface{}{ + "results": s.results, + "chunkMode": s.chunkMode, + "fieldsMap": s.FieldsMap, + "fieldsInv": s.FieldsInv, + } + if s.opaque == nil { + s.opaque = map[int]resetable{} + for i, x := range segmentSections { + s.opaque[int(i)] = x.InitOpaque(args) + } + } else { + for k, v := range args { + for _, op := range s.opaque { + op.Set(k, v) + } + } + } + s.processDocuments() storedIndexOffset, err := s.writeStoredFields() diff --git a/section_synonym.go b/section_synonym.go index b4604191..35c14d3d 100644 --- a/section_synonym.go +++ b/section_synonym.go @@ -45,6 +45,13 @@ type synonymIndexOpaque struct { // indicates whether the following structs are initialized init bool + // FieldsMap maps field name to field id and must be set in + // the index opaque using the key "fieldsMap" + // used for ensuring accurate mapping between fieldID and + // thesaurusID + // name -> field id + FieldsMap map[string]uint16 + // ThesaurusMap adds 1 to thesaurus id to avoid zero value issues // name -> thesaurus id + 1 ThesaurusMap map[string]uint16 @@ -89,6 +96,8 @@ func (so *synonymIndexOpaque) Set(key string, value interface{}) { switch key { case "results": so.results = value.([]index.Document) + case "fieldsMap": + so.FieldsMap = value.(map[string]uint16) } } @@ -124,7 +133,7 @@ func (so *synonymIndexOpaque) Reset() (err error) { func (so *synonymIndexOpaque) process(field index.SynonymField, fieldID uint16, docNum uint32) { if !so.init && so.results != nil { - so.realloc(fieldID) + so.realloc() so.init = true } @@ -146,7 +155,7 @@ func (so *synonymIndexOpaque) process(field index.SynonymField, fieldID uint16, }) } -func (so *synonymIndexOpaque) realloc(fieldID uint16) { +func (so *synonymIndexOpaque) realloc() { var pidNext int var sidNext uint32 so.ThesaurusMap = map[string]uint16{} @@ -155,7 +164,8 @@ func (so *synonymIndexOpaque) realloc(fieldID uint16) { for _, result := range so.results { if synDoc, ok := result.(index.SynonymDocument); ok { synDoc.VisitSynonymField(func(synField index.SynonymField) { - so.getOrDefineThesaurus(fieldID, synField.Name()) + fieldIDPlus1 := so.FieldsMap[synField.Name()] + so.getOrDefineThesaurus(fieldIDPlus1-1, synField.Name()) }) } } @@ -163,8 +173,8 @@ func (so *synonymIndexOpaque) realloc(fieldID uint16) { for _, result := range so.results { if synDoc, ok := result.(index.SynonymDocument); ok { synDoc.VisitSynonymField(func(synField index.SynonymField) { - - thesaurusID := uint16(so.getOrDefineThesaurus(fieldID, synField.Name())) + fieldIDPlus1 := so.FieldsMap[synField.Name()] + thesaurusID := uint16(so.getOrDefineThesaurus(fieldIDPlus1-1, synField.Name())) thesaurus := so.Thesauri[thesaurusID] thesaurusKeys := so.ThesaurusKeys[thesaurusID] @@ -378,12 +388,6 @@ func (s *synonymIndexSection) Process(opaque map[int]resetable, docNum uint32, f return } if sf, ok := field.(index.SynonymField); ok { - // at this point we have a synonym document being processed - // and this document is expected to have a single field - // which is a synonym field. - // we consider the - // fieldName as the thesaurusName and - // fieldID as the thesaurusID. so := s.getSynonymIndexOpaque(opaque) so.process(sf, fieldID, docNum) } diff --git a/thesaurus_test.go b/thesaurus_test.go index 67849ecd..79611d22 100644 --- a/thesaurus_test.go +++ b/thesaurus_test.go @@ -22,6 +22,7 @@ import ( "errors" + "github.com/RoaringBitmap/roaring" index "github.com/blevesearch/bleve_index_api" segment "github.com/blevesearch/scorch_segment_api/v2" ) @@ -63,8 +64,8 @@ func buildTestSegmentForThesaurus(results []index.Document) (*SegmentBase, error return seg.(*SegmentBase), err } -func extractSynonymsForTermFromThesaurus(thes segment.Thesaurus, term string) ([]string, error) { - list, err := thes.SynonymsList([]byte(term), nil, nil) +func extractSynonymsForTermFromThesaurus(thes segment.Thesaurus, term string, except *roaring.Bitmap) ([]string, error) { + list, err := thes.SynonymsList([]byte(term), except, nil) if err != nil { return nil, err } @@ -89,7 +90,7 @@ func extractSynonymsForTermFromThesaurus(thes segment.Thesaurus, term string) ([ return synonyms, nil } -func testSegmentSynonymAccuracy(collectionName string, testSynonymMap map[string][]string, seg segment.Segment) error { +func checkWithDeletes(except *roaring.Bitmap, collectionName string, testSynonymMap map[string][]string, seg segment.Segment) error { dict, err := seg.Dictionary(collectionName) if err != nil { return err @@ -115,7 +116,7 @@ func testSegmentSynonymAccuracy(collectionName string, testSynonymMap map[string return errors.New("expected a thesaurus") } for term, expectedSynonyms := range testSynonymMap { - synonyms, err := extractSynonymsForTermFromThesaurus(thes, term) + synonyms, err := extractSynonymsForTermFromThesaurus(thes, term, except) if err != nil { return err } @@ -133,6 +134,28 @@ func testSegmentSynonymAccuracy(collectionName string, testSynonymMap map[string return nil } +func testSegmentSynonymAccuracy(collSynMap map[string][]testSynonymDefinition, seg segment.Segment) error { + for collectionName, testSynonymMap := range collSynMap { + expectedSynonymMap := createExpectedSynonymMap(testSynonymMap) + err := checkWithDeletes(nil, collectionName, expectedSynonymMap, seg) + if err != nil { + return err + } + for i := 0; i < len(testSynonymMap); i++ { + except := roaring.New() + except.Add(uint32(i)) + modifiedSynonymMap := append([]testSynonymDefinition{}, testSynonymMap[:i]...) + modifiedSynonymMap = append(modifiedSynonymMap, testSynonymMap[i+1:]...) + expectedSynonymMap = createExpectedSynonymMap(modifiedSynonymMap) + err = checkWithDeletes(except, collectionName, expectedSynonymMap, seg) + if err != nil { + return err + } + } + } + return nil +} + type testSynonymDefinition struct { terms []string synonyms []string @@ -152,78 +175,119 @@ func createExpectedSynonymMap(input []testSynonymDefinition) map[string][]string return rv } -func TestThesaurusSingleSegment(t *testing.T) { - err := os.RemoveAll("/tmp/scorch.zap") +func buildSegment(testSynonymDefinitions map[string][]testSynonymDefinition) (segment.Segment, error) { + tmpDir, err := os.MkdirTemp("", "zap-") if err != nil { - t.Fatalf("error removing directory: %v", err) - } - collectionName := "coll1" - testSynonymDefinitions := []testSynonymDefinition{ - { - terms: nil, - synonyms: []string{ - "adeptness", - "aptitude", - "facility", - "faculty", - "capacity", - "power", - "knack", - "proficiency", - "ability", - }, - }, - { - terms: []string{"afflict"}, - synonyms: []string{ - "affect", - "bother", - "distress", - "oppress", - "trouble", - "torment", - }, - }, - { - terms: []string{"capacity"}, - synonyms: []string{ - "volume", - "content", - "size", - "dimensions", - "measure", - }, - }, + return nil, err + } + + err = os.RemoveAll(tmpDir) + if err != nil { + return nil, err } var testSynonymDocuments []index.Document - for i, testSynonymDefinition := range testSynonymDefinitions { - testSynonymDocuments = append(testSynonymDocuments, buildTestSynonymDocument( - strconv.Itoa(i), - collectionName, - testSynonymDefinition.terms, - testSynonymDefinition.synonyms, - )) + for collName, synDefs := range testSynonymDefinitions { + for i, testSynonymDefinition := range synDefs { + testSynonymDocuments = append(testSynonymDocuments, buildTestSynonymDocument( + strconv.Itoa(i), + collName, + testSynonymDefinition.terms, + testSynonymDefinition.synonyms, + )) + } } sb, err := buildTestSegmentForThesaurus(testSynonymDocuments) if err != nil { - t.Fatalf("error building test seg: %v", err) + return nil, err + } + err = PersistSegmentBase(sb, tmpDir) + if err != nil { + return nil, err } - err = PersistSegmentBase(sb, "/tmp/scorch.zap") + seg, err := zapPlugin.Open(tmpDir) if err != nil { - t.Fatalf("error persisting seg: %v", err) + return nil, err } - seg, err := zapPlugin.Open("/tmp/scorch.zap") + err = testSegmentSynonymAccuracy(testSynonymDefinitions, seg) if err != nil { - t.Fatalf("error opening seg: %v", err) + return nil, err + } + return seg, nil +} + +func TestSingleSegmentThesaurus(t *testing.T) { + firstCollectionName := "coll0" + secondCollectionName := "coll1" + testSynonymDefinitions := map[string][]testSynonymDefinition{ + firstCollectionName: { + { + terms: nil, + synonyms: []string{ + "adeptness", + "aptitude", + "facility", + "faculty", + "capacity", + "power", + "knack", + "proficiency", + "ability", + }, + }, + { + terms: []string{"afflict"}, + synonyms: []string{ + "affect", + "bother", + "distress", + "oppress", + "trouble", + "torment", + }, + }, + { + terms: []string{"capacity"}, + synonyms: []string{ + "volume", + "content", + "size", + "dimensions", + "measure", + }, + }, + }, + secondCollectionName: { + { + synonyms: []string{ + "absolutely", + "unqualifiedly", + "unconditionally", + "unreservedly", + "unexceptionally", + "unequivocally", + }, + }, + { + terms: []string{"abrupt"}, + synonyms: []string{ + "sudden", + "hasty", + "quick", + "precipitate", + "snappy", + }, + }, + }, + } + + seg1, err := buildSegment(testSynonymDefinitions) + if err != nil { + t.Fatalf("error building segment: %v", err) } defer func() { - cerr := seg.Close() + cerr := seg1.Close() if cerr != nil { t.Fatalf("error closing seg: %v", err) } }() - err = testSegmentSynonymAccuracy(collectionName, createExpectedSynonymMap(testSynonymDefinitions), seg) - if err != nil { - t.Fatalf("error testing segment: %v", err) - } } From ba3f3d59d261f66657246da051eb7b77bf7517b0 Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Tue, 15 Oct 2024 19:33:24 +0530 Subject: [PATCH 05/28] merge --- thesaurus_test.go | 72 +++++++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 70 insertions(+), 2 deletions(-) diff --git a/thesaurus_test.go b/thesaurus_test.go index 79611d22..f8d8d08a 100644 --- a/thesaurus_test.go +++ b/thesaurus_test.go @@ -215,7 +215,37 @@ func buildSegment(testSynonymDefinitions map[string][]testSynonymDefinition) (se return seg, nil } -func TestSingleSegmentThesaurus(t *testing.T) { +func mergeSegments(segs []segment.Segment, drops []*roaring.Bitmap, testSynonymDefinitions map[string][]testSynonymDefinition) error { + tmpDir, err := os.MkdirTemp("", "zap-") + if err != nil { + return err + } + err = os.RemoveAll(tmpDir) + if err != nil { + return err + } + // Test Merging of multiple segments + _, _, err = zapPlugin.Merge(segs, drops, tmpDir, nil, nil) + if err != nil { + return err + } + + seg, err := zapPlugin.Open(tmpDir) + if err != nil { + return err + } + err = testSegmentSynonymAccuracy(testSynonymDefinitions, seg) + if err != nil { + return err + } + cerr := seg.Close() + if cerr != nil { + return err + } + return nil +} + +func TestThesaurus(t *testing.T) { firstCollectionName := "coll0" secondCollectionName := "coll1" testSynonymDefinitions := map[string][]testSynonymDefinition{ @@ -279,7 +309,7 @@ func TestSingleSegmentThesaurus(t *testing.T) { }, }, } - + // single segment test seg1, err := buildSegment(testSynonymDefinitions) if err != nil { t.Fatalf("error building segment: %v", err) @@ -290,4 +320,42 @@ func TestSingleSegmentThesaurus(t *testing.T) { t.Fatalf("error closing seg: %v", err) } }() + + // multiple segment test + numSegs := 3 + numDocs := 5 + segData := make([]map[string][]testSynonymDefinition, numSegs) + + segData[0] = make(map[string][]testSynonymDefinition) + segData[0][firstCollectionName] = testSynonymDefinitions[firstCollectionName][:2] // 2 docs + + segData[1] = make(map[string][]testSynonymDefinition) + segData[1][firstCollectionName] = testSynonymDefinitions[firstCollectionName][2:] + segData[1][secondCollectionName] = testSynonymDefinitions[secondCollectionName][:1] // 2 docs + + segData[2] = make(map[string][]testSynonymDefinition) + segData[2][secondCollectionName] = testSynonymDefinitions[secondCollectionName][1:] // 1 doc + + segs := make([]segment.Segment, numSegs) + for i, data := range segData { + seg, err := buildSegment(data) + if err != nil { + t.Fatalf("error building segment: %v", err) + } + segs[i] = seg + } + drops := make([]*roaring.Bitmap, numDocs) + for i := 0; i < numDocs; i++ { + drops[i] = roaring.New() + } + err = mergeSegments(segs, drops, testSynonymDefinitions) + if err != nil { + t.Fatalf("error merging segments: %v", err) + } + for _, seg := range segs { + cerr := seg.Close() + if cerr != nil { + t.Fatalf("error closing seg: %v", err) + } + } } From 5234df29e36f707fefa6df0a07d47ac79e88599b Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Wed, 6 Nov 2024 16:12:48 +0530 Subject: [PATCH 06/28] fix size API impl --- synonym_posting.go | 38 ++++++++++++++++++++++++++++++++++++++ 1 file changed, 38 insertions(+) diff --git a/synonym_posting.go b/synonym_posting.go index 77ad5762..c965c16d 100644 --- a/synonym_posting.go +++ b/synonym_posting.go @@ -18,12 +18,26 @@ import ( "bytes" "encoding/binary" "fmt" + "reflect" "github.com/RoaringBitmap/roaring" "github.com/RoaringBitmap/roaring/roaring64" segment "github.com/blevesearch/scorch_segment_api/v2" ) +var reflectStaticSizeSynonymsList int +var reflectStaticSizeSynonymsIterator int +var reflectStaticSizeSynonym int + +func init() { + var sl SynonymsList + reflectStaticSizeSynonymsList = int(reflect.TypeOf(sl).Size()) + var si SynonymsIterator + reflectStaticSizeSynonymsIterator = int(reflect.TypeOf(si).Size()) + var s Synonym + reflectStaticSizePosting = int(reflect.TypeOf(s).Size()) +} + type SynonymsList struct { sb *SegmentBase synonymsOffset uint64 @@ -38,6 +52,16 @@ type SynonymsList struct { // represents an immutable, empty synonyms list var emptySynonymsList = &SynonymsList{} +func (p *SynonymsList) Size() int { + sizeInBytes := reflectStaticSizeSynonymsList + SizeOfPtr + + if p.except != nil { + sizeInBytes += int(p.except.GetSizeInBytes()) + } + + return sizeInBytes +} + // Iterator returns an iterator for this postings list func (s *SynonymsList) Iterator(prealloc segment.SynonymsIterator) segment.SynonymsIterator { if s.synonyms == nil { @@ -113,6 +137,13 @@ type SynonymsIterator struct { var emptySynonymsIterator = &SynonymsIterator{} +func (i *SynonymsIterator) Size() int { + sizeInBytes := reflectStaticSizeSynonymsIterator + SizeOfPtr + + i.nextSyn.Size() + + return sizeInBytes +} + func (i *SynonymsIterator) Next() (segment.Synonym, error) { return i.next() } @@ -173,6 +204,13 @@ type Synonym struct { docNum uint32 } +func (p *Synonym) Size() int { + sizeInBytes := reflectStaticSizePosting + SizeOfPtr + + len(p.term) + + return sizeInBytes +} + func (s *Synonym) Term() string { return s.term } From 83f1bcb18a2c660aab2131eaaae4b9facc078c6b Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Thu, 14 Nov 2024 19:45:23 +0530 Subject: [PATCH 07/28] test fixes --- doc_test.go | 51 +++------- section_synonym.go | 16 ++-- thesaurus_test.go | 228 +++++++++++++++++++++++---------------------- 3 files changed, 138 insertions(+), 157 deletions(-) diff --git a/doc_test.go b/doc_test.go index 0b0a3a03..ffc8da93 100644 --- a/doc_test.go +++ b/doc_test.go @@ -177,31 +177,10 @@ func (s *stubField) Compose(field string, length int, freq index.TokenFrequencie } -// ----------------------------------------------------------------------------- -type stubSynonymDefinition struct { - term string - synonyms []string -} - -func (s *stubSynonymDefinition) Term() string { - return s.term -} - -func (s *stubSynonymDefinition) Synonyms() []string { - return s.synonyms -} - -func newStubSynonymDefinition(term string, synonyms []string) index.SynonymDefinition { - return &stubSynonymDefinition{ - term: term, - synonyms: synonyms, - } -} - // ----------------------------------------------------------------------------- type stubSynonymField struct { - name string - synonymDefs []index.SynonymDefinition + name string + synonymMap map[string][]string } func (s *stubSynonymField) Name() string { @@ -240,9 +219,9 @@ func (s *stubSynonymField) NumPlainTextBytes() uint64 { return 0 } -func (sf *stubSynonymField) VisitSynonymDefinitions(visitor func(index.SynonymDefinition)) { - for _, def := range sf.synonymDefs { - visitor(def) +func (sf *stubSynonymField) VisitSynonymDefinitions(visitor func(term string, synonyms []string)) { + for term, synonyms := range sf.synonymMap { + visitor(term, synonyms) } } @@ -251,19 +230,19 @@ func analyzeStubTerm(term string) string { return lowerCaseTerm } -func newStubSynonymField(name, analyzer string, defs []index.SynonymDefinition) index.SynonymField { - analyzedSynonymDefs := make([]index.SynonymDefinition, 0, len(defs)) - for _, def := range defs { - analyzedTerm := analyzeStubTerm(def.Term()) - analyzedSynonyms := make([]string, 0, len(def.Synonyms())) - for _, syn := range def.Synonyms() { +func newStubSynonymField(name, analyzer string, synonymMap map[string][]string) index.SynonymField { + analyzedSynonymDefs := make(map[string][]string, len(synonymMap)) + for term, synonyms := range synonymMap { + analyzedTerm := analyzeStubTerm(term) + analyzedSynonyms := make([]string, 0, len(synonyms)) + for _, syn := range synonyms { analyzedSynonyms = append(analyzedSynonyms, analyzeStubTerm(syn)) } - analyzedSynonymDefs = append(analyzedSynonymDefs, newStubSynonymDefinition(analyzedTerm, analyzedSynonyms)) + analyzedSynonymDefs[analyzedTerm] = analyzedSynonyms } return &stubSynonymField{ - name: name, - synonymDefs: analyzedSynonymDefs, + name: name, + synonymMap: analyzedSynonymDefs, } } @@ -305,7 +284,7 @@ func (s *stubSynonymDocument) AddIDField() { s.fields = append(s.fields, newStubFieldSplitString("_id", nil, s.id, true, false, false)) } -func (s *stubSynonymDocument) VisitSynonymField(visitor index.SynonymFieldVisitor) { +func (s *stubSynonymDocument) VisitSynonymFields(visitor index.SynonymFieldVisitor) { for _, f := range s.fields { if sf, ok := f.(index.SynonymField); ok { visitor(sf) diff --git a/section_synonym.go b/section_synonym.go index 35c14d3d..6828028a 100644 --- a/section_synonym.go +++ b/section_synonym.go @@ -143,12 +143,12 @@ func (so *synonymIndexOpaque) process(field index.SynonymField, fieldID uint16, termSynMap := so.SynonymTermToID[tid] - field.VisitSynonymDefinitions(func(def index.SynonymDefinition) { - pid := thesaurus[def.Term()] - 1 + field.VisitSynonymDefinitions(func(term string, synonyms []string) { + pid := thesaurus[term] - 1 bs := so.Synonyms[pid] - for _, syn := range def.Synonyms() { + for _, syn := range synonyms { code := encodeSynonym(termSynMap[syn], docNum) bs.Add(code) } @@ -163,7 +163,7 @@ func (so *synonymIndexOpaque) realloc() { for _, result := range so.results { if synDoc, ok := result.(index.SynonymDocument); ok { - synDoc.VisitSynonymField(func(synField index.SynonymField) { + synDoc.VisitSynonymFields(func(synField index.SynonymField) { fieldIDPlus1 := so.FieldsMap[synField.Name()] so.getOrDefineThesaurus(fieldIDPlus1-1, synField.Name()) }) @@ -172,7 +172,7 @@ func (so *synonymIndexOpaque) realloc() { for _, result := range so.results { if synDoc, ok := result.(index.SynonymDocument); ok { - synDoc.VisitSynonymField(func(synField index.SynonymField) { + synDoc.VisitSynonymFields(func(synField index.SynonymField) { fieldIDPlus1 := so.FieldsMap[synField.Name()] thesaurusID := uint16(so.getOrDefineThesaurus(fieldIDPlus1-1, synField.Name())) @@ -183,8 +183,7 @@ func (so *synonymIndexOpaque) realloc() { termSynMap := so.SynonymTermToID[thesaurusID] - synField.VisitSynonymDefinitions(func(synDef index.SynonymDefinition) { - term := synDef.Term() + synField.VisitSynonymDefinitions(func(term string, synonyms []string) { _, exists := thesaurus[term] if !exists { pidNext++ @@ -193,8 +192,7 @@ func (so *synonymIndexOpaque) realloc() { thesaurus[term] = pidPlus1 thesaurusKeys = append(thesaurusKeys, term) } - syns := synDef.Synonyms() - for _, syn := range syns { + for _, syn := range synonyms { _, exists := termSynMap[syn] if !exists { sidNext++ diff --git a/thesaurus_test.go b/thesaurus_test.go index f8d8d08a..735e4fd0 100644 --- a/thesaurus_test.go +++ b/thesaurus_test.go @@ -42,19 +42,18 @@ func createEquivalentSynonymMap(input []string, resultMap map[string][]string) m } func buildTestSynonymDocument(id string, collection string, terms []string, synonyms []string) index.Document { - synDefs := make(map[string][]string) + var synonymMap map[string][]string if terms == nil { - synDefs = createEquivalentSynonymMap(synonyms, synDefs) + numEntries := len(synonyms) * (len(synonyms) - 1) + synonymMap = make(map[string][]string, numEntries) + synonymMap = createEquivalentSynonymMap(synonyms, synonymMap) } else { + synonymMap = make(map[string][]string, len(terms)) for _, term := range terms { - synDefs[term] = synonyms + synonymMap[term] = synonyms } } - synonymDefs := make([]index.SynonymDefinition, 0, len(synDefs)) - for term, synonyms := range synDefs { - synonymDefs = append(synonymDefs, newStubSynonymDefinition(term, synonyms)) - } - synDoc := newStubSynonymDocument(id, newStubSynonymField(collection, "standard", synonymDefs)) + synDoc := newStubSynonymDocument(id, newStubSynonymField(collection, "standard", synonymMap)) synDoc.AddIDField() return synDoc } @@ -121,20 +120,26 @@ func checkWithDeletes(except *roaring.Bitmap, collectionName string, testSynonym return err } if len(synonyms) != len(expectedSynonyms) { - return errors.New("unexpected number of synonyms") + return errors.New("unexpected number of synonyms, expected: " + + strconv.Itoa(len(expectedSynonyms)) + " got: " + + strconv.Itoa(len(synonyms)) + " for term: " + term + " when excepting: " + except.String()) } sort.Strings(synonyms) sort.Strings(expectedSynonyms) for i, synonym := range synonyms { if synonym != expectedSynonyms[i] { - return errors.New("unexpected synonym") + return errors.New("unexpected synonym" + synonym + " for term: " + term) } } } return nil } -func testSegmentSynonymAccuracy(collSynMap map[string][]testSynonymDefinition, seg segment.Segment) error { +func testSegmentSynonymAccuracy(testSynonymDefinitions []testSynonymDefinition, seg segment.Segment) error { + collSynMap := make(map[string][]testSynonymDefinition) + for _, testSynonymDefinition := range testSynonymDefinitions { + collSynMap[testSynonymDefinition.collectionName] = append(collSynMap[testSynonymDefinition.collectionName], testSynonymDefinition) + } for collectionName, testSynonymMap := range collSynMap { expectedSynonymMap := createExpectedSynonymMap(testSynonymMap) err := checkWithDeletes(nil, collectionName, expectedSynonymMap, seg) @@ -157,8 +162,9 @@ func testSegmentSynonymAccuracy(collSynMap map[string][]testSynonymDefinition, s } type testSynonymDefinition struct { - terms []string - synonyms []string + collectionName string + terms []string + synonyms []string } func createExpectedSynonymMap(input []testSynonymDefinition) map[string][]string { @@ -175,7 +181,7 @@ func createExpectedSynonymMap(input []testSynonymDefinition) map[string][]string return rv } -func buildSegment(testSynonymDefinitions map[string][]testSynonymDefinition) (segment.Segment, error) { +func buildSegment(testSynonymDefinitions []testSynonymDefinition) (segment.Segment, error) { tmpDir, err := os.MkdirTemp("", "zap-") if err != nil { return nil, err @@ -186,15 +192,13 @@ func buildSegment(testSynonymDefinitions map[string][]testSynonymDefinition) (se return nil, err } var testSynonymDocuments []index.Document - for collName, synDefs := range testSynonymDefinitions { - for i, testSynonymDefinition := range synDefs { - testSynonymDocuments = append(testSynonymDocuments, buildTestSynonymDocument( - strconv.Itoa(i), - collName, - testSynonymDefinition.terms, - testSynonymDefinition.synonyms, - )) - } + for i, testSynonymDefinition := range testSynonymDefinitions { + testSynonymDocuments = append(testSynonymDocuments, buildTestSynonymDocument( + strconv.Itoa(i), + testSynonymDefinition.collectionName, + testSynonymDefinition.terms, + testSynonymDefinition.synonyms, + )) } sb, err := buildTestSegmentForThesaurus(testSynonymDocuments) if err != nil { @@ -215,8 +219,8 @@ func buildSegment(testSynonymDefinitions map[string][]testSynonymDefinition) (se return seg, nil } -func mergeSegments(segs []segment.Segment, drops []*roaring.Bitmap, testSynonymDefinitions map[string][]testSynonymDefinition) error { - tmpDir, err := os.MkdirTemp("", "zap-") +func mergeSegments(segs []segment.Segment, drops []*roaring.Bitmap, testSynonymDefinitions []testSynonymDefinition) error { + tmpDir, err := os.MkdirTemp("", "mergedzap-") if err != nil { return err } @@ -245,67 +249,68 @@ func mergeSegments(segs []segment.Segment, drops []*roaring.Bitmap, testSynonymD return nil } -func TestThesaurus(t *testing.T) { +func TestSynonymSegment(t *testing.T) { firstCollectionName := "coll0" secondCollectionName := "coll1" - testSynonymDefinitions := map[string][]testSynonymDefinition{ - firstCollectionName: { - { - terms: nil, - synonyms: []string{ - "adeptness", - "aptitude", - "facility", - "faculty", - "capacity", - "power", - "knack", - "proficiency", - "ability", - }, + testSynonymDefinitions := []testSynonymDefinition{ + { + collectionName: firstCollectionName, + terms: nil, + synonyms: []string{ + "adeptness", + "aptitude", + "facility", + "faculty", + "capacity", + "power", + "knack", + "proficiency", + "ability", }, - { - terms: []string{"afflict"}, - synonyms: []string{ - "affect", - "bother", - "distress", - "oppress", - "trouble", - "torment", - }, + }, + { + collectionName: firstCollectionName, + terms: []string{"afflict"}, + synonyms: []string{ + "affect", + "bother", + "distress", + "oppress", + "trouble", + "torment", }, - { - terms: []string{"capacity"}, - synonyms: []string{ - "volume", - "content", - "size", - "dimensions", - "measure", - }, + }, + { + collectionName: firstCollectionName, + terms: []string{"capacity"}, + synonyms: []string{ + "volume", + "content", + "size", + "dimensions", + "measure", }, }, - secondCollectionName: { - { - synonyms: []string{ - "absolutely", - "unqualifiedly", - "unconditionally", - "unreservedly", - "unexceptionally", - "unequivocally", - }, + { + collectionName: secondCollectionName, + synonyms: []string{ + "absolutely", + "unqualifiedly", + "unconditionally", + "unreservedly", + "unexceptionally", + "unequivocally", }, - { - terms: []string{"abrupt"}, - synonyms: []string{ - "sudden", - "hasty", - "quick", - "precipitate", - "snappy", - }, + }, + { + collectionName: secondCollectionName, + terms: []string{"abrupt"}, + synonyms: []string{ + "sudden", + "hasty", + "quick", + "precipitate", + "snappy", }, }, } @@ -321,41 +326,40 @@ func TestThesaurus(t *testing.T) { } }() - // multiple segment test - numSegs := 3 - numDocs := 5 - segData := make([]map[string][]testSynonymDefinition, numSegs) + // // multiple segment test + // numSegs := 3 + // numDocs := 5 + // segData := make([][]testSynonymDefinition, numSegs) - segData[0] = make(map[string][]testSynonymDefinition) - segData[0][firstCollectionName] = testSynonymDefinitions[firstCollectionName][:2] // 2 docs + // segData[0] = make([]testSynonymDefinition, 0) + // segData[0] = testSynonymDefinitions[:2] // 2 docs - segData[1] = make(map[string][]testSynonymDefinition) - segData[1][firstCollectionName] = testSynonymDefinitions[firstCollectionName][2:] - segData[1][secondCollectionName] = testSynonymDefinitions[secondCollectionName][:1] // 2 docs + // segData[1] = make([]testSynonymDefinition, 0) + // segData[1] = testSynonymDefinitions[2:4] // 2 docs - segData[2] = make(map[string][]testSynonymDefinition) - segData[2][secondCollectionName] = testSynonymDefinitions[secondCollectionName][1:] // 1 doc + // segData[2] = make([]testSynonymDefinition, 0) + // segData[2] = testSynonymDefinitions[4:] // 1 doc - segs := make([]segment.Segment, numSegs) - for i, data := range segData { - seg, err := buildSegment(data) - if err != nil { - t.Fatalf("error building segment: %v", err) - } - segs[i] = seg - } - drops := make([]*roaring.Bitmap, numDocs) - for i := 0; i < numDocs; i++ { - drops[i] = roaring.New() - } - err = mergeSegments(segs, drops, testSynonymDefinitions) - if err != nil { - t.Fatalf("error merging segments: %v", err) - } - for _, seg := range segs { - cerr := seg.Close() - if cerr != nil { - t.Fatalf("error closing seg: %v", err) - } - } + // segs := make([]segment.Segment, numSegs) + // for i, data := range segData { + // seg, err := buildSegment(data) + // if err != nil { + // t.Fatalf("error building segment: %v", err) + // } + // segs[i] = seg + // } + // drops := make([]*roaring.Bitmap, numDocs) + // for i := 0; i < numDocs; i++ { + // drops[i] = roaring.New() + // } + // err = mergeSegments(segs, drops, testSynonymDefinitions) + // if err != nil { + // t.Fatalf("error merging segments: %v", err) + // } + // for _, seg := range segs { + // cerr := seg.Close() + // if cerr != nil { + // t.Fatalf("error closing seg: %v", err) + // } + // } } From ce33fef5167cebfa112f8f945b0d1173b7964094 Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Fri, 15 Nov 2024 12:48:32 +0530 Subject: [PATCH 08/28] fix tests --- section_inverted_text_index.go | 5 --- synonym_test.go | 15 ------- thesaurus_test.go | 82 +++++++++++++++++----------------- 3 files changed, 41 insertions(+), 61 deletions(-) delete mode 100644 synonym_test.go diff --git a/section_inverted_text_index.go b/section_inverted_text_index.go index 51702721..86a15f6c 100644 --- a/section_inverted_text_index.go +++ b/section_inverted_text_index.go @@ -451,11 +451,6 @@ func (io *invertedIndexOpaque) writeDicts(w *CountHashWriter) (dictOffsets []uin for fieldID, terms := range io.DictKeys { dict := io.Dicts[fieldID] - // dict is nil if the field is excluded from inverted index - // processing, so skip it - if len(dict) == 0 { - continue - } if cap(docTermMap) < len(io.results) { docTermMap = make([][]byte, len(io.results)) diff --git a/synonym_test.go b/synonym_test.go deleted file mode 100644 index fc710c55..00000000 --- a/synonym_test.go +++ /dev/null @@ -1,15 +0,0 @@ -// Copyright (c) 2024 Couchbase, 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 zap diff --git a/thesaurus_test.go b/thesaurus_test.go index 735e4fd0..a545c2ff 100644 --- a/thesaurus_test.go +++ b/thesaurus_test.go @@ -59,7 +59,7 @@ func buildTestSynonymDocument(id string, collection string, terms []string, syno } func buildTestSegmentForThesaurus(results []index.Document) (*SegmentBase, error) { - seg, _, err := zapPlugin.newWithChunkMode(results, 1024) + seg, _, err := zapPlugin.newWithChunkMode(results, DefaultChunkMode) return seg.(*SegmentBase), err } @@ -182,15 +182,6 @@ func createExpectedSynonymMap(input []testSynonymDefinition) map[string][]string } func buildSegment(testSynonymDefinitions []testSynonymDefinition) (segment.Segment, error) { - tmpDir, err := os.MkdirTemp("", "zap-") - if err != nil { - return nil, err - } - - err = os.RemoveAll(tmpDir) - if err != nil { - return nil, err - } var testSynonymDocuments []index.Document for i, testSynonymDefinition := range testSynonymDefinitions { testSynonymDocuments = append(testSynonymDocuments, buildTestSynonymDocument( @@ -204,6 +195,15 @@ func buildSegment(testSynonymDefinitions []testSynonymDefinition) (segment.Segme if err != nil { return nil, err } + tmpDir, err := os.MkdirTemp("", "zap-") + if err != nil { + return nil, err + } + + err = os.RemoveAll(tmpDir) + if err != nil { + return nil, err + } err = PersistSegmentBase(sb, tmpDir) if err != nil { return nil, err @@ -327,39 +327,39 @@ func TestSynonymSegment(t *testing.T) { }() // // multiple segment test - // numSegs := 3 - // numDocs := 5 - // segData := make([][]testSynonymDefinition, numSegs) + numSegs := 3 + numDocs := 5 + segData := make([][]testSynonymDefinition, numSegs) - // segData[0] = make([]testSynonymDefinition, 0) - // segData[0] = testSynonymDefinitions[:2] // 2 docs + segData[0] = make([]testSynonymDefinition, 0) + segData[0] = testSynonymDefinitions[:2] // 2 docs - // segData[1] = make([]testSynonymDefinition, 0) - // segData[1] = testSynonymDefinitions[2:4] // 2 docs + segData[1] = make([]testSynonymDefinition, 0) + segData[1] = testSynonymDefinitions[2:4] // 2 docs - // segData[2] = make([]testSynonymDefinition, 0) - // segData[2] = testSynonymDefinitions[4:] // 1 doc + segData[2] = make([]testSynonymDefinition, 0) + segData[2] = testSynonymDefinitions[4:] // 1 doc - // segs := make([]segment.Segment, numSegs) - // for i, data := range segData { - // seg, err := buildSegment(data) - // if err != nil { - // t.Fatalf("error building segment: %v", err) - // } - // segs[i] = seg - // } - // drops := make([]*roaring.Bitmap, numDocs) - // for i := 0; i < numDocs; i++ { - // drops[i] = roaring.New() - // } - // err = mergeSegments(segs, drops, testSynonymDefinitions) - // if err != nil { - // t.Fatalf("error merging segments: %v", err) - // } - // for _, seg := range segs { - // cerr := seg.Close() - // if cerr != nil { - // t.Fatalf("error closing seg: %v", err) - // } - // } + segs := make([]segment.Segment, numSegs) + for i, data := range segData { + seg, err := buildSegment(data) + if err != nil { + t.Fatalf("error building segment: %v", err) + } + segs[i] = seg + } + drops := make([]*roaring.Bitmap, numDocs) + for i := 0; i < numDocs; i++ { + drops[i] = roaring.New() + } + err = mergeSegments(segs, drops, testSynonymDefinitions) + if err != nil { + t.Fatalf("error merging segments: %v", err) + } + for _, seg := range segs { + cerr := seg.Close() + if cerr != nil { + t.Fatalf("error closing seg: %v", err) + } + } } From 877152d5fdb594f62f0bd191d69e76a055873c9e Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Fri, 15 Nov 2024 15:54:42 +0530 Subject: [PATCH 09/28] refactor tests --- doc_test.go | 63 ++++++++++++++++++++++++++++++++++------------ section_synonym.go | 4 +-- thesaurus_test.go | 52 ++++++++++++++++++-------------------- 3 files changed, 73 insertions(+), 46 deletions(-) diff --git a/doc_test.go b/doc_test.go index ffc8da93..3a4ebd14 100644 --- a/doc_test.go +++ b/doc_test.go @@ -179,7 +179,11 @@ func (s *stubField) Compose(field string, length int, freq index.TokenFrequencie // ----------------------------------------------------------------------------- type stubSynonymField struct { - name string + name string + analyzer string + input []string + synonyms []string + synonymMap map[string][]string } @@ -200,7 +204,18 @@ func (s *stubSynonymField) EncodedFieldType() byte { } func (s *stubSynonymField) Analyze() { - + var analyzedInput []string + if len(s.input) > 0 { + analyzedInput = make([]string, 0, len(s.input)) + for _, term := range s.input { + analyzedInput = append(analyzedInput, analyzeStubTerm(term, s.analyzer)) + } + } + analyzedSynonyms := make([]string, 0, len(s.synonyms)) + for _, syn := range s.synonyms { + analyzedSynonyms = append(analyzedSynonyms, analyzeStubTerm(syn, s.analyzer)) + } + s.synonymMap = processSynonymData(analyzedInput, analyzedSynonyms) } func (s *stubSynonymField) Options() index.FieldIndexingOptions { @@ -219,30 +234,46 @@ func (s *stubSynonymField) NumPlainTextBytes() uint64 { return 0 } -func (sf *stubSynonymField) VisitSynonymDefinitions(visitor func(term string, synonyms []string)) { +func (sf *stubSynonymField) IterateSynonyms(visitor func(term string, synonyms []string)) { for term, synonyms := range sf.synonymMap { visitor(term, synonyms) } } -func analyzeStubTerm(term string) string { +func processSynonymData(input []string, synonyms []string) map[string][]string { + var synonymMap map[string][]string + if len(input) > 0 { + // Map each term to the same list of synonyms. + synonymMap = make(map[string][]string, len(input)) + for _, term := range input { + synonymMap[term] = append([]string(nil), synonyms...) // Avoid sharing slices. + } + } else { + synonymMap = make(map[string][]string, len(synonyms)) + // Precompute a map where each synonym points to all other synonyms. + for i, elem := range synonyms { + synonymMap[elem] = make([]string, 0, len(synonyms)-1) + for j, otherElem := range synonyms { + if i != j { + synonymMap[elem] = append(synonymMap[elem], otherElem) + } + } + } + } + return synonymMap +} + +func analyzeStubTerm(term string, analyzer string) string { lowerCaseTerm := strings.ToLower(term) return lowerCaseTerm } -func newStubSynonymField(name, analyzer string, synonymMap map[string][]string) index.SynonymField { - analyzedSynonymDefs := make(map[string][]string, len(synonymMap)) - for term, synonyms := range synonymMap { - analyzedTerm := analyzeStubTerm(term) - analyzedSynonyms := make([]string, 0, len(synonyms)) - for _, syn := range synonyms { - analyzedSynonyms = append(analyzedSynonyms, analyzeStubTerm(syn)) - } - analyzedSynonymDefs[analyzedTerm] = analyzedSynonyms - } +func newStubSynonymField(name string, analyzer string, input []string, synonyms []string) index.SynonymField { return &stubSynonymField{ - name: name, - synonymMap: analyzedSynonymDefs, + name: name, + analyzer: analyzer, + input: input, + synonyms: synonyms, } } diff --git a/section_synonym.go b/section_synonym.go index 6828028a..ef237bce 100644 --- a/section_synonym.go +++ b/section_synonym.go @@ -143,7 +143,7 @@ func (so *synonymIndexOpaque) process(field index.SynonymField, fieldID uint16, termSynMap := so.SynonymTermToID[tid] - field.VisitSynonymDefinitions(func(term string, synonyms []string) { + field.IterateSynonyms(func(term string, synonyms []string) { pid := thesaurus[term] - 1 bs := so.Synonyms[pid] @@ -183,7 +183,7 @@ func (so *synonymIndexOpaque) realloc() { termSynMap := so.SynonymTermToID[thesaurusID] - synField.VisitSynonymDefinitions(func(term string, synonyms []string) { + synField.IterateSynonyms(func(term string, synonyms []string) { _, exists := thesaurus[term] if !exists { pidNext++ diff --git a/thesaurus_test.go b/thesaurus_test.go index a545c2ff..f4e6b992 100644 --- a/thesaurus_test.go +++ b/thesaurus_test.go @@ -27,33 +27,15 @@ import ( segment "github.com/blevesearch/scorch_segment_api/v2" ) -func createEquivalentSynonymMap(input []string, resultMap map[string][]string) map[string][]string { - if resultMap == nil { - resultMap = make(map[string][]string) - } - for _, elem := range input { - for _, otherElem := range input { - if elem != otherElem { - resultMap[elem] = append(resultMap[elem], otherElem) - } - } - } - return resultMap -} - -func buildTestSynonymDocument(id string, collection string, terms []string, synonyms []string) index.Document { - var synonymMap map[string][]string - if terms == nil { - numEntries := len(synonyms) * (len(synonyms) - 1) - synonymMap = make(map[string][]string, numEntries) - synonymMap = createEquivalentSynonymMap(synonyms, synonymMap) - } else { - synonymMap = make(map[string][]string, len(terms)) - for _, term := range terms { - synonymMap[term] = synonyms - } - } - synDoc := newStubSynonymDocument(id, newStubSynonymField(collection, "standard", synonymMap)) +func buildTestSynonymDocument(id string, synonymSource string, terms []string, synonyms []string) index.Document { + // Create the synonym document using stubs. + stubAnalyzer := "standard" + // Create the synonym field. + synField := newStubSynonymField(synonymSource, stubAnalyzer, terms, synonyms) + // Analyze the synonyms. + synField.Analyze() + // Create the synonym document. + synDoc := newStubSynonymDocument(id, synField) synDoc.AddIDField() return synDoc } @@ -122,7 +104,7 @@ func checkWithDeletes(except *roaring.Bitmap, collectionName string, testSynonym if len(synonyms) != len(expectedSynonyms) { return errors.New("unexpected number of synonyms, expected: " + strconv.Itoa(len(expectedSynonyms)) + " got: " + - strconv.Itoa(len(synonyms)) + " for term: " + term + " when excepting: " + except.String()) + strconv.Itoa(len(synonyms)) + " for term: " + term) } sort.Strings(synonyms) sort.Strings(expectedSynonyms) @@ -167,6 +149,20 @@ type testSynonymDefinition struct { synonyms []string } +func createEquivalentSynonymMap(input []string, resultMap map[string][]string) map[string][]string { + if resultMap == nil { + resultMap = make(map[string][]string) + } + for _, elem := range input { + for _, otherElem := range input { + if elem != otherElem { + resultMap[elem] = append(resultMap[elem], otherElem) + } + } + } + return resultMap +} + func createExpectedSynonymMap(input []testSynonymDefinition) map[string][]string { rv := make(map[string][]string) for _, testSynonymDefinition := range input { From 8bbd3234578d06ce138148a2a41c05972a09b2aa Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Mon, 9 Dec 2024 18:41:12 +0530 Subject: [PATCH 10/28] bug fixes --- section_synonym.go | 2 +- synonym_posting.go | 10 +++------- thesaurus.go | 45 +++++++++++++++++++++++++++++++++++++++++++++ 3 files changed, 49 insertions(+), 8 deletions(-) diff --git a/section_synonym.go b/section_synonym.go index ef237bce..cf52bc6e 100644 --- a/section_synonym.go +++ b/section_synonym.go @@ -719,7 +719,7 @@ func mergeSynonyms(synItr *SynonymsIterator, newDocNums []uint64, newRoaring *ro synTermMap map[uint32]string, termSynMap map[string]uint32, newSynonymID uint32) (uint32, error) { next, err := synItr.Next() for next != nil && err == nil { - synNewDocNum := newDocNums[next.DocNum()] + synNewDocNum := newDocNums[next.Number()] if synNewDocNum == docDropped { return 0, fmt.Errorf("see hit with dropped docNum") } diff --git a/synonym_posting.go b/synonym_posting.go index c965c16d..9bec7761 100644 --- a/synonym_posting.go +++ b/synonym_posting.go @@ -35,7 +35,7 @@ func init() { var si SynonymsIterator reflectStaticSizeSynonymsIterator = int(reflect.TypeOf(si).Size()) var s Synonym - reflectStaticSizePosting = int(reflect.TypeOf(s).Size()) + reflectStaticSizeSynonym = int(reflect.TypeOf(s).Size()) } type SynonymsList struct { @@ -205,7 +205,7 @@ type Synonym struct { } func (p *Synonym) Size() int { - sizeInBytes := reflectStaticSizePosting + SizeOfPtr + + sizeInBytes := reflectStaticSizeSynonym + SizeOfPtr + len(p.term) return sizeInBytes @@ -215,11 +215,7 @@ func (s *Synonym) Term() string { return s.term } -func (s *Synonym) SynonymID() uint32 { - return s.synID -} - -func (s *Synonym) DocNum() uint32 { +func (s *Synonym) Number() uint32 { return s.docNum } diff --git a/thesaurus.go b/thesaurus.go index 28515f9d..e488ae6c 100644 --- a/thesaurus.go +++ b/thesaurus.go @@ -19,6 +19,7 @@ import ( "fmt" "github.com/RoaringBitmap/roaring" + index "github.com/blevesearch/bleve_index_api" segment "github.com/blevesearch/scorch_segment_api/v2" "github.com/blevesearch/vellum" ) @@ -112,3 +113,47 @@ func (t *Thesaurus) Contains(key []byte) (bool, error) { } return false, nil } + +// AutomatonIterator returns an iterator which only visits terms +// having the the vellum automaton and start/end key range +func (t *Thesaurus) AutomatonIterator(a segment.Automaton, + startKeyInclusive, endKeyExclusive []byte) segment.ThesaurusIterator { + if t.fst != nil { + rv := &ThesaurusIterator{ + t: t, + } + + itr, err := t.fst.Search(a, startKeyInclusive, endKeyExclusive) + if err == nil { + rv.itr = itr + } else if err != vellum.ErrIteratorDone { + rv.err = err + } + + return rv + } + return emptyThesaurusIterator +} + +var emptyThesaurusIterator = &ThesaurusIterator{} + +// ThesaurusIterator is an iterator for term dictionary +type ThesaurusIterator struct { + t *Thesaurus + itr vellum.Iterator + err error + entry index.ThesaurusEntry +} + +// Next returns the next entry in the dictionary +func (i *ThesaurusIterator) Next() (*index.ThesaurusEntry, error) { + if i.err != nil && i.err != vellum.ErrIteratorDone { + return nil, i.err + } else if i.itr == nil || i.err == vellum.ErrIteratorDone { + return nil, nil + } + term, _ := i.itr.Current() + i.entry.Term = string(term) + i.err = i.itr.Next() + return &i.entry, nil +} From 2f8298c74e1233e84891a5a09bc91cf81307449b Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Tue, 10 Dec 2024 18:36:13 +0530 Subject: [PATCH 11/28] fix interface change bug --- thesaurus_test.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/thesaurus_test.go b/thesaurus_test.go index f4e6b992..1ae2d559 100644 --- a/thesaurus_test.go +++ b/thesaurus_test.go @@ -85,7 +85,7 @@ func checkWithDeletes(except *roaring.Bitmap, collectionName string, testSynonym return errors.New("expected empty postings list") } } - synSeg, ok := seg.(segment.SynonymSegment) + synSeg, ok := seg.(segment.ThesaurusSegment) if !ok { return errors.New("expected synonym segment") } From 5e4016743490b26faf8c8db7e2c31e7e31fceb52 Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Wed, 11 Dec 2024 18:48:03 +0530 Subject: [PATCH 12/28] minor cleanup --- doc_test.go | 2 -- section_inverted_text_index.go | 4 ++-- segment.go | 6 +----- synonym_cache.go | 3 +-- synonym_posting.go | 2 -- 5 files changed, 4 insertions(+), 13 deletions(-) diff --git a/doc_test.go b/doc_test.go index 3a4ebd14..476a4672 100644 --- a/doc_test.go +++ b/doc_test.go @@ -330,5 +330,3 @@ func newStubSynonymDocument(id string, synonymField index.SynonymField) index.Sy } return rv } - -// ----------------------------------------------------------------------------- diff --git a/section_inverted_text_index.go b/section_inverted_text_index.go index 86a15f6c..0cbf3357 100644 --- a/section_inverted_text_index.go +++ b/section_inverted_text_index.go @@ -450,8 +450,6 @@ func (io *invertedIndexOpaque) writeDicts(w *CountHashWriter) (dictOffsets []uin } for fieldID, terms := range io.DictKeys { - dict := io.Dicts[fieldID] - if cap(docTermMap) < len(io.results) { docTermMap = make([][]byte, len(io.results)) } else { @@ -461,6 +459,8 @@ func (io *invertedIndexOpaque) writeDicts(w *CountHashWriter) (dictOffsets []uin } } + dict := io.Dicts[fieldID] + for _, term := range terms { // terms are already sorted pid := dict[term] - 1 diff --git a/segment.go b/segment.go index 971418d7..5bf0abc0 100644 --- a/segment.go +++ b/segment.go @@ -478,7 +478,7 @@ func (sb *SegmentBase) dictionary(field string) (rv *Dictionary, err error) { return rv, nil } -// Thesaurus returns the term thesaurus for the specified field +// Thesaurus returns the thesaurus with the specified name, or an empty thesaurus if not found. func (s *SegmentBase) Thesaurus(name string) (segment.Thesaurus, error) { thesaurus, err := s.thesaurus(name) if err == nil && thesaurus == nil { @@ -499,14 +499,10 @@ func (sb *SegmentBase) thesaurus(name string) (rv *Thesaurus, err error) { name: name, fieldID: fieldIDPlus1 - 1, } - // the below loop loads the following: - // 1. doc values(first 2 iterations) - adhering to the sections format. never - // valid values for synonym section. for i := 0; i < 2; i++ { _, n := binary.Uvarint(sb.mem[thesaurusStart : thesaurusStart+binary.MaxVarintLen64]) thesaurusStart += uint64(n) } - // 2. thesaurus location - valid value for synonym section. thesLoc, n := binary.Uvarint(sb.mem[thesaurusStart : thesaurusStart+binary.MaxVarintLen64]) thesaurusStart += uint64(n) fst, synTermMap, err := sb.synIndexCache.loadOrCreate(rv.fieldID, sb.mem[thesLoc:]) diff --git a/synonym_cache.go b/synonym_cache.go index 060cd875..acf39906 100644 --- a/synonym_cache.go +++ b/synonym_cache.go @@ -65,7 +65,6 @@ func (sc *synonymIndexCache) loadOrCreate(fieldID uint16, mem []byte) (*vellum.F func (sc *synonymIndexCache) createAndCacheLOCKED(fieldID uint16, mem []byte) (*vellum.FST, map[uint32][]byte, error) { var pos uint64 - // read the length of the vellum data vellumLen, read := binary.Uvarint(mem[pos : pos+binary.MaxVarintLen64]) if vellumLen == 0 || read <= 0 { return nil, nil, fmt.Errorf("vellum length is 0") @@ -77,12 +76,12 @@ func (sc *synonymIndexCache) createAndCacheLOCKED(fieldID uint16, mem []byte) (* return nil, nil, fmt.Errorf("vellum err: %v", err) } pos += vellumLen - synTermMap := make(map[uint32][]byte) numSyns, n := binary.Uvarint(mem[pos : pos+binary.MaxVarintLen64]) pos += uint64(n) if numSyns == 0 { return nil, nil, fmt.Errorf("no synonyms found") } + synTermMap := make(map[uint32][]byte, numSyns) for i := 0; i < int(numSyns); i++ { synID, n := binary.Uvarint(mem[pos : pos+binary.MaxVarintLen64]) pos += uint64(n) diff --git a/synonym_posting.go b/synonym_posting.go index 9bec7761..db3ab52c 100644 --- a/synonym_posting.go +++ b/synonym_posting.go @@ -90,9 +90,7 @@ func (s *SynonymsList) iterator(rv *SynonymsIterator) *SynonymsIterator { rv.except = s.except rv.Actual = s.synonyms.Iterator() rv.ActualBM = s.synonyms - rv.synIDTermMap = s.synIDTermMap - return rv } From ab86d466181b537f320d91eef436455eab47f781 Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Wed, 11 Dec 2024 18:51:48 +0530 Subject: [PATCH 13/28] clean --- section_synonym.go | 2 -- 1 file changed, 2 deletions(-) diff --git a/section_synonym.go b/section_synonym.go index cf52bc6e..93d8ce2b 100644 --- a/section_synonym.go +++ b/section_synonym.go @@ -739,5 +739,3 @@ func mergeSynonyms(synItr *SynonymsIterator, newDocNums []uint64, newRoaring *ro } return newSynonymID, nil } - -// ----------------------------------------------------------------------------- From 508221aa42eafcdd03343ab8b79395f0badd4074 Mon Sep 17 00:00:00 2001 From: Abhinav Dangeti Date: Wed, 11 Dec 2024 09:39:36 -0700 Subject: [PATCH 14/28] Upgrade vellum, bleve_index_api, scorch_segment_api --- go.mod | 6 +++--- go.sum | 12 ++++++------ 2 files changed, 9 insertions(+), 9 deletions(-) diff --git a/go.mod b/go.mod index 13b2e2e9..2d67daf3 100644 --- a/go.mod +++ b/go.mod @@ -4,11 +4,11 @@ go 1.21 require ( github.com/RoaringBitmap/roaring v1.9.3 - github.com/blevesearch/bleve_index_api v1.1.13 + github.com/blevesearch/bleve_index_api v1.2.0 github.com/blevesearch/go-faiss v1.0.23 github.com/blevesearch/mmap-go v1.0.4 - github.com/blevesearch/scorch_segment_api/v2 v2.2.16 - github.com/blevesearch/vellum v1.0.11 + github.com/blevesearch/scorch_segment_api/v2 v2.3.0 + github.com/blevesearch/vellum v1.1.0 github.com/golang/snappy v0.0.1 github.com/spf13/cobra v1.7.0 ) diff --git a/go.sum b/go.sum index d15a1d18..44e4a5f7 100644 --- a/go.sum +++ b/go.sum @@ -2,16 +2,16 @@ github.com/RoaringBitmap/roaring v1.9.3 h1:t4EbC5qQwnisr5PrP9nt0IRhRTb9gMUgQF4t4 github.com/RoaringBitmap/roaring v1.9.3/go.mod h1:6AXUsoIEzDTFFQCe1RbGA6uFONMhvejWj5rqITANK90= github.com/bits-and-blooms/bitset v1.12.0 h1:U/q1fAF7xXRhFCrhROzIfffYnu+dlS38vCZtmFVPHmA= github.com/bits-and-blooms/bitset v1.12.0/go.mod h1:7hO7Gc7Pp1vODcmWvKMRA9BNmbv6a/7QIWpPxHddWR8= -github.com/blevesearch/bleve_index_api v1.1.13 h1:+nrA6oRJr85aCPyqaeZtsruObwKojutfonHJin/BP48= -github.com/blevesearch/bleve_index_api v1.1.13/go.mod h1:PbcwjIcRmjhGbkS/lJCpfgVSMROV6TRubGGAODaK1W8= +github.com/blevesearch/bleve_index_api v1.2.0 h1:/DXMMWBwx/UmGKM1xDhTwDoJI5yQrG6rqRWPFcOgUVo= +github.com/blevesearch/bleve_index_api v1.2.0/go.mod h1:PbcwjIcRmjhGbkS/lJCpfgVSMROV6TRubGGAODaK1W8= github.com/blevesearch/go-faiss v1.0.23 h1:Wmc5AFwDLKGl2L6mjLX1Da3vCL0EKa2uHHSorcIS1Uc= github.com/blevesearch/go-faiss v1.0.23/go.mod h1:OMGQwOaRRYxrmeNdMrXJPvVx8gBnvE5RYrr0BahNnkk= github.com/blevesearch/mmap-go v1.0.4 h1:OVhDhT5B/M1HNPpYPBKIEJaD0F3Si+CrEKULGCDPWmc= github.com/blevesearch/mmap-go v1.0.4/go.mod h1:EWmEAOmdAS9z/pi/+Toxu99DnsbhG1TIxUoRmJw/pSs= -github.com/blevesearch/scorch_segment_api/v2 v2.2.16 h1:uGvKVvG7zvSxCwcm4/ehBa9cCEuZVE+/zvrSl57QUVY= -github.com/blevesearch/scorch_segment_api/v2 v2.2.16/go.mod h1:VF5oHVbIFTu+znY1v30GjSpT5+9YFs9dV2hjvuh34F0= -github.com/blevesearch/vellum v1.0.11 h1:SJI97toEFTtA9WsDZxkyGTaBWFdWl1n2LEDCXLCq/AU= -github.com/blevesearch/vellum v1.0.11/go.mod h1:QgwWryE8ThtNPxtgWJof5ndPfx0/YMBh+W2weHKPw8Y= +github.com/blevesearch/scorch_segment_api/v2 v2.3.0 h1:vxCjbXAkkEBSb4AB3Iqgr/EJcPyYRsiGxpcvsS8E1Dw= +github.com/blevesearch/scorch_segment_api/v2 v2.3.0/go.mod h1:5y+TgXYSx+xJGaCwSlvy9G/UJBIY5wzvIkhvhBm2ATc= +github.com/blevesearch/vellum v1.1.0 h1:CinkGyIsgVlYf8Y2LUQHvdelgXr6PYuvoDIajq6yR9w= +github.com/blevesearch/vellum v1.1.0/go.mod h1:QgwWryE8ThtNPxtgWJof5ndPfx0/YMBh+W2weHKPw8Y= github.com/cpuguy83/go-md2man/v2 v2.0.2/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o= github.com/davecgh/go-spew v1.1.0/go.mod h1:J7Y8YcW2NihsgmVo/mv3lAwl/skON4iLHjSsI+c5H38= github.com/davecgh/go-spew v1.1.1 h1:vj9j/u1bqnvCEfJOwUhtlOARqs3+rkHYY13jYWTU97c= From 042af033960dd151c9187ad075b4dcfffa11e326 Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Thu, 12 Dec 2024 18:34:25 +0530 Subject: [PATCH 15/28] command line tooling --- cmd/zap/cmd/synonym.go | 140 +++++++++++++++++++++++++++++++++++++++++ segment.go | 21 ++++++- thesaurus_test.go | 70 ++++++++++++--------- 3 files changed, 202 insertions(+), 29 deletions(-) create mode 100644 cmd/zap/cmd/synonym.go diff --git a/cmd/zap/cmd/synonym.go b/cmd/zap/cmd/synonym.go new file mode 100644 index 00000000..ebc7fe89 --- /dev/null +++ b/cmd/zap/cmd/synonym.go @@ -0,0 +1,140 @@ +// Copyright (c) 2024 Couchbase, 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 cmd + +import ( + "bytes" + "encoding/binary" + "fmt" + + "github.com/RoaringBitmap/roaring/roaring64" + "github.com/blevesearch/vellum" + "github.com/spf13/cobra" +) + +var thesCmd = &cobra.Command{ + Use: "thes [path] [name]", + Short: "thes prints the thesaurus with the specified name", + Long: `The thes command lets you print the thesaurus with the specified name.`, + RunE: func(cmd *cobra.Command, args []string) error { + pos := segment.FieldsIndexOffset() + if pos == 0 { + // this is the case only for older file formats + return fmt.Errorf("file format not supported") + } + if len(args) < 2 { + return fmt.Errorf("must specify thesaurus name") + } + + pos, err := segment.ThesAddr(args[1]) + if err != nil { + return fmt.Errorf("error determining address: %v", err) + } + fmt.Printf("thesaurus with name starts at %d (%x)\n", pos, pos) + + data := segment.Data() + vellumLen, read := binary.Uvarint(data[pos : pos+binary.MaxVarintLen64]) + pos += uint64(read) + fmt.Printf("vellum length: %d\n", vellumLen) + + fstBytes := data[pos : pos+vellumLen] + pos += vellumLen + fst, err := vellum.Load(fstBytes) + if err != nil { + return fmt.Errorf("thesaurus name %s vellum err: %v", args[1], err) + } + fmt.Printf("raw vellum data:\n % x\n", fstBytes) + + numSyns, n := binary.Uvarint(data[pos : pos+binary.MaxVarintLen64]) + pos += uint64(n) + if numSyns == 0 { + return fmt.Errorf("no synonyms found") + } + synTermMap := make(map[uint32][]byte, numSyns) + for i := 0; i < int(numSyns); i++ { + synID, n := binary.Uvarint(data[pos : pos+binary.MaxVarintLen64]) + pos += uint64(n) + termLen, n := binary.Uvarint(data[pos : pos+binary.MaxVarintLen64]) + pos += uint64(n) + if termLen == 0 { + return fmt.Errorf("term length is 0") + } + term := data[pos : pos+uint64(termLen)] + pos += uint64(termLen) + synTermMap[uint32(synID)] = term + } + + fmt.Printf("termID to term mapping:\n") + fmt.Printf(" termID\tterm\n") + for k, v := range synTermMap { + fmt.Printf(" %d\t%s\n", k, string(v)) + } + fmt.Printf("thesaurus (term -> [{termID|docNum},...]):\n") + var totalTerms int + itr, err := fst.Iterator(nil, nil) + for err == nil { + var sl *roaring64.Bitmap + currTerm, currVal := itr.Current() + sl, err = readSynonymsList(currVal, data) + if err != nil { + return err + } + sitr := sl.Iterator() + printStr := fmt.Sprintf(" %s -> [", currTerm) + for sitr.HasNext() { + encodedVal := sitr.Next() + tID, docNum := decodeSynonym(encodedVal) + str := fmt.Sprintf("{%d|%d},", tID, docNum) + printStr += str + } + printStr = printStr[:len(printStr)-1] + "]" + fmt.Printf("%s\n", printStr) + totalTerms++ + err = itr.Next() + } + fmt.Printf("Total terms in thesaurus : %d\n", totalTerms) + if err != nil && err != vellum.ErrIteratorDone { + return fmt.Errorf("error iterating thesaurus: %v", err) + } + return nil + }, +} + +func readSynonymsList(postingsOffset uint64, data []byte) (*roaring64.Bitmap, error) { + var n uint64 + var read int + + var postingsLen uint64 + postingsLen, read = binary.Uvarint(data[postingsOffset : postingsOffset+binary.MaxVarintLen64]) + n += uint64(read) + + buf := bytes.NewReader(data[postingsOffset+n : postingsOffset+n+postingsLen]) + r := roaring64.NewBitmap() + + _, err := r.ReadFrom(buf) + if err != nil { + return nil, fmt.Errorf("error loading roaring bitmap: %v", err) + } + + return r, nil +} + +func decodeSynonym(synonymCode uint64) (synonymID uint32, docID uint32) { + return uint32(synonymCode >> 32), uint32(synonymCode) +} + +func init() { + RootCmd.AddCommand(thesCmd) +} diff --git a/segment.go b/segment.go index 5bf0abc0..13079c29 100644 --- a/segment.go +++ b/segment.go @@ -695,7 +695,7 @@ func (s *Segment) Close() (err error) { } func (s *Segment) closeActual() (err error) { - // clear contents from the vector index cache before un-mmapping + // clear contents from the vector and synonym index cache before un-mmapping s.vecIndexCache.Clear() s.synIndexCache.Clear() @@ -767,6 +767,25 @@ func (s *Segment) DictAddr(field string) (uint64, error) { return s.dictLocs[fieldIDPlus1-1], nil } +// ThesAddr is a helper function to compute the file offset where the +// thesaurus is stored with the specified name. +func (s *Segment) ThesAddr(name string) (uint64, error) { + fieldIDPlus1, ok := s.fieldsMap[name] + if !ok { + return 0, fmt.Errorf("no such thesaurus '%s'", name) + } + thesaurusStart := s.fieldsSectionsMap[fieldIDPlus1-1][SectionSynonymIndex] + if thesaurusStart == 0 { + return 0, fmt.Errorf("no such thesaurus '%s'", name) + } + for i := 0; i < 2; i++ { + _, n := binary.Uvarint(s.mem[thesaurusStart : thesaurusStart+binary.MaxVarintLen64]) + thesaurusStart += uint64(n) + } + thesLoc, _ := binary.Uvarint(s.mem[thesaurusStart : thesaurusStart+binary.MaxVarintLen64]) + return thesLoc, nil +} + func (s *Segment) getSectionDvOffsets(fieldID int, secID uint16) (uint64, uint64, uint64, error) { // Version is gonna be 16 var fieldLocStart uint64 = fieldNotUninverted diff --git a/thesaurus_test.go b/thesaurus_test.go index 1ae2d559..47dc1339 100644 --- a/thesaurus_test.go +++ b/thesaurus_test.go @@ -177,7 +177,7 @@ func createExpectedSynonymMap(input []testSynonymDefinition) map[string][]string return rv } -func buildSegment(testSynonymDefinitions []testSynonymDefinition) (segment.Segment, error) { +func buildSegment(testSynonymDefinitions []testSynonymDefinition) (segment.Segment, string, error) { var testSynonymDocuments []index.Document for i, testSynonymDefinition := range testSynonymDefinitions { testSynonymDocuments = append(testSynonymDocuments, buildTestSynonymDocument( @@ -189,68 +189,68 @@ func buildSegment(testSynonymDefinitions []testSynonymDefinition) (segment.Segme } sb, err := buildTestSegmentForThesaurus(testSynonymDocuments) if err != nil { - return nil, err + return nil, "", err } tmpDir, err := os.MkdirTemp("", "zap-") if err != nil { - return nil, err + return nil, "", err } err = os.RemoveAll(tmpDir) if err != nil { - return nil, err + return nil, "", err } err = PersistSegmentBase(sb, tmpDir) if err != nil { - return nil, err + return nil, "", err } seg, err := zapPlugin.Open(tmpDir) if err != nil { - return nil, err + return nil, "", err } err = testSegmentSynonymAccuracy(testSynonymDefinitions, seg) if err != nil { - return nil, err + return nil, "", err } - return seg, nil + return seg, tmpDir, nil } -func mergeSegments(segs []segment.Segment, drops []*roaring.Bitmap, testSynonymDefinitions []testSynonymDefinition) error { +func mergeSegments(segs []segment.Segment, drops []*roaring.Bitmap, testSynonymDefinitions []testSynonymDefinition) (string, error) { tmpDir, err := os.MkdirTemp("", "mergedzap-") if err != nil { - return err + return "", err } err = os.RemoveAll(tmpDir) if err != nil { - return err + return "", err } // Test Merging of multiple segments _, _, err = zapPlugin.Merge(segs, drops, tmpDir, nil, nil) if err != nil { - return err + return "", err } seg, err := zapPlugin.Open(tmpDir) if err != nil { - return err + return "", err } err = testSegmentSynonymAccuracy(testSynonymDefinitions, seg) if err != nil { - return err + return "", err } cerr := seg.Close() if cerr != nil { - return err + return "", err } - return nil + return tmpDir, nil } func TestSynonymSegment(t *testing.T) { - firstCollectionName := "coll0" - secondCollectionName := "coll1" + synonymSourceOneName := "coll0" + synonymSourceTwoName := "coll1" testSynonymDefinitions := []testSynonymDefinition{ { - collectionName: firstCollectionName, + collectionName: synonymSourceOneName, terms: nil, synonyms: []string{ "adeptness", @@ -265,7 +265,7 @@ func TestSynonymSegment(t *testing.T) { }, }, { - collectionName: firstCollectionName, + collectionName: synonymSourceOneName, terms: []string{"afflict"}, synonyms: []string{ "affect", @@ -277,7 +277,7 @@ func TestSynonymSegment(t *testing.T) { }, }, { - collectionName: firstCollectionName, + collectionName: synonymSourceOneName, terms: []string{"capacity"}, synonyms: []string{ "volume", @@ -288,7 +288,7 @@ func TestSynonymSegment(t *testing.T) { }, }, { - collectionName: secondCollectionName, + collectionName: synonymSourceTwoName, synonyms: []string{ "absolutely", "unqualifiedly", @@ -299,7 +299,7 @@ func TestSynonymSegment(t *testing.T) { }, }, { - collectionName: secondCollectionName, + collectionName: synonymSourceTwoName, terms: []string{"abrupt"}, synonyms: []string{ "sudden", @@ -311,7 +311,7 @@ func TestSynonymSegment(t *testing.T) { }, } // single segment test - seg1, err := buildSegment(testSynonymDefinitions) + seg1, dir, err := buildSegment(testSynonymDefinitions) if err != nil { t.Fatalf("error building segment: %v", err) } @@ -320,6 +320,10 @@ func TestSynonymSegment(t *testing.T) { if cerr != nil { t.Fatalf("error closing seg: %v", err) } + err := os.RemoveAll(dir) + if err != nil { + t.Fatalf("error removing dir: %v", err) + } }() // // multiple segment test @@ -337,25 +341,35 @@ func TestSynonymSegment(t *testing.T) { segData[2] = testSynonymDefinitions[4:] // 1 doc segs := make([]segment.Segment, numSegs) + dirs := make([]string, numSegs) for i, data := range segData { - seg, err := buildSegment(data) + seg, dir, err := buildSegment(data) if err != nil { t.Fatalf("error building segment: %v", err) } segs[i] = seg + dirs[i] = dir } drops := make([]*roaring.Bitmap, numDocs) for i := 0; i < numDocs; i++ { drops[i] = roaring.New() } - err = mergeSegments(segs, drops, testSynonymDefinitions) + mergeDir, err := mergeSegments(segs, drops, testSynonymDefinitions) if err != nil { t.Fatalf("error merging segments: %v", err) } - for _, seg := range segs { - cerr := seg.Close() + for i := 0; i < numSegs; i++ { + cerr := segs[i].Close() if cerr != nil { t.Fatalf("error closing seg: %v", err) } + err := os.RemoveAll(dirs[i]) + if err != nil { + t.Fatalf("error removing dir: %v", err) + } + err = os.RemoveAll(mergeDir) + if err != nil { + t.Fatalf("error removing dir: %v", err) + } } } From 801ea518097b137ab19ff2a9f2b4f456d8d56da2 Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Thu, 12 Dec 2024 20:38:43 +0530 Subject: [PATCH 16/28] update zap.md --- zap.md | 85 ++++++++++++++++++++++++++++++++++++++++++++++------------ 1 file changed, 68 insertions(+), 17 deletions(-) diff --git a/zap.md b/zap.md index 675ac56c..6a7a0655 100644 --- a/zap.md +++ b/zap.md @@ -28,9 +28,9 @@ ### Chunked data - [--------] - [ ] - [--------] + [--------] + [ ] + [--------] ## Overview @@ -65,7 +65,6 @@ Footer section describes the configuration of particular ZAP file. The format of V. Version. CC. CRC32. - ## Stored Fields Stored Fields Index is `D#` consecutive 64-bit unsigned integers - offsets, where relevant Stored Fields Data records are located. @@ -114,10 +113,9 @@ Sections Index is a set of NF uint64 addresses (0 through F# - 1) each of which NS. Number of index sections Sn. nth index section - ## Inverted Text Index Section -Each fields has its own types of indexes in separate sections as indicated above. This can be a vector index or inverted text index. +Each field has its own types of indexes in separate sections as indicated above. This can be a vector index or inverted text index. In case of inverted text index, the dictionary is encoded in [Vellum](https://github.com/couchbase/vellum) format. Dictionary consists of pairs `(term, offset)`, where `offset` indicates the position of postings (list of documents) for this particular term. @@ -151,6 +149,8 @@ In case of inverted text index, the dictionary is encoded in [Vellum](https://gi | | | | |================================================================+- Vector Index Section | | | + | +================================================================+- Synonym Index Section + | | | | |================================================================+- Sections Info +-----------------------------+ | | | | @@ -162,22 +162,73 @@ In case of inverted text index, the dictionary is encoded in [Vellum](https://gi ITI - Inverted Text Index +## Synonym Index Section -## Doc Values +In a synonyms index, the relationship between a term and its synonyms is represented using a Thesaurus. The Thesaurus is encoded in the [Vellum](https://github.com/couchbase/vellum) format and consists of pairs in the form `(term, offset)`. Here, the offset specifies the position of the postings list containing the synonyms for the given term. The postings list is stored as a Roaring64 bitmap, with each entry representing an encoded synonym for the term. -DocValue start and end offsets are stored within the section content of each field. This allows each field having its own type of index to choose whether to store the doc values or not. For example, it may not make sense to store doc values for vector indexing and so, the offsets can be invalid ones for it whereas the fields having text indexing may have valid doc values offsets. + |================================================================+- Inverted Text Index Section + | | + |================================================================+- Vector Index Section + | | + +================================================================+- Synonym Index Section + | | + | (Offset) +~~~~~+----------+...+---+ | + | +--------->| RL | ROARING64 BITMAP | | + | | +~~~~~+----------+...+---+ +-------------------+ + | |(Term -> Offset) | + | +--------+ | + | | Term ID to Term map (NST Entries) | + | +~~~~+~~~~+~~~~~[{~~~~~+~~~~+~~~~~~}{~~~~~+~~~~+~~~~~~}...{~~~~~+~~~~+~~~~~~}] | + | +->| VL | VD | NST || TID | TL | Term || TID | TL | Term | | TID | TL | Term | | + | | +~~~~+~~~~+~~~~~[{~~~~~+~~~~+~~~~~~}{~~~~~+~~~~+~~~~~~}...{~~~~~+~~~~+~~~~~~}] | + | | | + | +----------------------------+ | + | | | + | +~~~~~~~~~~+~~~~~~~~+~~~~~~~~~~~~~~~~~+ | + +-----> DV Start | DV End | ThesaurusOffset | | + | | +~~~~~~~~~~+~~~~~~~~+~~~~~~~~~~~~~~~~~+ +------------------+ + | | | + | | | + | |================================================================+- Sections Info + +-----------------------------+ | + | | | + | +-------+-----+-----+------+~~~~~~~~+~~~~~~~~+--+...+--+ | + | | ... | SI | SI ADDR | NS | Length | Name | | + | +-------+-----+------------+~~~~~~~~+~~~~~~~~+--+...+--+ | + +================================================================+ + + SI - Synonym Index + VL - Vellum Length + VD - Vellum Data (TERM -> OFFSET) + RL - Roaring64 Length + NST - Number of entries in the term ID to term map + TID - Term ID + TL - Term Length +### Synonym Encoding - +================================================================+ - | +------...--+ | - | +->+ DocValues +<-+ | - | | +------...--+ | | - |==|=================|===========================================+- Inverted Text - ++~+~~~~~~~~~+~~~~~~~+~~+~~~~~~~~+-----------------------...--+ | Index Section - || DV START | DV END | LENGTH | VELLUM DATA: TERM -> OFFSET| | - ++~~~~~~~~~~~+~~~~~~~~~~+~~~~~~~~+-----------------------...--+ | - +================================================================+ + ROARING64 BITMAP + + [{~~~~~+~~~~~}{~~~~~+~~~~~}...{~~~~~+~~~~~}] + | TID | DN || TID | DN | | TID | DN | + [{~~~~~+~~~~~}{~~~~~+~~~~~}...{~~~~~+~~~~~}] + + TID - Term ID + DN - Document Number + +## Doc Values + +DocValue start and end offsets are stored within the section content of each field. This allows each field having its own type of index to choose whether to store the doc values or not. For example, it may not make sense to store doc values for vector indexing and so, the offsets can be invalid ones for it whereas the fields having text indexing may have valid doc values offsets. + +================================================================+ + | +------...--+ | + | +->+ DocValues +<-+ | + | | +------...--+ | | + |==|=================|===========================================+- Inverted Text + ++~+~~~~~~~~~+~~~~~~~+~~+~~~~~~~~+-----------------------...--+ | Index Section + || DV START | DV END | LENGTH | VELLUM DATA: TERM -> OFFSET| | + ++~~~~~~~~~~~+~~~~~~~~~~+~~~~~~~~+-----------------------...--+ | + +================================================================+ DocValues is chunked Snappy-compressed values for each document and field. From c5634c0cac1efb17eb7b948a1c1788089b0bc06d Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Thu, 12 Dec 2024 20:44:31 +0530 Subject: [PATCH 17/28] small formatting fixes in zap.md --- zap.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/zap.md b/zap.md index 6a7a0655..ddcda5ad 100644 --- a/zap.md +++ b/zap.md @@ -186,7 +186,7 @@ In a synonyms index, the relationship between a term and its synonyms is represe | | | | +~~~~~~~~~~+~~~~~~~~+~~~~~~~~~~~~~~~~~+ | +-----> DV Start | DV End | ThesaurusOffset | | - | | +~~~~~~~~~~+~~~~~~~~+~~~~~~~~~~~~~~~~~+ +------------------+ + | | +~~~~~~~~~~+~~~~~~~~+~~~~~~~~~~~~~~~~~+ +-------------------+ | | | | | | | |================================================================+- Sections Info @@ -199,7 +199,7 @@ In a synonyms index, the relationship between a term and its synonyms is represe SI - Synonym Index VL - Vellum Length - VD - Vellum Data (TERM -> OFFSET) + VD - Vellum Data (Term -> Offset) RL - Roaring64 Length NST - Number of entries in the term ID to term map TID - Term ID From f9896a9d0fde8e7cd87643d281e585184eb3e07d Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Thu, 12 Dec 2024 21:18:26 +0530 Subject: [PATCH 18/28] fix cmd --- cmd/zap/cmd/synonym.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/cmd/zap/cmd/synonym.go b/cmd/zap/cmd/synonym.go index ebc7fe89..a9b9a8d6 100644 --- a/cmd/zap/cmd/synonym.go +++ b/cmd/zap/cmd/synonym.go @@ -42,7 +42,7 @@ var thesCmd = &cobra.Command{ if err != nil { return fmt.Errorf("error determining address: %v", err) } - fmt.Printf("thesaurus with name starts at %d (%x)\n", pos, pos) + fmt.Printf("thesaurus with name %s starts at %d (%x)\n", args[1], pos, pos) data := segment.Data() vellumLen, read := binary.Uvarint(data[pos : pos+binary.MaxVarintLen64]) From 1b5ea945ee7b1624ad6ff65142b9497b5c746892 Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Thu, 12 Dec 2024 22:11:04 +0530 Subject: [PATCH 19/28] thes -> thesaurus --- cmd/zap/cmd/synonym.go | 6 +++--- section_synonym.go | 5 ++--- 2 files changed, 5 insertions(+), 6 deletions(-) diff --git a/cmd/zap/cmd/synonym.go b/cmd/zap/cmd/synonym.go index a9b9a8d6..d6ee14c6 100644 --- a/cmd/zap/cmd/synonym.go +++ b/cmd/zap/cmd/synonym.go @@ -25,9 +25,9 @@ import ( ) var thesCmd = &cobra.Command{ - Use: "thes [path] [name]", - Short: "thes prints the thesaurus with the specified name", - Long: `The thes command lets you print the thesaurus with the specified name.`, + Use: "thesaurus [path] [name]", + Short: "thesaurus prints the thesaurus with the specified name", + Long: `The thesaurus command lets you print the thesaurus with the specified name.`, RunE: func(cmd *cobra.Command, args []string) error { pos := segment.FieldsIndexOffset() if pos == 0 { diff --git a/section_synonym.go b/section_synonym.go index 93d8ce2b..a89a696a 100644 --- a/section_synonym.go +++ b/section_synonym.go @@ -195,10 +195,9 @@ func (so *synonymIndexOpaque) realloc() { for _, syn := range synonyms { _, exists := termSynMap[syn] if !exists { + termSynMap[syn] = sidNext + synTermMap[sidNext] = syn sidNext++ - sidPlus1 := sidNext - termSynMap[syn] = sidPlus1 - synTermMap[sidPlus1] = syn } } }) From 946a3694efd9b0d8b6dbf9687b1ecc442b949de7 Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Thu, 12 Dec 2024 22:22:07 +0530 Subject: [PATCH 20/28] minor zapx.md changes --- zap.md | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/zap.md b/zap.md index ddcda5ad..31868232 100644 --- a/zap.md +++ b/zap.md @@ -202,19 +202,21 @@ In a synonyms index, the relationship between a term and its synonyms is represe VD - Vellum Data (Term -> Offset) RL - Roaring64 Length NST - Number of entries in the term ID to term map - TID - Term ID + TID - Term ID (32-bit) TL - Term Length ### Synonym Encoding ROARING64 BITMAP - + + Each 64-bit entry consists of two parts: the first 32 bits represent the Term ID (TID), and the next 32 bits represent the Document Number (DN). + [{~~~~~+~~~~~}{~~~~~+~~~~~}...{~~~~~+~~~~~}] | TID | DN || TID | DN | | TID | DN | [{~~~~~+~~~~~}{~~~~~+~~~~~}...{~~~~~+~~~~~}] - TID - Term ID - DN - Document Number + TID - Term ID (32-bit) + DN - Document Number (32-bit) ## Doc Values From cdad5b4dec7be6980c5bb6866cc9ba6b3ed552de Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Fri, 13 Dec 2024 12:56:16 +0530 Subject: [PATCH 21/28] address code review --- cmd/zap/cmd/synonym.go | 6 +++--- segment.go | 19 ++++++------------- zap.md | 3 ++- 3 files changed, 11 insertions(+), 17 deletions(-) diff --git a/cmd/zap/cmd/synonym.go b/cmd/zap/cmd/synonym.go index d6ee14c6..fbc579b4 100644 --- a/cmd/zap/cmd/synonym.go +++ b/cmd/zap/cmd/synonym.go @@ -24,7 +24,7 @@ import ( "github.com/spf13/cobra" ) -var thesCmd = &cobra.Command{ +var thesaurusCmd = &cobra.Command{ Use: "thesaurus [path] [name]", Short: "thesaurus prints the thesaurus with the specified name", Long: `The thesaurus command lets you print the thesaurus with the specified name.`, @@ -38,7 +38,7 @@ var thesCmd = &cobra.Command{ return fmt.Errorf("must specify thesaurus name") } - pos, err := segment.ThesAddr(args[1]) + pos, err := segment.ThesaurusAddr(args[1]) if err != nil { return fmt.Errorf("error determining address: %v", err) } @@ -136,5 +136,5 @@ func decodeSynonym(synonymCode uint64) (synonymID uint32, docID uint32) { } func init() { - RootCmd.AddCommand(thesCmd) + RootCmd.AddCommand(thesaurusCmd) } diff --git a/segment.go b/segment.go index 13079c29..3be1a671 100644 --- a/segment.go +++ b/segment.go @@ -479,22 +479,14 @@ func (sb *SegmentBase) dictionary(field string) (rv *Dictionary, err error) { } // Thesaurus returns the thesaurus with the specified name, or an empty thesaurus if not found. -func (s *SegmentBase) Thesaurus(name string) (segment.Thesaurus, error) { - thesaurus, err := s.thesaurus(name) - if err == nil && thesaurus == nil { - return emptyThesaurus, nil - } - return thesaurus, err -} - -func (sb *SegmentBase) thesaurus(name string) (rv *Thesaurus, err error) { +func (sb *SegmentBase) Thesaurus(name string) (segment.Thesaurus, error) { fieldIDPlus1 := sb.fieldsMap[name] if fieldIDPlus1 == 0 { return nil, nil } thesaurusStart := sb.fieldsSectionsMap[fieldIDPlus1-1][SectionSynonymIndex] if thesaurusStart > 0 { - rv = &Thesaurus{ + rv := &Thesaurus{ sb: sb, name: name, fieldID: fieldIDPlus1 - 1, @@ -515,8 +507,9 @@ func (sb *SegmentBase) thesaurus(name string) (rv *Thesaurus, err error) { if err != nil { return nil, fmt.Errorf("thesaurus name %s vellum reader err: %v", name, err) } + return rv, nil } - return rv, nil + return emptyThesaurus, nil } // visitDocumentCtx holds data structures that are reusable across @@ -767,9 +760,9 @@ func (s *Segment) DictAddr(field string) (uint64, error) { return s.dictLocs[fieldIDPlus1-1], nil } -// ThesAddr is a helper function to compute the file offset where the +// ThesaurusAddr is a helper function to compute the file offset where the // thesaurus is stored with the specified name. -func (s *Segment) ThesAddr(name string) (uint64, error) { +func (s *Segment) ThesaurusAddr(name string) (uint64, error) { fieldIDPlus1, ok := s.fieldsMap[name] if !ok { return 0, fmt.Errorf("no such thesaurus '%s'", name) diff --git a/zap.md b/zap.md index 31868232..ef413643 100644 --- a/zap.md +++ b/zap.md @@ -209,7 +209,8 @@ In a synonyms index, the relationship between a term and its synonyms is represe ROARING64 BITMAP - Each 64-bit entry consists of two parts: the first 32 bits represent the Term ID (TID), and the next 32 bits represent the Document Number (DN). + Each 64-bit entry consists of two parts: the first 32 bits represent the Term ID (TID), + and the next 32 bits represent the Document Number (DN). [{~~~~~+~~~~~}{~~~~~+~~~~~}...{~~~~~+~~~~~}] | TID | DN || TID | DN | | TID | DN | From 3609ffc5f71b1d99026454ba60bf563f16cbe6ac Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Fri, 13 Dec 2024 13:03:26 +0530 Subject: [PATCH 22/28] fix --- segment.go | 15 +++++++++++---- zap.md | 6 +++--- 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/segment.go b/segment.go index 3be1a671..ecb46794 100644 --- a/segment.go +++ b/segment.go @@ -479,14 +479,22 @@ func (sb *SegmentBase) dictionary(field string) (rv *Dictionary, err error) { } // Thesaurus returns the thesaurus with the specified name, or an empty thesaurus if not found. -func (sb *SegmentBase) Thesaurus(name string) (segment.Thesaurus, error) { +func (s *SegmentBase) Thesaurus(name string) (segment.Thesaurus, error) { + thesaurus, err := s.thesaurus(name) + if err == nil && thesaurus == nil { + return emptyThesaurus, nil + } + return thesaurus, err +} + +func (sb *SegmentBase) thesaurus(name string) (rv *Thesaurus, err error) { fieldIDPlus1 := sb.fieldsMap[name] if fieldIDPlus1 == 0 { return nil, nil } thesaurusStart := sb.fieldsSectionsMap[fieldIDPlus1-1][SectionSynonymIndex] if thesaurusStart > 0 { - rv := &Thesaurus{ + rv = &Thesaurus{ sb: sb, name: name, fieldID: fieldIDPlus1 - 1, @@ -507,9 +515,8 @@ func (sb *SegmentBase) Thesaurus(name string) (segment.Thesaurus, error) { if err != nil { return nil, fmt.Errorf("thesaurus name %s vellum reader err: %v", name, err) } - return rv, nil } - return emptyThesaurus, nil + return rv, nil } // visitDocumentCtx holds data structures that are reusable across diff --git a/zap.md b/zap.md index ef413643..8bafac62 100644 --- a/zap.md +++ b/zap.md @@ -212,9 +212,9 @@ In a synonyms index, the relationship between a term and its synonyms is represe Each 64-bit entry consists of two parts: the first 32 bits represent the Term ID (TID), and the next 32 bits represent the Document Number (DN). - [{~~~~~+~~~~~}{~~~~~+~~~~~}...{~~~~~+~~~~~}] - | TID | DN || TID | DN | | TID | DN | - [{~~~~~+~~~~~}{~~~~~+~~~~~}...{~~~~~+~~~~~}] + [{~~~~~+~~~~}{~~~~~+~~~~}...{~~~~~+~~~~}] + | TID | DN || TID | DN | | TID | DN | + [{~~~~~+~~~~}{~~~~~+~~~~}...{~~~~~+~~~~}] TID - Term ID (32-bit) DN - Document Number (32-bit) From bde3bf815f8a5f665d179e35fadae2d6897d2387 Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Fri, 13 Dec 2024 14:07:10 +0530 Subject: [PATCH 23/28] review comments --- section_faiss_vector_index.go | 2 +- section_inverted_text_index.go | 18 +++++++++--------- section_synonym.go | 2 +- 3 files changed, 11 insertions(+), 11 deletions(-) diff --git a/section_faiss_vector_index.go b/section_faiss_vector_index.go index ef712317..6242d2b4 100644 --- a/section_faiss_vector_index.go +++ b/section_faiss_vector_index.go @@ -36,7 +36,7 @@ const defaultFaissOMPThreads = 1 func init() { rand.Seed(time.Now().UTC().UnixNano()) registerSegmentSection(SectionFaissVectorIndex, &faissVectorIndexSection{}) - invertedIndexExclusionChecks = append(invertedIndexExclusionChecks, func(field index.Field) bool { + invertedTextIndexSectionExclusionChecks = append(invertedTextIndexSectionExclusionChecks, func(field index.Field) bool { _, ok := field.(index.VectorField) return ok }) diff --git a/section_inverted_text_index.go b/section_inverted_text_index.go index 0cbf3357..680472e0 100644 --- a/section_inverted_text_index.go +++ b/section_inverted_text_index.go @@ -34,14 +34,14 @@ func init() { type invertedTextIndexSection struct { } -// This function checks whether the inverted index section should process -// a particular field, avoiding unnecessary work if another section will handle it. -var isFieldExcludedFromInvertedIndex = func(field index.Field) bool { - for _, excludeField := range invertedIndexExclusionChecks { +// This function checks whether the inverted text index section should avoid processing +// a particular field, preventing unnecessary work if another section will handle it. +var isFieldExcludedFromInvertedTextIndexSection = func(field index.Field) bool { + for _, excludeField := range invertedTextIndexSectionExclusionChecks { if excludeField(field) { // atleast one section has agreed to exclude this field - // from inverted index processing and has agreed to process it - // independently + // from inverted text index section processing and has + // agreed to process it independently return true } } @@ -50,11 +50,11 @@ var isFieldExcludedFromInvertedIndex = func(field index.Field) bool { return false } -// List of checks to determine if a field is excluded from the inverted index section -var invertedIndexExclusionChecks = make([]func(field index.Field) bool, 0) +// List of checks to determine if a field is excluded from the inverted text index section +var invertedTextIndexSectionExclusionChecks = make([]func(field index.Field) bool, 0) func (i *invertedTextIndexSection) Process(opaque map[int]resetable, docNum uint32, field index.Field, fieldID uint16) { - if !isFieldExcludedFromInvertedIndex(field) { + if !isFieldExcludedFromInvertedTextIndexSection(field) { invIndexOpaque := i.getInvertedIndexOpaque(opaque) invIndexOpaque.process(field, fieldID, docNum) } diff --git a/section_synonym.go b/section_synonym.go index a89a696a..f9d91069 100644 --- a/section_synonym.go +++ b/section_synonym.go @@ -31,7 +31,7 @@ import ( func init() { registerSegmentSection(SectionSynonymIndex, &synonymIndexSection{}) - invertedIndexExclusionChecks = append(invertedIndexExclusionChecks, func(field index.Field) bool { + invertedTextIndexSectionExclusionChecks = append(invertedTextIndexSectionExclusionChecks, func(field index.Field) bool { _, ok := field.(index.SynonymField) return ok }) From 5794708e88d42207d5e8c0e77120f72d1f0d93e6 Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Fri, 13 Dec 2024 14:13:04 +0530 Subject: [PATCH 24/28] rename file --- section_synonym.go => section_synonym_index.go | 0 1 file changed, 0 insertions(+), 0 deletions(-) rename section_synonym.go => section_synonym_index.go (100%) diff --git a/section_synonym.go b/section_synonym_index.go similarity index 100% rename from section_synonym.go rename to section_synonym_index.go From 079fe42912404d8b0c879e8d4fd65df64779951b Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Fri, 13 Dec 2024 18:01:58 +0530 Subject: [PATCH 25/28] add code commentary and refactor --- section_synonym_index.go | 155 ++++++++++++++++++++++++--------------- synonym_cache.go | 16 +++- synonym_posting.go | 27 +++++-- 3 files changed, 131 insertions(+), 67 deletions(-) diff --git a/section_synonym_index.go b/section_synonym_index.go index f9d91069..daa2b653 100644 --- a/section_synonym_index.go +++ b/section_synonym_index.go @@ -18,7 +18,6 @@ import ( "bytes" "encoding/binary" "fmt" - "io" "math" "sort" @@ -49,7 +48,7 @@ type synonymIndexOpaque struct { // the index opaque using the key "fieldsMap" // used for ensuring accurate mapping between fieldID and // thesaurusID - // name -> field id + // name -> field id + 1 FieldsMap map[string]uint16 // ThesaurusMap adds 1 to thesaurus id to avoid zero value issues @@ -92,6 +91,7 @@ type synonymIndexOpaque struct { thesaurusAddrs map[int]int } +// Set the fieldsMap and results in the synonym index opaque before the section processes a synonym field. func (so *synonymIndexOpaque) Set(key string, value interface{}) { switch key { case "results": @@ -101,12 +101,13 @@ func (so *synonymIndexOpaque) Set(key string, value interface{}) { } } +// Reset the synonym index opaque after a batch of documents have been processed into a segment. func (so *synonymIndexOpaque) Reset() (err error) { // cleanup stuff over here so.results = nil so.init = false so.ThesaurusMap = nil - so.ThesaurusInv = nil + so.ThesaurusInv = so.ThesaurusInv[:0] for i := range so.Thesauri { so.Thesauri[i] = nil } @@ -124,21 +125,25 @@ func (so *synonymIndexOpaque) Reset() (err error) { err = so.builder.Reset(&so.builderBuf) } so.FieldIDtoThesaurusID = nil - so.SynonymTermToID = nil - so.SynonymIDtoTerm = nil + so.SynonymTermToID = so.SynonymTermToID[:0] + so.SynonymIDtoTerm = so.SynonymIDtoTerm[:0] so.tmp0 = so.tmp0[:0] return err } func (so *synonymIndexOpaque) process(field index.SynonymField, fieldID uint16, docNum uint32) { + // if this is the first time we are processing a synonym field in this batch + // we need to allocate memory for the thesauri and related data structures if !so.init && so.results != nil { so.realloc() so.init = true } + // get the thesaurus id for this field tid := so.FieldIDtoThesaurusID[fieldID] + // get the thesaurus for this field thesaurus := so.Thesauri[tid] termSynMap := so.SynonymTermToID[tid] @@ -155,12 +160,16 @@ func (so *synonymIndexOpaque) process(field index.SynonymField, fieldID uint16, }) } +// a one-time call to allocate memory for the thesauri and synonyms which takes +// all the documents in the result batch and the fieldsMap and predetermines the +// size of the data structures in the synonymIndexOpaque func (so *synonymIndexOpaque) realloc() { var pidNext int var sidNext uint32 so.ThesaurusMap = map[string]uint16{} so.FieldIDtoThesaurusID = map[uint16]int{} + // count the number of unique thesauri from the batch of documents for _, result := range so.results { if synDoc, ok := result.(index.SynonymDocument); ok { synDoc.VisitSynonymFields(func(synField index.SynonymField) { @@ -174,7 +183,7 @@ func (so *synonymIndexOpaque) realloc() { if synDoc, ok := result.(index.SynonymDocument); ok { synDoc.VisitSynonymFields(func(synField index.SynonymField) { fieldIDPlus1 := so.FieldsMap[synField.Name()] - thesaurusID := uint16(so.getOrDefineThesaurus(fieldIDPlus1-1, synField.Name())) + thesaurusID := so.getOrDefineThesaurus(fieldIDPlus1-1, synField.Name()) thesaurus := so.Thesauri[thesaurusID] thesaurusKeys := so.ThesaurusKeys[thesaurusID] @@ -183,6 +192,7 @@ func (so *synonymIndexOpaque) realloc() { termSynMap := so.SynonymTermToID[thesaurusID] + // iterate over all the term-synonyms pair from the field synField.IterateSynonyms(func(term string, synonyms []string) { _, exists := thesaurus[term] if !exists { @@ -226,9 +236,11 @@ func (so *synonymIndexOpaque) realloc() { } } +// getOrDefineThesaurus returns the thesaurus id for the given field id and thesaurus name. func (so *synonymIndexOpaque) getOrDefineThesaurus(fieldID uint16, thesaurusName string) int { thesaurusIDPlus1, exists := so.ThesaurusMap[thesaurusName] if !exists { + // need to create a new thesaurusID for this thesaurusName and thesaurusIDPlus1 = uint16(len(so.ThesaurusInv) + 1) so.ThesaurusMap[thesaurusName] = thesaurusIDPlus1 so.ThesaurusInv = append(so.ThesaurusInv, thesaurusName) @@ -239,6 +251,7 @@ func (so *synonymIndexOpaque) getOrDefineThesaurus(fieldID uint16, thesaurusName so.SynonymTermToID = append(so.SynonymTermToID, make(map[string]uint32)) + // map the fieldID to the thesaurusID so.FieldIDtoThesaurusID[fieldID] = int(thesaurusIDPlus1 - 1) n := len(so.ThesaurusKeys) @@ -253,6 +266,7 @@ func (so *synonymIndexOpaque) getOrDefineThesaurus(fieldID uint16, thesaurusName return int(thesaurusIDPlus1 - 1) } +// grabBuf returns a reusable buffer of the given size from the synonymIndexOpaque. func (so *synonymIndexOpaque) grabBuf(size int) []byte { buf := so.tmp0 if cap(buf) < size { @@ -369,6 +383,9 @@ func (s *synonymIndexSection) getSynonymIndexOpaque(opaque map[int]resetable) *s return opaque[SectionSynonymIndex].(*synonymIndexOpaque) } +// Implementations of the Section interface for the synonym index section. +// InitOpaque initializes the synonym index opaque, which sets the FieldsMap and +// results in the opaque before the section processes a synonym field. func (s *synonymIndexSection) InitOpaque(args map[string]interface{}) resetable { rv := &synonymIndexOpaque{ thesaurusAddrs: map[int]int{}, @@ -380,6 +397,8 @@ func (s *synonymIndexSection) InitOpaque(args map[string]interface{}) resetable return rv } +// Process processes a synonym field by adding the synonyms to the thesaurus +// pointed to by the fieldID, implements the Process API for the synonym index section. func (s *synonymIndexSection) Process(opaque map[int]resetable, docNum uint32, field index.Field, fieldID uint16) { if fieldID == math.MaxUint16 { return @@ -390,12 +409,19 @@ func (s *synonymIndexSection) Process(opaque map[int]resetable, docNum uint32, f } } +// Persist serializes and writes the thesauri processed to the writer, along +// with the synonym postings lists, and the synonym term map. Implements the +// Persist API for the synonym index section. func (s *synonymIndexSection) Persist(opaque map[int]resetable, w *CountHashWriter) (n int64, err error) { synIndexOpaque := s.getSynonymIndexOpaque(opaque) _, err = synIndexOpaque.writeThesauri(w) return 0, err } +// AddrForField returns the file offset of the thesaurus for the given fieldID, +// it uses the FieldIDtoThesaurusID map to translate the fieldID to the thesaurusID, +// and returns the corresponding thesaurus offset from the thesaurusAddrs map. +// Implements the AddrForField API for the synonym index section. func (s *synonymIndexSection) AddrForField(opaque map[int]resetable, fieldID int) int { synIndexOpaque := s.getSynonymIndexOpaque(opaque) tid, exists := synIndexOpaque.FieldIDtoThesaurusID[uint16(fieldID)] @@ -405,6 +431,10 @@ func (s *synonymIndexSection) AddrForField(opaque map[int]resetable, fieldID int return synIndexOpaque.thesaurusAddrs[tid] } +// Merge merges the thesauri, synonym postings lists and synonym term maps from +// the segments into a single thesaurus and serializes and writes the merged +// thesaurus and associated data to the writer. Implements the Merge API for the +// synonym index section. func (s *synonymIndexSection) Merge(opaque map[int]resetable, segments []*SegmentBase, drops []*roaring.Bitmap, fieldsInv []string, newDocNumsIn [][]uint64, w *CountHashWriter, closeCh chan struct{}) error { @@ -421,10 +451,31 @@ func (s *synonymIndexSection) Merge(opaque map[int]resetable, segments []*Segmen // ----------------------------------------------------------------------------- +// encodeSynonym encodes a synonymID and a docID into a single uint64 value. +// The encoding format splits the 64 bits as follows: +// +// 63 32 31 0 +// +-----------+----------+ +// | synonymID | docNum | +// +-----------+----------+ +// +// The upper 32 bits (63-32) store the synonymID, and the lower 32 bits (31-0) store the docID. +// +// Parameters: +// +// synonymID - A 32-bit unsigned integer representing the ID of the synonym. +// docID - A 32-bit unsigned integer representing the document ID. +// +// Returns: +// +// A 64-bit unsigned integer that combines the synonymID and docID. func encodeSynonym(synonymID uint32, docID uint32) uint64 { return uint64(synonymID)<<32 | uint64(docID) } +// writeSynonyms serilizes and writes the synonym postings list to the writer, by first +// serializing the postings list to a byte slice and then writing the length +// of the byte slice followed by the byte slice itself. func writeSynonyms(postings *roaring64.Bitmap, w *CountHashWriter, bufMaxVarintLen64 []byte) ( offset uint64, err error) { termCardinality := postings.GetCardinality() @@ -434,7 +485,18 @@ func writeSynonyms(postings *roaring64.Bitmap, w *CountHashWriter, bufMaxVarintL postingsOffset := uint64(w.Count()) - err = writeRoaringSynonymWithLen(postings, w, bufMaxVarintLen64) + buf, err := postings.ToBytes() + if err != nil { + return 0, err + } + // write out the length + n := binary.PutUvarint(bufMaxVarintLen64, uint64(len(buf))) + _, err = w.Write(bufMaxVarintLen64[:n]) + if err != nil { + return 0, err + } + // write out the roaring bytes + _, err = w.Write(buf) if err != nil { return 0, err } @@ -442,6 +504,9 @@ func writeSynonyms(postings *roaring64.Bitmap, w *CountHashWriter, bufMaxVarintL return postingsOffset, nil } +// writeSynTermMap serializes and writes the synonym term map to the writer, by first +// writing the length of the map followed by the map entries, where each entry +// consists of the synonym ID, the length of the term, and the term itself. func writeSynTermMap(synTermMap map[uint32]string, w *CountHashWriter, bufMaxVarintLen64 []byte) error { if len(synTermMap) == 0 { return nil @@ -474,31 +539,6 @@ func writeSynTermMap(synTermMap map[uint32]string, w *CountHashWriter, bufMaxVar return nil } -// writes out the length of the roaring bitmap in bytes as varint -// then writes out the roaring bitmap itself -func writeRoaringSynonymWithLen(r *roaring64.Bitmap, w io.Writer, - reuseBufVarint []byte) error { - buf, err := r.ToBytes() - if err != nil { - return err - } - - // write out the length - n := binary.PutUvarint(reuseBufVarint, uint64(len(buf))) - _, err = w.Write(reuseBufVarint[:n]) - if err != nil { - return err - } - - // write out the roaring bytes - _, err = w.Write(buf) - if err != nil { - return err - } - - return nil -} - func mergeAndPersistSynonymSection(segments []*SegmentBase, dropsIn []*roaring.Bitmap, fieldsInv []string, newDocNumsIn [][]uint64, w *CountHashWriter, closeCh chan struct{}) (map[int]int, map[uint16]int, error) { @@ -626,7 +666,27 @@ func mergeAndPersistSynonymSection(segments []*SegmentBase, dropsIn []*roaring.B } synItr = synonyms.iterator(synItr) - newSynonymID, err = mergeSynonyms(synItr, newDocNums[itrI], newRoaring, synTermMap, termSynMap, newSynonymID) + var next seg.Synonym + next, err = synItr.Next() + for next != nil && err == nil { + synNewDocNum := newDocNums[itrI][next.Number()] + if synNewDocNum == docDropped { + return nil, nil, fmt.Errorf("see hit with dropped docNum") + } + nextTerm := next.Term() + var synNewID uint32 + if synID, ok := termSynMap[nextTerm]; ok { + synNewID = synID + } else { + synNewID = newSynonymID + termSynMap[nextTerm] = newSynonymID + synTermMap[newSynonymID] = nextTerm + newSynonymID++ + } + synNewCode := encodeSynonym(synNewID, uint32(synNewDocNum)) + newRoaring.Add(synNewCode) + next, err = synItr.Next() + } if err != nil { return nil, nil, err } @@ -687,6 +747,9 @@ func mergeAndPersistSynonymSection(segments []*SegmentBase, dropsIn []*roaring.B thesStart := w.Count() + // the synonym index section does not have any doc value data + // so we write two special entries to indicate that + // the field is not uninverted and the thesaurus offset n = binary.PutUvarint(bufMaxVarintLen64, fieldNotUninverted) _, err = w.Write(bufMaxVarintLen64[:n]) if err != nil { @@ -699,6 +762,7 @@ func mergeAndPersistSynonymSection(segments []*SegmentBase, dropsIn []*roaring.B return nil, nil, err } + // write out the thesaurus offset from which the vellum data starts n = binary.PutUvarint(bufMaxVarintLen64, thesOffset) _, err = w.Write(bufMaxVarintLen64[:n]) if err != nil { @@ -713,28 +777,3 @@ func mergeAndPersistSynonymSection(segments []*SegmentBase, dropsIn []*roaring.B return thesaurusAddrs, fieldIDtoThesaurusID, nil } - -func mergeSynonyms(synItr *SynonymsIterator, newDocNums []uint64, newRoaring *roaring64.Bitmap, - synTermMap map[uint32]string, termSynMap map[string]uint32, newSynonymID uint32) (uint32, error) { - next, err := synItr.Next() - for next != nil && err == nil { - synNewDocNum := newDocNums[next.Number()] - if synNewDocNum == docDropped { - return 0, fmt.Errorf("see hit with dropped docNum") - } - nextTerm := next.Term() - var synNewID uint32 - if synID, ok := termSynMap[nextTerm]; ok { - synNewID = synID - } else { - synNewID = newSynonymID - termSynMap[nextTerm] = newSynonymID - synTermMap[newSynonymID] = nextTerm - newSynonymID++ - } - synNewCode := encodeSynonym(synNewID, uint32(synNewDocNum)) - newRoaring.Add(synNewCode) - next, err = synItr.Next() - } - return newSynonymID, nil -} diff --git a/synonym_cache.go b/synonym_cache.go index acf39906..0b8d56c2 100644 --- a/synonym_cache.go +++ b/synonym_cache.go @@ -24,24 +24,28 @@ import ( func newSynonymIndexCache() *synonymIndexCache { return &synonymIndexCache{ - cache: make(map[uint16]*synonymCacheEntry), - closeCh: make(chan struct{}), + cache: make(map[uint16]*synonymCacheEntry), } } type synonymIndexCache struct { - closeCh chan struct{} - m sync.RWMutex + m sync.RWMutex cache map[uint16]*synonymCacheEntry } +// Clear clears the synonym cache which would mean tha the termID to term map would no longer be available. func (sc *synonymIndexCache) Clear() { sc.m.Lock() sc.cache = nil sc.m.Unlock() } +// loadOrCreate loads the synonym index cache for the specified fieldID if it is already present, +// or creates it if not. The synonym index cache for a fieldID consists of a tuple: +// - A Vellum FST (Finite State Transducer) representing the thesaurus. +// - A map associating synonym IDs to their corresponding terms. +// This function returns the loaded or newly created tuple (FST and map). func (sc *synonymIndexCache) loadOrCreate(fieldID uint16, mem []byte) (*vellum.FST, map[uint32][]byte, error) { sc.m.RLock() entry, ok := sc.cache[fieldID] @@ -63,6 +67,7 @@ func (sc *synonymIndexCache) loadOrCreate(fieldID uint16, mem []byte) (*vellum.F return sc.createAndCacheLOCKED(fieldID, mem) } +// createAndCacheLOCKED creates the synonym index cache for the specified fieldID and caches it. func (sc *synonymIndexCache) createAndCacheLOCKED(fieldID uint16, mem []byte) (*vellum.FST, map[uint32][]byte, error) { var pos uint64 vellumLen, read := binary.Uvarint(mem[pos : pos+binary.MaxVarintLen64]) @@ -98,6 +103,7 @@ func (sc *synonymIndexCache) createAndCacheLOCKED(fieldID uint16, mem []byte) (* return fst, synTermMap, nil } +// insertLOCKED inserts the vellum FST and the map of synonymID to term into the cache for the specified fieldID. func (sc *synonymIndexCache) insertLOCKED(fieldID uint16, fst *vellum.FST, synTermMap map[uint32][]byte) { _, ok := sc.cache[fieldID] if !ok { @@ -108,6 +114,8 @@ func (sc *synonymIndexCache) insertLOCKED(fieldID uint16, fst *vellum.FST, synTe } } +// synonymCacheEntry is a tuple of the vellum FST and the map of synonymID to term, +// and is the value stored in the synonym cache, for a given fieldID. type synonymCacheEntry struct { fst *vellum.FST synTermMap map[uint32][]byte diff --git a/synonym_posting.go b/synonym_posting.go index db3ab52c..a9e5755b 100644 --- a/synonym_posting.go +++ b/synonym_posting.go @@ -38,6 +38,7 @@ func init() { reflectStaticSizeSynonym = int(reflect.TypeOf(s).Size()) } +// SynonymsList represents a list of synonyms for a term, stored in a Roaring64 bitmap. type SynonymsList struct { sb *SegmentBase synonymsOffset uint64 @@ -49,7 +50,7 @@ type SynonymsList struct { buffer *bytes.Reader } -// represents an immutable, empty synonyms list +// immutable, empty synonyms list var emptySynonymsList = &SynonymsList{} func (p *SynonymsList) Size() int { @@ -62,7 +63,8 @@ func (p *SynonymsList) Size() int { return sizeInBytes } -// Iterator returns an iterator for this postings list +// Iterator creates and returns a SynonymsIterator for the SynonymsList. +// If the synonyms bitmap is nil, it returns an empty iterator. func (s *SynonymsList) Iterator(prealloc segment.SynonymsIterator) segment.SynonymsIterator { if s.synonyms == nil { return emptySynonymsIterator @@ -80,6 +82,8 @@ func (s *SynonymsList) Iterator(prealloc segment.SynonymsIterator) segment.Synon return s.iterator(preallocSI) } +// iterator initializes a SynonymsIterator for the SynonymsList and returns it. +// If a preallocated iterator is provided, it resets and reuses it; otherwise, it creates a new one. func (s *SynonymsList) iterator(rv *SynonymsIterator) *SynonymsIterator { if rv == nil { rv = &SynonymsIterator{} @@ -94,6 +98,8 @@ func (s *SynonymsList) iterator(rv *SynonymsIterator) *SynonymsIterator { return rv } +// read initializes a SynonymsList by reading data from the given synonymsOffset in the Thesaurus. +// It reads and parses the Roaring64 bitmap that represents the synonyms. func (rv *SynonymsList) read(synonymsOffset uint64, t *Thesaurus) error { rv.synonymsOffset = synonymsOffset @@ -121,7 +127,8 @@ func (rv *SynonymsList) read(synonymsOffset uint64, t *Thesaurus) error { } // ----------------------------------------------------------------------------- -// SynonymsIterator provides a way to iterate through the synonyms list + +// SynonymsIterator provides a way to iterate through the synonyms list. type SynonymsIterator struct { synonyms *SynonymsList except *roaring.Bitmap @@ -133,6 +140,7 @@ type SynonymsIterator struct { nextSyn Synonym } +// immutable, empty synonyms iterator var emptySynonymsIterator = &SynonymsIterator{} func (i *SynonymsIterator) Size() int { @@ -142,10 +150,13 @@ func (i *SynonymsIterator) Size() int { return sizeInBytes } +// Next returns the next Synonym in the iteration or an error if the end is reached. func (i *SynonymsIterator) Next() (segment.Synonym, error) { return i.next() } +// next retrieves the next synonym from the iterator, populates the nextSyn field, +// and returns it. If no valid synonym is found, it returns an error. func (i *SynonymsIterator) next() (segment.Synonym, error) { synID, docNum, exists, err := i.nextSynonym() if err != nil || !exists { @@ -165,12 +176,14 @@ func (i *SynonymsIterator) next() (segment.Synonym, error) { i.nextSyn = Synonym{} // clear the struct rv := &i.nextSyn rv.term = string(term) - rv.synID = synID rv.docNum = docNum return rv, nil } +// nextSynonym decodes the next synonym from the roaring bitmap iterator, +// ensuring it is not in the "except" set. Returns the synonymID, docNum, +// and a flag indicating success. func (i *SynonymsIterator) nextSynonym() (uint32, uint32, bool, error) { // If no synonyms are available, return early if i.Actual == nil || i.synonyms == nil || i.synonyms == emptySynonymsList { @@ -196,12 +209,13 @@ func (i *SynonymsIterator) nextSynonym() (uint32, uint32, bool, error) { return 0, 0, false, nil } +// Synonym represents a single synonym, containing the term, synonymID, and document number. type Synonym struct { term string - synID uint32 docNum uint32 } +// Size returns the memory size of the Synonym, including the length of the term string. func (p *Synonym) Size() int { sizeInBytes := reflectStaticSizeSynonym + SizeOfPtr + len(p.term) @@ -209,14 +223,17 @@ func (p *Synonym) Size() int { return sizeInBytes } +// Term returns the term of the Synonym. func (s *Synonym) Term() string { return s.term } +// Number returns the document number of the Synonym. func (s *Synonym) Number() uint32 { return s.docNum } +// decodeSynonym decodes a synonymCode into its synonymID and document ID components. func decodeSynonym(synonymCode uint64) (synonymID uint32, docID uint32) { return uint32(synonymCode >> 32), uint32(synonymCode) } From ddd8f768b28c4e6911d2b7fe3f0bb942a2666183 Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Mon, 16 Dec 2024 13:47:40 +0530 Subject: [PATCH 26/28] add comment --- section_inverted_text_index.go | 8 ++++++++ 1 file changed, 8 insertions(+) diff --git a/section_inverted_text_index.go b/section_inverted_text_index.go index 680472e0..db7b1a91 100644 --- a/section_inverted_text_index.go +++ b/section_inverted_text_index.go @@ -36,6 +36,14 @@ type invertedTextIndexSection struct { // This function checks whether the inverted text index section should avoid processing // a particular field, preventing unnecessary work if another section will handle it. +// +// NOTE: The exclusion check is applicable only to the InvertedTextIndexSection +// because it serves as a catch-all section. This section processes every field +// unless explicitly excluded, similar to a "default" case in a switch statement. +// Other sections, such as VectorSection and SynonymSection, rely on inclusion +// checks to process only specific field types (e.g., index.VectorField or +// index.SynonymField). Any new section added in the future must define its +// special field type and inclusion logic explicitly. var isFieldExcludedFromInvertedTextIndexSection = func(field index.Field) bool { for _, excludeField := range invertedTextIndexSectionExclusionChecks { if excludeField(field) { From 0d80cdefcbd06c7cdc6a160f07d6fcf0a7e4fbef Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Thu, 19 Dec 2024 14:59:50 +0530 Subject: [PATCH 27/28] review comments --- section_synonym_index.go | 12 +++++++++++- thesaurus.go | 2 +- thesaurus_test.go | 2 +- 3 files changed, 13 insertions(+), 3 deletions(-) diff --git a/section_synonym_index.go b/section_synonym_index.go index daa2b653..08e5a619 100644 --- a/section_synonym_index.go +++ b/section_synonym_index.go @@ -83,11 +83,21 @@ type synonymIndexOpaque struct { // synonym postings list -> synonym bitmap Synonyms []*roaring64.Bitmap - builder *vellum.Builder + // A reusable vellum FST builder that will be stored in the synonym opaque + // and reused across multiple document batches during the persist phase + // of the synonym index section, the FST builder is used to build the + // FST for each thesaurus, which maps terms to their corresponding synonym bitmaps. + builder *vellum.Builder + + // A reusable buffer for the vellum FST builder. It streams data written + // into the builder into a byte slice. The final byte slice represents + // the serialized vellum FST, which will be written to disk builderBuf bytes.Buffer + // A reusable buffer for temporary use within the synonym index opaque tmp0 []byte + // A map linking thesaurus IDs to their corresponding thesaurus' file offsets thesaurusAddrs map[int]int } diff --git a/thesaurus.go b/thesaurus.go index e488ae6c..f4f0d002 100644 --- a/thesaurus.go +++ b/thesaurus.go @@ -35,7 +35,7 @@ type Thesaurus struct { fstReader *vellum.Reader } -// // represents an immutable, empty Thesaurus +// represents an immutable, empty Thesaurus var emptyThesaurus = &Thesaurus{} // SynonymsList returns the synonyms list for the specified term diff --git a/thesaurus_test.go b/thesaurus_test.go index 47dc1339..8f205e75 100644 --- a/thesaurus_test.go +++ b/thesaurus_test.go @@ -326,7 +326,7 @@ func TestSynonymSegment(t *testing.T) { } }() - // // multiple segment test + // multiple segment test numSegs := 3 numDocs := 5 segData := make([][]testSynonymDefinition, numSegs) From c94e9161d6124da693d28ca2eda3d0fdd631b403 Mon Sep 17 00:00:00 2001 From: CascadingRadium Date: Thu, 19 Dec 2024 21:24:36 +0530 Subject: [PATCH 28/28] addess comments --- section_synonym_index.go | 15 ++++++--------- segment.go | 13 +++++++------ 2 files changed, 13 insertions(+), 15 deletions(-) diff --git a/section_synonym_index.go b/section_synonym_index.go index 08e5a619..9bfd804d 100644 --- a/section_synonym_index.go +++ b/section_synonym_index.go @@ -145,7 +145,7 @@ func (so *synonymIndexOpaque) Reset() (err error) { func (so *synonymIndexOpaque) process(field index.SynonymField, fieldID uint16, docNum uint32) { // if this is the first time we are processing a synonym field in this batch // we need to allocate memory for the thesauri and related data structures - if !so.init && so.results != nil { + if !so.init { so.realloc() so.init = true } @@ -434,6 +434,9 @@ func (s *synonymIndexSection) Persist(opaque map[int]resetable, w *CountHashWrit // Implements the AddrForField API for the synonym index section. func (s *synonymIndexSection) AddrForField(opaque map[int]resetable, fieldID int) int { synIndexOpaque := s.getSynonymIndexOpaque(opaque) + if synIndexOpaque == nil || synIndexOpaque.FieldIDtoThesaurusID == nil { + return 0 + } tid, exists := synIndexOpaque.FieldIDtoThesaurusID[uint16(fieldID)] if !exists { return 0 @@ -578,8 +581,6 @@ func mergeAndPersistSynonymSection(segments []*SegmentBase, dropsIn []*roaring.B fieldIDtoThesaurusID := make(map[uint16]int) - synTermMap := make(map[uint32]string) - termSynMap := make(map[string]uint32) var thesaurusID int var newSynonymID uint32 @@ -591,12 +592,8 @@ func mergeAndPersistSynonymSection(segments []*SegmentBase, dropsIn []*roaring.B thesauri = thesauri[:0] itrs = itrs[:0] newSynonymID = 0 - for syn := range synTermMap { - delete(synTermMap, syn) - } - for syn := range termSynMap { - delete(termSynMap, syn) - } + synTermMap := make(map[uint32]string) + termSynMap := make(map[string]uint32) for segmentI, segment := range segments { // check for the closure in meantime diff --git a/segment.go b/segment.go index ecb46794..41abde25 100644 --- a/segment.go +++ b/segment.go @@ -492,19 +492,20 @@ func (sb *SegmentBase) thesaurus(name string) (rv *Thesaurus, err error) { if fieldIDPlus1 == 0 { return nil, nil } - thesaurusStart := sb.fieldsSectionsMap[fieldIDPlus1-1][SectionSynonymIndex] - if thesaurusStart > 0 { + pos := sb.fieldsSectionsMap[fieldIDPlus1-1][SectionSynonymIndex] + if pos > 0 { rv = &Thesaurus{ sb: sb, name: name, fieldID: fieldIDPlus1 - 1, } + // skip the doc value offsets as doc values are not supported in thesaurus for i := 0; i < 2; i++ { - _, n := binary.Uvarint(sb.mem[thesaurusStart : thesaurusStart+binary.MaxVarintLen64]) - thesaurusStart += uint64(n) + _, n := binary.Uvarint(sb.mem[pos : pos+binary.MaxVarintLen64]) + pos += uint64(n) } - thesLoc, n := binary.Uvarint(sb.mem[thesaurusStart : thesaurusStart+binary.MaxVarintLen64]) - thesaurusStart += uint64(n) + thesLoc, n := binary.Uvarint(sb.mem[pos : pos+binary.MaxVarintLen64]) + pos += uint64(n) fst, synTermMap, err := sb.synIndexCache.loadOrCreate(rv.fieldID, sb.mem[thesLoc:]) if err != nil { return nil, fmt.Errorf("thesaurus name %s err: %v", name, err)