github.com/m3db/m3@v1.5.0/src/m3ninx/index/segment/builder/builder.go (about) 1 // Copyright (c) 2018 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 builder 22 23 import ( 24 "errors" 25 "fmt" 26 "runtime" 27 "sync" 28 29 "github.com/m3db/m3/src/m3ninx/doc" 30 "github.com/m3db/m3/src/m3ninx/index" 31 "github.com/m3db/m3/src/m3ninx/index/segment" 32 "github.com/m3db/m3/src/m3ninx/postings" 33 "github.com/m3db/m3/src/m3ninx/util" 34 35 "github.com/cespare/xxhash/v2" 36 "github.com/twotwotwo/sorts" 37 ) 38 39 var ( 40 errDocNotFound = errors.New("doc not found") 41 errClosed = errors.New("builder closed") 42 ) 43 44 const ( 45 // Slightly buffer the work to avoid blocking main thread. 46 indexQueueSize = 2 << 9 // 1024 47 entriesPerIndexJob = 32 48 ) 49 50 var ( 51 globalIndexWorkers = &indexWorkers{} 52 fieldsMapSetOptions = fieldsMapSetUnsafeOptions{ 53 // Builder takes ownership of keys and docs so it's ok 54 // to avoid copying and finalizing keys. 55 NoCopyKey: true, 56 NoFinalizeKey: true, 57 } 58 ) 59 60 type indexWorkers struct { 61 sync.RWMutex 62 builders int 63 queues []chan indexJob 64 } 65 66 type indexJob struct { 67 wg *sync.WaitGroup 68 69 opts Options 70 71 entries [entriesPerIndexJob]indexJobEntry 72 usedEntries int 73 74 shard int 75 shardedFields *shardedFields 76 77 batchErr *index.BatchPartialError 78 } 79 80 type indexJobEntry struct { 81 id postings.ID 82 field doc.Field 83 docIdx int 84 } 85 86 func (w *indexWorkers) registerBuilder() { 87 w.Lock() 88 defer w.Unlock() 89 90 preIncBuilders := w.builders 91 w.builders++ 92 93 if preIncBuilders != 0 { 94 return // Already initialized. 95 } 96 97 // Need to initialize structures, prepare all num CPU 98 // worker queues, even if we don't use all of them. 99 n := runtime.GOMAXPROCS(0) 100 if cap(w.queues) == 0 { 101 w.queues = make([]chan indexJob, 0, n) 102 } else { 103 // Reuse existing queues slice. 104 w.queues = w.queues[:0] 105 } 106 107 // Start the workers. 108 for i := 0; i < n; i++ { 109 indexQueue := make(chan indexJob, indexQueueSize) 110 w.queues = append(w.queues, indexQueue) 111 go w.indexWorker(indexQueue) 112 } 113 } 114 115 func (w *indexWorkers) indexWorker(indexQueue <-chan indexJob) { 116 for job := range indexQueue { 117 for i := 0; i < job.usedEntries; i++ { 118 entry := job.entries[i] 119 terms, ok := job.shardedFields.fields.ShardedGet(job.shard, entry.field.Name) 120 if !ok { 121 // NB(bodu): Check again within the lock to make sure we aren't making concurrent map writes. 122 terms = newTerms(job.opts) 123 job.shardedFields.fields.ShardedSetUnsafe(job.shard, entry.field.Name, 124 terms, fieldsMapSetOptions) 125 } 126 127 // If empty field, track insertion of this key into the fields 128 // collection for correct response when retrieving all fields. 129 newField := terms.size() == 0 130 // NB(bodu): Bulk of the cpu time during insertion is spent inside of terms.post(). 131 err := terms.post(entry.field.Value, entry.id) 132 if err != nil { 133 job.batchErr.AddWithLock(index.BatchError{Err: err, Idx: entry.docIdx}) 134 } 135 if err == nil && newField { 136 newEntry := uniqueField{ 137 field: entry.field.Name, 138 postingsList: terms.postingsListUnion, 139 } 140 job.shardedFields.uniqueFields[job.shard] = 141 append(job.shardedFields.uniqueFields[job.shard], newEntry) 142 } 143 } 144 145 job.wg.Done() 146 } 147 } 148 149 func (w *indexWorkers) indexJob(job indexJob) { 150 w.queues[job.shard] <- job 151 } 152 153 func (w *indexWorkers) unregisterBuilder() { 154 w.Lock() 155 defer w.Unlock() 156 157 w.builders-- 158 159 if w.builders != 0 { 160 return // Still have registered builders, cannot spin down yet. 161 } 162 163 // Close the workers. 164 for i := range w.queues { 165 close(w.queues[i]) 166 w.queues[i] = nil 167 } 168 w.queues = w.queues[:0] 169 } 170 171 type builderStatus struct { 172 sync.RWMutex 173 closed bool 174 } 175 176 type builder struct { 177 opts Options 178 newUUIDFn util.NewUUIDFn 179 180 batchSizeOne index.Batch 181 docs []doc.Metadata 182 idSet *IDsMap 183 shardedJobs []indexJob 184 shardedFields *shardedFields 185 concurrency int 186 187 status builderStatus 188 } 189 190 type shardedFields struct { 191 fields *shardedFieldsMap 192 uniqueFields [][]uniqueField 193 } 194 195 // NewBuilderFromDocuments returns a builder from documents, it is 196 // not thread safe and is optimized for insertion speed and a 197 // final build step when documents are indexed. 198 func NewBuilderFromDocuments(opts Options) (segment.CloseableDocumentsBuilder, error) { 199 b := &builder{ 200 opts: opts, 201 newUUIDFn: opts.NewUUIDFn(), 202 batchSizeOne: index.Batch{ 203 Docs: make([]doc.Metadata, 1), 204 }, 205 idSet: NewIDsMap(IDsMapOptions{ 206 InitialSize: opts.InitialCapacity(), 207 }), 208 shardedFields: &shardedFields{}, 209 } 210 // Indiciate we need to spin up workers if we haven't already. 211 globalIndexWorkers.registerBuilder() 212 b.SetIndexConcurrency(opts.Concurrency()) 213 return b, nil 214 } 215 216 func (b *builder) SetIndexConcurrency(value int) { 217 b.status.Lock() 218 defer b.status.Unlock() 219 220 if b.concurrency == value { 221 return // No-op 222 } 223 224 b.concurrency = value 225 226 // Nothing to migrate, jobs only used during a batch insertion. 227 b.shardedJobs = make([]indexJob, b.concurrency) 228 229 // Take refs to existing fields to migrate. 230 existingUniqueFields := b.shardedFields.uniqueFields 231 existingFields := b.shardedFields.fields 232 233 b.shardedFields.uniqueFields = make([][]uniqueField, 0, b.concurrency) 234 b.shardedFields.fields = newShardedFieldsMap(b.concurrency, b.opts.InitialCapacity()) 235 236 for i := 0; i < b.concurrency; i++ { 237 // Give each shard a fraction of the configured initial capacity. 238 shardInitialCapacity := b.opts.InitialCapacity() 239 if shardInitialCapacity > 0 { 240 shardInitialCapacity /= b.concurrency 241 } 242 243 shardUniqueFields := make([]uniqueField, 0, shardInitialCapacity) 244 b.shardedFields.uniqueFields = 245 append(b.shardedFields.uniqueFields, shardUniqueFields) 246 } 247 248 // Migrate data from existing unique fields. 249 if existingUniqueFields != nil { 250 for _, fields := range existingUniqueFields { 251 for _, field := range fields { 252 // Calculate the new shard for the field. 253 newShard := b.calculateShardWithRLock(field.field) 254 255 // Append to the correct shard. 256 b.shardedFields.uniqueFields[newShard] = 257 append(b.shardedFields.uniqueFields[newShard], field) 258 } 259 } 260 } 261 262 // Migrate from fields. 263 if existingFields != nil { 264 for _, fields := range existingFields.data { 265 for _, entry := range fields.Iter() { 266 field := entry.Key() 267 terms := entry.Value() 268 269 // Calculate the new shard for the field. 270 newShard := b.calculateShardWithRLock(field) 271 272 // Set with new correct shard. 273 b.shardedFields.fields.ShardedSetUnsafe(newShard, field, 274 terms, fieldsMapSetOptions) 275 } 276 } 277 } 278 } 279 280 func (b *builder) IndexConcurrency() int { 281 b.status.RLock() 282 defer b.status.RUnlock() 283 284 return b.concurrency 285 } 286 287 func (b *builder) Reset() { 288 b.status.Lock() 289 defer b.status.Unlock() 290 291 // Reset the documents slice. 292 var empty doc.Metadata 293 for i := range b.docs { 294 b.docs[i] = empty 295 } 296 b.docs = b.docs[:0] 297 298 // Remove all entries in the ID set. 299 b.idSet.Reset() 300 301 // Keep fields around, just reset the terms set for each one. 302 b.shardedFields.fields.ResetTermsSets() 303 304 // Reset the unique fields slice 305 var emptyField uniqueField 306 for i, shardUniqueFields := range b.shardedFields.uniqueFields { 307 for i := range shardUniqueFields { 308 shardUniqueFields[i] = emptyField 309 } 310 b.shardedFields.uniqueFields[i] = shardUniqueFields[:0] 311 } 312 } 313 314 func (b *builder) Insert(d doc.Metadata) ([]byte, error) { 315 b.status.Lock() 316 defer b.status.Unlock() 317 318 // Use a preallocated slice to make insert able to avoid alloc 319 // a slice to call insert batch with. 320 b.batchSizeOne.Docs[0] = d 321 err := b.insertBatchWithLock(b.batchSizeOne) 322 if err != nil { 323 if errs := err.Errs(); len(errs) == 1 { 324 // Return concrete error instead of the batch partial error. 325 return nil, errs[0].Err 326 } 327 // Fallback to returning batch partial error if not what we expect. 328 return nil, err 329 } 330 last := b.docs[len(b.docs)-1] 331 return last.ID, nil 332 } 333 334 func (b *builder) InsertBatch(batch index.Batch) error { 335 b.status.Lock() 336 defer b.status.Unlock() 337 338 if b.status.closed { 339 return errClosed 340 } 341 342 // NB(r): This switch is required or else *index.BatchPartialError 343 // is returned as a non-nil wrapped "error" even though it is not 344 // an error and underlying error is nil. 345 if err := b.insertBatchWithLock(batch); err != nil { 346 return err 347 } 348 return nil 349 } 350 351 func (b *builder) resetShardedJobs() { 352 // Reset sharded jobs using memset optimization. 353 var jobZeroed indexJob 354 for i := range b.shardedJobs { 355 b.shardedJobs[i] = jobZeroed 356 } 357 } 358 359 func (b *builder) insertBatchWithLock(batch index.Batch) *index.BatchPartialError { 360 // NB(r): This is all kept in a single method to make the 361 // insertion path avoid too much function call overhead. 362 wg := &sync.WaitGroup{} 363 batchErr := index.NewBatchPartialError() 364 365 // Reset shared resources and at cleanup too to remove refs. 366 b.resetShardedJobs() 367 defer b.resetShardedJobs() 368 369 // Enqueue docs for indexing. 370 for i, d := range batch.Docs { 371 // Validate doc 372 if err := d.Validate(); err != nil { 373 batchErr.Add(index.BatchError{Err: err, Idx: i}) 374 continue 375 } 376 377 // Generate ID if needed. 378 if !d.HasID() { 379 id, err := b.newUUIDFn() 380 if err != nil { 381 batchErr.Add(index.BatchError{Err: err, Idx: i}) 382 continue 383 } 384 385 d.ID = id 386 387 // Update the document in the batch since we added an ID to it. 388 batch.Docs[i] = d 389 } 390 391 // Avoid duplicates. 392 if _, ok := b.idSet.Get(d.ID); ok { 393 batchErr.Add(index.BatchError{Err: index.ErrDuplicateID, Idx: i}) 394 continue 395 } 396 397 // Write to document set. 398 b.idSet.SetUnsafe(d.ID, struct{}{}, IDsMapSetUnsafeOptions{ 399 NoCopyKey: true, 400 NoFinalizeKey: true, 401 }) 402 403 // Every new document just gets the next available id. 404 postingsListID := len(b.docs) 405 b.docs = append(b.docs, d) 406 407 // Index the terms. 408 for _, f := range d.Fields { 409 b.queueIndexJobEntryWithLock(wg, postings.ID(postingsListID), f, i, batchErr) 410 } 411 b.queueIndexJobEntryWithLock(wg, postings.ID(postingsListID), doc.Field{ 412 Name: doc.IDReservedFieldName, 413 Value: d.ID, 414 }, i, batchErr) 415 } 416 417 // Enqueue any partially filled sharded jobs. 418 for shard := 0; shard < b.concurrency; shard++ { 419 if b.shardedJobs[shard].usedEntries > 0 { 420 b.flushShardedIndexJobWithLock(shard, wg, batchErr) 421 } 422 } 423 424 // Wait for all the concurrent indexing jobs to finish. 425 wg.Wait() 426 427 if !batchErr.IsEmpty() { 428 return batchErr 429 } 430 return nil 431 } 432 433 func (b *builder) queueIndexJobEntryWithLock( 434 wg *sync.WaitGroup, 435 id postings.ID, 436 field doc.Field, 437 docIdx int, 438 batchErr *index.BatchPartialError, 439 ) { 440 shard := b.calculateShardWithRLock(field.Name) 441 entryIndex := b.shardedJobs[shard].usedEntries 442 b.shardedJobs[shard].usedEntries++ 443 b.shardedJobs[shard].entries[entryIndex].id = id 444 b.shardedJobs[shard].entries[entryIndex].field = field 445 b.shardedJobs[shard].entries[entryIndex].docIdx = docIdx 446 447 numEntries := b.shardedJobs[shard].usedEntries 448 if numEntries != entriesPerIndexJob { 449 return 450 } 451 452 // Ready to flush this job since all entries are used. 453 b.flushShardedIndexJobWithLock(shard, wg, batchErr) 454 455 // Reset for reuse. 456 b.shardedJobs[shard] = indexJob{} 457 } 458 459 func (b *builder) flushShardedIndexJobWithLock( 460 shard int, 461 wg *sync.WaitGroup, 462 batchErr *index.BatchPartialError, 463 ) { 464 // Set common fields. 465 b.shardedJobs[shard].shard = shard 466 b.shardedJobs[shard].wg = wg 467 b.shardedJobs[shard].batchErr = batchErr 468 b.shardedJobs[shard].shardedFields = b.shardedFields 469 b.shardedJobs[shard].opts = b.opts 470 471 // Enqueue job. 472 wg.Add(1) 473 globalIndexWorkers.indexJob(b.shardedJobs[shard]) 474 } 475 476 func (b *builder) calculateShardWithRLock(field []byte) int { 477 return int(xxhash.Sum64(field) % uint64(b.concurrency)) 478 } 479 480 func (b *builder) AllDocs() (index.IDDocIterator, error) { 481 b.status.RLock() 482 defer b.status.RUnlock() 483 484 rangeIter := postings.NewRangeIterator(0, postings.ID(len(b.docs))) 485 return index.NewIDDocIterator(b, rangeIter), nil 486 } 487 488 func (b *builder) Metadata(id postings.ID) (doc.Metadata, error) { 489 b.status.RLock() 490 defer b.status.RUnlock() 491 492 idx := int(id) 493 if idx < 0 || idx >= len(b.docs) { 494 return doc.Metadata{}, errDocNotFound 495 } 496 497 return b.docs[idx], nil 498 } 499 500 func (b *builder) Docs() []doc.Metadata { 501 b.status.RLock() 502 defer b.status.RUnlock() 503 504 return b.docs 505 } 506 507 func (b *builder) FieldsIterable() segment.FieldsPostingsListIterable { 508 return b 509 } 510 511 func (b *builder) TermsIterable() segment.TermsIterable { 512 return b 513 } 514 515 func (b *builder) FieldsPostingsList() (segment.FieldsPostingsListIterator, error) { 516 // NB(r): Need write lock since sort in newOrderedFieldsPostingsListIter 517 // and SetConcurrency causes sharded fields to change. 518 b.status.Lock() 519 defer b.status.Unlock() 520 521 return newOrderedFieldsPostingsListIter(b.shardedFields.uniqueFields), nil 522 } 523 524 func (b *builder) Terms(field []byte) (segment.TermsIterator, error) { 525 // NB(r): Need write lock since sort if required below 526 // and SetConcurrency causes sharded fields to change. 527 b.status.Lock() 528 defer b.status.Unlock() 529 530 shard := b.calculateShardWithRLock(field) 531 terms, ok := b.shardedFields.fields.ShardedGet(shard, field) 532 if !ok { 533 return nil, fmt.Errorf("field not found: %s", string(field)) 534 } 535 536 // NB(r): Ensure always sorted so can be used to build an FST which 537 // requires in order insertion. 538 terms.sortIfRequired() 539 540 return newTermsIter(terms.uniqueTerms), nil 541 } 542 543 func (b *builder) Close() error { 544 b.status.Lock() 545 defer b.status.Unlock() 546 547 b.status.closed = true 548 // Indiciate we could possibly spin down workers if no builders open. 549 globalIndexWorkers.unregisterBuilder() 550 return nil 551 } 552 553 var ( 554 sortConcurrencyLock sync.RWMutex 555 ) 556 557 // SetSortConcurrency sets the sort concurrency for when 558 // building segments, unfortunately this must be set globally 559 // since github.com/twotwotwo/sorts does not provide an 560 // ability to set parallelism on call to sort. 561 func SetSortConcurrency(value int) { 562 sortConcurrencyLock.Lock() 563 sorts.MaxProcs = value 564 sortConcurrencyLock.Unlock() 565 }