You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by rm...@apache.org on 2011/11/16 20:09:42 UTC
svn commit: r1202842 [3/6] - in /lucene/dev/trunk: ./ dev-tools/eclipse/
dev-tools/idea/lucene/contrib/ dev-tools/idea/lucene/contrib/instantiated/
dev-tools/maven/lucene/contrib/
dev-tools/maven/lucene/contrib/instantiated/ lucene/ lucene/contrib/ luc...
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=1202842&r1=1202841&r2=1202842&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 Wed Nov 16 19:09:35 2011
@@ -25,12 +25,12 @@ import java.util.List;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.IndexReader.FieldOption;
-import org.apache.lucene.index.MergePolicy.MergeAbortedException;
import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.FieldInfosWriter;
import org.apache.lucene.index.codecs.FieldsConsumer;
import org.apache.lucene.index.codecs.StoredFieldsWriter;
import org.apache.lucene.index.codecs.PerDocConsumer;
-import org.apache.lucene.store.CompoundFileDirectory;
+import org.apache.lucene.index.codecs.TermVectorsWriter;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexOutput;
@@ -52,10 +52,6 @@ final class SegmentMerger {
private final String segment;
private final int termIndexInterval;
- /** Maximum number of contiguous documents to bulk-copy
- when merging term vectors */
- private final static int MAX_RAW_MERGE_DOCS = 4192;
-
private final Codec codec;
private final IOContext context;
@@ -128,41 +124,15 @@ final class SegmentMerger {
mergeNorms();
if (mergeState.fieldInfos.hasVectors()) {
- mergeVectors(segmentWriteState);
+ int numMerged = mergeVectors();
+ assert numMerged == mergeState.mergedDocCount;
}
// write FIS once merge is done. IDV might change types or drops fields
- mergeState.fieldInfos.write(directory, segment + "." + IndexFileNames.FIELD_INFOS_EXTENSION);
+ FieldInfosWriter fieldInfosWriter = codec.fieldInfosFormat().getFieldInfosWriter();
+ fieldInfosWriter.write(directory, segment, mergeState.fieldInfos, context);
return mergeState;
}
- /**
- * NOTE: this method creates a compound file for all files returned by
- * info.files(). While, generally, this may include separate norms and
- * deletion files, this SegmentInfo must not reference such files when this
- * method is called, because they are not allowed within a compound file.
- */
- final Collection<String> createCompoundFile(String fileName, final SegmentInfo info, IOContext context)
- throws IOException {
-
- // Now merge all added files
- Collection<String> files = info.files();
- CompoundFileDirectory cfsDir = new CompoundFileDirectory(directory, fileName, context, true);
- try {
- for (String file : files) {
- assert !IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION)
- : ".del file is not allowed in .cfs: " + file;
- assert !IndexFileNames.isSeparateNormsFile(file)
- : "separate norms file (.s[0-9]+) is not allowed in .cfs: " + file;
- directory.copy(cfsDir, file, file, context);
- mergeState.checkAbort.work(directory.fileLength(file));
- }
- } finally {
- cfsDir.close();
- }
-
- return files;
- }
-
private static void addIndexed(IndexReader reader, FieldInfos fInfos,
Collection<String> names, boolean storeTermVectors,
boolean storePositionWithTermVector, boolean storeOffsetWithTermVector,
@@ -244,138 +214,27 @@ final class SegmentMerger {
* @throws IOException if there is a low-level IO error
*/
private int mergeFields() throws CorruptIndexException, IOException {
- int docCount = 0;
-
final StoredFieldsWriter fieldsWriter = codec.storedFieldsFormat().fieldsWriter(directory, segment, context);
+
try {
- docCount = fieldsWriter.merge(mergeState);
+ return fieldsWriter.merge(mergeState);
} finally {
fieldsWriter.close();
}
-
- return docCount;
}
/**
* Merge the TermVectors from each of the segments into the new one.
* @throws IOException
*/
- private final void mergeVectors(SegmentWriteState segmentWriteState) throws IOException {
- TermVectorsWriter termVectorsWriter = new TermVectorsWriter(directory, segment, mergeState.fieldInfos, context);
- // Used for bulk-reading raw bytes for term vectors
- int rawDocLengths[] = new int[MAX_RAW_MERGE_DOCS];
- int rawDocLengths2[] = new int[MAX_RAW_MERGE_DOCS];
+ private final int mergeVectors() throws IOException {
+ final TermVectorsWriter termVectorsWriter = codec.termVectorsFormat().vectorsWriter(directory, segment, context);
+
try {
- int idx = 0;
- for (final MergeState.IndexReaderAndLiveDocs reader : mergeState.readers) {
- final SegmentReader matchingSegmentReader = mergeState.matchingSegmentReaders[idx++];
- TermVectorsReader matchingVectorsReader = null;
- if (matchingSegmentReader != null) {
- TermVectorsReader vectorsReader = matchingSegmentReader.getTermVectorsReader();
-
- // If the TV* files are an older format then they cannot read raw docs:
- if (vectorsReader != null && vectorsReader.canReadRawDocs()) {
- matchingVectorsReader = vectorsReader;
- }
- }
- if (reader.liveDocs != null) {
- copyVectorsWithDeletions(termVectorsWriter, matchingVectorsReader, reader, rawDocLengths, rawDocLengths2);
- } else {
- copyVectorsNoDeletions(termVectorsWriter, matchingVectorsReader, reader, rawDocLengths, rawDocLengths2);
- }
- }
+ return termVectorsWriter.merge(mergeState);
} finally {
termVectorsWriter.close();
}
-
- final String fileName = IndexFileNames.segmentFileName(segment, "", IndexFileNames.VECTORS_INDEX_EXTENSION);
- final long tvxSize = directory.fileLength(fileName);
- final int mergedDocs = segmentWriteState.numDocs;
-
- if (4+((long) mergedDocs)*16 != tvxSize)
- // This is most likely a bug in Sun JRE 1.6.0_04/_05;
- // we detect that the bug has struck, here, and
- // throw an exception to prevent the corruption from
- // entering the index. See LUCENE-1282 for
- // details.
- throw new RuntimeException("mergeVectors produced an invalid result: mergedDocs is " + mergedDocs + " but tvx size is " + tvxSize + " file=" + fileName + " file exists?=" + directory.fileExists(fileName) + "; now aborting this merge to prevent index corruption");
- }
-
- private void copyVectorsWithDeletions(final TermVectorsWriter termVectorsWriter,
- final TermVectorsReader matchingVectorsReader,
- final MergeState.IndexReaderAndLiveDocs reader,
- int rawDocLengths[],
- int rawDocLengths2[])
- throws IOException, MergeAbortedException {
- final int maxDoc = reader.reader.maxDoc();
- final Bits liveDocs = reader.liveDocs;
- if (matchingVectorsReader != null) {
- // We can bulk-copy because the fieldInfos are "congruent"
- for (int docNum = 0; docNum < maxDoc;) {
- if (!liveDocs.get(docNum)) {
- // skip deleted docs
- ++docNum;
- continue;
- }
- // We can optimize this case (doing a bulk byte copy) since the field
- // numbers are identical
- int start = docNum, numDocs = 0;
- do {
- docNum++;
- numDocs++;
- if (docNum >= maxDoc) break;
- if (!liveDocs.get(docNum)) {
- docNum++;
- break;
- }
- } while(numDocs < MAX_RAW_MERGE_DOCS);
-
- matchingVectorsReader.rawDocs(rawDocLengths, rawDocLengths2, start, numDocs);
- termVectorsWriter.addRawDocuments(matchingVectorsReader, rawDocLengths, rawDocLengths2, numDocs);
- mergeState.checkAbort.work(300 * numDocs);
- }
- } else {
- for (int docNum = 0; docNum < maxDoc; docNum++) {
- if (!liveDocs.get(docNum)) {
- // 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.reader.getTermFreqVectors(docNum);
- termVectorsWriter.addAllDocVectors(vectors);
- mergeState.checkAbort.work(300);
- }
- }
- }
-
- private void copyVectorsNoDeletions(final TermVectorsWriter termVectorsWriter,
- final TermVectorsReader matchingVectorsReader,
- final MergeState.IndexReaderAndLiveDocs reader,
- int rawDocLengths[],
- int rawDocLengths2[])
- throws IOException, MergeAbortedException {
- final int maxDoc = reader.reader.maxDoc();
- if (matchingVectorsReader != null) {
- // We can bulk-copy because the fieldInfos are "congruent"
- int docCount = 0;
- while (docCount < maxDoc) {
- int len = Math.min(MAX_RAW_MERGE_DOCS, maxDoc - docCount);
- matchingVectorsReader.rawDocs(rawDocLengths, rawDocLengths2, docCount, len);
- termVectorsWriter.addRawDocuments(matchingVectorsReader, rawDocLengths, rawDocLengths2, len);
- docCount += len;
- mergeState.checkAbort.work(300 * len);
- }
- } else {
- for (int docNum = 0; docNum < maxDoc; docNum++) {
- // NOTE: it's very important to first assign to vectors then pass it to
- // termVectorsWriter.addAllDocVectors; see LUCENE-1282
- TermFreqVector[] vectors = reader.reader.getTermFreqVectors(docNum);
- termVectorsWriter.addAllDocVectors(vectors);
- mergeState.checkAbort.work(300);
- }
- }
}
private final void mergeTerms(SegmentWriteState segmentWriteState) throws CorruptIndexException, IOException {
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java Wed Nov 16 19:09:35 2011
@@ -32,6 +32,7 @@ import org.apache.lucene.store.IndexInpu
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.codecs.StoredFieldsReader;
import org.apache.lucene.index.codecs.PerDocValues;
+import org.apache.lucene.index.codecs.TermVectorsReader;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.Bits;
@@ -659,19 +660,16 @@ public class SegmentReader extends Index
/**
* Create a clone from the initial TermVectorsReader and store it in the ThreadLocal.
* @return TermVectorsReader
+ * @lucene.internal
*/
- TermVectorsReader getTermVectorsReader() {
+ public TermVectorsReader getTermVectorsReader() {
TermVectorsReader tvReader = termVectorsLocal.get();
if (tvReader == null) {
TermVectorsReader orig = core.getTermVectorsReaderOrig();
if (orig == null) {
return null;
} else {
- try {
- tvReader = (TermVectorsReader) orig.clone();
- } catch (CloneNotSupportedException cnse) {
- return null;
- }
+ tvReader = orig.clone();
}
termVectorsLocal.set(tvReader);
}
@@ -689,67 +687,15 @@ public class SegmentReader extends Index
* @throws IOException
*/
@Override
- public TermFreqVector getTermFreqVector(int docNumber, String field) throws IOException {
- // Check if this field is invalid or has no stored term vector
+ public Fields getTermVectors(int docID) throws IOException {
ensureOpen();
- FieldInfo fi = core.fieldInfos.fieldInfo(field);
- if (fi == null || !fi.storeTermVector)
- return null;
-
- TermVectorsReader termVectorsReader = getTermVectorsReader();
- if (termVectorsReader == null)
- return null;
-
- return termVectorsReader.get(docNumber, field);
- }
-
-
- @Override
- public void getTermFreqVector(int docNumber, String field, TermVectorMapper mapper) throws IOException {
- ensureOpen();
- FieldInfo fi = core.fieldInfos.fieldInfo(field);
- if (fi == null || !fi.storeTermVector)
- return;
-
TermVectorsReader termVectorsReader = getTermVectorsReader();
if (termVectorsReader == null) {
- return;
+ return null;
}
-
-
- termVectorsReader.get(docNumber, field, mapper);
+ return termVectorsReader.get(docID);
}
-
- @Override
- public void getTermFreqVector(int docNumber, TermVectorMapper mapper) throws IOException {
- ensureOpen();
-
- TermVectorsReader termVectorsReader = getTermVectorsReader();
- if (termVectorsReader == null)
- return;
-
- termVectorsReader.get(docNumber, mapper);
- }
-
- /** Return an array of term frequency vectors for the specified document.
- * The array contains a vector for each vectorized field in the document.
- * Each vector vector contains term numbers and frequencies for all terms
- * in a given vectorized field.
- * If no such fields existed, the method returns null.
- * @throws IOException
- */
- @Override
- public TermFreqVector[] getTermFreqVectors(int docNumber) throws IOException {
- ensureOpen();
-
- TermVectorsReader termVectorsReader = getTermVectorsReader();
- if (termVectorsReader == null)
- return null;
-
- return termVectorsReader.get(docNumber);
- }
-
/** {@inheritDoc} */
@Override
public String toString() {
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsConsumer.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsConsumer.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/StoredFieldsConsumer.java Wed Nov 16 19:09:35 2011
@@ -45,12 +45,12 @@ final class StoredFieldsConsumer {
private int numStoredFields;
private IndexableField[] storedFields;
- private int[] fieldNumbers;
+ private FieldInfo[] fieldInfos;
public void reset() {
numStoredFields = 0;
storedFields = new IndexableField[1];
- fieldNumbers = new int[1];
+ fieldInfos = new FieldInfo[1];
}
public void startDocument() {
@@ -116,7 +116,7 @@ final class StoredFieldsConsumer {
if (fieldsWriter != null && numStoredFields > 0) {
fieldsWriter.startDocument(numStoredFields);
for (int i = 0; i < numStoredFields; i++) {
- fieldsWriter.writeField(fieldNumbers[i], storedFields[i]);
+ fieldsWriter.writeField(fieldInfos[i], storedFields[i]);
}
lastDocID++;
}
@@ -131,14 +131,14 @@ final class StoredFieldsConsumer {
IndexableField[] newArray = new IndexableField[newSize];
System.arraycopy(storedFields, 0, newArray, 0, numStoredFields);
storedFields = newArray;
- }
-
- if (numStoredFields == fieldNumbers.length) {
- fieldNumbers = ArrayUtil.grow(fieldNumbers);
+
+ FieldInfo[] newInfoArray = new FieldInfo[newSize];
+ System.arraycopy(fieldInfos, 0, newInfoArray, 0, numStoredFields);
+ fieldInfos = newInfoArray;
}
storedFields[numStoredFields] = field;
- fieldNumbers[numStoredFields] = fieldInfo.number;
+ fieldInfos[numStoredFields] = fieldInfo;
numStoredFields++;
assert docState.testPoint("StoredFieldsWriterPerThread.processFields.writeField");
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/Terms.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/Terms.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/Terms.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/Terms.java Wed Nov 16 19:09:35 2011
@@ -37,8 +37,11 @@ public abstract class Terms {
private final CloseableThreadLocal<TermsEnum> threadEnums = new CloseableThreadLocal<TermsEnum>();
/** Returns an iterator that will step through all
- * terms. This method will not return null.*/
- public abstract TermsEnum iterator() throws IOException;
+ * terms. This method will not return null. If you have
+ * a previous TermsEnum, for example from a different
+ * field, you can pass it for possible reuse if the
+ * implementation can do so. */
+ public abstract TermsEnum iterator(TermsEnum reuse) throws IOException;
/** Returns a TermsEnum that iterates over all terms that
* are accepted by the provided {@link
@@ -59,9 +62,9 @@ public abstract class Terms {
throw new IllegalArgumentException("please use CompiledAutomaton.getTermsEnum instead");
}
if (startTerm == null) {
- return new AutomatonTermsEnum(iterator(), compiled);
+ return new AutomatonTermsEnum(iterator(null), compiled);
} else {
- return new AutomatonTermsEnum(iterator(), compiled) {
+ return new AutomatonTermsEnum(iterator(null), compiled) {
@Override
protected BytesRef nextSeekTerm(BytesRef term) throws IOException {
if (term == null) {
@@ -186,7 +189,7 @@ public abstract class Terms {
/**
* Returns a thread-private {@link TermsEnum} instance. Obtaining
* {@link TermsEnum} from this method might be more efficient than using
- * {@link #iterator()} directly since this method doesn't necessarily create a
+ * {@link #iterator(TermsEnum)} directly since this method doesn't necessarily create a
* new {@link TermsEnum} instance.
* <p>
* NOTE: {@link TermsEnum} instances obtained from this method must not be
@@ -201,7 +204,7 @@ public abstract class Terms {
public TermsEnum getThreadTermsEnum() throws IOException {
TermsEnum termsEnum = threadEnums.get();
if (termsEnum == null) {
- termsEnum = iterator();
+ termsEnum = iterator(null);
threadEnums.set(termsEnum);
}
return termsEnum;
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=1202842&r1=1202841&r2=1202842&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 Wed Nov 16 19:09:35 2011
@@ -29,7 +29,7 @@ import org.apache.lucene.util.BytesRef;
* field. It stores these tokens in a hash table, and
* allocates separate byte streams per token. Consumers of
* this class, eg {@link FreqProxTermsWriter} and {@link
- * TermVectorsTermsWriter}, write their own byte streams
+ * TermVectorsConsumer}, write their own byte streams
* under each term.
*/
final class TermsHash extends InvertedDocConsumer {
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/TieredMergePolicy.java Wed Nov 16 19:09:35 2011
@@ -338,7 +338,7 @@ public class TieredMergePolicy extends M
for(int idx = tooBigCount; idx<infosSorted.size(); idx++) {
final SegmentInfo info = infosSorted.get(idx);
if (merging.contains(info)) {
- mergingBytes += info.sizeInBytes(true);
+ mergingBytes += info.sizeInBytes();
} else if (!toBeMerged.contains(info)) {
eligible.add(info);
}
@@ -430,7 +430,7 @@ public class TieredMergePolicy extends M
final long segBytes = size(info);
totAfterMergeBytes += segBytes;
totAfterMergeBytesFloored += floorSize(segBytes);
- totBeforeMergeBytes += info.sizeInBytes(true);
+ totBeforeMergeBytes += info.sizeInBytes();
}
// Measure "skew" of the merge, which can range
@@ -632,7 +632,7 @@ public class TieredMergePolicy extends M
// Segment size in bytes, pro-rated by % deleted
private long size(SegmentInfo info) throws IOException {
- final long byteSize = info.sizeInBytes(true);
+ final long byteSize = info.sizeInBytes();
final int delCount = writer.get().numDeletedDocs(info);
final double delRatio = (info.docCount <= 0 ? 0.0f : ((double)delCount / (double)info.docCount));
assert delRatio <= 1.0;
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java Wed Nov 16 19:09:35 2011
@@ -130,7 +130,6 @@ public class BlockTermsReader extends Fi
seekDir(in, dirOffset);
final int numFields = in.readVInt();
-
for(int i=0;i<numFields;i++) {
final int field = in.readVInt();
final long numTerms = in.readVLong();
@@ -208,6 +207,11 @@ public class BlockTermsReader extends Fi
return fields.get(field);
}
+ @Override
+ public int getUniqueFieldCount() {
+ return fields.size();
+ }
+
// Iterates through all fields
private class TermFieldsEnum extends FieldsEnum {
final Iterator<FieldReader> it;
@@ -229,8 +233,8 @@ public class BlockTermsReader extends Fi
}
@Override
- public TermsEnum terms() throws IOException {
- return current.iterator();
+ public Terms terms() throws IOException {
+ return current;
}
}
@@ -263,7 +267,7 @@ public class BlockTermsReader extends Fi
}
@Override
- public TermsEnum iterator() throws IOException {
+ public TermsEnum iterator(TermsEnum reuse) throws IOException {
return new SegmentTermsEnum();
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/BlockTreeTermsReader.java Wed Nov 16 19:09:35 2011
@@ -19,11 +19,8 @@ package org.apache.lucene.index.codecs;
import java.io.ByteArrayOutputStream;
import java.io.Closeable;
-import java.io.FileOutputStream;
import java.io.IOException;
-import java.io.OutputStreamWriter;
import java.io.PrintStream;
-import java.io.Writer;
import java.util.Collection;
import java.util.Comparator;
import java.util.Iterator;
@@ -221,6 +218,11 @@ public class BlockTreeTermsReader extend
return fields.get(field);
}
+ @Override
+ public int getUniqueFieldCount() {
+ return fields.size();
+ }
+
// Iterates through all fields
private class TermFieldsEnum extends FieldsEnum {
final Iterator<FieldReader> it;
@@ -242,8 +244,8 @@ public class BlockTreeTermsReader extend
}
@Override
- public TermsEnum terms() throws IOException {
- return current.iterator();
+ public Terms terms() throws IOException {
+ return current;
}
}
@@ -454,7 +456,7 @@ public class BlockTreeTermsReader extend
}
@Override
- public TermsEnum iterator() throws IOException {
+ public TermsEnum iterator(TermsEnum reuse) throws IOException {
return new SegmentTermsEnum();
}
@@ -914,8 +916,6 @@ public class BlockTreeTermsReader extend
for(int idx=0;idx<=target.length;idx++) {
- boolean lastIsSubBlock = false;
-
while (true) {
final int savePos = currentFrame.suffixesReader.getPosition();
final int saveStartBytePos = currentFrame.startBytePos;
@@ -950,7 +950,6 @@ public class BlockTreeTermsReader extend
return;
}
}
- lastIsSubBlock = isSubBlock;
continue;
} else if (cmp == 0) {
//if (DEBUG) System.out.println(" return term=" + brToString(term));
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/Codec.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/Codec.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/Codec.java Wed Nov 16 19:09:35 2011
@@ -46,6 +46,10 @@ public abstract class Codec implements N
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
postingsFormat().files(dir, info, "", files);
storedFieldsFormat().files(dir, info, files);
+ termVectorsFormat().files(dir, info, files);
+ fieldInfosFormat().files(dir, info, files);
+ // TODO: segmentInfosFormat should be allowed to declare additional files
+ // if it wants, in addition to segments_N
docValuesFormat().files(dir, info, files);
}
@@ -58,6 +62,12 @@ public abstract class Codec implements N
/** Encodes/decodes stored fields */
public abstract StoredFieldsFormat storedFieldsFormat();
+ /** Encodes/decodes term vectors */
+ public abstract TermVectorsFormat termVectorsFormat();
+
+ /** Encodes/decodes field infos file */
+ public abstract FieldInfosFormat fieldInfosFormat();
+
/** Encodes/decodes segments file */
public abstract SegmentInfosFormat segmentInfosFormat();
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java Wed Nov 16 19:09:35 2011
@@ -18,6 +18,8 @@ package org.apache.lucene.index.codecs;
*/
import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexFormatTooOldException;
@@ -51,7 +53,7 @@ public class DefaultSegmentInfosReader e
infos.setGlobalFieldMapVersion(input.readLong());
}
for (int i = input.readInt(); i > 0; i--) { // read segmentInfos
- SegmentInfo si = new SegmentInfo(directory, format, input);
+ SegmentInfo si = readSegmentInfo(directory, format, input);
if (si.getVersion() == null) {
// Could be a 3.0 - try to open the doc stores - if it fails, it's a
// 2.x segment, and an IndexFormatTooOldException will be thrown,
@@ -90,4 +92,100 @@ public class DefaultSegmentInfosReader e
infos.userData = input.readStringStringMap();
}
+
+ // if we make a preflex impl we can remove a lot of this hair...
+ public SegmentInfo readSegmentInfo(Directory dir, int format, ChecksumIndexInput input) throws IOException {
+ final String version;
+ if (format <= DefaultSegmentInfosWriter.FORMAT_3_1) {
+ version = input.readString();
+ } else {
+ version = null;
+ }
+ final String name = input.readString();
+ final int docCount = input.readInt();
+ final long delGen = input.readLong();
+ final int docStoreOffset = input.readInt();
+ final String docStoreSegment;
+ final boolean docStoreIsCompoundFile;
+ if (docStoreOffset != -1) {
+ docStoreSegment = input.readString();
+ docStoreIsCompoundFile = input.readByte() == SegmentInfo.YES;
+ } else {
+ 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;
+ }
+
+ final int numNormGen = input.readInt();
+ final Map<Integer,Long> normGen;
+ if (numNormGen == SegmentInfo.NO) {
+ normGen = null;
+ } else {
+ normGen = new HashMap<Integer, Long>();
+ for(int j=0;j<numNormGen;j++) {
+ int fieldNumber = j;
+ if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
+ fieldNumber = input.readInt();
+ }
+
+ normGen.put(fieldNumber, input.readLong());
+ }
+ }
+ final boolean isCompoundFile = input.readByte() == SegmentInfo.YES;
+
+ final int delCount = input.readInt();
+ assert delCount <= docCount;
+
+ final int hasProx = input.readByte();
+
+ final Codec codec;
+ // note: if the codec is not available: Codec.forName will throw an exception.
+ if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
+ codec = Codec.forName(input.readString());
+ } else {
+ codec = Codec.forName("Lucene3x");
+ }
+ final Map<String,String> diagnostics = input.readStringStringMap();
+
+ final int hasVectors;
+ if (format <= DefaultSegmentInfosWriter.FORMAT_HAS_VECTORS) {
+ hasVectors = input.readByte();
+ } else {
+ final String storesSegment;
+ final String ext;
+ final boolean storeIsCompoundFile;
+ if (docStoreOffset != -1) {
+ storesSegment = docStoreSegment;
+ storeIsCompoundFile = docStoreIsCompoundFile;
+ ext = IndexFileNames.COMPOUND_FILE_STORE_EXTENSION;
+ } else {
+ storesSegment = name;
+ storeIsCompoundFile = isCompoundFile;
+ ext = IndexFileNames.COMPOUND_FILE_EXTENSION;
+ }
+ final Directory dirToTest;
+ if (storeIsCompoundFile) {
+ dirToTest = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(storesSegment, "", ext), IOContext.READONCE, false);
+ } else {
+ dirToTest = dir;
+ }
+ try {
+ // TODO: remove this manual file check or push to preflex codec
+ hasVectors = dirToTest.fileExists(IndexFileNames.segmentFileName(storesSegment, "", DefaultTermVectorsReader.VECTORS_INDEX_EXTENSION)) ? SegmentInfo.YES : SegmentInfo.NO;
+ } finally {
+ if (isCompoundFile) {
+ dirToTest.close();
+ }
+ }
+ }
+
+ return new SegmentInfo(dir, version, name, docCount, delGen, docStoreOffset,
+ docStoreSegment, docStoreIsCompoundFile, normGen, isCompoundFile,
+ delCount, hasProx, codec, diagnostics, hasVectors);
+ }
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java Wed Nov 16 19:09:35 2011
@@ -18,6 +18,8 @@ package org.apache.lucene.index.codecs;
*/
import java.io.IOException;
+import java.util.Map;
+import java.util.Map.Entry;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentInfos;
@@ -50,6 +52,7 @@ public class DefaultSegmentInfosWriter e
/** This must always point to the most recent file format.
* whenever you add a new format, make it 1 smaller (negative version logic)! */
+ // TODO: move this, as its currently part of required preamble
public static final int FORMAT_CURRENT = FORMAT_4_0;
/** This must always point to the first supported file format. */
@@ -68,7 +71,7 @@ public class DefaultSegmentInfosWriter e
out.writeLong(infos.getGlobalFieldMapVersion());
out.writeInt(infos.size()); // write infos
for (SegmentInfo si : infos) {
- si.write(out);
+ writeInfo(out, si);
}
out.writeStringStringMap(infos.getUserData());
success = true;
@@ -80,6 +83,40 @@ public class DefaultSegmentInfosWriter e
}
}
+ /** Save a single segment's info. */
+ private void writeInfo(IndexOutput output, SegmentInfo si) throws IOException {
+ assert si.getDelCount() <= si.docCount: "delCount=" + si.getDelCount() + " docCount=" + si.docCount + " segment=" + si.name;
+ // Write the Lucene version that created this segment, since 3.1
+ output.writeString(si.getVersion());
+ output.writeString(si.name);
+ output.writeInt(si.docCount);
+ output.writeLong(si.getDelGen());
+
+ output.writeInt(si.getDocStoreOffset());
+ if (si.getDocStoreOffset() != -1) {
+ output.writeString(si.getDocStoreSegment());
+ output.writeByte((byte) (si.getDocStoreIsCompoundFile() ? 1:0));
+ }
+
+ Map<Integer,Long> normGen = si.getNormGen();
+ if (normGen == null) {
+ output.writeInt(SegmentInfo.NO);
+ } else {
+ output.writeInt(normGen.size());
+ for (Entry<Integer,Long> entry : normGen.entrySet()) {
+ output.writeInt(entry.getKey());
+ output.writeLong(entry.getValue());
+ }
+ }
+
+ output.writeByte((byte) (si.getUseCompoundFile() ? SegmentInfo.YES : SegmentInfo.NO));
+ output.writeInt(si.getDelCount());
+ output.writeByte((byte) (si.getHasProxInternal()));
+ output.writeString(si.getCodec().getName());
+ output.writeStringStringMap(si.getDiagnostics());
+ output.writeByte((byte) (si.getHasVectorsInternal()));
+ }
+
protected IndexOutput createOutput(Directory dir, String segmentFileName, IOContext context)
throws IOException {
IndexOutput plainOut = dir.createOutput(segmentFileName, context);
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultStoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultStoredFieldsReader.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultStoredFieldsReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultStoredFieldsReader.java Wed Nov 16 19:09:35 2011
@@ -32,7 +32,6 @@ import org.apache.lucene.store.AlreadyCl
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.util.CloseableThreadLocal;
import org.apache.lucene.util.IOUtils;
import java.io.Closeable;
@@ -50,16 +49,7 @@ public final class DefaultStoredFieldsRe
private final static int FORMAT_SIZE = 4;
private final FieldInfos fieldInfos;
- private CloseableThreadLocal<IndexInput> fieldsStreamTL = new CloseableThreadLocal<IndexInput>();
-
- // The main fieldStream, used only for cloning.
- private final IndexInput cloneableFieldsStream;
-
- // This is a clone of cloneableFieldsStream used for reading documents.
- // It should not be cloned outside of a synchronized context.
private final IndexInput fieldsStream;
-
- private final IndexInput cloneableIndexStream;
private final IndexInput indexStream;
private int numTotalDocs;
private int size;
@@ -70,8 +60,6 @@ public final class DefaultStoredFieldsRe
// file. This will be 0 if we have our own private file.
private int docStoreOffset;
- private boolean isOriginal = false;
-
/** Returns a cloned FieldsReader that shares open
* IndexInputs with the original one. It is the caller's
* job not to close the original FieldsReader until all
@@ -80,7 +68,7 @@ public final class DefaultStoredFieldsRe
@Override
public DefaultStoredFieldsReader clone() {
ensureOpen();
- return new DefaultStoredFieldsReader(fieldInfos, numTotalDocs, size, format, docStoreOffset, cloneableFieldsStream, cloneableIndexStream);
+ return new DefaultStoredFieldsReader(fieldInfos, numTotalDocs, size, format, docStoreOffset, (IndexInput)fieldsStream.clone(), (IndexInput)indexStream.clone());
}
/** Verifies that the code version which wrote the segment is supported. */
@@ -102,16 +90,14 @@ public final class DefaultStoredFieldsRe
// Used only by clone
private DefaultStoredFieldsReader(FieldInfos fieldInfos, int numTotalDocs, int size, int format, int docStoreOffset,
- IndexInput cloneableFieldsStream, IndexInput cloneableIndexStream) {
+ IndexInput fieldsStream, IndexInput indexStream) {
this.fieldInfos = fieldInfos;
this.numTotalDocs = numTotalDocs;
this.size = size;
this.format = format;
this.docStoreOffset = docStoreOffset;
- this.cloneableFieldsStream = cloneableFieldsStream;
- this.cloneableIndexStream = cloneableIndexStream;
- fieldsStream = (IndexInput) cloneableFieldsStream.clone();
- indexStream = (IndexInput) cloneableIndexStream.clone();
+ this.fieldsStream = fieldsStream;
+ this.indexStream = indexStream;
}
public DefaultStoredFieldsReader(Directory d, SegmentInfo si, FieldInfos fn, IOContext context) throws IOException {
@@ -119,24 +105,20 @@ public final class DefaultStoredFieldsRe
final int docStoreOffset = si.getDocStoreOffset();
final int size = si.docCount;
boolean success = false;
- isOriginal = true;
+ fieldInfos = fn;
try {
- fieldInfos = fn;
-
- cloneableFieldsStream = d.openInput(IndexFileNames.segmentFileName(segment, "", DefaultStoredFieldsWriter.FIELDS_EXTENSION), context);
+ fieldsStream = d.openInput(IndexFileNames.segmentFileName(segment, "", DefaultStoredFieldsWriter.FIELDS_EXTENSION), context);
final String indexStreamFN = IndexFileNames.segmentFileName(segment, "", DefaultStoredFieldsWriter.FIELDS_INDEX_EXTENSION);
- cloneableIndexStream = d.openInput(indexStreamFN, context);
+ indexStream = d.openInput(indexStreamFN, context);
- format = cloneableIndexStream.readInt();
+ format = indexStream.readInt();
if (format < DefaultStoredFieldsWriter.FORMAT_MINIMUM)
- throw new IndexFormatTooOldException(cloneableIndexStream, format, DefaultStoredFieldsWriter.FORMAT_MINIMUM, DefaultStoredFieldsWriter.FORMAT_CURRENT);
+ throw new IndexFormatTooOldException(indexStream, format, DefaultStoredFieldsWriter.FORMAT_MINIMUM, DefaultStoredFieldsWriter.FORMAT_CURRENT);
if (format > DefaultStoredFieldsWriter.FORMAT_CURRENT)
- throw new IndexFormatTooNewException(cloneableIndexStream, format, DefaultStoredFieldsWriter.FORMAT_MINIMUM, DefaultStoredFieldsWriter.FORMAT_CURRENT);
+ throw new IndexFormatTooNewException(indexStream, format, DefaultStoredFieldsWriter.FORMAT_MINIMUM, DefaultStoredFieldsWriter.FORMAT_CURRENT);
- fieldsStream = (IndexInput) cloneableFieldsStream.clone();
-
- final long indexSize = cloneableIndexStream.length() - FORMAT_SIZE;
+ final long indexSize = indexStream.length() - FORMAT_SIZE;
if (docStoreOffset != -1) {
// We read only a slice out of this shared fields file
@@ -154,8 +136,6 @@ public final class DefaultStoredFieldsRe
throw new CorruptIndexException("doc counts differ for segment " + segment + ": fieldsReader shows " + this.size + " but segmentInfo shows " + si.docCount);
}
}
-
- indexStream = (IndexInput) cloneableIndexStream.clone();
numTotalDocs = (int) (indexSize >> 3);
success = true;
} finally {
@@ -180,18 +160,14 @@ public final class DefaultStoredFieldsRe
}
/**
- * Closes the underlying {@link org.apache.lucene.store.IndexInput} streams, including any ones associated with a
- * lazy implementation of a Field. This means that the Fields values will not be accessible.
+ * Closes the underlying {@link org.apache.lucene.store.IndexInput} streams.
+ * This means that the Fields values will not be accessible.
*
* @throws IOException
*/
public final void close() throws IOException {
if (!closed) {
- if (isOriginal) {
- IOUtils.close(fieldsStream, indexStream, fieldsStreamTL, cloneableFieldsStream, cloneableIndexStream);
- } else {
- IOUtils.close(fieldsStream, indexStream, fieldsStreamTL);
- }
+ IOUtils.close(fieldsStream, indexStream);
closed = true;
}
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultStoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultStoredFieldsWriter.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultStoredFieldsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/DefaultStoredFieldsWriter.java Wed Nov 16 19:09:35 2011
@@ -20,6 +20,7 @@ import java.io.IOException;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.MergeState;
@@ -72,8 +73,8 @@ public final class DefaultStoredFieldsWr
/** Extension of stored fields index file */
public static final String FIELDS_INDEX_EXTENSION = "fdx";
- private Directory directory;
- private String segment;
+ private final Directory directory;
+ private final String segment;
private IndexOutput fieldsStream;
private IndexOutput indexStream;
@@ -118,20 +119,19 @@ public final class DefaultStoredFieldsWr
public void abort() {
try {
close();
- } catch (IOException ignored) {
- }
+ } catch (IOException ignored) {}
+
try {
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", FIELDS_EXTENSION));
- } catch (IOException ignored) {
- }
+ } catch (IOException ignored) {}
+
try {
directory.deleteFile(IndexFileNames.segmentFileName(segment, "", FIELDS_INDEX_EXTENSION));
- } catch (IOException ignored) {
- }
+ } catch (IOException ignored) {}
}
- public final void writeField(int fieldNumber, IndexableField field) throws IOException {
- fieldsStream.writeVInt(fieldNumber);
+ public final void writeField(FieldInfo info, IndexableField field) throws IOException {
+ fieldsStream.writeVInt(info.number);
int bits = 0;
final BytesRef bytes;
final String string;
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/FieldsConsumer.java Wed Nov 16 19:09:35 2011
@@ -17,14 +17,14 @@ package org.apache.lucene.index.codecs;
* limitations under the License.
*/
+import java.io.Closeable;
+import java.io.IOException;
+
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.Fields;
import org.apache.lucene.index.FieldsEnum;
import org.apache.lucene.index.MergeState;
-import org.apache.lucene.index.TermsEnum;
-
-import java.io.IOException;
-import java.io.Closeable;
+import org.apache.lucene.index.Terms;
/** Abstract API that consumes terms, doc, freq, prox and
* payloads postings. Concrete implementations of this
@@ -48,10 +48,10 @@ public abstract class FieldsConsumer imp
while((field = fieldsEnum.next()) != null) {
mergeState.fieldInfo = mergeState.fieldInfos.fieldInfo(field);
assert mergeState.fieldInfo != null : "FieldInfo for field is null: "+ field;
- TermsEnum terms = fieldsEnum.terms();
+ Terms terms = fieldsEnum.terms();
if (terms != null) {
final TermsConsumer termsConsumer = addField(mergeState.fieldInfo);
- termsConsumer.merge(mergeState, terms);
+ termsConsumer.merge(mergeState, terms.iterator(null));
}
}
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/StoredFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/StoredFieldsReader.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/StoredFieldsReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/StoredFieldsReader.java Wed Nov 16 19:09:35 2011
@@ -1,11 +1,5 @@
package org.apache.lucene.index.codecs;
-import java.io.Closeable;
-import java.io.IOException;
-
-import org.apache.lucene.index.CorruptIndexException;
-import org.apache.lucene.index.StoredFieldVisitor;
-
/**
* Copyright 2004 The Apache Software Foundation
*
@@ -22,6 +16,12 @@ import org.apache.lucene.index.StoredFie
* the License.
*/
+import java.io.Closeable;
+import java.io.IOException;
+
+import org.apache.lucene.index.CorruptIndexException;
+import org.apache.lucene.index.StoredFieldVisitor;
+
/**
* Codec API for reading stored fields:
*
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/StoredFieldsWriter.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/StoredFieldsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/StoredFieldsWriter.java Wed Nov 16 19:09:35 2011
@@ -4,6 +4,7 @@ import java.io.Closeable;
import java.io.IOException;
import org.apache.lucene.document.Document;
+import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.MergeState;
@@ -31,7 +32,7 @@ import org.apache.lucene.util.Bits;
* <ol>
* <li>For every document, {@link #startDocument(int)} is called,
* informing the Codec how many fields will be written.
- * <li>{@link #writeField(int, IndexableField)} is called for
+ * <li>{@link #writeField(FieldInfo, IndexableField)} is called for
* each field in the document.
* <li>After all documents have been written, {@link #finish(int)}
* is called for verification/sanity-checks.
@@ -43,14 +44,14 @@ import org.apache.lucene.util.Bits;
public abstract class StoredFieldsWriter implements Closeable {
/** Called before writing the stored fields of the document.
- * {@link #writeField(int, IndexableField)} will be called
+ * {@link #writeField(FieldInfo, IndexableField)} will be called
* <code>numStoredFields</code> times. Note that this is
* called even if the document has no stored fields, in
* this case <code>numStoredFields</code> will be zero. */
public abstract void startDocument(int numStoredFields) throws IOException;
/** Writes a single stored field. */
- public abstract void writeField(int fieldNumber, IndexableField field) throws IOException;
+ public abstract void writeField(FieldInfo info, IndexableField field) throws IOException;
/** Aborts writing entirely, implementation should remove
* any partially-written files, etc. */
@@ -67,7 +68,7 @@ public abstract class StoredFieldsWriter
/** Merges in the stored fields from the readers in
* <code>mergeState</code>. The default implementation skips
* over deleted documents, and uses {@link #startDocument(int)},
- * {@link #writeField(int, IndexableField)}, and {@link #finish(int)},
+ * {@link #writeField(FieldInfo, IndexableField)}, and {@link #finish(int)},
* returning the number of documents that were written.
* Implementations can override this method for more sophisticated
* merging (bulk-byte copying, etc). */
@@ -110,7 +111,7 @@ public abstract class StoredFieldsWriter
for (IndexableField field : doc) {
if (field.fieldType().stored()) {
- writeField(fieldInfos.fieldNumber(field.name()), field);
+ writeField(fieldInfos.fieldInfo(field.name()), field);
}
}
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xCodec.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xCodec.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xCodec.java Wed Nov 16 19:09:35 2011
@@ -24,14 +24,18 @@ import org.apache.lucene.index.PerDocWri
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.DefaultFieldInfosFormat;
import org.apache.lucene.index.codecs.DefaultStoredFieldsFormat;
import org.apache.lucene.index.codecs.DefaultSegmentInfosFormat;
+import org.apache.lucene.index.codecs.DefaultTermVectorsFormat;
import org.apache.lucene.index.codecs.DocValuesFormat;
+import org.apache.lucene.index.codecs.FieldInfosFormat;
import org.apache.lucene.index.codecs.StoredFieldsFormat;
import org.apache.lucene.index.codecs.PerDocConsumer;
import org.apache.lucene.index.codecs.PerDocValues;
import org.apache.lucene.index.codecs.PostingsFormat;
import org.apache.lucene.index.codecs.SegmentInfosFormat;
+import org.apache.lucene.index.codecs.TermVectorsFormat;
import org.apache.lucene.store.Directory;
/**
@@ -48,6 +52,12 @@ public class Lucene3xCodec extends Codec
private final StoredFieldsFormat fieldsFormat = new DefaultStoredFieldsFormat();
// TODO: this should really be a different impl
+ private final TermVectorsFormat vectorsFormat = new DefaultTermVectorsFormat();
+
+ // TODO: this should really be a different impl
+ private final FieldInfosFormat fieldInfosFormat = new DefaultFieldInfosFormat();
+
+ // TODO: this should really be a different impl
// also if we want preflex to *really* be read-only it should throw exception for the writer?
// this way IR.commit fails on delete/undelete/setNorm/etc ?
private final SegmentInfosFormat infosFormat = new DefaultSegmentInfosFormat();
@@ -82,6 +92,16 @@ public class Lucene3xCodec extends Codec
public StoredFieldsFormat storedFieldsFormat() {
return fieldsFormat;
}
+
+ @Override
+ public TermVectorsFormat termVectorsFormat() {
+ return vectorsFormat;
+ }
+
+ @Override
+ public FieldInfosFormat fieldInfosFormat() {
+ return fieldInfosFormat;
+ }
@Override
public SegmentInfosFormat segmentInfosFormat() {
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xFields.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xFields.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene3x/Lucene3xFields.java Wed Nov 16 19:09:35 2011
@@ -25,7 +25,6 @@ import java.util.Iterator;
import java.util.Map;
import java.util.TreeMap;
-import org.apache.lucene.index.CorruptIndexException;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
@@ -38,7 +37,6 @@ import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.codecs.FieldsProducer;
-import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.IndexInput;
@@ -163,6 +161,11 @@ public class Lucene3xFields extends Fiel
}
@Override
+ public int getUniqueFieldCount() {
+ return preTerms.size();
+ }
+
+ @Override
public long getUniqueTermCount() throws IOException {
return getTermsDict().size();
}
@@ -196,12 +199,10 @@ public class Lucene3xFields extends Fiel
private class PreFlexFieldsEnum extends FieldsEnum {
final Iterator<FieldInfo> it;
- private final PreTermsEnum termsEnum;
FieldInfo current;
public PreFlexFieldsEnum() throws IOException {
it = fields.values().iterator();
- termsEnum = new PreTermsEnum();
}
@Override
@@ -215,9 +216,8 @@ public class Lucene3xFields extends Fiel
}
@Override
- public TermsEnum terms() throws IOException {
- termsEnum.reset(current);
- return termsEnum;
+ public Terms terms() throws IOException {
+ return Lucene3xFields.this.terms(current.name);
}
}
@@ -228,7 +228,7 @@ public class Lucene3xFields extends Fiel
}
@Override
- public TermsEnum iterator() throws IOException {
+ public TermsEnum iterator(TermsEnum reuse) throws IOException {
PreTermsEnum termsEnum = new PreTermsEnum();
termsEnum.reset(fieldInfo);
return termsEnum;
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/lucene40/Lucene40Codec.java Wed Nov 16 19:09:35 2011
@@ -19,12 +19,16 @@ package org.apache.lucene.index.codecs.l
import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.index.codecs.DefaultDocValuesFormat;
+import org.apache.lucene.index.codecs.DefaultFieldInfosFormat;
import org.apache.lucene.index.codecs.DefaultStoredFieldsFormat;
import org.apache.lucene.index.codecs.DefaultSegmentInfosFormat;
+import org.apache.lucene.index.codecs.DefaultTermVectorsFormat;
import org.apache.lucene.index.codecs.DocValuesFormat;
+import org.apache.lucene.index.codecs.FieldInfosFormat;
import org.apache.lucene.index.codecs.StoredFieldsFormat;
import org.apache.lucene.index.codecs.PostingsFormat;
import org.apache.lucene.index.codecs.SegmentInfosFormat;
+import org.apache.lucene.index.codecs.TermVectorsFormat;
import org.apache.lucene.index.codecs.perfield.PerFieldPostingsFormat;
/**
@@ -39,7 +43,9 @@ import org.apache.lucene.index.codecs.pe
// (it writes a minor version, etc).
public class Lucene40Codec extends Codec {
private final StoredFieldsFormat fieldsFormat = new DefaultStoredFieldsFormat();
+ private final TermVectorsFormat vectorsFormat = new DefaultTermVectorsFormat();
private final DocValuesFormat docValuesFormat = new DefaultDocValuesFormat();
+ private final FieldInfosFormat fieldInfosFormat = new DefaultFieldInfosFormat();
private final SegmentInfosFormat infosFormat = new DefaultSegmentInfosFormat();
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@Override
@@ -56,6 +62,11 @@ public class Lucene40Codec extends Codec
public StoredFieldsFormat storedFieldsFormat() {
return fieldsFormat;
}
+
+ @Override
+ public TermVectorsFormat termVectorsFormat() {
+ return vectorsFormat;
+ }
@Override
public DocValuesFormat docValuesFormat() {
@@ -68,6 +79,11 @@ public class Lucene40Codec extends Codec
}
@Override
+ public FieldInfosFormat fieldInfosFormat() {
+ return fieldInfosFormat;
+ }
+
+ @Override
public SegmentInfosFormat segmentInfosFormat() {
return infosFormat;
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryPostingsFormat.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/memory/MemoryPostingsFormat.java Wed Nov 16 19:09:35 2011
@@ -723,7 +723,7 @@ public class MemoryPostingsFormat extend
}
@Override
- public TermsEnum iterator() {
+ public TermsEnum iterator(TermsEnum reuse) {
return new FSTTermsEnum(field, fst);
}
@@ -768,8 +768,9 @@ public class MemoryPostingsFormat extend
return current.field.name;
}
- public TermsEnum terms() {
- return current.iterator();
+ @Override
+ public Terms terms() {
+ return current;
}
};
}
@@ -780,6 +781,11 @@ public class MemoryPostingsFormat extend
}
@Override
+ public int getUniqueFieldCount() {
+ return fields.size();
+ }
+
+ @Override
public void close() {
// Drop ref to FST:
for(TermsReader termsReader : fields.values()) {
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldPostingsFormat.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/perfield/PerFieldPostingsFormat.java Wed Nov 16 19:09:35 2011
@@ -34,7 +34,6 @@ import org.apache.lucene.index.SegmentIn
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.index.SegmentWriteState;
import org.apache.lucene.index.Terms;
-import org.apache.lucene.index.TermsEnum;
import org.apache.lucene.index.codecs.FieldsConsumer;
import org.apache.lucene.index.codecs.FieldsProducer;
import org.apache.lucene.index.codecs.PostingsFormat;
@@ -213,7 +212,7 @@ public abstract class PerFieldPostingsFo
}
@Override
- public String next() {
+ public String next() throws IOException {
if (it.hasNext()) {
current = it.next();
} else {
@@ -224,13 +223,8 @@ public abstract class PerFieldPostingsFo
}
@Override
- public TermsEnum terms() throws IOException {
- final Terms terms = fields.get(current).terms(current);
- if (terms != null) {
- return terms.iterator();
- } else {
- return TermsEnum.EMPTY;
- }
+ public Terms terms() throws IOException {
+ return fields.get(current).terms(current);
}
}
@@ -246,6 +240,11 @@ public abstract class PerFieldPostingsFo
}
@Override
+ public int getUniqueFieldCount() {
+ return fields.size();
+ }
+
+ @Override
public void close() throws IOException {
IOUtils.close(formats.values());
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextCodec.java Wed Nov 16 19:09:35 2011
@@ -19,11 +19,12 @@ package org.apache.lucene.index.codecs.s
import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.index.codecs.DefaultDocValuesFormat;
-import org.apache.lucene.index.codecs.DefaultSegmentInfosFormat;
import org.apache.lucene.index.codecs.DocValuesFormat;
+import org.apache.lucene.index.codecs.FieldInfosFormat;
import org.apache.lucene.index.codecs.PostingsFormat;
import org.apache.lucene.index.codecs.SegmentInfosFormat;
import org.apache.lucene.index.codecs.StoredFieldsFormat;
+import org.apache.lucene.index.codecs.TermVectorsFormat;
/**
* plain text index format.
@@ -34,11 +35,11 @@ import org.apache.lucene.index.codecs.St
public final class SimpleTextCodec extends Codec {
private final PostingsFormat postings = new SimpleTextPostingsFormat();
private final StoredFieldsFormat storedFields = new SimpleTextStoredFieldsFormat();
-
+ private final SegmentInfosFormat segmentInfos = new SimpleTextSegmentInfosFormat();
+ private final FieldInfosFormat fieldInfosFormat = new SimpleTextFieldInfosFormat();
+ private final TermVectorsFormat vectorsFormat = new SimpleTextTermVectorsFormat();
// TODO: need a plain-text impl
private final DocValuesFormat docValues = new DefaultDocValuesFormat();
- // TODO: need a plain-text impl
- private final SegmentInfosFormat segmentInfos = new DefaultSegmentInfosFormat();
public SimpleTextCodec() {
super("SimpleText");
@@ -58,6 +59,16 @@ public final class SimpleTextCodec exten
public StoredFieldsFormat storedFieldsFormat() {
return storedFields;
}
+
+ @Override
+ public TermVectorsFormat termVectorsFormat() {
+ return vectorsFormat;
+ }
+
+ @Override
+ public FieldInfosFormat fieldInfosFormat() {
+ return fieldInfosFormat;
+ }
@Override
public SegmentInfosFormat segmentInfosFormat() {
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextFieldsReader.java Wed Nov 16 19:09:35 2011
@@ -87,8 +87,8 @@ class SimpleTextFieldsReader extends Fie
}
@Override
- public TermsEnum terms() throws IOException {
- return SimpleTextFieldsReader.this.terms(current).iterator();
+ public Terms terms() throws IOException {
+ return SimpleTextFieldsReader.this.terms(current);
}
}
@@ -523,7 +523,7 @@ class SimpleTextFieldsReader extends Fie
}
@Override
- public TermsEnum iterator() throws IOException {
+ public TermsEnum iterator(TermsEnum reuse) throws IOException {
if (fst != null) {
return new SimpleTextTermsEnum(fst, indexOptions);
} else {
@@ -582,6 +582,11 @@ class SimpleTextFieldsReader extends Fie
}
@Override
+ public int getUniqueFieldCount() {
+ return -1;
+ }
+
+ @Override
public void close() throws IOException {
in.close();
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextStoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextStoredFieldsWriter.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextStoredFieldsWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/codecs/simpletext/SimpleTextStoredFieldsWriter.java Wed Nov 16 19:09:35 2011
@@ -19,6 +19,7 @@ package org.apache.lucene.index.codecs.s
import java.io.IOException;
+import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.IndexFileNames;
import org.apache.lucene.index.IndexableField;
import org.apache.lucene.index.codecs.StoredFieldsWriter;
@@ -87,9 +88,9 @@ public class SimpleTextStoredFieldsWrite
}
@Override
- public void writeField(int fieldNumber, IndexableField field) throws IOException {
+ public void writeField(FieldInfo info, IndexableField field) throws IOException {
write(FIELD);
- write(Integer.toString(fieldNumber));
+ write(Integer.toString(info.number));
newLine();
write(NAME);
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheImpl.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheImpl.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheImpl.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FieldCacheImpl.java Wed Nov 16 19:09:35 2011
@@ -333,7 +333,7 @@ class FieldCacheImpl implements FieldCac
setDocsWithField = false;
}
}
- final TermsEnum termsEnum = terms.iterator();
+ final TermsEnum termsEnum = terms.iterator(null);
DocsEnum docs = null;
try {
while(true) {
@@ -406,7 +406,7 @@ class FieldCacheImpl implements FieldCac
setDocsWithField = false;
}
}
- final TermsEnum termsEnum = terms.iterator();
+ final TermsEnum termsEnum = terms.iterator(null);
DocsEnum docs = null;
try {
while(true) {
@@ -505,7 +505,7 @@ class FieldCacheImpl implements FieldCac
setDocsWithField = false;
}
}
- final TermsEnum termsEnum = terms.iterator();
+ final TermsEnum termsEnum = terms.iterator(null);
DocsEnum docs = null;
try {
while(true) {
@@ -574,7 +574,7 @@ class FieldCacheImpl implements FieldCac
// Fast case: all docs have this field:
return new Bits.MatchAllBits(maxDoc);
}
- final TermsEnum termsEnum = terms.iterator();
+ final TermsEnum termsEnum = terms.iterator(null);
DocsEnum docs = null;
while(true) {
final BytesRef term = termsEnum.next();
@@ -655,7 +655,7 @@ class FieldCacheImpl implements FieldCac
setDocsWithField = false;
}
}
- final TermsEnum termsEnum = terms.iterator();
+ final TermsEnum termsEnum = terms.iterator(null);
DocsEnum docs = null;
try {
while(true) {
@@ -743,7 +743,7 @@ class FieldCacheImpl implements FieldCac
setDocsWithField = false;
}
}
- final TermsEnum termsEnum = terms.iterator();
+ final TermsEnum termsEnum = terms.iterator(null);
DocsEnum docs = null;
try {
while(true) {
@@ -832,7 +832,7 @@ class FieldCacheImpl implements FieldCac
setDocsWithField = false;
}
}
- final TermsEnum termsEnum = terms.iterator();
+ final TermsEnum termsEnum = terms.iterator(null);
DocsEnum docs = null;
try {
while(true) {
@@ -1128,7 +1128,7 @@ class FieldCacheImpl implements FieldCac
int termOrd = 1;
if (terms != null) {
- final TermsEnum termsEnum = terms.iterator();
+ final TermsEnum termsEnum = terms.iterator(null);
DocsEnum docs = null;
while(true) {
@@ -1253,7 +1253,7 @@ class FieldCacheImpl implements FieldCac
if (terms != null) {
int termCount = 0;
- final TermsEnum termsEnum = terms.iterator();
+ final TermsEnum termsEnum = terms.iterator(null);
DocsEnum docs = null;
while(true) {
if (termCount++ == termCountHardLimit) {
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyQuery.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyQuery.java Wed Nov 16 19:09:35 2011
@@ -139,7 +139,7 @@ public class FuzzyQuery extends MultiTer
@Override
protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
if (!termLongEnough) { // can only match if it's exact
- return new SingleTermsEnum(terms.iterator(), term.bytes());
+ return new SingleTermsEnum(terms.iterator(null), term.bytes());
}
return new FuzzyTermsEnum(terms, atts, getTerm(), minimumSimilarity, prefixLength);
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/FuzzyTermsEnum.java Wed Nov 16 19:09:35 2011
@@ -406,7 +406,7 @@ public final class FuzzyTermsEnum extend
* @throws IOException
*/
public LinearFuzzyTermsEnum() throws IOException {
- super(terms.iterator());
+ super(terms.iterator(null));
this.text = new int[termLength - realPrefixLength];
System.arraycopy(termText, realPrefixLength, text, 0, text.length);
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/NumericRangeQuery.java Wed Nov 16 19:09:35 2011
@@ -305,7 +305,7 @@ public final class NumericRangeQuery<T e
// very strange: java.lang.Number itsself is not Comparable, but all subclasses used here are
return (min != null && max != null && ((Comparable<T>) min).compareTo(max) > 0) ?
TermsEnum.EMPTY :
- new NumericRangeTermsEnum(terms.iterator());
+ new NumericRangeTermsEnum(terms.iterator(null));
}
/** Returns <code>true</code> if the lower endpoint is inclusive */
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PrefixQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PrefixQuery.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PrefixQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/PrefixQuery.java Wed Nov 16 19:09:35 2011
@@ -45,7 +45,7 @@ public class PrefixQuery extends MultiTe
@Override
protected TermsEnum getTermsEnum(Terms terms, AttributeSource atts) throws IOException {
- TermsEnum tenum = terms.iterator();
+ TermsEnum tenum = terms.iterator(null);
if (prefix.bytes().length == 0) {
// no prefix -- match all terms for this field:
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/search/TermRangeQuery.java Wed Nov 16 19:09:35 2011
@@ -103,7 +103,7 @@ public class TermRangeQuery extends Mult
return TermsEnum.EMPTY;
}
- TermsEnum tenum = terms.iterator();
+ TermsEnum tenum = terms.iterator(null);
if ((lowerTerm == null || (includeLower && lowerTerm.length == 0)) && upperTerm == null) {
return tenum;
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/StringHelper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/StringHelper.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/StringHelper.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/StringHelper.java Wed Nov 16 19:09:35 2011
@@ -43,6 +43,22 @@ public abstract class StringHelper {
return i;
return len;
}
+
+ /**
+ * Compares two byte[] arrays, element by element, and returns the
+ * number of elements common to both arrays.
+ *
+ * @param bytes1 The first byte[] to compare
+ * @param bytes2 The second byte[] to compare
+ * @return The number of common elements.
+ */
+ public static int bytesDifference(byte[] bytes1, int off1, int len1, byte[] bytes2, int off2, int len2) {
+ int len = len1 < len2 ? len1 : len2;
+ for (int i = 0; i < len; i++)
+ if (bytes1[i+off1] != bytes2[i+off2])
+ return i;
+ return len;
+ }
private StringHelper() {
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/automaton/CompiledAutomaton.java Wed Nov 16 19:09:35 2011
@@ -198,13 +198,13 @@ public class CompiledAutomaton {
case NONE:
return TermsEnum.EMPTY;
case ALL:
- return terms.iterator();
+ return terms.iterator(null);
case SINGLE:
- return new SingleTermsEnum(terms.iterator(), term);
+ return new SingleTermsEnum(terms.iterator(null), term);
case PREFIX:
// TODO: this is very likely faster than .intersect,
// but we should test and maybe cutover
- return new PrefixTermsEnum(terms.iterator(), term);
+ return new PrefixTermsEnum(terms.iterator(null), term);
case NORMAL:
return terms.intersect(this, null);
default:
Modified: lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/index/ThreadedIndexingAndSearchingTestCase.java Wed Nov 16 19:09:35 2011
@@ -348,7 +348,7 @@ public abstract class ThreadedIndexingAn
if (terms == null) {
continue;
}
- TermsEnum termsEnum = terms.iterator();
+ TermsEnum termsEnum = terms.iterator(null);
int seenTermCount = 0;
int shift;
int trigger;
Modified: lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/index/codecs/ramonly/RAMOnlyPostingsFormat.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/index/codecs/ramonly/RAMOnlyPostingsFormat.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/index/codecs/ramonly/RAMOnlyPostingsFormat.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/index/codecs/ramonly/RAMOnlyPostingsFormat.java Wed Nov 16 19:09:35 2011
@@ -109,6 +109,11 @@ public class RAMOnlyPostingsFormat exten
}
@Override
+ public int getUniqueFieldCount() {
+ return fieldToTerms.size();
+ }
+
+ @Override
public FieldsEnum iterator() {
return new RAMFieldsEnum(this);
}
@@ -150,7 +155,7 @@ public class RAMOnlyPostingsFormat exten
}
@Override
- public TermsEnum iterator() {
+ public TermsEnum iterator(TermsEnum reuse) {
return new RAMTermsEnum(RAMOnlyPostingsFormat.RAMField.this);
}
@@ -300,8 +305,8 @@ public class RAMOnlyPostingsFormat exten
}
@Override
- public TermsEnum terms() {
- return new RAMTermsEnum(postings.fieldToTerms.get(current));
+ public Terms terms() {
+ return postings.fieldToTerms.get(current);
}
}
Modified: lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/store/MockDirectoryWrapper.java Wed Nov 16 19:09:35 2011
@@ -149,8 +149,7 @@ public class MockDirectoryWrapper extend
@Override
public synchronized void sync(Collection<String> names) throws IOException {
maybeYield();
- for (String name : names)
- maybeThrowDeterministicException();
+ maybeThrowDeterministicException();
if (crashed)
throw new IOException("cannot sync after crash");
unSyncedFiles.removeAll(names);
@@ -349,9 +348,18 @@ public class MockDirectoryWrapper extend
return new HashSet<String>(openFilesDeleted);
}
+ private boolean failOnCreateOutput = true;
+
+ public void setFailOnCreateOutput(boolean v) {
+ failOnCreateOutput = v;
+ }
+
@Override
public synchronized IndexOutput createOutput(String name, IOContext context) throws IOException {
maybeYield();
+ if (failOnCreateOutput) {
+ maybeThrowDeterministicException();
+ }
if (crashed)
throw new IOException("cannot createOutput after crash");
init();
@@ -416,10 +424,19 @@ public class MockDirectoryWrapper extend
openFileHandles.put(c, new RuntimeException("unclosed Index" + handle.name() + ": " + name));
}
-
+
+ private boolean failOnOpenInput = true;
+
+ public void setFailOnOpenInput(boolean v) {
+ failOnOpenInput = v;
+ }
+
@Override
public synchronized IndexInput openInput(String name, IOContext context) throws IOException {
maybeYield();
+ if (failOnOpenInput) {
+ maybeThrowDeterministicException();
+ }
if (!delegate.fileExists(name))
throw new FileNotFoundException(name);
Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/Test2BTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/Test2BTerms.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/Test2BTerms.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/Test2BTerms.java Wed Nov 16 19:09:35 2011
@@ -225,7 +225,7 @@ public class Test2BTerms extends LuceneT
private List<BytesRef> findTerms(IndexReader r) throws IOException {
System.out.println("TEST: findTerms");
- final TermsEnum termsEnum = MultiFields.getTerms(r, "field").iterator();
+ final TermsEnum termsEnum = MultiFields.getTerms(r, "field").iterator(null);
final List<BytesRef> savedTerms = new ArrayList<BytesRef>();
int nextSave = _TestUtil.nextInt(random, 500000, 1000000);
BytesRef term;
@@ -243,7 +243,7 @@ public class Test2BTerms extends LuceneT
System.out.println("TEST: run " + terms.size() + " terms on reader=" + r);
IndexSearcher s = new IndexSearcher(r);
Collections.shuffle(terms);
- TermsEnum termsEnum = MultiFields.getTerms(r, "field").iterator();
+ TermsEnum termsEnum = MultiFields.getTerms(r, "field").iterator(null);
boolean failed = false;
for(int iter=0;iter<10*terms.size();iter++) {
final BytesRef term = terms.get(random.nextInt(terms.size()));
Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java Wed Nov 16 19:09:35 2011
@@ -32,12 +32,16 @@ import org.apache.lucene.document.TextFi
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
import org.apache.lucene.index.codecs.Codec;
import org.apache.lucene.index.codecs.DefaultDocValuesFormat;
+import org.apache.lucene.index.codecs.DefaultFieldInfosFormat;
import org.apache.lucene.index.codecs.DefaultStoredFieldsFormat;
import org.apache.lucene.index.codecs.DefaultSegmentInfosFormat;
+import org.apache.lucene.index.codecs.DefaultTermVectorsFormat;
import org.apache.lucene.index.codecs.DocValuesFormat;
+import org.apache.lucene.index.codecs.FieldInfosFormat;
import org.apache.lucene.index.codecs.StoredFieldsFormat;
import org.apache.lucene.index.codecs.PostingsFormat;
import org.apache.lucene.index.codecs.SegmentInfosFormat;
+import org.apache.lucene.index.codecs.TermVectorsFormat;
import org.apache.lucene.index.codecs.lucene40.Lucene40Codec;
import org.apache.lucene.index.codecs.pulsing.Pulsing40PostingsFormat;
import org.apache.lucene.search.DocIdSetIterator;
@@ -1160,6 +1164,16 @@ public class TestAddIndexes extends Luce
public StoredFieldsFormat storedFieldsFormat() {
return new DefaultStoredFieldsFormat();
}
+
+ @Override
+ public TermVectorsFormat termVectorsFormat() {
+ return new DefaultTermVectorsFormat();
+ }
+
+ @Override
+ public FieldInfosFormat fieldInfosFormat() {
+ return new DefaultFieldInfosFormat();
+ }
@Override
public SegmentInfosFormat segmentInfosFormat() {
Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1202842&r1=1202841&r2=1202842&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Wed Nov 16 19:09:35 2011
@@ -35,6 +35,8 @@ import org.apache.lucene.document.String
import org.apache.lucene.document.TextField;
import org.apache.lucene.index.FieldInfo.IndexOptions;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
+import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.FieldInfosReader;
import org.apache.lucene.search.DocIdSetIterator;
import org.apache.lucene.search.FieldCache;
import org.apache.lucene.search.IndexSearcher;
@@ -44,6 +46,7 @@ import org.apache.lucene.search.TermQuer
import org.apache.lucene.search.similarities.DefaultSimilarity;
import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
import org.apache.lucene.store.RAMDirectory;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
@@ -272,7 +275,7 @@ public class TestBackwardsCompatibility
assertEquals("wrong number of hits", expectedCount, hitCount);
for(int i=0;i<hitCount;i++) {
reader.document(hits[i].doc);
- reader.getTermFreqVectors(hits[i].doc);
+ reader.getTermVectors(hits[i].doc);
}
}
@@ -311,9 +314,8 @@ public class TestBackwardsCompatibility
assertEquals("field with non-ascii name", f.stringValue());
}
- TermFreqVector tfv = reader.getTermFreqVector(i, "utf8");
+ Terms tfv = reader.getTermVectors(i).terms("utf8");
assertNotNull("docID=" + i + " index=" + indexDir.getName(), tfv);
- assertTrue(tfv instanceof TermPositionVector);
} else
// Only ID 7 is deleted
assertEquals(7, i);
@@ -564,7 +566,8 @@ public class TestBackwardsCompatibility
// "content", and then set our expected file names below
// accordingly:
CompoundFileDirectory cfsReader = new CompoundFileDirectory(dir, "_0.cfs", newIOContext(random), false);
- FieldInfos fieldInfos = new FieldInfos(cfsReader, "_0.fnm");
+ FieldInfosReader infosReader = Codec.getDefault().fieldInfosFormat().getFieldInfosReader();
+ FieldInfos fieldInfos = infosReader.read(cfsReader, "_0", IOContext.READONCE);
int contentFieldIndex = -1;
for (FieldInfo fi : fieldInfos) {
if (fi.name.equals("content")) {
@@ -654,7 +657,7 @@ public class TestBackwardsCompatibility
_TestUtil.unzip(getDataFile("index." + oldNames[i] + ".zip"), oldIndexDir);
Directory dir = newFSDirectory(oldIndexDir);
IndexReader r = IndexReader.open(dir);
- TermsEnum terms = MultiFields.getFields(r).terms("content").iterator();
+ TermsEnum terms = MultiFields.getFields(r).terms("content").iterator(null);
BytesRef t = terms.next();
assertNotNull(t);