You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by us...@apache.org on 2011/05/02 00:38:36 UTC
svn commit: r1098427 [4/5] - in /lucene/dev/trunk: ./ lucene/
lucene/src/java/org/apache/lucene/index/
lucene/src/test-framework/org/apache/lucene/search/
lucene/src/test-framework/org/apache/lucene/store/
lucene/src/test-framework/org/apache/lucene/ut...
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1098427&r1=1098426&r2=1098427&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Sun May 1 22:38:33 2011
@@ -39,24 +39,24 @@ import org.apache.lucene.util.ReaderUtil
/**
* The SegmentMerger class combines two or more Segments, represented by an IndexReader ({@link #add},
- * into a single Segment. After adding the appropriate readers, call the merge method to combine the
+ * into a single Segment. After adding the appropriate readers, call the merge method to combine the
* segments.
- *
+ *
* @see #merge
* @see #add
*/
final class SegmentMerger {
-
+
/** norms header placeholder */
- static final byte[] NORMS_HEADER = new byte[]{'N','R','M',-1};
-
+ static final byte[] NORMS_HEADER = new byte[]{'N','R','M',-1};
+
private Directory directory;
private String segment;
private int termIndexInterval = IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL;
private List<IndexReader> readers = new ArrayList<IndexReader>();
private final FieldInfos fieldInfos;
-
+
private int mergedDocs;
private final MergeState.CheckAbort checkAbort;
@@ -64,13 +64,13 @@ final class SegmentMerger {
/** Maximum number of contiguous documents to bulk-copy
when merging stored fields */
private final static int MAX_RAW_MERGE_DOCS = 4192;
-
+
private final CodecProvider codecs;
private Codec codec;
private SegmentWriteState segmentWriteState;
private PayloadProcessorProvider payloadProcessorProvider;
-
+
SegmentMerger(Directory dir, int termIndexInterval, String name, MergePolicy.OneMerge merge, CodecProvider codecs, PayloadProcessorProvider payloadProcessorProvider, FieldInfos fieldInfos) {
this.payloadProcessorProvider = payloadProcessorProvider;
directory = dir;
@@ -135,10 +135,10 @@ final class SegmentMerger {
for (String file : files) {
cfsWriter.addFile(file);
}
-
+
// Perform the merge
cfsWriter.close();
-
+
return files;
}
@@ -196,13 +196,12 @@ final class SegmentMerger {
}
/**
- *
+ *
* @return The number of documents in all of the readers
* @throws CorruptIndexException if the index is corrupt
* @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;
@@ -265,7 +264,7 @@ final class SegmentMerger {
throw new RuntimeException("mergeFields produced an invalid result: docCount is " + docCount + " but fdx file size is " + fdxFileLength + " file=" + fileName + " file exists?=" + directory.fileExists(fileName) + "; now aborting this merge to prevent index corruption");
segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codecInfo, null);
-
+
return docCount;
}
@@ -283,7 +282,7 @@ final class SegmentMerger {
++j;
continue;
}
- // We can optimize this case (doing a bulk byte copy) since the field
+ // We can optimize this case (doing a bulk byte copy) since the field
// numbers are identical
int start = j, numDocs = 0;
do {
@@ -295,7 +294,7 @@ final class SegmentMerger {
break;
}
} while(numDocs < MAX_RAW_MERGE_DOCS);
-
+
IndexInput stream = matchingFieldsReader.rawDocs(rawDocLengths, start, numDocs);
fieldsWriter.addRawDocuments(stream, rawDocLengths, numDocs);
docCount += numDocs;
@@ -349,7 +348,7 @@ final class SegmentMerger {
* @throws IOException
*/
private final void mergeVectors() throws IOException {
- TermVectorsWriter termVectorsWriter =
+ TermVectorsWriter termVectorsWriter =
new TermVectorsWriter(directory, segment, fieldInfos);
try {
@@ -369,7 +368,7 @@ final class SegmentMerger {
copyVectorsWithDeletions(termVectorsWriter, matchingVectorsReader, reader);
} else {
copyVectorsNoDeletions(termVectorsWriter, matchingVectorsReader, reader);
-
+
}
}
} finally {
@@ -402,7 +401,7 @@ final class SegmentMerger {
++docNum;
continue;
}
- // We can optimize this case (doing a bulk byte copy) since the field
+ // We can optimize this case (doing a bulk byte copy) since the field
// numbers are identical
int start = docNum, numDocs = 0;
do {
@@ -414,7 +413,7 @@ final class SegmentMerger {
break;
}
} while(numDocs < MAX_RAW_MERGE_DOCS);
-
+
matchingVectorsReader.rawDocs(rawDocLengths, rawDocLengths2, start, numDocs);
termVectorsWriter.addRawDocuments(matchingVectorsReader, rawDocLengths, rawDocLengths2, numDocs);
checkAbort.work(300 * numDocs);
@@ -425,7 +424,7 @@ final class SegmentMerger {
// skip deleted docs
continue;
}
-
+
// NOTE: it's very important to first assign to vectors then pass it to
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
TermFreqVector[] vectors = reader.getTermFreqVectors(docNum);
@@ -434,7 +433,7 @@ final class SegmentMerger {
}
}
}
-
+
private void copyVectorsNoDeletions(final TermVectorsWriter termVectorsWriter,
final TermVectorsReader matchingVectorsReader,
final IndexReader reader)
@@ -470,7 +469,7 @@ final class SegmentMerger {
// Let CodecProvider decide which codec will be used to write
// the new segment:
-
+
int docBase = 0;
final List<Fields> fields = new ArrayList<Fields>();
@@ -498,7 +497,7 @@ final class SegmentMerger {
mergeState.readerCount = readers.size();
mergeState.fieldInfos = fieldInfos;
mergeState.mergedDocCount = mergedDocs;
-
+
// Remap docIDs
mergeState.delCounts = new int[mergeState.readerCount];
mergeState.docMaps = new int[mergeState.readerCount][];
@@ -536,7 +535,7 @@ final class SegmentMerger {
}
assert delCount == mergeState.delCounts[i]: "reader delCount=" + mergeState.delCounts[i] + " vs recomputed delCount=" + delCount;
}
-
+
if (payloadProcessorProvider != null) {
mergeState.dirPayloadProcessor[i] = payloadProcessorProvider.getDirProcessor(reader.directory());
}
@@ -549,7 +548,7 @@ final class SegmentMerger {
// apart when we step through the docs enums in
// MultiDocsEnum.
mergeState.multiDeletedDocs = new MultiBits(bits, bitsStarts);
-
+
try {
consumer.merge(mergeState,
new MultiFields(fields.toArray(Fields.EMPTY_ARRAY),
@@ -568,7 +567,7 @@ final class SegmentMerger {
int[] getDelCounts() {
return mergeState.delCounts;
}
-
+
public boolean getAnyNonBulkMerges() {
assert matchedCount <= readers.size();
return matchedCount != readers.size();
@@ -579,7 +578,7 @@ final class SegmentMerger {
try {
for (FieldInfo fi : fieldInfos) {
if (fi.isIndexed && !fi.omitNorms) {
- if (output == null) {
+ if (output == null) {
output = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.NORMS_EXTENSION));
output.writeBytes(NORMS_HEADER,NORMS_HEADER.length);
}
@@ -610,7 +609,7 @@ final class SegmentMerger {
}
}
} finally {
- if (output != null) {
+ if (output != null) {
output.close();
}
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java?rev=1098427&r1=1098426&r2=1098427&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java Sun May 1 22:38:33 2011
@@ -18,7 +18,8 @@ package org.apache.lucene.index;
*/
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 @@ import org.apache.lucene.util.RamUsageEs
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 @@ final class StoredFieldsWriter {
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 @@ final class StoredFieldsWriter {
}
}
- synchronized void finishDocument(PerDoc perDoc) throws IOException {
+ void finishDocument() throws IOException {
assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument start");
+
initFieldsWriter();
+ fill(docState.docID);
- fill(perDoc.docID);
+ if (fieldsWriter != null && numStoredFields > 0) {
+ fieldsWriter.startDocument(numStoredFields);
+ for (int i = 0; i < numStoredFields; i++) {
+ fieldsWriter.writeField(fieldNumbers[i], storedFields[i]);
+ }
+ lastDocID++;
+ }
- // Append stored fields to the real FieldsWriter:
- fieldsWriter.flushDocument(perDoc.numStoredFields, perDoc.fdt);
- lastDocID++;
- perDoc.reset();
- free(perDoc);
+ reset();
assert docWriter.writer.testPoint("StoredFieldsWriter.finishDocument end");
}
- 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;
+ 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
- void abort() {
- reset();
- free(this);
+ if (numStoredFields == fieldNumbers.length) {
+ fieldNumbers = ArrayUtil.grow(fieldNumbers);
}
- @Override
- public long sizeInBytes() {
- return buffer.getSizeInBytes();
- }
+ storedFields[numStoredFields] = field;
+ fieldNumbers[numStoredFields] = fieldInfo.number;
+ numStoredFields++;
- @Override
- public void finish() throws IOException {
- finishDocument(this);
- }
+ assert docState.testPoint("StoredFieldsWriterPerThread.processFields.writeField");
}
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java?rev=1098427&r1=1098426&r2=1098427&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriter.java Sun May 1 22:38:33 2011
@@ -17,49 +17,48 @@ package org.apache.lucene.index;
* 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<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
+ void flush(Map<FieldInfo, TermsHashConsumerPerField> 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,33 +67,10 @@ final class TermVectorsTermsWriter exten
hasVectors = false;
}
- for (Map.Entry<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> entry : threadsAndFields.entrySet()) {
- for (final TermsHashConsumerPerField field : entry.getValue() ) {
- 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];
+ for (final TermsHashConsumerPerField field : fieldsToFlush.values() ) {
+ TermVectorsTermsWriterPerField perField = (TermVectorsTermsWriterPerField) field;
+ perField.termsHashPerField.reset();
+ perField.shrinkHash();
}
}
@@ -112,18 +88,17 @@ final class TermVectorsTermsWriter exten
}
}
- 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));
-
+
tvx.writeInt(TermVectorsReader.FORMAT_CURRENT);
tvd.writeInt(TermVectorsReader.FORMAT_CURRENT);
tvf.writeInt(TermVectorsReader.FORMAT_CURRENT);
@@ -132,39 +107,44 @@ final class TermVectorsTermsWriter exten
}
}
- 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<perDoc.numVectorFields;i++) {
- tvd.writeVInt(perDoc.fieldNumbers[i]);
- }
- assert 0 == perDoc.fieldPointers[0];
- long lastPos = perDoc.fieldPointers[0];
- for(int i=1;i<perDoc.numVectorFields;i++) {
- long pos = perDoc.fieldPointers[i];
+ tvd.writeVInt(numVectorFields);
+ if (numVectorFields > 0) {
+ for(int i=0;i<numVectorFields;i++) {
+ tvd.writeVInt(perFields[i].fieldInfo.number);
+ }
+ long lastPos = tvf.getFilePointer();
+ perFields[0].finishDocument();
+ for(int i=1;i<numVectorFields;i++) {
+ long pos = tvf.getFilePointer();
tvd.writeVLong(pos-lastPos);
lastPos = pos;
+ perFields[i].finishDocument();
}
- perDoc.perDocTvf.writeTo(tvf);
- perDoc.numVectorFields = 0;
}
- assert lastDocID == perDoc.docID;
+ assert lastDocID == docState.docID;
lastDocID++;
- perDoc.reset();
- free(perDoc);
+ termsHash.reset();
+ reset();
assert docWriter.writer.testPoint("TermVectorsTermsWriter.finishDocument end");
}
@@ -189,55 +169,58 @@ final class TermVectorsTermsWriter exten
}
tvx = tvd = tvf = null;
lastDocID = 0;
- }
- synchronized void free(PerDoc doc) {
- assert freeCount < docFreeList.length;
- docFreeList[freeCount++] = doc;
+ reset();
}
- class PerDoc extends DocumentsWriter.DocWriter {
+ int numVectorFields;
- final DocumentsWriter.PerDocBuffer buffer = docWriter.newPerDocBuffer();
- RAMOutputStream perDocTvf = new RAMOutputStream(buffer);
+ TermVectorsTermsWriterPerField[] perFields;
- int numVectorFields;
+ void reset() {
+ numVectorFields = 0;
+ perFields = new TermVectorsTermsWriterPerField[1];
+ }
- int[] fieldNumbers = new int[1];
- long[] fieldPointers = new long[1];
+ @Override
+ public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo) {
+ return new TermVectorsTermsWriterPerField(termsHashPerField, this, fieldInfo);
+ }
- void reset() {
- perDocTvf.reset();
- buffer.recycle();
- numVectorFields = 0;
+ void addFieldToFlush(TermVectorsTermsWriterPerField fieldToFlush) {
+ if (numVectorFields == perFields.length) {
+ int newSize = ArrayUtil.oversize(numVectorFields + 1, RamUsageEstimator.NUM_BYTES_OBJECT_REF);
+ TermVectorsTermsWriterPerField[] newArray = new TermVectorsTermsWriterPerField[newSize];
+ System.arraycopy(perFields, 0, newArray, 0, numVectorFields);
+ perFields = newArray;
}
- @Override
- void abort() {
- reset();
- free(this);
- }
+ perFields[numVectorFields++] = fieldToFlush;
+ }
- void addField(final int fieldNumber) {
- if (numVectorFields == fieldNumbers.length) {
- fieldNumbers = ArrayUtil.grow(fieldNumbers);
- }
- if (numVectorFields == fieldPointers.length) {
- fieldPointers = ArrayUtil.grow(fieldPointers);
- }
- fieldNumbers[numVectorFields] = fieldNumber;
- fieldPointers[numVectorFields] = perDocTvf.getFilePointer();
- numVectorFields++;
- }
+ @Override
+ void startDocument() throws IOException {
+ assert clearLastVectorFieldName();
+ reset();
+ }
- @Override
- public long sizeInBytes() {
- return buffer.getSizeInBytes();
- }
+ // Called only by assert
+ final boolean clearLastVectorFieldName() {
+ lastVectorFieldName = null;
+ return true;
+ }
- @Override
- public void finish() throws IOException {
- finishDocument(this);
+ // Called only by assert
+ String lastVectorFieldName;
+ final boolean vectorFieldsInOrder(FieldInfo fi) {
+ try {
+ if (lastVectorFieldName != null)
+ return lastVectorFieldName.compareTo(fi.name) < 0;
+ else
+ return true;
+ } finally {
+ lastVectorFieldName = fi.name;
}
}
+
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java?rev=1098427&r1=1098426&r2=1098427&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java Sun May 1 22:38:33 2011
@@ -28,11 +28,10 @@ import org.apache.lucene.util.RamUsageEs
final class TermVectorsTermsWriterPerField extends TermsHashConsumerPerField {
- final TermVectorsTermsWriterPerThread perThread;
final TermsHashPerField termsHashPerField;
final TermVectorsTermsWriter termsWriter;
final FieldInfo fieldInfo;
- final DocumentsWriter.DocState docState;
+ final DocumentsWriterPerThread.DocState docState;
final FieldInvertState fieldState;
boolean doVectors;
@@ -41,11 +40,10 @@ final class TermVectorsTermsWriterPerFie
int maxNumPostings;
OffsetAttribute offsetAttribute = null;
-
- public TermVectorsTermsWriterPerField(TermsHashPerField termsHashPerField, TermVectorsTermsWriterPerThread perThread, FieldInfo fieldInfo) {
+
+ public TermVectorsTermsWriterPerField(TermsHashPerField termsHashPerField, TermVectorsTermsWriter termsWriter, FieldInfo fieldInfo) {
this.termsHashPerField = termsHashPerField;
- this.perThread = perThread;
- this.termsWriter = perThread.termsWriter;
+ this.termsWriter = termsWriter;
this.fieldInfo = fieldInfo;
docState = termsHashPerField.docState;
fieldState = termsHashPerField.fieldState;
@@ -72,22 +70,12 @@ final class TermVectorsTermsWriterPerFie
}
if (doVectors) {
- if (perThread.doc == null) {
- perThread.doc = termsWriter.getPerDoc();
- perThread.doc.docID = docState.docID;
- assert perThread.doc.numVectorFields == 0;
- assert 0 == perThread.doc.perDocTvf.length();
- assert 0 == perThread.doc.perDocTvf.getFilePointer();
- }
-
- assert perThread.doc.docID == docState.docID;
-
+ termsWriter.hasVectors = true;
if (termsHashPerField.bytesHash.size() != 0) {
// Only necessary if previous doc hit a
// non-aborting exception while writing vectors in
// this field:
termsHashPerField.reset();
- perThread.termsHashPerThread.reset(false);
}
}
@@ -95,42 +83,42 @@ final class TermVectorsTermsWriterPerFie
//perThread.postingsCount = 0;
return doVectors;
- }
+ }
public void abort() {}
/** Called once per field per document if term vectors
* are enabled, to write the vectors to
* RAMOutputStream, which is then quickly flushed to
- * the real term vectors files in the Directory. */
- @Override
+ * the real term vectors files in the Directory. */ @Override
void finish() throws IOException {
+ if (!doVectors || termsHashPerField.bytesHash.size() == 0)
+ return;
+ termsWriter.addFieldToFlush(this);
+ }
+
+ void finishDocument() throws IOException {
assert docState.testPoint("TermVectorsTermsWriterPerField.finish start");
final int numPostings = termsHashPerField.bytesHash.size();
- final BytesRef flushTerm = perThread.flushTerm;
+ final BytesRef flushTerm = termsWriter.flushTerm;
assert numPostings >= 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?
@@ -140,21 +128,21 @@ final class TermVectorsTermsWriterPerFie
byte bits = 0x0;
if (doVectorPositions)
bits |= TermVectorsReader.STORE_POSITIONS_WITH_TERMVECTOR;
- if (doVectorOffsets)
+ if (doVectorOffsets)
bits |= TermVectorsReader.STORE_OFFSET_WITH_TERMVECTOR;
tvf.writeByte(bits);
int lastLen = 0;
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<numPostings;j++) {
final int termID = termIDs[j];
final int freq = postings.freqs[termID];
-
+
// Get BytesRef
termBytePool.setBytesRef(flushTerm, postings.textStarts[termID]);
@@ -192,20 +180,13 @@ final class TermVectorsTermsWriterPerFie
}
termsHashPerField.reset();
-
- // NOTE: we clear, per-field, at the thread level,
- // because term vectors fully write themselves on each
- // field; this saves RAM (eg if large doc has two large
- // fields w/ term vectors on) because we recycle/reuse
- // all RAM after each field:
- perThread.termsHashPerThread.reset(false);
}
void shrinkHash() {
termsHashPerField.shrinkHash(maxNumPostings);
maxNumPostings = 0;
}
-
+
@Override
void start(Fieldable f) {
if (doVectorOffsets) {
@@ -225,7 +206,7 @@ final class TermVectorsTermsWriterPerFie
if (doVectorOffsets) {
int startOffset = fieldState.offset + offsetAttribute.startOffset();
int endOffset = fieldState.offset + offsetAttribute.endOffset();
-
+
termsHashPerField.writeVInt(1, startOffset);
termsHashPerField.writeVInt(1, endOffset - startOffset);
postings.lastOffsets[termID] = endOffset;
@@ -243,13 +224,13 @@ final class TermVectorsTermsWriterPerFie
assert docState.testPoint("TermVectorsTermsWriterPerField.addTerm start");
TermVectorsPostingsArray postings = (TermVectorsPostingsArray) termsHashPerField.postingsArray;
-
+
postings.freqs[termID]++;
if (doVectorOffsets) {
int startOffset = fieldState.offset + offsetAttribute.startOffset();
int endOffset = fieldState.offset + offsetAttribute.endOffset();
-
+
termsHashPerField.writeVInt(1, startOffset - postings.lastOffsets[termID]);
termsHashPerField.writeVInt(1, endOffset - startOffset);
postings.lastOffsets[termID] = endOffset;
@@ -280,7 +261,7 @@ final class TermVectorsTermsWriterPerFie
int[] freqs; // How many times this term occurred in the current doc
int[] lastOffsets; // Last offset we saw
int[] lastPositions; // Last position where this term occurred
-
+
@Override
ParallelPostingsArray newInstance(int size) {
return new TermVectorsPostingsArray(size);
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java?rev=1098427&r1=1098426&r2=1098427&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermVectorsWriter.java Sun May 1 22:38:33 2011
@@ -20,12 +20,13 @@ package org.apache.lucene.index;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.StringHelper;
import java.io.IOException;
final class TermVectorsWriter {
-
+
private IndexOutput tvx = null, tvd = null, tvf = null;
private FieldInfos fieldInfos;
@@ -46,7 +47,7 @@ final class TermVectorsWriter {
/**
* Add a complete document specified by all its term vectors. If document has no
* term vectors, add value for tvx.
- *
+ *
* @param vectors
* @throws IOException
*/
@@ -99,7 +100,7 @@ final class TermVectorsWriter {
final int[] freqs = vectors[i].getTermFrequencies();
for (int j=0; j<numTerms; j++) {
-
+
int start = j == 0 ? 0 : StringHelper.bytesDifference(terms[j-1].bytes,
terms[j-1].length,
terms[j].bytes,
@@ -181,30 +182,11 @@ final class TermVectorsWriter {
assert tvd.getFilePointer() == tvdPosition;
assert tvf.getFilePointer() == tvfPosition;
}
-
+
/** Close all streams. */
final void close() throws IOException {
// make an effort to close all streams we can but remember and re-throw
// the first exception encountered in this process
- IOException keep = null;
- if (tvx != null)
- try {
- tvx.close();
- } catch (IOException e) {
- keep = e;
- }
- if (tvd != null)
- try {
- tvd.close();
- } catch (IOException e) {
- if (keep == null) keep = e;
- }
- if (tvf != null)
- try {
- tvf.close();
- } catch (IOException e) {
- if (keep == null) keep = e;
- }
- if (keep != null) throw (IOException) keep.fillInStackTrace();
+ IOUtils.closeSafely(tvx, tvd, tvf);
}
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHash.java?rev=1098427&r1=1098426&r2=1098427&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHash.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHash.java Sun May 1 22:38:33 2011
@@ -18,12 +18,12 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
-import java.util.Collection;
import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
import java.util.Map;
+import org.apache.lucene.util.ByteBlockPool;
+import org.apache.lucene.util.BytesRef;
+
/** This class implements {@link InvertedDocConsumer}, which
* is passed each token produced by the analyzer on each
* field. It stores these tokens in a hash table, and
@@ -36,78 +36,118 @@ final class TermsHash extends InvertedDo
final TermsHashConsumer consumer;
final TermsHash nextTermsHash;
- final DocumentsWriter docWriter;
+ final DocumentsWriterPerThread docWriter;
+
+ final IntBlockPool intPool;
+ final ByteBlockPool bytePool;
+ ByteBlockPool termBytePool;
+
+ final boolean primary;
+ final DocumentsWriterPerThread.DocState docState;
+
+ // Used when comparing postings via termRefComp, in TermsHashPerField
+ final BytesRef tr1 = new BytesRef();
+ final BytesRef tr2 = new BytesRef();
- boolean trackAllocations;
+ // Used by perField to obtain terms from the analysis chain
+ final BytesRef termBytesRef = new BytesRef(10);
- public TermsHash(final DocumentsWriter docWriter, boolean trackAllocations, final TermsHashConsumer consumer, final TermsHash nextTermsHash) {
+ final boolean trackAllocations;
+
+
+ public TermsHash(final DocumentsWriterPerThread docWriter, final TermsHashConsumer consumer, boolean trackAllocations, final TermsHash nextTermsHash) {
+ this.docState = docWriter.docState;
this.docWriter = docWriter;
this.consumer = consumer;
+ this.trackAllocations = trackAllocations;
this.nextTermsHash = nextTermsHash;
- this.trackAllocations = trackAllocations;
+ intPool = new IntBlockPool(docWriter);
+ bytePool = new ByteBlockPool(docWriter.byteBlockAllocator);
+
+ if (nextTermsHash != null) {
+ // We are primary
+ primary = true;
+ termBytePool = bytePool;
+ nextTermsHash.termBytePool = bytePool;
+ } else {
+ primary = false;
+ }
}
@Override
- InvertedDocConsumerPerThread addThread(DocInverterPerThread docInverterPerThread) {
- return new TermsHashPerThread(docInverterPerThread, this, nextTermsHash, null);
+ public void abort() {
+ reset();
+ try {
+ consumer.abort();
+ } finally {
+ if (nextTermsHash != null) {
+ nextTermsHash.abort();
+ }
+ }
}
- TermsHashPerThread addThread(DocInverterPerThread docInverterPerThread, TermsHashPerThread primaryPerThread) {
- return new TermsHashPerThread(docInverterPerThread, this, nextTermsHash, primaryPerThread);
- }
+ // Clear all state
+ void reset() {
+ intPool.reset();
+ bytePool.reset();
- @Override
- public void abort() {
- consumer.abort();
- if (nextTermsHash != null)
- nextTermsHash.abort();
+ if (primary) {
+ bytePool.reset();
+ }
}
@Override
- synchronized void flush(Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException {
- Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> childThreadsAndFields = new HashMap<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>>();
- Map<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> nextThreadsAndFields;
-
- if (nextTermsHash != null)
- nextThreadsAndFields = new HashMap<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>>();
- else
- nextThreadsAndFields = null;
+ void flush(Map<FieldInfo,InvertedDocConsumerPerField> fieldsToFlush, final SegmentWriteState state) throws IOException {
+ Map<FieldInfo,TermsHashConsumerPerField> childFields = new HashMap<FieldInfo,TermsHashConsumerPerField>();
+ Map<FieldInfo,InvertedDocConsumerPerField> nextChildFields;
+
+ if (nextTermsHash != null) {
+ nextChildFields = new HashMap<FieldInfo,InvertedDocConsumerPerField>();
+ } else {
+ nextChildFields = null;
+ }
- for (final Map.Entry<InvertedDocConsumerPerThread,Collection<InvertedDocConsumerPerField>> entry : threadsAndFields.entrySet()) {
+ for (final Map.Entry<FieldInfo,InvertedDocConsumerPerField> entry : fieldsToFlush.entrySet()) {
+ TermsHashPerField perField = (TermsHashPerField) entry.getValue();
+ childFields.put(entry.getKey(), perField.consumer);
+ if (nextTermsHash != null) {
+ nextChildFields.put(entry.getKey(), perField.nextPerField);
+ }
+ }
- TermsHashPerThread perThread = (TermsHashPerThread) entry.getKey();
+ consumer.flush(childFields, state);
- Collection<InvertedDocConsumerPerField> fields = entry.getValue();
+ if (nextTermsHash != null) {
+ nextTermsHash.flush(nextChildFields, state);
+ }
+ }
- Iterator<InvertedDocConsumerPerField> fieldsIt = fields.iterator();
- Collection<TermsHashConsumerPerField> childFields = new HashSet<TermsHashConsumerPerField>();
- Collection<InvertedDocConsumerPerField> nextChildFields;
+ @Override
+ InvertedDocConsumerPerField addField(DocInverterPerField docInverterPerField, final FieldInfo fieldInfo) {
+ return new TermsHashPerField(docInverterPerField, this, nextTermsHash, fieldInfo);
+ }
- if (nextTermsHash != null)
- nextChildFields = new HashSet<InvertedDocConsumerPerField>();
- else
- nextChildFields = null;
+ @Override
+ public boolean freeRAM() {
+ return false;
+ }
- while(fieldsIt.hasNext()) {
- TermsHashPerField perField = (TermsHashPerField) fieldsIt.next();
- childFields.add(perField.consumer);
- if (nextTermsHash != null)
- nextChildFields.add(perField.nextPerField);
+ @Override
+ void finishDocument() throws IOException {
+ try {
+ consumer.finishDocument(this);
+ } finally {
+ if (nextTermsHash != null) {
+ nextTermsHash.consumer.finishDocument(nextTermsHash);
}
-
- childThreadsAndFields.put(perThread.consumer, childFields);
- if (nextTermsHash != null)
- nextThreadsAndFields.put(perThread.nextPerThread, nextChildFields);
}
-
- consumer.flush(childThreadsAndFields, state);
-
- if (nextTermsHash != null)
- nextTermsHash.flush(nextThreadsAndFields, state);
}
@Override
- synchronized public boolean freeRAM() {
- return false;
+ void startDocument() throws IOException {
+ consumer.startDocument();
+ if (nextTermsHash != null) {
+ nextTermsHash.consumer.startDocument();
+ }
}
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java?rev=1098427&r1=1098426&r2=1098427&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java Sun May 1 22:38:33 2011
@@ -18,11 +18,12 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
-import java.util.Collection;
import java.util.Map;
abstract class TermsHashConsumer {
- abstract TermsHashConsumerPerThread addThread(TermsHashPerThread perThread);
- abstract void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException;
+ abstract void flush(Map<FieldInfo, TermsHashConsumerPerField> 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);
+}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java?rev=1098427&r1=1098426&r2=1098427&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java Sun May 1 22:38:33 2011
@@ -34,9 +34,10 @@ final class TermsHashPerField extends In
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;
@@ -52,27 +53,27 @@ final class TermsHashPerField extends In
final FieldInfo fieldInfo;
final BytesRefHash bytesHash;
-
+
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);
+ 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;
}
@@ -80,7 +81,7 @@ final class TermsHashPerField extends In
void shrinkHash(int targetSize) {
// Fully free the bytesHash on each flush but keep the pool untouched
// bytesHash.clear will clear the ByteStartArray and in turn the ParallelPostingsArray too
- bytesHash.clear(false);
+ bytesHash.clear(false);
}
public void reset() {
@@ -90,7 +91,7 @@ final class TermsHashPerField extends In
}
@Override
- synchronized public void abort() {
+ public void abort() {
reset();
if (nextPerField != null)
nextPerField.abort();
@@ -99,14 +100,13 @@ final class TermsHashPerField extends In
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<BytesRef> termComp) {
return bytesHash.sort(termComp);
@@ -124,7 +124,7 @@ final class TermsHashPerField extends In
nextPerField.start(f);
}
}
-
+
@Override
boolean start(Fieldable[] fields, int count) throws IOException {
doCall = consumer.start(fields, count);
@@ -143,11 +143,12 @@ final class TermsHashPerField extends In
// 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 @@ final class TermsHashPerField extends In
} 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 @@ final class TermsHashPerField extends In
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 @@ final class TermsHashPerField extends In
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 @@ final class TermsHashPerField extends In
} 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);
}
@@ -278,7 +279,7 @@ final class TermsHashPerField extends In
if (nextPerField != null)
nextPerField.finish();
}
-
+
private static final class PostingsBytesStartArray extends BytesStartArray {
private final TermsHashPerField perField;
@@ -289,10 +290,10 @@ final class TermsHashPerField extends In
this.perField = perField;
this.bytesUsed = bytesUsed;
}
-
+
@Override
public int[] init() {
- if(perField.postingsArray == null) {
+ if(perField.postingsArray == null) {
perField.postingsArray = perField.consumer.createPostingsArray(2);
bytesUsed.addAndGet(perField.postingsArray.size * perField.postingsArray.bytesPerPosting());
}
@@ -312,7 +313,7 @@ final class TermsHashPerField extends In
@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;
Modified: lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java?rev=1098427&r1=1098426&r2=1098427&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/search/QueryUtils.java Sun May 1 22:38:33 2011
@@ -2,13 +2,14 @@ package org.apache.lucene.search;
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;
Modified: lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1098427&r1=1098426&r2=1098427&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/store/MockDirectoryWrapper.java Sun May 1 22:38:33 2011
@@ -34,6 +34,7 @@ import java.util.Set;
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 @@ public class MockDirectoryWrapper extend
private Set<String> createdFiles;
Set<String> openFilesForWrite = new HashSet<String>();
volatile boolean crashed;
+ private ThrottledIndexOutput throttledOutput;
// use this for tracking files for crash.
// additionally: provides debugging information in case you leave one open
@@ -114,6 +116,10 @@ public class MockDirectoryWrapper extend
public void setPreventDoubleWrite(boolean value) {
preventDoubleWrite = value;
}
+
+ public void setThrottledIndexOutput(ThrottledIndexOutput throttledOutput) {
+ this.throttledOutput = throttledOutput;
+ }
@Override
public synchronized void sync(Collection<String> names) throws IOException {
@@ -348,7 +354,7 @@ public class MockDirectoryWrapper extend
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 @@ public class MockDirectoryWrapper extend
maybeYield();
delegate.copy(to, src, dest);
}
+
}
Modified: lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java?rev=1098427&r1=1098426&r2=1098427&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java Sun May 1 22:38:33 2011
@@ -116,7 +116,7 @@ public abstract class LuceneTestCase ext
* If this is set, it is the only method that should run.
*/
static final String TEST_METHOD;
-
+
/** Create indexes in this directory, optimally use a subdir, named after the test */
public static final File TEMP_DIR;
static {
@@ -163,11 +163,11 @@ public abstract class LuceneTestCase ext
* multiply it by the number of iterations
*/
public static final int RANDOM_MULTIPLIER = Integer.parseInt(System.getProperty("tests.multiplier", "1"));
-
+
private int savedBoolMaxClauseCount;
private volatile Thread.UncaughtExceptionHandler savedUncaughtExceptionHandler = null;
-
+
/** Used to track if setUp and tearDown are called correctly from subclasses */
private boolean setup;
@@ -189,28 +189,28 @@ public abstract class LuceneTestCase ext
private static class UncaughtExceptionEntry {
public final Thread thread;
public final Throwable exception;
-
+
public UncaughtExceptionEntry(Thread thread, Throwable exception) {
this.thread = thread;
this.exception = exception;
}
}
private List<UncaughtExceptionEntry> uncaughtExceptions = Collections.synchronizedList(new ArrayList<UncaughtExceptionEntry>());
-
+
// saves default codec: we do this statically as many build indexes in @beforeClass
private static String savedDefaultCodec;
// default codec: not set when we use a per-field provider.
private static Codec codec;
// default codec provider
private static CodecProvider savedCodecProvider;
-
+
private static Locale locale;
private static Locale savedLocale;
private static TimeZone timeZone;
private static TimeZone savedTimeZone;
-
+
private static Map<MockDirectoryWrapper,StackTraceElement[]> stores;
-
+
private static final String[] TEST_CODECS = new String[] {"MockSep", "MockFixedIntBlock", "MockVariableIntBlock", "MockRandom"};
private static void swapCodec(Codec c, CodecProvider cp) {
@@ -288,7 +288,7 @@ public abstract class LuceneTestCase ext
// randomly picks from core and test codecs
static String pickRandomCodec(Random rnd) {
- int idx = rnd.nextInt(CodecProvider.CORE_CODECS.length +
+ int idx = rnd.nextInt(CodecProvider.CORE_CODECS.length +
TEST_CODECS.length);
if (idx < CodecProvider.CORE_CODECS.length) {
return CodecProvider.CORE_CODECS[idx];
@@ -321,7 +321,7 @@ public abstract class LuceneTestCase ext
/** @deprecated (4.0) until we fix no-fork problems in solr tests */
@Deprecated
private static List<String> testClassesRun = new ArrayList<String>();
-
+
@BeforeClass
public static void beforeClassLuceneTestCaseJ4() {
staticSeed = "random".equals(TEST_SEED) ? seedRand.nextLong() : TwoLongs.fromString(TEST_SEED).l1;
@@ -347,7 +347,7 @@ public abstract class LuceneTestCase ext
TimeZone.setDefault(timeZone);
testsFailed = false;
}
-
+
@AfterClass
public static void afterClassLuceneTestCaseJ4() {
if (! "false".equals(TEST_CLEAN_THREADS)) {
@@ -363,12 +363,12 @@ public abstract class LuceneTestCase ext
if ("randomPerField".equals(TEST_CODEC)) {
if (cp instanceof RandomCodecProvider)
codecDescription = cp.toString();
- else
+ else
codecDescription = "PreFlex";
} else {
codecDescription = codec.toString();
}
-
+
if (CodecProvider.getDefault() == savedCodecProvider)
removeTestCodecs(codec, CodecProvider.getDefault());
CodecProvider.setDefault(savedCodecProvider);
@@ -398,14 +398,14 @@ public abstract class LuceneTestCase ext
stores = null;
// if verbose or tests failed, report some information back
if (VERBOSE || testsFailed)
- System.err.println("NOTE: test params are: codec=" + codecDescription +
- ", locale=" + locale +
+ System.err.println("NOTE: test params are: codec=" + codecDescription +
+ ", locale=" + locale +
", timezone=" + (timeZone == null ? "(null)" : timeZone.getID()));
if (testsFailed) {
System.err.println("NOTE: all tests run in this JVM:");
System.err.println(Arrays.toString(testClassesRun.toArray()));
- System.err.println("NOTE: " + System.getProperty("os.name") + " "
- + System.getProperty("os.version") + " "
+ System.err.println("NOTE: " + System.getProperty("os.name") + " "
+ + System.getProperty("os.version") + " "
+ System.getProperty("os.arch") + "/"
+ System.getProperty("java.vendor") + " "
+ System.getProperty("java.version") + " "
@@ -428,7 +428,7 @@ public abstract class LuceneTestCase ext
}
private static boolean testsFailed; /* true if any tests failed */
-
+
// This is how we get control when errors occur.
// Think of this as start/end/success/failed
// events.
@@ -463,7 +463,7 @@ public abstract class LuceneTestCase ext
LuceneTestCase.this.name = method.getName();
super.starting(method);
}
-
+
};
@Before
@@ -481,7 +481,7 @@ public abstract class LuceneTestCase ext
savedUncaughtExceptionHandler.uncaughtException(t, e);
}
});
-
+
savedBoolMaxClauseCount = BooleanQuery.getMaxClauseCount();
}
@@ -513,7 +513,7 @@ public abstract class LuceneTestCase ext
if ("perMethod".equals(TEST_CLEAN_THREADS)) {
int rogueThreads = threadCleanup("test method: '" + getName() + "'");
if (rogueThreads > 0) {
- System.err.println("RESOURCE LEAK: test method: '" + getName()
+ System.err.println("RESOURCE LEAK: test method: '" + getName()
+ "' left " + rogueThreads + " thread(s) running");
// TODO: fail, but print seed for now.
if (!testsFailed && uncaughtExceptions.isEmpty()) {
@@ -535,18 +535,18 @@ public abstract class LuceneTestCase ext
fail("Some threads threw uncaught exceptions!");
}
- // calling assertSaneFieldCaches here isn't as useful as having test
- // classes call it directly from the scope where the index readers
- // are used, because they could be gc'ed just before this tearDown
+ // calling assertSaneFieldCaches here isn't as useful as having test
+ // classes call it directly from the scope where the index readers
+ // are used, because they could be gc'ed just before this tearDown
// method is called.
//
// But it's better then nothing.
//
- // If you are testing functionality that you know for a fact
- // "violates" FieldCache sanity, then you should either explicitly
+ // If you are testing functionality that you know for a fact
+ // "violates" FieldCache sanity, then you should either explicitly
// call purgeFieldCache at the end of your test method, or refactor
- // your Test class so that the inconsistant FieldCache usages are
- // isolated in distinct test methods
+ // your Test class so that the inconsistant FieldCache usages are
+ // isolated in distinct test methods
assertSaneFieldCaches(getTestLabel());
} finally {
@@ -557,14 +557,14 @@ public abstract class LuceneTestCase ext
private final static int THREAD_STOP_GRACE_MSEC = 50;
// jvm-wide list of 'rogue threads' we found, so they only get reported once.
private final static IdentityHashMap<Thread,Boolean> rogueThreads = new IdentityHashMap<Thread,Boolean>();
-
+
static {
// just a hack for things like eclipse test-runner threads
for (Thread t : Thread.getAllStackTraces().keySet()) {
rogueThreads.put(t, true);
}
}
-
+
/**
* Looks for leftover running threads, trying to kill them off,
* so they don't fail future tests.
@@ -575,20 +575,20 @@ public abstract class LuceneTestCase ext
Thread[] stillRunning = new Thread[Thread.activeCount()+1];
int threadCount = 0;
int rogueCount = 0;
-
+
if ((threadCount = Thread.enumerate(stillRunning)) > 1) {
while (threadCount == stillRunning.length) {
// truncated response
stillRunning = new Thread[stillRunning.length*2];
threadCount = Thread.enumerate(stillRunning);
}
-
+
for (int i = 0; i < threadCount; i++) {
Thread t = stillRunning[i];
-
- if (t.isAlive() &&
- !rogueThreads.containsKey(t) &&
- t != Thread.currentThread() &&
+
+ if (t.isAlive() &&
+ !rogueThreads.containsKey(t) &&
+ t != Thread.currentThread() &&
/* its ok to keep your searcher across test cases */
(t.getName().startsWith("LuceneTestCase") && context.startsWith("test method")) == false) {
System.err.println("WARNING: " + context + " left thread running: " + t);
@@ -613,7 +613,7 @@ public abstract class LuceneTestCase ext
}
return rogueCount;
}
-
+
/**
* Asserts that FieldCacheSanityChecker does not detect any
* problems with FieldCache.DEFAULT.
@@ -656,13 +656,13 @@ public abstract class LuceneTestCase ext
}
}
-
+
// @deprecated (4.0) These deprecated methods should be removed soon, when all tests using no Epsilon are fixed:
@Deprecated
static public void assertEquals(double expected, double actual) {
assertEquals(null, expected, actual);
}
-
+
@Deprecated
static public void assertEquals(String message, double expected, double actual) {
assertEquals(message, Double.valueOf(expected), Double.valueOf(actual));
@@ -677,18 +677,18 @@ public abstract class LuceneTestCase ext
static public void assertEquals(String message, float expected, float actual) {
assertEquals(message, Float.valueOf(expected), Float.valueOf(actual));
}
-
+
// Replacement for Assume jUnit class, so we can add a message with explanation:
-
+
private static final class TestIgnoredException extends RuntimeException {
TestIgnoredException(String msg) {
super(msg);
}
-
+
TestIgnoredException(String msg, Throwable t) {
super(msg, t);
}
-
+
@Override
public String getMessage() {
StringBuilder sb = new StringBuilder(super.getMessage());
@@ -696,7 +696,7 @@ public abstract class LuceneTestCase ext
sb.append(" - ").append(getCause());
return sb.toString();
}
-
+
// only this one is called by our code, exception is not used outside this class:
@Override
public void printStackTrace(PrintStream s) {
@@ -708,19 +708,19 @@ public abstract class LuceneTestCase ext
}
}
}
-
+
public static void assumeTrue(String msg, boolean b) {
Assume.assumeNoException(b ? null : new TestIgnoredException(msg));
}
-
+
public static void assumeFalse(String msg, boolean b) {
assumeTrue(msg, !b);
}
-
+
public static void assumeNoException(String msg, Exception e) {
Assume.assumeNoException(e == null ? null : new TestIgnoredException(msg, e));
}
-
+
public static <T> Set<T> asSet(T... args) {
return new HashSet<T>(Arrays.asList(args));
}
@@ -778,7 +778,7 @@ public abstract class LuceneTestCase ext
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()) {
@@ -864,7 +864,7 @@ public abstract class LuceneTestCase ext
public static MockDirectoryWrapper newDirectory() throws IOException {
return newDirectory(random);
}
-
+
/**
* Returns a new Directory instance, using the specified random.
* See {@link #newDirectory()} for more information.
@@ -875,7 +875,7 @@ public abstract class LuceneTestCase ext
stores.put(dir, Thread.currentThread().getStackTrace());
return dir;
}
-
+
/**
* Returns a new Directory instance, with contents copied from the
* provided directory. See {@link #newDirectory()} for more
@@ -884,23 +884,23 @@ public abstract class LuceneTestCase ext
public static MockDirectoryWrapper newDirectory(Directory d) throws IOException {
return newDirectory(random, d);
}
-
+
/** Returns a new FSDirectory instance over the given file, which must be a folder. */
public static MockDirectoryWrapper newFSDirectory(File f) throws IOException {
return newFSDirectory(f, null);
}
-
+
/** Returns a new FSDirectory instance over the given file, which must be a folder. */
public static MockDirectoryWrapper newFSDirectory(File f, LockFactory lf) throws IOException {
String fsdirClass = TEST_DIRECTORY;
if (fsdirClass.equals("random")) {
fsdirClass = FS_DIRECTORIES[random.nextInt(FS_DIRECTORIES.length)];
}
-
+
if (fsdirClass.indexOf(".") == -1) {// if not fully qualified, assume .store
fsdirClass = "org.apache.lucene.store." + fsdirClass;
}
-
+
Class<? extends FSDirectory> clazz;
try {
try {
@@ -908,11 +908,11 @@ public abstract class LuceneTestCase ext
} catch (ClassCastException e) {
// TEST_DIRECTORY is not a sub-class of FSDirectory, so draw one at random
fsdirClass = FS_DIRECTORIES[random.nextInt(FS_DIRECTORIES.length)];
-
+
if (fsdirClass.indexOf(".") == -1) {// if not fully qualified, assume .store
fsdirClass = "org.apache.lucene.store." + fsdirClass;
}
-
+
clazz = Class.forName(fsdirClass).asSubclass(FSDirectory.class);
}
MockDirectoryWrapper dir = new MockDirectoryWrapper(random, newFSDirectoryImpl(clazz, f, lf));
@@ -922,7 +922,7 @@ public abstract class LuceneTestCase ext
throw new RuntimeException(e);
}
}
-
+
/**
* Returns a new Directory instance, using the specified random
* with contents copied from the provided directory. See
@@ -980,44 +980,44 @@ public abstract class LuceneTestCase ext
public static Field newField(Random random, String name, String value, Store store, Index index, TermVector tv) {
if (!index.isIndexed())
return new Field(name, value, store, index);
-
+
if (!store.isStored() && random.nextBoolean())
store = Store.YES; // randomly store it
-
+
tv = randomTVSetting(random, tv);
-
+
return new Field(name, value, store, index, tv);
}
-
- static final TermVector tvSettings[] = {
- TermVector.NO, TermVector.YES, TermVector.WITH_OFFSETS,
- TermVector.WITH_POSITIONS, TermVector.WITH_POSITIONS_OFFSETS
+
+ static final TermVector tvSettings[] = {
+ TermVector.NO, TermVector.YES, TermVector.WITH_OFFSETS,
+ TermVector.WITH_POSITIONS, TermVector.WITH_POSITIONS_OFFSETS
};
-
+
private static TermVector randomTVSetting(Random random, TermVector minimum) {
switch(minimum) {
case NO: return tvSettings[_TestUtil.nextInt(random, 0, tvSettings.length-1)];
case YES: return tvSettings[_TestUtil.nextInt(random, 1, tvSettings.length-1)];
- case WITH_OFFSETS: return random.nextBoolean() ? TermVector.WITH_OFFSETS
+ case WITH_OFFSETS: return random.nextBoolean() ? TermVector.WITH_OFFSETS
: TermVector.WITH_POSITIONS_OFFSETS;
- case WITH_POSITIONS: return random.nextBoolean() ? TermVector.WITH_POSITIONS
+ case WITH_POSITIONS: return random.nextBoolean() ? TermVector.WITH_POSITIONS
: TermVector.WITH_POSITIONS_OFFSETS;
default: return TermVector.WITH_POSITIONS_OFFSETS;
}
}
-
+
/** return a random Locale from the available locales on the system */
public static Locale randomLocale(Random random) {
Locale locales[] = Locale.getAvailableLocales();
return locales[random.nextInt(locales.length)];
}
-
+
/** return a random TimeZone from the available timezones on the system */
public static TimeZone randomTimeZone(Random random) {
String tzIds[] = TimeZone.getAvailableIDs();
return TimeZone.getTimeZone(tzIds[random.nextInt(tzIds.length)]);
}
-
+
/** return a Locale object equivalent to its programmatic name */
public static Locale localeForName(String localeName) {
String elements[] = localeName.split("\\_");
@@ -1039,7 +1039,7 @@ public abstract class LuceneTestCase ext
"RAMDirectory",
FS_DIRECTORIES[0], FS_DIRECTORIES[1], FS_DIRECTORIES[2]
};
-
+
public static String randomDirectory(Random random) {
if (random.nextInt(10) == 0) {
return CORE_DIRECTORIES[random.nextInt(CORE_DIRECTORIES.length)];
@@ -1065,7 +1065,7 @@ public abstract class LuceneTestCase ext
}
return d;
}
-
+
static Directory newDirectoryImpl(Random random, String clazzName) {
if (clazzName.equals("random"))
clazzName = randomDirectory(random);
@@ -1086,9 +1086,9 @@ public abstract class LuceneTestCase ext
return clazz.newInstance();
} catch (Exception e) {
throw new RuntimeException(e);
- }
+ }
}
-
+
/** create a new searcher over the reader.
* This searcher might randomly use threads. */
public static IndexSearcher newSearcher(IndexReader r) throws IOException {
@@ -1109,8 +1109,8 @@ public abstract class LuceneTestCase ext
}
} else {
int threads = 0;
- final ExecutorService ex = (random.nextBoolean()) ? null
- : Executors.newFixedThreadPool(threads = _TestUtil.nextInt(random, 1, 8),
+ final ExecutorService ex = (random.nextBoolean()) ? null
+ : Executors.newFixedThreadPool(threads = _TestUtil.nextInt(random, 1, 8),
new NamedThreadFactory("LuceneTestCase"));
if (ex != null && VERBOSE) {
System.out.println("NOTE: newSearcher using ExecutorService with " + threads + " threads");
@@ -1135,12 +1135,12 @@ public abstract class LuceneTestCase ext
public String getName() {
return this.name;
}
-
+
/** Gets a resource from the classpath as {@link File}. This method should only be used,
* if a real file is needed. To get a stream, code should prefer
* {@link Class#getResourceAsStream} using {@code this.getClass()}.
*/
-
+
protected File getDataFile(String name) throws IOException {
try {
return new File(this.getClass().getResource(name).toURI());
@@ -1151,11 +1151,11 @@ public abstract class LuceneTestCase ext
// We get here from InterceptTestCaseEvents on the 'failed' event....
public void reportAdditionalFailureInfo() {
- System.err.println("NOTE: reproduce with: ant test -Dtestcase=" + getClass().getSimpleName()
+ System.err.println("NOTE: reproduce with: ant test -Dtestcase=" + getClass().getSimpleName()
+ " -Dtestmethod=" + getName() + " -Dtests.seed=" + new TwoLongs(staticSeed, seed)
+ reproduceWithExtraParams());
}
-
+
// extra params that were overridden needed to reproduce the command
private String reproduceWithExtraParams() {
StringBuilder sb = new StringBuilder();
@@ -1171,12 +1171,12 @@ public abstract class LuceneTestCase ext
private static long staticSeed;
// seed for individual test methods, changed in @before
private long seed;
-
+
private static final Random seedRand = new Random();
protected static final Random random = new Random(0);
private String name = "<unknown>";
-
+
/**
* Annotation for tests that should only be run during nightly builds.
*/
@@ -1184,7 +1184,7 @@ public abstract class LuceneTestCase ext
@Inherited
@Retention(RetentionPolicy.RUNTIME)
public @interface Nightly {}
-
+
/** optionally filters the tests to be run by TEST_METHOD */
public static class LuceneTestCaseRunner extends BlockJUnit4ClassRunner {
private List<FrameworkMethod> testMethods;
@@ -1214,11 +1214,11 @@ public abstract class LuceneTestCase ext
testMethods.add(new FrameworkMethod(m));
}
}
-
+
if (testMethods.isEmpty()) {
throw new RuntimeException("No runnable methods!");
}
-
+
if (TEST_NIGHTLY == false) {
if (getTestClass().getJavaClass().isAnnotationPresent(Nightly.class)) {
/* the test class is annotated with nightly, remove all methods */
@@ -1279,9 +1279,9 @@ public abstract class LuceneTestCase ext
@Override
public boolean shouldRun(Description d) {
return TEST_METHOD == null || d.getMethodName().equals(TEST_METHOD);
- }
+ }
};
-
+
try {
f.apply(this);
} catch (NoTestsRemainException e) {
@@ -1289,12 +1289,12 @@ public abstract class LuceneTestCase ext
}
}
}
-
+
private static class RandomCodecProvider extends CodecProvider {
private List<Codec> knownCodecs = new ArrayList<Codec>();
private Map<String,Codec> previousMappings = new HashMap<String,Codec>();
private final int perFieldSeed;
-
+
RandomCodecProvider(Random random) {
this.perFieldSeed = random.nextInt();
register(new StandardCodec());
@@ -1326,13 +1326,13 @@ public abstract class LuceneTestCase ext
}
return codec.name;
}
-
+
@Override
public synchronized String toString() {
return "RandomCodecProvider: " + previousMappings.toString();
}
}
-
+
@Ignore("just a hack")
public final void alwaysIgnoredTestMethod() {}
}
Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java?rev=1098427&r1=1098426&r2=1098427&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestConcurrentMergeScheduler.java Sun May 1 22:38:33 2011
@@ -50,7 +50,7 @@ public class TestConcurrentMergeSchedule
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())) {