github.com/balzaczyy/golucene@v0.0.0-20151210033525-d0be9ee89713/core/index/documentsWriter.go (about) 1 package index 2 3 import ( 4 "container/list" 5 "fmt" 6 "github.com/balzaczyy/golucene/core/analysis" 7 "github.com/balzaczyy/golucene/core/index/model" 8 "github.com/balzaczyy/golucene/core/store" 9 "github.com/balzaczyy/golucene/core/util" 10 "sync" 11 "sync/atomic" 12 ) 13 14 // index/DocumentsWriter.java 15 16 /* 17 This class accepts multiple added documents and directly writes 18 segment files. 19 20 Each added document is passed to the indexing chain, which in turn 21 processes the document into the different codec formats. Some format 22 write bytes to files immediately, e.g. stored fields and term vectors, 23 while others are buffered by the indexing chain and written only on 24 flush. 25 26 Other consumers e.g. FreqProxTermsWriter and NormsConsumer, buffer 27 bytes in RAM and flush only when a new segment is produced. 28 29 Once we have used our allowed RAM buffer, or the number of aded docs 30 is large enough (in the case we are flushing by doc count instead of 31 RAM usage), we create a real segment and flush it to the Directory. 32 33 Goroutines: 34 35 Multiple Goroutines are allowed into AddDocument at once. There is an 36 initial synchronized call to ThreadState() which allocates a 37 TheadState for this goroutine. The same goroutine will get the same 38 ThreadState over time (goroutine affinity) so that if there are 39 consistent patterns (for example each goroutine is indexing a 40 different content source) then we make better use of RAM. Then 41 processDocument() is called on tha tThreadState without 42 synchronization (most of the "heavy lifting" is in this call). 43 Finally the synchronized "finishDocument" is called to flush changes 44 to the directory. 45 46 When flush is called by IndexWriter we forcefully idle all goroutines 47 and flush only once they are all idle. This means you can call flush 48 with a given goroutine even while other goroutines are actively 49 adding/deleting documents. 50 51 Exceptions: 52 53 Because this class directly updates in-memory posting lists, and 54 flushes stored fields and term vectors directly to files in the 55 directory, there are certain limited times when an error can corrupt 56 this state. For example, a disk full while flushing stored fields 57 leaves this file in a corrupt state. Or, a memory issue while 58 appending to the in-memory posting lists can corrupt that posting 59 list. We call such errors "aborting errors". In these cases we must 60 call abort() to discard all docs added since the last flush. 61 62 All other errors ("non-aborting errors") can still partially update 63 the index structures. These updates are consistent, but, they 64 represent only a part of the document seen up until the error was hit. 65 When this happens, we immediately mark the document as deleted so 66 that the document is always atomically ("all or none") added to the 67 index. 68 */ 69 type DocumentsWriter struct { 70 sync.Locker 71 72 directory store.Directory 73 closed bool // volatile 74 infoStream util.InfoStream 75 config LiveIndexWriterConfig 76 numDocsInRAM int32 // atomic 77 78 // TODO: cut over to BytesRefHash in BufferedUpdates 79 deleteQueue *DocumentsWriterDeleteQueue // volatile 80 ticketQueue *DocumentsWriterFlushQueue 81 // we preserve changes during a full flush since IW might not 82 // checkout before we release all changes. NRT Readers otherwise 83 // suddenly return true from isCurrent() while there are actually 84 // changes currently committed. See also anyChanges() & 85 // flushAllThreads() 86 pendingChangesInCurrentFullFlush bool // volatile 87 88 perThreadPool *DocumentsWriterPerThreadPool 89 flushPolicy FlushPolicy 90 flushControl *DocumentsWriterFlushControl 91 writer *IndexWriter 92 eventsLock *sync.RWMutex 93 events *list.List // synchronized 94 // for asserts 95 currentFullFlushDelQueue *DocumentsWriterDeleteQueue 96 } 97 98 func newDocumentsWriter(writer *IndexWriter, config LiveIndexWriterConfig, 99 directory store.Directory) *DocumentsWriter { 100 ans := &DocumentsWriter{ 101 Locker: &sync.Mutex{}, 102 deleteQueue: newDocumentsWriterDeleteQueue(), 103 ticketQueue: newDocumentsWriterFlushQueue(), 104 directory: directory, 105 config: config, 106 infoStream: config.InfoStream(), 107 perThreadPool: config.indexerThreadPool(), 108 flushPolicy: config.flushPolicy(), 109 writer: writer, 110 eventsLock: &sync.RWMutex{}, 111 events: list.New(), 112 } 113 ans.flushControl = newDocumentsWriterFlushControl(ans, config, writer.bufferedUpdatesStream) 114 return ans 115 } 116 117 func (dw *DocumentsWriter) applyAllDeletes(deleteQueue *DocumentsWriterDeleteQueue) (bool, error) { 118 if dw.flushControl.getAndResetApplyAllDeletes() { 119 if deleteQueue != nil && !dw.flushControl.fullFlush { 120 err := dw.ticketQueue.addDeletes(deleteQueue) 121 if err != nil { 122 return false, err 123 } 124 } 125 dw.putEvent(applyDeletesEvent) // apply deletes event forces a purge 126 return true, nil 127 } 128 return false, nil 129 } 130 131 func (w *DocumentsWriter) purgeBuffer(writer *IndexWriter, forced bool) (int, error) { 132 // forced flag is ignored since Go doesn't encourage tryLock idea 133 return w.ticketQueue.forcePurge(writer) 134 } 135 136 func (dw *DocumentsWriter) ensureOpen() { 137 assert2(!dw.closed, "this IndexWriter is closed") 138 } 139 140 /* 141 Called if we hit an error at a bad time (when updating the index 142 files) and must discard all currently buffered docs. This resets our 143 state, discarding any docs added since last flush. 144 */ 145 func (dw *DocumentsWriter) abort(writer *IndexWriter) { 146 dw.Lock() 147 defer dw.Unlock() 148 149 var success = false 150 var newFilesSet = make(map[string]bool) 151 defer func() { 152 if dw.infoStream.IsEnabled("DW") { 153 dw.infoStream.Message("DW", "done abort; abortedFiles=%v success=%v", newFilesSet, success) 154 } 155 }() 156 157 dw.deleteQueue.clear() 158 if dw.infoStream.IsEnabled("DW") { 159 dw.infoStream.Message("DW", "abort") 160 } 161 dw.perThreadPool.foreach(func(perThread *ThreadState) { 162 dw.abortThreadState(perThread, newFilesSet) 163 }) 164 dw.flushControl.abortPendingFlushes(newFilesSet) 165 dw.putEvent(newDeleteNewFilesEvent(newFilesSet)) 166 dw.flushControl.waitForFlush() 167 success = true 168 } 169 170 func (dw *DocumentsWriter) abortThreadState(perThread *ThreadState, newFiles map[string]bool) { 171 if perThread.isActive { // we might be closed 172 if perThread.dwpt != nil { 173 defer func() { 174 perThread.dwpt.checkAndResetHasAborted() 175 dw.flushControl.doOnAbort(perThread) 176 }() 177 dw.subtractFlushedNumDocs(perThread.dwpt.numDocsInRAM) 178 perThread.dwpt.abort(newFiles) 179 } else { 180 dw.flushControl.doOnAbort(perThread) 181 } 182 } else { 183 assert(dw.closed) 184 } 185 } 186 187 func (dw *DocumentsWriter) anyChanges() bool { 188 if dw.infoStream.IsEnabled("DW") { 189 dw.infoStream.Message("DW", 190 "anyChanges? numDocsInRAM=%v deletes=%v, hasTickets=%v pendingChangesInFullFlush=%v", 191 atomic.LoadInt32(&dw.numDocsInRAM), dw.deleteQueue.anyChanges(), 192 dw.ticketQueue.hasTickets(), dw.pendingChangesInCurrentFullFlush) 193 } 194 // Changes are either in a DWPT or in the deleteQueue. 195 // Yet if we currently flush deletes and/or dwpt, there 196 // could be a window where all changes are in the ticket queue 197 // before they are published to the IW, ie, we need to check if the 198 // ticket queue has any tickets. 199 return atomic.LoadInt32(&dw.numDocsInRAM) != 0 || dw.deleteQueue.anyChanges() || 200 dw.ticketQueue.hasTickets() || dw.pendingChangesInCurrentFullFlush 201 } 202 203 func (dw *DocumentsWriter) close() { 204 dw.closed = true 205 dw.flushControl.close() 206 } 207 208 func (dw *DocumentsWriter) preUpdate() (bool, error) { 209 dw.ensureOpen() 210 var hasEvents = false 211 if dw.flushControl.anyStalledThreads() || dw.flushControl.numQueuedFlushes() > 0 { 212 // Help out flushing any queued DWPTs so we can un-stall: 213 if dw.infoStream.IsEnabled("DW") { 214 dw.infoStream.Message("DW", "DocumentsWriter has queued dwpt; will hijack this thread to flush pending segment(s)") 215 } 216 for { 217 // Try pick up pending threads here if possible 218 for flushingDWPT := dw.flushControl.nextPendingFlush(); flushingDWPT != nil; { 219 // Don't push the delete here since the update could fail! 220 ok, err := dw.doFlush(flushingDWPT) 221 if err != nil { 222 return false, err 223 } 224 if ok { 225 hasEvents = true 226 } 227 } 228 229 if dw.infoStream.IsEnabled("DW") { 230 if dw.flushControl.anyStalledThreads() { 231 dw.infoStream.Message("DW", "WARNING DocumentsWriter has stalled threads; waiting") 232 } 233 } 234 235 dw.flushControl.waitIfStalled() // block if stalled 236 if dw.flushControl.numQueuedFlushes() == 0 { 237 break 238 } 239 } 240 } 241 return hasEvents, nil 242 } 243 244 func (dw *DocumentsWriter) postUpdate(flushingDWPT *DocumentsWriterPerThread, hasEvents bool) (bool, error) { 245 ok, err := dw.applyAllDeletes(dw.deleteQueue) 246 if err != nil { 247 return false, err 248 } 249 if ok { 250 hasEvents = true 251 } 252 if flushingDWPT != nil { 253 ok, err = dw.doFlush(flushingDWPT) 254 if err != nil { 255 return false, err 256 } 257 if ok { 258 hasEvents = true 259 } 260 } else { 261 nextPendingFlush := dw.flushControl.nextPendingFlush() 262 if nextPendingFlush != nil { 263 ok, err = dw.doFlush(nextPendingFlush) 264 if err != nil { 265 return false, err 266 } 267 if ok { 268 hasEvents = true 269 } 270 } 271 } 272 return hasEvents, nil 273 } 274 275 func (dw *DocumentsWriter) ensureInitialized(state *ThreadState) { 276 if state.isActive && state.dwpt == nil { 277 infos := model.NewFieldInfosBuilder(dw.writer.globalFieldNumberMap) 278 state.dwpt = newDocumentsWriterPerThread(dw.writer.newSegmentName(), 279 dw.directory, dw.config, dw.infoStream, dw.deleteQueue, infos, &dw.writer.pendingNumDocs) 280 } 281 } 282 283 // L428 284 func (dw *DocumentsWriter) updateDocument(doc []model.IndexableField, 285 analyzer analysis.Analyzer, delTerm *Term) (bool, error) { 286 287 hasEvents, err := dw.preUpdate() 288 if err != nil { 289 return false, err 290 } 291 292 flushingDWPT, err := func() (*DocumentsWriterPerThread, error) { 293 perThread := dw.flushControl.obtainAndLock() 294 defer dw.flushControl.perThreadPool.release(perThread) 295 296 if !perThread.isActive { 297 dw.ensureOpen() 298 panic("perThread is not active but we are still open") 299 } 300 dw.ensureInitialized(perThread) 301 assert(perThread.dwpt != nil) 302 dwpt := perThread.dwpt 303 dwptNuMDocs := dwpt.numDocsInRAM 304 305 err := func() error { 306 defer func() { 307 // We don't know whether the document actually counted as 308 // being indexed, so we must subtract here to accumulate our 309 // separate counter: 310 atomic.AddInt32(&dw.numDocsInRAM, int32(dwpt.numDocsInRAM-dwptNuMDocs)) 311 if dwpt.checkAndResetHasAborted() { 312 if len(dwpt.filesToDelete) > 0 { 313 dw.putEvent(newDeleteNewFilesEvent(dwpt.filesToDelete)) 314 } 315 dw.subtractFlushedNumDocs(dwptNuMDocs) 316 dw.flushControl.doOnAbort(perThread) 317 } 318 }() 319 320 err := dwpt.updateDocument(doc, analyzer, delTerm) 321 if err != nil { 322 return err 323 } 324 return nil 325 }() 326 if err != nil { 327 return nil, err 328 } 329 330 isUpdate := delTerm != nil 331 return dw.flushControl.doAfterDocument(perThread, isUpdate), nil 332 }() 333 if err != nil { 334 return false, err 335 } 336 337 return dw.postUpdate(flushingDWPT, hasEvents) 338 } 339 340 func (dw *DocumentsWriter) doFlush(flushingDWPT *DocumentsWriterPerThread) (bool, error) { 341 var hasEvents = false 342 for flushingDWPT != nil { 343 hasEvents = true 344 stop, err := func() (bool, error) { 345 defer func() { 346 dw.flushControl.doAfterFlush(flushingDWPT) 347 flushingDWPT.checkAndResetHasAborted() 348 }() 349 350 assertn(dw.currentFullFlushDelQueue == nil || 351 flushingDWPT.deleteQueue == dw.currentFullFlushDelQueue, 352 "expected: %v but was %v %v", 353 dw.currentFullFlushDelQueue, 354 flushingDWPT.deleteQueue, 355 dw.flushControl.fullFlush) 356 357 /* 358 Since, with DWPT, the flush process is concurrent and several 359 DWPT could flush at the same time, we must maintain the order 360 or the flushes before we can apply the flushed segment and 361 the frozen global deletes it is buffering. The reason for 362 this is that the global deletes mark a certain point in time 363 where we took a DWPT out of rotation and freeze the global 364 deletes. 365 366 Example: A flush 'A' starts and freezes the global deletes, 367 then flush 'B' starts and freezes all deletes occurred since 368 'A' has started. If 'B' finishes before 'A', we need to wait 369 until 'A' is done, otherwise the deletes frozen by 'B' are 370 not applied to 'A' and we might miss to deletes documents in 371 'A'. 372 */ 373 374 err := func() error { 375 var success = false 376 var ticket *SegmentFlushTicket 377 defer func() { 378 if !success && ticket != nil { 379 // In the case of a failure, make sure we are making 380 // progress and apply all the deletes since the segment 381 // flush failed since the flush ticket could hold global 382 // deletes. See FlushTicket.canPublish(). 383 dw.ticketQueue.markTicketFailed(ticket) 384 } 385 }() 386 387 // Each flush is assigned a ticket in the order they acquire the ticketQueue lock 388 ticket = dw.ticketQueue.addFlushTicket(flushingDWPT) 389 390 flushingDocsInRAM := flushingDWPT.numDocsInRAM 391 err := func() error { 392 var dwptSuccess = false 393 defer func() { 394 dw.subtractFlushedNumDocs(flushingDocsInRAM) 395 if len(flushingDWPT.filesToDelete) > 0 { 396 dw.putEvent(newDeleteNewFilesEvent(flushingDWPT.filesToDelete)) 397 hasEvents = true 398 } 399 if !dwptSuccess { 400 dw.putEvent(newFlushFailedEvent(flushingDWPT.segmentInfo)) 401 hasEvents = true 402 } 403 }() 404 405 // flush concurrently without locking 406 newSegment, err := flushingDWPT.flush() 407 if err != nil { 408 return err 409 } 410 dw.ticketQueue.addSegment(ticket, newSegment) 411 dwptSuccess = true 412 return nil 413 }() 414 if err != nil { 415 return err 416 } 417 // flush was successful once we reached this point - new seg. 418 // has been assigned to the ticket! 419 success = true 420 return nil 421 }() 422 if err != nil { 423 return false, err 424 } 425 // Now we are done and try to flush the ticket queue if the 426 // head of the queue has already finished the flush. 427 if dw.ticketQueue.ticketCount() >= dw.perThreadPool.numActiveThreadState() { 428 // This means there is a backlog: the one thread in 429 // innerPurge can't keep up with all other threads flusing 430 // segments. In this case we forcefully stall the producers. 431 dw.putEvent(forcedPurgeEvent) 432 return true, nil 433 } 434 return false, nil 435 }() 436 if err != nil { 437 return false, err 438 } 439 if stop { 440 break 441 } 442 443 flushingDWPT = dw.flushControl.nextPendingFlush() 444 } 445 if hasEvents { 446 dw.putEvent(mergePendingEvent) 447 } 448 // If deletes alone are consuming > 1/2 our RAM buffer, force them 449 // all to apply now. This is to prevent too-frequent flushing of a 450 // long tail tiny segments: 451 if ramBufferSizeMB := dw.config.RAMBufferSizeMB(); ramBufferSizeMB != DISABLE_AUTO_FLUSH && 452 dw.flushControl.deleteBytesUsed() > int64(1024*1024*ramBufferSizeMB/2) { 453 454 if dw.infoStream.IsEnabled("DW") { 455 dw.infoStream.Message("DW", "force apply deletes bytesUsed=%v vs ramBuffer=%v", 456 dw.flushControl.deleteBytesUsed(), 1024*1024*ramBufferSizeMB) 457 } 458 hasEvents = true 459 ok, err := dw.applyAllDeletes(dw.deleteQueue) 460 if err != nil { 461 return false, err 462 } 463 if !ok { 464 dw.putEvent(applyDeletesEvent) 465 } 466 } 467 468 return hasEvents, nil 469 } 470 471 func (dw *DocumentsWriter) subtractFlushedNumDocs(numFlushed int) { 472 oldValue := atomic.LoadInt32(&dw.numDocsInRAM) 473 for !atomic.CompareAndSwapInt32(&dw.numDocsInRAM, oldValue, oldValue-int32(numFlushed)) { 474 oldValue = atomic.LoadInt32(&dw.numDocsInRAM) 475 } 476 assert(atomic.LoadInt32(&dw.numDocsInRAM) >= 0) 477 } 478 479 /* 480 FlushAllThreads is synced by IW fullFlushLock. Flushing all threads 481 is a two stage operation; the caller must ensure (in try/finally) 482 that finishFLush is called after this method, to release the flush 483 lock in DWFlushControl 484 */ 485 func (dw *DocumentsWriter) flushAllThreads(indexWriter *IndexWriter) (bool, error) { 486 if dw.infoStream.IsEnabled("DW") { 487 dw.infoStream.Message("DW", "startFullFlush") 488 } 489 490 flushingDeleteQueue := func() *DocumentsWriterDeleteQueue { 491 dw.Lock() 492 defer dw.Unlock() 493 dw.pendingChangesInCurrentFullFlush = dw.anyChanges() 494 dq := dw.deleteQueue 495 // Cut over to a new delete queue. This must be synced on the 496 // flush control otherwise a new DWPT could sneak into the loop 497 // with an already flushing delete queue 498 dw.flushControl.markForFullFlush() // swaps the delQueue synced on FlushControl 499 dw.currentFullFlushDelQueue = dq 500 return dq 501 }() 502 assert(dw.currentFullFlushDelQueue != nil) 503 assert(dw.currentFullFlushDelQueue != dw.deleteQueue) 504 505 return func() (bool, error) { 506 anythingFlushed := false 507 defer func() { assert(flushingDeleteQueue == dw.currentFullFlushDelQueue) }() 508 509 flushingDWPT := dw.flushControl.nextPendingFlush() 510 for flushingDWPT != nil { 511 flushed, err := dw.doFlush(flushingDWPT) 512 if err != nil { 513 return false, err 514 } 515 if flushed { 516 anythingFlushed = true 517 } 518 next := dw.flushControl.nextPendingFlush() 519 assert(next != flushingDWPT) 520 flushingDWPT = next 521 } 522 523 // If a concurrent flush is still in flight wait for it 524 dw.flushControl.waitForFlush() 525 if !anythingFlushed && flushingDeleteQueue.anyChanges() { 526 // apply deletes if we did not flush any document 527 if dw.infoStream.IsEnabled("DW") { 528 dw.infoStream.Message("DW", "flush naked frozen global deletes") 529 } 530 err := dw.ticketQueue.addDeletes(flushingDeleteQueue) 531 if err != nil { 532 return false, err 533 } 534 } 535 _, err := dw.ticketQueue.forcePurge(indexWriter) 536 if err != nil { 537 return false, err 538 } 539 assert(!flushingDeleteQueue.anyChanges() && !dw.ticketQueue.hasTickets()) 540 return anythingFlushed, nil 541 }() 542 } 543 544 func (dw *DocumentsWriter) finishFullFlush(success bool) { 545 defer func() { dw.pendingChangesInCurrentFullFlush = false }() 546 if dw.infoStream.IsEnabled("DW") { 547 dw.infoStream.Message("DW", "finishFullFlush success=%v", success) 548 } 549 dw.currentFullFlushDelQueue = nil 550 if success { 551 // Release the flush lock 552 dw.flushControl.finishFullFlush() 553 } else { 554 newFilesSet := make(map[string]bool) 555 dw.flushControl.abortFullFlushes(newFilesSet) 556 dw.putEvent(newDeleteNewFilesEvent(newFilesSet)) 557 } 558 } 559 560 func (dw *DocumentsWriter) putEvent(event Event) { 561 dw.eventsLock.Lock() 562 defer dw.eventsLock.Unlock() 563 dw.events.PushBack(event) 564 } 565 566 func (dw *DocumentsWriter) processEvents(writer *IndexWriter, 567 triggerMerge, forcePurge bool) (processed bool, err error) { 568 dw.eventsLock.RLock() 569 defer dw.eventsLock.RUnlock() 570 571 for e := dw.events.Front(); e != nil; e = e.Next() { 572 dw.events.Remove(e) 573 processed = true 574 if err = e.Value.(Event)(writer, triggerMerge, forcePurge); err != nil { 575 break 576 } 577 } 578 return 579 } 580 581 func (dw *DocumentsWriter) assertEventQueueAfterClose() { 582 dw.eventsLock.RLock() 583 defer dw.eventsLock.RUnlock() 584 585 for e := dw.events.Front(); e != nil; e = e.Next() { 586 // TODO find a better way to compare event type 587 assert(fmt.Sprintf("%v", e.Value) == fmt.Sprintf("%v", mergePendingEvent)) 588 } 589 }