--- lucene/CHANGES.txt 2011-04-29 13:14:04.293476900 +0200 +++ lucene/CHANGES.txt 2011-04-29 14:58:58.936050500 +0200 @@ -174,6 +174,70 @@ successful commit. The corresponding file format changes are backwards- compatible. (Michael Busch, Simon Willnauer) +* LUCENE-2956, LUCENE-2573, LUCENE-2324, LUCENE-2555: Changes from + DocumentsWriterPerThread: + + - IndexWriter now uses a DocumentsWriter per thread when indexing documents. + Each DocumentsWriterPerThread indexes documents in its own private segment, + and the in memory segments are no longer merged on flush. Instead, each + segment is separately flushed to disk and subsequently merged with normal + segment merging. + + - DocumentsWriterPerThread (DWPT) is now flushed concurrently based on a + FlushPolicy. When a DWPT is flushed, a fresh DWPT is swapped in so that + indexing may continue concurrently with flushing. The selected + DWPT flushes all its RAM resident documents do disk. Note: Segment flushes + don't flush all RAM resident documents but only the documents private to + the DWPT selected for flushing. + + - Flushing is now controlled by FlushPolicy that is called for every add, + update or delete on IndexWriter. By default DWPTs are flushed either on + maxBufferedDocs per DWPT or the global active used memory. Once the active + memory exceeds ramBufferSizeMB only the largest DWPT is selected for + flushing and the memory used by this DWPT is substracted from the active + memory and added to a flushing memory pool, which can lead to temporarily + higher memory usage due to ongoing indexing. + + - IndexWriter now can utilize ramBufferSize > 2048 MB. Each DWPT can address + up to 2048 MB memory such that the ramBufferSize is now bounded by the max + number of DWPT avaliable in the used DocumentsWriterPerThreadPool. + IndexWriters net memory consumption can grow far beyond the 2048 MB limit if + the applicatoin can use all available DWPTs. To prevent a DWPT from + exhausting its address space IndexWriter will forcefully flush a DWPT if its + hard memory limit is exceeded. The RAMPerThreadHardLimitMB can be controlled + via IndexWriterConfig and defaults to 1945 MB. + Since IndexWriter flushes DWPT concurrently not all memory is released + immediately. Applications should still use a ramBufferSize significantly + lower than the JVMs avaliable heap memory since under high load multiple + flushing DWPT can consume substantial transient memory when IO performance + is slow relative to indexing rate. + + - IndexWriter#commit now doesn't block concurrent indexing while flushing all + 'currently' RAM resident documents to disk. Yet, flushes that occur while a + a full flush is running are queued and will happen after all DWPT involved + in the full flush are done flushing. Applications using multiple threads + during indexing and trigger a full flush (eg call commmit() or open a new + NRT reader) can use significantly more transient memory. + + - IndexWriter#addDocument and IndexWriter.updateDocument can block indexing + threads if the number of active + number of flushing DWPT exceed a + safety limit. By default this happens if 2 * max number available thread + states (DWPTPool) is exceeded. This safety limit prevents applications from + exhausting their available memory if flushing can't keep up with + concurrently indexing threads. + + - IndexWriter only applies and flushes deletes if the maxBufferedDelTerms + limit is reached during indexing. No segment flushes will be triggered + due to this setting. + + - IndexWriter#flush(boolean, boolean) doesn't synchronized on IndexWriter + anymore. A dedicated flushLock has been introduced to prevent multiple full- + flushes happening concurrently. + + - DocumentsWriter doesn't write shared doc stores anymore. + + (Mike McCandless, Michael Busch, Simon Willnauer) + API Changes * LUCENE-2302, LUCENE-1458, LUCENE-2111, LUCENE-2514: Terms are no longer --- lucene/src/java/org/apache/lucene/index/BufferedDeletes.java 2011-02-01 18:38:12.012018200 +0100 +++ lucene/src/java/org/apache/lucene/index/BufferedDeletes.java 2011-04-29 15:01:00.013975800 +0200 @@ -72,13 +72,18 @@ public static final Integer MAX_INT = Integer.valueOf(Integer.MAX_VALUE); - final AtomicLong bytesUsed = new AtomicLong(); + final AtomicLong bytesUsed; private final static boolean VERBOSE_DELETES = false; long gen; - public BufferedDeletes(boolean sortTerms) { + this(sortTerms, new AtomicLong()); + } + + BufferedDeletes(boolean sortTerms, AtomicLong bytesUsed) { + assert bytesUsed != null; + this.bytesUsed = bytesUsed; if (sortTerms) { terms = new TreeMap(); } else { --- lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java 2011-02-18 13:53:43.701144600 +0100 +++ lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java 2011-04-29 15:01:07.395397900 +0200 @@ -33,8 +33,8 @@ import org.apache.lucene.search.Scorer; import org.apache.lucene.search.Weight; -/* Tracks the stream of {@link BuffereDeletes}. - * When DocumensWriter flushes, its buffered +/* Tracks the stream of {@link BufferedDeletes}. + * When DocumentsWriterPerThread flushes, its buffered * deletes are appended to this stream. We later * apply these deletes (resolve them to the actual * docIDs, per segment) when a merge is started @@ -82,17 +82,27 @@ // Appends a new packet of buffered deletes to the stream, // setting its generation: - public synchronized void push(FrozenBufferedDeletes packet) { + public synchronized long push(FrozenBufferedDeletes packet) { + /* + * The insert operation must be atomic. If we let threads increment the gen + * and push the packet afterwards we risk that packets are out of order. + * With DWPT this is possible if two or more flushes are racing for pushing + * updates. If the pushed packets get our of order would loose documents + * since deletes are applied to the wrong segments. + */ + packet.setDelGen(nextGen++); assert packet.any(); assert checkDeleteStats(); - assert packet.gen < nextGen; + assert packet.delGen() < nextGen; + assert deletes.isEmpty() || deletes.get(deletes.size()-1).delGen() < packet.delGen() : "Delete packets must be in order"; deletes.add(packet); numTerms.addAndGet(packet.numTermDeletes); bytesUsed.addAndGet(packet.bytesUsed); if (infoStream != null) { - message("push deletes " + packet + " delGen=" + packet.gen + " packetCount=" + deletes.size()); + message("push deletes " + packet + " delGen=" + packet.delGen() + " packetCount=" + deletes.size()); } assert checkDeleteStats(); + return packet.delGen(); } public synchronized void clear() { @@ -132,7 +142,7 @@ } // Sorts SegmentInfos from smallest to biggest bufferedDelGen: - private static final Comparator sortByDelGen = new Comparator() { + private static final Comparator sortSegInfoByDelGen = new Comparator() { // @Override -- not until Java 1.6 public int compare(SegmentInfo si1, SegmentInfo si2) { final long cmp = si1.getBufferedDeletesGen() - si2.getBufferedDeletesGen(); @@ -147,7 +157,7 @@ @Override public boolean equals(Object other) { - return sortByDelGen == other; + return sortSegInfoByDelGen == other; } }; @@ -174,7 +184,7 @@ SegmentInfos infos2 = new SegmentInfos(); infos2.addAll(infos); - Collections.sort(infos2, sortByDelGen); + Collections.sort(infos2, sortSegInfoByDelGen); BufferedDeletes coalescedDeletes = null; boolean anyNewDeletes = false; @@ -191,19 +201,30 @@ final SegmentInfo info = infos2.get(infosIDX); final long segGen = info.getBufferedDeletesGen(); - if (packet != null && segGen < packet.gen) { + if (packet != null && segGen < packet.delGen()) { //System.out.println(" coalesce"); if (coalescedDeletes == null) { coalescedDeletes = new BufferedDeletes(true); } + if (!packet.isSegmentPrivate) { + /* + * Only coalesce if we are NOT on a segment private del packet: the segment private del packet + * must only applied to segments with the same delGen. Yet, if a segment is already deleted + * from the SI since it had no more documents remaining after some del packets younger than + * its segPrivate packet (higher delGen) have been applied, the segPrivate packet has not been + * removed. + */ coalescedDeletes.update(packet); + } + delIDX--; - } else if (packet != null && segGen == packet.gen) { + } else if (packet != null && segGen == packet.delGen()) { + assert packet.isSegmentPrivate : "Packet and Segments deletegen can only match on a segment private del packet"; //System.out.println(" eq"); // Lock order: IW -> BD -> RP assert readerPool.infoIsLive(info); - SegmentReader reader = readerPool.get(info, false); + final SegmentReader reader = readerPool.get(info, false); int delCount = 0; final boolean segAllDeletes; try { @@ -213,7 +234,7 @@ delCount += applyQueryDeletes(coalescedDeletes.queriesIterable(), reader); } //System.out.println(" del exact"); - // Don't delete by Term here; DocumentsWriter + // Don't delete by Term here; DocumentsWriterPerThread // already did that on flush: delCount += applyQueryDeletes(packet.queriesIterable(), reader); segAllDeletes = reader.numDocs() == 0; @@ -236,7 +257,12 @@ if (coalescedDeletes == null) { coalescedDeletes = new BufferedDeletes(true); } - coalescedDeletes.update(packet); + + /* + * Since we are on a segment private del packet we must not + * update the coalescedDeletes here! We can simply advance to the + * next packet and seginfo. + */ delIDX--; infosIDX--; info.setBufferedDeletesGen(nextGen); @@ -285,7 +311,7 @@ return new ApplyDeletesResult(anyNewDeletes, nextGen++, allDeleted); } - public synchronized long getNextGen() { + synchronized long getNextGen() { return nextGen++; } @@ -303,10 +329,9 @@ if (infoStream != null) { message("prune sis=" + segmentInfos + " minGen=" + minGen + " packetCount=" + deletes.size()); } - final int limit = deletes.size(); for(int delIDX=0;delIDX= minGen) { + if (deletes.get(delIDX).delGen() >= minGen) { prune(delIDX); assert checkDeleteStats(); return; --- lucene/src/java/org/apache/lucene/index/ByteSliceWriter.java 2011-01-26 17:07:32.312150000 +0100 +++ lucene/src/java/org/apache/lucene/index/ByteSliceWriter.java 2011-04-29 15:01:04.477231000 +0200 @@ -81,6 +81,6 @@ } public int getAddress() { - return upto + (offset0 & DocumentsWriter.BYTE_BLOCK_NOT_MASK); + return upto + (offset0 & DocumentsWriterPerThread.BYTE_BLOCK_NOT_MASK); } } \ No newline at end of file --- lucene/src/java/org/apache/lucene/index/DocConsumer.java 2011-01-26 17:07:33.479216800 +0100 +++ lucene/src/java/org/apache/lucene/index/DocConsumer.java 2011-04-29 15:01:31.985804400 +0200 @@ -18,11 +18,12 @@ */ import java.io.IOException; -import java.util.Collection; abstract class DocConsumer { - abstract DocConsumerPerThread addThread(DocumentsWriterThreadState perThread) throws IOException; - abstract void flush(final Collection threads, final SegmentWriteState state) throws IOException; + abstract void processDocument(FieldInfos fieldInfos) throws IOException; + abstract void finishDocument() throws IOException; + abstract void flush(final SegmentWriteState state) throws IOException; abstract void abort(); abstract boolean freeRAM(); + abstract void doAfterFlush(); } --- lucene/src/java/org/apache/lucene/index/DocConsumerPerThread.java 2011-03-22 19:30:26.269097400 +0100 +++ lucene/src/java/org/apache/lucene/index/DocConsumerPerThread.java 1970-01-01 01:00:00.000000000 +0100 @@ -1,34 +0,0 @@ -package org.apache.lucene.index; - -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import java.io.IOException; - -abstract class DocConsumerPerThread { - - /** Process the document. If there is - * something for this document to be done in docID order, - * you should encapsulate that as a - * DocumentsWriter.DocWriter and return it. - * DocumentsWriter then calls finish() on this object - * when it's its turn. */ - abstract DocumentsWriter.DocWriter processDocument(FieldInfos fieldInfos) throws IOException; - - abstract void doAfterFlush(); - abstract void abort(); -} --- lucene/src/java/org/apache/lucene/index/DocFieldConsumer.java 2011-03-22 19:30:26.350102000 +0100 +++ lucene/src/java/org/apache/lucene/index/DocFieldConsumer.java 2011-04-29 15:01:05.874310900 +0200 @@ -18,22 +18,25 @@ */ import java.io.IOException; -import java.util.Collection; import java.util.Map; abstract class DocFieldConsumer { - /** Called when DocumentsWriter decides to create a new + /** Called when DocumentsWriterPerThread decides to create a new * segment */ - abstract void flush(Map> threadsAndFields, SegmentWriteState state) throws IOException; + abstract void flush(Map fieldsToFlush, SegmentWriteState state) throws IOException; /** Called when an aborting exception is hit */ abstract void abort(); - /** Add a new thread */ - abstract DocFieldConsumerPerThread addThread(DocFieldProcessorPerThread docFieldProcessorPerThread) throws IOException; - - /** Called when DocumentsWriter is using too much RAM. + /** Called when DocumentsWriterPerThread is using too much RAM. * The consumer should free RAM, if possible, returning * true if any RAM was in fact freed. */ abstract boolean freeRAM(); - } + + abstract void startDocument() throws IOException; + + abstract DocFieldConsumerPerField addField(FieldInfo fi); + + abstract void finishDocument() throws IOException; + +} --- lucene/src/java/org/apache/lucene/index/DocFieldConsumerPerField.java 2011-01-26 17:07:32.469159000 +0100 +++ lucene/src/java/org/apache/lucene/index/DocFieldConsumerPerField.java 2011-04-29 15:01:06.824365300 +0200 @@ -24,4 +24,5 @@ /** Processes all occurrences of a single field */ abstract void processFields(Fieldable[] fields, int count) throws IOException; abstract void abort(); + abstract FieldInfo getFieldInfo(); } --- lucene/src/java/org/apache/lucene/index/DocFieldConsumerPerThread.java 2011-01-26 17:07:32.684171300 +0100 +++ lucene/src/java/org/apache/lucene/index/DocFieldConsumerPerThread.java 1970-01-01 01:00:00.000000000 +0100 @@ -1,27 +0,0 @@ -package org.apache.lucene.index; - -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import java.io.IOException; - -abstract class DocFieldConsumerPerThread { - abstract void startDocument() throws IOException; - abstract DocumentsWriter.DocWriter finishDocument() throws IOException; - abstract DocFieldConsumerPerField addField(FieldInfo fi); - abstract void abort(); -} --- lucene/src/java/org/apache/lucene/index/DocFieldConsumers.java 1970-01-01 01:00:00.000000000 +0100 +++ lucene/src/java/org/apache/lucene/index/DocFieldConsumers.java 2011-04-29 15:01:08.077437000 +0200 @@ -0,0 +1,90 @@ +package org.apache.lucene.index; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +/** This is just a "splitter" class: it lets you wrap two + * DocFieldConsumer instances as a single consumer. */ + +final class DocFieldConsumers extends DocFieldConsumer { + final DocFieldConsumer one; + final DocFieldConsumer two; + final DocumentsWriterPerThread.DocState docState; + + public DocFieldConsumers(DocFieldProcessor processor, DocFieldConsumer one, DocFieldConsumer two) { + this.one = one; + this.two = two; + this.docState = processor.docState; + } + + @Override + public void flush(Map fieldsToFlush, SegmentWriteState state) throws IOException { + + Map oneFieldsToFlush = new HashMap(); + Map twoFieldsToFlush = new HashMap(); + + for (Map.Entry fieldToFlush : fieldsToFlush.entrySet()) { + DocFieldConsumersPerField perField = (DocFieldConsumersPerField) fieldToFlush.getValue(); + oneFieldsToFlush.put(fieldToFlush.getKey(), perField.one); + twoFieldsToFlush.put(fieldToFlush.getKey(), perField.two); + } + + one.flush(oneFieldsToFlush, state); + two.flush(twoFieldsToFlush, state); + } + + @Override + public void abort() { + try { + one.abort(); + } finally { + two.abort(); + } + } + + @Override + public boolean freeRAM() { + boolean any = one.freeRAM(); + any |= two.freeRAM(); + return any; + } + + @Override + public void finishDocument() throws IOException { + try { + one.finishDocument(); + } finally { + two.finishDocument(); + } + } + + @Override + public void startDocument() throws IOException { + one.startDocument(); + two.startDocument(); + } + + @Override + public DocFieldConsumerPerField addField(FieldInfo fi) { + return new DocFieldConsumersPerField(this, fi, one.addField(fi), two.addField(fi)); + } + +} --- lucene/src/java/org/apache/lucene/index/DocFieldConsumersPerField.java 1970-01-01 01:00:00.000000000 +0100 +++ lucene/src/java/org/apache/lucene/index/DocFieldConsumersPerField.java 2011-04-29 15:01:09.994546600 +0200 @@ -0,0 +1,56 @@ +package org.apache.lucene.index; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; +import org.apache.lucene.document.Fieldable; + +final class DocFieldConsumersPerField extends DocFieldConsumerPerField { + + final DocFieldConsumerPerField one; + final DocFieldConsumerPerField two; + final DocFieldConsumers parent; + final FieldInfo fieldInfo; + + public DocFieldConsumersPerField(DocFieldConsumers parent, FieldInfo fi, DocFieldConsumerPerField one, DocFieldConsumerPerField two) { + this.parent = parent; + this.one = one; + this.two = two; + this.fieldInfo = fi; + } + + @Override + public void processFields(Fieldable[] fields, int count) throws IOException { + one.processFields(fields, count); + two.processFields(fields, count); + } + + @Override + public void abort() { + try { + one.abort(); + } finally { + two.abort(); + } + } + + @Override + FieldInfo getFieldInfo() { + return fieldInfo; + } +} --- lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java 2011-03-22 19:30:26.313099900 +0100 +++ lucene/src/java/org/apache/lucene/index/DocFieldProcessor.java 2011-04-29 15:01:03.163155900 +0200 @@ -19,8 +19,13 @@ import java.io.IOException; import java.util.Collection; -import java.util.Map; import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; + +import org.apache.lucene.document.Document; +import org.apache.lucene.document.Fieldable; /** @@ -33,26 +38,39 @@ final class DocFieldProcessor extends DocConsumer { - final DocumentsWriter docWriter; final DocFieldConsumer consumer; final StoredFieldsWriter fieldsWriter; - public DocFieldProcessor(DocumentsWriter docWriter, DocFieldConsumer consumer) { - this.docWriter = docWriter; + // Holds all fields seen in current doc + DocFieldProcessorPerField[] fields = new DocFieldProcessorPerField[1]; + int fieldCount; + + // Hash table for all fields ever seen + DocFieldProcessorPerField[] fieldHash = new DocFieldProcessorPerField[2]; + int hashMask = 1; + int totalFieldCount; + + float docBoost; + int fieldGen; + final DocumentsWriterPerThread.DocState docState; + + public DocFieldProcessor(DocumentsWriterPerThread docWriter, DocFieldConsumer consumer) { + this.docState = docWriter.docState; this.consumer = consumer; fieldsWriter = new StoredFieldsWriter(docWriter); } @Override - public void flush(Collection threads, SegmentWriteState state) throws IOException { + public void flush(SegmentWriteState state) throws IOException { - Map> childThreadsAndFields = new HashMap>(); - for ( DocConsumerPerThread thread : threads) { - DocFieldProcessorPerThread perThread = (DocFieldProcessorPerThread) thread; - childThreadsAndFields.put(perThread.consumer, perThread.fields()); + Map childFields = new HashMap(); + Collection fields = fields(); + for (DocFieldConsumerPerField f : fields) { + childFields.put(f.getFieldInfo(), f); } + fieldsWriter.flush(state); - consumer.flush(childThreadsAndFields, state); + consumer.flush(childFields, state); // Important to save after asking consumer to flush so // consumer can alter the FieldInfo* if necessary. EG, @@ -64,17 +82,234 @@ @Override public void abort() { + for(int i=0;i fields() { + Collection fields = new HashSet(); + for(int i=0;i fieldHash.length; + + final DocFieldProcessorPerField newHashArray[] = new DocFieldProcessorPerField[newHashSize]; + + // Rehash + int newHashMask = newHashSize-1; + for(int j=0;j docFields = doc.getFields(); + final int numDocFields = docFields.size(); + + // Absorb any new fields first seen in this document. + // Also absorb any changes to fields we had already + // seen before (eg suddenly turning on norms or + // vectors, etc.): + + for(int i=0;i= fieldHash.length/2) + rehash(); + } else { + fieldInfos.addOrUpdate(fp.fieldInfo.name, field.isIndexed(), field.isTermVectorStored(), + field.isStorePositionWithTermVector(), field.isStoreOffsetWithTermVector(), + field.getOmitNorms(), false, field.getOmitTermFreqAndPositions()); + } + + if (thisFieldGen != fp.lastGen) { + + // First time we're seeing this field for this doc + fp.fieldCount = 0; + + if (fieldCount == fields.length) { + final int newSize = fields.length*2; + DocFieldProcessorPerField newArray[] = new DocFieldProcessorPerField[newSize]; + System.arraycopy(fields, 0, newArray, 0, fieldCount); + fields = newArray; + } + + fields[fieldCount++] = fp; + fp.lastGen = thisFieldGen; + } + + fp.addField(field); + + if (field.isStored()) { + fieldsWriter.addField(field, fp.fieldInfo); + } + } + + // If we are writing vectors then we must visit + // fields in sorted order so they are written in + // sorted order. TODO: we actually only need to + // sort the subset of fields that have vectors + // enabled; we could save [small amount of] CPU + // here. + quickSort(fields, 0, fieldCount-1); + + for(int i=0;i= hi) + return; + else if (hi == 1+lo) { + if (array[lo].fieldInfo.name.compareTo(array[hi].fieldInfo.name) > 0) { + final DocFieldProcessorPerField tmp = array[lo]; + array[lo] = array[hi]; + array[hi] = tmp; + } + return; + } + + int mid = (lo + hi) >>> 1; + + if (array[lo].fieldInfo.name.compareTo(array[mid].fieldInfo.name) > 0) { + DocFieldProcessorPerField tmp = array[lo]; + array[lo] = array[mid]; + array[mid] = tmp; + } + + if (array[mid].fieldInfo.name.compareTo(array[hi].fieldInfo.name) > 0) { + DocFieldProcessorPerField tmp = array[mid]; + array[mid] = array[hi]; + array[hi] = tmp; + + if (array[lo].fieldInfo.name.compareTo(array[mid].fieldInfo.name) > 0) { + DocFieldProcessorPerField tmp2 = array[lo]; + array[lo] = array[mid]; + array[mid] = tmp2; + } + } + + int left = lo + 1; + int right = hi - 1; + + if (left >= right) + return; + + DocFieldProcessorPerField partition = array[mid]; + + for (; ;) { + while (array[right].fieldInfo.name.compareTo(partition.fieldInfo.name) > 0) + --right; + + while (left < right && array[left].fieldInfo.name.compareTo(partition.fieldInfo.name) <= 0) + ++left; + + if (left < right) { + DocFieldProcessorPerField tmp = array[left]; + array[left] = array[right]; + array[right] = tmp; + --right; + } else { + break; + } + } + + quickSort(array, lo, left); + quickSort(array, left + 1, hi); } } --- lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java 2011-01-26 17:07:32.352152300 +0100 +++ lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerField.java 2011-04-29 15:01:05.413284600 +0200 @@ -18,6 +18,8 @@ */ import org.apache.lucene.document.Fieldable; +import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.RamUsageEstimator; /** * Holds all per thread, per field state. @@ -34,11 +36,22 @@ int fieldCount; Fieldable[] fields = new Fieldable[1]; - public DocFieldProcessorPerField(final DocFieldProcessorPerThread perThread, final FieldInfo fieldInfo) { - this.consumer = perThread.consumer.addField(fieldInfo); + public DocFieldProcessorPerField(final DocFieldProcessor docFieldProcessor, final FieldInfo fieldInfo) { + this.consumer = docFieldProcessor.consumer.addField(fieldInfo); this.fieldInfo = fieldInfo; } + public void addField(Fieldable field) { + if (fieldCount == fields.length) { + int newSize = ArrayUtil.oversize(fieldCount + 1, RamUsageEstimator.NUM_BYTES_OBJECT_REF); + Fieldable[] newArray = new Fieldable[newSize]; + System.arraycopy(fields, 0, newArray, 0, fieldCount); + fields = newArray; + } + + fields[fieldCount++] = field; + } + public void abort() { consumer.abort(); } --- lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java 2011-04-20 23:25:31.169989300 +0200 +++ lucene/src/java/org/apache/lucene/index/DocFieldProcessorPerThread.java 1970-01-01 01:00:00.000000000 +0100 @@ -1,307 +0,0 @@ -package org.apache.lucene.index; - -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import java.util.Comparator; -import java.util.Collection; -import java.util.HashSet; -import java.util.List; -import java.io.IOException; -import org.apache.lucene.document.Document; -import org.apache.lucene.document.Fieldable; -import org.apache.lucene.util.ArrayUtil; -import org.apache.lucene.util.RamUsageEstimator; - -/** - * Gathers all Fieldables for a document under the same - * name, updates FieldInfos, and calls per-field consumers - * to process field by field. - * - * Currently, only a single thread visits the fields, - * sequentially, for processing. - */ - -final class DocFieldProcessorPerThread extends DocConsumerPerThread { - - float docBoost; - int fieldGen; - final DocFieldProcessor docFieldProcessor; - final DocFieldConsumerPerThread consumer; - - // Holds all fields seen in current doc - DocFieldProcessorPerField[] fields = new DocFieldProcessorPerField[1]; - int fieldCount; - - // Hash table for all fields seen in current segment - DocFieldProcessorPerField[] fieldHash = new DocFieldProcessorPerField[2]; - int hashMask = 1; - int totalFieldCount; - - final StoredFieldsWriterPerThread fieldsWriter; - - final DocumentsWriter.DocState docState; - - public DocFieldProcessorPerThread(DocumentsWriterThreadState threadState, DocFieldProcessor docFieldProcessor) throws IOException { - this.docState = threadState.docState; - this.docFieldProcessor = docFieldProcessor; - this.consumer = docFieldProcessor.consumer.addThread(this); - fieldsWriter = docFieldProcessor.fieldsWriter.addThread(docState); - } - - @Override - public void abort() { - for(int i=0;i fields() { - Collection fields = new HashSet(); - for(int i=0;i fieldHash.length; - - final DocFieldProcessorPerField newHashArray[] = new DocFieldProcessorPerField[newHashSize]; - - // Rehash - int newHashMask = newHashSize-1; - for(int j=0;j docFields = doc.getFields(); - final int numDocFields = docFields.size(); - - // Absorb any new fields first seen in this document. - // Also absorb any changes to fields we had already - // seen before (eg suddenly turning on norms or - // vectors, etc.): - - for(int i=0;i= fieldHash.length/2) - rehash(); - } else { - fieldInfos.addOrUpdate(fp.fieldInfo.name, field.isIndexed(), field.isTermVectorStored(), - field.isStorePositionWithTermVector(), field.isStoreOffsetWithTermVector(), - field.getOmitNorms(), false, field.getOmitTermFreqAndPositions()); - } - if (thisFieldGen != fp.lastGen) { - - // First time we're seeing this field for this doc - fp.fieldCount = 0; - - if (fieldCount == fields.length) { - final int newSize = fields.length*2; - DocFieldProcessorPerField newArray[] = new DocFieldProcessorPerField[newSize]; - System.arraycopy(fields, 0, newArray, 0, fieldCount); - fields = newArray; - } - - fields[fieldCount++] = fp; - fp.lastGen = thisFieldGen; - } - - if (fp.fieldCount == fp.fields.length) { - Fieldable[] newArray = new Fieldable[fp.fields.length*2]; - System.arraycopy(fp.fields, 0, newArray, 0, fp.fieldCount); - fp.fields = newArray; - } - - fp.fields[fp.fieldCount++] = field; - if (field.isStored()) { - fieldsWriter.addField(field, fp.fieldInfo); - } - } - - // If we are writing vectors then we must visit - // fields in sorted order so they are written in - // sorted order. TODO: we actually only need to - // sort the subset of fields that have vectors - // enabled; we could save [small amount of] CPU - // here. - ArrayUtil.quickSort(fields, 0, fieldCount, fieldsComp); - - for(int i=0;i fieldsComp = new Comparator() { - public int compare(DocFieldProcessorPerField o1, DocFieldProcessorPerField o2) { - return o1.fieldInfo.name.compareTo(o2.fieldInfo.name); - } - }; - - PerDoc[] docFreeList = new PerDoc[1]; - int freeCount; - int allocCount; - - synchronized PerDoc getPerDoc() { - if (freeCount == 0) { - allocCount++; - if (allocCount > docFreeList.length) { - // Grow our free list up front to make sure we have - // enough space to recycle all outstanding PerDoc - // instances - assert allocCount == 1+docFreeList.length; - docFreeList = new PerDoc[ArrayUtil.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)]; - } - return new PerDoc(); - } else - return docFreeList[--freeCount]; - } - - synchronized void freePerDoc(PerDoc perDoc) { - assert freeCount < docFreeList.length; - docFreeList[freeCount++] = perDoc; - } - - class PerDoc extends DocumentsWriter.DocWriter { - - DocumentsWriter.DocWriter one; - DocumentsWriter.DocWriter two; - - @Override - public long sizeInBytes() { - return one.sizeInBytes() + two.sizeInBytes(); - } - - @Override - public void finish() throws IOException { - try { - try { - one.finish(); - } finally { - two.finish(); - } - } finally { - freePerDoc(this); - } - } - - @Override - public void abort() { - try { - try { - one.abort(); - } finally { - two.abort(); - } - } finally { - freePerDoc(this); - } - } - } -} \ No newline at end of file --- lucene/src/java/org/apache/lucene/index/DocInverter.java 2011-03-22 19:30:26.434106800 +0100 +++ lucene/src/java/org/apache/lucene/index/DocInverter.java 2011-04-29 15:01:32.647842300 +0200 @@ -18,12 +18,13 @@ */ import java.io.IOException; -import java.util.Collection; import java.util.HashMap; -import java.util.HashSet; - import java.util.Map; +import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; +import org.apache.lucene.analysis.tokenattributes.OffsetAttribute; +import org.apache.lucene.util.AttributeSource; + /** This is a DocFieldConsumer that inverts each field, * separately, from a Document, and accepts a @@ -34,43 +35,73 @@ final InvertedDocConsumer consumer; final InvertedDocEndConsumer endConsumer; - public DocInverter(InvertedDocConsumer consumer, InvertedDocEndConsumer endConsumer) { + final DocumentsWriterPerThread.DocState docState; + + final FieldInvertState fieldState = new FieldInvertState(); + + final SingleTokenAttributeSource singleToken = new SingleTokenAttributeSource(); + + static class SingleTokenAttributeSource extends AttributeSource { + final CharTermAttribute termAttribute; + final OffsetAttribute offsetAttribute; + + private SingleTokenAttributeSource() { + termAttribute = addAttribute(CharTermAttribute.class); + offsetAttribute = addAttribute(OffsetAttribute.class); + } + + public void reinit(String stringValue, int startOffset, int endOffset) { + termAttribute.setEmpty().append(stringValue); + offsetAttribute.setOffset(startOffset, endOffset); + } + } + + // Used to read a string value for a field + final ReusableStringReader stringReader = new ReusableStringReader(); + + public DocInverter(DocumentsWriterPerThread.DocState docState, InvertedDocConsumer consumer, InvertedDocEndConsumer endConsumer) { + this.docState = docState; this.consumer = consumer; this.endConsumer = endConsumer; } @Override - void flush(Map> threadsAndFields, SegmentWriteState state) throws IOException { - - Map> childThreadsAndFields = new HashMap>(); - Map> endChildThreadsAndFields = new HashMap>(); - - for (Map.Entry> entry : threadsAndFields.entrySet() ) { + void flush(Map fieldsToFlush, SegmentWriteState state) throws IOException { + Map childFieldsToFlush = new HashMap(); + Map endChildFieldsToFlush = new HashMap(); - DocInverterPerThread perThread = (DocInverterPerThread) entry.getKey(); + for (Map.Entry fieldToFlush : fieldsToFlush.entrySet()) { + DocInverterPerField perField = (DocInverterPerField) fieldToFlush.getValue(); + childFieldsToFlush.put(fieldToFlush.getKey(), perField.consumer); + endChildFieldsToFlush.put(fieldToFlush.getKey(), perField.endConsumer); + } - Collection childFields = new HashSet(); - Collection endChildFields = new HashSet(); - for (final DocFieldConsumerPerField field: entry.getValue() ) { - DocInverterPerField perField = (DocInverterPerField) field; - childFields.add(perField.consumer); - endChildFields.add(perField.endConsumer); + consumer.flush(childFieldsToFlush, state); + endConsumer.flush(endChildFieldsToFlush, state); } - childThreadsAndFields.put(perThread.consumer, childFields); - endChildThreadsAndFields.put(perThread.endConsumer, endChildFields); + @Override + public void startDocument() throws IOException { + consumer.startDocument(); + endConsumer.startDocument(); } - consumer.flush(childThreadsAndFields, state); - endConsumer.flush(endChildThreadsAndFields, state); + public void finishDocument() throws IOException { + // TODO: allow endConsumer.finishDocument to also return + // a DocWriter + endConsumer.finishDocument(); + consumer.finishDocument(); } @Override void abort() { + try { consumer.abort(); + } finally { endConsumer.abort(); } + } @Override public boolean freeRAM() { @@ -78,7 +109,8 @@ } @Override - public DocFieldConsumerPerThread addThread(DocFieldProcessorPerThread docFieldProcessorPerThread) { - return new DocInverterPerThread(docFieldProcessorPerThread, this); + public DocFieldConsumerPerField addField(FieldInfo fi) { + return new DocInverterPerField(this, fi); } + } --- lucene/src/java/org/apache/lucene/index/DocInverterPerField.java 2011-01-26 17:07:31.151083600 +0100 +++ lucene/src/java/org/apache/lucene/index/DocInverterPerField.java 2011-04-29 15:00:55.726730500 +0200 @@ -35,20 +35,20 @@ final class DocInverterPerField extends DocFieldConsumerPerField { - final private DocInverterPerThread perThread; - final private FieldInfo fieldInfo; + final private DocInverter parent; + final FieldInfo fieldInfo; final InvertedDocConsumerPerField consumer; final InvertedDocEndConsumerPerField endConsumer; - final DocumentsWriter.DocState docState; + final DocumentsWriterPerThread.DocState docState; final FieldInvertState fieldState; - public DocInverterPerField(DocInverterPerThread perThread, FieldInfo fieldInfo) { - this.perThread = perThread; + public DocInverterPerField(DocInverter parent, FieldInfo fieldInfo) { + this.parent = parent; this.fieldInfo = fieldInfo; - docState = perThread.docState; - fieldState = perThread.fieldState; - this.consumer = perThread.consumer.addField(this, fieldInfo); - this.endConsumer = perThread.endConsumer.addField(this, fieldInfo); + docState = parent.docState; + fieldState = parent.fieldState; + this.consumer = parent.consumer.addField(this, fieldInfo); + this.endConsumer = parent.endConsumer.addField(this, fieldInfo); } @Override @@ -80,8 +80,8 @@ if (!field.isTokenized()) { // un-tokenized field String stringValue = field.stringValue(); final int valueLength = stringValue.length(); - perThread.singleToken.reinit(stringValue, 0, valueLength); - fieldState.attributeSource = perThread.singleToken; + parent.singleToken.reinit(stringValue, 0, valueLength); + fieldState.attributeSource = parent.singleToken; consumer.start(field); boolean success = false; @@ -89,9 +89,10 @@ consumer.add(); success = true; } finally { - if (!success) + if (!success) { docState.docWriter.setAborting(); } + } fieldState.offset += valueLength; fieldState.length++; fieldState.position++; @@ -114,8 +115,8 @@ if (stringValue == null) { throw new IllegalArgumentException("field must have either TokenStream, String or Reader value"); } - perThread.stringReader.init(stringValue); - reader = perThread.stringReader; + parent.stringReader.init(stringValue); + reader = parent.stringReader; } // Tokenize field and add to postingTable @@ -166,9 +167,10 @@ consumer.add(); success = true; } finally { - if (!success) + if (!success) { docState.docWriter.setAborting(); } + } fieldState.length++; fieldState.position++; @@ -195,4 +197,9 @@ consumer.finish(); endConsumer.finish(); } + + @Override + FieldInfo getFieldInfo() { + return fieldInfo; + } } --- lucene/src/java/org/apache/lucene/index/DocInverterPerThread.java 2011-01-26 17:07:33.538220200 +0100 +++ lucene/src/java/org/apache/lucene/index/DocInverterPerThread.java 1970-01-01 01:00:00.000000000 +0100 @@ -1,92 +0,0 @@ -package org.apache.lucene.index; - -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import java.io.IOException; - -import org.apache.lucene.util.AttributeSource; -import org.apache.lucene.analysis.tokenattributes.OffsetAttribute; -import org.apache.lucene.analysis.tokenattributes.CharTermAttribute; - -/** This is a DocFieldConsumer that inverts each field, - * separately, from a Document, and accepts a - * InvertedTermsConsumer to process those terms. */ - -final class DocInverterPerThread extends DocFieldConsumerPerThread { - final DocInverter docInverter; - final InvertedDocConsumerPerThread consumer; - final InvertedDocEndConsumerPerThread endConsumer; - final SingleTokenAttributeSource singleToken = new SingleTokenAttributeSource(); - - static class SingleTokenAttributeSource extends AttributeSource { - final CharTermAttribute termAttribute; - final OffsetAttribute offsetAttribute; - - private SingleTokenAttributeSource() { - termAttribute = addAttribute(CharTermAttribute.class); - offsetAttribute = addAttribute(OffsetAttribute.class); - } - - public void reinit(String stringValue, int startOffset, int endOffset) { - termAttribute.setEmpty().append(stringValue); - offsetAttribute.setOffset(startOffset, endOffset); - } - } - - final DocumentsWriter.DocState docState; - - final FieldInvertState fieldState = new FieldInvertState(); - - // Used to read a string value for a field - final ReusableStringReader stringReader = new ReusableStringReader(); - - public DocInverterPerThread(DocFieldProcessorPerThread docFieldProcessorPerThread, DocInverter docInverter) { - this.docInverter = docInverter; - docState = docFieldProcessorPerThread.docState; - consumer = docInverter.consumer.addThread(this); - endConsumer = docInverter.endConsumer.addThread(this); - } - - @Override - public void startDocument() throws IOException { - consumer.startDocument(); - endConsumer.startDocument(); - } - - @Override - public DocumentsWriter.DocWriter finishDocument() throws IOException { - // TODO: allow endConsumer.finishDocument to also return - // a DocWriter - endConsumer.finishDocument(); - return consumer.finishDocument(); - } - - @Override - void abort() { - try { - consumer.abort(); - } finally { - endConsumer.abort(); - } - } - - @Override - public DocFieldConsumerPerField addField(FieldInfo fi) { - return new DocInverterPerField(this, fi); - } -} --- lucene/src/java/org/apache/lucene/index/DocumentsWriter.java 2011-03-26 11:40:50.284430700 +0100 +++ lucene/src/java/org/apache/lucene/index/DocumentsWriter.java 2011-04-29 15:01:12.261676300 +0200 @@ -19,36 +19,27 @@ import java.io.IOException; import java.io.PrintStream; -import java.text.NumberFormat; -import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; -import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedList; import java.util.List; -import java.util.concurrent.atomic.AtomicLong; +import java.util.Queue; +import java.util.concurrent.atomic.AtomicInteger; import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.document.Document; +import org.apache.lucene.index.DocumentsWriterPerThread.FlushedSegment; +import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain; +import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState; +import org.apache.lucene.index.FieldInfos.FieldNumberBiMap; import org.apache.lucene.search.Query; import org.apache.lucene.search.SimilarityProvider; import org.apache.lucene.store.AlreadyClosedException; import org.apache.lucene.store.Directory; -import org.apache.lucene.store.RAMFile; -import org.apache.lucene.util.ArrayUtil; -import org.apache.lucene.util.BitVector; -import org.apache.lucene.util.RamUsageEstimator; -import org.apache.lucene.util.RecyclingByteBlockAllocator; -import org.apache.lucene.util.ThreadInterruptedException; - -import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK; -import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE; /** * This class accepts multiple added documents and directly - * writes a single segment file. It does this more - * efficiently than creating a single segment per document - * (with DocumentWriter) and doing standard merges on those - * segments. + * writes segment files. * * Each added document is passed to the {@link DocConsumer}, * which in turn processes the document and interacts with @@ -111,266 +102,117 @@ */ final class DocumentsWriter { - final AtomicLong bytesUsed = new AtomicLong(0); - IndexWriter writer; Directory directory; - String segment; // Current segment we are working on - - private int nextDocID; // Next docID to be added - private int numDocs; // # of docs added, but not yet flushed - - // Max # ThreadState instances; if there are more threads - // than this they share ThreadStates - private DocumentsWriterThreadState[] threadStates = new DocumentsWriterThreadState[0]; - private final HashMap threadBindings = new HashMap(); - - boolean bufferIsFull; // True when it's time to write segment - private boolean aborting; // True if an abort is pending - - PrintStream infoStream; - SimilarityProvider similarityProvider; + private volatile boolean closed; - // max # simultaneous threads; if there are more than - // this, they wait for others to finish first - private final int maxThreadStates; - - // TODO: cutover to BytesRefHash - // Deletes for our still-in-RAM (to be flushed next) segment - private BufferedDeletes pendingDeletes = new BufferedDeletes(false); - - static class DocState { - DocumentsWriter docWriter; - Analyzer analyzer; PrintStream infoStream; SimilarityProvider similarityProvider; - int docID; - Document doc; - String maxTermPrefix; - - // Only called by asserts - public boolean testPoint(String name) { - return docWriter.writer.testPoint(name); - } - - public void clear() { - // don't hold onto doc nor analyzer, in case it is - // largish: - doc = null; - analyzer = null; - } - } - - /** Consumer returns this on each doc. This holds any - * state that must be flushed synchronized "in docID - * order". We gather these and flush them in order. */ - abstract static class DocWriter { - DocWriter next; - int docID; - abstract void finish() throws IOException; - abstract void abort(); - abstract long sizeInBytes(); - - void setNext(DocWriter next) { - this.next = next; - } - } - - /** - * Create and return a new DocWriterBuffer. - */ - PerDocBuffer newPerDocBuffer() { - return new PerDocBuffer(); - } - - /** - * RAMFile buffer for DocWriters. - */ - class PerDocBuffer extends RAMFile { - - /** - * Allocate bytes used from shared pool. - */ - @Override - protected byte[] newBuffer(int size) { - assert size == PER_DOC_BLOCK_SIZE; - return perDocAllocator.getByteBlock(); - } - - /** - * Recycle the bytes used. - */ - synchronized void recycle() { - if (buffers.size() > 0) { - setLength(0); - - // Recycle the blocks - perDocAllocator.recycleByteBlocks(buffers); - buffers.clear(); - sizeInBytes = 0; - - assert numBuffers() == 0; - } - } - } - - /** - * The IndexingChain must define the {@link #getChain(DocumentsWriter)} method - * which returns the DocConsumer that the DocumentsWriter calls to process the - * documents. - */ - abstract static class IndexingChain { - abstract DocConsumer getChain(DocumentsWriter documentsWriter); - } - - static final IndexingChain defaultIndexingChain = new IndexingChain() { - - @Override - DocConsumer getChain(DocumentsWriter documentsWriter) { - /* - This is the current indexing chain: - - DocConsumer / DocConsumerPerThread - --> code: DocFieldProcessor / DocFieldProcessorPerThread - --> DocFieldConsumer / DocFieldConsumerPerThread / DocFieldConsumerPerField - --> code: DocFieldConsumers / DocFieldConsumersPerThread / DocFieldConsumersPerField - --> code: DocInverter / DocInverterPerThread / DocInverterPerField - --> InvertedDocConsumer / InvertedDocConsumerPerThread / InvertedDocConsumerPerField - --> code: TermsHash / TermsHashPerThread / TermsHashPerField - --> TermsHashConsumer / TermsHashConsumerPerThread / TermsHashConsumerPerField - --> code: FreqProxTermsWriter / FreqProxTermsWriterPerThread / FreqProxTermsWriterPerField - --> code: TermVectorsTermsWriter / TermVectorsTermsWriterPerThread / TermVectorsTermsWriterPerField - --> InvertedDocEndConsumer / InvertedDocConsumerPerThread / InvertedDocConsumerPerField - --> code: NormsWriter / NormsWriterPerThread / NormsWriterPerField - --> code: StoredFieldsWriter / StoredFieldsWriterPerThread / StoredFieldsWriterPerField - */ - // Build up indexing chain: - - final TermsHashConsumer termVectorsWriter = new TermVectorsTermsWriter(documentsWriter); - final TermsHashConsumer freqProxWriter = new FreqProxTermsWriter(); - /* - * nesting TermsHash instances here to allow the secondary (TermVectors) share the interned postings - * via a shared ByteBlockPool. See TermsHashPerField for details. - */ - final TermsHash termVectorsTermHash = new TermsHash(documentsWriter, false, termVectorsWriter, null); - final InvertedDocConsumer termsHash = new TermsHash(documentsWriter, true, freqProxWriter, termVectorsTermHash); - final NormsWriter normsWriter = new NormsWriter(); - final DocInverter docInverter = new DocInverter(termsHash, normsWriter); - return new DocFieldProcessor(documentsWriter, docInverter); - } - }; + List newFiles; - final DocConsumer consumer; + final IndexWriter indexWriter; - // How much RAM we can use before flushing. This is 0 if - // we are flushing by doc count instead. + private AtomicInteger numDocsInRAM = new AtomicInteger(0); - private final IndexWriterConfig config; + // TODO: cut over to BytesRefHash in BufferedDeletes + volatile DocumentsWriterDeleteQueue deleteQueue = new DocumentsWriterDeleteQueue(); + private final Queue ticketQueue = new LinkedList(); - private boolean closed; - private FieldInfos fieldInfos; + private Collection abortedFiles; // List of files that were written before last abort() - private final BufferedDeletesStream bufferedDeletesStream; - private final IndexWriter.FlushControl flushControl; + final IndexingChain chain; - DocumentsWriter(IndexWriterConfig config, Directory directory, IndexWriter writer, IndexingChain indexingChain, FieldInfos fieldInfos, + final DocumentsWriterPerThreadPool perThreadPool; + final FlushPolicy flushPolicy; + final DocumentsWriterFlushControl flushControl; + final Healthiness healthiness; + DocumentsWriter(IndexWriterConfig config, Directory directory, IndexWriter writer, FieldNumberBiMap globalFieldNumbers, BufferedDeletesStream bufferedDeletesStream) throws IOException { this.directory = directory; - this.writer = writer; + this.indexWriter = writer; this.similarityProvider = config.getSimilarityProvider(); - this.maxThreadStates = config.getMaxThreadStates(); - this.fieldInfos = fieldInfos; - this.bufferedDeletesStream = bufferedDeletesStream; - flushControl = writer.flushControl; - consumer = config.getIndexingChain().getChain(this); - this.config = config; + this.perThreadPool = config.getIndexerThreadPool(); + this.chain = config.getIndexingChain(); + this.perThreadPool.initialize(this, globalFieldNumbers, config); + final FlushPolicy configuredPolicy = config.getFlushPolicy(); + if (configuredPolicy == null) { + flushPolicy = new FlushByRamOrCountsPolicy(); + } else { + flushPolicy = configuredPolicy; + } + flushPolicy.init(this); + + healthiness = new Healthiness(); + final long maxRamPerDWPT = config.getRAMPerThreadHardLimitMB() * 1024 * 1024; + flushControl = new DocumentsWriterFlushControl(this, healthiness, maxRamPerDWPT); + } + + synchronized void deleteQueries(final Query... queries) throws IOException { + deleteQueue.addDelete(queries); + flushControl.doOnDelete(); + if (flushControl.doApplyAllDeletes()) { + applyAllDeletes(deleteQueue); } - - // Buffer a specific docID for deletion. Currently only - // used when we hit a exception when adding a document - synchronized void deleteDocID(int docIDUpto) { - pendingDeletes.addDocID(docIDUpto); - // NOTE: we do not trigger flush here. This is - // potentially a RAM leak, if you have an app that tries - // to add docs but every single doc always hits a - // non-aborting exception. Allowing a flush here gets - // very messy because we are only invoked when handling - // exceptions so to do this properly, while handling an - // exception we'd have to go off and flush new deletes - // which is risky (likely would hit some other - // confounding exception). } - boolean deleteQueries(Query... queries) { - final boolean doFlush = flushControl.waitUpdate(0, queries.length); - synchronized(this) { - for (Query query : queries) { - pendingDeletes.addQuery(query, numDocs); - } + // TODO: we could check w/ FreqProxTermsWriter: if the + // term doesn't exist, don't bother buffering into the + // per-DWPT map (but still must go into the global map) + synchronized void deleteTerms(final Term... terms) throws IOException { + final DocumentsWriterDeleteQueue deleteQueue = this.deleteQueue; + deleteQueue.addDelete(terms); + flushControl.doOnDelete(); + if (flushControl.doApplyAllDeletes()) { + applyAllDeletes(deleteQueue); } - return doFlush; } - boolean deleteQuery(Query query) { - final boolean doFlush = flushControl.waitUpdate(0, 1); - synchronized(this) { - pendingDeletes.addQuery(query, numDocs); - } - return doFlush; + DocumentsWriterDeleteQueue currentDeleteSession() { + return deleteQueue; } - boolean deleteTerms(Term... terms) { - final boolean doFlush = flushControl.waitUpdate(0, terms.length); - synchronized(this) { - for (Term term : terms) { - pendingDeletes.addTerm(term, numDocs); - } - } - return doFlush; + private void applyAllDeletes(DocumentsWriterDeleteQueue deleteQueue) throws IOException { + if (deleteQueue != null) { + synchronized (ticketQueue) { + // Freeze and insert the delete flush ticket in the queue + ticketQueue.add(new FlushTicket(deleteQueue.freezeGlobalBuffer(null), false)); + applyFlushTickets(); } - - // TODO: we could check w/ FreqProxTermsWriter: if the - // term doesn't exist, don't bother buffering into the - // per-DWPT map (but still must go into the global map) - boolean deleteTerm(Term term, boolean skipWait) { - final boolean doFlush = flushControl.waitUpdate(0, 1, skipWait); - synchronized(this) { - pendingDeletes.addTerm(term, numDocs); } - return doFlush; + indexWriter.applyAllDeletes(); + indexWriter.flushCount.incrementAndGet(); } - /** If non-null, various details of indexing are printed - * here. */ synchronized void setInfoStream(PrintStream infoStream) { this.infoStream = infoStream; - for(int i=0;i it = perThreadPool.getAllPerThreadsIterator(); + while (it.hasNext()) { + it.next().perThread.docState.infoStream = infoStream; } } - /** Get current segment name we are writing. */ - synchronized String getSegment() { - return segment; + /** Returns how many docs are currently buffered in RAM. */ + int getNumDocs() { + return numDocsInRAM.get(); } - /** Returns how many docs are currently buffered in RAM. */ - synchronized int getNumDocs() { - return numDocs; + Collection abortedFiles() { + return abortedFiles; } - void message(String message) { + // returns boolean for asserts + boolean message(String message) { if (infoStream != null) { - writer.message("DW: " + message); + indexWriter.message("DW: " + message); } + return true; } - synchronized void setAborting() { - if (infoStream != null) { - message("setAborting"); + private void ensureOpen() throws AlreadyClosedException { + if (closed) { + throw new AlreadyClosedException("this IndexWriter is closed"); } - aborting = true; } /** Called if we hit an exception at a bad time (when @@ -378,816 +220,335 @@ * currently buffered docs. This resets our state, * discarding any docs added since last flush. */ synchronized void abort() throws IOException { - if (infoStream != null) { - message("docWriter: abort"); - } - boolean success = false; - try { - - // Forcefully remove waiting ThreadStates from line - waitQueue.abort(); - - // Wait for all other threads to finish with - // DocumentsWriter: - waitIdle(); + synchronized (this) { + deleteQueue.clear(); + } + try { if (infoStream != null) { - message("docWriter: abort waitIdle done"); + message("docWriter: abort"); } - assert 0 == waitQueue.numWaiting: "waitQueue.numWaiting=" + waitQueue.numWaiting; - - waitQueue.waitingBytes = 0; - - pendingDeletes.clear(); + final Iterator threadsIterator = perThreadPool.getActivePerThreadsIterator(); - for (DocumentsWriterThreadState threadState : threadStates) + while (threadsIterator.hasNext()) { + ThreadState perThread = threadsIterator.next(); + perThread.lock(); try { - threadState.consumer.abort(); - } catch (Throwable t) { + if (perThread.isActive()) { // we might be closed + perThread.perThread.abort(); + perThread.perThread.checkAndResetHasAborted(); + } else { + assert closed; + } + } finally { + perThread.unlock(); } - - try { - consumer.abort(); - } catch (Throwable t) { } - // Reset all postings data - doAfterFlush(); success = true; } finally { - aborting = false; - notifyAll(); if (infoStream != null) { - message("docWriter: done abort; success=" + success); + message("docWriter: done abort; abortedFiles=" + abortedFiles + " success=" + success); } } } - /** Reset after a flush */ - private void doAfterFlush() throws IOException { - // All ThreadStates should be idle when we are called - assert allThreadsIdle(); - for (DocumentsWriterThreadState threadState : threadStates) { - threadState.consumer.doAfterFlush(); + boolean anyChanges() { + return numDocsInRAM.get() != 0 || anyDeletions(); } - threadBindings.clear(); - waitQueue.reset(); - segment = null; - fieldInfos = new FieldInfos(fieldInfos); - numDocs = 0; - nextDocID = 0; - bufferIsFull = false; - for(int i=0;i BD - final long delGen = bufferedDeletesStream.getNextGen(); - if (pendingDeletes.any()) { - if (segmentInfos.size() > 0 || newSegment != null) { - final FrozenBufferedDeletes packet = new FrozenBufferedDeletes(pendingDeletes, delGen); - if (infoStream != null) { - message("flush: push buffered deletes startSize=" + pendingDeletes.bytesUsed.get() + " frozenSize=" + packet.bytesUsed); - } - bufferedDeletesStream.push(packet); - if (infoStream != null) { - message("flush: delGen=" + packet.gen); - } - if (newSegment != null) { - newSegment.setBufferedDeletesGen(packet.gen); - } - } else { - if (infoStream != null) { - message("flush: drop buffered deletes: no segments"); - } - // We can safely discard these deletes: since - // there are no segments, the deletions cannot - // affect anything. - } - pendingDeletes.clear(); - } else if (newSegment != null) { - newSegment.setBufferedDeletesGen(delGen); - } + //for testing + public int getNumBufferedDeleteTerms() { + return deleteQueue.numGlobalTermDeletes(); } public boolean anyDeletions() { - return pendingDeletes.any(); - } - - /** Flush all pending docs to a new segment */ - // Lock order: IW -> DW - synchronized SegmentInfo flush(IndexWriter writer, IndexFileDeleter deleter, MergePolicy mergePolicy, SegmentInfos segmentInfos) throws IOException { - - final long startTime = System.currentTimeMillis(); - - // We change writer's segmentInfos: - assert Thread.holdsLock(writer); - - waitIdle(); - - if (numDocs == 0) { - // nothing to do! - if (infoStream != null) { - message("flush: no docs; skipping"); - } - // Lock order: IW -> DW -> BD - pushDeletes(null, segmentInfos); - return null; + return deleteQueue.anyChanges(); } - if (aborting) { - if (infoStream != null) { - message("flush: skip because aborting is set"); - } - return null; + void close() { + closed = true; + flushControl.setClosed(); } - boolean success = false; - - SegmentInfo newSegment; - - try { - assert nextDocID == numDocs; - assert waitQueue.numWaiting == 0; - assert waitQueue.waitingBytes == 0; + boolean updateDocument(final Document doc, final Analyzer analyzer, + final Term delTerm) throws CorruptIndexException, IOException { + ensureOpen(); + boolean maybeMerge = false; + final boolean isUpdate = delTerm != null; + if (healthiness.anyStalledThreads()) { + // Help out flushing any pending DWPTs so we can un-stall: if (infoStream != null) { - message("flush postings as segment " + segment + " numDocs=" + numDocs); + message("WARNING DocumentsWriter has stalled threads; will hijack this thread to flush pending segment(s)"); } - final SegmentWriteState flushState = new SegmentWriteState(infoStream, directory, segment, fieldInfos, - numDocs, writer.getConfig().getTermIndexInterval(), - fieldInfos.buildSegmentCodecs(true), - pendingDeletes); - // Apply delete-by-docID now (delete-byDocID only - // happens when an exception is hit processing that - // doc, eg if analyzer has some problem w/ the text): - if (pendingDeletes.docIDs.size() > 0) { - flushState.deletedDocs = new BitVector(numDocs); - for(int delDocID : pendingDeletes.docIDs) { - flushState.deletedDocs.set(delDocID); + // Try pick up pending threads here if possible + DocumentsWriterPerThread flushingDWPT; + while ((flushingDWPT = flushControl.nextPendingFlush()) != null) { + // Don't push the delete here since the update could fail! + maybeMerge = doFlush(flushingDWPT); + if (!healthiness.anyStalledThreads()) { + break; } - pendingDeletes.bytesUsed.addAndGet(-pendingDeletes.docIDs.size() * BufferedDeletes.BYTES_PER_DEL_DOCID); - pendingDeletes.docIDs.clear(); } - newSegment = new SegmentInfo(segment, numDocs, directory, false, fieldInfos.hasProx(), flushState.segmentCodecs, false, fieldInfos); - - Collection threads = new HashSet(); - for (DocumentsWriterThreadState threadState : threadStates) { - threads.add(threadState.consumer); + if (infoStream != null && healthiness.anyStalledThreads()) { + message("WARNING DocumentsWriter still has stalled threads; waiting"); } - double startMBUsed = bytesUsed()/1024./1024.; - - consumer.flush(threads, flushState); + healthiness.waitIfStalled(); // block if stalled - newSegment.setHasVectors(flushState.hasVectors); - - if (infoStream != null) { - message("new segment has " + (flushState.hasVectors ? "vectors" : "no vectors")); - if (flushState.deletedDocs != null) { - message("new segment has " + flushState.deletedDocs.count() + " deleted docs"); + if (infoStream != null && healthiness.anyStalledThreads()) { + message("WARNING DocumentsWriter done waiting"); } - message("flushedFiles=" + newSegment.files()); - message("flushed codecs=" + newSegment.getSegmentCodecs()); } - if (mergePolicy.useCompoundFile(segmentInfos, newSegment)) { - final String cfsFileName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION); - - if (infoStream != null) { - message("flush: create compound file \"" + cfsFileName + "\""); - } + final ThreadState perThread = perThreadPool.getAndLock(Thread.currentThread(), + this, doc); + final DocumentsWriterPerThread flushingDWPT; - CompoundFileWriter cfsWriter = new CompoundFileWriter(directory, cfsFileName); - for(String fileName : newSegment.files()) { - cfsWriter.addFile(fileName); - } - cfsWriter.close(); - deleter.deleteNewFiles(newSegment.files()); - newSegment.setUseCompoundFile(true); - } - - // Must write deleted docs after the CFS so we don't - // slurp the del file into CFS: - if (flushState.deletedDocs != null) { - final int delCount = flushState.deletedDocs.count(); - assert delCount > 0; - newSegment.setDelCount(delCount); - newSegment.advanceDelGen(); - final String delFileName = newSegment.getDelFileName(); - if (infoStream != null) { - message("flush: write " + delCount + " deletes to " + delFileName); - } - boolean success2 = false; try { - // TODO: in the NRT case it'd be better to hand - // this del vector over to the - // shortly-to-be-opened SegmentReader and let it - // carry the changes; there's no reason to use - // filesystem as intermediary here. - flushState.deletedDocs.write(directory, delFileName); - success2 = true; - } finally { - if (!success2) { - try { - directory.deleteFile(delFileName); - } catch (Throwable t) { - // suppress this so we keep throwing the - // original exception - } - } - } - } - - if (infoStream != null) { - message("flush: segment=" + newSegment); - final double newSegmentSizeNoStore = newSegment.sizeInBytes(false)/1024./1024.; - final double newSegmentSize = newSegment.sizeInBytes(true)/1024./1024.; - message(" ramUsed=" + nf.format(startMBUsed) + " MB" + - " newFlushedSize=" + nf.format(newSegmentSize) + " MB" + - " (" + nf.format(newSegmentSizeNoStore) + " MB w/o doc stores)" + - " docs/MB=" + nf.format(numDocs / newSegmentSize) + - " new/old=" + nf.format(100.0 * newSegmentSizeNoStore / startMBUsed) + "%"); - } - - success = true; - } finally { - notifyAll(); - if (!success) { - if (segment != null) { - deleter.refresh(segment); - } - abort(); - } - } - - doAfterFlush(); - - // Lock order: IW -> DW -> BD - pushDeletes(newSegment, segmentInfos); - if (infoStream != null) { - message("flush time " + (System.currentTimeMillis()-startTime) + " msec"); - } - - return newSegment; - } - - synchronized void close() { - closed = true; - notifyAll(); - } - - /** Returns a free (idle) ThreadState that may be used for - * indexing this one document. This call also pauses if a - * flush is pending. If delTerm is non-null then we - * buffer this deleted term after the thread state has - * been acquired. */ - synchronized DocumentsWriterThreadState getThreadState(Document doc, Term delTerm) throws IOException { - - final Thread currentThread = Thread.currentThread(); - assert !Thread.holdsLock(writer); - - // First, find a thread state. If this thread already - // has affinity to a specific ThreadState, use that one - // again. - DocumentsWriterThreadState state = threadBindings.get(currentThread); - if (state == null) { - - // First time this thread has called us since last - // flush. Find the least loaded thread state: - DocumentsWriterThreadState minThreadState = null; - for(int i=0;i= maxThreadStates)) { - state = minThreadState; - state.numThreads++; - } else { - // Just create a new "private" thread state - DocumentsWriterThreadState[] newArray = new DocumentsWriterThreadState[1+threadStates.length]; - if (threadStates.length > 0) { - System.arraycopy(threadStates, 0, newArray, 0, threadStates.length); - } - state = newArray[threadStates.length] = new DocumentsWriterThreadState(this); - threadStates = newArray; - } - threadBindings.put(currentThread, state); - } - - // Next, wait until my thread state is idle (in case - // it's shared with other threads), and no flush/abort - // pending - waitReady(state); - - // Allocate segment name if this is the first doc since - // last flush: - if (segment == null) { - segment = writer.newSegmentName(); - assert numDocs == 0; - } - - state.docState.docID = nextDocID++; - - if (delTerm != null) { - pendingDeletes.addTerm(delTerm, state.docState.docID); - } - numDocs++; - state.isIdle = false; - return state; + if (!perThread.isActive()) { + ensureOpen(); + assert false: "perThread is not active but we are still open"; } - boolean addDocument(Document doc, Analyzer analyzer) throws CorruptIndexException, IOException { - return updateDocument(doc, analyzer, null); - } - - boolean updateDocument(Document doc, Analyzer analyzer, Term delTerm) - throws CorruptIndexException, IOException { - - // Possibly trigger a flush, or wait until any running flush completes: - boolean doFlush = flushControl.waitUpdate(1, delTerm != null ? 1 : 0); - - // This call is synchronized but fast - final DocumentsWriterThreadState state = getThreadState(doc, delTerm); - - final DocState docState = state.docState; - docState.doc = doc; - docState.analyzer = analyzer; - - boolean success = false; + final DocumentsWriterPerThread dwpt = perThread.perThread; try { - // This call is not synchronized and does all the - // work - final DocWriter perDoc; - try { - perDoc = state.consumer.processDocument(fieldInfos); + dwpt.updateDocument(doc, analyzer, delTerm); + numDocsInRAM.incrementAndGet(); } finally { - docState.clear(); + if (dwpt.checkAndResetHasAborted()) { + flushControl.doOnAbort(perThread); } - - // This call is synchronized but fast - finishDocument(state, perDoc); - - success = true; - } finally { - if (!success) { - - // If this thread state had decided to flush, we - // must clear it so another thread can flush - if (doFlush) { - flushControl.clearFlushPending(); } - - if (infoStream != null) { - message("exception in updateDocument aborting=" + aborting); - } - - synchronized(this) { - - state.isIdle = true; - notifyAll(); - - if (aborting) { - abort(); - } else { - skipDocWriter.docID = docState.docID; - boolean success2 = false; - try { - waitQueue.add(skipDocWriter); - success2 = true; + flushingDWPT = flushControl.doAfterDocument(perThread, isUpdate); } finally { - if (!success2) { - abort(); - return false; - } - } - - // Immediately mark this document as deleted - // since likely it was partially added. This - // keeps indexing as "all or none" (atomic) when - // adding a document: - deleteDocID(state.docState.docID); - } - } - } + perThread.unlock(); } - doFlush |= flushControl.flushByRAMUsage("new document"); - - return doFlush; - } - - public synchronized void waitIdle() { - while (!allThreadsIdle()) { - try { - wait(); - } catch (InterruptedException ie) { - throw new ThreadInterruptedException(ie); - } - } - } - - synchronized void waitReady(DocumentsWriterThreadState state) { - while (!closed && (!state.isIdle || aborting)) { - try { - wait(); - } catch (InterruptedException ie) { - throw new ThreadInterruptedException(ie); - } + if (flushingDWPT != null) { + maybeMerge |= doFlush(flushingDWPT); + } else { + final DocumentsWriterPerThread nextPendingFlush = flushControl.nextPendingFlush(); + if (nextPendingFlush != null) { + maybeMerge |= doFlush(nextPendingFlush); } - - if (closed) { - throw new AlreadyClosedException("this IndexWriter is closed"); } + return maybeMerge; } - /** Does the synchronized work to finish/flush the - * inverted document. */ - private void finishDocument(DocumentsWriterThreadState perThread, DocWriter docWriter) throws IOException { - - // Must call this w/o holding synchronized(this) else - // we'll hit deadlock: - balanceRAM(); - - synchronized(this) { - - assert docWriter == null || docWriter.docID == perThread.docState.docID; - - if (aborting) { + private boolean doFlush(DocumentsWriterPerThread flushingDWPT) throws IOException { + boolean maybeMerge = false; + while (flushingDWPT != null) { + maybeMerge = true; + boolean success = false; + FlushTicket ticket = null; - // We are currently aborting, and another thread is - // waiting for me to become idle. We just forcefully - // idle this threadState; it will be fully reset by - // abort() - if (docWriter != null) { try { - docWriter.abort(); - } catch (Throwable t) { - } - } - - perThread.isIdle = true; - - // wakes up any threads waiting on the wait queue - notifyAll(); - - return; - } - - final boolean doPause; - - if (docWriter != null) { - doPause = waitQueue.add(docWriter); - } else { - skipDocWriter.docID = perThread.docState.docID; - doPause = waitQueue.add(skipDocWriter); - } - - if (doPause) { - waitForWaitQueue(); - } - - perThread.isIdle = true; - - // wakes up any threads waiting on the wait queue - notifyAll(); - } - } - - synchronized void waitForWaitQueue() { - do { + assert currentFullFlushDelQueue == null + || flushingDWPT.deleteQueue == currentFullFlushDelQueue : "expected: " + + currentFullFlushDelQueue + "but was: " + flushingDWPT.deleteQueue + + " " + flushControl.isFullFlush(); + /* + * Since with DWPT the flush process is concurrent and several DWPT + * could flush at the same time we must maintain the order of the + * flushes before we can apply the flushed segment and the frozen global + * deletes it is buffering. The reason for this is that the global + * deletes mark a certain point in time where we took a DWPT out of + * rotation and freeze the global deletes. + * + * Example: A flush 'A' starts and freezes the global deletes, then + * flush 'B' starts and freezes all deletes occurred since 'A' has + * started. if 'B' finishes before 'A' we need to wait until 'A' is done + * otherwise the deletes frozen by 'B' are not applied to 'A' and we + * might miss to deletes documents in 'A'. + */ try { - wait(); - } catch (InterruptedException ie) { - throw new ThreadInterruptedException(ie); - } - } while (!waitQueue.doResume()); + synchronized (ticketQueue) { + // Each flush is assigned a ticket in the order they accquire the ticketQueue lock + ticket = new FlushTicket(flushingDWPT.prepareFlush(), true); + ticketQueue.add(ticket); } - private static class SkipDocWriter extends DocWriter { - @Override - void finish() { + // flush concurrently without locking + final FlushedSegment newSegment = flushingDWPT.flush(); + synchronized (ticketQueue) { + ticket.segment = newSegment; } - @Override - void abort() { + // flush was successful once we reached this point - new seg. has been assigned to the ticket! + success = true; + } finally { + if (!success && ticket != null) { + synchronized (ticketQueue) { + // In the case of a failure make sure we are making progress and + // apply all the deletes since the segment flush failed since the flush + // ticket could hold global deletes see FlushTicket#canPublish() + ticket.isSegmentFlush = false; } - @Override - long sizeInBytes() { - return 0; } } - final SkipDocWriter skipDocWriter = new SkipDocWriter(); - - NumberFormat nf = NumberFormat.getInstance(); - - /* Initial chunks size of the shared byte[] blocks used to - store postings data */ - final static int BYTE_BLOCK_NOT_MASK = ~BYTE_BLOCK_MASK; - - /* if you increase this, you must fix field cache impl for - * getTerms/getTermsIndex requires <= 32768. */ - final static int MAX_TERM_LENGTH_UTF8 = BYTE_BLOCK_SIZE-2; - - /* Initial chunks size of the shared int[] blocks used to - store postings data */ - final static int INT_BLOCK_SHIFT = 13; - final static int INT_BLOCK_SIZE = 1 << INT_BLOCK_SHIFT; - final static int INT_BLOCK_MASK = INT_BLOCK_SIZE - 1; - - private List freeIntBlocks = new ArrayList(); - - /* Allocate another int[] from the shared pool */ - synchronized int[] getIntBlock() { - final int size = freeIntBlocks.size(); - final int[] b; - if (0 == size) { - b = new int[INT_BLOCK_SIZE]; - bytesUsed.addAndGet(INT_BLOCK_SIZE*RamUsageEstimator.NUM_BYTES_INT); - } else { - b = freeIntBlocks.remove(size-1); - } - return b; - } - - long bytesUsed() { - return bytesUsed.get() + pendingDeletes.bytesUsed.get(); + /* + * Now we are done and try to flush the ticket queue if the head of the + * queue has already finished the flush. + */ + applyFlushTickets(); + } finally { + flushControl.doAfterFlush(flushingDWPT); + flushingDWPT.checkAndResetHasAborted(); + indexWriter.flushCount.incrementAndGet(); } - /* Return int[]s to the pool */ - synchronized void recycleIntBlocks(int[][] blocks, int start, int end) { - for(int i=start;i= ramBufferSize; } - - if (doBalance) { - - if (infoStream != null) { - message(" RAM: balance allocations: usedMB=" + toMB(bytesUsed()) + - " vs trigger=" + toMB(ramBufferSize) + - " deletesMB=" + toMB(deletesRAMUsed) + - " byteBlockFree=" + toMB(byteBlockAllocator.bytesUsed()) + - " perDocFree=" + toMB(perDocAllocator.bytesUsed())); } - final long startBytesUsed = bytesUsed() + deletesRAMUsed; - - int iter = 0; - - // We free equally from each pool in 32 KB - // chunks until we are below our threshold - // (freeLevel) - - boolean any = true; - - final long freeLevel = (long) (0.95 * ramBufferSize); - - while(bytesUsed()+deletesRAMUsed > freeLevel) { - - synchronized(this) { - if (0 == perDocAllocator.numBufferedBlocks() && - 0 == byteBlockAllocator.numBufferedBlocks() && - 0 == freeIntBlocks.size() && !any) { - // Nothing else to free -- must flush now. - bufferIsFull = bytesUsed()+deletesRAMUsed > ramBufferSize; + private void finishFlush(FlushedSegment newSegment, FrozenBufferedDeletes bufferedDeletes) + throws IOException { + // Finish the flushed segment and publish it to IndexWriter + if (newSegment == null) { + assert bufferedDeletes != null; + if (bufferedDeletes != null && bufferedDeletes.any()) { + indexWriter.bufferedDeletesStream.push(bufferedDeletes); if (infoStream != null) { - if (bytesUsed()+deletesRAMUsed > ramBufferSize) { - message(" nothing to free; set bufferIsFull"); - } else { - message(" nothing to free"); - } - } - break; + message("flush: push buffered deletes: " + bufferedDeletes); } - - if ((0 == iter % 4) && byteBlockAllocator.numBufferedBlocks() > 0) { - byteBlockAllocator.freeBlocks(1); - } - if ((1 == iter % 4) && freeIntBlocks.size() > 0) { - freeIntBlocks.remove(freeIntBlocks.size()-1); - bytesUsed.addAndGet(-INT_BLOCK_SIZE * RamUsageEstimator.NUM_BYTES_INT); } - if ((2 == iter % 4) && perDocAllocator.numBufferedBlocks() > 0) { - perDocAllocator.freeBlocks(32); // Remove upwards of 32 blocks (each block is 1K) + } else { + publishFlushedSegment(newSegment, bufferedDeletes); } } - if ((3 == iter % 4) && any) { - // Ask consumer to free any recycled state - any = consumer.freeRAM(); + final void subtractFlushedNumDocs(int numFlushed) { + int oldValue = numDocsInRAM.get(); + while (!numDocsInRAM.compareAndSet(oldValue, oldValue - numFlushed)) { + oldValue = numDocsInRAM.get(); } - - iter++; } + /** + * Publishes the flushed segment, segment private deletes (if any) and its + * associated global delete (if present) to IndexWriter. The actual + * publishing operation is synced on IW -> BDS so that the {@link SegmentInfo}'s + * delete generation is always GlobalPacket_deleteGeneration + 1 + */ + private void publishFlushedSegment(FlushedSegment newSegment, FrozenBufferedDeletes globalPacket) + throws IOException { + assert newSegment != null; + final SegmentInfo segInfo = indexWriter.prepareFlushedSegment(newSegment); + final BufferedDeletes deletes = newSegment.segmentDeletes; + FrozenBufferedDeletes packet = null; + if (deletes != null && deletes.any()) { + // Segment private delete + packet = new FrozenBufferedDeletes(deletes, true); if (infoStream != null) { - message(" after free: freedMB=" + nf.format((startBytesUsed-bytesUsed()-deletesRAMUsed)/1024./1024.) + " usedMB=" + nf.format((bytesUsed()+deletesRAMUsed)/1024./1024.)); - } + message("flush: push buffered seg private deletes: " + packet); } } - final WaitQueue waitQueue = new WaitQueue(); - - private class WaitQueue { - DocWriter[] waiting; - int nextWriteDocID; - int nextWriteLoc; - int numWaiting; - long waitingBytes; - - public WaitQueue() { - waiting = new DocWriter[10]; + // now publish! + indexWriter.publishFlushedSegment(segInfo, packet, globalPacket); } - synchronized void reset() { - // NOTE: nextWriteLoc doesn't need to be reset - assert numWaiting == 0; - assert waitingBytes == 0; - nextWriteDocID = 0; - } - - synchronized boolean doResume() { - final double mb = config.getRAMBufferSizeMB(); - final long waitQueueResumeBytes; - if (mb == IndexWriterConfig.DISABLE_AUTO_FLUSH) { - waitQueueResumeBytes = 2*1024*1024; - } else { - waitQueueResumeBytes = (long) (mb*1024*1024*0.05); - } - return waitingBytes <= waitQueueResumeBytes; - } - - synchronized boolean doPause() { - final double mb = config.getRAMBufferSizeMB(); - final long waitQueuePauseBytes; - if (mb == IndexWriterConfig.DISABLE_AUTO_FLUSH) { - waitQueuePauseBytes = 4*1024*1024; - } else { - waitQueuePauseBytes = (long) (mb*1024*1024*0.1); - } - return waitingBytes > waitQueuePauseBytes; + // for asserts + private volatile DocumentsWriterDeleteQueue currentFullFlushDelQueue = null; + // for asserts + private synchronized boolean setFlushingDeleteQueue(DocumentsWriterDeleteQueue session) { + currentFullFlushDelQueue = session; + return true; } - synchronized void abort() { - int count = 0; - for(int i=0;i= nextWriteDocID; - - if (doc.docID == nextWriteDocID) { - writeDocument(doc); - while(true) { - doc = waiting[nextWriteLoc]; - if (doc != null) { - numWaiting--; - waiting[nextWriteLoc] = null; - waitingBytes -= doc.sizeInBytes(); - writeDocument(doc); + final void finishFullFlush(boolean success) { + assert setFlushingDeleteQueue(null); + if (success) { + // Release the flush lock + flushControl.finishFullFlush(); } else { - break; + flushControl.abortFullFlushes(); } } - } else { - // I finished before documents that were added - // before me. This can easily happen when I am a - // small doc and the docs before me were large, or, - // just due to luck in the thread scheduling. Just - // add myself to the queue and when that large doc - // finishes, it will flush me: - int gap = doc.docID - nextWriteDocID; - if (gap >= waiting.length) { - // Grow queue - DocWriter[] newArray = new DocWriter[ArrayUtil.oversize(gap, RamUsageEstimator.NUM_BYTES_OBJECT_REF)]; - assert nextWriteLoc >= 0; - System.arraycopy(waiting, nextWriteLoc, newArray, 0, waiting.length-nextWriteLoc); - System.arraycopy(waiting, 0, newArray, waiting.length-nextWriteLoc, nextWriteLoc); - nextWriteLoc = 0; - waiting = newArray; - gap = doc.docID - nextWriteDocID; - } - - int loc = nextWriteLoc + gap; - if (loc >= waiting.length) { - loc -= waiting.length; - } - - // We should only wrap one time - assert loc < waiting.length; - - // Nobody should be in my spot! - assert waiting[loc] == null; - waiting[loc] = doc; - numWaiting++; - waitingBytes += doc.sizeInBytes(); + static final class FlushTicket { + final FrozenBufferedDeletes frozenDeletes; + /* access to non-final members must be synchronized on DW#ticketQueue */ + FlushedSegment segment; + boolean isSegmentFlush; + + FlushTicket(FrozenBufferedDeletes frozenDeletes, boolean isSegmentFlush) { + this.frozenDeletes = frozenDeletes; + this.isSegmentFlush = isSegmentFlush; } - return doPause(); + boolean canPublish() { + return (!isSegmentFlush || segment != null); } } } --- lucene/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java 1970-01-01 01:00:00.000000000 +0100 +++ lucene/src/java/org/apache/lucene/index/DocumentsWriterDeleteQueue.java 2011-04-29 15:01:05.953315500 +0200 @@ -0,0 +1,396 @@ +package org.apache.lucene.index; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +import java.util.concurrent.atomic.AtomicReferenceFieldUpdater; +import java.util.concurrent.locks.ReentrantLock; + +import org.apache.lucene.search.Query; + +/** + * {@link DocumentsWriterDeleteQueue} is a non-blocking linked pending deletes + * 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 + * DWPTs and a global delete pool. Each of the DWPT and the global pool need to + * maintain their 'own' head of the queue (as a DeleteSlice instance per DWPT). + * The difference between the DWPT and the global pool is that the DWPT starts + * maintaining a head once it has added its first document since for its segments + * private deletes only the deletes after that document are relevant. The global + * pool instead starts maintaining the head once this instance is created by + * taking the sentinel instance as its initial head. + *

+ * 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. + * All nodes in the list that are still relevant should be either directly or + * indirectly referenced by one of the DWPT's private {@link DeleteSlice} or by + * the global {@link BufferedDeletes} slice. + *

+ * Each DWPT as well as the global delete pool maintain their private + * DeleteSlice instance. In the DWPT case updating a slice is equivalent to + * atomically finishing the document. The slice update guarantees a "happens + * before" relationship to all other updates in the same indexing session. When a + * DWPT updates a document it: + * + *

    + *
  1. consumes a document and finishes its processing
  2. + *
  3. updates its private {@link DeleteSlice} either by calling + * {@link #updateSlice(DeleteSlice)} or {@link #add(Term, DeleteSlice)} (if the + * document has a delTerm)
  4. + *
  5. applies all deletes in the slice to its private {@link BufferedDeletes} + * and resets it
  6. + *
  7. increments its internal document id
  8. + *
+ * + * 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 + * 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. + * + */ +final class DocumentsWriterDeleteQueue { + + private volatile Node tail; + + private static final AtomicReferenceFieldUpdater tailUpdater = AtomicReferenceFieldUpdater + .newUpdater(DocumentsWriterDeleteQueue.class, Node.class, "tail"); + + private final DeleteSlice globalSlice; + private final BufferedDeletes globalBufferedDeletes; + /* only acquired to update the global deletes */ + private final ReentrantLock globalBufferLock = new ReentrantLock(); + + final long generation; + + DocumentsWriterDeleteQueue() { + this(0); + } + + DocumentsWriterDeleteQueue(long generation) { + this(new BufferedDeletes(false), generation); + } + + DocumentsWriterDeleteQueue(BufferedDeletes globalBufferedDeletes, long generation) { + this.globalBufferedDeletes = globalBufferedDeletes; + this.generation = generation; + /* + * we use a sentinel instance as our initial tail. No slice will ever try to + * apply this tail since the head is always omitted. + */ + tail = new Node(null); // sentinel + globalSlice = new DeleteSlice(tail); + } + + void addDelete(Query... queries) { + add(new QueryArrayNode(queries)); + tryApplyGlobalSlice(); + } + + void addDelete(Term... terms) { + add(new TermArrayNode(terms)); + tryApplyGlobalSlice(); + } + + /** + * invariant for document update + */ + void add(Term term, DeleteSlice slice) { + final TermNode termNode = new TermNode(term); + add(termNode); + /* + * this is an update request where the term is the updated documents + * delTerm. in that case we need to guarantee that this insert is atomic + * with regards to the given delete slice. This means if two threads try to + * update the same document with in turn the same delTerm one of them must + * win. By taking the node we have created for our del term as the new tail + * it is guaranteed that if another thread adds the same right after us we + * will apply this delete next time we update our slice and one of the two + * competing updates wins! + */ + slice.sliceTail = termNode; + assert slice.sliceHead != slice.sliceTail : "slice head and tail must differ after add"; + tryApplyGlobalSlice(); // TODO doing this each time is not necessary maybe + // we can do it just every n times or so? + } + + void add(Node item) { + /* + * this non-blocking / 'wait-free' linked list add was inspired by Apache + * Harmony's ConcurrentLinkedQueue Implementation. + */ + while (true) { + final Node currentTail = this.tail; + final Node tailNext = currentTail.next; + if (tail == currentTail) { + if (tailNext != null) { + /* + * we are in intermediate state here. the tails next pointer has been + * advanced but the tail itself might not be updated yet. help to + * advance the tail and try again updating it. + */ + tailUpdater.compareAndSet(this, currentTail, tailNext); // can fail + } else { + /* + * we are in quiescent state and can try to insert the item to the + * current tail if we fail to insert we just retry the operation since + * somebody else has already added its item + */ + if (currentTail.casNext(null, item)) { + /* + * now that we are done we need to advance the tail while another + * thread could have advanced it already so we can ignore the return + * type of this CAS call + */ + tailUpdater.compareAndSet(this, currentTail, item); + return; + } + } + } + } + } + + boolean anyChanges() { + globalBufferLock.lock(); + try { + return !globalSlice.isEmpty() || globalBufferedDeletes.any(); + } finally { + globalBufferLock.unlock(); + } + } + + void tryApplyGlobalSlice() { + if (globalBufferLock.tryLock()) { + /* + * 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 + * deletes that have been added after the current in-flight global slices + * tail the next time we can get the lock! + */ + try { + if (updateSlice(globalSlice)) { + globalSlice.apply(globalBufferedDeletes, BufferedDeletes.MAX_INT); + } + } finally { + globalBufferLock.unlock(); + } + } + } + + FrozenBufferedDeletes freezeGlobalBuffer(DeleteSlice callerSlice) { + globalBufferLock.lock(); + /* + * 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 + * queue. + */ + final Node currentTail = tail; // take the current tail make this local any + // Changes after this call are applied later + // and not relevant here + if (callerSlice != null) { + // Update the callers slices so we are on the same page + callerSlice.sliceTail = currentTail; + } + try { + if (globalSlice.sliceTail != currentTail) { + globalSlice.sliceTail = currentTail; + globalSlice.apply(globalBufferedDeletes, BufferedDeletes.MAX_INT); + } + + final FrozenBufferedDeletes packet = new FrozenBufferedDeletes( + globalBufferedDeletes, false); + globalBufferedDeletes.clear(); + return packet; + } finally { + globalBufferLock.unlock(); + } + } + + DeleteSlice newSlice() { + return new DeleteSlice(tail); + } + + boolean updateSlice(DeleteSlice slice) { + if (slice.sliceTail != tail) { // If we are the same just + slice.sliceTail = tail; + return true; + } + return false; + } + + static class DeleteSlice { + // No need to be volatile, slices are thread captive (only accessed by one thread)! + Node sliceHead; // we don't apply this one + Node sliceTail; + + DeleteSlice(Node currentTail) { + assert currentTail != null; + /* + * Initially this is a 0 length slice pointing to the 'current' tail of + * the queue. Once we update the slice we only need to assign the tail and + * have a new slice + */ + sliceHead = sliceTail = currentTail; + } + + void apply(BufferedDeletes del, int docIDUpto) { + if (sliceHead == sliceTail) { + // 0 length slice + return; + } + /* + * 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 + * tail in this slice are not equal then there will be at least one more + * non-null node in the slice! + */ + Node current = sliceHead; + do { + current = current.next; + assert current != null : "slice property violated between the head on the tail must not be a null node"; + current.apply(del, docIDUpto); + } while (current != sliceTail); + reset(); + } + + void reset() { + // Reset to a 0 length slice + sliceHead = sliceTail; + } + + /** + * Returns true iff the given item is identical to the item + * hold by the slices tail, otherwise false. + */ + boolean isTailItem(Object item) { + return sliceTail.item == item; + } + + boolean isEmpty() { + return sliceHead == sliceTail; + } + } + + public int numGlobalTermDeletes() { + return globalBufferedDeletes.numTermDeletes.get(); + } + + void clear() { + globalBufferLock.lock(); + try { + final Node currentTail = tail; + globalSlice.sliceHead = globalSlice.sliceTail = currentTail; + globalBufferedDeletes.clear(); + } finally { + globalBufferLock.unlock(); + } + } + + private static class Node { + volatile Node next; + final Object item; + + private Node(Object item) { + this.item = item; + } + + static final AtomicReferenceFieldUpdater nextUpdater = AtomicReferenceFieldUpdater + .newUpdater(Node.class, Node.class, "next"); + + void apply(BufferedDeletes bufferedDeletes, int docIDUpto) { + assert false : "sentinel item must never be applied"; + } + + boolean casNext(Node cmp, Node val) { + return nextUpdater.compareAndSet(this, cmp, val); + } + } + + private static final class TermNode extends Node { + + TermNode(Term term) { + super(term); + } + + @Override + void apply(BufferedDeletes bufferedDeletes, int docIDUpto) { + bufferedDeletes.addTerm((Term) item, docIDUpto); + } + } + + private static final class QueryArrayNode extends Node { + QueryArrayNode(Query[] query) { + super(query); + } + + @Override + void apply(BufferedDeletes bufferedDeletes, int docIDUpto) { + final Query[] queries = (Query[]) item; + for (Query query : queries) { + bufferedDeletes.addQuery(query, docIDUpto); + } + } + } + + private static final class TermArrayNode extends Node { + TermArrayNode(Term[] term) { + super(term); + } + + @Override + void apply(BufferedDeletes bufferedDeletes, int docIDUpto) { + final Term[] terms = (Term[]) item; + for (Term term : terms) { + bufferedDeletes.addTerm(term, docIDUpto); + } + } + } + + + private boolean forceApplyGlobalSlice() { + globalBufferLock.lock(); + final Node currentTail = tail; + try { + if (globalSlice.sliceTail != currentTail) { + globalSlice.sliceTail = currentTail; + globalSlice.apply(globalBufferedDeletes, BufferedDeletes.MAX_INT); + } + return globalBufferedDeletes.any(); + } finally { + globalBufferLock.unlock(); + } + } + + public int getBufferedDeleteTermsSize() { + globalBufferLock.lock(); + try { + forceApplyGlobalSlice(); + return globalBufferedDeletes.terms.size(); + } finally { + globalBufferLock.unlock(); + } + } + + public long bytesUsed() { + return globalBufferedDeletes.bytesUsed.get(); + } + + @Override + public String toString() { + return "DWDQ: [ generation: " + generation + " ]"; + } + + +} --- lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java 1970-01-01 01:00:00.000000000 +0100 +++ lucene/src/java/org/apache/lucene/index/DocumentsWriterFlushControl.java 2011-04-29 15:00:55.802734900 +0200 @@ -0,0 +1,394 @@ +package org.apache.lucene.index; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +import java.util.ArrayList; +import java.util.HashMap; +import java.util.Iterator; +import java.util.LinkedList; +import java.util.Queue; +import java.util.concurrent.atomic.AtomicBoolean; + +import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState; +import org.apache.lucene.util.ThreadInterruptedException; + +/** + * This class controls {@link DocumentsWriterPerThread} flushing during + * indexing. It tracks the memory consumption per + * {@link DocumentsWriterPerThread} and uses a configured {@link FlushPolicy} to + * decide if a {@link DocumentsWriterPerThread} must flush. + *

+ * In addition to the {@link FlushPolicy} the flush control might set certain + * {@link DocumentsWriterPerThread} as flush pending iff a + * {@link DocumentsWriterPerThread} exceeds the + * {@link IndexWriterConfig#getRAMPerThreadHardLimitMB()} to prevent address + * space exhaustion. + */ +public final class DocumentsWriterFlushControl { + + private final long hardMaxBytesPerDWPT; + private long activeBytes = 0; + private long flushBytes = 0; + private volatile int numPending = 0; + private volatile int numFlushing = 0; + final AtomicBoolean flushDeletes = new AtomicBoolean(false); + private boolean fullFlush = false; + private Queue flushQueue = new LinkedList(); + // only for safety reasons if a DWPT is close to the RAM limit + private Queue blockedFlushes = new LinkedList(); + + + long peakActiveBytes = 0;// only with assert + long peakFlushBytes = 0;// only with assert + long peakNetBytes = 0;// only with assert + private final Healthiness healthiness; + private final DocumentsWriterPerThreadPool perThreadPool; + private final FlushPolicy flushPolicy; + private boolean closed = false; + private final HashMap flushingWriters = new HashMap(); + private final DocumentsWriter documentsWriter; + + DocumentsWriterFlushControl(DocumentsWriter documentsWriter, + Healthiness healthiness, long hardMaxBytesPerDWPT) { + this.healthiness = healthiness; + this.perThreadPool = documentsWriter.perThreadPool; + this.flushPolicy = documentsWriter.flushPolicy; + this.hardMaxBytesPerDWPT = hardMaxBytesPerDWPT; + this.documentsWriter = documentsWriter; + } + + public synchronized long activeBytes() { + return activeBytes; + } + + public synchronized long flushBytes() { + return flushBytes; + } + + public synchronized long netBytes() { + return flushBytes + activeBytes; + } + + private void commitPerThreadBytes(ThreadState perThread) { + final long delta = perThread.perThread.bytesUsed() + - perThread.bytesUsed; + perThread.bytesUsed += delta; + /* + * We need to differentiate here if we are pending since setFlushPending + * moves the perThread memory to the flushBytes and we could be set to + * pending during a delete + */ + if (perThread.flushPending) { + flushBytes += delta; + } else { + activeBytes += delta; + } + assert updatePeaks(delta); + } + + // only for asserts + private boolean updatePeaks(long delta) { + peakActiveBytes = Math.max(peakActiveBytes, activeBytes); + peakFlushBytes = Math.max(peakFlushBytes, flushBytes); + peakNetBytes = Math.max(peakNetBytes, netBytes()); + return true; + } + + synchronized DocumentsWriterPerThread doAfterDocument(ThreadState perThread, + boolean isUpdate) { + commitPerThreadBytes(perThread); + if (!perThread.flushPending) { + if (isUpdate) { + flushPolicy.onUpdate(this, perThread); + } else { + flushPolicy.onInsert(this, perThread); + } + if (!perThread.flushPending && perThread.bytesUsed > hardMaxBytesPerDWPT) { + // Safety check to prevent a single DWPT exceeding its RAM limit. This + // is super important since we can not address more than 2048 MB per DWPT + setFlushPending(perThread); + if (fullFlush) { + DocumentsWriterPerThread toBlock = internalTryCheckOutForFlush(perThread, false); + assert toBlock != null; + blockedFlushes.add(toBlock); + } + } + } + final DocumentsWriterPerThread flushingDWPT = tryCheckoutForFlush(perThread, false); + healthiness.updateStalled(this); + return flushingDWPT; + } + + synchronized void doAfterFlush(DocumentsWriterPerThread dwpt) { + assert flushingWriters.containsKey(dwpt); + try { + numFlushing--; + Long bytes = flushingWriters.remove(dwpt); + flushBytes -= bytes.longValue(); + perThreadPool.recycle(dwpt); + healthiness.updateStalled(this); + } finally { + notifyAll(); + } + } + + public synchronized boolean anyFlushing() { + return numFlushing != 0; + } + + public synchronized void waitForFlush() { + if (numFlushing != 0) { + try { + this.wait(); + } catch (InterruptedException e) { + throw new ThreadInterruptedException(e); + } + } + } + + /** + * Sets flush pending state on the given {@link ThreadState}. The + * {@link ThreadState} must have indexed at least on Document and must not be + * already pending. + */ + public synchronized void setFlushPending(ThreadState perThread) { + assert !perThread.flushPending; + if (perThread.perThread.getNumDocsInRAM() > 0) { + perThread.flushPending = true; // write access synced + final long bytes = perThread.bytesUsed; + flushBytes += bytes; + activeBytes -= bytes; + numPending++; // write access synced + } // don't assert on numDocs since we could hit an abort excp. while selecting that dwpt for flushing + + } + + synchronized void doOnAbort(ThreadState state) { + if (state.flushPending) { + flushBytes -= state.bytesUsed; + } else { + activeBytes -= state.bytesUsed; + } + // Take it out of the loop this DWPT is stale + perThreadPool.replaceForFlush(state, closed); + healthiness.updateStalled(this); + } + + synchronized DocumentsWriterPerThread tryCheckoutForFlush( + ThreadState perThread, boolean setPending) { + if (fullFlush) { + return null; + } + return internalTryCheckOutForFlush(perThread, setPending); + } + + private DocumentsWriterPerThread internalTryCheckOutForFlush( + ThreadState perThread, boolean setPending) { + if (setPending && !perThread.flushPending) { + setFlushPending(perThread); + } + if (perThread.flushPending) { + // We are pending so all memory is already moved to flushBytes + if (perThread.tryLock()) { + try { + if (perThread.isActive()) { + assert perThread.isHeldByCurrentThread(); + final DocumentsWriterPerThread dwpt; + final long bytes = perThread.bytesUsed; // do that before + // replace! + dwpt = perThreadPool.replaceForFlush(perThread, closed); + assert !flushingWriters.containsKey(dwpt) : "DWPT is already flushing"; + // Record the flushing DWPT to reduce flushBytes in doAfterFlush + flushingWriters.put(dwpt, Long.valueOf(bytes)); + numPending--; // write access synced + numFlushing++; + return dwpt; + } + } finally { + perThread.unlock(); + } + } + } + return null; + } + + @Override + public String toString() { + return "DocumentsWriterFlushControl [activeBytes=" + activeBytes + + ", flushBytes=" + flushBytes + "]"; + } + + DocumentsWriterPerThread nextPendingFlush() { + synchronized (this) { + DocumentsWriterPerThread poll = flushQueue.poll(); + if (poll != null) { + return poll; + } + } + if (numPending > 0) { + final Iterator allActiveThreads = perThreadPool + .getActivePerThreadsIterator(); + while (allActiveThreads.hasNext() && numPending > 0) { + ThreadState next = allActiveThreads.next(); + if (next.flushPending) { + final DocumentsWriterPerThread dwpt = tryCheckoutForFlush(next, false); + if (dwpt != null) { + return dwpt; + } + } + } + } + return null; + } + + synchronized void setClosed() { + // set by DW to signal that we should not release new DWPT after close + this.closed = true; + } + + /** + * Returns an iterator that provides access to all currently active {@link ThreadState}s + */ + public Iterator allActiveThreads() { + return perThreadPool.getActivePerThreadsIterator(); + } + + synchronized void doOnDelete() { + // pass null this is a global delete no update + flushPolicy.onDelete(this, null); + } + + /** + * Returns the number of delete terms in the global pool + */ + public int getNumGlobalTermDeletes() { + return documentsWriter.deleteQueue.numGlobalTermDeletes(); + } + + int numFlushingDWPT() { + return numFlushing; + } + + public boolean doApplyAllDeletes() { + return flushDeletes.getAndSet(false); + } + + public void setApplyAllDeletes() { + flushDeletes.set(true); + } + + int numActiveDWPT() { + return this.perThreadPool.getMaxThreadStates(); + } + + void markForFullFlush() { + final DocumentsWriterDeleteQueue flushingQueue; + synchronized (this) { + assert !fullFlush; + fullFlush = true; + flushingQueue = documentsWriter.deleteQueue; + // Set a new delete queue - all subsequent DWPT will use this queue until + // we do another full flush + DocumentsWriterDeleteQueue newQueue = new DocumentsWriterDeleteQueue(flushingQueue.generation+1); + documentsWriter.deleteQueue = newQueue; + } + final Iterator allActiveThreads = perThreadPool + .getActivePerThreadsIterator(); + final ArrayList toFlush = new ArrayList(); + while (allActiveThreads.hasNext()) { + final ThreadState next = allActiveThreads.next(); + next.lock(); + try { + if (!next.isActive()) { + continue; + } + assert next.perThread.deleteQueue == flushingQueue + || next.perThread.deleteQueue == documentsWriter.deleteQueue : " flushingQueue: " + + flushingQueue + + " currentqueue: " + + documentsWriter.deleteQueue + + " perThread queue: " + + next.perThread.deleteQueue + + " numDocsInRam: " + next.perThread.getNumDocsInRAM(); + if (next.perThread.deleteQueue != flushingQueue) { + // this one is already a new DWPT + continue; + } + if (next.perThread.getNumDocsInRAM() > 0 ) { + final DocumentsWriterPerThread dwpt = next.perThread; // just for assert + final DocumentsWriterPerThread flushingDWPT = internalTryCheckOutForFlush(next, true); + assert flushingDWPT != null : "DWPT must never be null here since we hold the lock and it holds documents"; + assert dwpt == flushingDWPT : "flushControl returned different DWPT"; + toFlush.add(flushingDWPT); + } else { + // get the new delete queue from DW + next.perThread.initialize(); + } + } finally { + next.unlock(); + } + } + synchronized (this) { + assert assertBlockedFlushes(flushingQueue); + flushQueue.addAll(blockedFlushes); + blockedFlushes.clear(); + flushQueue.addAll(toFlush); + } + } + + synchronized void finishFullFlush() { + assert fullFlush; + assert flushQueue.isEmpty(); + try { + if (!blockedFlushes.isEmpty()) { + assert assertBlockedFlushes(documentsWriter.deleteQueue); + flushQueue.addAll(blockedFlushes); + blockedFlushes.clear(); + } + } finally { + fullFlush = false; + } + } + + boolean assertBlockedFlushes(DocumentsWriterDeleteQueue flushingQueue) { + Queue flushes = this.blockedFlushes; + for (DocumentsWriterPerThread documentsWriterPerThread : flushes) { + assert documentsWriterPerThread.deleteQueue == flushingQueue; + } + return true; + } + + synchronized void abortFullFlushes() { + try { + for (DocumentsWriterPerThread dwpt : flushQueue) { + doAfterFlush(dwpt); + } + for (DocumentsWriterPerThread dwpt : blockedFlushes) { + doAfterFlush(dwpt); + } + + } finally { + fullFlush = false; + flushQueue.clear(); + blockedFlushes.clear(); + } + } + + synchronized boolean isFullFlush() { + return fullFlush; + } +} \ No newline at end of file --- lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java 1970-01-01 01:00:00.000000000 +0100 +++ lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java 2011-04-29 15:01:00.831022500 +0200 @@ -0,0 +1,496 @@ +package org.apache.lucene.index; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_MASK; +import static org.apache.lucene.util.ByteBlockPool.BYTE_BLOCK_SIZE; + +import java.io.IOException; +import java.io.PrintStream; +import java.text.NumberFormat; +import java.util.concurrent.atomic.AtomicLong; + +import org.apache.lucene.analysis.Analyzer; +import org.apache.lucene.document.Document; +import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice; +import org.apache.lucene.search.SimilarityProvider; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.BitVector; +import org.apache.lucene.util.ByteBlockPool.Allocator; +import org.apache.lucene.util.RamUsageEstimator; + +public class DocumentsWriterPerThread { + + /** + * The IndexingChain must define the {@link #getChain(DocumentsWriter)} method + * which returns the DocConsumer that the DocumentsWriter calls to process the + * documents. + */ + abstract static class IndexingChain { + abstract DocConsumer getChain(DocumentsWriterPerThread documentsWriterPerThread); + } + + + static final IndexingChain defaultIndexingChain = new IndexingChain() { + + @Override + DocConsumer getChain(DocumentsWriterPerThread documentsWriterPerThread) { + /* + This is the current indexing chain: + + DocConsumer / DocConsumerPerThread + --> code: DocFieldProcessor / DocFieldProcessorPerThread + --> DocFieldConsumer / DocFieldConsumerPerThread / DocFieldConsumerPerField + --> code: DocFieldConsumers / DocFieldConsumersPerThread / DocFieldConsumersPerField + --> code: DocInverter / DocInverterPerThread / DocInverterPerField + --> InvertedDocConsumer / InvertedDocConsumerPerThread / InvertedDocConsumerPerField + --> code: TermsHash / TermsHashPerThread / TermsHashPerField + --> TermsHashConsumer / TermsHashConsumerPerThread / TermsHashConsumerPerField + --> code: FreqProxTermsWriter / FreqProxTermsWriterPerThread / FreqProxTermsWriterPerField + --> code: TermVectorsTermsWriter / TermVectorsTermsWriterPerThread / TermVectorsTermsWriterPerField + --> InvertedDocEndConsumer / InvertedDocConsumerPerThread / InvertedDocConsumerPerField + --> code: NormsWriter / NormsWriterPerThread / NormsWriterPerField + --> code: StoredFieldsWriter / StoredFieldsWriterPerThread / StoredFieldsWriterPerField + */ + + // Build up indexing chain: + + final TermsHashConsumer termVectorsWriter = new TermVectorsTermsWriter(documentsWriterPerThread); + final TermsHashConsumer freqProxWriter = new FreqProxTermsWriter(); + + final InvertedDocConsumer termsHash = new TermsHash(documentsWriterPerThread, freqProxWriter, true, + new TermsHash(documentsWriterPerThread, termVectorsWriter, false, null)); + final NormsWriter normsWriter = new NormsWriter(); + final DocInverter docInverter = new DocInverter(documentsWriterPerThread.docState, termsHash, normsWriter); + return new DocFieldProcessor(documentsWriterPerThread, docInverter); + } + }; + + static class DocState { + final DocumentsWriterPerThread docWriter; + Analyzer analyzer; + PrintStream infoStream; + SimilarityProvider similarityProvider; + int docID; + Document doc; + String maxTermPrefix; + + DocState(DocumentsWriterPerThread docWriter) { + this.docWriter = docWriter; + } + + // Only called by asserts + public boolean testPoint(String name) { + return docWriter.writer.testPoint(name); + } + + public void clear() { + // don't hold onto doc nor analyzer, in case it is + // largish: + doc = null; + analyzer = null; + } + } + + static class FlushedSegment { + final SegmentInfo segmentInfo; + final BufferedDeletes segmentDeletes; + final BitVector deletedDocuments; + + private FlushedSegment(SegmentInfo segmentInfo, + BufferedDeletes segmentDeletes, BitVector deletedDocuments) { + this.segmentInfo = segmentInfo; + this.segmentDeletes = segmentDeletes; + this.deletedDocuments = deletedDocuments; + } + } + + /** Called if we hit an exception at a bad time (when + * updating the index files) and must discard all + * currently buffered docs. This resets our state, + * discarding any docs added since last flush. */ + void abort() throws IOException { + hasAborted = aborting = true; + try { + if (infoStream != null) { + message("docWriter: now abort"); + } + try { + consumer.abort(); + } catch (Throwable t) { + } + + pendingDeletes.clear(); + deleteSlice = deleteQueue.newSlice(); + // Reset all postings data + doAfterFlush(); + + } finally { + aborting = false; + if (infoStream != null) { + message("docWriter: done abort"); + } + } + } + + final DocumentsWriter parent; + final IndexWriter writer; + final Directory directory; + final DocState docState; + final DocConsumer consumer; + final AtomicLong bytesUsed; + + SegmentWriteState flushState; + //Deletes for our still-in-RAM (to be flushed next) segment + BufferedDeletes pendingDeletes; + String segment; // Current segment we are working on + boolean aborting = false; // True if an abort is pending + boolean hasAborted = false; // True if the last exception throws by #updateDocument was aborting + + private FieldInfos fieldInfos; + private final PrintStream infoStream; + private int numDocsInRAM; + private int flushedDocCount; + DocumentsWriterDeleteQueue deleteQueue; + DeleteSlice deleteSlice; + private final NumberFormat nf = NumberFormat.getInstance(); + + + public DocumentsWriterPerThread(Directory directory, DocumentsWriter parent, + FieldInfos fieldInfos, IndexingChain indexingChain) { + this.directory = directory; + this.parent = parent; + this.fieldInfos = fieldInfos; + this.writer = parent.indexWriter; + this.infoStream = parent.indexWriter.getInfoStream(); + this.docState = new DocState(this); + this.docState.similarityProvider = parent.indexWriter.getConfig() + .getSimilarityProvider(); + + consumer = indexingChain.getChain(this); + bytesUsed = new AtomicLong(0); + pendingDeletes = new BufferedDeletes(false); + initialize(); + } + + public DocumentsWriterPerThread(DocumentsWriterPerThread other, FieldInfos fieldInfos) { + this(other.directory, other.parent, fieldInfos, other.parent.chain); + } + + void initialize() { + deleteQueue = parent.deleteQueue; + assert numDocsInRAM == 0 : "num docs " + numDocsInRAM; + pendingDeletes.clear(); + deleteSlice = null; + } + + void setAborting() { + aborting = true; + } + + boolean checkAndResetHasAborted() { + final boolean retval = hasAborted; + hasAborted = false; + return retval; + } + + public void updateDocument(Document doc, Analyzer analyzer, Term delTerm) throws IOException { + assert writer.testPoint("DocumentsWriterPerThread addDocument start"); + assert deleteQueue != null; + docState.doc = doc; + docState.analyzer = analyzer; + docState.docID = numDocsInRAM; + if (segment == null) { + // this call is synchronized on IndexWriter.segmentInfos + segment = writer.newSegmentName(); + assert numDocsInRAM == 0; + } + + boolean success = false; + try { + try { + consumer.processDocument(fieldInfos); + } finally { + docState.clear(); + } + success = true; + } finally { + if (!success) { + if (!aborting) { + // mark document as deleted + deleteDocID(docState.docID); + numDocsInRAM++; + } else { + abort(); + } + } + } + success = false; + try { + consumer.finishDocument(); + success = true; + } finally { + if (!success) { + abort(); + } + } + finishDocument(delTerm); + } + + private void finishDocument(Term delTerm) throws IOException { + /* + * here we actually finish the document in two steps 1. push the delete into + * the queue and update our slice. 2. increment the DWPT private document + * id. + * + * the updated slice we get from 1. holds all the deletes that have occurred + * since we updated the slice the last time. + */ + if (deleteSlice == null) { + deleteSlice = deleteQueue.newSlice(); + if (delTerm != null) { + deleteQueue.add(delTerm, deleteSlice); + deleteSlice.reset(); + } + + } else { + if (delTerm != null) { + deleteQueue.add(delTerm, deleteSlice); + assert deleteSlice.isTailItem(delTerm) : "expected the delete term as the tail item"; + deleteSlice.apply(pendingDeletes, numDocsInRAM); + } else if (deleteQueue.updateSlice(deleteSlice)) { + deleteSlice.apply(pendingDeletes, numDocsInRAM); + } + } + ++numDocsInRAM; + } + + // Buffer a specific docID for deletion. Currently only + // used when we hit a exception when adding a document + void deleteDocID(int docIDUpto) { + pendingDeletes.addDocID(docIDUpto); + // NOTE: we do not trigger flush here. This is + // potentially a RAM leak, if you have an app that tries + // to add docs but every single doc always hits a + // non-aborting exception. Allowing a flush here gets + // very messy because we are only invoked when handling + // exceptions so to do this properly, while handling an + // exception we'd have to go off and flush new deletes + // which is risky (likely would hit some other + // confounding exception). + } + + /** + * Returns the number of delete terms in this {@link DocumentsWriterPerThread} + */ + public int numDeleteTerms() { + // public for FlushPolicy + return pendingDeletes.numTermDeletes.get(); + } + + /** + * Returns the number of RAM resident documents in this {@link DocumentsWriterPerThread} + */ + public int getNumDocsInRAM() { + // public for FlushPolicy + return numDocsInRAM; + } + + SegmentCodecs getCodec() { + return flushState.segmentCodecs; + } + + /** Reset after a flush */ + private void doAfterFlush() throws IOException { + segment = null; + consumer.doAfterFlush(); + fieldInfos = new FieldInfos(fieldInfos); + parent.subtractFlushedNumDocs(numDocsInRAM); + numDocsInRAM = 0; + } + + /** + * Prepares this DWPT for flushing. This method will freeze and return the + * {@link DocumentsWriterDeleteQueue}s global buffer and apply all pending + * deletes to this DWPT. + */ + FrozenBufferedDeletes prepareFlush() { + assert numDocsInRAM > 0; + final FrozenBufferedDeletes globalDeletes = deleteQueue.freezeGlobalBuffer(deleteSlice); + /* deleteSlice can possibly be null if we have hit non-aborting exceptions during indexing and never succeeded + adding a document. */ + if (deleteSlice != null) { + // apply all deletes before we flush and release the delete slice + deleteSlice.apply(pendingDeletes, numDocsInRAM); + assert deleteSlice.isEmpty(); + deleteSlice = null; + } + return globalDeletes; + } + + /** Flush all pending docs to a new segment */ + FlushedSegment flush() throws IOException { + assert numDocsInRAM > 0; + assert deleteSlice == null : "all deletes must be applied in prepareFlush"; + flushState = new SegmentWriteState(infoStream, directory, segment, fieldInfos, + numDocsInRAM, writer.getConfig().getTermIndexInterval(), + fieldInfos.buildSegmentCodecs(true), pendingDeletes); + final double startMBUsed = parent.flushControl.netBytes() / 1024. / 1024.; + // Apply delete-by-docID now (delete-byDocID only + // happens when an exception is hit processing that + // doc, eg if analyzer has some problem w/ the text): + if (pendingDeletes.docIDs.size() > 0) { + flushState.deletedDocs = new BitVector(numDocsInRAM); + for(int delDocID : pendingDeletes.docIDs) { + flushState.deletedDocs.set(delDocID); + } + pendingDeletes.bytesUsed.addAndGet(-pendingDeletes.docIDs.size() * BufferedDeletes.BYTES_PER_DEL_DOCID); + pendingDeletes.docIDs.clear(); + } + + if (infoStream != null) { + message("flush postings as segment " + flushState.segmentName + " numDocs=" + numDocsInRAM); + } + + if (aborting) { + if (infoStream != null) { + message("flush: skip because aborting is set"); + } + return null; + } + + boolean success = false; + + try { + + SegmentInfo newSegment = new SegmentInfo(segment, flushState.numDocs, directory, false, fieldInfos.hasProx(), flushState.segmentCodecs, false, fieldInfos); + consumer.flush(flushState); + pendingDeletes.terms.clear(); + newSegment.setHasVectors(flushState.hasVectors); + + if (infoStream != null) { + message("new segment has " + (flushState.deletedDocs == null ? 0 : flushState.deletedDocs.count()) + " deleted docs"); + message("new segment has " + (flushState.hasVectors ? "vectors" : "no vectors")); + message("flushedFiles=" + newSegment.files()); + message("flushed codecs=" + newSegment.getSegmentCodecs()); + } + flushedDocCount += flushState.numDocs; + + final BufferedDeletes segmentDeletes; + if (pendingDeletes.queries.isEmpty()) { + pendingDeletes.clear(); + segmentDeletes = null; + } else { + segmentDeletes = pendingDeletes; + pendingDeletes = new BufferedDeletes(false); + } + + if (infoStream != null) { + final double newSegmentSizeNoStore = newSegment.sizeInBytes(false)/1024./1024.; + final double newSegmentSize = newSegment.sizeInBytes(true)/1024./1024.; + message("flushed: segment=" + newSegment + + " ramUsed=" + nf.format(startMBUsed) + " MB" + + " newFlushedSize=" + nf.format(newSegmentSize) + " MB" + + " (" + nf.format(newSegmentSizeNoStore) + " MB w/o doc stores)" + + " docs/MB=" + nf.format(flushedDocCount / newSegmentSize) + + " new/old=" + nf.format(100.0 * newSegmentSizeNoStore / startMBUsed) + "%"); + } + doAfterFlush(); + success = true; + + return new FlushedSegment(newSegment, segmentDeletes, flushState.deletedDocs); + } finally { + if (!success) { + if (segment != null) { + synchronized(parent.indexWriter) { + parent.indexWriter.deleter.refresh(segment); + } + } + abort(); + } + } + } + + /** Get current segment name we are writing. */ + String getSegment() { + return segment; + } + + long bytesUsed() { + return bytesUsed.get() + pendingDeletes.bytesUsed.get(); + } + + FieldInfos getFieldInfos() { + return fieldInfos; + } + + void message(String message) { + writer.message("DWPT: " + message); + } + + /* Initial chunks size of the shared byte[] blocks used to + store postings data */ + final static int BYTE_BLOCK_NOT_MASK = ~BYTE_BLOCK_MASK; + + /* if you increase this, you must fix field cache impl for + * getTerms/getTermsIndex requires <= 32768 */ + final static int MAX_TERM_LENGTH_UTF8 = BYTE_BLOCK_SIZE-2; + + /* Initial chunks size of the shared int[] blocks used to + store postings data */ + final static int INT_BLOCK_SHIFT = 13; + final static int INT_BLOCK_SIZE = 1 << INT_BLOCK_SHIFT; + final static int INT_BLOCK_MASK = INT_BLOCK_SIZE - 1; + + /* Allocate another int[] from the shared pool */ + int[] getIntBlock() { + int[] b = new int[INT_BLOCK_SIZE]; + bytesUsed.addAndGet(INT_BLOCK_SIZE*RamUsageEstimator.NUM_BYTES_INT); + return b; + } + + void recycleIntBlocks(int[][] blocks, int offset, int length) { + bytesUsed.addAndGet(-(length *(INT_BLOCK_SIZE*RamUsageEstimator.NUM_BYTES_INT))); + } + + final Allocator byteBlockAllocator = new DirectTrackingAllocator(); + + + private class DirectTrackingAllocator extends Allocator { + public DirectTrackingAllocator() { + this(BYTE_BLOCK_SIZE); + } + + public DirectTrackingAllocator(int blockSize) { + super(blockSize); + } + + public byte[] getByteBlock() { + bytesUsed.addAndGet(blockSize); + return new byte[blockSize]; + } + @Override + public void recycleByteBlocks(byte[][] blocks, int start, int end) { + bytesUsed.addAndGet(-((end-start)* blockSize)); + for (int i = start; i < end; i++) { + blocks[i] = null; + } + } + + }; +} --- lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java 1970-01-01 01:00:00.000000000 +0100 +++ lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThreadPool.java 2011-04-29 15:00:59.613952900 +0200 @@ -0,0 +1,272 @@ +package org.apache.lucene.index; +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.util.Iterator; +import java.util.concurrent.locks.ReentrantLock; + +import org.apache.lucene.document.Document; +import org.apache.lucene.index.FieldInfos.FieldNumberBiMap; +import org.apache.lucene.index.SegmentCodecs.SegmentCodecsBuilder; +import org.apache.lucene.index.codecs.CodecProvider; +import org.apache.lucene.util.SetOnce; + +/** + * {@link DocumentsWriterPerThreadPool} controls {@link ThreadState} instances + * and their thread assignments during indexing. Each {@link ThreadState} holds + * a reference to a {@link DocumentsWriterPerThread} that is once a + * {@link ThreadState} is obtained from the pool exclusively used for indexing a + * single document by the obtaining thread. Each indexing thread must obtain + * such a {@link ThreadState} to make progress. Depending on the + * {@link DocumentsWriterPerThreadPool} implementation {@link ThreadState} + * assignments might differ from document to document. + *

+ * Once a {@link DocumentsWriterPerThread} is selected for flush the thread pool + * is reusing the flushing {@link DocumentsWriterPerThread}s ThreadState with a + * new {@link DocumentsWriterPerThread} instance. + *

+ */ +public abstract class DocumentsWriterPerThreadPool { + /** The maximum number of simultaneous threads that may be + * indexing documents at once in IndexWriter; if more + * than this many threads arrive they will wait for + * others to finish. */ + public final static int DEFAULT_MAX_THREAD_STATES = 8; + + /** + * {@link ThreadState} references and guards a + * {@link DocumentsWriterPerThread} instance that is used during indexing to + * build a in-memory index segment. {@link ThreadState} also holds all flush + * related per-thread data controlled by {@link DocumentsWriterFlushControl}. + *

+ * A {@link ThreadState}, its methods and members should only accessed by one + * thread a time. Users must acquire the lock via {@link ThreadState#lock()} + * and release the lock in a finally block via {@link ThreadState#unlock()} + * before accessing the state. + */ + @SuppressWarnings("serial") + public final static class ThreadState extends ReentrantLock { + // package private for FlushPolicy + DocumentsWriterPerThread perThread; + // write access guarded by DocumentsWriterFlushControl + volatile boolean flushPending = false; + // write access guarded by DocumentsWriterFlushControl + long bytesUsed = 0; + // guarded by Reentrant lock + private boolean isActive = true; + + ThreadState(DocumentsWriterPerThread perThread) { + this.perThread = perThread; + } + + /** + * Resets the internal {@link DocumentsWriterPerThread} with the given one. + * if the given DWPT is null this ThreadState is marked as inactive and should not be used + * for indexing anymore. + * @see #isActive() + */ + void resetWriter(DocumentsWriterPerThread perThread) { + assert this.isHeldByCurrentThread(); + if (perThread == null) { + isActive = false; + } + this.perThread = perThread; + this.bytesUsed = 0; + this.flushPending = false; + } + + /** + * Returns true if this ThreadState is still open. This will + * only return false iff the DW has been closed and this + * ThreadState is already checked out for flush. + */ + boolean isActive() { + assert this.isHeldByCurrentThread(); + return isActive; + } + + /** + * Returns the number of currently active bytes in this ThreadState's + * {@link DocumentsWriterPerThread} + */ + public long getBytesUsedPerThread() { + assert this.isHeldByCurrentThread(); + // public for FlushPolicy + return bytesUsed; + } + + /** + * Returns this {@link ThreadState}s {@link DocumentsWriterPerThread} + */ + public DocumentsWriterPerThread getDocumentsWriterPerThread() { + assert this.isHeldByCurrentThread(); + // public for FlushPolicy + return perThread; + } + + /** + * Returns true iff this {@link ThreadState} is marked as flush + * pending otherwise false + */ + public boolean isFlushPending() { + return flushPending; + } + } + + private final ThreadState[] perThreads; + private volatile int numThreadStatesActive; + private CodecProvider codecProvider; + private FieldNumberBiMap globalFieldMap; + private final SetOnce documentsWriter = new SetOnce(); + + /** + * Creates a new {@link DocumentsWriterPerThreadPool} with max. + * {@link #DEFAULT_MAX_THREAD_STATES} thread states. + */ + public DocumentsWriterPerThreadPool() { + this(DEFAULT_MAX_THREAD_STATES); + } + + public DocumentsWriterPerThreadPool(int maxNumPerThreads) { + maxNumPerThreads = (maxNumPerThreads < 1) ? DEFAULT_MAX_THREAD_STATES : maxNumPerThreads; + perThreads = new ThreadState[maxNumPerThreads]; + numThreadStatesActive = 0; + } + + public void initialize(DocumentsWriter documentsWriter, FieldNumberBiMap globalFieldMap, IndexWriterConfig config) { + this.documentsWriter.set(documentsWriter); // thread pool is bound to DW + final CodecProvider codecs = config.getCodecProvider(); + this.codecProvider = codecs; + this.globalFieldMap = globalFieldMap; + for (int i = 0; i < perThreads.length; i++) { + final FieldInfos infos = globalFieldMap.newFieldInfos(SegmentCodecsBuilder.create(codecs)); + perThreads[i] = new ThreadState(new DocumentsWriterPerThread(documentsWriter.directory, documentsWriter, infos, documentsWriter.chain)); + } + } + + /** + * Returns the max number of {@link ThreadState} instances available in this + * {@link DocumentsWriterPerThreadPool} + */ + public int getMaxThreadStates() { + return perThreads.length; + } + + /** + * Returns a new {@link ThreadState} iff any new state is available otherwise + * null. + *

+ * NOTE: the returned {@link ThreadState} is already locked iff non- + * null. + * + * @return a new {@link ThreadState} iff any new state is available otherwise + * null + */ + public synchronized ThreadState newThreadState() { + if (numThreadStatesActive < perThreads.length) { + final ThreadState threadState = perThreads[numThreadStatesActive]; + threadState.lock(); // lock so nobody else will get this ThreadState + numThreadStatesActive++; // increment will publish the ThreadState + threadState.perThread.initialize(); + return threadState; + } + return null; + } + + protected DocumentsWriterPerThread replaceForFlush(ThreadState threadState, boolean closed) { + assert threadState.isHeldByCurrentThread(); + final DocumentsWriterPerThread dwpt = threadState.perThread; + if (!closed) { + final FieldInfos infos = globalFieldMap.newFieldInfos(SegmentCodecsBuilder.create(codecProvider)); + final DocumentsWriterPerThread newDwpt = new DocumentsWriterPerThread(dwpt, infos); + newDwpt.initialize(); + threadState.resetWriter(newDwpt); + } else { + threadState.resetWriter(null); + } + return dwpt; + } + + public void recycle(DocumentsWriterPerThread dwpt) { + // don't recycle DWPT by default + } + + public abstract ThreadState getAndLock(Thread requestingThread, DocumentsWriter documentsWriter, Document doc); + + //public abstract void clearThreadBindings(ThreadState perThread); + + //public abstract void clearAllThreadBindings(); + + /** + * Returns an iterator providing access to all {@link ThreadState} + * instances. + */ + // TODO: new Iterator per indexed doc is overkill...? + public Iterator getAllPerThreadsIterator() { + return getPerThreadsIterator(this.perThreads.length); + } + + /** + * Returns an iterator providing access to all active {@link ThreadState} + * instances. + *

+ * Note: The returned iterator will only iterator + * {@link ThreadState}s that are active at the point in time when this method + * has been called. + * + */ + // TODO: new Iterator per indexed doc is overkill...? + public Iterator getActivePerThreadsIterator() { + return getPerThreadsIterator(numThreadStatesActive); + } + + private Iterator getPerThreadsIterator(final int upto) { + return new Iterator() { + int i = 0; + + public boolean hasNext() { + return i < upto; + } + + public ThreadState next() { + return perThreads[i++]; + } + + public void remove() { + throw new UnsupportedOperationException("remove() not supported."); + } + }; + } + + /** + * Returns the ThreadState with the minimum estimated number of threads + * waiting to acquire its lock or null if no {@link ThreadState} + * is yet visible to the calling thread. + */ + protected ThreadState minContendedThreadState() { + ThreadState minThreadState = null; + // TODO: new Iterator per indexed doc is overkill...? + final Iterator it = getActivePerThreadsIterator(); + while (it.hasNext()) { + final ThreadState state = it.next(); + if (minThreadState == null || state.getQueueLength() < minThreadState.getQueueLength()) { + minThreadState = state; + } + } + return minThreadState; + } +} --- lucene/src/java/org/apache/lucene/index/DocumentsWriterThreadState.java 2011-01-26 17:07:32.771176300 +0100 +++ lucene/src/java/org/apache/lucene/index/DocumentsWriterThreadState.java 1970-01-01 01:00:00.000000000 +0100 @@ -1,47 +0,0 @@ -package org.apache.lucene.index; - -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import java.io.IOException; - -/** Used by DocumentsWriter to maintain per-thread state. - * We keep a separate Posting hash and other state for each - * thread and then merge postings hashes from all threads - * when writing the segment. */ -final class DocumentsWriterThreadState { - - boolean isIdle = true; // false if this is currently in use by a thread - int numThreads = 1; // Number of threads that share this instance - final DocConsumerPerThread consumer; - final DocumentsWriter.DocState docState; - - final DocumentsWriter docWriter; - - public DocumentsWriterThreadState(DocumentsWriter docWriter) throws IOException { - this.docWriter = docWriter; - docState = new DocumentsWriter.DocState(); - docState.infoStream = docWriter.infoStream; - docState.similarityProvider = docWriter.similarityProvider; - docState.docWriter = docWriter; - consumer = docWriter.consumer.addThread(this); - } - - void doAfterFlush() { - numThreads = 0; - } -} --- lucene/src/java/org/apache/lucene/index/FieldsWriter.java 2011-03-22 19:30:26.299099100 +0100 +++ lucene/src/java/org/apache/lucene/index/FieldsWriter.java 2011-04-29 15:01:01.710072800 +0200 @@ -22,9 +22,8 @@ import org.apache.lucene.document.Document; import org.apache.lucene.document.Fieldable; import org.apache.lucene.store.Directory; -import org.apache.lucene.store.RAMOutputStream; -import org.apache.lucene.store.IndexOutput; import org.apache.lucene.store.IndexInput; +import org.apache.lucene.store.IndexOutput; import org.apache.lucene.util.IOUtils; final class FieldsWriter { @@ -83,10 +82,9 @@ // and adds a new entry for this document into the index // stream. This assumes the buffer was already written // in the correct fields format. - void flushDocument(int numStoredFields, RAMOutputStream buffer) throws IOException { + void startDocument(int numStoredFields) throws IOException { indexStream.writeLong(fieldsStream.getFilePointer()); fieldsStream.writeVInt(numStoredFields); - buffer.writeTo(fieldsStream); } void skipDocument() throws IOException { @@ -121,8 +119,8 @@ } } - final void writeField(FieldInfo fi, Fieldable field) throws IOException { - fieldsStream.writeVInt(fi.number); + final void writeField(int fieldNumber, Fieldable field) throws IOException { + fieldsStream.writeVInt(fieldNumber); byte bits = 0; if (field.isTokenized()) bits |= FieldsWriter.FIELD_IS_TOKENIZED; @@ -175,10 +173,9 @@ fieldsStream.writeVInt(storedCount); - for (Fieldable field : fields) { if (field.isStored()) - writeField(fieldInfos.fieldInfo(field.name()), field); + writeField(fieldInfos.fieldNumber(field.name()), field); } } } --- lucene/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java 1970-01-01 01:00:00.000000000 +0100 +++ lucene/src/java/org/apache/lucene/index/FlushByRamOrCountsPolicy.java 2011-04-29 15:01:06.025319600 +0200 @@ -0,0 +1,128 @@ +package org.apache.lucene.index; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState; + +/** + * Default {@link FlushPolicy} implementation that flushes based on RAM used, + * document count and number of buffered deletes depending on the IndexWriter's + * {@link IndexWriterConfig}. + * + *

    + *
  • {@link #onDelete(DocumentsWriterFlushControl, ThreadState)} - flushes + * based on the global number of buffered delete terms iff + * {@link IndexWriterConfig#getMaxBufferedDeleteTerms()} is enabled
  • + *
  • {@link #onInsert(DocumentsWriterFlushControl, ThreadState)} - flushes + * either on the number of documents per {@link DocumentsWriterPerThread} ( + * {@link DocumentsWriterPerThread#getNumDocsInRAM()}) or on the global active + * memory consumption in the current indexing session iff + * {@link IndexWriterConfig#getMaxBufferedDocs()} or + * {@link IndexWriterConfig#getRAMBufferSizeMB()} is enabled respectively
  • + *
  • {@link #onUpdate(DocumentsWriterFlushControl, ThreadState)} - calls + * {@link #onInsert(DocumentsWriterFlushControl, ThreadState)} and + * {@link #onDelete(DocumentsWriterFlushControl, ThreadState)} in order
  • + *
+ * All {@link IndexWriterConfig} settings are used to mark + * {@link DocumentsWriterPerThread} as flush pending during indexing with + * respect to their live updates. + *

+ * If {@link IndexWriterConfig#setRAMBufferSizeMB(double)} is enabled, the + * largest ram consuming {@link DocumentsWriterPerThread} will be marked as + * pending iff the global active RAM consumption is >= the configured max RAM + * buffer. + */ +public class FlushByRamOrCountsPolicy extends FlushPolicy { + + @Override + public void onDelete(DocumentsWriterFlushControl control, ThreadState state) { + if (flushOnDeleteTerms()) { + // Flush this state by num del terms + final int maxBufferedDeleteTerms = indexWriterConfig + .getMaxBufferedDeleteTerms(); + if (control.getNumGlobalTermDeletes() >= maxBufferedDeleteTerms) { + control.setApplyAllDeletes(); + } + } + final DocumentsWriter writer = this.writer.get(); + // If deletes alone are consuming > 1/2 our RAM + // buffer, force them all to apply now. This is to + // prevent too-frequent flushing of a long tail of + // tiny segments: + if ((flushOnRAM() && + writer.deleteQueue.bytesUsed() > (1024*1024*indexWriterConfig.getRAMBufferSizeMB()/2))) { + control.setApplyAllDeletes(); + if (writer.infoStream != null) { + writer.message("force apply deletes bytesUsed=" + writer.deleteQueue.bytesUsed() + " vs ramBuffer=" + (1024*1024*indexWriterConfig.getRAMBufferSizeMB())); + } + } + } + + @Override + public void onInsert(DocumentsWriterFlushControl control, ThreadState state) { + if (flushOnDocCount() + && state.perThread.getNumDocsInRAM() >= indexWriterConfig + .getMaxBufferedDocs()) { + // Flush this state by num docs + control.setFlushPending(state); + } else if (flushOnRAM()) {// flush by RAM + final long limit = (long) (indexWriterConfig.getRAMBufferSizeMB() * 1024.d * 1024.d); + final long totalRam = control.activeBytes(); + if (totalRam >= limit) { + markLargestWriterPending(control, state, totalRam); + } + } + } + + /** + * Marks the most ram consuming active {@link DocumentsWriterPerThread} flush + * pending + */ + protected void markLargestWriterPending(DocumentsWriterFlushControl control, + ThreadState perThreadState, final long currentBytesPerThread) { + control + .setFlushPending(findLargestNonPendingWriter(control, perThreadState)); + } + + /** + * Returns true if this {@link FlushPolicy} flushes on + * {@link IndexWriterConfig#getMaxBufferedDocs()}, otherwise + * false. + */ + protected boolean flushOnDocCount() { + return indexWriterConfig.getMaxBufferedDocs() != IndexWriterConfig.DISABLE_AUTO_FLUSH; + } + + /** + * Returns true if this {@link FlushPolicy} flushes on + * {@link IndexWriterConfig#getMaxBufferedDeleteTerms()}, otherwise + * false. + */ + protected boolean flushOnDeleteTerms() { + return indexWriterConfig.getMaxBufferedDeleteTerms() != IndexWriterConfig.DISABLE_AUTO_FLUSH; + } + + /** + * Returns true if this {@link FlushPolicy} flushes on + * {@link IndexWriterConfig#getRAMBufferSizeMB()}, otherwise + * false. + */ + protected boolean flushOnRAM() { + return indexWriterConfig.getRAMBufferSizeMB() != IndexWriterConfig.DISABLE_AUTO_FLUSH; + } +} --- lucene/src/java/org/apache/lucene/index/FlushPolicy.java 1970-01-01 01:00:00.000000000 +0100 +++ lucene/src/java/org/apache/lucene/index/FlushPolicy.java 2011-04-29 15:00:56.104752200 +0200 @@ -0,0 +1,131 @@ +package org.apache.lucene.index; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +import java.util.Iterator; + +import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState; +import org.apache.lucene.store.Directory; +import org.apache.lucene.util.SetOnce; + +/** + * {@link FlushPolicy} controls when segments are flushed from a RAM resident + * internal data-structure to the {@link IndexWriter}s {@link Directory}. + *

+ * Segments are traditionally flushed by: + *

    + *
  • RAM consumption - configured via + * {@link IndexWriterConfig#setRAMBufferSizeMB(double)}
  • + *
  • Number of RAM resident documents - configured via + * {@link IndexWriterConfig#setMaxBufferedDocs(int)}
  • + *
  • Number of buffered delete terms/queries - configured via + * {@link IndexWriterConfig#setMaxBufferedDeleteTerms(int)}
  • + *
+ * + * The {@link IndexWriter} consults a provided {@link FlushPolicy} to control the + * flushing process. The policy is informed for each added or + * updated document as well as for each delete term. Based on the + * {@link FlushPolicy}, the information provided via {@link ThreadState} and + * {@link DocumentsWriterFlushControl}, the {@link FlushPolicy} decides if a + * {@link DocumentsWriterPerThread} needs flushing and mark it as + * flush-pending via + * {@link DocumentsWriterFlushControl#setFlushPending(ThreadState)}. + * + * @see ThreadState + * @see DocumentsWriterFlushControl + * @see DocumentsWriterPerThread + * @see IndexWriterConfig#setFlushPolicy(FlushPolicy) + */ +public abstract class FlushPolicy { + protected final SetOnce writer = new SetOnce(); + protected IndexWriterConfig indexWriterConfig; + + /** + * Called for each delete term. If this is a delete triggered due to an update + * the given {@link ThreadState} is non-null. + *

+ * Note: This method is called synchronized on the given + * {@link DocumentsWriterFlushControl} and it is guaranteed that the calling + * thread holds the lock on the given {@link ThreadState} + */ + public abstract void onDelete(DocumentsWriterFlushControl control, + ThreadState state); + + /** + * Called for each document update on the given {@link ThreadState}'s + * {@link DocumentsWriterPerThread}. + *

+ * Note: This method is called synchronized on the given + * {@link DocumentsWriterFlushControl} and it is guaranteed that the calling + * thread holds the lock on the given {@link ThreadState} + */ + public void onUpdate(DocumentsWriterFlushControl control, ThreadState state) { + onInsert(control, state); + if (!state.flushPending) { + onDelete(control, state); + } + } + + /** + * Called for each document addition on the given {@link ThreadState}s + * {@link DocumentsWriterPerThread}. + *

+ * Note: This method is synchronized by the given + * {@link DocumentsWriterFlushControl} and it is guaranteed that the calling + * thread holds the lock on the given {@link ThreadState} + */ + public abstract void onInsert(DocumentsWriterFlushControl control, + ThreadState state); + + /** + * Called by {@link DocumentsWriter} to initialize the FlushPolicy + */ + protected synchronized void init(DocumentsWriter docsWriter) { + writer.set(docsWriter); + indexWriterConfig = docsWriter.indexWriter.getConfig(); + } + + /** + * Returns the current most RAM consuming non-pending {@link ThreadState} with + * at least one indexed document. + *

+ * This method will never return null + */ + protected ThreadState findLargestNonPendingWriter( + DocumentsWriterFlushControl control, ThreadState perThreadState) { + assert perThreadState.perThread.getNumDocsInRAM() > 0; + long maxRamSoFar = perThreadState.bytesUsed; + // the dwpt which needs to be flushed eventually + ThreadState maxRamUsingThreadState = perThreadState; + assert !perThreadState.flushPending : "DWPT should have flushed"; + Iterator activePerThreadsIterator = control.allActiveThreads(); + while (activePerThreadsIterator.hasNext()) { + ThreadState next = activePerThreadsIterator.next(); + if (!next.flushPending) { + final long nextRam = next.bytesUsed; + if (nextRam > maxRamSoFar && next.perThread.getNumDocsInRAM() > 0) { + maxRamSoFar = nextRam; + maxRamUsingThreadState = next; + } + } + } + assert writer.get().message( + "set largest ram consuming thread pending on lower watermark"); + return maxRamUsingThreadState; + } + +} --- lucene/src/java/org/apache/lucene/index/FreqProxFieldMergeState.java 2011-01-26 17:07:31.933128400 +0100 +++ lucene/src/java/org/apache/lucene/index/FreqProxFieldMergeState.java 1970-01-01 01:00:00.000000000 +0100 @@ -1,115 +0,0 @@ -package org.apache.lucene.index; - -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import java.io.IOException; -import java.util.Comparator; - -import org.apache.lucene.util.ByteBlockPool; -import org.apache.lucene.util.BytesRef; - -import org.apache.lucene.index.FreqProxTermsWriterPerField.FreqProxPostingsArray; - -// TODO FI: some of this is "generic" to TermsHash* so we -// should factor it out so other consumers don't have to -// duplicate this code - -/** Used by DocumentsWriter to merge the postings from - * multiple ThreadStates when creating a segment */ -final class FreqProxFieldMergeState { - - final FreqProxTermsWriterPerField field; - final int numPostings; - private final ByteBlockPool bytePool; - final int[] termIDs; - final FreqProxPostingsArray postings; - int currentTermID; - - final BytesRef text = new BytesRef(); - - private int postingUpto = -1; - - final ByteSliceReader freq = new ByteSliceReader(); - final ByteSliceReader prox = new ByteSliceReader(); - - int docID; - int termFreq; - - public FreqProxFieldMergeState(FreqProxTermsWriterPerField field, Comparator termComp) { - this.field = field; - this.numPostings = field.termsHashPerField.bytesHash.size(); - this.bytePool = field.perThread.termsHashPerThread.bytePool; - this.termIDs = field.termsHashPerField.sortPostings(termComp); - this.postings = (FreqProxPostingsArray) field.termsHashPerField.postingsArray; - } - - boolean nextTerm() throws IOException { - postingUpto++; - if (postingUpto == numPostings) { - return false; - } - - currentTermID = termIDs[postingUpto]; - docID = 0; - - // Get BytesRef - final int textStart = postings.textStarts[currentTermID]; - bytePool.setBytesRef(text, textStart); - - field.termsHashPerField.initReader(freq, currentTermID, 0); - if (!field.fieldInfo.omitTermFreqAndPositions) { - field.termsHashPerField.initReader(prox, currentTermID, 1); - } - - // Should always be true - boolean result = nextDoc(); - assert result; - - return true; - } - - public boolean nextDoc() throws IOException { - if (freq.eof()) { - if (postings.lastDocCodes[currentTermID] != -1) { - // Return last doc - docID = postings.lastDocIDs[currentTermID]; - if (!field.omitTermFreqAndPositions) - termFreq = postings.docFreqs[currentTermID]; - postings.lastDocCodes[currentTermID] = -1; - return true; - } else - // EOF - return false; - } - - final int code = freq.readVInt(); - if (field.omitTermFreqAndPositions) - docID += code; - else { - docID += code >>> 1; - if ((code & 1) != 0) - termFreq = 1; - else - termFreq = freq.readVInt(); - } - - assert docID != postings.lastDocIDs[currentTermID]; - - return true; - } -} --- lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java 2011-04-20 23:25:31.185589300 +0200 +++ lucene/src/java/org/apache/lucene/index/FreqProxTermsWriter.java 2011-04-29 15:01:03.856195500 +0200 @@ -19,53 +19,33 @@ import java.io.IOException; import java.util.ArrayList; -import java.util.Collection; -import java.util.Comparator; import java.util.List; import java.util.Map; import org.apache.lucene.index.codecs.FieldsConsumer; -import org.apache.lucene.index.codecs.PostingsConsumer; -import org.apache.lucene.index.codecs.TermStats; -import org.apache.lucene.index.codecs.TermsConsumer; -import org.apache.lucene.util.BitVector; import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.CollectionUtil; final class FreqProxTermsWriter extends TermsHashConsumer { @Override - public TermsHashConsumerPerThread addThread(TermsHashPerThread perThread) { - return new FreqProxTermsWriterPerThread(perThread); - } - - @Override void abort() {} - private int flushedDocCount; - // TODO: would be nice to factor out more of this, eg the // FreqProxFieldMergeState, and code to visit all Fields // under the same FieldInfo together, up into TermsHash*. // Other writers would presumably share alot of this... @Override - public void flush(Map> threadsAndFields, final SegmentWriteState state) throws IOException { + public void flush(Map fieldsToFlush, final SegmentWriteState state) throws IOException { // Gather all FieldData's that have postings, across all // ThreadStates List allFields = new ArrayList(); - flushedDocCount = state.numDocs; - - for (Map.Entry> entry : threadsAndFields.entrySet()) { - - Collection fields = entry.getValue(); - - - for (final TermsHashConsumerPerField i : fields) { - final FreqProxTermsWriterPerField perField = (FreqProxTermsWriterPerField) i; - if (perField.termsHashPerField.bytesHash.size() > 0) + for (TermsHashConsumerPerField f : fieldsToFlush.values()) { + final FreqProxTermsWriterPerField perField = (FreqProxTermsWriterPerField) f; + if (perField.termsHashPerField.bytesHash.size() > 0) { allFields.add(perField); } } @@ -77,6 +57,8 @@ final FieldsConsumer consumer = state.segmentCodecs.codec().fieldsConsumer(state); + TermsHash termsHash = null; + /* Current writer chain: FieldsConsumer @@ -89,257 +71,48 @@ -> IMPL: FormatPostingsPositionsWriter */ - int start = 0; - while(start < numAllFields) { - final FieldInfo fieldInfo = allFields.get(start).fieldInfo; - final String fieldName = fieldInfo.name; - - int end = start+1; - while(end < numAllFields && allFields.get(end).fieldInfo.name.equals(fieldName)) - end++; - - FreqProxTermsWriterPerField[] fields = new FreqProxTermsWriterPerField[end-start]; - for(int i=start;i> entry : threadsAndFields.entrySet()) { - FreqProxTermsWriterPerThread perThread = (FreqProxTermsWriterPerThread) entry.getKey(); - perThread.termsHashPerThread.reset(true); + if (termsHash != null) { + termsHash.reset(); } consumer.close(); } BytesRef payload; - /* Walk through all unique text tokens (Posting - * instances) found in this field and serialize them - * into a single RAM segment. */ - void appendPostings(String fieldName, SegmentWriteState state, - FreqProxTermsWriterPerField[] fields, - FieldsConsumer consumer) - throws CorruptIndexException, IOException { - - int numFields = fields.length; - - final BytesRef text = new BytesRef(); - - final FreqProxFieldMergeState[] mergeStates = new FreqProxFieldMergeState[numFields]; - - final TermsConsumer termsConsumer = consumer.addField(fields[0].fieldInfo); - final Comparator termComp = termsConsumer.getComparator(); - - for(int i=0;i termsIterable() { return new Iterable() { // @Override -- not until Java 1.6 --- lucene/src/java/org/apache/lucene/index/Healthiness.java 1970-01-01 01:00:00.000000000 +0100 +++ lucene/src/java/org/apache/lucene/index/Healthiness.java 2011-04-29 15:00:56.592780100 +0200 @@ -0,0 +1,121 @@ +package org.apache.lucene.index; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +import java.util.concurrent.locks.AbstractQueuedSynchronizer; + +import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState; + +/** + * Controls the health status of a {@link DocumentsWriter} sessions. This class + * used to block incoming indexing threads if flushing significantly slower than + * indexing to ensure the {@link DocumentsWriter}s healthiness. If flushing is + * significantly slower than indexing the net memory used within an + * {@link IndexWriter} session can increase very quickly and easily exceed the + * JVM's available memory. + *

+ * To prevent OOM Errors and ensure IndexWriter's stability this class blocks + * incoming threads from indexing once 2 x number of available + * {@link ThreadState}s in {@link DocumentsWriterPerThreadPool} is exceeded. + * Once flushing catches up and the number of flushing DWPT is equal or lower + * than the number of active {@link ThreadState}s threads are released and can + * continue indexing. + */ +//TODO: rename this to DocumentsWriterStallControl (or something like that)? +final class Healthiness { + + @SuppressWarnings("serial") + private static final class Sync extends AbstractQueuedSynchronizer { + volatile boolean hasBlockedThreads = false; // only with assert + + Sync() { + setState(0); + } + + boolean isHealthy() { + return getState() == 0; + } + + boolean trySetStalled() { + int state = getState(); + return compareAndSetState(state, state + 1); + } + + boolean tryReset() { + final int oldState = getState(); + if (oldState == 0) + return true; + if (compareAndSetState(oldState, 0)) { + releaseShared(0); + return true; + } + return false; + } + + @Override + public int tryAcquireShared(int acquires) { + assert maybeSetHasBlocked(getState()); + return getState() == 0 ? 1 : -1; + } + + // only used for testing + private boolean maybeSetHasBlocked(int state) { + hasBlockedThreads |= getState() != 0; + return true; + } + + @Override + public boolean tryReleaseShared(int newState) { + return (getState() == 0); + } + } + + private final Sync sync = new Sync(); + volatile boolean wasStalled = false; // only with asserts + + boolean anyStalledThreads() { + return !sync.isHealthy(); + } + + /** + * Update the stalled flag status. This method will set the stalled flag to + * true iff the number of flushing + * {@link DocumentsWriterPerThread} is greater than the number of active + * {@link DocumentsWriterPerThread}. Otherwise it will reset the + * {@link Healthiness} to healthy and release all threads waiting on + * {@link #waitIfStalled()} + */ + void updateStalled(DocumentsWriterFlushControl flushControl) { + do { + // if we have more flushing DWPT than numActiveDWPT we stall! + while (flushControl.numActiveDWPT() < flushControl.numFlushingDWPT()) { + if (sync.trySetStalled()) { + assert wasStalled = true; + return; + } + } + } while (!sync.tryReset()); + } + + void waitIfStalled() { + sync.acquireShared(0); + } + + boolean hasBlocked() { + return sync.hasBlockedThreads; + } +} \ No newline at end of file --- lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java 2011-01-26 17:07:33.291206000 +0100 +++ lucene/src/java/org/apache/lucene/index/IndexFileDeleter.java 2011-04-29 15:01:15.437858000 +0200 @@ -21,7 +21,13 @@ import java.io.FilenameFilter; import java.io.IOException; import java.io.PrintStream; -import java.util.*; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Date; +import java.util.HashMap; +import java.util.List; +import java.util.Map; import org.apache.lucene.index.codecs.CodecProvider; import org.apache.lucene.store.Directory; @@ -417,7 +423,7 @@ public void checkpoint(SegmentInfos segmentInfos, boolean isCommit) throws IOException { if (infoStream != null) { - message("now checkpoint \"" + segmentInfos.getCurrentSegmentFileName() + "\" [" + segmentInfos.size() + " segments " + "; isCommit = " + isCommit + "]"); + message("now checkpoint \"" + segmentInfos + "\" [" + segmentInfos.size() + " segments " + "; isCommit = " + isCommit + "]"); } // Try again now to delete any previously un-deletable --- lucene/src/java/org/apache/lucene/index/IndexWriter.java 2011-04-03 19:24:37.867758700 +0200 +++ lucene/src/java/org/apache/lucene/index/IndexWriter.java 2011-04-29 15:01:14.150784300 +0200 @@ -35,6 +35,7 @@ import org.apache.lucene.analysis.Analyzer; import org.apache.lucene.document.Document; +import org.apache.lucene.index.DocumentsWriterPerThread.FlushedSegment; import org.apache.lucene.index.FieldInfos.FieldNumberBiMap; import org.apache.lucene.index.IndexWriterConfig.OpenMode; import org.apache.lucene.index.PayloadProcessorProvider.DirPayloadProcessor; @@ -46,6 +47,7 @@ import org.apache.lucene.store.Directory; import org.apache.lucene.store.Lock; import org.apache.lucene.store.LockObtainFailedException; +import org.apache.lucene.util.BitVector; import org.apache.lucene.util.Bits; import org.apache.lucene.util.Constants; import org.apache.lucene.util.ThreadInterruptedException; @@ -54,17 +56,16 @@ /** An IndexWriter creates and maintains an index. -

The create argument to the {@link - #IndexWriter(Directory, IndexWriterConfig) constructor} determines +

The {@link OpenMode} option on + {@link IndexWriterConfig#setOpenMode(OpenMode)} determines whether a new index is created, or whether an existing index is - opened. Note that you can open an index with create=true + opened. Note that you can open an index with {@link OpenMode#CREATE} even while readers are using the index. The old readers will continue to search the "point in time" snapshot they had opened, - and won't see the newly created index until they re-open. There are - also {@link #IndexWriter(Directory, IndexWriterConfig) constructors} - with no create argument which will create a new index - if there is not already an index at the provided path and otherwise - open the existing index.

+ and won't see the newly created index until they re-open. If + {@link OpenMode#CREATE_OR_APPEND} is used IndexWriter will create a + new index if there is not already an index at the provided path + and otherwise open the existing index.

In either case, documents are added with {@link #addDocument(Document) addDocument} and removed with {@link #deleteDocuments(Term)} or {@link @@ -76,15 +77,19 @@

These changes are buffered in memory and periodically flushed to the {@link Directory} (during the above method - calls). A flush is triggered when there are enough - buffered deletes (see {@link IndexWriterConfig#setMaxBufferedDeleteTerms}) - or enough added documents since the last flush, whichever - is sooner. For the added documents, flushing is triggered - either by RAM usage of the documents (see {@link - IndexWriterConfig#setRAMBufferSizeMB}) or the number of added documents. - The default is to flush when RAM usage hits 16 MB. For + calls). A flush is triggered when there are enough added documents + since the last flush. Flushing is triggered either by RAM usage of the + documents (see {@link IndexWriterConfig#setRAMBufferSizeMB}) or the + number of added documents (see {@link IndexWriterConfig#setMaxBufferedDocs(int)}). + The default is to flush when RAM usage hits + {@value IndexWriterConfig#DEFAULT_RAM_BUFFER_SIZE_MB} MB. For best indexing speed you should flush by RAM usage with a - large RAM buffer. Note that flushing just moves the + large RAM buffer. Additionally, if IndexWriter reaches the configured number of + buffered deletes (see {@link IndexWriterConfig#setMaxBufferedDeleteTerms}) + the deleted terms and queries are flushed and applied to existing segments. + In contrast to the other flush options {@link IndexWriterConfig#setRAMBufferSizeMB} and + {@link IndexWriterConfig#setMaxBufferedDocs(int)}, deleted terms + won't trigger a segment flush. Note that flushing just moves the internal buffered state in IndexWriter into the index, but these changes are not visible to IndexReader until either {@link #commit()} or {@link #close} is called. A flush may @@ -195,7 +200,7 @@ * printed to infoStream, if set (see {@link * #setInfoStream}). */ - public final static int MAX_TERM_LENGTH = DocumentsWriter.MAX_TERM_LENGTH_UTF8; + public final static int MAX_TERM_LENGTH = DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8; // The normal read buffer size defaults to 1024, but // increasing this during merging seems to yield @@ -225,7 +230,7 @@ final FieldNumberBiMap globalFieldNumberMap; private DocumentsWriter docWriter; - private IndexFileDeleter deleter; + final IndexFileDeleter deleter; private Set segmentsToOptimize = new HashSet(); // used by optimize to note those needing optimization private int optimizeMaxNumSegments; @@ -247,8 +252,8 @@ private long mergeGen; private boolean stopMerges; - private final AtomicInteger flushCount = new AtomicInteger(); - private final AtomicInteger flushDeletesCount = new AtomicInteger(); + final AtomicInteger flushCount = new AtomicInteger(); + final AtomicInteger flushDeletesCount = new AtomicInteger(); final ReaderPool readerPool = new ReaderPool(); final BufferedDeletesStream bufferedDeletesStream; @@ -345,25 +350,52 @@ if (infoStream != null) { message("flush at getReader"); } - // Do this up front before flushing so that the readers // obtained during this flush are pooled, the first time // this method is called: poolReaders = true; - + final IndexReader r; + doBeforeFlush(); + final boolean anySegmentFlushed; + /* + * for releasing a NRT reader we must ensure that + * DW doesn't add any segments or deletes until we are + * done with creating the NRT DirectoryReader. + * We release the two stage full flush after we are done opening the + * directory reader! + */ + synchronized (fullFlushLock) { + boolean success = false; + try { + anySegmentFlushed = docWriter.flushAllThreads(); + if (!anySegmentFlushed) { + // prevent double increment since docWriter#doFlush increments the flushcount + // if we flushed anything. + flushCount.incrementAndGet(); + } + success = true; // Prevent segmentInfos from changing while opening the // reader; in theory we could do similar retry logic, // just like we do when loading segments_N - IndexReader r; synchronized(this) { - flush(false, applyAllDeletes); + maybeApplyDeletes(applyAllDeletes); r = new DirectoryReader(this, segmentInfos, config.getReaderTermsIndexDivisor(), codecs, applyAllDeletes); if (infoStream != null) { message("return reader version=" + r.getVersion() + " reader=" + r); } } + } finally { + if (!success && infoStream != null) { + message("hit exception during while NRT reader"); + } + // Done: finish the full flush! + docWriter.finishFullFlush(success); + doAfterFlush(); + } + } + if (anySegmentFlushed) { maybeMerge(); - + } if (infoStream != null) { message("getReader took " + (System.currentTimeMillis() - tStart) + " msec"); } @@ -790,8 +822,7 @@ // start with previous field numbers, but new FieldInfos globalFieldNumberMap = segmentInfos.getOrLoadGlobalFieldNumberMap(directory); - docWriter = new DocumentsWriter(config, directory, this, conf.getIndexingChain(), - globalFieldNumberMap.newFieldInfos(SegmentCodecsBuilder.create(codecs)), bufferedDeletesStream); + docWriter = new DocumentsWriter(config, directory, this, globalFieldNumberMap, bufferedDeletesStream); docWriter.setInfoStream(infoStream); // Default deleter (for backwards compatibility) is @@ -1196,22 +1227,7 @@ * @throws IOException if there is a low-level IO error */ public void addDocument(Document doc, Analyzer analyzer) throws CorruptIndexException, IOException { - ensureOpen(); - boolean doFlush = false; - boolean success = false; - try { - try { - doFlush = docWriter.updateDocument(doc, analyzer, null); - success = true; - } finally { - if (!success && infoStream != null) - message("hit exception adding document"); - } - if (doFlush) - flush(true, false); - } catch (OutOfMemoryError oom) { - handleOOM(oom, "addDocument"); - } + updateDocument(null, doc, analyzer); } /** @@ -1228,9 +1244,7 @@ public void deleteDocuments(Term term) throws CorruptIndexException, IOException { ensureOpen(); try { - if (docWriter.deleteTerm(term, false)) { - flush(true, false); - } + docWriter.deleteTerms(term); } catch (OutOfMemoryError oom) { handleOOM(oom, "deleteDocuments(Term)"); } @@ -1238,7 +1252,8 @@ /** * Deletes the document(s) containing any of the - * terms. All deletes are flushed at the same time. + * terms. All given deletes are applied and flushed atomically + * at the same time. * *

NOTE: if this method hits an OutOfMemoryError * you should immediately close the writer. See NOTE: if this method hits an OutOfMemoryError * you should immediately close the writer. See 0; + newSegment.setDelCount(delCount); + newSegment.advanceDelGen(); + final String delFileName = newSegment.getDelFileName(); + if (infoStream != null) { + message("flush: write " + delCount + " deletes to " + delFileName); + } + boolean success2 = false; + try { + // TODO: in the NRT case it'd be better to hand + // this del vector over to the + // shortly-to-be-opened SegmentReader and let it + // carry the changes; there's no reason to use + // filesystem as intermediary here. + flushedSegment.deletedDocuments.write(directory, delFileName); + success2 = true; + } finally { + if (!success2) { + try { + directory.deleteFile(delFileName); + } catch (Throwable t) { + // suppress this so we keep throwing the + // original exception + } + } + } + } + + success = true; + } finally { + if (!success) { + if (infoStream != null) { + message("hit exception " + + "reating compound file for newly flushed segment " + newSegment.name); + } + + synchronized(this) { + deleter.refresh(newSegment.name); + } + } + } + return newSegment; + } + + /** + * Atomically adds the segment private delete packet and publishes the flushed + * segments SegmentInfo to the index writer. NOTE: use + * {@link #prepareFlushedSegment(FlushedSegment)} to obtain the + * {@link SegmentInfo} for the flushed segment. + * + * @see #prepareFlushedSegment(FlushedSegment) + */ + synchronized void publishFlushedSegment(SegmentInfo newSegment, + FrozenBufferedDeletes packet, FrozenBufferedDeletes globalPacket) throws IOException { + // Lock order IW -> BDS + synchronized (bufferedDeletesStream) { + if (globalPacket != null && globalPacket.any()) { + bufferedDeletesStream.push(globalPacket); + } + // Publishing the segment must be synched on IW -> BDS to make the sure + // that no merge prunes away the seg. private delete packet + final long nextGen; + if (packet != null && packet.any()) { + nextGen = bufferedDeletesStream.push(packet); + } else { + // Since we don't have a delete packet to apply we can get a new + // generation right away + nextGen = bufferedDeletesStream.getNextGen(); + } + newSegment.setBufferedDeletesGen(nextGen); + segmentInfos.add(newSegment); + checkpoint(); + } + } + + synchronized boolean useCompoundFile(SegmentInfo segmentInfo) throws IOException { + return mergePolicy.useCompoundFile(segmentInfos, segmentInfo); + } + private synchronized void resetMergeExceptions() { mergeExceptions = new ArrayList(); mergeGen++; @@ -2230,6 +2359,7 @@ merger.add(reader); int docCount = merger.merge(); // merge 'em + final FieldInfos fieldInfos = merger.fieldInfos(); SegmentInfo info = new SegmentInfo(mergedName, docCount, directory, false, fieldInfos.hasProx(), merger.getSegmentCodecs(), @@ -2441,6 +2571,10 @@ } } + // Ensures only one flush() is actually flushing segments + // at a time: + private final Object fullFlushLock = new Object(); + /** * Flush all in-memory buffered updates (adds and deletes) * to the Directory. @@ -2464,64 +2598,67 @@ } } - // TODO: this method should not have to be entirely - // synchronized, ie, merges should be allowed to commit - // even while a flush is happening - private synchronized boolean doFlush(boolean applyAllDeletes) throws CorruptIndexException, IOException { - + private boolean doFlush(boolean applyAllDeletes) throws CorruptIndexException, IOException { if (hitOOM) { throw new IllegalStateException("this writer hit an OutOfMemoryError; cannot flush"); } doBeforeFlush(); - assert testPoint("startDoFlush"); - - // We may be flushing because it was triggered by doc - // count, del count, ram usage (in which case flush - // pending is already set), or we may be flushing - // due to external event eg getReader or commit is - // called (in which case we now set it, and this will - // pause all threads): - flushControl.setFlushPendingNoWait("explicit flush"); - boolean success = false; - try { if (infoStream != null) { message(" start flush: applyAllDeletes=" + applyAllDeletes); message(" index before flush " + segString()); } + final boolean anySegmentFlushed; - final SegmentInfo newSegment = docWriter.flush(this, deleter, mergePolicy, segmentInfos); - if (newSegment != null) { - setDiagnostics(newSegment, "flush"); - segmentInfos.add(newSegment); - checkpoint(); + synchronized (fullFlushLock) { + try { + anySegmentFlushed = docWriter.flushAllThreads(); + success = true; + } finally { + docWriter.finishFullFlush(success); } - - if (!applyAllDeletes) { - // If deletes alone are consuming > 1/2 our RAM - // buffer, force them all to apply now. This is to - // prevent too-frequent flushing of a long tail of - // tiny segments: - if (flushControl.getFlushDeletes() || - (config.getRAMBufferSizeMB() != IndexWriterConfig.DISABLE_AUTO_FLUSH && - bufferedDeletesStream.bytesUsed() > (1024*1024*config.getRAMBufferSizeMB()/2))) { - applyAllDeletes = true; - if (infoStream != null) { - message("force apply deletes bytesUsed=" + bufferedDeletesStream.bytesUsed() + " vs ramBuffer=" + (1024*1024*config.getRAMBufferSizeMB())); } + success = false; + synchronized(this) { + maybeApplyDeletes(applyAllDeletes); + doAfterFlush(); + if (!anySegmentFlushed) { + // flushCount is incremented in flushAllThreads + flushCount.incrementAndGet(); + } + success = true; + return anySegmentFlushed; + } + } catch (OutOfMemoryError oom) { + handleOOM(oom, "doFlush"); + // never hit + return false; + } finally { + if (!success && infoStream != null) + message("hit exception during flush"); } } + final synchronized void maybeApplyDeletes(boolean applyAllDeletes) throws IOException { if (applyAllDeletes) { if (infoStream != null) { message("apply all deletes during flush"); } + applyAllDeletes(); + } else if (infoStream != null) { + message("don't apply deletes now delTermCount=" + bufferedDeletesStream.numTerms() + " bytesUsed=" + bufferedDeletesStream.bytesUsed()); + } + + } + + final synchronized void applyAllDeletes() throws IOException { flushDeletesCount.incrementAndGet(); - final BufferedDeletesStream.ApplyDeletesResult result = bufferedDeletesStream.applyDeletes(readerPool, segmentInfos); + final BufferedDeletesStream.ApplyDeletesResult result = bufferedDeletesStream + .applyDeletes(readerPool, segmentInfos); if (result.anyDeletes) { checkpoint(); } @@ -2529,7 +2666,7 @@ if (infoStream != null) { message("drop 100% deleted segments: " + result.allDeleted); } - for(SegmentInfo info : result.allDeleted) { + for (SegmentInfo info : result.allDeleted) { // If a merge has already registered for this // segment, we leave it in the readerPool; the // merge will skip merging it and will then drop @@ -2544,28 +2681,6 @@ checkpoint(); } bufferedDeletesStream.prune(segmentInfos); - assert !bufferedDeletesStream.any(); - flushControl.clearDeletes(); - } else if (infoStream != null) { - message("don't apply deletes now delTermCount=" + bufferedDeletesStream.numTerms() + " bytesUsed=" + bufferedDeletesStream.bytesUsed()); - } - - doAfterFlush(); - flushCount.incrementAndGet(); - - success = true; - - return newSegment != null; - - } catch (OutOfMemoryError oom) { - handleOOM(oom, "doFlush"); - // never hit - return false; - } finally { - flushControl.clearFlushPending(); - if (!success && infoStream != null) - message("hit exception during flush"); - } } /** Expert: Return the total size of all index files currently cached in memory. @@ -2573,7 +2688,14 @@ */ public final long ramSizeInBytes() { ensureOpen(); - return docWriter.bytesUsed() + bufferedDeletesStream.bytesUsed(); + return docWriter.flushControl.netBytes() + bufferedDeletesStream.bytesUsed(); + } + + // for testing only + DocumentsWriter getDocsWriter() { + boolean test = false; + assert test = true; + return test?docWriter: null; } /** Expert: Return the number of documents currently @@ -2723,7 +2845,7 @@ message("merged segment " + merge.info + " is 100% deleted" + (keepFullyDeletedSegments ? "" : "; skipping insert")); } - final Set mergedAway = new HashSet(merge.segments); + final Set mergedAway = new HashSet(merge.segments); int segIdx = 0; int newSegIdx = 0; boolean inserted = false; @@ -2998,7 +3120,6 @@ // Lock order: IW -> BD bufferedDeletesStream.prune(segmentInfos); - Map details = new HashMap(); details.put("optimize", Boolean.toString(merge.optimize)); details.put("mergeFactor", Integer.toString(merge.segments.size())); @@ -3019,11 +3140,11 @@ mergingSegments.add(merge.info); } - private void setDiagnostics(SegmentInfo info, String source) { + static void setDiagnostics(SegmentInfo info, String source) { setDiagnostics(info, source, null); } - private void setDiagnostics(SegmentInfo info, String source, Map details) { + private static void setDiagnostics(SegmentInfo info, String source, Map details) { Map diagnostics = new HashMap(); diagnostics.put("source", source); diagnostics.put("lucene.version", Constants.LUCENE_VERSION); @@ -3325,12 +3446,12 @@ // For test purposes. final int getBufferedDeleteTermsSize() { - return docWriter.getPendingDeletes().terms.size(); + return docWriter.getBufferedDeleteTermsSize(); } // For test purposes. final int getNumBufferedDeleteTerms() { - return docWriter.getPendingDeletes().numTermDeletes.get(); + return docWriter.getNumBufferedDeleteTerms(); } // utility routines for tests @@ -3655,124 +3776,4 @@ public PayloadProcessorProvider getPayloadProcessorProvider() { return payloadProcessorProvider; } - - // decides when flushes happen - final class FlushControl { - - private boolean flushPending; - private boolean flushDeletes; - private int delCount; - private int docCount; - private boolean flushing; - - private synchronized boolean setFlushPending(String reason, boolean doWait) { - if (flushPending || flushing) { - if (doWait) { - while(flushPending || flushing) { - try { - wait(); - } catch (InterruptedException ie) { - throw new ThreadInterruptedException(ie); - } - } - } - return false; - } else { - if (infoStream != null) { - message("now trigger flush reason=" + reason); - } - flushPending = true; - return flushPending; - } - } - - public synchronized void setFlushPendingNoWait(String reason) { - setFlushPending(reason, false); - } - - public synchronized boolean getFlushPending() { - return flushPending; - } - - public synchronized boolean getFlushDeletes() { - return flushDeletes; - } - - public synchronized void clearFlushPending() { - if (infoStream != null) { - message("clearFlushPending"); - } - flushPending = false; - flushDeletes = false; - docCount = 0; - notifyAll(); - } - - public synchronized void clearDeletes() { - delCount = 0; - } - - public synchronized boolean waitUpdate(int docInc, int delInc) { - return waitUpdate(docInc, delInc, false); - } - - public synchronized boolean waitUpdate(int docInc, int delInc, boolean skipWait) { - while(flushPending) { - try { - wait(); - } catch (InterruptedException ie) { - throw new ThreadInterruptedException(ie); - } - } - - // skipWait is only used when a thread is BOTH adding - // a doc and buffering a del term, and, the adding of - // the doc already triggered a flush - if (skipWait) { - docCount += docInc; - delCount += delInc; - return false; - } - - final int maxBufferedDocs = config.getMaxBufferedDocs(); - if (maxBufferedDocs != IndexWriterConfig.DISABLE_AUTO_FLUSH && - (docCount+docInc) >= maxBufferedDocs) { - return setFlushPending("maxBufferedDocs", true); - } - docCount += docInc; - - final int maxBufferedDeleteTerms = config.getMaxBufferedDeleteTerms(); - if (maxBufferedDeleteTerms != IndexWriterConfig.DISABLE_AUTO_FLUSH && - (delCount+delInc) >= maxBufferedDeleteTerms) { - flushDeletes = true; - return setFlushPending("maxBufferedDeleteTerms", true); - } - delCount += delInc; - - return flushByRAMUsage("add delete/doc"); - } - - public synchronized boolean flushByRAMUsage(String reason) { - final double ramBufferSizeMB = config.getRAMBufferSizeMB(); - if (ramBufferSizeMB != IndexWriterConfig.DISABLE_AUTO_FLUSH) { - final long limit = (long) (ramBufferSizeMB*1024*1024); - long used = bufferedDeletesStream.bytesUsed() + docWriter.bytesUsed(); - if (used >= limit) { - - // DocumentsWriter may be able to free up some - // RAM: - // Lock order: FC -> DW - docWriter.balanceRAM(); - - used = bufferedDeletesStream.bytesUsed() + docWriter.bytesUsed(); - if (used >= limit) { - return setFlushPending("ram full: " + reason, false); - } - } - } - return false; - } - } - - final FlushControl flushControl = new FlushControl(); } --- lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java 2011-04-03 19:24:37.925762000 +0200 +++ lucene/src/java/org/apache/lucene/index/IndexWriterConfig.java 2011-04-29 15:01:33.117869200 +0200 @@ -18,7 +18,7 @@ */ import org.apache.lucene.analysis.Analyzer; -import org.apache.lucene.index.DocumentsWriter.IndexingChain; +import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain; import org.apache.lucene.index.IndexWriter.IndexReaderWarmer; import org.apache.lucene.index.codecs.CodecProvider; import org.apache.lucene.search.IndexSearcher; @@ -82,18 +82,14 @@ */ public static long WRITE_LOCK_TIMEOUT = 1000; - /** The maximum number of simultaneous threads that may be - * indexing documents at once in IndexWriter; if more - * than this many threads arrive they will wait for - * others to finish. */ - public final static int DEFAULT_MAX_THREAD_STATES = 8; - /** Default setting for {@link #setReaderPooling}. */ public final static boolean DEFAULT_READER_POOLING = false; /** Default value is 1. Change using {@link #setReaderTermsIndexDivisor(int)}. */ public static final int DEFAULT_READER_TERMS_INDEX_DIVISOR = IndexReader.DEFAULT_TERMS_INDEX_DIVISOR; + /** Default value is 1945. Change using {@link #setRAMPerThreadHardLimitMB(int)} */ + public static final int DEFAULT_RAM_PER_THREAD_HARD_LIMIT_MB = 1945; /** * Sets the default (for any instance) maximum time to wait for a write lock * (in milliseconds). @@ -127,9 +123,11 @@ private volatile IndexReaderWarmer mergedSegmentWarmer; private volatile CodecProvider codecProvider; private volatile MergePolicy mergePolicy; - private volatile int maxThreadStates; + private volatile DocumentsWriterPerThreadPool indexerThreadPool; private volatile boolean readerPooling; private volatile int readerTermsIndexDivisor; + private volatile FlushPolicy flushPolicy; + private volatile int perThreadHardLimitMB; private Version matchVersion; @@ -153,13 +151,14 @@ maxBufferedDeleteTerms = DEFAULT_MAX_BUFFERED_DELETE_TERMS; ramBufferSizeMB = DEFAULT_RAM_BUFFER_SIZE_MB; maxBufferedDocs = DEFAULT_MAX_BUFFERED_DOCS; - indexingChain = DocumentsWriter.defaultIndexingChain; + indexingChain = DocumentsWriterPerThread.defaultIndexingChain; mergedSegmentWarmer = null; codecProvider = CodecProvider.getDefault(); mergePolicy = new TieredMergePolicy(); - maxThreadStates = DEFAULT_MAX_THREAD_STATES; readerPooling = DEFAULT_READER_POOLING; + indexerThreadPool = new ThreadAffinityDocumentsWriterThreadPool(); readerTermsIndexDivisor = DEFAULT_READER_TERMS_INDEX_DIVISOR; + perThreadHardLimitMB = DEFAULT_RAM_PER_THREAD_HARD_LIMIT_MB; } @Override @@ -343,15 +342,16 @@ /** * Determines the minimal number of delete terms required before the buffered - * in-memory delete terms are applied and flushed. If there are documents - * buffered in memory at the time, they are merged and a new segment is - * created. - - *

Disabled by default (writer flushes by RAM usage). + * in-memory delete terms and queries are applied and flushed. + *

Disabled by default (writer flushes by RAM usage).

+ *

+ * NOTE: This setting won't trigger a segment flush. + *

* * @throws IllegalArgumentException if maxBufferedDeleteTerms * is enabled but smaller than 1 * @see #setRAMBufferSizeMB + * @see #setFlushPolicy(FlushPolicy) * *

Takes effect immediately, but only the next time a * document is added, updated or deleted. @@ -366,8 +366,8 @@ } /** - * Returns the number of buffered deleted terms that will trigger a flush if - * enabled. + * Returns the number of buffered deleted terms that will trigger a flush of all + * buffered deletes if enabled. * * @see #setMaxBufferedDeleteTerms(int) */ @@ -380,32 +380,40 @@ * and deletions before they are flushed to the Directory. Generally for * faster indexing performance it's best to flush by RAM usage instead of * document count and use as large a RAM buffer as you can. - * *

* When this is set, the writer will flush whenever buffered documents and * deletions use this much RAM. Pass in {@link #DISABLE_AUTO_FLUSH} to prevent * triggering a flush due to RAM usage. Note that if flushing by document * count is also enabled, then the flush will be triggered by whichever comes * first. - * + *

+ * The maximum RAM limit is inherently determined by the JVMs available memory. + * Yet, an {@link IndexWriter} session can consume a significantly larger amount + * of memory than the given RAM limit since this limit is just an indicator when + * to flush memory resident documents to the Directory. Flushes are likely happen + * concurrently while other threads adding documents to the writer. For application + * stability the available memory in the JVM should be significantly larger than + * the RAM buffer used for indexing. *

* NOTE: the account of RAM usage for pending deletions is only * approximate. Specifically, if you delete by Query, Lucene currently has no * way to measure the RAM usage of individual Queries so the accounting will * under-estimate and you should compensate by either calling commit() * periodically yourself, or by using {@link #setMaxBufferedDeleteTerms(int)} - * to flush by count instead of RAM usage (each buffered delete Query counts - * as one). - * + * to flush and apply buffered deletes by count instead of RAM usage + * (for each buffered delete Query a constant number of bytes is used to estimate + * RAM usage). Note that enabling {@link #setMaxBufferedDeleteTerms(int)} will + * not trigger any segment flushes. + *

+ * NOTE: It's not guaranteed that all memory resident documents are flushed + * once this limit is exceeded. Depending on the configured {@link FlushPolicy} only a + * subset of the buffered documents are flushed and therefore only parts of the RAM + * buffer is released. *

- * NOTE: because IndexWriter uses ints when managing its - * internal storage, the absolute maximum value for this setting is somewhat - * less than 2048 MB. The precise limit depends on various factors, such as - * how large your documents are, how many fields have norms, etc., so it's - * best to set this value comfortably under 2048. * - *

* The default value is {@link #DEFAULT_RAM_BUFFER_SIZE_MB}. + * @see #setFlushPolicy(FlushPolicy) + * @see #setRAMPerThreadHardLimitMB(int) * *

Takes effect immediately, but only the next time a * document is added, updated or deleted. @@ -413,12 +421,9 @@ * @throws IllegalArgumentException * if ramBufferSize is enabled but non-positive, or it disables * ramBufferSize when maxBufferedDocs is already disabled + * */ public IndexWriterConfig setRAMBufferSizeMB(double ramBufferSizeMB) { - if (ramBufferSizeMB > 2048.0) { - throw new IllegalArgumentException("ramBufferSize " + ramBufferSizeMB - + " is too large; should be comfortably less than 2048"); - } if (ramBufferSizeMB != DISABLE_AUTO_FLUSH && ramBufferSizeMB <= 0.0) throw new IllegalArgumentException( "ramBufferSize should be > 0.0 MB when enabled"); @@ -453,7 +458,7 @@ * document is added, updated or deleted. * * @see #setRAMBufferSizeMB(double) - * + * @see #setFlushPolicy(FlushPolicy) * @throws IllegalArgumentException * if maxBufferedDocs is enabled but smaller than 2, or it disables * maxBufferedDocs when ramBufferSize is already disabled @@ -529,22 +534,33 @@ return mergePolicy; } - /** - * Sets the max number of simultaneous threads that may be indexing documents - * at once in IndexWriter. Values < 1 are invalid and if passed - * maxThreadStates will be set to - * {@link #DEFAULT_MAX_THREAD_STATES}. - * - *

Only takes effect when IndexWriter is first created. */ - public IndexWriterConfig setMaxThreadStates(int maxThreadStates) { - this.maxThreadStates = maxThreadStates < 1 ? DEFAULT_MAX_THREAD_STATES : maxThreadStates; + /** Expert: Sets the {@link DocumentsWriterPerThreadPool} instance used by the + * IndexWriter to assign thread-states to incoming indexing threads. If no + * {@link DocumentsWriterPerThreadPool} is set {@link IndexWriter} will use + * {@link ThreadAffinityDocumentsWriterThreadPool} with max number of + * thread-states set to {@value DocumentsWriterPerThreadPool#DEFAULT_MAX_THREAD_STATES} (see + * {@link DocumentsWriterPerThreadPool#DEFAULT_MAX_THREAD_STATES}). + *

+ *

+ * NOTE: The given {@link DocumentsWriterPerThreadPool} instance must not be used with + * other {@link IndexWriter} instances once it has been initialized / associated with an + * {@link IndexWriter}. + *

+ *

+ * NOTE: This only takes effect when IndexWriter is first created.

*/ + public IndexWriterConfig setIndexerThreadPool(DocumentsWriterPerThreadPool threadPool) { + if(threadPool == null) { + throw new IllegalArgumentException("DocumentsWriterPerThreadPool must not be nul"); + } + this.indexerThreadPool = threadPool; return this; } - /** Returns the max number of simultaneous threads that - * may be indexing documents at once in IndexWriter. */ - public int getMaxThreadStates() { - return maxThreadStates; + /** Returns the configured {@link DocumentsWriterPerThreadPool} instance. + * @see #setIndexerThreadPool(DocumentsWriterPerThreadPool) + * @return the configured {@link DocumentsWriterPerThreadPool} instance.*/ + public DocumentsWriterPerThreadPool getIndexerThreadPool() { + return this.indexerThreadPool; } /** By default, IndexWriter does not pool the @@ -572,7 +588,7 @@ * *

Only takes effect when IndexWriter is first created. */ IndexWriterConfig setIndexingChain(IndexingChain indexingChain) { - this.indexingChain = indexingChain == null ? DocumentsWriter.defaultIndexingChain : indexingChain; + this.indexingChain = indexingChain == null ? DocumentsWriterPerThread.defaultIndexingChain : indexingChain; return this; } @@ -604,6 +620,53 @@ return readerTermsIndexDivisor; } + /** + * Expert: Controls when segments are flushed to disk during indexing. + * The {@link FlushPolicy} initialized during {@link IndexWriter} instantiation and once initialized + * the given instance is bound to this {@link IndexWriter} and should not be used with another writer. + * @see #setMaxBufferedDeleteTerms(int) + * @see #setMaxBufferedDocs(int) + * @see #setRAMBufferSizeMB(double) + */ + public IndexWriterConfig setFlushPolicy(FlushPolicy flushPolicy) { + this.flushPolicy = flushPolicy; + return this; + } + + /** + * Expert: Sets the maximum memory consumption per thread triggering a forced + * flush if exceeded. A {@link DocumentsWriterPerThread} is forcefully flushed + * once it exceeds this limit even if the {@link #getRAMBufferSizeMB()} has + * not been exceeded. This is a safety limit to prevent a + * {@link DocumentsWriterPerThread} from address space exhaustion due to its + * internal 32 bit signed integer based memory addressing. + * The given value must be less that 2GB (2048MB) + * + * @see #DEFAULT_RAM_PER_THREAD_HARD_LIMIT_MB + */ + public IndexWriterConfig setRAMPerThreadHardLimitMB(int perThreadHardLimitMB) { + if (perThreadHardLimitMB <= 0 || perThreadHardLimitMB >= 2048) { + throw new IllegalArgumentException("PerThreadHardLimit must be greater than 0 and less than 2048MB"); + } + this.perThreadHardLimitMB = perThreadHardLimitMB; + return this; + } + + /** + * Returns the max amount of memory each {@link DocumentsWriterPerThread} can + * consume until forcefully flushed. + * @see #setRAMPerThreadHardLimitMB(int) + */ + public int getRAMPerThreadHardLimitMB() { + return perThreadHardLimitMB; + } + /** + * @see #setFlushPolicy(FlushPolicy) + */ + public FlushPolicy getFlushPolicy() { + return flushPolicy; + } + @Override public String toString() { StringBuilder sb = new StringBuilder(); @@ -623,9 +686,13 @@ sb.append("mergedSegmentWarmer=").append(mergedSegmentWarmer).append("\n"); sb.append("codecProvider=").append(codecProvider).append("\n"); sb.append("mergePolicy=").append(mergePolicy).append("\n"); - sb.append("maxThreadStates=").append(maxThreadStates).append("\n"); + sb.append("indexerThreadPool=").append(indexerThreadPool).append("\n"); sb.append("readerPooling=").append(readerPooling).append("\n"); sb.append("readerTermsIndexDivisor=").append(readerTermsIndexDivisor).append("\n"); + sb.append("flushPolicy=").append(flushPolicy).append("\n"); + sb.append("perThreadHardLimitMB=").append(perThreadHardLimitMB).append("\n"); + return sb.toString(); } + } --- lucene/src/java/org/apache/lucene/index/IntBlockPool.java 2011-01-26 17:07:33.015190300 +0100 +++ lucene/src/java/org/apache/lucene/index/IntBlockPool.java 2011-04-29 15:01:12.470688200 +0200 @@ -1,5 +1,7 @@ package org.apache.lucene.index; +import java.util.Arrays; + /** * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -22,24 +24,24 @@ public int[][] buffers = new int[10][]; int bufferUpto = -1; // Which buffer we are upto - public int intUpto = DocumentsWriter.INT_BLOCK_SIZE; // Where we are in head buffer + public int intUpto = DocumentsWriterPerThread.INT_BLOCK_SIZE; // Where we are in head buffer public int[] buffer; // Current head buffer - public int intOffset = -DocumentsWriter.INT_BLOCK_SIZE; // Current head offset + public int intOffset = -DocumentsWriterPerThread.INT_BLOCK_SIZE; // Current head offset - final private DocumentsWriter docWriter; + final private DocumentsWriterPerThread docWriter; - public IntBlockPool(DocumentsWriter docWriter) { + public IntBlockPool(DocumentsWriterPerThread docWriter) { this.docWriter = docWriter; } public void reset() { if (bufferUpto != -1) { - if (bufferUpto > 0) - // Recycle all but the first buffer - docWriter.recycleIntBlocks(buffers, 1, 1+bufferUpto); - // Reuse first buffer + if (bufferUpto > 0) { + docWriter.recycleIntBlocks(buffers, 1, bufferUpto-1); + Arrays.fill(buffers, 1, bufferUpto, null); + } bufferUpto = 0; intUpto = 0; intOffset = 0; @@ -57,7 +59,7 @@ bufferUpto++; intUpto = 0; - intOffset += DocumentsWriter.INT_BLOCK_SIZE; + intOffset += DocumentsWriterPerThread.INT_BLOCK_SIZE; } } --- lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java 2011-03-22 19:30:26.329100800 +0100 +++ lucene/src/java/org/apache/lucene/index/InvertedDocConsumer.java 2011-04-29 15:01:04.934257200 +0200 @@ -17,20 +17,22 @@ * limitations under the License. */ -import java.util.Collection; -import java.util.Map; import java.io.IOException; +import java.util.Map; abstract class InvertedDocConsumer { - /** Add a new thread */ - abstract InvertedDocConsumerPerThread addThread(DocInverterPerThread docInverterPerThread); - /** Abort (called after hitting AbortException) */ abstract void abort(); /** Flush a new segment */ - abstract void flush(Map> threadsAndFields, SegmentWriteState state) throws IOException; + abstract void flush(Map fieldsToFlush, SegmentWriteState state) throws IOException; + + abstract InvertedDocConsumerPerField addField(DocInverterPerField docInverterPerField, FieldInfo fieldInfo); + + abstract void startDocument() throws IOException; + + abstract void finishDocument() throws IOException; /** Attempt to free RAM, returning true if any RAM was * freed */ --- lucene/src/java/org/apache/lucene/index/InvertedDocConsumerPerThread.java 2011-01-26 17:07:33.466216100 +0100 +++ lucene/src/java/org/apache/lucene/index/InvertedDocConsumerPerThread.java 1970-01-01 01:00:00.000000000 +0100 @@ -1,27 +0,0 @@ -package org.apache.lucene.index; - -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import java.io.IOException; - -abstract class InvertedDocConsumerPerThread { - abstract void startDocument() throws IOException; - abstract InvertedDocConsumerPerField addField(DocInverterPerField docInverterPerField, FieldInfo fieldInfo); - abstract DocumentsWriter.DocWriter finishDocument() throws IOException; - abstract void abort(); -} --- lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java 2011-03-22 19:30:26.371103200 +0100 +++ lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumer.java 2011-04-29 15:01:08.256447200 +0200 @@ -17,12 +17,13 @@ * limitations under the License. */ -import java.util.Collection; -import java.util.Map; import java.io.IOException; +import java.util.Map; abstract class InvertedDocEndConsumer { - abstract InvertedDocEndConsumerPerThread addThread(DocInverterPerThread docInverterPerThread); - abstract void flush(Map> threadsAndFields, SegmentWriteState state) throws IOException; + abstract void flush(Map fieldsToFlush, SegmentWriteState state) throws IOException; abstract void abort(); + abstract InvertedDocEndConsumerPerField addField(DocInverterPerField docInverterPerField, FieldInfo fieldInfo); + abstract void startDocument() throws IOException; + abstract void finishDocument() throws IOException; } --- lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumerPerThread.java 2011-01-26 17:07:33.238203000 +0100 +++ lucene/src/java/org/apache/lucene/index/InvertedDocEndConsumerPerThread.java 1970-01-01 01:00:00.000000000 +0100 @@ -1,25 +0,0 @@ -package org.apache.lucene.index; - -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -abstract class InvertedDocEndConsumerPerThread { - abstract void startDocument(); - abstract InvertedDocEndConsumerPerField addField(DocInverterPerField docInverterPerField, FieldInfo fieldInfo); - abstract void finishDocument(); - abstract void abort(); -} --- lucene/src/java/org/apache/lucene/index/NormsWriter.java 2011-03-22 19:30:26.441107200 +0100 +++ lucene/src/java/org/apache/lucene/index/NormsWriter.java 2011-04-29 15:01:33.482890100 +0200 @@ -19,11 +19,7 @@ import java.io.IOException; import java.util.Collection; -import java.util.Iterator; -import java.util.HashMap; import java.util.Map; -import java.util.List; -import java.util.ArrayList; import org.apache.lucene.store.IndexOutput; @@ -36,10 +32,6 @@ final class NormsWriter extends InvertedDocEndConsumer { - @Override - public InvertedDocEndConsumerPerThread addThread(DocInverterPerThread docInverterPerThread) { - return new NormsWriterPerThread(docInverterPerThread, this); - } @Override public void abort() {} @@ -50,40 +42,11 @@ /** Produce _X.nrm if any document had a field with norms * not disabled */ @Override - public void flush(Map> threadsAndFields, SegmentWriteState state) throws IOException { - - final Map> byField = new HashMap>(); - + public void flush(Map fieldsToFlush, SegmentWriteState state) throws IOException { if (!state.fieldInfos.hasNorms()) { return; } - // Typically, each thread will have encountered the same - // field. So first we collate by field, ie, all - // per-thread field instances that correspond to the - // same FieldInfo - for (final Map.Entry> entry : threadsAndFields.entrySet()) { - final Collection fields = entry.getValue(); - final Iterator fieldsIt = fields.iterator(); - - while (fieldsIt.hasNext()) { - final NormsWriterPerField perField = (NormsWriterPerField) fieldsIt.next(); - - if (perField.upto > 0) { - // It has some norms - List l = byField.get(perField.fieldInfo); - if (l == null) { - l = new ArrayList(); - byField.put(perField.fieldInfo, l); - } - l.add(perField); - } else - // Remove this field since we haven't seen it - // since the previous flush - fieldsIt.remove(); - } - } - final String normsFileName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.NORMS_EXTENSION); IndexOutput normsOut = state.directory.createOutput(normsFileName); @@ -93,60 +56,25 @@ int normCount = 0; for (FieldInfo fi : state.fieldInfos) { - final List toMerge = byField.get(fi); + final NormsWriterPerField toWrite = (NormsWriterPerField) fieldsToFlush.get(fi); int upto = 0; - if (toMerge != null) { - - final int numFields = toMerge.size(); - + if (toWrite != null && toWrite.upto > 0) { normCount++; - final NormsWriterPerField[] fields = new NormsWriterPerField[numFields]; - int[] uptos = new int[numFields]; - - for(int j=0;j 0) { - - assert uptos[0] < fields[0].docIDs.length : " uptos[0]=" + uptos[0] + " len=" + (fields[0].docIDs.length); - - int minLoc = 0; - int minDocID = fields[0].docIDs[uptos[0]]; - - for(int j=1;j { - final NormsWriterPerThread perThread; final FieldInfo fieldInfo; - final DocumentsWriter.DocState docState; + final DocumentsWriterPerThread.DocState docState; final Similarity similarity; // Holds all docID/norm pairs we've seen @@ -46,10 +45,9 @@ upto = 0; } - public NormsWriterPerField(final DocInverterPerField docInverterPerField, final NormsWriterPerThread perThread, final FieldInfo fieldInfo) { - this.perThread = perThread; + public NormsWriterPerField(final DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) { this.fieldInfo = fieldInfo; - docState = perThread.docState; + docState = docInverterPerField.docState; fieldState = docInverterPerField.fieldState; similarity = docState.similarityProvider.get(fieldInfo.name); } --- lucene/src/java/org/apache/lucene/index/NormsWriterPerThread.java 2011-01-26 17:07:32.892183200 +0100 +++ lucene/src/java/org/apache/lucene/index/NormsWriterPerThread.java 1970-01-01 01:00:00.000000000 +0100 @@ -1,45 +0,0 @@ -package org.apache.lucene.index; - -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -final class NormsWriterPerThread extends InvertedDocEndConsumerPerThread { - final NormsWriter normsWriter; - final DocumentsWriter.DocState docState; - - public NormsWriterPerThread(DocInverterPerThread docInverterPerThread, NormsWriter normsWriter) { - this.normsWriter = normsWriter; - docState = docInverterPerThread.docState; - } - - @Override - InvertedDocEndConsumerPerField addField(DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) { - return new NormsWriterPerField(docInverterPerField, this, fieldInfo); - } - - @Override - void abort() {} - - @Override - void startDocument() {} - @Override - void finishDocument() {} - - boolean freeRAM() { - return false; - } -} --- lucene/src/java/org/apache/lucene/index/SegmentInfo.java 2011-04-03 19:24:37.611744000 +0200 +++ lucene/src/java/org/apache/lucene/index/SegmentInfo.java 2011-04-29 15:01:03.544177700 +0200 @@ -73,10 +73,13 @@ private volatile long sizeInBytesNoStore = -1; // total byte size of all but the store files (computed on demand) private volatile long sizeInBytesWithStore = -1; // total byte size of all of our files (computed on demand) + //TODO: LUCENE-2555: remove once we don't need to support shared doc stores (pre 4.0) private int docStoreOffset; // if this segment shares stored fields & vectors, this // offset is where in that file this segment's docs begin + //TODO: LUCENE-2555: remove once we don't need to support shared doc stores (pre 4.0) private String docStoreSegment; // name used to derive fields/vectors file we share with // other segments + //TODO: LUCENE-2555: remove once we don't need to support shared doc stores (pre 4.0) private boolean docStoreIsCompoundFile; // whether doc store files are stored in compound file (*.cfx) private int delCount; // How many deleted docs in this segment @@ -182,11 +185,13 @@ docStoreSegment = name; docStoreIsCompoundFile = false; } + if (format > DefaultSegmentInfosWriter.FORMAT_4_0) { // pre-4.0 indexes write a byte if there is a single norms file byte b = input.readByte(); assert 1 == b; } + int numNormGen = input.readInt(); if (numNormGen == NO) { normGen = null; @@ -465,37 +470,72 @@ assert delCount <= docCount; } + /** + * @deprecated shared doc stores are not supported in >= 4.0 + */ + @Deprecated public int getDocStoreOffset() { + // TODO: LUCENE-2555: remove once we don't need to support shared doc stores (pre 4.0) return docStoreOffset; } + /** + * @deprecated shared doc stores are not supported in >= 4.0 + */ + @Deprecated public boolean getDocStoreIsCompoundFile() { + // TODO: LUCENE-2555: remove once we don't need to support shared doc stores (pre 4.0) return docStoreIsCompoundFile; } - void setDocStoreIsCompoundFile(boolean v) { - docStoreIsCompoundFile = v; + /** + * @deprecated shared doc stores are not supported in >= 4.0 + */ + @Deprecated + public void setDocStoreIsCompoundFile(boolean docStoreIsCompoundFile) { + // TODO: LUCENE-2555: remove once we don't need to support shared doc stores (pre 4.0) + this.docStoreIsCompoundFile = docStoreIsCompoundFile; clearFilesCache(); } - public String getDocStoreSegment() { - return docStoreSegment; + /** + * @deprecated shared doc stores are not supported in >= 4.0 + */ + @Deprecated + void setDocStore(int offset, String segment, boolean isCompoundFile) { + // TODO: LUCENE-2555: remove once we don't need to support shared doc stores (pre 4.0) + docStoreOffset = offset; + docStoreSegment = segment; + docStoreIsCompoundFile = isCompoundFile; + clearFilesCache(); } - public void setDocStoreSegment(String segment) { - docStoreSegment = segment; + /** + * @deprecated shared doc stores are not supported in >= 4.0 + */ + @Deprecated + public String getDocStoreSegment() { + // TODO: LUCENE-2555: remove once we don't need to support shared doc stores (pre 4.0) + return docStoreSegment; } + /** + * @deprecated shared doc stores are not supported in >= 4.0 + */ + @Deprecated void setDocStoreOffset(int offset) { + // TODO: LUCENE-2555: remove once we don't need to support shared doc stores (pre 4.0) docStoreOffset = offset; clearFilesCache(); } - void setDocStore(int offset, String segment, boolean isCompoundFile) { - docStoreOffset = offset; - docStoreSegment = segment; - docStoreIsCompoundFile = isCompoundFile; - clearFilesCache(); + /** + * @deprecated shared doc stores are not supported in 4.0 + */ + @Deprecated + public void setDocStoreSegment(String docStoreSegment) { + // TODO: LUCENE-2555: remove once we don't need to support shared doc stores (pre 4.0) + this.docStoreSegment = docStoreSegment; } /** Save this segment's info. */ @@ -507,12 +547,14 @@ output.writeString(name); output.writeInt(docCount); output.writeLong(delGen); + output.writeInt(docStoreOffset); if (docStoreOffset != -1) { output.writeString(docStoreSegment); output.writeByte((byte) (docStoreIsCompoundFile ? 1:0)); } + if (normGen == null) { output.writeInt(NO); } else { --- lucene/src/java/org/apache/lucene/index/SegmentMerger.java 2011-03-26 11:40:50.227427500 +0100 +++ lucene/src/java/org/apache/lucene/index/SegmentMerger.java 2011-04-29 15:01:01.928085200 +0200 @@ -202,7 +202,6 @@ * @throws IOException if there is a low-level IO error */ private int mergeFields() throws CorruptIndexException, IOException { - for (IndexReader reader : readers) { if (reader instanceof SegmentReader) { SegmentReader segmentReader = (SegmentReader) reader; --- lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java 2011-03-22 19:30:26.393104500 +0100 +++ lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java 2011-04-29 15:01:10.411570500 +0200 @@ -18,7 +18,8 @@ */ import java.io.IOException; -import org.apache.lucene.store.RAMOutputStream; + +import org.apache.lucene.document.Fieldable; import org.apache.lucene.util.ArrayUtil; import org.apache.lucene.util.RamUsageEstimator; @@ -26,22 +27,38 @@ final class StoredFieldsWriter { FieldsWriter fieldsWriter; - final DocumentsWriter docWriter; + final DocumentsWriterPerThread docWriter; int lastDocID; - PerDoc[] docFreeList = new PerDoc[1]; int freeCount; - public StoredFieldsWriter(DocumentsWriter docWriter) { + final DocumentsWriterPerThread.DocState docState; + + public StoredFieldsWriter(DocumentsWriterPerThread docWriter) { this.docWriter = docWriter; + this.docState = docWriter.docState; + } + + private int numStoredFields; + private Fieldable[] storedFields; + private int[] fieldNumbers; + + public void reset() { + numStoredFields = 0; + storedFields = new Fieldable[1]; + fieldNumbers = new int[1]; } - public StoredFieldsWriterPerThread addThread(DocumentsWriter.DocState docState) throws IOException { - return new StoredFieldsWriterPerThread(docState, this); + public void startDocument() { + reset(); } - synchronized public void flush(SegmentWriteState state) throws IOException { - if (state.numDocs > lastDocID) { + public void flush(SegmentWriteState state) throws IOException { + + if (state.numDocs > 0) { + // It's possible that all documents seen in this segment + // hit non-aborting exceptions, in which case we will + // not have yet init'd the FieldsWriter: initFieldsWriter(); fill(state.numDocs); } @@ -67,23 +84,9 @@ int allocCount; - synchronized PerDoc getPerDoc() { - if (freeCount == 0) { - allocCount++; - if (allocCount > docFreeList.length) { - // Grow our free list up front to make sure we have - // enough space to recycle all outstanding PerDoc - // instances - assert allocCount == 1+docFreeList.length; - docFreeList = new PerDoc[ArrayUtil.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)]; - } - return new PerDoc(); - } else { - return docFreeList[--freeCount]; - } - } + void abort() { + reset(); - synchronized void abort() { if (fieldsWriter != null) { fieldsWriter.abort(); fieldsWriter = null; @@ -101,53 +104,40 @@ } } - synchronized void finishDocument(PerDoc perDoc) throws IOException { + void finishDocument() throws IOException { assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument start"); - initFieldsWriter(); - fill(perDoc.docID); + initFieldsWriter(); + fill(docState.docID); - // Append stored fields to the real FieldsWriter: - fieldsWriter.flushDocument(perDoc.numStoredFields, perDoc.fdt); - lastDocID++; - perDoc.reset(); - free(perDoc); - assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument end"); + if (fieldsWriter != null && numStoredFields > 0) { + fieldsWriter.startDocument(numStoredFields); + for (int i = 0; i < numStoredFields; i++) { + fieldsWriter.writeField(fieldNumbers[i], storedFields[i]); } - - synchronized void free(PerDoc perDoc) { - assert freeCount < docFreeList.length; - assert 0 == perDoc.numStoredFields; - assert 0 == perDoc.fdt.length(); - assert 0 == perDoc.fdt.getFilePointer(); - docFreeList[freeCount++] = perDoc; - } - - class PerDoc extends DocumentsWriter.DocWriter { - final DocumentsWriter.PerDocBuffer buffer = docWriter.newPerDocBuffer(); - RAMOutputStream fdt = new RAMOutputStream(buffer); - int numStoredFields; - - void reset() { - fdt.reset(); - buffer.recycle(); - numStoredFields = 0; + lastDocID++; } - @Override - void abort() { reset(); - free(this); + assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument end"); } - @Override - public long sizeInBytes() { - return buffer.getSizeInBytes(); + public void addField(Fieldable field, FieldInfo fieldInfo) throws IOException { + if (numStoredFields == storedFields.length) { + int newSize = ArrayUtil.oversize(numStoredFields + 1, RamUsageEstimator.NUM_BYTES_OBJECT_REF); + Fieldable[] newArray = new Fieldable[newSize]; + System.arraycopy(storedFields, 0, newArray, 0, numStoredFields); + storedFields = newArray; } - @Override - public void finish() throws IOException { - finishDocument(this); + if (numStoredFields == fieldNumbers.length) { + fieldNumbers = ArrayUtil.grow(fieldNumbers); } + + storedFields[numStoredFields] = field; + fieldNumbers[numStoredFields] = fieldInfo.number; + numStoredFields++; + + assert docState.testPoint("StoredFieldsWriterPerThread.processFields.writeField"); } } --- lucene/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java 2011-03-22 19:30:26.386104100 +0100 +++ lucene/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java 1970-01-01 01:00:00.000000000 +0100 @@ -1,79 +0,0 @@ -package org.apache.lucene.index; - -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import java.io.IOException; -import org.apache.lucene.store.IndexOutput; -import org.apache.lucene.document.Fieldable; - -final class StoredFieldsWriterPerThread { - - final FieldsWriter localFieldsWriter; - final StoredFieldsWriter storedFieldsWriter; - final DocumentsWriter.DocState docState; - - StoredFieldsWriter.PerDoc doc; - - public StoredFieldsWriterPerThread(DocumentsWriter.DocState docState, StoredFieldsWriter storedFieldsWriter) throws IOException { - this.storedFieldsWriter = storedFieldsWriter; - this.docState = docState; - localFieldsWriter = new FieldsWriter((IndexOutput) null, (IndexOutput) null); - } - - public void startDocument() { - if (doc != null) { - // Only happens if previous document hit non-aborting - // exception while writing stored fields into - // localFieldsWriter: - doc.reset(); - doc.docID = docState.docID; - } - } - - public void addField(Fieldable field, FieldInfo fieldInfo) throws IOException { - if (doc == null) { - doc = storedFieldsWriter.getPerDoc(); - doc.docID = docState.docID; - localFieldsWriter.setFieldsStream(doc.fdt); - assert doc.numStoredFields == 0: "doc.numStoredFields=" + doc.numStoredFields; - assert 0 == doc.fdt.length(); - assert 0 == doc.fdt.getFilePointer(); - } - - localFieldsWriter.writeField(fieldInfo, field); - assert docState.testPoint("StoredFieldsWriterPerThread.processFields.writeField"); - doc.numStoredFields++; - } - - public DocumentsWriter.DocWriter finishDocument() { - // If there were any stored fields in this doc, doc will - // be non-null; else it's null. - try { - return doc; - } finally { - doc = null; - } - } - - public void abort() { - if (doc != null) { - doc.abort(); - doc = null; - } - } -} --- lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java 2011-01-26 17:07:32.725173700 +0100 +++ lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java 2011-04-29 15:01:10.751589900 +0200 @@ -17,49 +17,48 @@ * limitations under the License. */ +import java.io.IOException; +import java.util.Map; + import org.apache.lucene.store.IndexOutput; -import org.apache.lucene.store.RAMOutputStream; import org.apache.lucene.util.ArrayUtil; +import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.IOUtils; import org.apache.lucene.util.RamUsageEstimator; -import java.io.IOException; -import java.util.Collection; - -import java.util.Map; - final class TermVectorsTermsWriter extends TermsHashConsumer { - final DocumentsWriter docWriter; - PerDoc[] docFreeList = new PerDoc[1]; + final DocumentsWriterPerThread docWriter; int freeCount; IndexOutput tvx; IndexOutput tvd; IndexOutput tvf; int lastDocID; + + final DocumentsWriterPerThread.DocState docState; + final BytesRef flushTerm = new BytesRef(); + + // Used by perField when serializing the term vectors + final ByteSliceReader vectorSliceReader = new ByteSliceReader(); boolean hasVectors; - public TermVectorsTermsWriter(DocumentsWriter docWriter) { + public TermVectorsTermsWriter(DocumentsWriterPerThread docWriter) { this.docWriter = docWriter; + docState = docWriter.docState; } @Override - public TermsHashConsumerPerThread addThread(TermsHashPerThread termsHashPerThread) { - return new TermVectorsTermsWriterPerThread(termsHashPerThread, this); - } - - @Override - synchronized void flush(Map> threadsAndFields, final SegmentWriteState state) throws IOException { + void flush(Map fieldsToFlush, final SegmentWriteState state) throws IOException { if (tvx != null) { // At least one doc in this run had term vectors enabled fill(state.numDocs); + assert state.segmentName != null; + String idxName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_INDEX_EXTENSION); tvx.close(); tvf.close(); tvd.close(); tvx = tvd = tvf = null; - assert state.segmentName != null; - String idxName = IndexFileNames.segmentFileName(state.segmentName, "", IndexFileNames.VECTORS_INDEX_EXTENSION); - if (4 + ((long) state.numDocs) * 16 != state.directory.fileLength(idxName)) { + if (4+((long) state.numDocs)*16 != state.directory.fileLength(idxName)) { throw new RuntimeException("after flush: tvx size mismatch: " + state.numDocs + " docs vs " + state.directory.fileLength(idxName) + " length in bytes of " + idxName + " file exists?=" + state.directory.fileExists(idxName)); } @@ -68,34 +67,11 @@ hasVectors = false; } - for (Map.Entry> entry : threadsAndFields.entrySet()) { - for (final TermsHashConsumerPerField field : entry.getValue() ) { + for (final TermsHashConsumerPerField field : fieldsToFlush.values() ) { TermVectorsTermsWriterPerField perField = (TermVectorsTermsWriterPerField) field; perField.termsHashPerField.reset(); perField.shrinkHash(); } - - TermVectorsTermsWriterPerThread perThread = (TermVectorsTermsWriterPerThread) entry.getKey(); - perThread.termsHashPerThread.reset(true); - } - } - - int allocCount; - - synchronized PerDoc getPerDoc() { - if (freeCount == 0) { - allocCount++; - if (allocCount > docFreeList.length) { - // Grow our free list up front to make sure we have - // enough space to recycle all outstanding PerDoc - // instances - assert allocCount == 1+docFreeList.length; - docFreeList = new PerDoc[ArrayUtil.oversize(allocCount, RamUsageEstimator.NUM_BYTES_OBJECT_REF)]; - } - return new PerDoc(); - } else { - return docFreeList[--freeCount]; - } } /** Fills in no-term-vectors for all docs we haven't seen @@ -112,14 +88,13 @@ } } - synchronized void initTermVectorsWriter() throws IOException { + private final void initTermVectorsWriter() throws IOException { if (tvx == null) { // If we hit an exception while init'ing the term // vector output files, we must abort this segment // because those files will be in an unknown // state: - hasVectors = true; tvx = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_INDEX_EXTENSION)); tvd = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION)); tvf = docWriter.directory.createOutput(IndexFileNames.segmentFileName(docWriter.getSegment(), "", IndexFileNames.VECTORS_FIELDS_EXTENSION)); @@ -132,39 +107,44 @@ } } - synchronized void finishDocument(PerDoc perDoc) throws IOException { + @Override + void finishDocument(TermsHash termsHash) throws IOException { assert docWriter.writer.testPoint("TermVectorsTermsWriter.finishDocument start"); + if (!hasVectors) { + return; + } + initTermVectorsWriter(); - fill(perDoc.docID); + fill(docState.docID); // Append term vectors to the real outputs: - tvx.writeLong(tvd.getFilePointer()); + long pointer = tvd.getFilePointer(); + tvx.writeLong(pointer); tvx.writeLong(tvf.getFilePointer()); - tvd.writeVInt(perDoc.numVectorFields); - if (perDoc.numVectorFields > 0) { - for(int i=0;i 0) { + for(int i=0;i= 0; - if (!doVectors || numPostings == 0) - return; - if (numPostings > maxNumPostings) maxNumPostings = numPostings; - final IndexOutput tvf = perThread.doc.perDocTvf; - // This is called once, after inverting all occurrences // of a given field in the doc. At this point we flush // our hash into the DocWriter. assert fieldInfo.storeTermVector; - assert perThread.vectorFieldsInOrder(fieldInfo); + assert termsWriter.vectorFieldsInOrder(fieldInfo); - perThread.doc.addField(termsHashPerField.fieldInfo.number); TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray; + final IndexOutput tvf = termsWriter.tvf; // TODO: we may want to make this sort in same order // as Codec's terms dict? @@ -148,8 +136,8 @@ byte[] lastBytes = null; int lastStart = 0; - final ByteSliceReader reader = perThread.vectorSliceReader; - final ByteBlockPool termBytePool = perThread.termsHashPerThread.termBytePool; + final ByteSliceReader reader = termsWriter.vectorSliceReader; + final ByteBlockPool termBytePool = termsHashPerField.termBytePool; for(int j=0;j> threadsAndFields, final SegmentWriteState state) throws IOException { - Map> childThreadsAndFields = new HashMap>(); - Map> nextThreadsAndFields; - - if (nextTermsHash != null) - nextThreadsAndFields = new HashMap>(); - else - nextThreadsAndFields = null; - - for (final Map.Entry> entry : threadsAndFields.entrySet()) { + // Clear all state + void reset() { + intPool.reset(); + bytePool.reset(); - TermsHashPerThread perThread = (TermsHashPerThread) entry.getKey(); + if (primary) { + bytePool.reset(); + } + } - Collection fields = entry.getValue(); + @Override + void flush(Map fieldsToFlush, final SegmentWriteState state) throws IOException { + Map childFields = new HashMap(); + Map nextChildFields; + + if (nextTermsHash != null) { + nextChildFields = new HashMap(); + } else { + nextChildFields = null; + } - Iterator fieldsIt = fields.iterator(); - Collection childFields = new HashSet(); - Collection nextChildFields; + for (final Map.Entry entry : fieldsToFlush.entrySet()) { + TermsHashPerField perField = (TermsHashPerField) entry.getValue(); + childFields.put(entry.getKey(), perField.consumer); + if (nextTermsHash != null) { + nextChildFields.put(entry.getKey(), perField.nextPerField); + } + } - if (nextTermsHash != null) - nextChildFields = new HashSet(); - else - nextChildFields = null; + consumer.flush(childFields, state); - while(fieldsIt.hasNext()) { - TermsHashPerField perField = (TermsHashPerField) fieldsIt.next(); - childFields.add(perField.consumer); - if (nextTermsHash != null) - nextChildFields.add(perField.nextPerField); + if (nextTermsHash != null) { + nextTermsHash.flush(nextChildFields, state); + } } - childThreadsAndFields.put(perThread.consumer, childFields); - if (nextTermsHash != null) - nextThreadsAndFields.put(perThread.nextPerThread, nextChildFields); + @Override + InvertedDocConsumerPerField addField(DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) { + return new TermsHashPerField(docInverterPerField, this, nextTermsHash, fieldInfo); } - consumer.flush(childThreadsAndFields, state); + @Override + public boolean freeRAM() { + return false; + } - if (nextTermsHash != null) - nextTermsHash.flush(nextThreadsAndFields, state); + @Override + void finishDocument() throws IOException { + try { + consumer.finishDocument(this); + } finally { + if (nextTermsHash != null) { + nextTermsHash.consumer.finishDocument(nextTermsHash); + } + } } @Override - synchronized public boolean freeRAM() { - return false; + void startDocument() throws IOException { + consumer.startDocument(); + if (nextTermsHash != null) { + nextTermsHash.consumer.startDocument(); + } } } --- lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java 2011-03-22 19:30:26.380103700 +0100 +++ lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java 2011-04-29 15:01:08.425456900 +0200 @@ -18,11 +18,12 @@ */ import java.io.IOException; -import java.util.Collection; import java.util.Map; abstract class TermsHashConsumer { - abstract TermsHashConsumerPerThread addThread(TermsHashPerThread perThread); - abstract void flush(Map> threadsAndFields, final SegmentWriteState state) throws IOException; + abstract void flush(Map fieldsToFlush, final SegmentWriteState state) throws IOException; abstract void abort(); - } + abstract void startDocument() throws IOException; + abstract void finishDocument(TermsHash termsHash) throws IOException; + abstract public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo); +} --- lucene/src/java/org/apache/lucene/index/TermsHashConsumerPerThread.java 2011-01-26 17:07:31.569107500 +0100 +++ lucene/src/java/org/apache/lucene/index/TermsHashConsumerPerThread.java 1970-01-01 01:00:00.000000000 +0100 @@ -1,27 +0,0 @@ -package org.apache.lucene.index; - -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import java.io.IOException; - -abstract class TermsHashConsumerPerThread { - abstract void startDocument() throws IOException; - abstract DocumentsWriter.DocWriter finishDocument() throws IOException; - abstract public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo); - abstract public void abort(); -} --- lucene/src/java/org/apache/lucene/index/TermsHashPerField.java 2011-03-22 08:16:51.300516600 +0100 +++ lucene/src/java/org/apache/lucene/index/TermsHashPerField.java 2011-04-29 15:01:02.978145300 +0200 @@ -34,9 +34,10 @@ final TermsHashConsumerPerField consumer; + final TermsHash termsHash; + final TermsHashPerField nextPerField; - final TermsHashPerThread perThread; - final DocumentsWriter.DocState docState; + final DocumentsWriterPerThread.DocState docState; final FieldInvertState fieldState; TermToBytesRefAttribute termAtt; BytesRef termBytesRef; @@ -56,23 +57,23 @@ ParallelPostingsArray postingsArray; private final AtomicLong bytesUsed; - public TermsHashPerField(DocInverterPerField docInverterPerField, final TermsHashPerThread perThread, final TermsHashPerThread nextPerThread, final FieldInfo fieldInfo) { - this.perThread = perThread; - intPool = perThread.intPool; - bytePool = perThread.bytePool; - termBytePool = perThread.termBytePool; - docState = perThread.docState; - bytesUsed = perThread.termsHash.trackAllocations?perThread.termsHash.docWriter.bytesUsed:new AtomicLong(); - + public TermsHashPerField(DocInverterPerField docInverterPerField, final TermsHash termsHash, final TermsHash nextTermsHash, final FieldInfo fieldInfo) { + intPool = termsHash.intPool; + bytePool = termsHash.bytePool; + termBytePool = termsHash.termBytePool; + docState = termsHash.docState; + this.termsHash = termsHash; + bytesUsed = termsHash.trackAllocations ? termsHash.docWriter.bytesUsed + : new AtomicLong(); fieldState = docInverterPerField.fieldState; - this.consumer = perThread.consumer.addField(this, fieldInfo); + this.consumer = termsHash.consumer.addField(this, fieldInfo); PostingsBytesStartArray byteStarts = new PostingsBytesStartArray(this, bytesUsed); bytesHash = new BytesRefHash(termBytePool, HASH_INIT_SIZE, byteStarts); streamCount = consumer.getStreamCount(); numPostingInt = 2*streamCount; this.fieldInfo = fieldInfo; - if (nextPerThread != null) - nextPerField = (TermsHashPerField) nextPerThread.addField(docInverterPerField, fieldInfo); + if (nextTermsHash != null) + nextPerField = (TermsHashPerField) nextTermsHash.addField(docInverterPerField, fieldInfo); else nextPerField = null; } @@ -90,7 +91,7 @@ } @Override - synchronized public void abort() { + public void abort() { reset(); if (nextPerField != null) nextPerField.abort(); @@ -99,14 +100,13 @@ public void initReader(ByteSliceReader reader, int termID, int stream) { assert stream < streamCount; int intStart = postingsArray.intStarts[termID]; - final int[] ints = intPool.buffers[intStart >> DocumentsWriter.INT_BLOCK_SHIFT]; - final int upto = intStart & DocumentsWriter.INT_BLOCK_MASK; + final int[] ints = intPool.buffers[intStart >> DocumentsWriterPerThread.INT_BLOCK_SHIFT]; + final int upto = intStart & DocumentsWriterPerThread.INT_BLOCK_MASK; reader.init(bytePool, postingsArray.byteStarts[termID]+stream*ByteBlockPool.FIRST_LEVEL_SIZE, ints[upto+stream]); } - /** Collapse the hash table & sort in-place. */ public int[] sortPostings(Comparator termComp) { return bytesHash.sort(termComp); @@ -143,11 +143,12 @@ // First time we are seeing this token since we last // flushed the hash. // Init stream slices - if (numPostingInt + intPool.intUpto > DocumentsWriter.INT_BLOCK_SIZE) + if (numPostingInt + intPool.intUpto > DocumentsWriterPerThread.INT_BLOCK_SIZE) intPool.nextBuffer(); - if (ByteBlockPool.BYTE_BLOCK_SIZE - bytePool.byteUpto < numPostingInt*ByteBlockPool.FIRST_LEVEL_SIZE) + if (ByteBlockPool.BYTE_BLOCK_SIZE - bytePool.byteUpto < numPostingInt*ByteBlockPool.FIRST_LEVEL_SIZE) { bytePool.nextBuffer(); + } intUptos = intPool.buffer; intUptoStart = intPool.intUpto; @@ -166,8 +167,8 @@ } else { termID = (-termID)-1; int intStart = postingsArray.intStarts[termID]; - intUptos = intPool.buffers[intStart >> DocumentsWriter.INT_BLOCK_SHIFT]; - intUptoStart = intStart & DocumentsWriter.INT_BLOCK_MASK; + intUptos = intPool.buffers[intStart >> DocumentsWriterPerThread.INT_BLOCK_SHIFT]; + intUptoStart = intStart & DocumentsWriterPerThread.INT_BLOCK_MASK; consumer.addTerm(termID); } } @@ -192,7 +193,7 @@ if (docState.maxTermPrefix == null) { final int saved = termBytesRef.length; try { - termBytesRef.length = Math.min(30, DocumentsWriter.MAX_TERM_LENGTH_UTF8); + termBytesRef.length = Math.min(30, DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8); docState.maxTermPrefix = termBytesRef.toString(); } finally { termBytesRef.length = saved; @@ -204,7 +205,7 @@ if (termID >= 0) {// New posting bytesHash.byteStart(termID); // Init stream slices - if (numPostingInt + intPool.intUpto > DocumentsWriter.INT_BLOCK_SIZE) { + if (numPostingInt + intPool.intUpto > DocumentsWriterPerThread.INT_BLOCK_SIZE) { intPool.nextBuffer(); } @@ -229,8 +230,8 @@ } else { termID = (-termID)-1; final int intStart = postingsArray.intStarts[termID]; - intUptos = intPool.buffers[intStart >> DocumentsWriter.INT_BLOCK_SHIFT]; - intUptoStart = intStart & DocumentsWriter.INT_BLOCK_MASK; + intUptos = intPool.buffers[intStart >> DocumentsWriterPerThread.INT_BLOCK_SHIFT]; + intUptoStart = intStart & DocumentsWriterPerThread.INT_BLOCK_MASK; consumer.addTerm(termID); } @@ -312,7 +313,7 @@ @Override public int[] clear() { if(perField.postingsArray != null) { - bytesUsed.addAndGet(-perField.postingsArray.size * perField.postingsArray.bytesPerPosting()); + bytesUsed.addAndGet(-(perField.postingsArray.size * perField.postingsArray.bytesPerPosting())); perField.postingsArray = null; } return null; --- lucene/src/java/org/apache/lucene/index/TermsHashPerThread.java 2011-03-22 08:16:51.314517400 +0100 +++ lucene/src/java/org/apache/lucene/index/TermsHashPerThread.java 1970-01-01 01:00:00.000000000 +0100 @@ -1,96 +0,0 @@ -package org.apache.lucene.index; - -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -import org.apache.lucene.util.ByteBlockPool; - -import java.io.IOException; - -final class TermsHashPerThread extends InvertedDocConsumerPerThread { - - final TermsHash termsHash; - final TermsHashConsumerPerThread consumer; - final TermsHashPerThread nextPerThread; // the secondary is currently consumed by TermVectorsWriter - // see secondary entry point in TermsHashPerField#add(int) - - final IntBlockPool intPool; - final ByteBlockPool bytePool; - final ByteBlockPool termBytePool; - - final boolean primary; - final DocumentsWriter.DocState docState; - - public TermsHashPerThread(DocInverterPerThread docInverterPerThread, final TermsHash termsHash, final TermsHash nextTermsHash, final TermsHashPerThread primaryPerThread) { - docState = docInverterPerThread.docState; - - this.termsHash = termsHash; - this.consumer = termsHash.consumer.addThread(this); - - intPool = new IntBlockPool(termsHash.docWriter); - bytePool = new ByteBlockPool(termsHash.docWriter.byteBlockAllocator); // use the allocator from the docWriter which tracks the used bytes - primary = nextTermsHash != null; - if (primary) { - // We are primary - termBytePool = bytePool; - nextPerThread = nextTermsHash.addThread(docInverterPerThread, this); // this will be the primaryPerThread in the secondary - assert nextPerThread != null; - } else { - assert primaryPerThread != null; - termBytePool = primaryPerThread.bytePool; // we are secondary and share the byte pool with the primary - nextPerThread = null; - } - } - - @Override - InvertedDocConsumerPerField addField(DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) { - return new TermsHashPerField(docInverterPerField, this, nextPerThread, fieldInfo); - } - - @Override - synchronized public void abort() { - reset(true); - consumer.abort(); - if (primary) - nextPerThread.abort(); - } - - @Override - public void startDocument() throws IOException { - consumer.startDocument(); - if (primary) - nextPerThread.consumer.startDocument(); - } - - @Override - public DocumentsWriter.DocWriter finishDocument() throws IOException { - final DocumentsWriter.DocWriter doc = consumer.finishDocument(); - final DocumentsWriter.DocWriter docFromSecondary = primary? nextPerThread.consumer.finishDocument():null; - if (doc == null) - return docFromSecondary; - else { - doc.setNext(docFromSecondary); - return doc; - } - } - - // Clear all state - void reset(boolean recyclePostings) { - intPool.reset(); - bytePool.reset(); - } -} --- lucene/src/java/org/apache/lucene/index/ThreadAffinityDocumentsWriterThreadPool.java 1970-01-01 01:00:00.000000000 +0100 +++ lucene/src/java/org/apache/lucene/index/ThreadAffinityDocumentsWriterThreadPool.java 2011-04-29 15:01:13.800764300 +0200 @@ -0,0 +1,96 @@ +package org.apache.lucene.index; +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +import org.apache.lucene.document.Document; + +/** + * A {@link DocumentsWriterPerThreadPool} implementation that tries to assign an + * indexing thread to the same {@link ThreadState} each time the thread tries to + * obtain a {@link ThreadState}. Once a new {@link ThreadState} is created it is + * associated with the creating thread. Subsequently, if the threads associated + * {@link ThreadState} is not in use it will be associated with the requesting + * thread. Otherwise, if the {@link ThreadState} is used by another thread + * {@link ThreadAffinityDocumentsWriterThreadPool} tries to find the currently + * minimal contended {@link ThreadState}. + */ +public class ThreadAffinityDocumentsWriterThreadPool extends DocumentsWriterPerThreadPool { + private Map threadBindings = new ConcurrentHashMap(); + + /** + * Creates a new {@link DocumentsWriterPerThreadPool} with max. + * {@link #DEFAULT_MAX_THREAD_STATES} thread states. + */ + public ThreadAffinityDocumentsWriterThreadPool() { + this(DEFAULT_MAX_THREAD_STATES); + } + + public ThreadAffinityDocumentsWriterThreadPool(int maxNumPerThreads) { + super(maxNumPerThreads); + assert getMaxThreadStates() >= 1; + } + + @Override + public ThreadState getAndLock(Thread requestingThread, DocumentsWriter documentsWriter, Document doc) { + ThreadState threadState = threadBindings.get(requestingThread); + if (threadState != null) { + if (threadState.tryLock()) { + return threadState; + } + } + ThreadState minThreadState = null; + + + /* TODO -- another thread could lock the minThreadState we just got while + we should somehow prevent this. */ + // Find the state that has minimum number of threads waiting + minThreadState = minContendedThreadState(); + if (minThreadState == null || minThreadState.hasQueuedThreads()) { + final ThreadState newState = newThreadState(); // state is already locked if non-null + if (newState != null) { + assert newState.isHeldByCurrentThread(); + threadBindings.put(requestingThread, newState); + return newState; + } else if (minThreadState == null) { + /* + * no new threadState available we just take the minContented one + * This must return a valid thread state since we accessed the + * synced context in newThreadState() above. + */ + minThreadState = minContendedThreadState(); + } + } + assert minThreadState != null: "ThreadState is null"; + + minThreadState.lock(); + return minThreadState; + } + + /* + @Override + public void clearThreadBindings(ThreadState perThread) { + threadBindings.clear(); + } + + @Override + public void clearAllThreadBindings() { + threadBindings.clear(); + } + */ +} --- lucene/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java 2011-04-14 09:38:41.310850900 +0200 +++ lucene/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java 2011-04-29 14:59:37.191238600 +0200 @@ -50,7 +50,7 @@ boolean isClose = false; StackTraceElement[] trace = new Exception().getStackTrace(); for (int i = 0; i < trace.length; i++) { - if ("doFlush".equals(trace[i].getMethodName())) { + if ("flush".equals(trace[i].getMethodName())) { isDoFlush = true; } if ("close".equals(trace[i].getMethodName())) { --- lucene/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java 1970-01-01 01:00:00.000000000 +0100 +++ lucene/src/test/org/apache/lucene/index/TestDocumentsWriterDeleteQueue.java 2011-04-29 14:59:50.957025900 +0200 @@ -0,0 +1,218 @@ +package org.apache.lucene.index; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to You under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.lucene.index.DocumentsWriterDeleteQueue.DeleteSlice; +import org.apache.lucene.search.TermQuery; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.ThreadInterruptedException; + +/** + * Unit test for {@link DocumentsWriterDeleteQueue} + */ +public class TestDocumentsWriterDeleteQueue extends LuceneTestCase { + + public void testUpdateDelteSlices() { + DocumentsWriterDeleteQueue queue = new DocumentsWriterDeleteQueue(); + final int size = 200 + random.nextInt(500) * RANDOM_MULTIPLIER; + Integer[] ids = new Integer[size]; + for (int i = 0; i < ids.length; i++) { + ids[i] = random.nextInt(); + } + Term template = new Term("id"); + DeleteSlice slice1 = queue.newSlice(); + DeleteSlice slice2 = queue.newSlice(); + BufferedDeletes bd1 = new BufferedDeletes(false); + BufferedDeletes bd2 = new BufferedDeletes(false); + int last1 = 0; + int last2 = 0; + Set uniqueValues = new HashSet(); + for (int j = 0; j < ids.length; j++) { + Integer i = ids[j]; + // create an array here since we compare identity below against tailItem + Term[] term = new Term[] {template.createTerm(i.toString())}; + uniqueValues.add(term[0]); + queue.addDelete(term); + if (random.nextInt(20) == 0 || j == ids.length - 1) { + queue.updateSlice(slice1); + assertTrue(slice1.isTailItem(term)); + slice1.apply(bd1, j); + assertAllBetween(last1, j, bd1, ids); + last1 = j + 1; + } + if (random.nextInt(10) == 5 || j == ids.length - 1) { + queue.updateSlice(slice2); + assertTrue(slice2.isTailItem(term)); + slice2.apply(bd2, j); + assertAllBetween(last2, j, bd2, ids); + last2 = j + 1; + } + assertEquals(uniqueValues.size(), queue.numGlobalTermDeletes()); + } + assertEquals(uniqueValues, bd1.terms.keySet()); + assertEquals(uniqueValues, bd2.terms.keySet()); + assertEquals(uniqueValues, new HashSet(Arrays.asList(queue + .freezeGlobalBuffer(null).terms))); + assertEquals("num deletes must be 0 after freeze", 0, queue + .numGlobalTermDeletes()); + } + + private void assertAllBetween(int start, int end, BufferedDeletes deletes, + Integer[] ids) { + Term template = new Term("id"); + for (int i = start; i <= end; i++) { + assertEquals(Integer.valueOf(end), deletes.terms.get(template + .createTerm(ids[i].toString()))); + } + } + + public void testClear() { + DocumentsWriterDeleteQueue queue = new DocumentsWriterDeleteQueue(); + Term template = new Term("id"); + assertFalse(queue.anyChanges()); + queue.clear(); + assertFalse(queue.anyChanges()); + final int size = 200 + random.nextInt(500) * RANDOM_MULTIPLIER; + int termsSinceFreeze = 0; + int queriesSinceFreeze = 0; + for (int i = 0; i < size; i++) { + Term term = template.createTerm("" + i); + if (random.nextInt(10) == 0) { + queue.addDelete(new TermQuery(term)); + queriesSinceFreeze++; + } else { + queue.addDelete(term); + termsSinceFreeze++; + } + assertTrue(queue.anyChanges()); + if (random.nextInt(10) == 0) { + queue.clear(); + queue.tryApplyGlobalSlice(); + assertFalse(queue.anyChanges()); + } + } + + } + + public void testAnyChanges() { + DocumentsWriterDeleteQueue queue = new DocumentsWriterDeleteQueue(); + Term template = new Term("id"); + final int size = 200 + random.nextInt(500) * RANDOM_MULTIPLIER; + int termsSinceFreeze = 0; + int queriesSinceFreeze = 0; + for (int i = 0; i < size; i++) { + Term term = template.createTerm("" + i); + if (random.nextInt(10) == 0) { + queue.addDelete(new TermQuery(term)); + queriesSinceFreeze++; + } else { + queue.addDelete(term); + termsSinceFreeze++; + } + assertTrue(queue.anyChanges()); + if (random.nextInt(5) == 0) { + FrozenBufferedDeletes freezeGlobalBuffer = queue + .freezeGlobalBuffer(null); + assertEquals(termsSinceFreeze, freezeGlobalBuffer.terms.length); + assertEquals(queriesSinceFreeze, freezeGlobalBuffer.queries.length); + queriesSinceFreeze = 0; + termsSinceFreeze = 0; + assertFalse(queue.anyChanges()); + } + } + } + + public void testStressDeleteQueue() throws InterruptedException { + DocumentsWriterDeleteQueue queue = new DocumentsWriterDeleteQueue(); + Set uniqueValues = new HashSet(); + final int size = 10000 + random.nextInt(500) * RANDOM_MULTIPLIER; + Integer[] ids = new Integer[size]; + Term template = new Term("id"); + for (int i = 0; i < ids.length; i++) { + ids[i] = random.nextInt(); + uniqueValues.add(template.createTerm(ids[i].toString())); + } + CountDownLatch latch = new CountDownLatch(1); + AtomicInteger index = new AtomicInteger(0); + final int numThreads = 2 + random.nextInt(5); + UpdateThread[] threads = new UpdateThread[numThreads]; + for (int i = 0; i < threads.length; i++) { + threads[i] = new UpdateThread(queue, index, ids, latch); + threads[i].start(); + } + latch.countDown(); + for (int i = 0; i < threads.length; i++) { + threads[i].join(); + } + + for (UpdateThread updateThread : threads) { + DeleteSlice slice = updateThread.slice; + queue.updateSlice(slice); + BufferedDeletes deletes = updateThread.deletes; + slice.apply(deletes, BufferedDeletes.MAX_INT); + assertEquals(uniqueValues, deletes.terms.keySet()); + } + queue.tryApplyGlobalSlice(); + assertEquals(uniqueValues, new HashSet(Arrays.asList(queue + .freezeGlobalBuffer(null).terms))); + assertEquals("num deletes must be 0 after freeze", 0, queue + .numGlobalTermDeletes()); + } + + private static class UpdateThread extends Thread { + final DocumentsWriterDeleteQueue queue; + final AtomicInteger index; + final Integer[] ids; + final DeleteSlice slice; + final BufferedDeletes deletes; + final CountDownLatch latch; + + protected UpdateThread(DocumentsWriterDeleteQueue queue, + AtomicInteger index, Integer[] ids, CountDownLatch latch) { + this.queue = queue; + this.index = index; + this.ids = ids; + this.slice = queue.newSlice(); + deletes = new BufferedDeletes(false); + this.latch = latch; + } + + @Override + public void run() { + try { + latch.await(); + } catch (InterruptedException e) { + throw new ThreadInterruptedException(e); + } + Term template = new Term("id"); + int i = 0; + while ((i = index.getAndIncrement()) < ids.length) { + Term term = template.createTerm(ids[i].toString()); + queue.add(term, slice); + assertTrue(slice.isTailItem(term)); + slice.apply(deletes, BufferedDeletes.MAX_INT); + } + } + } + +} --- lucene/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java 1970-01-01 01:00:00.000000000 +0100 +++ lucene/src/test/org/apache/lucene/index/TestFlushByRamOrCountsPolicy.java 2011-04-29 14:59:51.227041400 +0200 @@ -0,0 +1,432 @@ +package org.apache.lucene.index; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.lucene.analysis.MockAnalyzer; +import org.apache.lucene.document.Document; +import org.apache.lucene.index.DocumentsWriterPerThreadPool.ThreadState; +import org.apache.lucene.store.Directory; +import org.apache.lucene.store.LockObtainFailedException; +import org.apache.lucene.store.MockDirectoryWrapper; +import org.apache.lucene.util.LineFileDocs; +import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.ThrottledIndexOutput; +import org.junit.Before; + +public class TestFlushByRamOrCountsPolicy extends LuceneTestCase { + + private LineFileDocs lineDocFile; + + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + lineDocFile = new LineFileDocs(random); + } + + public void testFlushByRam() throws CorruptIndexException, + LockObtainFailedException, IOException, InterruptedException { + int[] numThreads = new int[] { 3 + random.nextInt(12), 1 }; + for (int i = 0; i < numThreads.length; i++) { + runFlushByRam(numThreads[i], + 1 + random.nextInt(10) + random.nextDouble(), false); + } + + for (int i = 0; i < numThreads.length; i++) { + // with a 512 mb ram buffer we should never stall + runFlushByRam(numThreads[i], 512.d, true); + } + } + + protected void runFlushByRam(int numThreads, double maxRam, + boolean ensureNotStalled) throws IOException, CorruptIndexException, + LockObtainFailedException, InterruptedException { + final int numDocumentsToIndex = 50 + random.nextInt(150); + AtomicInteger numDocs = new AtomicInteger(numDocumentsToIndex); + Directory dir = newDirectory(); + MockDefaultFlushPolicy flushPolicy = new MockDefaultFlushPolicy(); + IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, + new MockAnalyzer(random)).setFlushPolicy(flushPolicy); + + final int numDWPT = 1 + random.nextInt(8); + DocumentsWriterPerThreadPool threadPool = new ThreadAffinityDocumentsWriterThreadPool( + numDWPT); + iwc.setIndexerThreadPool(threadPool); + iwc.setRAMBufferSizeMB(1 + random.nextInt(10) + random.nextDouble()); + iwc.setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH); + iwc.setMaxBufferedDeleteTerms(IndexWriterConfig.DISABLE_AUTO_FLUSH); + IndexWriter writer = new IndexWriter(dir, iwc); + assertFalse(flushPolicy.flushOnDocCount()); + assertFalse(flushPolicy.flushOnDeleteTerms()); + assertTrue(flushPolicy.flushOnRAM()); + DocumentsWriter docsWriter = writer.getDocsWriter(); + assertNotNull(docsWriter); + DocumentsWriterFlushControl flushControl = docsWriter.flushControl; + assertEquals(" bytes must be 0 after init", 0, flushControl.flushBytes()); + + IndexThread[] threads = new IndexThread[numThreads]; + for (int x = 0; x < threads.length; x++) { + threads[x] = new IndexThread(numDocs, numThreads, writer, lineDocFile, + false); + threads[x].start(); + } + + for (int x = 0; x < threads.length; x++) { + threads[x].join(); + } + final long maxRAMBytes = (long) (iwc.getRAMBufferSizeMB() * 1024. * 1024.); + assertEquals(" all flushes must be due numThreads=" + numThreads, 0, + flushControl.flushBytes()); + assertEquals(numDocumentsToIndex, writer.numDocs()); + assertEquals(numDocumentsToIndex, writer.maxDoc()); + assertTrue("peak bytes without flush exceeded watermark", + flushPolicy.peakBytesWithoutFlush <= maxRAMBytes); + assertActiveBytesAfter(flushControl); + if (flushPolicy.hasMarkedPending) { + assertTrue(maxRAMBytes < flushControl.peakActiveBytes); + } + if (ensureNotStalled) { + assertFalse(docsWriter.healthiness.wasStalled); + } + writer.close(); + assertEquals(0, flushControl.activeBytes()); + dir.close(); + } + + public void testFlushDocCount() throws CorruptIndexException, + LockObtainFailedException, IOException, InterruptedException { + int[] numThreads = new int[] { 3 + random.nextInt(12), 1 }; + for (int i = 0; i < numThreads.length; i++) { + + final int numDocumentsToIndex = 50 + random.nextInt(150); + AtomicInteger numDocs = new AtomicInteger(numDocumentsToIndex); + Directory dir = newDirectory(); + MockDefaultFlushPolicy flushPolicy = new MockDefaultFlushPolicy(); + IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, + new MockAnalyzer(random)).setFlushPolicy(flushPolicy); + + final int numDWPT = 1 + random.nextInt(8); + DocumentsWriterPerThreadPool threadPool = new ThreadAffinityDocumentsWriterThreadPool( + numDWPT); + iwc.setIndexerThreadPool(threadPool); + iwc.setMaxBufferedDocs(2 + random.nextInt(50)); + iwc.setRAMBufferSizeMB(IndexWriterConfig.DISABLE_AUTO_FLUSH); + iwc.setMaxBufferedDeleteTerms(IndexWriterConfig.DISABLE_AUTO_FLUSH); + IndexWriter writer = new IndexWriter(dir, iwc); + assertTrue(flushPolicy.flushOnDocCount()); + assertFalse(flushPolicy.flushOnDeleteTerms()); + assertFalse(flushPolicy.flushOnRAM()); + DocumentsWriter docsWriter = writer.getDocsWriter(); + assertNotNull(docsWriter); + DocumentsWriterFlushControl flushControl = docsWriter.flushControl; + assertEquals(" bytes must be 0 after init", 0, flushControl.flushBytes()); + + IndexThread[] threads = new IndexThread[numThreads[i]]; + for (int x = 0; x < threads.length; x++) { + threads[x] = new IndexThread(numDocs, numThreads[i], writer, + lineDocFile, false); + threads[x].start(); + } + + for (int x = 0; x < threads.length; x++) { + threads[x].join(); + } + + assertEquals(" all flushes must be due numThreads=" + numThreads[i], 0, + flushControl.flushBytes()); + assertEquals(numDocumentsToIndex, writer.numDocs()); + assertEquals(numDocumentsToIndex, writer.maxDoc()); + assertTrue("peak bytes without flush exceeded watermark", + flushPolicy.peakDocCountWithoutFlush <= iwc.getMaxBufferedDocs()); + assertActiveBytesAfter(flushControl); + writer.close(); + assertEquals(0, flushControl.activeBytes()); + dir.close(); + } + } + + public void testRandom() throws IOException, InterruptedException { + final int numThreads = 1 + random.nextInt(8); + final int numDocumentsToIndex = 100 + random.nextInt(300); + AtomicInteger numDocs = new AtomicInteger(numDocumentsToIndex); + Directory dir = newDirectory(); + IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, + new MockAnalyzer(random)); + MockDefaultFlushPolicy flushPolicy = new MockDefaultFlushPolicy(); + iwc.setFlushPolicy(flushPolicy); + + final int numDWPT = 1 + random.nextInt(8); + DocumentsWriterPerThreadPool threadPool = new ThreadAffinityDocumentsWriterThreadPool( + numDWPT); + iwc.setIndexerThreadPool(threadPool); + + IndexWriter writer = new IndexWriter(dir, iwc); + DocumentsWriter docsWriter = writer.getDocsWriter(); + assertNotNull(docsWriter); + DocumentsWriterFlushControl flushControl = docsWriter.flushControl; + + assertEquals(" bytes must be 0 after init", 0, flushControl.flushBytes()); + + IndexThread[] threads = new IndexThread[numThreads]; + for (int x = 0; x < threads.length; x++) { + threads[x] = new IndexThread(numDocs, numThreads, writer, lineDocFile, + true); + threads[x].start(); + } + + for (int x = 0; x < threads.length; x++) { + threads[x].join(); + } + assertEquals(" all flushes must be due", 0, flushControl.flushBytes()); + assertEquals(numDocumentsToIndex, writer.numDocs()); + assertEquals(numDocumentsToIndex, writer.maxDoc()); + if (flushPolicy.flushOnRAM() && !flushPolicy.flushOnDocCount() + && !flushPolicy.flushOnDeleteTerms()) { + final long maxRAMBytes = (long) (iwc.getRAMBufferSizeMB() * 1024. * 1024.); + assertTrue("peak bytes without flush exceeded watermark", + flushPolicy.peakBytesWithoutFlush <= maxRAMBytes); + if (flushPolicy.hasMarkedPending) { + assertTrue("max: " + maxRAMBytes + " " + flushControl.peakActiveBytes, + maxRAMBytes <= flushControl.peakActiveBytes); + } + } + assertActiveBytesAfter(flushControl); + writer.commit(); + assertEquals(0, flushControl.activeBytes()); + IndexReader r = IndexReader.open(dir); + assertEquals(numDocumentsToIndex, r.numDocs()); + assertEquals(numDocumentsToIndex, r.maxDoc()); + if (!flushPolicy.flushOnRAM()) { + assertFalse("never stall if we don't flush on RAM", docsWriter.healthiness.wasStalled); + assertFalse("never block if we don't flush on RAM", docsWriter.healthiness.hasBlocked()); + } + r.close(); + writer.close(); + dir.close(); + } + + public void testHealthyness() throws InterruptedException, + CorruptIndexException, LockObtainFailedException, IOException { + + int[] numThreads = new int[] { 4 + random.nextInt(8), 1 }; + final int numDocumentsToIndex = 50 + random.nextInt(50); + for (int i = 0; i < numThreads.length; i++) { + AtomicInteger numDocs = new AtomicInteger(numDocumentsToIndex); + MockDirectoryWrapper dir = newDirectory(); + // mock a very slow harddisk here so that flushing is very slow + dir.setThrottledIndexOutput(new ThrottledIndexOutput(ThrottledIndexOutput + .mBitsToBytes(40 + random.nextInt(10)), 5 + random.nextInt(5), null)); + IndexWriterConfig iwc = newIndexWriterConfig(TEST_VERSION_CURRENT, + new MockAnalyzer(random)); + iwc.setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH); + iwc.setMaxBufferedDeleteTerms(IndexWriterConfig.DISABLE_AUTO_FLUSH); + FlushPolicy flushPolicy = new FlushByRamOrCountsPolicy(); + iwc.setFlushPolicy(flushPolicy); + + DocumentsWriterPerThreadPool threadPool = new ThreadAffinityDocumentsWriterThreadPool( + numThreads[i]== 1 ? 1 : 2); + iwc.setIndexerThreadPool(threadPool); + // with such a small ram buffer we should be stalled quiet quickly + iwc.setRAMBufferSizeMB(0.25); + IndexWriter writer = new IndexWriter(dir, iwc); + IndexThread[] threads = new IndexThread[numThreads[i]]; + for (int x = 0; x < threads.length; x++) { + threads[x] = new IndexThread(numDocs, numThreads[i], writer, + lineDocFile, false); + threads[x].start(); + } + + for (int x = 0; x < threads.length; x++) { + threads[x].join(); + } + DocumentsWriter docsWriter = writer.getDocsWriter(); + assertNotNull(docsWriter); + DocumentsWriterFlushControl flushControl = docsWriter.flushControl; + assertEquals(" all flushes must be due", 0, flushControl.flushBytes()); + assertEquals(numDocumentsToIndex, writer.numDocs()); + assertEquals(numDocumentsToIndex, writer.maxDoc()); + if (numThreads[i] == 1) { + assertFalse( + "single thread must not stall", + docsWriter.healthiness.wasStalled); + assertFalse( + "single thread must not block numThreads: " + numThreads[i], + docsWriter.healthiness.hasBlocked()); + // this assumption is too strict in this test +// } else { +// if (docsWriter.healthiness.wasStalled) { +// // TODO maybe this assumtion is too strickt +// assertTrue(" we should have blocked here numThreads: " +// + numThreads[i], docsWriter.healthiness.hasBlocked()); +// } + } + assertActiveBytesAfter(flushControl); + writer.close(true); + dir.close(); + } + } + + protected void assertActiveBytesAfter(DocumentsWriterFlushControl flushControl) { + Iterator allActiveThreads = flushControl.allActiveThreads(); + long bytesUsed = 0; + while (allActiveThreads.hasNext()) { + bytesUsed += allActiveThreads.next().perThread.bytesUsed(); + } + assertEquals(bytesUsed, flushControl.activeBytes()); + } + + public class IndexThread extends Thread { + IndexWriter writer; + IndexWriterConfig iwc; + LineFileDocs docs; + private AtomicInteger pendingDocs; + private final boolean doRandomCommit; + + public IndexThread(AtomicInteger pendingDocs, int numThreads, + IndexWriter writer, LineFileDocs docs, boolean doRandomCommit) { + this.pendingDocs = pendingDocs; + this.writer = writer; + iwc = writer.getConfig(); + this.docs = docs; + this.doRandomCommit = doRandomCommit; + } + + public void run() { + try { + long ramSize = 0; + while (pendingDocs.decrementAndGet() > -1) { + Document doc = docs.nextDoc(); + writer.addDocument(doc); + long newRamSize = writer.ramSizeInBytes(); + if (newRamSize != ramSize) { + ramSize = newRamSize; + } + if (doRandomCommit) { + int commit; + synchronized (random) { + commit = random.nextInt(20); + } + if (commit == 0) { + writer.commit(); + } + } + } + writer.commit(); + } catch (Throwable ex) { + throw new RuntimeException(ex); + } + } + } + + private static class MockDefaultFlushPolicy extends FlushByRamOrCountsPolicy { + long peakBytesWithoutFlush = Integer.MIN_VALUE; + long peakDocCountWithoutFlush = Integer.MIN_VALUE; + boolean hasMarkedPending = false; + + @Override + public void onDelete(DocumentsWriterFlushControl control, ThreadState state) { + final ArrayList pending = new ArrayList(); + final ArrayList notPending = new ArrayList(); + findPending(control, pending, notPending); + final boolean flushCurrent = state.flushPending; + final ThreadState toFlush; + if (state.flushPending) { + toFlush = state; + } else if (flushOnDeleteTerms() + && state.perThread.pendingDeletes.numTermDeletes.get() >= indexWriterConfig + .getMaxBufferedDeleteTerms()) { + toFlush = state; + } else { + toFlush = null; + } + super.onDelete(control, state); + if (toFlush != null) { + if (flushCurrent) { + assertTrue(pending.remove(toFlush)); + } else { + assertTrue(notPending.remove(toFlush)); + } + assertTrue(toFlush.flushPending); + hasMarkedPending = true; + } + + for (ThreadState threadState : notPending) { + assertFalse(threadState.flushPending); + } + } + + @Override + public void onInsert(DocumentsWriterFlushControl control, ThreadState state) { + final ArrayList pending = new ArrayList(); + final ArrayList notPending = new ArrayList(); + findPending(control, pending, notPending); + final boolean flushCurrent = state.flushPending; + long activeBytes = control.activeBytes(); + final ThreadState toFlush; + if (state.flushPending) { + toFlush = state; + } else if (flushOnDocCount() + && state.perThread.getNumDocsInRAM() >= indexWriterConfig + .getMaxBufferedDocs()) { + toFlush = state; + } else if (flushOnRAM() + && activeBytes >= (long) (indexWriterConfig.getRAMBufferSizeMB() * 1024. * 1024.)) { + toFlush = findLargestNonPendingWriter(control, state); + assertFalse(toFlush.flushPending); + } else { + toFlush = null; + } + super.onInsert(control, state); + if (toFlush != null) { + if (flushCurrent) { + assertTrue(pending.remove(toFlush)); + } else { + assertTrue(notPending.remove(toFlush)); + } + assertTrue(toFlush.flushPending); + hasMarkedPending = true; + } else { + peakBytesWithoutFlush = Math.max(activeBytes, peakBytesWithoutFlush); + peakDocCountWithoutFlush = Math.max(state.perThread.getNumDocsInRAM(), + peakDocCountWithoutFlush); + } + + for (ThreadState threadState : notPending) { + assertFalse(threadState.flushPending); + } + } + } + + static void findPending(DocumentsWriterFlushControl flushControl, + ArrayList pending, ArrayList notPending) { + Iterator allActiveThreads = flushControl.allActiveThreads(); + while (allActiveThreads.hasNext()) { + ThreadState next = allActiveThreads.next(); + if (next.flushPending) { + pending.add(next); + } else { + notPending.add(next); + } + } + } +} --- lucene/src/test/org/apache/lucene/index/TestIndexWriter.java 2011-04-22 21:46:44.034869900 +0200 +++ lucene/src/test/org/apache/lucene/index/TestIndexWriter.java 2011-04-29 14:59:32.824988800 +0200 @@ -782,13 +782,14 @@ writer.deleteDocuments(new Term("field", "aaa" + j)); _TestUtil.syncConcurrentMerges(writer); int flushCount = writer.getFlushCount(); + if (j == 1) lastFlushCount = flushCount; else if (j < 10) { // No new files should be created assertEquals(flushCount, lastFlushCount); } else if (10 == j) { - assertTrue(flushCount > lastFlushCount); + assertTrue("" + j, flushCount > lastFlushCount); lastFlushCount = flushCount; writer.getConfig().setRAMBufferSizeMB(0.000001); writer.getConfig().setMaxBufferedDeleteTerms(1); @@ -2678,20 +2679,8 @@ dir.close(); } - private static class FlushCountingIndexWriter extends IndexWriter { - int flushCount; - public FlushCountingIndexWriter(Directory dir, IndexWriterConfig iwc) throws IOException { - super(dir, iwc); - } - @Override - public void doAfterFlush() { - flushCount++; - } - } - public void testIndexingThenDeleting() throws Exception { final Random r = random; - Directory dir = newDirectory(); // note this test explicitly disables payloads final Analyzer analyzer = new Analyzer() { @@ -2700,7 +2689,7 @@ return new MockTokenizer(reader, MockTokenizer.WHITESPACE, true); } }; - FlushCountingIndexWriter w = new FlushCountingIndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, analyzer).setRAMBufferSizeMB(1.0).setMaxBufferedDocs(-1).setMaxBufferedDeleteTerms(-1)); + IndexWriter w = new IndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, analyzer).setRAMBufferSizeMB(1.0).setMaxBufferedDocs(IndexWriterConfig.DISABLE_AUTO_FLUSH).setMaxBufferedDeleteTerms(IndexWriterConfig.DISABLE_AUTO_FLUSH)); w.setInfoStream(VERBOSE ? System.out : null); Document doc = new Document(); doc.add(newField("field", "go 1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20", Field.Store.NO, Field.Index.ANALYZED)); @@ -2714,15 +2703,15 @@ } if (doIndexing) { // Add docs until a flush is triggered - final int startFlushCount = w.flushCount; - while(w.flushCount == startFlushCount) { + final int startFlushCount = w.getFlushCount(); + while(w.getFlushCount() == startFlushCount) { w.addDocument(doc); count++; } } else { // Delete docs until a flush is triggered - final int startFlushCount = w.flushCount; - while(w.flushCount == startFlushCount) { + final int startFlushCount = w.getFlushCount(); + while(w.getFlushCount() == startFlushCount) { w.deleteDocuments(new Term("foo", ""+count)); count++; } @@ -3111,7 +3100,7 @@ Directory dir = newDirectory(); RandomIndexWriter w = new RandomIndexWriter(random, dir, new StringSplitAnalyzer()); - char[] chars = new char[DocumentsWriter.MAX_TERM_LENGTH_UTF8]; + char[] chars = new char[DocumentsWriterPerThread.MAX_TERM_LENGTH_UTF8]; Arrays.fill(chars, 'x'); Document doc = new Document(); final String bigTerm = new String(chars); --- lucene/src/test/org/apache/lucene/index/TestIndexWriterConfig.java 2011-04-14 09:38:41.250848100 +0200 +++ lucene/src/test/org/apache/lucene/index/TestIndexWriterConfig.java 2011-04-29 14:59:35.968168600 +0200 @@ -24,7 +24,7 @@ import java.util.Set; import org.apache.lucene.analysis.MockAnalyzer; -import org.apache.lucene.index.DocumentsWriter.IndexingChain; +import org.apache.lucene.index.DocumentsWriterPerThread.IndexingChain; import org.apache.lucene.index.IndexWriterConfig.OpenMode; import org.apache.lucene.search.DefaultSimilarityProvider; import org.apache.lucene.search.IndexSearcher; @@ -41,7 +41,7 @@ // Does not implement anything - used only for type checking on IndexWriterConfig. @Override - DocConsumer getChain(DocumentsWriter documentsWriter) { + DocConsumer getChain(DocumentsWriterPerThread documentsWriter) { return null; } @@ -64,11 +64,15 @@ assertEquals(IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB, conf.getRAMBufferSizeMB(), 0.0); assertEquals(IndexWriterConfig.DEFAULT_MAX_BUFFERED_DOCS, conf.getMaxBufferedDocs()); assertEquals(IndexWriterConfig.DEFAULT_READER_POOLING, conf.getReaderPooling()); - assertTrue(DocumentsWriter.defaultIndexingChain == conf.getIndexingChain()); + assertTrue(DocumentsWriterPerThread.defaultIndexingChain == conf.getIndexingChain()); assertNull(conf.getMergedSegmentWarmer()); - assertEquals(IndexWriterConfig.DEFAULT_MAX_THREAD_STATES, conf.getMaxThreadStates()); assertEquals(IndexWriterConfig.DEFAULT_READER_TERMS_INDEX_DIVISOR, conf.getReaderTermsIndexDivisor()); assertEquals(TieredMergePolicy.class, conf.getMergePolicy().getClass()); + assertEquals(ThreadAffinityDocumentsWriterThreadPool.class, conf.getIndexerThreadPool().getClass()); + assertNull(conf.getFlushPolicy()); + assertEquals(IndexWriterConfig.DEFAULT_RAM_PER_THREAD_HARD_LIMIT_MB, conf.getRAMPerThreadHardLimitMB()); + + // Sanity check - validate that all getters are covered. Set getters = new HashSet(); @@ -91,7 +95,11 @@ getters.add("getMergePolicy"); getters.add("getMaxThreadStates"); getters.add("getReaderPooling"); + getters.add("getIndexerThreadPool"); getters.add("getReaderTermsIndexDivisor"); + getters.add("getFlushPolicy"); + getters.add("getRAMPerThreadHardLimitMB"); + for (Method m : IndexWriterConfig.class.getDeclaredMethods()) { if (m.getDeclaringClass() == IndexWriterConfig.class && m.getName().startsWith("get")) { assertTrue("method " + m.getName() + " is not tested for defaults", getters.contains(m.getName())); @@ -123,7 +131,6 @@ assertEquals(IndexWriterConfig.DISABLE_AUTO_FLUSH, IndexWriterConfig.DEFAULT_MAX_BUFFERED_DOCS); assertEquals(16.0, IndexWriterConfig.DEFAULT_RAM_BUFFER_SIZE_MB, 0.0); assertEquals(false, IndexWriterConfig.DEFAULT_READER_POOLING); - assertEquals(8, IndexWriterConfig.DEFAULT_MAX_THREAD_STATES); assertEquals(IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, IndexWriterConfig.DEFAULT_READER_TERMS_INDEX_DIVISOR); } @@ -183,11 +190,11 @@ assertTrue(IndexSearcher.getDefaultSimilarityProvider() == conf.getSimilarityProvider()); // Test IndexingChain - assertTrue(DocumentsWriter.defaultIndexingChain == conf.getIndexingChain()); + assertTrue(DocumentsWriterPerThread.defaultIndexingChain == conf.getIndexingChain()); conf.setIndexingChain(new MyIndexingChain()); assertEquals(MyIndexingChain.class, conf.getIndexingChain().getClass()); conf.setIndexingChain(null); - assertTrue(DocumentsWriter.defaultIndexingChain == conf.getIndexingChain()); + assertTrue(DocumentsWriterPerThread.defaultIndexingChain == conf.getIndexingChain()); try { conf.setMaxBufferedDeleteTerms(0); @@ -239,11 +246,19 @@ // this is expected } - assertEquals(IndexWriterConfig.DEFAULT_MAX_THREAD_STATES, conf.getMaxThreadStates()); - conf.setMaxThreadStates(5); - assertEquals(5, conf.getMaxThreadStates()); - conf.setMaxThreadStates(0); - assertEquals(IndexWriterConfig.DEFAULT_MAX_THREAD_STATES, conf.getMaxThreadStates()); + try { + conf.setRAMPerThreadHardLimitMB(2048); + fail("should not have succeeded to set RAMPerThreadHardLimitMB to >= 2048"); + } catch (IllegalArgumentException e) { + // this is expected + } + + try { + conf.setRAMPerThreadHardLimitMB(0); + fail("should not have succeeded to set RAMPerThreadHardLimitMB to 0"); + } catch (IllegalArgumentException e) { + // this is expected + } // Test MergePolicy assertEquals(TieredMergePolicy.class, conf.getMergePolicy().getClass()); --- lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java 2011-04-20 23:25:30.686388400 +0200 +++ lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java 2011-04-29 14:59:42.194524700 +0200 @@ -538,10 +538,13 @@ } // prevent throwing a random exception here!! final double randomIOExceptionRate = dir.getRandomIOExceptionRate(); + final long maxSizeInBytes = dir.getMaxSizeInBytes(); dir.setRandomIOExceptionRate(0.0); + dir.setMaxSizeInBytes(0); if (!success) { // Must force the close else the writer can have // open files which cause exc in MockRAMDir.close + modifier.rollback(); } @@ -552,6 +555,7 @@ TestIndexWriter.assertNoUnreferencedFiles(dir, "after writer.close"); } dir.setRandomIOExceptionRate(randomIOExceptionRate); + dir.setMaxSizeInBytes(maxSizeInBytes); // Finally, verify index is not corrupt, and, if // we succeeded, we see all docs changed, and if --- lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java 2011-04-14 09:38:41.174845400 +0200 +++ lucene/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java 2011-04-29 14:59:33.439024000 +0200 @@ -223,8 +223,9 @@ threads[i].join(); for(int i=0;i docs = new HashMap(); IndexWriter w = new MockIndexWriter(dir, newIndexWriterConfig( TEST_VERSION_CURRENT, new MockAnalyzer(random)).setOpenMode(OpenMode.CREATE) - .setRAMBufferSizeMB(0.1).setMaxBufferedDocs(maxBufferedDocs).setMaxThreadStates(maxThreadStates) + .setRAMBufferSizeMB(0.1).setMaxBufferedDocs(maxBufferedDocs).setIndexerThreadPool(new ThreadAffinityDocumentsWriterThreadPool(maxThreadStates)) .setReaderPooling(doReaderPooling).setMergePolicy(newLogMergePolicy())); w.setInfoStream(VERBOSE ? System.out : null); LogMergePolicy lmp = (LogMergePolicy) w.getConfig().getMergePolicy(); --- lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java 2011-04-14 09:38:45.708263400 +0200 +++ lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java 2011-04-29 15:02:02.553552800 +0200 @@ -2,13 +2,14 @@ import java.io.IOException; import java.util.Random; +import java.lang.reflect.Method; import junit.framework.Assert; import org.apache.lucene.analysis.MockAnalyzer; import org.apache.lucene.document.Document; -import org.apache.lucene.index.IndexReader.AtomicReaderContext; import org.apache.lucene.index.IndexReader; +import org.apache.lucene.index.IndexReader.AtomicReaderContext; import org.apache.lucene.index.IndexWriter; import org.apache.lucene.index.IndexWriterConfig; import org.apache.lucene.index.MultiReader; --- lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java 2011-04-11 14:48:16.992278900 +0200 +++ lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java 2011-04-29 15:02:09.135929300 +0200 @@ -34,6 +34,7 @@ import org.apache.lucene.index.IndexReader; import org.apache.lucene.index.codecs.CodecProvider; import org.apache.lucene.util.LuceneTestCase; +import org.apache.lucene.util.ThrottledIndexOutput; import org.apache.lucene.util._TestUtil; /** @@ -69,6 +70,7 @@ private Set createdFiles; Set openFilesForWrite = new HashSet(); volatile boolean crashed; + private ThrottledIndexOutput throttledOutput; // use this for tracking files for crash. // additionally: provides debugging information in case you leave one open @@ -115,6 +117,10 @@ preventDoubleWrite = value; } + public void setThrottledIndexOutput(ThrottledIndexOutput throttledOutput) { + this.throttledOutput = throttledOutput; + } + @Override public synchronized void sync(Collection names) throws IOException { maybeYield(); @@ -348,7 +354,7 @@ IndexOutput io = new MockIndexOutputWrapper(this, delegate.createOutput(name), name); openFileHandles.put(io, new RuntimeException("unclosed IndexOutput")); openFilesForWrite.add(name); - return io; + return throttledOutput == null ? io : throttledOutput.newFromDelegate(io); } @Override @@ -578,4 +584,5 @@ maybeYield(); delegate.copy(to, src, dest); } + } --- lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java 2011-04-27 15:56:49.222736300 +0200 +++ lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java 2011-04-29 15:02:10.531009100 +0200 @@ -778,7 +778,7 @@ c.setTermIndexInterval(_TestUtil.nextInt(r, 1, 1000)); } if (r.nextBoolean()) { - c.setMaxThreadStates(_TestUtil.nextInt(r, 1, 20)); + c.setIndexerThreadPool(new ThreadAffinityDocumentsWriterThreadPool(_TestUtil.nextInt(r, 1, 20))); } if (r.nextBoolean()) { --- lucene/src/test-framework/org/apache/lucene/util/ThrottledIndexOutput.java 1970-01-01 01:00:00.000000000 +0100 +++ lucene/src/test-framework/org/apache/lucene/util/ThrottledIndexOutput.java 2011-04-29 15:02:10.151987400 +0200 @@ -0,0 +1,147 @@ +package org.apache.lucene.util; + +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +import java.io.IOException; + +import org.apache.lucene.store.DataInput; +import org.apache.lucene.store.IndexOutput; + +public class ThrottledIndexOutput extends IndexOutput { + public static final int DEFAULT_MIN_WRITTEN_BYTES = 1024; + private final int bytesPerSecond; + private IndexOutput delegate; + private long flushDelayMillis; + private long closeDelayMillis; + private long seekDelayMillis; + private long pendingBytes; + private long minBytesWritten; + private long timeElapsed; + private final byte[] bytes = new byte[1]; + + public ThrottledIndexOutput newFromDelegate(IndexOutput output) { + return new ThrottledIndexOutput(bytesPerSecond, flushDelayMillis, + closeDelayMillis, seekDelayMillis, minBytesWritten, output); + } + + public ThrottledIndexOutput(int bytesPerSecond, long delayInMillis, + IndexOutput delegate) { + this(bytesPerSecond, delayInMillis, delayInMillis, delayInMillis, + DEFAULT_MIN_WRITTEN_BYTES, delegate); + } + + public ThrottledIndexOutput(int bytesPerSecond, long delays, + int minBytesWritten, IndexOutput delegate) { + this(bytesPerSecond, delays, delays, delays, minBytesWritten, delegate); + } + + public static final int mBitsToBytes(int mbits) { + return mbits * 125000; + } + + public ThrottledIndexOutput(int bytesPerSecond, long flushDelayMillis, + long closeDelayMillis, long seekDelayMillis, long minBytesWritten, + IndexOutput delegate) { + assert bytesPerSecond > 0; + this.delegate = delegate; + this.bytesPerSecond = bytesPerSecond; + this.flushDelayMillis = flushDelayMillis; + this.closeDelayMillis = closeDelayMillis; + this.seekDelayMillis = seekDelayMillis; + this.minBytesWritten = minBytesWritten; + } + + @Override + public void flush() throws IOException { + sleep(flushDelayMillis); + delegate.flush(); + } + + @Override + public void close() throws IOException { + sleep(closeDelayMillis + getDelay(true)); + delegate.close(); + + } + + @Override + public long getFilePointer() { + return delegate.getFilePointer(); + } + + @Override + public void seek(long pos) throws IOException { + sleep(seekDelayMillis); + delegate.seek(pos); + } + + @Override + public long length() throws IOException { + return delegate.length(); + } + + @Override + public void writeByte(byte b) throws IOException { + bytes[0] = b; + writeBytes(bytes, 0, 1); + } + + @Override + public void writeBytes(byte[] b, int offset, int length) throws IOException { + final long before = System.nanoTime(); + delegate.writeBytes(b, offset, length); + timeElapsed += System.nanoTime() - before; + pendingBytes += length; + sleep(getDelay(false)); + + } + + protected long getDelay(boolean closing) { + if (pendingBytes > 0 && (closing || pendingBytes > minBytesWritten)) { + long actualBps = (timeElapsed / pendingBytes) * 1000000000l; // nano to sec + if (actualBps > bytesPerSecond) { + long expected = (pendingBytes * 1000l / bytesPerSecond) ; + final long delay = expected - (timeElapsed / 1000000l) ; + pendingBytes = 0; + timeElapsed = 0; + return delay; + } + } + return 0; + + } + + private static final void sleep(long ms) { + if (ms <= 0) + return; + try { + Thread.sleep(ms); + } catch (InterruptedException e) { + throw new ThreadInterruptedException(e); + } + } + + @Override + public void setLength(long length) throws IOException { + delegate.setLength(length); + } + + @Override + public void copyBytes(DataInput input, long numBytes) throws IOException { + delegate.copyBytes(input, numBytes); + } +} --- solr/src/java/org/apache/solr/analysis/PathHierarchyTokenizerFactory.java 2011-03-26 11:40:45.728170100 +0100 +++ solr/src/java/org/apache/solr/analysis/PathHierarchyTokenizerFactory.java 2011-04-29 15:03:51.742798100 +0200 @@ -31,7 +31,7 @@ * <tokenizer class="solr.PathHierarchyTokenizerFactory" delimiter="\" replace="/"/> * </analyzer> * </fieldType> - * @version $Id: PathHierarchyTokenizerFactory.java 1085635 2011-03-26 03:34:42Z ryan $ + * @version $Id: PathHierarchyTokenizerFactory.java 1086876 2011-03-30 09:17:25Z simonw $ */ public class PathHierarchyTokenizerFactory extends BaseTokenizerFactory { --- solr/src/test-files/solr/conf/solrconfig-repeater.xml 2011-02-21 14:10:06.438543400 +0100 +++ solr/src/test-files/solr/conf/solrconfig-repeater.xml 2011-04-29 15:03:51.710796300 +0200 @@ -17,7 +17,7 @@ limitations under the License. --> -