github.com/balzaczyy/golucene@v0.0.0-20151210033525-d0be9ee89713/core/index/flushControl.go (about) 1 package index 2 3 import ( 4 "container/list" 5 "fmt" 6 "github.com/balzaczyy/golucene/core/util" 7 "math" 8 "sync" 9 "sync/atomic" 10 ) 11 12 // index/DocumentsWriterFlushControl.java 13 14 /* 15 This class controls DocumentsWriterPerThread (DWPT) flushing during 16 indexing. It tracks the memory consumption per DWPT and uses a 17 configured FlushPolicy to decide if a DWPT must flush. 18 19 In addition to the FlushPolicy the flush control might set certain 20 DWPT as flush pending iff a DWPT exceeds the RAMPerThreadHardLimitMB() 21 to prevent address space exhaustion. 22 */ 23 type DocumentsWriterFlushControl struct { 24 sync.Locker 25 condFlushWait *sync.Cond 26 27 hardMaxBytesPerDWPT int64 28 _activeBytes int64 29 _flushBytes int64 30 numPending int // volatile 31 numDocsSinceStalled int 32 flushDeletes int32 // atomic bool 33 fullFlush bool 34 flushQueue *list.List 35 // only for safety reasons if a DWPT is close to the RAM limit 36 blockedFlushes *list.List 37 flushingWriters map[*DocumentsWriterPerThread]int64 38 39 maxConfiguredRamBuffer float64 40 peakActiveBytes int64 // assert only 41 peakFlushBytes int64 // assert only 42 peakNetBytes int64 // assert only 43 peakDelta int64 // assert only 44 flushByRAMWasDisabled bool // assert only 45 46 *DocumentsWriterStallControl // mixin 47 48 perThreadPool *DocumentsWriterPerThreadPool 49 flushPolicy FlushPolicy 50 closed bool 51 52 documentsWriter *DocumentsWriter 53 config LiveIndexWriterConfig 54 bufferedUpdatesStream *BufferedUpdatesStream 55 infoStream util.InfoStream 56 57 fullFlushBuffer []*DocumentsWriterPerThread 58 } 59 60 func newDocumentsWriterFlushControl(documentsWriter *DocumentsWriter, 61 config LiveIndexWriterConfig, 62 bufferedUpdatesStream *BufferedUpdatesStream) *DocumentsWriterFlushControl { 63 64 objLock := &sync.Mutex{} 65 ans := &DocumentsWriterFlushControl{ 66 Locker: objLock, 67 condFlushWait: sync.NewCond(objLock), 68 flushQueue: list.New(), 69 blockedFlushes: list.New(), 70 flushingWriters: make(map[*DocumentsWriterPerThread]int64), 71 infoStream: config.InfoStream(), 72 perThreadPool: documentsWriter.perThreadPool, 73 flushPolicy: documentsWriter.flushPolicy, 74 config: config, 75 hardMaxBytesPerDWPT: int64(config.RAMPerThreadHardLimitMB()) * 1024 * 1024, 76 documentsWriter: documentsWriter, 77 bufferedUpdatesStream: bufferedUpdatesStream, 78 } 79 ans.DocumentsWriterStallControl = newDocumentsWriterStallControl() 80 return ans 81 } 82 83 func (fc *DocumentsWriterFlushControl) activeBytes() int64 { 84 fc.Lock() 85 defer fc.Unlock() 86 return fc._activeBytes 87 } 88 89 func (fc *DocumentsWriterFlushControl) flushBytes() int64 { 90 fc.Lock() 91 defer fc.Unlock() 92 return fc._flushBytes 93 } 94 95 func (fc *DocumentsWriterFlushControl) netBytes() int64 { 96 fc.Lock() 97 defer fc.Unlock() 98 return fc._netBytes() 99 } 100 101 func (fc *DocumentsWriterFlushControl) _netBytes() int64 { 102 return fc._activeBytes + fc._flushBytes 103 } 104 105 func (fc *DocumentsWriterFlushControl) assertMemory() { 106 maxRamMB := fc.config.RAMBufferSizeMB() 107 if fc.flushByRAMWasDisabled || maxRamMB == DISABLE_AUTO_FLUSH { 108 fc.flushByRAMWasDisabled = true 109 return 110 } 111 // for this assert we must be tolerant to ram buffer changes! 112 if maxRamMB > fc.maxConfiguredRamBuffer { 113 fc.maxConfiguredRamBuffer = maxRamMB 114 } 115 ram := fc._flushBytes + fc._activeBytes 116 ramBufferBytes := int64(fc.maxConfiguredRamBuffer) * 1024 * 1024 117 // take peakDelta into account - worst case is that all flushing, 118 // pending and blocked DWPT had maxMem and the last doc had the 119 // peakDelta 120 121 /* 122 2 * ramBufferBytes 123 -> before we stall we need to across the 2xRAM Buffer border 124 this is still a valid limit 125 (numPending + numFlusingDWPT() + numBlockedFlushes()) * peakDelta 126 -> those are the total number of DWPT that are not active but 127 not yet fully flushed. all of them could theoretically be taken 128 out of the loop once they crossed the RAM buffer and the last 129 document was the peak delta 130 (numDocsSinceStalled * peakDelta) 131 -> at any given time, there could be n threads in flight that 132 crossed the stall control before we reached the limit and each 133 of them could hold a peak document 134 */ 135 expected := 2*ramBufferBytes + int64(fc.numPending+len(fc.flushingWriters)+ 136 fc.blockedFlushes.Len())*fc.peakDelta + 137 int64(fc.numDocsSinceStalled)*fc.peakDelta 138 139 // the expected ram consumption is an upper bound at this point and 140 // not really the expected consumption 141 if fc.peakDelta < (ramBufferBytes >> 1) { 142 /* 143 If we are indexing with very low maxRamBuffer, like 0.1MB. 144 Memory can easily overflow if we check out some DWPT based on 145 docCount and have several DWPT in flight indexing large 146 documents (compared to the ram buffer). This means that those 147 DWPT and their threads will not hit the stall control before 148 asserting the memory which would in turn fail. To prevent this 149 we only assert if the largest document seen is smaller tan the 150 1/2 of the maxRamBufferMB 151 */ 152 assertn(ram <= expected, 153 "actual mem: %v, expected mem: %v, flush mem: %v, active mem: %v, "+ 154 "pending DWPT: %v, flushing DWPT: %v, blocked DWPT: %v, "+ 155 "peakDelta mem: %v bytes, ramBufferBytes=%v, maxConfiguredRamBuffer=%v", 156 ram, expected, fc._flushBytes, fc._activeBytes, fc.numPending, 157 len(fc.flushingWriters), fc.blockedFlushes.Len(), fc.peakDelta, 158 ramBufferBytes, fc.maxConfiguredRamBuffer) 159 } 160 } 161 162 func (fc *DocumentsWriterFlushControl) commitPerThreadBytes(perThread *ThreadState) { 163 delta := perThread.dwpt.bytesUsed() - perThread.bytesUsed 164 perThread.bytesUsed += delta 165 // We need to differentiate here if we are pending since 166 // setFlushPending moves the perThread memory to the flushBytes and 167 // we could be set to pending during a delete 168 if perThread.flushPending { 169 fc._flushBytes += delta 170 } else { 171 fc._activeBytes += delta 172 } 173 fc.assertUpdatePeaks(delta) 174 } 175 176 func (fc *DocumentsWriterFlushControl) assertUpdatePeaks(delta int64) { 177 if fc.peakActiveBytes < fc._activeBytes { 178 fc.peakActiveBytes = fc._activeBytes 179 } 180 if fc.peakFlushBytes < fc._flushBytes { 181 fc.peakFlushBytes = fc._flushBytes 182 } 183 if n := fc._netBytes(); fc.peakNetBytes < n { 184 fc.peakNetBytes = n 185 } 186 if fc.peakDelta < delta { 187 fc.peakDelta = delta 188 } 189 } 190 191 func (fc *DocumentsWriterFlushControl) doAfterDocument(perThread *ThreadState, isUpdate bool) *DocumentsWriterPerThread { 192 fc.Lock() 193 defer fc.Unlock() 194 195 defer func() { 196 stalled := fc.updateStallState() 197 fc.assertNumDocsSinceStalled(stalled) 198 fc.assertMemory() 199 }() 200 201 fc.commitPerThreadBytes(perThread) 202 if !perThread.flushPending { 203 if isUpdate { 204 fc.flushPolicy.onUpdate(fc, perThread) 205 } else { 206 fc.flushPolicy.onInsert(fc, perThread) 207 } 208 if !perThread.flushPending && perThread.bytesUsed > fc.hardMaxBytesPerDWPT { 209 // Safety check to prevent a single DWPT exceeding its RAM 210 // limit. This is super important since we can not address more 211 // than 2048 MB per DWPT 212 fc._setFlushPending(perThread) 213 } 214 } 215 var flushingDWPT *DocumentsWriterPerThread 216 if fc.fullFlush { 217 if perThread.flushPending { 218 fc.checkoutAndBlock(perThread) 219 flushingDWPT = fc.nextPendingFlush() 220 } 221 } else { 222 flushingDWPT = fc._tryCheckoutForFlush(perThread) 223 } 224 return flushingDWPT 225 } 226 227 /* 228 updates the number of documents "finished" while we are in a stalled 229 state. this is important for asserting memory upper bounds since it 230 corresponds to the number of threads that are in-flight and crossed 231 the stall control check before we actually stalled. 232 */ 233 func (fc *DocumentsWriterFlushControl) assertNumDocsSinceStalled(stalled bool) { 234 if fc.stalled { 235 fc.numDocsSinceStalled++ 236 } else { 237 fc.numDocsSinceStalled = 0 238 } 239 } 240 241 func (fc *DocumentsWriterFlushControl) doAfterFlush(dwpt *DocumentsWriterPerThread) { 242 fc.Lock() 243 defer fc.Unlock() 244 245 bytes, ok := fc.flushingWriters[dwpt] 246 assert(ok) 247 248 defer func() { 249 defer func() { 250 fc.condFlushWait.Signal() 251 }() 252 fc.updateStallState() 253 }() 254 255 delete(fc.flushingWriters, dwpt) 256 fc._flushBytes -= bytes 257 // fc.perThreadPool.recycle(dwpt) 258 fc.assertMemory() 259 } 260 261 func (fc *DocumentsWriterFlushControl) updateStallState() bool { 262 var limit int64 = math.MaxInt64 263 if maxRamMB := fc.config.RAMBufferSizeMB(); maxRamMB != DISABLE_AUTO_FLUSH { 264 limit = int64(2 * 1024 * 1024 * maxRamMB) 265 } 266 // We block indexing threads if net byte grows due to slow flushes, 267 // yet, for small ram buffers and large documents, we can easily 268 // reach the limit without any ongoing flushes. We need ensure that 269 // we don't stall/block if an ongoing or pending flush can not free 270 // up enough memory to release the stall lock. 271 stall := (fc._activeBytes+fc._flushBytes) > limit && 272 fc._activeBytes < limit && 273 !fc.closed 274 fc.updateStalled(stall) 275 return stall 276 } 277 278 func (fc *DocumentsWriterFlushControl) waitForFlush() { 279 fc.Lock() 280 defer fc.Unlock() 281 282 for len(fc.flushingWriters) > 0 { 283 fc.condFlushWait.Wait() 284 } 285 } 286 287 /* 288 Sets flush pending state on the given ThreadState. The ThreadState 289 must have indexed at least one Document and must not be already 290 pending. 291 */ 292 func (fc *DocumentsWriterFlushControl) setFlushPending(perThread *ThreadState) { 293 fc.Lock() 294 defer fc.Unlock() 295 fc._setFlushPending(perThread) 296 } 297 298 func (fc *DocumentsWriterFlushControl) _setFlushPending(perThread *ThreadState) { 299 assert(!perThread.flushPending) 300 if perThread.dwpt.numDocsInRAM > 0 { 301 perThread.flushPending = true // write access synced 302 bytes := perThread.bytesUsed 303 fc._flushBytes += bytes 304 fc._activeBytes -= bytes 305 fc.numPending++ // write access synced 306 fc.assertMemory() 307 } 308 // don't assert on numDocs since we could hit an abort except while 309 // selecting that dwpt for flushing 310 } 311 312 func (fc *DocumentsWriterFlushControl) doOnAbort(state *ThreadState) { 313 fc.Lock() 314 defer fc.Unlock() 315 defer fc.updateStallState() 316 if state.flushPending { 317 fc._flushBytes -= state.bytesUsed 318 } else { 319 fc._activeBytes -= state.bytesUsed 320 } 321 fc.assertMemory() 322 // Take it out of the loop this DWPT is stale 323 fc.perThreadPool.reset(state, fc.closed) 324 } 325 326 func (fc *DocumentsWriterFlushControl) tryCheckoutForFlush(perThread *ThreadState) *DocumentsWriterPerThread { 327 fc.Lock() 328 defer fc.Unlock() 329 return fc._tryCheckoutForFlush(perThread) 330 } 331 332 func (fc *DocumentsWriterFlushControl) _tryCheckoutForFlush(perThread *ThreadState) *DocumentsWriterPerThread { 333 if perThread.flushPending { 334 return fc.internalTryCheckOutForFlush(perThread) 335 } 336 return nil 337 } 338 339 func (fc *DocumentsWriterFlushControl) checkoutAndBlock(perThread *ThreadState) { 340 // perThread is already locked 341 assert2(perThread.flushPending, "can not block non-pending threadstate") 342 assert2(fc.fullFlush, "can not block if fullFlush == false") 343 bytes := perThread.bytesUsed 344 dwpt := fc.perThreadPool.reset(perThread, fc.closed) 345 fc.numPending-- 346 fc.blockedFlushes.PushBack(&BlockedFlush{dwpt, bytes}) 347 } 348 349 func (fc *DocumentsWriterFlushControl) internalTryCheckOutForFlush(perThread *ThreadState) *DocumentsWriterPerThread { 350 // perThread is already locked 351 assert(perThread.flushPending) 352 defer fc.updateStallState() 353 // We are pending so all memory is already moved to flushBytes 354 if perThread.isActive && perThread.dwpt != nil { 355 bytes := perThread.bytesUsed // do that before replace 356 dwpt := fc.perThreadPool.reset(perThread, fc.closed) 357 _, ok := fc.flushingWriters[dwpt] 358 assert2(!ok, "DWPT is already flushing") 359 // Record the flushing DWPT to reduce flushBytes in doAfterFlush 360 fc.flushingWriters[dwpt] = bytes 361 fc.numPending-- // write access synced 362 return dwpt 363 } 364 return nil 365 } 366 367 func (fc *DocumentsWriterFlushControl) String() string { 368 return fmt.Sprintf("DocumentsWriterFlushControl [activeBytes=%v, flushBytes=%v]", 369 fc.activeBytes, fc.flushBytes) 370 } 371 372 func (fc *DocumentsWriterFlushControl) close() { 373 fc.Lock() 374 defer fc.Unlock() 375 // set by DW to signal that we should not release new DWPT after close 376 if !fc.closed { 377 fc.closed = true 378 } 379 } 380 381 /* Various statistics */ 382 383 func (fc *DocumentsWriterFlushControl) deleteBytesUsed() int64 { 384 return fc.documentsWriter.deleteQueue.RamBytesUsed() + fc.bufferedUpdatesStream.RamBytesUsed() 385 } 386 387 // L444 388 389 func (fc *DocumentsWriterFlushControl) obtainAndLock() *ThreadState { 390 perThread := fc.perThreadPool.lockAny() 391 var success = false 392 defer func() { 393 if !success { 394 fc.perThreadPool.release(perThread) 395 } 396 }() 397 398 if perThread.isActive && 399 perThread.dwpt != nil && 400 perThread.dwpt.deleteQueue != fc.documentsWriter.deleteQueue { 401 402 // Threre is a flush-all in process and this DWPT is now stale -- 403 // enroll it for flush and try for another DWPT: 404 fc.addFlushableState(perThread) 405 } 406 success = true 407 // simply return the ThreadState even in a flush all case since we 408 // already hold the lock 409 return perThread 410 } 411 412 func (fc *DocumentsWriterFlushControl) markForFullFlush() { 413 flushingQueue := func() *DocumentsWriterDeleteQueue { 414 fc.Lock() 415 defer fc.Unlock() 416 417 assert2(!fc.fullFlush, "called DWFC#markForFullFlush() while full flush is still running") 418 assertn(len(fc.fullFlushBuffer) == 0, "full flush buffer should be empty: ", fc.fullFlushBuffer) 419 420 fc.fullFlush = true 421 res := fc.documentsWriter.deleteQueue 422 // Set a new delete queue - all subsequent DWPT will use this 423 // queue untiil we do another full flush 424 fc.documentsWriter.deleteQueue = newDocumentsWriterDeleteQueueWithGeneration(res.generation + 1) 425 return res 426 }() 427 428 fc.perThreadPool.foreach(func(next *ThreadState) { 429 if !next.isActive || next.dwpt == nil { 430 if fc.closed && next.isActive { 431 next.deactivate() 432 } 433 return 434 } 435 assertn(next.dwpt.deleteQueue == flushingQueue || 436 next.dwpt.deleteQueue == fc.documentsWriter.deleteQueue, 437 " flushingQueue: %v currentQueue: %v perThread queue: %v numDocsInRAM: %v", 438 flushingQueue, fc.documentsWriter.deleteQueue, next.dwpt.deleteQueue, 439 next.dwpt.numDocsInRAM) 440 if next.dwpt.deleteQueue != flushingQueue { 441 // this one is already a new DWPT 442 return 443 } 444 fc.addFlushableState(next) 445 }) 446 447 func() { 448 fc.Lock() 449 defer fc.Unlock() 450 451 // make sure we move all DWPT that are where concurrently marked 452 // as pending and moved to blocked are moved over to the 453 // flushQueue. There is a chance that this happens since we 454 // marking DWPT for full flush without blocking indexing. 455 fc.pruneBlockedQueue(flushingQueue) 456 fc.assertBlockedFlushes(fc.documentsWriter.deleteQueue) 457 for _, dwpt := range fc.fullFlushBuffer { 458 fc.flushQueue.PushBack(dwpt) 459 } 460 fc.fullFlushBuffer = nil 461 fc.updateStallState() 462 }() 463 fc.assertActiveDeleteQueue(fc.documentsWriter.deleteQueue) 464 } 465 466 func (fc *DocumentsWriterFlushControl) assertActiveDeleteQueue(queue *DocumentsWriterDeleteQueue) { 467 fc.perThreadPool.foreach(func(next *ThreadState) { 468 n := 0 469 if next.dwpt != nil { 470 n = next.dwpt.numDocsInRAM 471 } 472 assertn(!next.isActive || next.dwpt == nil || next.dwpt.deleteQueue == queue, 473 "isInitialized: %v numDocs: %v", next.isActive && next.dwpt != nil, n) 474 }) 475 } 476 477 func (fc *DocumentsWriterFlushControl) nextPendingFlush() *DocumentsWriterPerThread { 478 numPending, fullFlush, dwpt := func() (int, bool, *DocumentsWriterPerThread) { 479 fc.Lock() 480 defer fc.Unlock() 481 482 if e := fc.flushQueue.Front(); e != nil { 483 fc.flushQueue.Remove(e) 484 pool := e.Value.(*DocumentsWriterPerThread) 485 fc.updateStallState() 486 return 0, false, pool 487 } 488 return fc.numPending, fc.fullFlush, nil 489 }() 490 if dwpt != nil { 491 return dwpt 492 } 493 494 if numPending > 0 && !fullFlush { 495 // don't check if we are doing a full flush 496 dwpt = fc.perThreadPool.find(func(next *ThreadState) interface{} { 497 if numPending > 0 && next.flushPending { 498 if dwpt := fc.tryCheckoutForFlush(next); dwpt != nil { 499 return dwpt 500 } 501 } 502 return nil 503 }).(*DocumentsWriterPerThread) 504 } 505 return dwpt 506 } 507 508 func (fc *DocumentsWriterFlushControl) addFlushableState(perThread *ThreadState) { 509 if fc.infoStream.IsEnabled("DWFC") { 510 fc.infoStream.Message("DWFC", "addFlushableState %v", perThread.dwpt) 511 } 512 dwpt := perThread.dwpt 513 assert(perThread.isActive && perThread.dwpt != nil) 514 assert(fc.fullFlush) 515 assert(dwpt.deleteQueue != fc.documentsWriter.deleteQueue) 516 if dwpt.numDocsInRAM > 0 { 517 func() { 518 fc.Lock() 519 defer fc.Unlock() 520 if !perThread.flushPending { 521 fc._setFlushPending(perThread) 522 } 523 flushingDWPT := fc.internalTryCheckOutForFlush(perThread) 524 assert2(flushingDWPT != nil, "DWPT must never be null here since we hold the lock and it holds documents") 525 assert2(dwpt == flushingDWPT, "flushControl returned different DWPT") 526 fc.fullFlushBuffer = append(fc.fullFlushBuffer, flushingDWPT) 527 }() 528 } else { 529 fc.perThreadPool.reset(perThread, fc.closed) // make this state inactive 530 } 531 } 532 533 func (fc *DocumentsWriterFlushControl) getAndResetApplyAllDeletes() bool { 534 return atomic.SwapInt32(&fc.flushDeletes, 0) == 1 535 } 536 537 /* 538 Prunes the blockedQueue by removing all DWPT that are associated with 539 the given flush queue. 540 */ 541 func (fc *DocumentsWriterFlushControl) pruneBlockedQueue(flushingQueue *DocumentsWriterDeleteQueue) { 542 for e := fc.blockedFlushes.Front(); e != nil; e = e.Next() { 543 if blockedFlush := e.Value.(*BlockedFlush); blockedFlush.dwpt.deleteQueue == flushingQueue { 544 fc.blockedFlushes.Remove(e) 545 _, ok := fc.flushingWriters[blockedFlush.dwpt] 546 assert2(!ok, "DWPT is already flushing") 547 // Record the flushing DWPT to reduce flushBytes in doAfterFlush 548 fc.flushingWriters[blockedFlush.dwpt] = blockedFlush.bytes 549 // don't decr pending here - its already done when DWPT is blocked 550 fc.flushQueue.PushBack(blockedFlush.dwpt) 551 } 552 } 553 } 554 555 func (fc *DocumentsWriterFlushControl) finishFullFlush() { 556 fc.Lock() 557 defer fc.Unlock() 558 559 assert(fc.fullFlush) 560 assert(fc.flushQueue.Len() == 0) 561 assert(len(fc.flushingWriters) == 0) 562 563 defer func() { fc.fullFlush = false }() 564 565 if fc.blockedFlushes.Len() > 0 { 566 fc.assertBlockedFlushes(fc.documentsWriter.deleteQueue) 567 fc.pruneBlockedQueue(fc.documentsWriter.deleteQueue) 568 assert(fc.blockedFlushes.Len() == 0) 569 } 570 } 571 572 func (fc *DocumentsWriterFlushControl) abortFullFlushes(newFiles map[string]bool) { 573 fc.Lock() 574 defer fc.Unlock() 575 defer func() { fc.fullFlush = false }() 576 } 577 578 func (fc *DocumentsWriterFlushControl) assertBlockedFlushes(flushingQueue *DocumentsWriterDeleteQueue) { 579 for e := fc.blockedFlushes.Front(); e != nil; e = e.Next() { 580 blockedFlush := e.Value.(*BlockedFlush) 581 assert(blockedFlush.dwpt.deleteQueue == flushingQueue) 582 } 583 } 584 585 func (fc *DocumentsWriterFlushControl) abortPendingFlushes(newFiles map[string]bool) { 586 fc.Lock() 587 defer fc.Unlock() 588 589 defer func() { 590 fc.flushQueue.Init() 591 fc.blockedFlushes.Init() 592 fc.updateStallState() 593 }() 594 595 for e := fc.flushQueue.Front(); e != nil; e = e.Next() { 596 dwpt := e.Value.(*DocumentsWriterPerThread) 597 fc.documentsWriter.subtractFlushedNumDocs(dwpt.numDocsInRAM) 598 fc.doAfterFlush(dwpt) 599 } 600 601 for e := fc.blockedFlushes.Front(); e != nil; e = e.Next() { 602 blockedFlush := e.Value.(*BlockedFlush) 603 fc.flushingWriters[blockedFlush.dwpt] = blockedFlush.bytes 604 fc.documentsWriter.subtractFlushedNumDocs(blockedFlush.dwpt.numDocsInRAM) 605 blockedFlush.dwpt.abort(newFiles) 606 fc.doAfterFlush(blockedFlush.dwpt) 607 } 608 } 609 610 func (fc *DocumentsWriterFlushControl) numQueuedFlushes() int { 611 fc.Lock() 612 defer fc.Unlock() 613 return fc.flushQueue.Len() 614 } 615 616 type BlockedFlush struct { 617 dwpt *DocumentsWriterPerThread 618 bytes int64 619 } 620 621 /* 622 This mehtod will block if too many DWPT are currently flushing and no 623 checked out DWPT are available. 624 */ 625 func (fc *DocumentsWriterFlushControl) waitIfStalled() { 626 if fc.infoStream.IsEnabled("DWFC") { 627 fc.infoStream.Message( 628 "DWFC", "waitIfStalled: numFlushesPending: %v netBytes: %v flushingBytes: %v fullFlush: %v", 629 fc.flushQueue.Len(), fc.netBytes(), fc.flushBytes(), fc.fullFlush) 630 } 631 fc.DocumentsWriterStallControl.waitIfStalled() 632 }