mirror of
https://github.com/apache/lucene.git
synced 2025-02-10 12:05:36 +00:00
LUCENE-3023: some clean-up and nocommits
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/branches/realtime_search@1094916 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
f0b56fd92e
commit
01ffe0ba84
@ -208,11 +208,11 @@ class BufferedDeletesStream {
|
|||||||
}
|
}
|
||||||
if (!packet.isSegmentPrivate) {
|
if (!packet.isSegmentPrivate) {
|
||||||
/*
|
/*
|
||||||
* only update the coalescededDeletes if we are NOT on a segment private del packet.
|
* Only coalesce if we are NOT on a segment private del packet: the segment private del packet
|
||||||
* the segment private del packet must only applied to segments with the same delGen.
|
* must only applied to segments with the same delGen. Yet, if a segment is already deleted
|
||||||
* Yet, if a segment is already deleted from the SI since it had no more documents remaining
|
* from the SI since it had no more documents remaining after some del packets younger than
|
||||||
* after some del packets younger than it segPrivate packet (hihger delGen) have been applied
|
* its segPrivate packet (higher delGen) have been applied, the segPrivate packet has not been
|
||||||
* the segPrivate packet has not been removed.
|
* removed.
|
||||||
*/
|
*/
|
||||||
coalescedDeletes.update(packet);
|
coalescedDeletes.update(packet);
|
||||||
}
|
}
|
||||||
@ -259,7 +259,7 @@ class BufferedDeletesStream {
|
|||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* since we are on a segment private del packet we must not
|
* Since we are on a segment private del packet we must not
|
||||||
* update the coalescedDeletes here! We can simply advance to the
|
* update the coalescedDeletes here! We can simply advance to the
|
||||||
* next packet and seginfo.
|
* next packet and seginfo.
|
||||||
*/
|
*/
|
||||||
|
@ -50,12 +50,10 @@ final class DocFieldProcessor extends DocConsumer {
|
|||||||
int hashMask = 1;
|
int hashMask = 1;
|
||||||
int totalFieldCount;
|
int totalFieldCount;
|
||||||
|
|
||||||
|
|
||||||
float docBoost;
|
float docBoost;
|
||||||
int fieldGen;
|
int fieldGen;
|
||||||
final DocumentsWriterPerThread.DocState docState;
|
final DocumentsWriterPerThread.DocState docState;
|
||||||
|
|
||||||
|
|
||||||
public DocFieldProcessor(DocumentsWriterPerThread docWriter, DocFieldConsumer consumer) {
|
public DocFieldProcessor(DocumentsWriterPerThread docWriter, DocFieldConsumer consumer) {
|
||||||
this.docState = docWriter.docState;
|
this.docState = docWriter.docState;
|
||||||
this.consumer = consumer;
|
this.consumer = consumer;
|
||||||
@ -254,7 +252,6 @@ final class DocFieldProcessor extends DocConsumer {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
void quickSort(DocFieldProcessorPerField[] array, int lo, int hi) {
|
void quickSort(DocFieldProcessorPerField[] array, int lo, int hi) {
|
||||||
if (lo >= hi)
|
if (lo >= hi)
|
||||||
return;
|
return;
|
||||||
|
@ -39,10 +39,7 @@ import org.apache.lucene.store.Directory;
|
|||||||
|
|
||||||
/**
|
/**
|
||||||
* This class accepts multiple added documents and directly
|
* This class accepts multiple added documents and directly
|
||||||
* writes a single segment file. It does this more
|
* writes segment files.
|
||||||
* efficiently than creating a single segment per document
|
|
||||||
* (with DocumentWriter) and doing standard merges on those
|
|
||||||
* segments.
|
|
||||||
*
|
*
|
||||||
* Each added document is passed to the {@link DocConsumer},
|
* Each added document is passed to the {@link DocConsumer},
|
||||||
* which in turn processes the document and interacts with
|
* which in turn processes the document and interacts with
|
||||||
@ -152,8 +149,11 @@ final class DocumentsWriter {
|
|||||||
}
|
}
|
||||||
|
|
||||||
synchronized boolean deleteQueries(final Query... queries) throws IOException {
|
synchronized boolean deleteQueries(final Query... queries) throws IOException {
|
||||||
final DocumentsWriterDeleteQueue deleteQueue = this.deleteQueue;
|
|
||||||
deleteQueue.addDelete(queries);
|
deleteQueue.addDelete(queries);
|
||||||
|
// nocommit -- shouldn't we check for doApplyAllDeletes
|
||||||
|
// here too?
|
||||||
|
// nocommit shouldn't this consult flush policy? or
|
||||||
|
// should this return void now?
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -165,9 +165,11 @@ final class DocumentsWriter {
|
|||||||
final DocumentsWriterDeleteQueue deleteQueue = this.deleteQueue;
|
final DocumentsWriterDeleteQueue deleteQueue = this.deleteQueue;
|
||||||
deleteQueue.addDelete(terms);
|
deleteQueue.addDelete(terms);
|
||||||
flushControl.doOnDelete();
|
flushControl.doOnDelete();
|
||||||
if (flushControl.flushDeletes.getAndSet(false)) {
|
if (flushControl.doApplyAllDeletes()) {
|
||||||
flushDeletes(deleteQueue);
|
applyAllDeletes(deleteQueue);
|
||||||
}
|
}
|
||||||
|
// nocommit shouldn't this consult flush policy? or
|
||||||
|
// should this return void now?
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -182,13 +184,13 @@ final class DocumentsWriter {
|
|||||||
return deleteQueue;
|
return deleteQueue;
|
||||||
}
|
}
|
||||||
|
|
||||||
private void flushDeletes(DocumentsWriterDeleteQueue deleteQueue) throws IOException {
|
private void applyAllDeletes(DocumentsWriterDeleteQueue deleteQueue) throws IOException {
|
||||||
if (deleteQueue != null) {
|
if (deleteQueue != null) {
|
||||||
synchronized (ticketQueue) {
|
synchronized (ticketQueue) {
|
||||||
// freeze and insert the delete flush ticket in the queue
|
// Freeze and insert the delete flush ticket in the queue
|
||||||
ticketQueue.add(new FlushTicket(deleteQueue.freezeGlobalBuffer(null), false));
|
ticketQueue.add(new FlushTicket(deleteQueue.freezeGlobalBuffer(null), false));
|
||||||
applyFlushTickets(null, null);
|
applyFlushTickets(null, null);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
indexWriter.applyAllDeletes();
|
indexWriter.applyAllDeletes();
|
||||||
indexWriter.flushCount.incrementAndGet();
|
indexWriter.flushCount.incrementAndGet();
|
||||||
@ -196,14 +198,9 @@ final class DocumentsWriter {
|
|||||||
|
|
||||||
synchronized void setInfoStream(PrintStream infoStream) {
|
synchronized void setInfoStream(PrintStream infoStream) {
|
||||||
this.infoStream = infoStream;
|
this.infoStream = infoStream;
|
||||||
pushConfigChange();
|
|
||||||
}
|
|
||||||
|
|
||||||
private final void pushConfigChange() {
|
|
||||||
final Iterator<ThreadState> it = perThreadPool.getAllPerThreadsIterator();
|
final Iterator<ThreadState> it = perThreadPool.getAllPerThreadsIterator();
|
||||||
while (it.hasNext()) {
|
while (it.hasNext()) {
|
||||||
DocumentsWriterPerThread perThread = it.next().perThread;
|
it.next().perThread.docState.infoStream = infoStream;
|
||||||
perThread.docState.infoStream = this.infoStream;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -218,8 +215,9 @@ final class DocumentsWriter {
|
|||||||
|
|
||||||
// returns boolean for asserts
|
// returns boolean for asserts
|
||||||
boolean message(String message) {
|
boolean message(String message) {
|
||||||
if (infoStream != null)
|
if (infoStream != null) {
|
||||||
indexWriter.message("DW: " + message);
|
indexWriter.message("DW: " + message);
|
||||||
|
}
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -297,45 +295,52 @@ final class DocumentsWriter {
|
|||||||
ensureOpen();
|
ensureOpen();
|
||||||
boolean maybeMerge = false;
|
boolean maybeMerge = false;
|
||||||
final boolean isUpdate = delTerm != null;
|
final boolean isUpdate = delTerm != null;
|
||||||
if (healthiness.isStalled()) {
|
if (healthiness.anyStalledThreads()) {
|
||||||
/*
|
|
||||||
* if we are allowed to hijack threads for flushing we try to flush out
|
// Help out flushing any pending DWPTs so we can un-stall:
|
||||||
* as many pending DWPT to release memory and get back healthy status.
|
|
||||||
*/
|
|
||||||
if (infoStream != null) {
|
if (infoStream != null) {
|
||||||
message("WARNING DocumentsWriter is stalled try to hijack thread to flush pending segment");
|
message("WARNING DocumentsWriter has stalled threads; will hijack this thread to flush pending segment(s)");
|
||||||
}
|
}
|
||||||
// try pick up pending threads here if possile
|
|
||||||
|
// Try pick up pending threads here if possible
|
||||||
DocumentsWriterPerThread flushingDWPT;
|
DocumentsWriterPerThread flushingDWPT;
|
||||||
while ( (flushingDWPT = flushControl.nextPendingFlush()) != null){
|
while ((flushingDWPT = flushControl.nextPendingFlush()) != null) {
|
||||||
// don't push the delete here since the update could fail!
|
// Don't push the delete here since the update could fail!
|
||||||
maybeMerge = doFlush(flushingDWPT);
|
maybeMerge = doFlush(flushingDWPT);
|
||||||
if (!healthiness.isStalled()) {
|
if (!healthiness.anyStalledThreads()) {
|
||||||
break;
|
break;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
if (infoStream != null && healthiness.isStalled()) {
|
|
||||||
message("WARNING DocumentsWriter is stalled might block thread until DocumentsWriter is not stalled anymore");
|
if (infoStream != null && healthiness.anyStalledThreads()) {
|
||||||
|
message("WARNING DocumentsWriter still has stalled threads; waiting");
|
||||||
}
|
}
|
||||||
|
|
||||||
healthiness.waitIfStalled(); // block if stalled
|
healthiness.waitIfStalled(); // block if stalled
|
||||||
|
|
||||||
|
if (infoStream != null && healthiness.anyStalledThreads()) {
|
||||||
|
message("WARNING DocumentsWriter done waiting");
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
final ThreadState perThread = perThreadPool.getAndLock(Thread.currentThread(),
|
final ThreadState perThread = perThreadPool.getAndLock(Thread.currentThread(),
|
||||||
this, doc);
|
this, doc);
|
||||||
final DocumentsWriterPerThread flushingDWPT;
|
final DocumentsWriterPerThread flushingDWPT;
|
||||||
final DocumentsWriterPerThread dwpt;
|
|
||||||
try {
|
try {
|
||||||
|
|
||||||
if (!perThread.isActive()) {
|
if (!perThread.isActive()) {
|
||||||
ensureOpen();
|
ensureOpen();
|
||||||
assert false: "perThread is not active but we are still open";
|
assert false: "perThread is not active but we are still open";
|
||||||
}
|
}
|
||||||
|
|
||||||
dwpt = perThread.perThread;
|
final DocumentsWriterPerThread dwpt = perThread.perThread;
|
||||||
try {
|
try {
|
||||||
dwpt.updateDocument(doc, analyzer, delTerm);
|
dwpt.updateDocument(doc, analyzer, delTerm);
|
||||||
numDocsInRAM.incrementAndGet();
|
numDocsInRAM.incrementAndGet();
|
||||||
} finally {
|
} finally {
|
||||||
if(dwpt.checkAndResetHasAborted()) {
|
if (dwpt.checkAndResetHasAborted()) {
|
||||||
flushControl.doOnAbort(perThread);
|
flushControl.doOnAbort(perThread);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
flushingDWPT = flushControl.doAfterDocument(perThread, isUpdate);
|
flushingDWPT = flushControl.doAfterDocument(perThread, isUpdate);
|
||||||
@ -376,46 +381,53 @@ final class DocumentsWriter {
|
|||||||
* might miss to deletes documents in 'A'.
|
* might miss to deletes documents in 'A'.
|
||||||
*/
|
*/
|
||||||
synchronized (ticketQueue) {
|
synchronized (ticketQueue) {
|
||||||
// each flush is assigned a ticket in the order they accquire the ticketQueue lock
|
// Each flush is assigned a ticket in the order they accquire the ticketQueue lock
|
||||||
ticket = new FlushTicket(flushingDWPT.prepareFlush(), true);
|
ticket = new FlushTicket(flushingDWPT.prepareFlush(), true);
|
||||||
ticketQueue.add(ticket);
|
ticketQueue.add(ticket);
|
||||||
}
|
}
|
||||||
|
|
||||||
// flush concurrently without locking
|
// flush concurrently without locking
|
||||||
final FlushedSegment newSegment = flushingDWPT.flush();
|
final FlushedSegment newSegment = flushingDWPT.flush();
|
||||||
|
|
||||||
|
// nocommit -- should this success = true be moved
|
||||||
|
// under the applyFlushTickets?
|
||||||
success = true;
|
success = true;
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* now we are done and try to flush the ticket queue if the head of the
|
* Now we are done and try to flush the ticket queue if the head of the
|
||||||
* queue has already finished the flush.
|
* queue has already finished the flush.
|
||||||
*/
|
*/
|
||||||
applyFlushTickets(ticket, newSegment);
|
applyFlushTickets(ticket, newSegment);
|
||||||
} finally {
|
} finally {
|
||||||
flushControl.doAfterFlush(flushingDWPT);
|
flushControl.doAfterFlush(flushingDWPT);
|
||||||
flushingDWPT.checkAndResetHasAborted();
|
flushingDWPT.checkAndResetHasAborted();
|
||||||
indexWriter.flushCount.incrementAndGet();
|
indexWriter.flushCount.incrementAndGet();
|
||||||
if (!success && ticket != null) {
|
if (!success && ticket != null) {
|
||||||
synchronized (ticketQueue) {
|
synchronized (ticketQueue) {
|
||||||
// in the case of a failure make sure we are making progress and
|
// nocommit -- shouldn't we drop the ticket in
|
||||||
// apply all the deletes since the segment flush failed
|
// this case?
|
||||||
ticket.isSegmentFlush = false;
|
// In the case of a failure make sure we are making progress and
|
||||||
|
// apply all the deletes since the segment flush failed
|
||||||
}
|
ticket.isSegmentFlush = false;
|
||||||
}
|
}
|
||||||
|
}
|
||||||
}
|
}
|
||||||
flushingDWPT = flushControl.nextPendingFlush() ;
|
flushingDWPT = flushControl.nextPendingFlush();
|
||||||
}
|
}
|
||||||
return maybeMerge;
|
return maybeMerge;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
private void applyFlushTickets(FlushTicket current, FlushedSegment segment) throws IOException {
|
private void applyFlushTickets(FlushTicket current, FlushedSegment segment) throws IOException {
|
||||||
synchronized (ticketQueue) {
|
synchronized (ticketQueue) {
|
||||||
if (current != null) {
|
if (current != null) {
|
||||||
// this is a segment FlushTicket so assign the flushed segment so we can make progress.
|
// nocommit -- can't caller set current.segment = segment?
|
||||||
|
// nocommit -- confused by this comment:
|
||||||
|
// This is a segment FlushTicket so assign the flushed segment so we can make progress.
|
||||||
assert segment != null;
|
assert segment != null;
|
||||||
current.segment = segment;
|
current.segment = segment;
|
||||||
}
|
}
|
||||||
while (true) {
|
while (true) {
|
||||||
// while we can publish flushes keep on making the queue empty.
|
// Keep publishing eligible flushed segments:
|
||||||
final FlushTicket head = ticketQueue.peek();
|
final FlushTicket head = ticketQueue.peek();
|
||||||
if (head != null && head.canPublish()) {
|
if (head != null && head.canPublish()) {
|
||||||
ticketQueue.poll();
|
ticketQueue.poll();
|
||||||
@ -426,11 +438,10 @@ final class DocumentsWriter {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
private void finishFlush(FlushedSegment newSegment, FrozenBufferedDeletes bufferedDeletes)
|
private void finishFlush(FlushedSegment newSegment, FrozenBufferedDeletes bufferedDeletes)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
// this is eventually finishing the flushed segment and publishing it to the IndexWriter
|
// Finish the flushed segment and publish it to IndexWriter
|
||||||
if (newSegment == null) {
|
if (newSegment == null) {
|
||||||
assert bufferedDeletes != null;
|
assert bufferedDeletes != null;
|
||||||
if (bufferedDeletes != null && bufferedDeletes.any()) {
|
if (bufferedDeletes != null && bufferedDeletes.any()) {
|
||||||
@ -442,9 +453,6 @@ final class DocumentsWriter {
|
|||||||
} else {
|
} else {
|
||||||
publishFlushedSegment(newSegment, bufferedDeletes);
|
publishFlushedSegment(newSegment, bufferedDeletes);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
final void subtractFlushedNumDocs(int numFlushed) {
|
final void subtractFlushedNumDocs(int numFlushed) {
|
||||||
@ -455,10 +463,10 @@ final class DocumentsWriter {
|
|||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* publishes the flushed segment, segment private deletes if any and its
|
* Publishes the flushed segment, segment private deletes (if any) and its
|
||||||
* associated global delete if present to the index writer. the actual
|
* associated global delete (if present) to IndexWriter. The actual
|
||||||
* publishing operation is synced on IW -> BDS so that the {@link SegmentInfo}
|
* publishing operation is synced on IW -> BDS so that the {@link SegmentInfo}'s
|
||||||
* 's delete generation is always GlobalPacket_deleteGeneration + 1
|
* delete generation is always GlobalPacket_deleteGeneration + 1
|
||||||
*/
|
*/
|
||||||
private void publishFlushedSegment(FlushedSegment newSegment, FrozenBufferedDeletes globalPacket)
|
private void publishFlushedSegment(FlushedSegment newSegment, FrozenBufferedDeletes globalPacket)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -467,12 +475,13 @@ final class DocumentsWriter {
|
|||||||
final BufferedDeletes deletes = newSegment.segmentDeletes;
|
final BufferedDeletes deletes = newSegment.segmentDeletes;
|
||||||
FrozenBufferedDeletes packet = null;
|
FrozenBufferedDeletes packet = null;
|
||||||
if (deletes != null && deletes.any()) {
|
if (deletes != null && deletes.any()) {
|
||||||
// segment private delete
|
// Segment private delete
|
||||||
packet = new FrozenBufferedDeletes(deletes, true);
|
packet = new FrozenBufferedDeletes(deletes, true);
|
||||||
if (infoStream != null) {
|
if (infoStream != null) {
|
||||||
message("flush: push buffered seg private deletes: " + packet);
|
message("flush: push buffered seg private deletes: " + packet);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// now publish!
|
// now publish!
|
||||||
indexWriter.publishFlushedSegment(segInfo, packet, globalPacket);
|
indexWriter.publishFlushedSegment(segInfo, packet, globalPacket);
|
||||||
}
|
}
|
||||||
@ -486,10 +495,9 @@ final class DocumentsWriter {
|
|||||||
}
|
}
|
||||||
|
|
||||||
/*
|
/*
|
||||||
* flushAllThreads is synced by IW fullFlushLock. Flushing all threads is a
|
* FlushAllThreads is synced by IW fullFlushLock. Flushing all threads is a
|
||||||
* two stage operations, the caller must ensure that #finishFlush is called
|
* two stage operation; the caller must ensure (in try/finally) that finishFlush
|
||||||
* after this method to release the flush lock in DWFlushControl - use try /
|
* is called after this method, to release the flush lock in DWFlushControl
|
||||||
* finally!
|
|
||||||
*/
|
*/
|
||||||
final boolean flushAllThreads(final boolean flushDeletes)
|
final boolean flushAllThreads(final boolean flushDeletes)
|
||||||
throws IOException {
|
throws IOException {
|
||||||
@ -497,9 +505,11 @@ final class DocumentsWriter {
|
|||||||
|
|
||||||
synchronized (this) {
|
synchronized (this) {
|
||||||
flushingDeleteQueue = deleteQueue;
|
flushingDeleteQueue = deleteQueue;
|
||||||
/* sets a new delete queue - this must be synced on the flush control
|
/* Cutover to a new delete queue. This must be synced on the flush control
|
||||||
* otherwise a new DWPT could sneak into the loop with an already flushing
|
* otherwise a new DWPT could sneak into the loop with an already flushing
|
||||||
* delete queue */
|
* delete queue */
|
||||||
|
// nocommit -- shouldn't we do this?:
|
||||||
|
// assert Thread.holdsLock(flushControl);
|
||||||
flushControl.markForFullFlush();
|
flushControl.markForFullFlush();
|
||||||
assert setFlushingDeleteQueue(flushingDeleteQueue);
|
assert setFlushingDeleteQueue(flushingDeleteQueue);
|
||||||
}
|
}
|
||||||
@ -509,18 +519,18 @@ final class DocumentsWriter {
|
|||||||
boolean anythingFlushed = false;
|
boolean anythingFlushed = false;
|
||||||
try {
|
try {
|
||||||
DocumentsWriterPerThread flushingDWPT;
|
DocumentsWriterPerThread flushingDWPT;
|
||||||
// now try help out with flushing
|
// Help out with flushing:
|
||||||
while ((flushingDWPT = flushControl.nextPendingFlush()) != null) {
|
while ((flushingDWPT = flushControl.nextPendingFlush()) != null) {
|
||||||
anythingFlushed |= doFlush(flushingDWPT);
|
anythingFlushed |= doFlush(flushingDWPT);
|
||||||
}
|
}
|
||||||
// if a concurrent flush is still in flight wait for it
|
// If a concurrent flush is still in flight wait for it
|
||||||
while (!flushControl.allFlushesDue()) {
|
while (flushControl.anyFlushing()) {
|
||||||
flushControl.waitForFlush();
|
flushControl.waitForFlush();
|
||||||
}
|
}
|
||||||
if (!anythingFlushed && flushDeletes) {
|
if (!anythingFlushed && flushDeletes) {
|
||||||
synchronized (ticketQueue) {
|
synchronized (ticketQueue) {
|
||||||
ticketQueue.add(new FlushTicket(flushingDeleteQueue.freezeGlobalBuffer(null), false));
|
ticketQueue.add(new FlushTicket(flushingDeleteQueue.freezeGlobalBuffer(null), false));
|
||||||
}
|
}
|
||||||
applyFlushTickets(null, null);
|
applyFlushTickets(null, null);
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
@ -532,13 +542,16 @@ final class DocumentsWriter {
|
|||||||
final void finishFullFlush(boolean success) {
|
final void finishFullFlush(boolean success) {
|
||||||
assert setFlushingDeleteQueue(null);
|
assert setFlushingDeleteQueue(null);
|
||||||
if (success) {
|
if (success) {
|
||||||
// release the flush lock
|
// Release the flush lock
|
||||||
flushControl.finishFullFlush();
|
flushControl.finishFullFlush();
|
||||||
} else {
|
} else {
|
||||||
flushControl.abortFullFlushes();
|
flushControl.abortFullFlushes();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// nocommit -- can we add comment justifying that these
|
||||||
|
// fields are safely changed across threads because they
|
||||||
|
// are always accessed in sync(ticketQueue)?
|
||||||
static final class FlushTicket {
|
static final class FlushTicket {
|
||||||
final FrozenBufferedDeletes frozenDeletes;
|
final FrozenBufferedDeletes frozenDeletes;
|
||||||
FlushedSegment segment;
|
FlushedSegment segment;
|
||||||
|
@ -23,15 +23,15 @@ import org.apache.lucene.search.Query;
|
|||||||
|
|
||||||
/**
|
/**
|
||||||
* {@link DocumentsWriterDeleteQueue} is a non-blocking linked pending deletes
|
* {@link DocumentsWriterDeleteQueue} is a non-blocking linked pending deletes
|
||||||
* queue. In contrast to other queue implementation we only maintain only the
|
* queue. In contrast to other queue implementation we only maintain the
|
||||||
* tail of the queue. A delete queue is always used in a context of a set of
|
* tail of the queue. A delete queue is always used in a context of a set of
|
||||||
* DWPT and a global delete pool. Each of the DWPT and the global pool need to
|
* DWPTs and a global delete pool. Each of the DWPT and the global pool need to
|
||||||
* maintain their 'own' head of the queue. The difference between the DWPT and
|
* maintain their 'own' head of the queue (as a DeleteSlice instance per DWPT).
|
||||||
* the global pool is that the DWPT starts maintaining a head once it has added
|
* The difference between the DWPT and the global pool is that the DWPT starts
|
||||||
* its first document since for its segments private deletes only the deletes
|
* maintaining a head once it has added its first document since for its segments
|
||||||
* after that document are relevant. The global pool instead starts maintaining
|
* private deletes only the deletes after that document are relevant. The global
|
||||||
* the head once this instance is created by taking the sentinel instance as its
|
* pool instead starts maintaining the head once this instance is created by
|
||||||
* initial head.
|
* taking the sentinel instance as its initial head.
|
||||||
* <p>
|
* <p>
|
||||||
* Since each {@link DeleteSlice} maintains its own head and the list is only
|
* Since each {@link DeleteSlice} maintains its own head and the list is only
|
||||||
* single linked the garbage collector takes care of pruning the list for us.
|
* single linked the garbage collector takes care of pruning the list for us.
|
||||||
@ -41,12 +41,12 @@ import org.apache.lucene.search.Query;
|
|||||||
* <p>
|
* <p>
|
||||||
* Each DWPT as well as the global delete pool maintain their private
|
* Each DWPT as well as the global delete pool maintain their private
|
||||||
* DeleteSlice instance. In the DWPT case updating a slice is equivalent to
|
* DeleteSlice instance. In the DWPT case updating a slice is equivalent to
|
||||||
* atomically finishing the document. The slice update guarantees a happens
|
* atomically finishing the document. The slice update guarantees a "happens
|
||||||
* before relationship to all other updates in the same indexing session. When a
|
* before" relationship to all other updates in the same indexing session. When a
|
||||||
* DWPT updates a document it
|
* DWPT updates a document it:
|
||||||
*
|
*
|
||||||
* <ol>
|
* <ol>
|
||||||
* <li>consumes a document finishes its processing</li>
|
* <li>consumes a document and finishes its processing</li>
|
||||||
* <li>updates its private {@link DeleteSlice} either by calling
|
* <li>updates its private {@link DeleteSlice} either by calling
|
||||||
* {@link #updateSlice(DeleteSlice)} or {@link #add(Term, DeleteSlice)} (if the
|
* {@link #updateSlice(DeleteSlice)} or {@link #add(Term, DeleteSlice)} (if the
|
||||||
* document has a delTerm)</li>
|
* document has a delTerm)</li>
|
||||||
@ -56,7 +56,7 @@ import org.apache.lucene.search.Query;
|
|||||||
* </ol>
|
* </ol>
|
||||||
*
|
*
|
||||||
* The DWPT also doesn't apply its current documents delete term until it has
|
* The DWPT also doesn't apply its current documents delete term until it has
|
||||||
* updated its delete slice which ensures the consistency of the update. if the
|
* updated its delete slice which ensures the consistency of the update. If the
|
||||||
* update fails before the DeleteSlice could have been updated the deleteTerm
|
* update fails before the DeleteSlice could have been updated the deleteTerm
|
||||||
* will also not be added to its private deletes neither to the global deletes.
|
* will also not be added to its private deletes neither to the global deletes.
|
||||||
*
|
*
|
||||||
@ -167,7 +167,7 @@ final class DocumentsWriterDeleteQueue {
|
|||||||
void tryApplyGlobalSlice() {
|
void tryApplyGlobalSlice() {
|
||||||
if (globalBufferLock.tryLock()) {
|
if (globalBufferLock.tryLock()) {
|
||||||
/*
|
/*
|
||||||
* the global buffer must be locked but we don't need to upate them if
|
* The global buffer must be locked but we don't need to upate them if
|
||||||
* there is an update going on right now. It is sufficient to apply the
|
* there is an update going on right now. It is sufficient to apply the
|
||||||
* deletes that have been added after the current in-flight global slices
|
* deletes that have been added after the current in-flight global slices
|
||||||
* tail the next time we can get the lock!
|
* tail the next time we can get the lock!
|
||||||
@ -175,7 +175,6 @@ final class DocumentsWriterDeleteQueue {
|
|||||||
try {
|
try {
|
||||||
if (updateSlice(globalSlice)) {
|
if (updateSlice(globalSlice)) {
|
||||||
globalSlice.apply(globalBufferedDeletes, BufferedDeletes.MAX_INT);
|
globalSlice.apply(globalBufferedDeletes, BufferedDeletes.MAX_INT);
|
||||||
|
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
globalBufferLock.unlock();
|
globalBufferLock.unlock();
|
||||||
@ -186,15 +185,15 @@ final class DocumentsWriterDeleteQueue {
|
|||||||
FrozenBufferedDeletes freezeGlobalBuffer(DeleteSlice callerSlice) {
|
FrozenBufferedDeletes freezeGlobalBuffer(DeleteSlice callerSlice) {
|
||||||
globalBufferLock.lock();
|
globalBufferLock.lock();
|
||||||
/*
|
/*
|
||||||
* here we are freezing the global buffer so we need to lock it, apply all
|
* Here we freeze the global buffer so we need to lock it, apply all
|
||||||
* deletes in the queue and reset the global slice to let the GC prune the
|
* deletes in the queue and reset the global slice to let the GC prune the
|
||||||
* queue.
|
* queue.
|
||||||
*/
|
*/
|
||||||
final Node currentTail = tail; // take the current tail make this local any
|
final Node currentTail = tail; // take the current tail make this local any
|
||||||
// changes after this call are applied later
|
// Changes after this call are applied later
|
||||||
// and not relevant here
|
// and not relevant here
|
||||||
if (callerSlice != null) {
|
if (callerSlice != null) {
|
||||||
// update the callers slices so we are on the same page
|
// Update the callers slices so we are on the same page
|
||||||
callerSlice.sliceTail = currentTail;
|
callerSlice.sliceTail = currentTail;
|
||||||
}
|
}
|
||||||
try {
|
try {
|
||||||
@ -217,7 +216,7 @@ final class DocumentsWriterDeleteQueue {
|
|||||||
}
|
}
|
||||||
|
|
||||||
boolean updateSlice(DeleteSlice slice) {
|
boolean updateSlice(DeleteSlice slice) {
|
||||||
if (slice.sliceTail != tail) { // if we are the same just
|
if (slice.sliceTail != tail) { // If we are the same just
|
||||||
slice.sliceTail = tail;
|
slice.sliceTail = tail;
|
||||||
return true;
|
return true;
|
||||||
}
|
}
|
||||||
@ -225,7 +224,7 @@ final class DocumentsWriterDeleteQueue {
|
|||||||
}
|
}
|
||||||
|
|
||||||
static class DeleteSlice {
|
static class DeleteSlice {
|
||||||
// no need to be volatile, slices are only access by one thread!
|
// No need to be volatile, slices are thread captive (only accessed by one thread)!
|
||||||
Node sliceHead; // we don't apply this one
|
Node sliceHead; // we don't apply this one
|
||||||
Node sliceTail;
|
Node sliceTail;
|
||||||
|
|
||||||
@ -245,7 +244,7 @@ final class DocumentsWriterDeleteQueue {
|
|||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
/*
|
/*
|
||||||
* when we apply a slice we take the head and get its next as our first
|
* When we apply a slice we take the head and get its next as our first
|
||||||
* item to apply and continue until we applied the tail. If the head and
|
* item to apply and continue until we applied the tail. If the head and
|
||||||
* tail in this slice are not equal then there will be at least one more
|
* tail in this slice are not equal then there will be at least one more
|
||||||
* non-null node in the slice!
|
* non-null node in the slice!
|
||||||
@ -260,7 +259,7 @@ final class DocumentsWriterDeleteQueue {
|
|||||||
}
|
}
|
||||||
|
|
||||||
void reset() {
|
void reset() {
|
||||||
// resetting to a 0 length slice
|
// Reset to a 0 length slice
|
||||||
sliceHead = sliceTail;
|
sliceHead = sliceTail;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -322,7 +321,6 @@ final class DocumentsWriterDeleteQueue {
|
|||||||
void apply(BufferedDeletes bufferedDeletes, int docIDUpto) {
|
void apply(BufferedDeletes bufferedDeletes, int docIDUpto) {
|
||||||
bufferedDeletes.addTerm((Term) item, docIDUpto);
|
bufferedDeletes.addTerm((Term) item, docIDUpto);
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
||||||
private static final class QueryArrayNode extends Node {
|
private static final class QueryArrayNode extends Node {
|
||||||
@ -376,6 +374,5 @@ final class DocumentsWriterDeleteQueue {
|
|||||||
} finally {
|
} finally {
|
||||||
globalBufferLock.unlock();
|
globalBufferLock.unlock();
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -40,7 +40,7 @@ import org.apache.lucene.util.ThreadInterruptedException;
|
|||||||
*/
|
*/
|
||||||
public final class DocumentsWriterFlushControl {
|
public final class DocumentsWriterFlushControl {
|
||||||
|
|
||||||
private final long maxBytesPerDWPT;
|
private final long hardMaxBytesPerDWPT;
|
||||||
private long activeBytes = 0;
|
private long activeBytes = 0;
|
||||||
private long flushBytes = 0;
|
private long flushBytes = 0;
|
||||||
private volatile int numPending = 0;
|
private volatile int numPending = 0;
|
||||||
@ -63,11 +63,11 @@ public final class DocumentsWriterFlushControl {
|
|||||||
private final DocumentsWriter documentsWriter;
|
private final DocumentsWriter documentsWriter;
|
||||||
|
|
||||||
DocumentsWriterFlushControl(DocumentsWriter documentsWriter,
|
DocumentsWriterFlushControl(DocumentsWriter documentsWriter,
|
||||||
Healthiness healthiness, long maxBytesPerDWPT) {
|
Healthiness healthiness, long hardMaxBytesPerDWPT) {
|
||||||
this.healthiness = healthiness;
|
this.healthiness = healthiness;
|
||||||
this.perThreadPool = documentsWriter.perThreadPool;
|
this.perThreadPool = documentsWriter.perThreadPool;
|
||||||
this.flushPolicy = documentsWriter.flushPolicy;
|
this.flushPolicy = documentsWriter.flushPolicy;
|
||||||
this.maxBytesPerDWPT = maxBytesPerDWPT;
|
this.hardMaxBytesPerDWPT = hardMaxBytesPerDWPT;
|
||||||
this.documentsWriter = documentsWriter;
|
this.documentsWriter = documentsWriter;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -85,8 +85,8 @@ public final class DocumentsWriterFlushControl {
|
|||||||
|
|
||||||
private void commitPerThreadBytes(ThreadState perThread) {
|
private void commitPerThreadBytes(ThreadState perThread) {
|
||||||
final long delta = perThread.perThread.bytesUsed()
|
final long delta = perThread.perThread.bytesUsed()
|
||||||
- perThread.perThreadBytes;
|
- perThread.bytesUsed;
|
||||||
perThread.perThreadBytes += delta;
|
perThread.bytesUsed += delta;
|
||||||
/*
|
/*
|
||||||
* We need to differentiate here if we are pending since setFlushPending
|
* We need to differentiate here if we are pending since setFlushPending
|
||||||
* moves the perThread memory to the flushBytes and we could be set to
|
* moves the perThread memory to the flushBytes and we could be set to
|
||||||
@ -100,6 +100,7 @@ public final class DocumentsWriterFlushControl {
|
|||||||
assert updatePeaks(delta);
|
assert updatePeaks(delta);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// only for asserts
|
||||||
private boolean updatePeaks(long delta) {
|
private boolean updatePeaks(long delta) {
|
||||||
peakActiveBytes = Math.max(peakActiveBytes, activeBytes);
|
peakActiveBytes = Math.max(peakActiveBytes, activeBytes);
|
||||||
peakFlushBytes = Math.max(peakFlushBytes, flushBytes);
|
peakFlushBytes = Math.max(peakFlushBytes, flushBytes);
|
||||||
@ -116,10 +117,9 @@ public final class DocumentsWriterFlushControl {
|
|||||||
} else {
|
} else {
|
||||||
flushPolicy.onInsert(this, perThread);
|
flushPolicy.onInsert(this, perThread);
|
||||||
}
|
}
|
||||||
if (!perThread.flushPending && perThread.perThreadBytes > maxBytesPerDWPT) {
|
if (!perThread.flushPending && perThread.bytesUsed > hardMaxBytesPerDWPT) {
|
||||||
// safety check to prevent a single DWPT exceeding its RAM limit. This
|
// Safety check to prevent a single DWPT exceeding its RAM limit. This
|
||||||
// is super
|
// is super important since we can not address more than 2048 MB per DWPT
|
||||||
// important since we can not address more than 2048 MB per DWPT
|
|
||||||
setFlushPending(perThread);
|
setFlushPending(perThread);
|
||||||
if (fullFlush) {
|
if (fullFlush) {
|
||||||
DocumentsWriterPerThread toBlock = internalTryCheckOutForFlush(perThread, false);
|
DocumentsWriterPerThread toBlock = internalTryCheckOutForFlush(perThread, false);
|
||||||
@ -146,8 +146,8 @@ public final class DocumentsWriterFlushControl {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized boolean allFlushesDue() {
|
public synchronized boolean anyFlushing() {
|
||||||
return numFlushing == 0;
|
return numFlushing != 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
public synchronized void waitForFlush() {
|
public synchronized void waitForFlush() {
|
||||||
@ -169,7 +169,7 @@ public final class DocumentsWriterFlushControl {
|
|||||||
assert !perThread.flushPending;
|
assert !perThread.flushPending;
|
||||||
if (perThread.perThread.getNumDocsInRAM() > 0) {
|
if (perThread.perThread.getNumDocsInRAM() > 0) {
|
||||||
perThread.flushPending = true; // write access synced
|
perThread.flushPending = true; // write access synced
|
||||||
final long bytes = perThread.perThreadBytes;
|
final long bytes = perThread.bytesUsed;
|
||||||
flushBytes += bytes;
|
flushBytes += bytes;
|
||||||
activeBytes -= bytes;
|
activeBytes -= bytes;
|
||||||
numPending++; // write access synced
|
numPending++; // write access synced
|
||||||
@ -179,19 +179,20 @@ public final class DocumentsWriterFlushControl {
|
|||||||
|
|
||||||
synchronized void doOnAbort(ThreadState state) {
|
synchronized void doOnAbort(ThreadState state) {
|
||||||
if (state.flushPending) {
|
if (state.flushPending) {
|
||||||
flushBytes -= state.perThreadBytes;
|
flushBytes -= state.bytesUsed;
|
||||||
} else {
|
} else {
|
||||||
activeBytes -= state.perThreadBytes;
|
activeBytes -= state.bytesUsed;
|
||||||
}
|
}
|
||||||
// take it out of the loop this DWPT is stale
|
// Take it out of the loop this DWPT is stale
|
||||||
perThreadPool.replaceForFlush(state, closed);
|
perThreadPool.replaceForFlush(state, closed);
|
||||||
healthiness.updateStalled(this);
|
healthiness.updateStalled(this);
|
||||||
}
|
}
|
||||||
|
|
||||||
synchronized DocumentsWriterPerThread tryCheckoutForFlush(
|
synchronized DocumentsWriterPerThread tryCheckoutForFlush(
|
||||||
ThreadState perThread, boolean setPending) {
|
ThreadState perThread, boolean setPending) {
|
||||||
if (fullFlush)
|
if (fullFlush) {
|
||||||
return null;
|
return null;
|
||||||
|
}
|
||||||
return internalTryCheckOutForFlush(perThread, setPending);
|
return internalTryCheckOutForFlush(perThread, setPending);
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -201,17 +202,17 @@ public final class DocumentsWriterFlushControl {
|
|||||||
setFlushPending(perThread);
|
setFlushPending(perThread);
|
||||||
}
|
}
|
||||||
if (perThread.flushPending) {
|
if (perThread.flushPending) {
|
||||||
// we are pending so all memory is already moved to flushBytes
|
// We are pending so all memory is already moved to flushBytes
|
||||||
if (perThread.tryLock()) {
|
if (perThread.tryLock()) {
|
||||||
try {
|
try {
|
||||||
if (perThread.isActive()) {
|
if (perThread.isActive()) {
|
||||||
assert perThread.isHeldByCurrentThread();
|
assert perThread.isHeldByCurrentThread();
|
||||||
final DocumentsWriterPerThread dwpt;
|
final DocumentsWriterPerThread dwpt;
|
||||||
final long bytes = perThread.perThreadBytes; // do that before
|
final long bytes = perThread.bytesUsed; // do that before
|
||||||
// replace!
|
// replace!
|
||||||
dwpt = perThreadPool.replaceForFlush(perThread, closed);
|
dwpt = perThreadPool.replaceForFlush(perThread, closed);
|
||||||
assert !flushingWriters.containsKey(dwpt) : "DWPT is already flushing";
|
assert !flushingWriters.containsKey(dwpt) : "DWPT is already flushing";
|
||||||
// record the flushing DWPT to reduce flushBytes in doAfterFlush
|
// Record the flushing DWPT to reduce flushBytes in doAfterFlush
|
||||||
flushingWriters.put(dwpt, Long.valueOf(bytes));
|
flushingWriters.put(dwpt, Long.valueOf(bytes));
|
||||||
numPending--; // write access synced
|
numPending--; // write access synced
|
||||||
numFlushing++;
|
numFlushing++;
|
||||||
@ -298,8 +299,12 @@ public final class DocumentsWriterFlushControl {
|
|||||||
return numFlushing;
|
return numFlushing;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void setFlushDeletes() {
|
public boolean doApplyAllDeletes() {
|
||||||
flushDeletes.set(true);
|
return flushDeletes.getAndSet(false);
|
||||||
|
}
|
||||||
|
|
||||||
|
public void setApplyAllDeletes() {
|
||||||
|
flushDeletes.set(true);
|
||||||
}
|
}
|
||||||
|
|
||||||
int numActiveDWPT() {
|
int numActiveDWPT() {
|
||||||
@ -312,7 +317,7 @@ public final class DocumentsWriterFlushControl {
|
|||||||
assert !fullFlush;
|
assert !fullFlush;
|
||||||
fullFlush = true;
|
fullFlush = true;
|
||||||
flushingQueue = documentsWriter.deleteQueue;
|
flushingQueue = documentsWriter.deleteQueue;
|
||||||
// set a new delete queue - all subsequent DWPT will use this queue until
|
// Set a new delete queue - all subsequent DWPT will use this queue until
|
||||||
// we do another full flush
|
// we do another full flush
|
||||||
documentsWriter.deleteQueue = new DocumentsWriterDeleteQueue(new BufferedDeletes(false));
|
documentsWriter.deleteQueue = new DocumentsWriterDeleteQueue(new BufferedDeletes(false));
|
||||||
}
|
}
|
||||||
@ -374,9 +379,9 @@ public final class DocumentsWriterFlushControl {
|
|||||||
}
|
}
|
||||||
|
|
||||||
} finally {
|
} finally {
|
||||||
|
fullFlush = false;
|
||||||
flushQueue.clear();
|
flushQueue.clear();
|
||||||
blockedFlushes.clear();
|
blockedFlushes.clear();
|
||||||
fullFlush = false;
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -44,7 +44,7 @@ public abstract class DocumentsWriterPerThreadPool {
|
|||||||
// write access guarded by DocumentsWriterFlushControl
|
// write access guarded by DocumentsWriterFlushControl
|
||||||
volatile boolean flushPending = false;
|
volatile boolean flushPending = false;
|
||||||
// write access guarded by DocumentsWriterFlushControl
|
// write access guarded by DocumentsWriterFlushControl
|
||||||
long perThreadBytes = 0;
|
long bytesUsed = 0;
|
||||||
|
|
||||||
// guarded by Reentrant lock
|
// guarded by Reentrant lock
|
||||||
private boolean isActive = true;
|
private boolean isActive = true;
|
||||||
@ -65,7 +65,7 @@ public abstract class DocumentsWriterPerThreadPool {
|
|||||||
isActive = false;
|
isActive = false;
|
||||||
}
|
}
|
||||||
this.perThread = perThread;
|
this.perThread = perThread;
|
||||||
this.perThreadBytes = 0;
|
this.bytesUsed = 0;
|
||||||
this.flushPending = false;
|
this.flushPending = false;
|
||||||
}
|
}
|
||||||
|
|
||||||
@ -86,7 +86,7 @@ public abstract class DocumentsWriterPerThreadPool {
|
|||||||
public long getBytesUsedPerThread() {
|
public long getBytesUsedPerThread() {
|
||||||
assert this.isHeldByCurrentThread();
|
assert this.isHeldByCurrentThread();
|
||||||
// public for FlushPolicy
|
// public for FlushPolicy
|
||||||
return perThreadBytes;
|
return bytesUsed;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
@ -162,9 +162,9 @@ public abstract class DocumentsWriterPerThreadPool {
|
|||||||
|
|
||||||
public abstract ThreadState getAndLock(Thread requestingThread, DocumentsWriter documentsWriter, Document doc);
|
public abstract ThreadState getAndLock(Thread requestingThread, DocumentsWriter documentsWriter, Document doc);
|
||||||
|
|
||||||
public abstract void clearThreadBindings(ThreadState perThread);
|
//public abstract void clearThreadBindings(ThreadState perThread);
|
||||||
|
|
||||||
public abstract void clearAllThreadBindings();
|
// public abstract void clearAllThreadBindings();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Returns an iterator providing access to all {@link ThreadState}
|
* Returns an iterator providing access to all {@link ThreadState}
|
||||||
|
@ -21,16 +21,16 @@ import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState;
|
|||||||
|
|
||||||
/**
|
/**
|
||||||
* Default {@link FlushPolicy} implementation that flushes based on RAM
|
* Default {@link FlushPolicy} implementation that flushes based on RAM
|
||||||
* Consumption, document count and number of buffered deletes depending on the
|
* used, document count and number of buffered deletes depending on the
|
||||||
* IndexWriters {@link IndexWriterConfig}. This {@link FlushPolicy} will only
|
* IndexWriter's {@link IndexWriterConfig}. This {@link FlushPolicy} will only
|
||||||
* respect settings which are not disabled during initialization (
|
* respect settings which are not disabled during initialization (
|
||||||
* {@link #init(DocumentsWriter)}). All enabled {@link IndexWriterConfig}
|
* {@link #init(DocumentsWriter)}) (nocommit what does that mean?). All enabled {@link IndexWriterConfig}
|
||||||
* settings are used to mark {@link DocumentsWriterPerThread} as flush pending
|
* settings are used to mark {@link DocumentsWriterPerThread} as flush pending
|
||||||
* during indexing with respect to thier live updates.
|
* during indexing with respect to their live updates.
|
||||||
* <p>
|
* <p>
|
||||||
* If {@link IndexWriterConfig#setRAMBufferSizeMB(double)} is enabled always the
|
* If {@link IndexWriterConfig#setRAMBufferSizeMB(double)} is enabled, the
|
||||||
* largest ram consuming {@link DocumentsWriterPerThread} will be marked as
|
* largest ram consuming {@link DocumentsWriterPerThread} will be marked as
|
||||||
* pending iff the global active RAM consumption is equals or higher the
|
* pending iff the global active RAM consumption is >= the
|
||||||
* configured max RAM buffer.
|
* configured max RAM buffer.
|
||||||
*/
|
*/
|
||||||
public class FlushByRamOrCountsPolicy extends FlushPolicy {
|
public class FlushByRamOrCountsPolicy extends FlushPolicy {
|
||||||
@ -38,10 +38,11 @@ public class FlushByRamOrCountsPolicy extends FlushPolicy {
|
|||||||
@Override
|
@Override
|
||||||
public void onDelete(DocumentsWriterFlushControl control, ThreadState state) {
|
public void onDelete(DocumentsWriterFlushControl control, ThreadState state) {
|
||||||
if (flushOnDeleteTerms()) {
|
if (flushOnDeleteTerms()) {
|
||||||
|
// Flush this state by num del terms
|
||||||
final int maxBufferedDeleteTerms = indexWriterConfig
|
final int maxBufferedDeleteTerms = indexWriterConfig
|
||||||
.getMaxBufferedDeleteTerms();
|
.getMaxBufferedDeleteTerms();
|
||||||
if (control.getNumGlobalTermDeletes() >= maxBufferedDeleteTerms) {
|
if (control.getNumGlobalTermDeletes() >= maxBufferedDeleteTerms) {
|
||||||
control.setFlushDeletes();
|
control.setApplyAllDeletes();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
@ -51,12 +52,12 @@ public class FlushByRamOrCountsPolicy extends FlushPolicy {
|
|||||||
if (flushOnDocCount()
|
if (flushOnDocCount()
|
||||||
&& state.perThread.getNumDocsInRAM() >= indexWriterConfig
|
&& state.perThread.getNumDocsInRAM() >= indexWriterConfig
|
||||||
.getMaxBufferedDocs()) {
|
.getMaxBufferedDocs()) {
|
||||||
control.setFlushPending(state); // flush by num docs
|
// Flush this state by num docs
|
||||||
|
control.setFlushPending(state);
|
||||||
} else {// flush by RAM
|
} else {// flush by RAM
|
||||||
if (flushOnRAM()) {
|
if (flushOnRAM()) {
|
||||||
final double ramBufferSizeMB = indexWriterConfig.getRAMBufferSizeMB();
|
final long limit = (long) (indexWriterConfig.getRAMBufferSizeMB() * 1024.d * 1024.d);
|
||||||
final long totalRam = control.activeBytes();
|
final long totalRam = control.activeBytes();
|
||||||
final long limit = (long) (ramBufferSizeMB * 1024.d * 1024.d);
|
|
||||||
if (totalRam >= limit) {
|
if (totalRam >= limit) {
|
||||||
markLargestWriterPending(control, state, totalRam);
|
markLargestWriterPending(control, state, totalRam);
|
||||||
}
|
}
|
||||||
|
@ -32,16 +32,16 @@ import org.apache.lucene.util.SetOnce;
|
|||||||
* {@link IndexWriterConfig#setRAMBufferSizeMB(double)}</li>
|
* {@link IndexWriterConfig#setRAMBufferSizeMB(double)}</li>
|
||||||
* <li>Number of RAM resident documents - configured via
|
* <li>Number of RAM resident documents - configured via
|
||||||
* {@link IndexWriterConfig#setMaxBufferedDocs(int)}</li>
|
* {@link IndexWriterConfig#setMaxBufferedDocs(int)}</li>
|
||||||
* <li>Number of buffered delete terms - configured via
|
* <li>Number of buffered delete terms/queries - configured via
|
||||||
* {@link IndexWriterConfig#setMaxBufferedDeleteTerms(int)}</li>
|
* {@link IndexWriterConfig#setMaxBufferedDeleteTerms(int)}</li>
|
||||||
* </ul>
|
* </ul>
|
||||||
*
|
*
|
||||||
* The {@link IndexWriter} uses a provided {@link FlushPolicy} to control the
|
* The {@link IndexWriter} consults a provided {@link FlushPolicy} to control the
|
||||||
* flushing process during indexing. The policy is informed for each added or
|
* flushing process. The policy is informed for each added or
|
||||||
* updated document as well as for each delete term. Based on the
|
* updated document as well as for each delete term. Based on the
|
||||||
* {@link FlushPolicy} the information provided via {@link ThreadState} and
|
* {@link FlushPolicy}, the information provided via {@link ThreadState} and
|
||||||
* {@link DocumentsWriterFlushControl} the {@link FlushPolicy} can decide if a
|
* {@link DocumentsWriterFlushControl}, the {@link FlushPolicy} decides if a
|
||||||
* {@link DocumentsWriterPerThread} needs flushing and can mark it as
|
* {@link DocumentsWriterPerThread} needs flushing and mark it as
|
||||||
* flush-pending via
|
* flush-pending via
|
||||||
* {@link DocumentsWriterFlushControl#setFlushPending(ThreadState)}.
|
* {@link DocumentsWriterFlushControl#setFlushPending(ThreadState)}.
|
||||||
*
|
*
|
||||||
@ -58,6 +58,7 @@ public abstract class FlushPolicy {
|
|||||||
* Called for each delete term. If this is a delete triggered due to an update
|
* Called for each delete term. If this is a delete triggered due to an update
|
||||||
* the given {@link ThreadState} is non-null.
|
* the given {@link ThreadState} is non-null.
|
||||||
* <p>
|
* <p>
|
||||||
|
* nocommit: what does this note mean...?
|
||||||
* Note: This method is synchronized by the given
|
* Note: This method is synchronized by the given
|
||||||
* {@link DocumentsWriterFlushControl} and it is guaranteed that the calling
|
* {@link DocumentsWriterFlushControl} and it is guaranteed that the calling
|
||||||
* thread holds the lock on the given {@link ThreadState}
|
* thread holds the lock on the given {@link ThreadState}
|
||||||
@ -66,9 +67,10 @@ public abstract class FlushPolicy {
|
|||||||
ThreadState state);
|
ThreadState state);
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Called for each document update on the given {@link ThreadState}s
|
* Called for each document update on the given {@link ThreadState}'s
|
||||||
* {@link DocumentsWriterPerThread}.
|
* {@link DocumentsWriterPerThread}.
|
||||||
* <p>
|
* <p>
|
||||||
|
* nocommit: what does this note mean...?
|
||||||
* Note: This method is synchronized by the given
|
* Note: This method is synchronized by the given
|
||||||
* {@link DocumentsWriterFlushControl} and it is guaranteed that the calling
|
* {@link DocumentsWriterFlushControl} and it is guaranteed that the calling
|
||||||
* thread holds the lock on the given {@link ThreadState}
|
* thread holds the lock on the given {@link ThreadState}
|
||||||
@ -103,6 +105,7 @@ public abstract class FlushPolicy {
|
|||||||
* Marks the most ram consuming active {@link DocumentsWriterPerThread} flush
|
* Marks the most ram consuming active {@link DocumentsWriterPerThread} flush
|
||||||
* pending
|
* pending
|
||||||
*/
|
*/
|
||||||
|
// nocommit -- move to default policy?
|
||||||
protected void markLargestWriterPending(DocumentsWriterFlushControl control,
|
protected void markLargestWriterPending(DocumentsWriterFlushControl control,
|
||||||
ThreadState perThreadState, final long currentBytesPerThread) {
|
ThreadState perThreadState, final long currentBytesPerThread) {
|
||||||
control
|
control
|
||||||
@ -117,7 +120,8 @@ public abstract class FlushPolicy {
|
|||||||
*/
|
*/
|
||||||
protected ThreadState findLargestNonPendingWriter(
|
protected ThreadState findLargestNonPendingWriter(
|
||||||
DocumentsWriterFlushControl control, ThreadState perThreadState) {
|
DocumentsWriterFlushControl control, ThreadState perThreadState) {
|
||||||
long maxRamSoFar = perThreadState.perThreadBytes;
|
assert perThreadState.perThread.getNumDocsInRAM() > 0;
|
||||||
|
long maxRamSoFar = perThreadState.bytesUsed;
|
||||||
// the dwpt which needs to be flushed eventually
|
// the dwpt which needs to be flushed eventually
|
||||||
ThreadState maxRamUsingThreadState = perThreadState;
|
ThreadState maxRamUsingThreadState = perThreadState;
|
||||||
assert !perThreadState.flushPending : "DWPT should have flushed";
|
assert !perThreadState.flushPending : "DWPT should have flushed";
|
||||||
@ -125,7 +129,7 @@ public abstract class FlushPolicy {
|
|||||||
while (activePerThreadsIterator.hasNext()) {
|
while (activePerThreadsIterator.hasNext()) {
|
||||||
ThreadState next = activePerThreadsIterator.next();
|
ThreadState next = activePerThreadsIterator.next();
|
||||||
if (!next.flushPending) {
|
if (!next.flushPending) {
|
||||||
final long nextRam = next.perThreadBytes;
|
final long nextRam = next.bytesUsed;
|
||||||
if (nextRam > maxRamSoFar && next.perThread.getNumDocsInRAM() > 0) {
|
if (nextRam > maxRamSoFar && next.perThread.getNumDocsInRAM() > 0) {
|
||||||
maxRamSoFar = nextRam;
|
maxRamSoFar = nextRam;
|
||||||
maxRamUsingThreadState = next;
|
maxRamUsingThreadState = next;
|
||||||
@ -137,6 +141,8 @@ public abstract class FlushPolicy {
|
|||||||
return maxRamUsingThreadState;
|
return maxRamUsingThreadState;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// nocommit -- I thought we pause based on "too many flush
|
||||||
|
// states pending"?
|
||||||
/**
|
/**
|
||||||
* Returns the max net memory which marks the upper watermark for the
|
* Returns the max net memory which marks the upper watermark for the
|
||||||
* DocumentsWriter to be healthy. If all flushing and active
|
* DocumentsWriter to be healthy. If all flushing and active
|
||||||
@ -154,6 +160,7 @@ public abstract class FlushPolicy {
|
|||||||
*/
|
*/
|
||||||
public long getMaxNetBytes() {
|
public long getMaxNetBytes() {
|
||||||
if (!flushOnRAM()) {
|
if (!flushOnRAM()) {
|
||||||
|
// nocommit explain that returning -1 is allowed?
|
||||||
return -1;
|
return -1;
|
||||||
}
|
}
|
||||||
final double ramBufferSizeMB = indexWriterConfig.getRAMBufferSizeMB();
|
final double ramBufferSizeMB = indexWriterConfig.getRAMBufferSizeMB();
|
||||||
@ -165,6 +172,8 @@ public abstract class FlushPolicy {
|
|||||||
* {@link IndexWriterConfig#getMaxBufferedDocs()}, otherwise
|
* {@link IndexWriterConfig#getMaxBufferedDocs()}, otherwise
|
||||||
* <code>false</code>.
|
* <code>false</code>.
|
||||||
*/
|
*/
|
||||||
|
// nocommit who needs this? policy shouldn't have to impl
|
||||||
|
// this? our default policy should?
|
||||||
protected boolean flushOnDocCount() {
|
protected boolean flushOnDocCount() {
|
||||||
return indexWriterConfig.getMaxBufferedDocs() != IndexWriterConfig.DISABLE_AUTO_FLUSH;
|
return indexWriterConfig.getMaxBufferedDocs() != IndexWriterConfig.DISABLE_AUTO_FLUSH;
|
||||||
}
|
}
|
||||||
@ -174,6 +183,8 @@ public abstract class FlushPolicy {
|
|||||||
* {@link IndexWriterConfig#getMaxBufferedDeleteTerms()}, otherwise
|
* {@link IndexWriterConfig#getMaxBufferedDeleteTerms()}, otherwise
|
||||||
* <code>false</code>.
|
* <code>false</code>.
|
||||||
*/
|
*/
|
||||||
|
// nocommit who needs this? policy shouldn't have to impl
|
||||||
|
// this? our default policy should?
|
||||||
protected boolean flushOnDeleteTerms() {
|
protected boolean flushOnDeleteTerms() {
|
||||||
return indexWriterConfig.getMaxBufferedDeleteTerms() != IndexWriterConfig.DISABLE_AUTO_FLUSH;
|
return indexWriterConfig.getMaxBufferedDeleteTerms() != IndexWriterConfig.DISABLE_AUTO_FLUSH;
|
||||||
}
|
}
|
||||||
@ -183,6 +194,8 @@ public abstract class FlushPolicy {
|
|||||||
* {@link IndexWriterConfig#getRAMBufferSizeMB()}, otherwise
|
* {@link IndexWriterConfig#getRAMBufferSizeMB()}, otherwise
|
||||||
* <code>false</code>.
|
* <code>false</code>.
|
||||||
*/
|
*/
|
||||||
|
// nocommit who needs this? policy shouldn't have to impl
|
||||||
|
// this? our default policy should?
|
||||||
protected boolean flushOnRAM() {
|
protected boolean flushOnRAM() {
|
||||||
return indexWriterConfig.getRAMBufferSizeMB() != IndexWriterConfig.DISABLE_AUTO_FLUSH;
|
return indexWriterConfig.getRAMBufferSizeMB() != IndexWriterConfig.DISABLE_AUTO_FLUSH;
|
||||||
}
|
}
|
||||||
|
@ -52,7 +52,7 @@ final class FreqProxTermsWriter extends TermsHashConsumer {
|
|||||||
|
|
||||||
final int numAllFields = allFields.size();
|
final int numAllFields = allFields.size();
|
||||||
|
|
||||||
// sort by field name
|
// Sort by field name
|
||||||
CollectionUtil.quickSort(allFields);
|
CollectionUtil.quickSort(allFields);
|
||||||
|
|
||||||
final FieldsConsumer consumer = state.segmentCodecs.codec().fieldsConsumer(state);
|
final FieldsConsumer consumer = state.segmentCodecs.codec().fieldsConsumer(state);
|
||||||
|
@ -83,10 +83,10 @@ final class Healthiness {
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private final Healthiness.Sync sync = new Sync();
|
private final Sync sync = new Sync();
|
||||||
volatile boolean wasStalled = false; // only with asserts
|
volatile boolean wasStalled = false; // only with asserts
|
||||||
|
|
||||||
boolean isStalled() {
|
boolean anyStalledThreads() {
|
||||||
return !sync.isHealthy();
|
return !sync.isHealthy();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -383,7 +383,7 @@ public class IndexWriter implements Closeable {
|
|||||||
if (!success && infoStream != null) {
|
if (!success && infoStream != null) {
|
||||||
message("hit exception during while NRT reader");
|
message("hit exception during while NRT reader");
|
||||||
}
|
}
|
||||||
// now we are done - finish the full flush!
|
// Done: finish the full flush!
|
||||||
docWriter.finishFullFlush(success);
|
docWriter.finishFullFlush(success);
|
||||||
doAfterFlush();
|
doAfterFlush();
|
||||||
}
|
}
|
||||||
@ -2073,7 +2073,7 @@ public class IndexWriter implements Closeable {
|
|||||||
if (useCompoundFile(newSegment)) {
|
if (useCompoundFile(newSegment)) {
|
||||||
String compoundFileName = IndexFileNames.segmentFileName(newSegment.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
|
String compoundFileName = IndexFileNames.segmentFileName(newSegment.name, "", IndexFileNames.COMPOUND_FILE_EXTENSION);
|
||||||
message("creating compound file " + compoundFileName);
|
message("creating compound file " + compoundFileName);
|
||||||
// Now build compound file
|
// Now build compound file
|
||||||
CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, compoundFileName);
|
CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, compoundFileName);
|
||||||
for(String fileName : newSegment.files()) {
|
for(String fileName : newSegment.files()) {
|
||||||
cfsWriter.addFile(fileName);
|
cfsWriter.addFile(fileName);
|
||||||
@ -2146,18 +2146,18 @@ public class IndexWriter implements Closeable {
|
|||||||
*/
|
*/
|
||||||
synchronized void publishFlushedSegment(SegmentInfo newSegment,
|
synchronized void publishFlushedSegment(SegmentInfo newSegment,
|
||||||
FrozenBufferedDeletes packet, FrozenBufferedDeletes globalPacket) throws IOException {
|
FrozenBufferedDeletes packet, FrozenBufferedDeletes globalPacket) throws IOException {
|
||||||
// lock order IW -> BDS
|
// Lock order IW -> BDS
|
||||||
synchronized (bufferedDeletesStream) {
|
synchronized (bufferedDeletesStream) {
|
||||||
if (globalPacket != null && globalPacket.any()) {
|
if (globalPacket != null && globalPacket.any()) {
|
||||||
bufferedDeletesStream.push(globalPacket);
|
bufferedDeletesStream.push(globalPacket);
|
||||||
}
|
}
|
||||||
// publishing the segment must be synched on IW -> BDS to make the sure
|
// Publishing the segment must be synched on IW -> BDS to make the sure
|
||||||
// that no merge prunes away the seg. private delete packet
|
// that no merge prunes away the seg. private delete packet
|
||||||
final long nextGen;
|
final long nextGen;
|
||||||
if (packet != null && packet.any()) {
|
if (packet != null && packet.any()) {
|
||||||
nextGen = bufferedDeletesStream.push(packet);
|
nextGen = bufferedDeletesStream.push(packet);
|
||||||
} else {
|
} else {
|
||||||
// since we don't have a delete packet to apply we can get a new
|
// Since we don't have a delete packet to apply we can get a new
|
||||||
// generation right away
|
// generation right away
|
||||||
nextGen = bufferedDeletesStream.getNextGen();
|
nextGen = bufferedDeletesStream.getNextGen();
|
||||||
}
|
}
|
||||||
@ -2572,7 +2572,11 @@ public class IndexWriter implements Closeable {
|
|||||||
message("commit: done");
|
message("commit: done");
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Ensures only one flush() is actually flushing segments
|
||||||
|
// at a time:
|
||||||
private final Object fullFlushLock = new Object();
|
private final Object fullFlushLock = new Object();
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Flush all in-memory buffered updates (adds and deletes)
|
* Flush all in-memory buffered updates (adds and deletes)
|
||||||
* to the Directory.
|
* to the Directory.
|
||||||
@ -2595,9 +2599,7 @@ public class IndexWriter implements Closeable {
|
|||||||
maybeMerge();
|
maybeMerge();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
// TODO: this method should not have to be entirely
|
|
||||||
// synchronized, ie, merges should be allowed to commit
|
|
||||||
// even while a flush is happening
|
|
||||||
private boolean doFlush(boolean applyAllDeletes) throws CorruptIndexException, IOException {
|
private boolean doFlush(boolean applyAllDeletes) throws CorruptIndexException, IOException {
|
||||||
if (hitOOM) {
|
if (hitOOM) {
|
||||||
throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot flush");
|
throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot flush");
|
||||||
@ -2645,6 +2647,8 @@ public class IndexWriter implements Closeable {
|
|||||||
|
|
||||||
final synchronized void maybeApplyDeletes(boolean applyAllDeletes) throws IOException {
|
final synchronized void maybeApplyDeletes(boolean applyAllDeletes) throws IOException {
|
||||||
if (!applyAllDeletes) {
|
if (!applyAllDeletes) {
|
||||||
|
// nocommit -- shouldn't this move into the default
|
||||||
|
// flush policy?
|
||||||
// If deletes alone are consuming > 1/2 our RAM
|
// If deletes alone are consuming > 1/2 our RAM
|
||||||
// buffer, force them all to apply now. This is to
|
// buffer, force them all to apply now. This is to
|
||||||
// prevent too-frequent flushing of a long tail of
|
// prevent too-frequent flushing of a long tail of
|
||||||
@ -2670,31 +2674,31 @@ public class IndexWriter implements Closeable {
|
|||||||
}
|
}
|
||||||
|
|
||||||
final synchronized void applyAllDeletes() throws IOException {
|
final synchronized void applyAllDeletes() throws IOException {
|
||||||
flushDeletesCount.incrementAndGet();
|
flushDeletesCount.incrementAndGet();
|
||||||
final BufferedDeletesStream.ApplyDeletesResult result = bufferedDeletesStream
|
final BufferedDeletesStream.ApplyDeletesResult result = bufferedDeletesStream
|
||||||
.applyDeletes(readerPool, segmentInfos);
|
.applyDeletes(readerPool, segmentInfos);
|
||||||
if (result.anyDeletes) {
|
if (result.anyDeletes) {
|
||||||
checkpoint();
|
checkpoint();
|
||||||
|
}
|
||||||
|
if (!keepFullyDeletedSegments && result.allDeleted != null) {
|
||||||
|
if (infoStream != null) {
|
||||||
|
message("drop 100% deleted segments: " + result.allDeleted);
|
||||||
}
|
}
|
||||||
if (!keepFullyDeletedSegments && result.allDeleted != null) {
|
for (SegmentInfo info : result.allDeleted) {
|
||||||
if (infoStream != null) {
|
// If a merge has already registered for this
|
||||||
message("drop 100% deleted segments: " + result.allDeleted);
|
// segment, we leave it in the readerPool; the
|
||||||
}
|
// merge will skip merging it and will then drop
|
||||||
for (SegmentInfo info : result.allDeleted) {
|
// it once it's done:
|
||||||
// If a merge has already registered for this
|
if (!mergingSegments.contains(info)) {
|
||||||
// segment, we leave it in the readerPool; the
|
segmentInfos.remove(info);
|
||||||
// merge will skip merging it and will then drop
|
if (readerPool != null) {
|
||||||
// it once it's done:
|
readerPool.drop(info);
|
||||||
if (!mergingSegments.contains(info)) {
|
|
||||||
segmentInfos.remove(info);
|
|
||||||
if (readerPool != null) {
|
|
||||||
readerPool.drop(info);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
checkpoint();
|
|
||||||
}
|
}
|
||||||
bufferedDeletesStream.prune(segmentInfos);
|
checkpoint();
|
||||||
|
}
|
||||||
|
bufferedDeletesStream.prune(segmentInfos);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Expert: Return the total size of all index files currently cached in memory.
|
/** Expert: Return the total size of all index files currently cached in memory.
|
||||||
|
@ -20,6 +20,8 @@ import java.util.concurrent.ConcurrentHashMap;
|
|||||||
|
|
||||||
import org.apache.lucene.document.Document;
|
import org.apache.lucene.document.Document;
|
||||||
|
|
||||||
|
// nocommit jdoc
|
||||||
|
// nocommit -- can/should apps set this via IWC
|
||||||
public class ThreadAffinityDocumentsWriterThreadPool extends DocumentsWriterPerThreadPool {
|
public class ThreadAffinityDocumentsWriterThreadPool extends DocumentsWriterPerThreadPool {
|
||||||
private Map<Thread, ThreadState> threadBindings = new ConcurrentHashMap<Thread, ThreadState>();
|
private Map<Thread, ThreadState> threadBindings = new ConcurrentHashMap<Thread, ThreadState>();
|
||||||
|
|
||||||
@ -37,8 +39,12 @@ public class ThreadAffinityDocumentsWriterThreadPool extends DocumentsWriterPerT
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
ThreadState minThreadState = null;
|
ThreadState minThreadState = null;
|
||||||
// find the state that has minimum amount of threads waiting
|
|
||||||
|
// Find the state that has minimum number of threads waiting
|
||||||
|
// noocommit -- can't another thread lock the
|
||||||
|
// minThreadState we just got?
|
||||||
minThreadState = minContendedThreadState();
|
minThreadState = minContendedThreadState();
|
||||||
|
|
||||||
if (minThreadState == null || minThreadState.hasQueuedThreads()) {
|
if (minThreadState == null || minThreadState.hasQueuedThreads()) {
|
||||||
ThreadState newState = newThreadState();
|
ThreadState newState = newThreadState();
|
||||||
if (newState != null) {
|
if (newState != null) {
|
||||||
@ -59,6 +65,7 @@ public class ThreadAffinityDocumentsWriterThreadPool extends DocumentsWriterPerT
|
|||||||
return minThreadState;
|
return minThreadState;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/*
|
||||||
@Override
|
@Override
|
||||||
public void clearThreadBindings(ThreadState perThread) {
|
public void clearThreadBindings(ThreadState perThread) {
|
||||||
threadBindings.clear();
|
threadBindings.clear();
|
||||||
@ -68,5 +75,5 @@ public class ThreadAffinityDocumentsWriterThreadPool extends DocumentsWriterPerT
|
|||||||
public void clearAllThreadBindings() {
|
public void clearAllThreadBindings() {
|
||||||
threadBindings.clear();
|
threadBindings.clear();
|
||||||
}
|
}
|
||||||
|
*/
|
||||||
}
|
}
|
||||||
|
Loading…
x
Reference in New Issue
Block a user