github.com/m3db/m3@v1.5.1-0.20231129193456-75a402aa583b/src/m3ninx/index/segment/mem/segment.go (about) 1 // Copyright (c) 2017 Uber Technologies, Inc. 2 // 3 // Permission is hereby granted, free of charge, to any person obtaining a copy 4 // of this software and associated documentation files (the "Software"), to deal 5 // in the Software without restriction, including without limitation the rights 6 // to use, copy, modify, merge, publish, distribute, sublicense, and/or sell 7 // copies of the Software, and to permit persons to whom the Software is 8 // furnished to do so, subject to the following conditions: 9 // 10 // The above copyright notice and this permission notice shall be included in 11 // all copies or substantial portions of the Software. 12 // 13 // THE SOFTWARE IS PROVIDED "AS IS", WITHOUT WARRANTY OF ANY KIND, EXPRESS OR 14 // IMPLIED, INCLUDING BUT NOT LIMITED TO THE WARRANTIES OF MERCHANTABILITY, 15 // FITNESS FOR A PARTICULAR PURPOSE AND NONINFRINGEMENT. IN NO EVENT SHALL THE 16 // AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER 17 // LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM, 18 // OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN 19 // THE SOFTWARE. 20 21 package mem 22 23 import ( 24 "errors" 25 re "regexp" 26 "sync" 27 28 "github.com/m3db/m3/src/m3ninx/doc" 29 "github.com/m3db/m3/src/m3ninx/index" 30 "github.com/m3db/m3/src/m3ninx/index/segment" 31 "github.com/m3db/m3/src/m3ninx/postings" 32 "github.com/m3db/m3/src/m3ninx/util" 33 ) 34 35 var ( 36 errSegmentSealed = errors.New("unable to seal, segment has already been sealed") 37 errSegmentIsUnsealed = errors.New("un-supported operation on an un-sealed mutable segment") 38 ) 39 40 // nolint: maligned 41 type memSegment struct { 42 offset int 43 plPool postings.Pool 44 newUUIDFn util.NewUUIDFn 45 46 state struct { 47 sync.RWMutex 48 closed bool 49 sealed bool 50 } 51 52 // Mapping of postings ID to document. 53 docs struct { 54 sync.RWMutex 55 data []doc.Metadata 56 } 57 58 // Mapping of term to postings list. 59 termsDict termsDictionary 60 61 writer struct { 62 sync.Mutex 63 idSet *idsMap 64 nextID postings.ID 65 } 66 readerID postings.AtomicID 67 } 68 69 // NewSegment returns a new in-memory mutable segment. It will start assigning 70 // postings IDs at the provided offset. 71 func NewSegment(opts Options) (segment.MutableSegment, error) { 72 s := &memSegment{ 73 plPool: opts.PostingsListPool(), 74 newUUIDFn: opts.NewUUIDFn(), 75 termsDict: newTermsDict(opts), 76 readerID: postings.NewAtomicID(0), 77 } 78 79 s.docs.data = make([]doc.Metadata, opts.InitialCapacity()) 80 81 s.writer.idSet = newIDsMap(256) 82 s.writer.nextID = 0 83 return s, nil 84 } 85 86 func (s *memSegment) SetIndexConcurrency(value int) { 87 // No-op, does not support concurrent indexing. 88 } 89 90 func (s *memSegment) IndexConcurrency() int { 91 return 1 92 } 93 94 func (s *memSegment) Reset() { 95 s.state.Lock() 96 defer s.state.Unlock() 97 98 s.state.sealed = false 99 100 s.termsDict.Reset() 101 s.readerID = postings.NewAtomicID(0) 102 103 var empty doc.Metadata 104 for i := range s.docs.data { 105 s.docs.data[i] = empty 106 } 107 s.docs.data = s.docs.data[:0] 108 109 s.writer.idSet.Reset() 110 s.writer.nextID = 0 111 } 112 113 func (s *memSegment) Size() int64 { 114 s.state.RLock() 115 closed := s.state.closed 116 size := int64(s.readerID.Load()) 117 s.state.RUnlock() 118 if closed { 119 return 0 120 } 121 return size 122 } 123 124 func (s *memSegment) Docs() []doc.Metadata { 125 s.state.RLock() 126 defer s.state.RUnlock() 127 128 s.docs.RLock() 129 defer s.docs.RUnlock() 130 131 return s.docs.data[:s.readerID.Load()] 132 } 133 134 func (s *memSegment) ContainsID(id []byte) (bool, error) { 135 s.state.RLock() 136 if s.state.closed { 137 s.state.RUnlock() 138 return false, segment.ErrClosed 139 } 140 141 contains := s.containsIDWithStateLock(id) 142 s.state.RUnlock() 143 return contains, nil 144 } 145 146 func (s *memSegment) containsIDWithStateLock(id []byte) bool { 147 return s.termsDict.ContainsTerm(doc.IDReservedFieldName, id) 148 } 149 150 func (s *memSegment) ContainsField(f []byte) (bool, error) { 151 s.state.RLock() 152 if s.state.closed { 153 s.state.RUnlock() 154 return false, segment.ErrClosed 155 } 156 157 contains := s.termsDict.ContainsField(f) 158 s.state.RUnlock() 159 return contains, nil 160 } 161 162 func (s *memSegment) Insert(d doc.Metadata) ([]byte, error) { 163 s.state.RLock() 164 defer s.state.RUnlock() 165 if s.state.closed { 166 return nil, segment.ErrClosed 167 } 168 169 { 170 s.writer.Lock() 171 defer s.writer.Unlock() 172 173 b := index.NewBatch([]doc.Metadata{d}) 174 b.AllowPartialUpdates = false 175 if err := s.prepareDocsWithLocks(b, nil); err != nil { 176 return nil, err 177 } 178 179 // Update the document in case we generated a UUID for it. 180 d = b.Docs[0] 181 182 if err := s.insertDocWithLocks(d); err != nil { 183 return nil, err 184 } 185 s.readerID.Inc() 186 } 187 188 return d.ID, nil 189 } 190 191 func (s *memSegment) InsertBatch(b index.Batch) error { 192 s.state.RLock() 193 defer s.state.RUnlock() 194 if s.state.closed { 195 return segment.ErrClosed 196 } 197 198 batchErr := index.NewBatchPartialError() 199 { 200 s.writer.Lock() 201 defer s.writer.Unlock() 202 203 if err := s.prepareDocsWithLocks(b, batchErr); err != nil { 204 return err 205 } 206 207 numInserts := uint32(0) 208 for i, d := range b.Docs { 209 // NB(prateek): we override a document to have no ID when 210 // it doesn't need to be inserted. 211 if !d.HasID() { 212 continue 213 } 214 if err := s.insertDocWithLocks(d); err != nil { 215 if !b.AllowPartialUpdates { 216 return err 217 } 218 batchErr.Add(index.BatchError{Err: err, Idx: i}) 219 continue 220 } 221 numInserts++ 222 } 223 s.readerID.Add(numInserts) 224 } 225 226 if !batchErr.IsEmpty() { 227 return batchErr 228 } 229 return nil 230 } 231 232 // prepareDocsWithLocks ensures the given documents can be inserted into the index. It 233 // must be called with the state and writer locks. 234 func (s *memSegment) prepareDocsWithLocks( 235 b index.Batch, 236 batchErr *index.BatchPartialError, 237 ) error { 238 s.writer.idSet.Reset() 239 240 var emptyDoc doc.Metadata 241 for i := 0; i < len(b.Docs); i++ { 242 d := b.Docs[i] 243 if err := d.Validate(); err != nil { 244 if !b.AllowPartialUpdates { 245 return err 246 } 247 batchErr.Add(index.BatchError{Err: err, Idx: i}) 248 b.Docs[i] = emptyDoc 249 continue 250 } 251 252 if d.HasID() { 253 if s.containsIDWithStateLock(d.ID) { 254 // The segment already contains this document so we can remove it from those 255 // we need to index. 256 b.Docs[i] = emptyDoc 257 continue 258 } 259 260 if _, ok := s.writer.idSet.Get(d.ID); ok { 261 if !b.AllowPartialUpdates { 262 return index.ErrDuplicateID 263 } 264 batchErr.Add(index.BatchError{Err: index.ErrDuplicateID, Idx: i}) 265 b.Docs[i] = emptyDoc 266 continue 267 } 268 } else { 269 id, err := s.newUUIDFn() 270 if err != nil { 271 if !b.AllowPartialUpdates { 272 return err 273 } 274 batchErr.Add(index.BatchError{Err: err, Idx: i}) 275 b.Docs[i] = emptyDoc 276 continue 277 } 278 279 d.ID = id 280 281 // Update the document in the batch since we added an ID to it. 282 b.Docs[i] = d 283 } 284 285 s.writer.idSet.SetUnsafe(d.ID, struct{}{}, idsMapSetUnsafeOptions{ 286 NoCopyKey: true, 287 NoFinalizeKey: true, 288 }) 289 } 290 291 return nil 292 } 293 294 // insertDocWithLocks inserts a document into the index. It must be called with the 295 // state and writer locks. 296 func (s *memSegment) insertDocWithLocks(d doc.Metadata) error { 297 nextID := s.writer.nextID 298 s.storeDocWithStateLock(nextID, d) 299 s.writer.nextID++ 300 return s.indexDocWithStateLock(nextID, d) 301 } 302 303 // indexDocWithStateLock indexes the fields of a document in the segment's terms 304 // dictionary. It must be called with the segment's state lock. 305 func (s *memSegment) indexDocWithStateLock(id postings.ID, d doc.Metadata) error { 306 for _, f := range d.Fields { 307 if err := s.termsDict.Insert(f, id); err != nil { 308 return err 309 } 310 } 311 return s.termsDict.Insert(doc.Field{ 312 Name: doc.IDReservedFieldName, 313 Value: d.ID, 314 }, id) 315 } 316 317 // storeDocWithStateLock stores a documents into the segment's mapping of postings 318 // IDs to documents. It must be called with the segment's state lock. 319 func (s *memSegment) storeDocWithStateLock(id postings.ID, d doc.Metadata) { 320 idx := int(id) 321 322 // Can return early if we have sufficient capacity. 323 { 324 s.docs.RLock() 325 size := len(s.docs.data) 326 if size > idx { 327 // NB(prateek): We only need a Read-lock here despite an insert operation because 328 // we're guaranteed to never have conflicts with docID (it's monotonically increasing), 329 // and have checked `i.docs.data` is large enough. 330 s.docs.data[idx] = d 331 s.docs.RUnlock() 332 return 333 } 334 s.docs.RUnlock() 335 } 336 337 // Otherwise we need to expand capacity. 338 { 339 s.docs.Lock() 340 size := len(s.docs.data) 341 342 // The slice has already been expanded since we released the lock. 343 if size > idx { 344 s.docs.data[idx] = d 345 s.docs.Unlock() 346 return 347 } 348 349 data := make([]doc.Metadata, 2*(size+1)) 350 copy(data, s.docs.data) 351 s.docs.data = data 352 s.docs.data[idx] = d 353 s.docs.Unlock() 354 } 355 } 356 357 func (s *memSegment) Reader() (segment.Reader, error) { 358 s.state.RLock() 359 defer s.state.RUnlock() 360 if s.state.closed { 361 return nil, segment.ErrClosed 362 } 363 364 limits := readerDocRange{ 365 startInclusive: postings.ID(0), 366 endExclusive: s.readerID.Load(), 367 } 368 return newReader(s, limits, s.plPool), nil 369 } 370 371 func (s *memSegment) AllDocs() (index.IDDocIterator, error) { 372 r, err := s.Reader() 373 if err != nil { 374 return nil, err 375 } 376 return r.AllDocs() 377 } 378 379 func (s *memSegment) matchTerm(field, term []byte) (postings.List, error) { 380 s.state.RLock() 381 defer s.state.RUnlock() 382 if s.state.closed { 383 return nil, segment.ErrClosed 384 } 385 386 return s.termsDict.MatchTerm(field, term), nil 387 } 388 389 func (s *memSegment) matchRegexp(field []byte, compiled *re.Regexp) (postings.List, error) { 390 s.state.RLock() 391 defer s.state.RUnlock() 392 if s.state.closed { 393 return nil, segment.ErrClosed 394 } 395 396 return s.termsDict.MatchRegexp(field, compiled), nil 397 } 398 399 func (s *memSegment) getDoc(id postings.ID) (doc.Metadata, error) { 400 s.state.RLock() 401 defer s.state.RUnlock() 402 if s.state.closed { 403 return doc.Metadata{}, segment.ErrClosed 404 } 405 406 idx := int(id) 407 408 s.docs.RLock() 409 if idx >= len(s.docs.data) { 410 s.docs.RUnlock() 411 return doc.Metadata{}, index.ErrDocNotFound 412 } 413 d := s.docs.data[idx] 414 s.docs.RUnlock() 415 416 return d, nil 417 } 418 419 func (s *memSegment) Close() error { 420 s.state.Lock() 421 defer s.state.Unlock() 422 if s.state.closed { 423 return segment.ErrClosed 424 } 425 426 s.state.closed = true 427 return nil 428 } 429 430 func (s *memSegment) IsSealed() bool { 431 s.state.Lock() 432 defer s.state.Unlock() 433 if s.state.closed { 434 return false 435 } 436 return s.state.sealed 437 } 438 439 func (s *memSegment) Seal() error { 440 s.state.Lock() 441 defer s.state.Unlock() 442 if s.state.closed { 443 return segment.ErrClosed 444 } 445 446 if s.state.sealed { 447 return errSegmentSealed 448 } 449 450 s.state.sealed = true 451 return nil 452 } 453 454 func (s *memSegment) Fields() (segment.FieldsIterator, error) { 455 s.state.RLock() 456 defer s.state.RUnlock() 457 if err := s.checkIsSealedWithRLock(); err != nil { 458 return nil, err 459 } 460 return s.termsDict.Fields(), nil 461 } 462 463 func (s *memSegment) FieldsPostingsList() (segment.FieldsPostingsListIterator, error) { 464 s.state.RLock() 465 defer s.state.RUnlock() 466 if err := s.checkIsSealedWithRLock(); err != nil { 467 return nil, err 468 } 469 return s.termsDict.FieldsPostingsList(), nil 470 } 471 472 func (s *memSegment) Terms(name []byte) (segment.TermsIterator, error) { 473 s.state.RLock() 474 defer s.state.RUnlock() 475 if err := s.checkIsSealedWithRLock(); err != nil { 476 return nil, err 477 } 478 return s.termsDict.Terms(name), nil 479 } 480 481 func (s *memSegment) FieldsIterable() segment.FieldsIterable { 482 return s 483 } 484 485 func (s *memSegment) FieldsPostingsListIterable() segment.FieldsPostingsListIterable { 486 return s 487 } 488 489 func (s *memSegment) TermsIterable() segment.TermsIterable { 490 return s 491 } 492 493 func (s *memSegment) checkIsSealedWithRLock() error { 494 if s.state.closed { 495 return segment.ErrClosed 496 } 497 if !s.state.sealed { 498 return errSegmentIsUnsealed 499 } 500 return nil 501 }