diff --git a/builder.go b/builder.go index 30285a2e4..c92920301 100644 --- a/builder.go +++ b/builder.go @@ -68,7 +68,7 @@ func newBuilder(path string, mapping mapping.IndexMapping, config map[string]int return nil, err } config["internal"] = map[string][]byte{ - string(mappingInternalKey): mappingBytes, + string(scorch.MappingInternalKey): mappingBytes, } // do not use real config, as these are options for the builder, diff --git a/index.go b/index.go index 3d2389884..6ab7ccd2a 100644 --- a/index.go +++ b/index.go @@ -329,6 +329,24 @@ func OpenUsing(path string, runtimeConfig map[string]interface{}) (Index, error) return openIndexUsing(path, runtimeConfig) } +// Update index at the specified path, must exist. +// The mapping used when created will be overwritten by the mapping provided +// for all Index/Search operations. +// Throws an error without any changes to the index if an unupdatable mapping is provided +func Update(path string, newParams string) (Index, error) { + return updateIndexUsing(path, nil, newParams) +} + +// UpdateUsing index at the specified path, must exist. +// The mapping used when created will be overwritten by the mapping provided +// for all Index/Search operations. +// The provided runtimeConfig can override settings +// persisted when the kvstore was created. +// Throws an error without any changes to the index if an unupdatable mapping is provided +func UpdateUsing(path string, runtimeConfig map[string]interface{}, newParams string) (Index, error) { + return updateIndexUsing(path, runtimeConfig, newParams) +} + // Builder is a limited interface, used to build indexes in an offline mode. // Items cannot be updated or deleted, and the caller MUST ensure a document is // indexed only once. diff --git a/index/scorch/optimize_knn.go b/index/scorch/optimize_knn.go index ca179574c..3b487882c 100644 --- a/index/scorch/optimize_knn.go +++ b/index/scorch/optimize_knn.go @@ -85,8 +85,15 @@ func (o *OptimizeVR) Finish() error { wg.Done() }() for field, vrs := range o.vrs { - vecIndex, err := segment.InterpretVectorIndex(field, - o.requiresFiltering, origSeg.deleted) + var vecIndex segment_api.VectorIndex + var err error + if info, ok := o.snapshot.updatedFields[field]; ok && info.All || info.Index { + vecIndex, err = segment.InterpretVectorIndex("", + o.requiresFiltering, origSeg.deleted) + } else { + vecIndex, err = segment.InterpretVectorIndex(field, + o.requiresFiltering, origSeg.deleted) + } if err != nil { errorsM.Lock() errors = append(errors, err) diff --git a/index/scorch/persister.go b/index/scorch/persister.go index d59f733df..85c4579ca 100644 --- a/index/scorch/persister.go +++ b/index/scorch/persister.go @@ -619,6 +619,18 @@ func prepareBoltSnapshot(snapshot *IndexSnapshot, tx *bolt.Tx, path string, return nil, nil, err } } + + // store updated field info + if segmentSnapshot.updatedFields != nil { + b, err := json.Marshal(segmentSnapshot.updatedFields) + if err != nil { + return nil, nil, err + } + err = snapshotSegmentBucket.Put(boltUpdatedFieldsKey, b) + if err != nil { + return nil, nil, err + } + } } return filenames, newSegmentPaths, nil @@ -722,6 +734,7 @@ var boltMetaDataSegmentTypeKey = []byte("type") var boltMetaDataSegmentVersionKey = []byte("version") var boltMetaDataTimeStamp = []byte("timeStamp") var boltStatsKey = []byte("stats") +var boltUpdatedFieldsKey = []byte("fields") var TotBytesWrittenKey = []byte("TotBytesWritten") func (s *Scorch) loadFromBolt() error { @@ -846,7 +859,7 @@ func (s *Scorch) loadSnapshot(snapshot *bolt.Bucket) (*IndexSnapshot, error) { segmentBucket := snapshot.Bucket(k) if segmentBucket == nil { _ = rv.DecRef() - return nil, fmt.Errorf("segment key, but bucket missing % x", k) + return nil, fmt.Errorf("segment key, but bucket missing %x", k) } segmentSnapshot, err := s.loadSegment(segmentBucket) if err != nil { @@ -860,6 +873,9 @@ func (s *Scorch) loadSnapshot(snapshot *bolt.Bucket) (*IndexSnapshot, error) { } rv.segment = append(rv.segment, segmentSnapshot) rv.offsets = append(rv.offsets, running) + if segmentSnapshot.updatedFields != nil { + rv.updatedFields = segmentSnapshot.updatedFields + } running += segmentSnapshot.segment.Count() } } @@ -872,13 +888,13 @@ func (s *Scorch) loadSegment(segmentBucket *bolt.Bucket) (*SegmentSnapshot, erro return nil, fmt.Errorf("segment path missing") } segmentPath := s.path + string(os.PathSeparator) + string(pathBytes) - segment, err := s.segPlugin.Open(segmentPath) + seg, err := s.segPlugin.Open(segmentPath) if err != nil { return nil, fmt.Errorf("error opening bolt segment: %v", err) } rv := &SegmentSnapshot{ - segment: segment, + segment: seg, cachedDocs: &cachedDocs{cache: nil}, cachedMeta: &cachedMeta{meta: nil}, } @@ -888,7 +904,7 @@ func (s *Scorch) loadSegment(segmentBucket *bolt.Bucket) (*SegmentSnapshot, erro r := bytes.NewReader(deletedBytes) _, err := deletedBitmap.ReadFrom(r) if err != nil { - _ = segment.Close() + _ = seg.Close() return nil, fmt.Errorf("error reading deleted bytes: %v", err) } if !deletedBitmap.IsEmpty() { @@ -902,11 +918,22 @@ func (s *Scorch) loadSegment(segmentBucket *bolt.Bucket) (*SegmentSnapshot, erro err := json.Unmarshal(statBytes, &statsMap) stats := &fieldStats{statMap: statsMap} if err != nil { - _ = segment.Close() + _ = seg.Close() return nil, fmt.Errorf("error reading stat bytes: %v", err) } rv.stats = stats } + updatedFieldBytes := segmentBucket.Get(boltUpdatedFieldsKey) + if updatedFieldBytes != nil { + var updatedFields map[string]index.FieldInfo + + err := json.Unmarshal(updatedFieldBytes, &updatedFields) + if err != nil { + _ = seg.Close() + return nil, fmt.Errorf("error reading updated field bytes: %v", err) + } + rv.updatedFields = updatedFields + } return rv, nil } diff --git a/index/scorch/scorch.go b/index/scorch/scorch.go index 429d1daa9..e4703ecb1 100644 --- a/index/scorch/scorch.go +++ b/index/scorch/scorch.go @@ -17,6 +17,7 @@ package scorch import ( "encoding/json" "fmt" + "log" "os" "path/filepath" "sync" @@ -36,6 +37,8 @@ const Version uint8 = 2 var ErrClosed = fmt.Errorf("scorch closed") +var MappingInternalKey = []byte("_mapping") + type Scorch struct { nextSegmentID uint64 stats Stats @@ -882,3 +885,82 @@ func (s *Scorch) CopyReader() index.CopyReader { func (s *Scorch) FireIndexEvent() { s.fireEvent(EventKindIndexStart, 0) } + +// Updates bolt db with the given field info. Existing field info already in bolt +// will be merged before persisting. The index mapping is also overwritted both +// in bolt as well as the index snapshot +func (s *Scorch) UpdateFields(fieldInfo map[string]*index.FieldInfo, mappingBytes []byte) error { + // Switch from pointer to value to marshal into a json for storage + updatedFields := make(map[string]index.FieldInfo) + for field, info := range fieldInfo { + updatedFields[field] = *info + } + err := s.updateBolt(updatedFields, mappingBytes) + if err != nil { + return err + } + s.root.m.Lock() + s.root.updatedFields = updatedFields + s.root.m.Unlock() + return nil +} + +// Merge and update deleted field info and rewrite index mapping +func (s *Scorch) updateBolt(fieldInfo map[string]index.FieldInfo, mappingBytes []byte) error { + return s.rootBolt.Update(func(tx *bolt.Tx) error { + snapshots := tx.Bucket(boltSnapshotsBucket) + if snapshots == nil { + return nil + } + + c := snapshots.Cursor() + for k, _ := c.Last(); k != nil; k, _ = c.Prev() { + _, _, err := decodeUvarintAscending(k) + if err != nil { + log.Printf("unable to parse segment epoch %x, continuing", k) + continue + } + snapshot := snapshots.Bucket(k) + cc := snapshot.Cursor() + for kk, _ := cc.First(); kk != nil; kk, _ = cc.Next() { + if kk[0] == boltInternalKey[0] { + internalBucket := snapshot.Bucket(kk) + if internalBucket == nil { + return fmt.Errorf("segment key, but bucket missing %x", kk) + } + err = internalBucket.Put(MappingInternalKey, mappingBytes) + if err != nil { + return err + } + } else if kk[0] != boltMetaDataKey[0] { + segmentBucket := snapshot.Bucket(kk) + if segmentBucket == nil { + return fmt.Errorf("segment key, but bucket missing %x", kk) + } + var updatedFields map[string]index.FieldInfo + updatedFieldBytes := segmentBucket.Get(boltUpdatedFieldsKey) + if updatedFieldBytes != nil { + err := json.Unmarshal(updatedFieldBytes, &updatedFields) + if err != nil { + return fmt.Errorf("error reading updated field bytes: %v", err) + } + for field, info := range fieldInfo { + updatedFields[field] = info + } + } else { + updatedFields = fieldInfo + } + b, err := json.Marshal(updatedFields) + if err != nil { + return err + } + err = segmentBucket.Put(boltUpdatedFieldsKey, b) + if err != nil { + return err + } + } + } + } + return nil + }) +} diff --git a/index/scorch/snapshot_index.go b/index/scorch/snapshot_index.go index ece32eee6..077d92e09 100644 --- a/index/scorch/snapshot_index.go +++ b/index/scorch/snapshot_index.go @@ -81,6 +81,8 @@ type IndexSnapshot struct { m2 sync.Mutex // Protects the fields that follow. fieldTFRs map[string][]*IndexSnapshotTermFieldReader // keyed by field, recycled TFR's + + updatedFields map[string]index.FieldInfo } func (i *IndexSnapshot) Segments() []*SegmentSnapshot { @@ -469,6 +471,12 @@ func (is *IndexSnapshot) Document(id string) (rv index.Document, err error) { // Keeping that TODO for now until we have a cleaner way. rvd.StoredFieldsSize += uint64(len(val)) + // Skip fields that are supposed to have deleted store values + if info, ok := is.updatedFields[name]; ok && + (info.All || info.Store) { + return true + } + // copy value, array positions to preserve them beyond the scope of this callback value := append([]byte(nil), val...) arrayPos := append([]uint64(nil), pos...) @@ -608,10 +616,21 @@ func (is *IndexSnapshot) TermFieldReader(ctx context.Context, term []byte, field segBytesRead := s.segment.BytesRead() rv.incrementBytesRead(segBytesRead) } - dict, err := s.segment.Dictionary(field) + + var dict segment.TermDictionary + var err error + + // Skip fields that are supposed to have no indexing + if info, ok := is.updatedFields[field]; ok && + (info.Index || info.All) { + dict, err = s.segment.Dictionary("") + } else { + dict, err = s.segment.Dictionary(field) + } if err != nil { return nil, err } + if dictStats, ok := dict.(segment.DiskStatsReporter); ok { bytesRead := dictStats.BytesRead() rv.incrementBytesRead(bytesRead) @@ -756,6 +775,17 @@ func (is *IndexSnapshot) documentVisitFieldTermsOnSegment( } } + // Filter out fields that are supposed to have no doc values + var filteredFields []string + for _, field := range vFields { + if info, ok := is.updatedFields[field]; ok && + (info.DocValues || info.All) { + continue + } else { + filteredFields = append(filteredFields, field) + } + } + var errCh chan error // cFields represents the fields that we'll need from the @@ -763,7 +793,7 @@ func (is *IndexSnapshot) documentVisitFieldTermsOnSegment( // if the caller happens to know we're on the same segmentIndex // from a previous invocation if cFields == nil { - cFields = subtractStrings(fields, vFields) + cFields = subtractStrings(fields, filteredFields) if !ss.cachedDocs.hasFields(cFields) { errCh = make(chan error, 1) @@ -778,8 +808,8 @@ func (is *IndexSnapshot) documentVisitFieldTermsOnSegment( } } - if ssvOk && ssv != nil && len(vFields) > 0 { - dvs, err = ssv.VisitDocValues(localDocNum, fields, visitor, dvs) + if ssvOk && ssv != nil && len(filteredFields) > 0 { + dvs, err = ssv.VisitDocValues(localDocNum, filteredFields, visitor, dvs) if err != nil { return nil, nil, err } diff --git a/index/scorch/snapshot_segment.go b/index/scorch/snapshot_segment.go index 96e59a31d..b9dcc7440 100644 --- a/index/scorch/snapshot_segment.go +++ b/index/scorch/snapshot_segment.go @@ -35,12 +35,13 @@ type SegmentSnapshot struct { // segment was mmaped recently, in which case // we consider the loading cost of the metadata // as part of IO stats. - mmaped uint32 - id uint64 - segment segment.Segment - deleted *roaring.Bitmap - creator string - stats *fieldStats + mmaped uint32 + id uint64 + segment segment.Segment + deleted *roaring.Bitmap + creator string + stats *fieldStats + updatedFields map[string]index.FieldInfo cachedMeta *cachedMeta diff --git a/index_impl.go b/index_impl.go index d59dfb9a1..52f2a4c3d 100644 --- a/index_impl.go +++ b/index_impl.go @@ -57,8 +57,6 @@ type indexImpl struct { const storePath = "store" -var mappingInternalKey = []byte("_mapping") - const SearchQueryStartCallbackKey = "_search_query_start_callback_key" const SearchQueryEndCallbackKey = "_search_query_end_callback_key" @@ -129,7 +127,7 @@ func newIndexUsing(path string, mapping mapping.IndexMapping, indexType string, if err != nil { return nil, err } - err = rv.i.SetInternal(mappingInternalKey, mappingBytes) + err = rv.i.SetInternal(scorch.MappingInternalKey, mappingBytes) if err != nil { return nil, err } @@ -202,7 +200,7 @@ func openIndexUsing(path string, runtimeConfig map[string]interface{}) (rv *inde } }() - mappingBytes, err := indexReader.GetInternal(mappingInternalKey) + mappingBytes, err := indexReader.GetInternal(scorch.MappingInternalKey) if err != nil { return nil, err } @@ -231,6 +229,131 @@ func openIndexUsing(path string, runtimeConfig map[string]interface{}) (rv *inde return rv, err } +func updateIndexUsing(path string, runtimeConfig map[string]interface{}, newParams string) (rv *indexImpl, err error) { + rv = &indexImpl{ + path: path, + name: path, + } + rv.stats = &IndexStat{i: rv} + + rv.meta, err = openIndexMeta(path) + if err != nil { + return nil, err + } + + // backwards compatibility if index type is missing + if rv.meta.IndexType == "" { + rv.meta.IndexType = upsidedown.Name + } + + storeConfig := rv.meta.Config + if storeConfig == nil { + storeConfig = map[string]interface{}{} + } + + var um *mapping.IndexMappingImpl + + if len(newParams) == 0 { + return nil, fmt.Errorf(("updated mapping is empty")) + } + + err = util.UnmarshalJSON([]byte(newParams), &um) + if err != nil { + return nil, fmt.Errorf("error parsing updated mapping JSON: %v\nmapping contents:\n%s", err, newParams) + } + + storeConfig["path"] = indexStorePath(path) + storeConfig["create_if_missing"] = false + storeConfig["error_if_exists"] = false + for rck, rcv := range runtimeConfig { + storeConfig[rck] = rcv + } + + // open the index + indexTypeConstructor := registry.IndexTypeConstructorByName(rv.meta.IndexType) + if indexTypeConstructor == nil { + return nil, ErrorUnknownIndexType + } + + rv.i, err = indexTypeConstructor(rv.meta.Storage, storeConfig, Config.analysisQueue) + if err != nil { + return nil, err + } + + err = rv.i.Open() + if err != nil { + return nil, err + } + defer func(rv *indexImpl) { + if !rv.open { + rv.i.Close() + } + }(rv) + + // now load the mapping + indexReader, err := rv.i.Reader() + if err != nil { + return nil, err + } + defer func() { + if cerr := indexReader.Close(); cerr != nil && err == nil { + err = cerr + } + }() + + mappingBytes, err := indexReader.GetInternal(scorch.MappingInternalKey) + if err != nil { + return nil, err + } + + var im *mapping.IndexMappingImpl + err = util.UnmarshalJSON(mappingBytes, &im) + if err != nil { + return nil, fmt.Errorf("error parsing mapping JSON: %v\nmapping contents:\n%s", err, string(mappingBytes)) + } + + // mark the index as open + rv.mutex.Lock() + defer rv.mutex.Unlock() + rv.open = true + + // validate the mapping + err = im.Validate() + if err != nil { + // note even if the mapping is invalid + // we still return an open usable index + return rv, err + } + + // Validate and update the index with the new mapping + if um != nil { + ui, ok := rv.i.(index.UpdateIndex) + if !ok { + return rv, fmt.Errorf("updated mapping present for unupdatable index") + } + + err = um.Validate() + if err != nil { + return rv, err + } + + fieldInfo, err := DeletedFields(im, um) + if err != nil { + return rv, err + } + + err = ui.UpdateFields(fieldInfo, []byte(newParams)) + if err != nil { + return rv, err + } + im = um + } + + rv.m = im + indexStats.Register(rv) + return rv, err +} + // Advanced returns internal index implementation func (i *indexImpl) Advanced() (index.Index, error) { return i.i, nil diff --git a/index_update.go b/index_update.go new file mode 100644 index 000000000..0b1de1e16 --- /dev/null +++ b/index_update.go @@ -0,0 +1,448 @@ +// Copyright (c) 2025 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 bleve + +import ( + "fmt" + + "github.com/blevesearch/bleve/v2/mapping" + index "github.com/blevesearch/bleve_index_api" +) + +// Store all the fields that interact with the data +// from a document path +type pathInfo struct { + fieldMapInfo []*fieldMapInfo + dynamic bool + path string + parentPath string +} + +// Store the field information with respect to the +// document paths +type fieldMapInfo struct { + fieldMapping *mapping.FieldMapping + rootName string + parent *pathInfo +} + +// Store all of the changes to defaults +type defaultInfo struct { + analyzer bool + dateTimeParser bool + synonymSource bool +} + +// Compare two index mappings to identify all of the updatable changes +func DeletedFields(ori, upd *mapping.IndexMappingImpl) (map[string]*index.FieldInfo, error) { + var err error + + defaultChanges, err := compareMappings(ori, upd) + if err != nil { + return nil, err + } + + // Check for new mappings present in the type mappings + // of the updated compared to the original + for name, updDMapping := range upd.TypeMapping { + err = checkUpdatedMapping(ori.TypeMapping[name], updDMapping) + if err != nil { + return nil, err + } + } + + // Check for new mappings present in the default mappings + // of the updated compared to the original + err = checkUpdatedMapping(ori.DefaultMapping, upd.DefaultMapping) + if err != nil { + return nil, err + } + + oriPaths := make(map[string]*pathInfo) + updPaths := make(map[string]*pathInfo) + + // Go through each mapping present in the original + // and consolidate according to the document paths + for name, oriDMapping := range ori.TypeMapping { + addPathInfo(oriPaths, "", oriDMapping, ori, nil, name) + } + addPathInfo(oriPaths, "", ori.DefaultMapping, ori, nil, "") + + // Go through each mapping present in the updated + // and consolidate according to the document paths + for name, updDMapping := range upd.TypeMapping { + addPathInfo(updPaths, "", updDMapping, ori, nil, name) + } + addPathInfo(updPaths, "", upd.DefaultMapping, ori, nil, "") + + // Compare both the mappings based on the document paths + // and create a list of index, docvalues, store differences + // for every single field possible + fieldInfo := make(map[string]*index.FieldInfo) + for path, info := range oriPaths { + err = addFieldInfo(fieldInfo, info, updPaths[path], defaultChanges) + if err != nil { + return nil, err + } + } + + // Remove entries from the list with no changes between the + // original and the updated mapping + for name, info := range fieldInfo { + if !info.All && !info.Index && !info.DocValues && !info.Store { + delete(fieldInfo, name) + } + if info.All { + if upd.IndexDynamic { + return nil, fmt.Errorf("Mapping cannot be removed when index dynamic is true") + } + if upd.StoreDynamic { + return nil, fmt.Errorf("Mapping cannot be removed when store dynamic is true") + } + if upd.DocValuesDynamic { + return nil, fmt.Errorf("Mapping cannot be removed when docvalues dynamic is true") + } + } + } + return fieldInfo, nil +} + +func compareMappings(ori, upd *mapping.IndexMappingImpl) (*defaultInfo, error) { + rv := &defaultInfo{} + + if ori.TypeField != upd.TypeField && + (len(ori.TypeMapping) != 0 || len(upd.TypeMapping) != 0) { + return nil, fmt.Errorf("type field cannot be changed when type mappings are present") + } + + if ori.DefaultType != upd.DefaultType { + return nil, fmt.Errorf("default type cannot be changed") + } + + if ori.DefaultAnalyzer != upd.DefaultAnalyzer { + rv.analyzer = true + } + + if ori.DefaultDateTimeParser != upd.DefaultDateTimeParser { + rv.dateTimeParser = true + } + + if ori.DefaultSynonymSource != upd.DefaultSynonymSource { + rv.synonymSource = true + } + + if ori.DefaultField != upd.DefaultField { + return nil, fmt.Errorf("default field cannot be changed") + } + + if ori.IndexDynamic != upd.IndexDynamic { + return nil, fmt.Errorf("index dynamic cannot be changed") + } + + if ori.StoreDynamic != upd.StoreDynamic { + return nil, fmt.Errorf(("store dynamic cannot be changed")) + } + + if ori.DocValuesDynamic != upd.DocValuesDynamic { + return nil, fmt.Errorf(("docvalues dynamic cannot be changed")) + } + + return rv, nil +} + +// Ensures updated document mapping does not contain new +// field mappings or document mappings +func checkUpdatedMapping(ori, upd *mapping.DocumentMapping) error { + + // Check to verify both original and updated are not nil + // and are enabled before proceeding + if ori == nil { + if upd == nil || !upd.Enabled { + return nil + } + return fmt.Errorf("updated index mapping contains new properties") + } + + if upd == nil || !upd.Enabled { + return nil + } + + var err error + // Recursively go through the child mappings + for name, updDMapping := range upd.Properties { + err = checkUpdatedMapping(ori.Properties[name], updDMapping) + if err != nil { + return err + } + } + + // Simple checks to ensure no new field mappings present + // in updated + for _, updFMapping := range upd.Fields { + var oriFMapping *mapping.FieldMapping + + for _, fMapping := range ori.Fields { + if updFMapping.Name == fMapping.Name { + oriFMapping = fMapping + } + } + if oriFMapping == nil { + return fmt.Errorf("updated index mapping contains new fields") + } + } + + return nil +} + +// Adds all of the field mappings while maintaining a tree of the document structure +// to ensure traversal and verification is possible incase of multiple mappings defined +// for a single field or multiple document fields' data getting written to a single zapx field +func addPathInfo(paths map[string]*pathInfo, name string, mp *mapping.DocumentMapping, + im *mapping.IndexMappingImpl, parent *pathInfo, rootName string) { + + // Early exit if mapping has been disabled + // Comparisions later on will be done with a nil object + if !mp.Enabled { + return + } + + // Consolidate path information like index dynamic across multiple + // mappings if path is the same + var pInfo *pathInfo + if val, ok := paths[name]; ok { + pInfo = val + } else { + pInfo = &pathInfo{ + fieldMapInfo: make([]*fieldMapInfo, 0), + } + pInfo.dynamic = mp.Dynamic && im.IndexDynamic + } + + pInfo.dynamic = (pInfo.dynamic || mp.Dynamic) && im.IndexDynamic + pInfo.path = name + if parent != nil { + pInfo.parentPath = parent.path + } + + // Recursively add path information for all child mappings + for cName, cMapping := range mp.Properties { + var pathName string + if name == "" { + pathName = cName + } else { + pathName = name + "." + cName + } + addPathInfo(paths, pathName, cMapping, im, pInfo, rootName) + } + + // Add field mapping information keeping the document structure intact + for _, fMap := range mp.Fields { + fieldMapInfo := &fieldMapInfo{ + fieldMapping: fMap, + rootName: rootName, + parent: pInfo, + } + pInfo.fieldMapInfo = append(pInfo.fieldMapInfo, fieldMapInfo) + } + + paths[name] = pInfo +} + +// Compare all of the fields at a particular document path and add its field information +func addFieldInfo(fInfo map[string]*index.FieldInfo, ori, upd *pathInfo, defaultChanges *defaultInfo) error { + + var info *index.FieldInfo + var updated bool + var err error + + // Assume deleted or disabled mapping if upd is nil. Checks for ori being nil + // or upd having mappings not in orihave already been done before this stage + if upd == nil { + for _, oriFMapInfo := range ori.fieldMapInfo { + info, updated, err = compareFieldMapping(oriFMapInfo.fieldMapping, nil, defaultChanges) + if err != nil { + return err + } + err = validateFieldInfo(info, updated, fInfo, ori, oriFMapInfo) + if err != nil { + return err + } + } + } else { + for _, oriFMapInfo := range ori.fieldMapInfo { + var updFMap *mapping.FieldMapping + // For multiple fields at a single document path, compare + // only with the matching ones + for _, updFMapInfo := range upd.fieldMapInfo { + if oriFMapInfo.rootName == updFMapInfo.rootName && + oriFMapInfo.fieldMapping.Name == updFMapInfo.fieldMapping.Name { + updFMap = updFMapInfo.fieldMapping + } + } + + info, updated, err = compareFieldMapping(oriFMapInfo.fieldMapping, updFMap, defaultChanges) + if err != nil { + return err + } + err = validateFieldInfo(info, updated, fInfo, ori, oriFMapInfo) + if err != nil { + return err + } + } + } + if err != nil { + return err + } + + return nil +} + +// Compares two field mappings against each other, checking for changes in index, store, doc values +// and complete deletiion of the mapping while noting that the changes made are doable based on +// other values like includeInAll and dynamic +// first return argument gives an empty fieldInfo if no changes detected +// second return argument gives a flag indicating whether any changes, if detected, are doable or if +// update is impossible +// third argument is an error explaining exactly why the change is not possible +func compareFieldMapping(original, updated *mapping.FieldMapping, defaultChanges *defaultInfo) (*index.FieldInfo, bool, error) { + + rv := &index.FieldInfo{} + + if updated == nil { + if original != nil && !original.IncludeInAll { + rv.All = true + return rv, true, nil + } else if original == nil { + return nil, false, nil + } + return nil, false, fmt.Errorf("deleted field present in '_all' field") + } else if original == nil { + return nil, false, fmt.Errorf("matching field not found in original index mapping") + } + + if original.Type != updated.Type { + return nil, false, fmt.Errorf("field type cannot be updated") + } + if original.Type == "text" { + if original.SynonymSource != updated.SynonymSource { + return nil, false, fmt.Errorf("synonym source cannot be changed for text field") + } else if original.SynonymSource == "inherit" && defaultChanges.synonymSource { + return nil, false, fmt.Errorf("synonym source cannot be changed for possible inherited text field") + } + if original.Analyzer != updated.Analyzer { + return nil, false, fmt.Errorf("analyzer cannot be updated for text fields") + } else if original.Analyzer == "inherit" && defaultChanges.analyzer { + return nil, false, fmt.Errorf("default analyzer changed for possible inherited text field") + } + } + if original.Type == "datetime" { + if original.DateFormat != updated.DateFormat { + return nil, false, fmt.Errorf("dateFormat cannot be updated for datetime fields") + } else if original.DateFormat == "inherit" && defaultChanges.dateTimeParser { + return nil, false, fmt.Errorf("default analyzer changed for possible inherited text field") + } + } + if original.Type == "vector" || original.Type == "vector_base64" { + if original.Dims != updated.Dims { + return nil, false, fmt.Errorf("dimensions cannot be updated for vector and vector_base64 fields") + } + if original.Similarity != updated.Similarity { + return nil, false, fmt.Errorf("similarity cannot be updated for vector and vector_base64 fields") + } + if original.VectorIndexOptimizedFor != updated.VectorIndexOptimizedFor { + return nil, false, fmt.Errorf("vectorIndexOptimizedFor cannot be updated for vector and vector_base64 fields") + } + } + if original.IncludeInAll != updated.IncludeInAll { + return nil, false, fmt.Errorf("includeInAll cannot be changed") + } + if original.IncludeTermVectors != updated.IncludeTermVectors { + return nil, false, fmt.Errorf("includeTermVectors cannot be changed") + } + if original.SkipFreqNorm != updated.SkipFreqNorm { + return nil, false, fmt.Errorf("skipFreqNorm cannot be changed") + } + + // Updating is not possible if store changes from true + // to false when the field is included in _all + if original.Store != updated.Store { + if updated.Store || updated.IncludeInAll { + return nil, false, fmt.Errorf("store cannot be changed if field present in `_all' field") + } else { + rv.Store = true + } + } + + // Updating is not possible if index changes from true + // to false when the field is included in _all + if original.Index != updated.Index { + if updated.Index || updated.IncludeInAll { + return nil, false, fmt.Errorf("index cannot be changed if field present in `_all' field") + } else { + rv.Index = true + rv.DocValues = true + } + } + + // Updating is not possible if docvalues changes from true + // to false when the field is included in _all + if original.DocValues != updated.DocValues { + if updated.DocValues || updated.IncludeInAll { + return nil, false, fmt.Errorf("docvalues cannot be changed if field present in `_all' field") + } else { + rv.DocValues = true + } + } + + if rv.All || rv.Index || rv.Store || rv.DocValues { + return rv, true, nil + } + return rv, false, nil +} + +// After identifying changes, validate against the existing changes incase of duplicate fields. +// In such a situation, any conflicting changes found will abort the update process +func validateFieldInfo(newInfo *index.FieldInfo, updated bool, fInfo map[string]*index.FieldInfo, + ori *pathInfo, oriFMapInfo *fieldMapInfo) error { + + var name string + if oriFMapInfo.parent.parentPath == "" { + if oriFMapInfo.fieldMapping.Name == "" { + name = oriFMapInfo.parent.path + } else { + name = oriFMapInfo.fieldMapping.Name + } + } else { + if oriFMapInfo.fieldMapping.Name == "" { + name = oriFMapInfo.parent.parentPath + "." + oriFMapInfo.parent.path + } else { + name = oriFMapInfo.parent.parentPath + "." + oriFMapInfo.fieldMapping.Name + } + } + if updated { + if ori.dynamic { + return fmt.Errorf("updated field is under a dynamic property") + } + } + if oldInfo, ok := fInfo[name]; ok { + if oldInfo.All != newInfo.All || oldInfo.Index != newInfo.Index || + oldInfo.DocValues != newInfo.DocValues || oldInfo.Store != newInfo.Store { + return fmt.Errorf("updated field impossible to verify because multiple mappings point to the same field name") + } + } else { + fInfo[name] = newInfo + } + return nil +} diff --git a/index_update_test.go b/index_update_test.go new file mode 100644 index 000000000..83c38e077 --- /dev/null +++ b/index_update_test.go @@ -0,0 +1,2498 @@ +// Copyright (c) 2025 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 bleve + +import ( + "encoding/json" + "reflect" + "testing" + + "github.com/blevesearch/bleve/v2/analysis/lang/en" + "github.com/blevesearch/bleve/v2/mapping" + index "github.com/blevesearch/bleve_index_api" +) + +func TestCompareFieldMapping(t *testing.T) { + tests := []struct { + original *mapping.FieldMapping + updated *mapping.FieldMapping + defaultChanges *defaultInfo + indexFieldInfo *index.FieldInfo + changed bool + err bool + }{ + { // both nil => no op + original: nil, + updated: nil, + defaultChanges: nil, + indexFieldInfo: nil, + changed: false, + err: false, + }, + { // updated nil => delete all + original: &mapping.FieldMapping{}, + updated: nil, + defaultChanges: &defaultInfo{ + analyzer: false, + dateTimeParser: false, + synonymSource: false, + }, + indexFieldInfo: &index.FieldInfo{ + All: true, + }, + changed: true, + err: false, + }, + { // type changed => not updatable + original: &mapping.FieldMapping{ + Type: "text", + }, + updated: &mapping.FieldMapping{ + Type: "datetime", + }, + defaultChanges: &defaultInfo{ + analyzer: false, + dateTimeParser: false, + synonymSource: false, + }, + indexFieldInfo: nil, + changed: false, + err: true, + }, + { // synonym source changed for text => not updatable + original: &mapping.FieldMapping{ + Type: "text", + SynonymSource: "a", + }, + updated: &mapping.FieldMapping{ + Type: "text", + SynonymSource: "b", + }, + defaultChanges: &defaultInfo{ + analyzer: false, + dateTimeParser: false, + synonymSource: false, + }, + indexFieldInfo: nil, + changed: false, + err: true, + }, + { // analyser changed for text => not updatable + original: &mapping.FieldMapping{ + Type: "text", + Analyzer: "a", + }, + updated: &mapping.FieldMapping{ + Type: "text", + Analyzer: "b", + }, + defaultChanges: &defaultInfo{ + analyzer: false, + dateTimeParser: false, + synonymSource: false, + }, + indexFieldInfo: nil, + changed: false, + err: true, + }, + { // default analyser changed when inherited => not updatable + original: &mapping.FieldMapping{ + Type: "text", + Analyzer: "inherit", + }, + updated: &mapping.FieldMapping{ + Type: "text", + Analyzer: "inherit", + }, + defaultChanges: &defaultInfo{ + analyzer: true, + dateTimeParser: false, + synonymSource: false, + }, + indexFieldInfo: nil, + changed: false, + err: true, + }, + { // default datetimeparser changed for inherited datetime field => not updatable + original: &mapping.FieldMapping{ + Type: "datetime", + DateFormat: "inherit", + }, + updated: &mapping.FieldMapping{ + Type: "datetime", + DateFormat: "inherit", + }, + defaultChanges: &defaultInfo{ + analyzer: false, + dateTimeParser: true, + synonymSource: false, + }, + indexFieldInfo: nil, + changed: false, + err: true, + }, + { // dims changed for vector => not updatable + original: &mapping.FieldMapping{ + Type: "vector", + Dims: 128, + Similarity: "l2_norm", + VectorIndexOptimizedFor: "memory-efficient", + }, + updated: &mapping.FieldMapping{ + Type: "vector", + Dims: 1024, + Similarity: "l2_norm", + VectorIndexOptimizedFor: "memory-efficient", + }, + defaultChanges: &defaultInfo{ + analyzer: false, + dateTimeParser: false, + synonymSource: false, + }, + indexFieldInfo: nil, + changed: false, + err: true, + }, + { // similarity changed for vectorbase64 => not updatable + original: &mapping.FieldMapping{ + Type: "vector_base64", + Similarity: "l2_norm", + Dims: 128, + VectorIndexOptimizedFor: "memory-efficient", + }, + updated: &mapping.FieldMapping{ + Type: "vector_base64", + Similarity: "dot_product", + Dims: 128, + VectorIndexOptimizedFor: "memory-efficient", + }, + defaultChanges: &defaultInfo{ + analyzer: false, + dateTimeParser: false, + synonymSource: false, + }, + indexFieldInfo: nil, + changed: false, + err: true, + }, + { // vectorindexoptimizedfor chagned for vector => not updatable + original: &mapping.FieldMapping{ + Type: "vector", + Similarity: "dot_product", + Dims: 128, + VectorIndexOptimizedFor: "memory-efficient", + }, + updated: &mapping.FieldMapping{ + Type: "vector", + Similarity: "dot_product", + Dims: 128, + VectorIndexOptimizedFor: "latency", + }, + defaultChanges: &defaultInfo{ + analyzer: false, + dateTimeParser: false, + synonymSource: false, + }, + indexFieldInfo: nil, + changed: false, + err: true, + }, + { // includeinall changed => not updatable + original: &mapping.FieldMapping{ + Type: "numeric", + IncludeInAll: true, + }, + updated: &mapping.FieldMapping{ + Type: "numeric", + IncludeInAll: false, + }, + defaultChanges: &defaultInfo{ + analyzer: false, + dateTimeParser: false, + synonymSource: false, + }, + indexFieldInfo: nil, + changed: false, + err: true, + }, + { //includetermvectors changed => not updatable + original: &mapping.FieldMapping{ + Type: "numeric", + IncludeTermVectors: false, + }, + updated: &mapping.FieldMapping{ + Type: "numeric", + IncludeTermVectors: true, + }, + defaultChanges: &defaultInfo{ + analyzer: false, + dateTimeParser: false, + synonymSource: false, + }, + indexFieldInfo: nil, + changed: false, + err: true, + }, + { // store changed after all checks => updatable with store delete + original: &mapping.FieldMapping{ + Type: "numeric", + SkipFreqNorm: true, + }, + updated: &mapping.FieldMapping{ + Type: "numeric", + SkipFreqNorm: false, + }, + defaultChanges: &defaultInfo{ + analyzer: false, + dateTimeParser: false, + synonymSource: false, + }, + indexFieldInfo: nil, + changed: false, + err: true, + }, + { // index changed after all checks => updatable with index and docvalues delete + original: &mapping.FieldMapping{ + Type: "geopoint", + Index: true, + }, + updated: &mapping.FieldMapping{ + Type: "geopoint", + Index: false, + }, + defaultChanges: &defaultInfo{ + analyzer: false, + dateTimeParser: false, + synonymSource: false, + }, + indexFieldInfo: &index.FieldInfo{ + Index: true, + DocValues: true, + }, + changed: true, + err: false, + }, + { // docvalues changed after all checks => docvalues delete + original: &mapping.FieldMapping{ + Type: "numeric", + DocValues: true, + }, + updated: &mapping.FieldMapping{ + Type: "numeric", + DocValues: false, + }, + defaultChanges: &defaultInfo{ + analyzer: false, + dateTimeParser: false, + synonymSource: false, + }, + indexFieldInfo: &index.FieldInfo{ + DocValues: true, + }, + changed: true, + err: false, + }, + { // no relavent changes => continue but no op + original: &mapping.FieldMapping{ + Name: "", + Type: "datetime", + Analyzer: "a", + Store: true, + Index: false, + IncludeTermVectors: true, + IncludeInAll: false, + DateFormat: "a", + DocValues: false, + SkipFreqNorm: true, + Dims: 128, + Similarity: "dot_product", + VectorIndexOptimizedFor: "memory-efficient", + SynonymSource: "a", + }, + updated: &mapping.FieldMapping{ + Name: "", + Type: "datetime", + Analyzer: "b", + Store: true, + Index: false, + IncludeTermVectors: true, + IncludeInAll: false, + DateFormat: "a", + DocValues: false, + SkipFreqNorm: true, + Dims: 256, + Similarity: "l2_norm", + VectorIndexOptimizedFor: "latency", + SynonymSource: "b", + }, + defaultChanges: &defaultInfo{ + analyzer: false, + dateTimeParser: false, + synonymSource: false, + }, + indexFieldInfo: &index.FieldInfo{}, + changed: false, + err: false, + }, + } + + for i, test := range tests { + rv, changed, err := compareFieldMapping(test.original, test.updated, test.defaultChanges) + + if err == nil && test.err || err != nil && !test.err { + t.Errorf("Unexpected error value for test %d, expecting %t, got %v\n", i, test.err, err) + } + if changed != test.changed { + t.Errorf("Unexpected changed value for test %d, expecting %t, got %t, err %v\n", i, test.changed, changed, err) + } + if rv == nil && test.indexFieldInfo != nil || rv != nil && test.indexFieldInfo == nil || !reflect.DeepEqual(rv, test.indexFieldInfo) { + t.Errorf("Unexpected index field info value for test %d, expecting %+v, got %+v, err %v", i, test.indexFieldInfo, rv, err) + } + } +} + +func TestCompareMappings(t *testing.T) { + tests := []struct { + original *mapping.IndexMappingImpl + updated *mapping.IndexMappingImpl + info *defaultInfo + err bool + }{ + { // changed type field when non empty mappings are present => error + original: &mapping.IndexMappingImpl{ + TypeField: "a", + TypeMapping: map[string]*mapping.DocumentMapping{ + "a": {}, + "b": {}, + }, + }, + updated: &mapping.IndexMappingImpl{ + TypeField: "b", + TypeMapping: map[string]*mapping.DocumentMapping{ + "a": {}, + "b": {}, + }, + }, + info: nil, + err: true, + }, + { // changed default type => error + original: &mapping.IndexMappingImpl{ + DefaultType: "a", + }, + updated: &mapping.IndexMappingImpl{ + DefaultType: "b", + }, + info: nil, + err: true, + }, + { // changed default analyzer => analyser true + original: &mapping.IndexMappingImpl{ + DefaultAnalyzer: "a", + }, + updated: &mapping.IndexMappingImpl{ + DefaultAnalyzer: "b", + }, + info: &defaultInfo{ + analyzer: true, + dateTimeParser: false, + synonymSource: false, + }, + err: false, + }, + { // changed default datetimeparser => datetimeparser true + original: &mapping.IndexMappingImpl{ + DefaultDateTimeParser: "a", + }, + updated: &mapping.IndexMappingImpl{ + DefaultDateTimeParser: "b", + }, + info: &defaultInfo{ + analyzer: false, + dateTimeParser: true, + synonymSource: false, + }, + err: false, + }, + { // changed default synonym source => synonym source true + original: &mapping.IndexMappingImpl{ + DefaultSynonymSource: "a", + }, + updated: &mapping.IndexMappingImpl{ + DefaultSynonymSource: "b", + }, + info: &defaultInfo{ + analyzer: false, + dateTimeParser: false, + synonymSource: true, + }, + err: false, + }, + { // changed default field => error + original: &mapping.IndexMappingImpl{ + DefaultField: "a", + }, + updated: &mapping.IndexMappingImpl{ + DefaultField: "b", + }, + info: nil, + err: true, + }, + { // changed index dynamic => error + original: &mapping.IndexMappingImpl{ + IndexDynamic: true, + }, + updated: &mapping.IndexMappingImpl{ + IndexDynamic: false, + }, + info: nil, + err: true, + }, + { // changed store dynamic => error + original: &mapping.IndexMappingImpl{ + StoreDynamic: false, + }, + updated: &mapping.IndexMappingImpl{ + StoreDynamic: true, + }, + info: nil, + err: true, + }, + { // changed docvalues dynamic => error + original: &mapping.IndexMappingImpl{ + DocValuesDynamic: true, + }, + updated: &mapping.IndexMappingImpl{ + DocValuesDynamic: false, + }, + info: nil, + err: true, + }, + } + + for i, test := range tests { + info, err := compareMappings(test.original, test.updated) + + if err == nil && test.err || err != nil && !test.err { + t.Errorf("Unexpected error value for test %d, expecting %t, got %v\n", i, test.err, err) + } + if info == nil && test.info != nil || info != nil && test.info == nil || !reflect.DeepEqual(info, test.info) { + t.Errorf("Unexpected default info value for test %d, expecting %+v, got %+v, err %v", i, test.info, info, err) + } + } +} + +func TestDeletedFields(t *testing.T) { + tests := []struct { + original *mapping.IndexMappingImpl + updated *mapping.IndexMappingImpl + fieldInfo map[string]*index.FieldInfo + err bool + }{ + { + // no change between original and updated having type and default mapping + // => empty fieldInfo with no error + original: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{ + "map1": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + "map2": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "b": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "c": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: false, + StoreDynamic: false, + DocValuesDynamic: false, + }, + updated: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{ + "map1": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + "map2": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "b": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "c": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: false, + StoreDynamic: false, + DocValuesDynamic: false, + }, + fieldInfo: map[string]*index.FieldInfo{}, + err: false, + }, + { + // no changes in type mappings and default mapping disabled with changes + // => empty fieldInfo with no error + original: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{ + "map1": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + "map2": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "b": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: false, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "c": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: false, + StoreDynamic: false, + DocValuesDynamic: false, + }, + updated: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{ + "map1": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + "map2": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "b": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: false, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "d": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: false, + StoreDynamic: false, + DocValuesDynamic: false, + }, + fieldInfo: map[string]*index.FieldInfo{}, + err: false, + }, + { + // new type mappings in updated => error + original: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{ + "map1": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + "map2": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "b": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: false, + StoreDynamic: false, + DocValuesDynamic: false, + }, + updated: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{ + "map1": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + "map2": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "c": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: false, + StoreDynamic: false, + DocValuesDynamic: false, + }, + fieldInfo: nil, + err: true, + }, + { + // new mappings in default mapping => error + original: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{}, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: false, + StoreDynamic: false, + DocValuesDynamic: false, + }, + updated: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{}, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "b": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: false, + StoreDynamic: false, + DocValuesDynamic: false, + }, + fieldInfo: nil, + err: true, + }, + { + // fully removed mapping in type with some dynamic => error + original: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{ + "map1": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + "map2": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "b": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "c": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: false, + StoreDynamic: false, + DocValuesDynamic: false, + }, + updated: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{ + "map1": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "c": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: true, + StoreDynamic: false, + DocValuesDynamic: false, + }, + fieldInfo: nil, + err: true, + }, + { + // semi removed mapping in default with some dynamic + // proper fieldInfo with no errors + original: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{ + "map1": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + "map2": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "b": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "c": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: false, + StoreDynamic: false, + DocValuesDynamic: false, + }, + updated: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{ + "map1": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + "map2": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "b": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: false, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "c": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: false, + StoreDynamic: false, + DocValuesDynamic: false, + }, + fieldInfo: map[string]*index.FieldInfo{ + "b": { + Index: true, + DocValues: true, + }, + }, + err: false, + }, + { + // two fields from diff paths with removed content matching + // => relavent fieldInfo + original: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{ + "map1": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + "map2": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "b": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: false, + StoreDynamic: false, + DocValuesDynamic: false, + }, + updated: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{ + "map1": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: false, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + "map2": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: false, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "b": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: false, + StoreDynamic: false, + DocValuesDynamic: false, + }, + fieldInfo: map[string]*index.FieldInfo{ + "a": { + Index: true, + DocValues: true, + }, + }, + err: false, + }, + { + // two fields from diff paths with removed content not matching + // => error + original: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{ + "map1": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + "map2": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "b": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: false, + StoreDynamic: false, + DocValuesDynamic: false, + }, + updated: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{ + "map1": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + "map2": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: false, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "b": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: false, + StoreDynamic: false, + DocValuesDynamic: false, + }, + fieldInfo: nil, + err: true, + }, + { + // two fields from the same path => relavent fieldInfo + original: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{ + "map1": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Name: "a", + Type: "numeric", + Index: true, + Store: true, + }, + { + Name: "b", + Type: "numeric", + Index: true, + Store: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "c": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: false, + StoreDynamic: false, + DocValuesDynamic: false, + }, + updated: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{ + "map1": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Name: "a", + Type: "numeric", + Index: false, + Store: true, + }, + { + Name: "b", + Type: "numeric", + Index: true, + Store: false, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "c": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: false, + StoreDynamic: false, + DocValuesDynamic: false, + }, + fieldInfo: map[string]*index.FieldInfo{ + "a": { + Index: true, + DocValues: true, + }, + "b": { + Store: true, + }, + }, + err: false, + }, + { + // one store, one index, one dynamic and one all removed in type and default + // => relavent fieldInfo without error + original: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{ + "map1": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + "map2": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "b": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Store: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + "map3": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "c": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + DocValues: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "d": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: true, + Store: true, + DocValues: true, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: false, + StoreDynamic: false, + DocValuesDynamic: false, + }, + updated: &mapping.IndexMappingImpl{ + TypeMapping: map[string]*mapping.DocumentMapping{ + "map1": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Index: false, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + "map2": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "b": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + Store: false, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + "map3": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "c": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "numeric", + DocValues: false, + }, + }, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + }, + DefaultMapping: &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "", + DefaultSynonymSource: "", + }, + IndexDynamic: false, + StoreDynamic: false, + DocValuesDynamic: false, + }, + fieldInfo: map[string]*index.FieldInfo{ + "a": { + Index: true, + DocValues: true, + }, + "b": { + Store: true, + }, + "c": { + DocValues: true, + }, + "d": { + All: true, + }, + }, + err: false, + }, + } + + for i, test := range tests { + info, err := DeletedFields(test.original, test.updated) + + if err == nil && test.err || err != nil && !test.err { + t.Errorf("Unexpected error value for test %d, expecting %t, got %v\n", i, test.err, err) + } + if info == nil && test.fieldInfo != nil || info != nil && test.fieldInfo == nil || !reflect.DeepEqual(info, test.fieldInfo) { + t.Errorf("Unexpected default info value for test %d, expecting %+v, got %+v, err %v", i, test.fieldInfo, info, err) + } + } +} + +func TestIndexUpdateText(t *testing.T) { + tmpIndexPath := createTmpIndexPath(t) + defer cleanupTmpIndexPath(t, tmpIndexPath) + + indexMappingBefore := mapping.NewIndexMapping() + indexMappingBefore.TypeMapping = map[string]*mapping.DocumentMapping{} + indexMappingBefore.DefaultMapping = &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "text", + Index: true, + Store: true, + }, + }, + DefaultAnalyzer: "standard", + DefaultSynonymSource: "", + }, + "b": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "text", + Index: true, + Store: true, + }, + }, + DefaultAnalyzer: "standard", + DefaultSynonymSource: "", + }, + "c": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "text", + Index: true, + Store: true, + }, + }, + DefaultAnalyzer: "standard", + DefaultSynonymSource: "", + }, + "d": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "text", + Index: true, + Store: true, + }, + }, + DefaultAnalyzer: "standard", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "standard", + DefaultSynonymSource: "", + } + indexMappingBefore.IndexDynamic = false + indexMappingBefore.StoreDynamic = false + indexMappingBefore.DocValuesDynamic = false + + index, err := New(tmpIndexPath, indexMappingBefore) + if err != nil { + t.Fatal(err) + } + doc1 := map[string]interface{}{"a": "xyz", "b": "abc", "c": "def", "d": "ghi"} + doc2 := map[string]interface{}{"a": "uvw", "b": "rst", "c": "klm", "d": "pqr"} + doc3 := map[string]interface{}{"a": "xyz", "b": "def", "c": "abc", "d": "mno"} + batch := index.NewBatch() + err = batch.Index("001", doc1) + if err != nil { + t.Fatal(err) + } + err = batch.Index("002", doc2) + if err != nil { + t.Fatal(err) + } + err = batch.Index("003", doc3) + if err != nil { + t.Fatal(err) + } + err = index.Batch(batch) + if err != nil { + t.Fatal(err) + } + err = index.Close() + if err != nil { + t.Fatal(err) + } + + indexMappingAfter := mapping.NewIndexMapping() + indexMappingAfter.TypeMapping = map[string]*mapping.DocumentMapping{} + indexMappingAfter.DefaultMapping = &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "text", + Index: true, + Store: true, + }, + }, + DefaultAnalyzer: "standard", + DefaultSynonymSource: "", + }, + "b": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "text", + Index: false, + Store: true, + }, + }, + DefaultAnalyzer: "standard", + DefaultSynonymSource: "", + }, + "c": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "text", + Index: true, + Store: false, + }, + }, + DefaultAnalyzer: "standard", + DefaultSynonymSource: "", + }, + }, + Fields: []*mapping.FieldMapping{}, + DefaultAnalyzer: "standard", + DefaultSynonymSource: "", + } + indexMappingAfter.IndexDynamic = false + indexMappingAfter.StoreDynamic = false + indexMappingAfter.DocValuesDynamic = false + + mappingString, err := json.Marshal(indexMappingAfter) + if err != nil { + t.Fatal(err) + } + index, err = Update(tmpIndexPath, string(mappingString)) + if err != nil { + t.Fatal(err) + } + + q1 := NewSearchRequest(NewQueryStringQuery("a:*")) + q1.Fields = append(q1.Fields, "a") + res1, err := index.Search(q1) + if err != nil { + t.Fatal(err) + } + if len(res1.Hits) != 3 { + t.Fatalf("Expected 3 hits, got %d\n", len(res1.Hits)) + } + if len(res1.Hits[0].Fields) != 1 { + t.Fatalf("Expected 1 field, got %d\n", len(res1.Hits[0].Fields)) + } + q2 := NewSearchRequest(NewQueryStringQuery("b:*")) + q2.Fields = append(q2.Fields, "b") + res2, err := index.Search(q2) + if err != nil { + t.Fatal(err) + } + if len(res2.Hits) != 0 { + t.Fatalf("Expected 0 hits, got %d\n", len(res2.Hits)) + } + q3 := NewSearchRequest(NewQueryStringQuery("c:*")) + q3.Fields = append(q3.Fields, "c") + res3, err := index.Search(q3) + if err != nil { + t.Fatal(err) + } + if len(res3.Hits) != 3 { + t.Fatalf("Expected 3 hits, got %d\n", len(res3.Hits)) + } + if len(res3.Hits[0].Fields) != 0 { + t.Fatalf("Expected 0 field, got %d\n", len(res3.Hits[0].Fields)) + } + q4 := NewSearchRequest(NewQueryStringQuery("d:*")) + q4.Fields = append(q4.Fields, "d") + res4, err := index.Search(q4) + if err != nil { + t.Fatal(err) + } + if len(res4.Hits) != 0 { + t.Fatalf("Expected 0 hits, got %d\n", len(res4.Hits)) + } +} + +func TestIndexUpdateVector(t *testing.T) { + tmpIndexPath := createTmpIndexPath(t) + defer cleanupTmpIndexPath(t, tmpIndexPath) + + indexMappingBefore := mapping.NewIndexMapping() + indexMappingBefore.TypeMapping = map[string]*mapping.DocumentMapping{} + indexMappingBefore.DefaultMapping = &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "vector", + Index: true, + Dims: 4, + Similarity: "l2_norm", + VectorIndexOptimizedFor: "latency", + }, + }, + }, + "b": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "vector", + Index: true, + Dims: 4, + Similarity: "l2_norm", + VectorIndexOptimizedFor: "latency", + }, + }, + }, + "c": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "vector_base64", + Index: true, + Dims: 4, + Similarity: "l2_norm", + VectorIndexOptimizedFor: "latency", + }, + }, + }, + "d": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "vector_base64", + Index: true, + Dims: 4, + Similarity: "l2_norm", + VectorIndexOptimizedFor: "latency", + }, + }, + }, + }, + Fields: []*mapping.FieldMapping{}, + } + indexMappingBefore.IndexDynamic = false + indexMappingBefore.StoreDynamic = false + indexMappingBefore.DocValuesDynamic = false + + index, err := New(tmpIndexPath, indexMappingBefore) + if err != nil { + t.Fatal(err) + } + doc1 := map[string]interface{}{"a": []float32{0.32894259691238403, 0.6973215341567993, 0.6835201978683472, 0.38296082615852356}, "b": []float32{0.32894259691238403, 0.6973215341567993, 0.6835201978683472, 0.38296082615852356}, "c": "L5MOPw7NID5SQMU9pHUoPw==", "d": "L5MOPw7NID5SQMU9pHUoPw=="} + doc2 := map[string]interface{}{"a": []float32{0.0018692062003538013, 0.41076546907424927, 0.5675257444381714, 0.45832985639572144}, "b": []float32{0.0018692062003538013, 0.41076546907424927, 0.5675257444381714, 0.45832985639572144}, "c": "czloP94ZCD71ldY+GbAOPw==", "d": "czloP94ZCD71ldY+GbAOPw=="} + doc3 := map[string]interface{}{"a": []float32{0.7853356599807739, 0.6904757618904114, 0.5643226504325867, 0.682637631893158}, "b": []float32{0.7853356599807739, 0.6904757618904114, 0.5643226504325867, 0.682637631893158}, "c": "Chh6P2lOqT47mjg/0odlPg==", "d": "Chh6P2lOqT47mjg/0odlPg=="} + batch := index.NewBatch() + err = batch.Index("001", doc1) + if err != nil { + t.Fatal(err) + } + err = batch.Index("002", doc2) + if err != nil { + t.Fatal(err) + } + err = batch.Index("003", doc3) + if err != nil { + t.Fatal(err) + } + err = index.Batch(batch) + if err != nil { + t.Fatal(err) + } + err = index.Close() + if err != nil { + t.Fatal(err) + } + + indexMappingAfter := mapping.NewIndexMapping() + indexMappingAfter.TypeMapping = map[string]*mapping.DocumentMapping{} + indexMappingAfter.DefaultMapping = &mapping.DocumentMapping{ + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{ + "a": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "vector", + Index: true, + Dims: 4, + Similarity: "l2_norm", + VectorIndexOptimizedFor: "latency", + }, + }, + }, + "c": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "vector_base64", + Index: true, + Dims: 4, + Similarity: "l2_norm", + VectorIndexOptimizedFor: "latency", + }, + }, + }, + "d": { + Enabled: true, + Dynamic: false, + Properties: map[string]*mapping.DocumentMapping{}, + Fields: []*mapping.FieldMapping{ + { + Type: "vector_base64", + Index: false, + Dims: 4, + Similarity: "l2_norm", + VectorIndexOptimizedFor: "latency", + }, + }, + }, + }, + Fields: []*mapping.FieldMapping{}, + } + indexMappingAfter.IndexDynamic = false + indexMappingAfter.StoreDynamic = false + indexMappingAfter.DocValuesDynamic = false + + mappingString, err := json.Marshal(indexMappingAfter) + if err != nil { + t.Fatal(err) + } + index, err = Update(tmpIndexPath, string(mappingString)) + if err != nil { + t.Fatal(err) + } + + q1 := NewSearchRequest(NewMatchNoneQuery()) + q1.AddKNN("a", []float32{1, 2, 3, 4}, 3, 1.0) + res1, err := index.Search(q1) + if err != nil { + t.Fatal(err) + } + if len(res1.Hits) != 3 { + t.Fatalf("Expected 3 hits, got %d\n", len(res1.Hits)) + } + q2 := NewSearchRequest(NewMatchNoneQuery()) + q2.AddKNN("e", []float32{1, 2, 3, 4}, 3, 1.0) + res2, err := index.Search(q2) + if err != nil { + t.Fatal(err) + } + if len(res2.Hits) != 0 { + t.Fatalf("Expected 0 hits, got %d\n", len(res2.Hits)) + } + q3 := NewSearchRequest(NewMatchNoneQuery()) + q3.AddKNN("c", []float32{1, 2, 3, 4}, 3, 1.0) + res3, err := index.Search(q3) + if err != nil { + t.Fatal(err) + } + if len(res3.Hits) != 3 { + t.Fatalf("Expected 3 hits, got %d\n", len(res3.Hits)) + } + q4 := NewSearchRequest(NewMatchNoneQuery()) + q4.AddKNN("d", []float32{1, 2, 3, 4}, 3, 1.0) + res4, err := index.Search(q4) + if err != nil { + t.Fatal(err) + } + if len(res4.Hits) != 0 { + t.Fatalf("Expected 0 hits, got %d\n", len(res4.Hits)) + } +} + +func TestIndexUpdateSynonym(t *testing.T) { + tmpIndexPath := createTmpIndexPath(t) + defer cleanupTmpIndexPath(t, tmpIndexPath) + + synonymCollection := "collection1" + synonymSourceName := "english" + analyzer := en.AnalyzerName + synonymSourceConfig := map[string]interface{}{ + "collection": synonymCollection, + "analyzer": analyzer, + } + + a := mapping.NewTextFieldMapping() + a.Analyzer = analyzer + a.SynonymSource = synonymSourceName + a.IncludeInAll = false + + b := mapping.NewTextFieldMapping() + b.Analyzer = analyzer + b.SynonymSource = synonymSourceName + b.IncludeInAll = false + + c := mapping.NewTextFieldMapping() + c.Analyzer = analyzer + c.SynonymSource = synonymSourceName + c.IncludeInAll = false + + indexMappingBefore := mapping.NewIndexMapping() + indexMappingBefore.DefaultMapping.AddFieldMappingsAt("a", a) + indexMappingBefore.DefaultMapping.AddFieldMappingsAt("b", b) + indexMappingBefore.DefaultMapping.AddFieldMappingsAt("c", c) + err := indexMappingBefore.AddSynonymSource(synonymSourceName, synonymSourceConfig) + if err != nil { + t.Fatal(err) + } + + indexMappingBefore.IndexDynamic = false + indexMappingBefore.StoreDynamic = false + indexMappingBefore.DocValuesDynamic = false + + index, err := New(tmpIndexPath, indexMappingBefore) + if err != nil { + t.Fatal(err) + } + + doc1 := map[string]interface{}{ + "a": `The hardworking employee consistently strives to exceed expectations. + His industrious nature makes him a valuable asset to any team. + His conscientious attention to detail ensures that projects are completed efficiently and accurately. + He remains persistent even in the face of challenges.`, + "b": `The hardworking employee consistently strives to exceed expectations. + His industrious nature makes him a valuable asset to any team. + His conscientious attention to detail ensures that projects are completed efficiently and accurately. + He remains persistent even in the face of challenges.`, + "c": `The hardworking employee consistently strives to exceed expectations. + His industrious nature makes him a valuable asset to any team. + His conscientious attention to detail ensures that projects are completed efficiently and accurately. + He remains persistent even in the face of challenges.`, + } + doc2 := map[string]interface{}{ + "a": `The tranquil surroundings of the retreat provide a perfect escape from the hustle and bustle of city life. + Guests enjoy the peaceful atmosphere, which is perfect for relaxation and rejuvenation. + The calm environment offers the ideal place to meditate and connect with nature. + Even the most stressed individuals find themselves feeling relaxed and at ease.`, + "b": `The tranquil surroundings of the retreat provide a perfect escape from the hustle and bustle of city life. + Guests enjoy the peaceful atmosphere, which is perfect for relaxation and rejuvenation. + The calm environment offers the ideal place to meditate and connect with nature. + Even the most stressed individuals find themselves feeling relaxed and at ease.`, + "c": `The tranquil surroundings of the retreat provide a perfect escape from the hustle and bustle of city life. + Guests enjoy the peaceful atmosphere, which is perfect for relaxation and rejuvenation. + The calm environment offers the ideal place to meditate and connect with nature. + Even the most stressed individuals find themselves feeling relaxed and at ease.`, + } + synDoc1 := &SynonymDefinition{Synonyms: []string{"hardworking", "industrious", "conscientious", "persistent", "focused", "devoted"}} + synDoc2 := &SynonymDefinition{Synonyms: []string{"tranquil", "peaceful", "calm", "relaxed", "unruffled"}} + + batch := index.NewBatch() + err = batch.IndexSynonym("001", synonymCollection, synDoc1) + if err != nil { + t.Fatal(err) + } + err = batch.IndexSynonym("002", synonymCollection, synDoc2) + if err != nil { + t.Fatal(err) + } + err = batch.Index("003", doc1) + if err != nil { + t.Fatal(err) + } + err = batch.Index("004", doc2) + if err != nil { + t.Fatal(err) + } + err = index.Batch(batch) + if err != nil { + t.Fatal(err) + } + err = index.Close() + if err != nil { + t.Fatal(err) + } + + indexMappingAfter := mapping.NewIndexMapping() + indexMappingAfter.DefaultMapping.AddFieldMappingsAt("a", a) + b.Index = false + indexMappingAfter.DefaultMapping.AddFieldMappingsAt("b", b) + err = indexMappingAfter.AddSynonymSource(synonymSourceName, synonymSourceConfig) + if err != nil { + t.Fatal(err) + } + + indexMappingAfter.IndexDynamic = false + indexMappingAfter.StoreDynamic = false + indexMappingAfter.DocValuesDynamic = false + + mappingString, err := json.Marshal(indexMappingAfter) + if err != nil { + t.Fatal(err) + } + index, err = Update(tmpIndexPath, string(mappingString)) + if err != nil { + t.Fatal(err) + } + + q1 := NewSearchRequest(NewQueryStringQuery("a:devoted")) + res1, err := index.Search(q1) + if err != nil { + t.Fatal(err) + } + if len(res1.Hits) != 1 { + t.Fatalf("Expected 1 hit, got %d\n", len(res1.Hits)) + } + + q2 := NewSearchRequest(NewQueryStringQuery("b:devoted")) + res2, err := index.Search(q2) + if err != nil { + t.Fatal(err) + } + if len(res2.Hits) != 0 { + t.Fatalf("Expected 0 hits, got %d\n", len(res2.Hits)) + } + + q3 := NewSearchRequest(NewQueryStringQuery("c:unruffled")) + res3, err := index.Search(q3) + if err != nil { + t.Fatal(err) + } + if len(res3.Hits) != 0 { + t.Fatalf("Expected 0 hits, got %d\n", len(res3.Hits)) + } +}