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/05/09 15:19:39 UTC
svn commit: r1101016 [4/10] - in /lucene/dev/branches/bulkpostings: ./
dev-tools/ dev-tools/eclipse/ dev-tools/idea/.idea/libraries/
dev-tools/idea/solr/ dev-tools/maven/lucene/contrib/ant/
dev-tools/maven/lucene/contrib/db/bdb-je/ dev-tools/maven/luce...
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfo.java Mon May 9 13:19:28 2011
@@ -17,21 +17,22 @@ package org.apache.lucene.index;
* limitations under the License.
*/
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.util.Constants;
-import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.CodecProvider;
-import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
import java.io.IOException;
-import java.util.Arrays;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
import java.util.List;
import java.util.Map;
+import java.util.Map.Entry;
import java.util.Set;
-import java.util.HashSet;
-import java.util.HashMap;
-import java.util.ArrayList;
+
+import org.apache.lucene.index.codecs.Codec;
+import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.util.Constants;
/**
* Information about a segment such as it's name, directory, and files related
@@ -62,7 +63,7 @@ public final class SegmentInfo {
* - NO says this field has no separate norms
* >= YES says this field has separate norms with the specified generation
*/
- private long[] normGen;
+ private Map<Integer,Long> normGen;
private boolean isCompoundFile;
@@ -83,6 +84,8 @@ public final class SegmentInfo {
private boolean hasProx; // True if this segment has any fields with omitTermFreqAndPositions==false
private boolean hasVectors; // True if this segment wrote term vectors
+
+ private FieldInfos fieldInfos;
private SegmentCodecs segmentCodecs;
@@ -100,7 +103,7 @@ public final class SegmentInfo {
private long bufferedDeletesGen;
public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile,
- boolean hasProx, SegmentCodecs segmentCodecs, boolean hasVectors) {
+ boolean hasProx, SegmentCodecs segmentCodecs, boolean hasVectors, FieldInfos fieldInfos) {
this.name = name;
this.docCount = docCount;
this.dir = dir;
@@ -113,13 +116,14 @@ public final class SegmentInfo {
this.hasVectors = hasVectors;
delCount = 0;
version = Constants.LUCENE_MAIN_VERSION;
+ this.fieldInfos = fieldInfos;
}
/**
* Copy everything from src SegmentInfo into our instance.
*/
void reset(SegmentInfo src) {
- clearFiles();
+ clearFilesCache();
version = src.version;
name = src.name;
docCount = src.docCount;
@@ -130,11 +134,14 @@ public final class SegmentInfo {
docStoreIsCompoundFile = src.docStoreIsCompoundFile;
hasVectors = src.hasVectors;
hasProx = src.hasProx;
+ fieldInfos = src.fieldInfos == null ? null : (FieldInfos) src.fieldInfos.clone();
if (src.normGen == null) {
normGen = null;
} else {
- normGen = new long[src.normGen.length];
- System.arraycopy(src.normGen, 0, normGen, 0, src.normGen.length);
+ normGen = new HashMap<Integer, Long>(src.normGen.size());
+ for (Entry<Integer,Long> entry : src.normGen.entrySet()) {
+ normGen.put(entry.getKey(), entry.getValue());
+ }
}
isCompoundFile = src.isCompoundFile;
delCount = src.delCount;
@@ -184,9 +191,14 @@ public final class SegmentInfo {
if (numNormGen == NO) {
normGen = null;
} else {
- normGen = new long[numNormGen];
+ normGen = new HashMap<Integer, Long>();
for(int j=0;j<numNormGen;j++) {
- normGen[j] = input.readLong();
+ int fieldNumber = j;
+ if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
+ fieldNumber = input.readInt();
+ }
+
+ normGen.put(fieldNumber, input.readLong());
}
}
isCompoundFile = input.readByte() == YES;
@@ -197,13 +209,12 @@ public final class SegmentInfo {
hasProx = input.readByte() == YES;
// System.out.println(Thread.currentThread().getName() + ": si.read hasProx=" + hasProx + " seg=" + name);
- segmentCodecs = new SegmentCodecs(codecs);
if (format <= DefaultSegmentInfosWriter.FORMAT_4_0) {
- segmentCodecs.read(input);
+ segmentCodecs = new SegmentCodecs(codecs, input);
} else {
// codec ID on FieldInfo is 0 so it will simply use the first codec available
// TODO what todo if preflex is not available in the provider? register it or fail?
- segmentCodecs.codecs = new Codec[] { codecs.lookup("PreFlex")};
+ segmentCodecs = new SegmentCodecs(codecs, new Codec[] { codecs.lookup("PreFlex")});
}
diagnostics = input.readStringStringMap();
@@ -237,6 +248,24 @@ public final class SegmentInfo {
}
}
}
+
+ synchronized void loadFieldInfos(Directory dir, boolean checkCompoundFile) throws IOException {
+ if (fieldInfos == null) {
+ Directory dir0 = dir;
+ if (isCompoundFile && checkCompoundFile) {
+ dir0 = new CompoundFileReader(dir, IndexFileNames.segmentFileName(name,
+ "", IndexFileNames.COMPOUND_FILE_EXTENSION));
+ }
+ try {
+ fieldInfos = new FieldInfos(dir0, IndexFileNames.segmentFileName(name,
+ "", IndexFileNames.FIELD_INFOS_EXTENSION));
+ } finally {
+ if (dir != dir0) {
+ dir0.close();
+ }
+ }
+ }
+ }
/**
* Returns total size in bytes of all of files used by this segment (if
@@ -280,7 +309,12 @@ public final class SegmentInfo {
public void setHasVectors(boolean v) {
hasVectors = v;
- clearFiles();
+ clearFilesCache();
+ }
+
+ public FieldInfos getFieldInfos() throws IOException {
+ loadFieldInfos(dir, true);
+ return fieldInfos;
}
public boolean hasDeletions() {
@@ -298,17 +332,18 @@ public final class SegmentInfo {
} else {
delGen++;
}
- clearFiles();
+ clearFilesCache();
}
void clearDelGen() {
delGen = NO;
- clearFiles();
+ clearFilesCache();
}
@Override
public Object clone() {
- SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, hasProx, segmentCodecs, false);
+ final SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, hasProx, segmentCodecs, hasVectors,
+ fieldInfos == null ? null : (FieldInfos) fieldInfos.clone());
si.docStoreOffset = docStoreOffset;
si.docStoreSegment = docStoreSegment;
si.docStoreIsCompoundFile = docStoreIsCompoundFile;
@@ -316,9 +351,11 @@ public final class SegmentInfo {
si.delCount = delCount;
si.diagnostics = new HashMap<String, String>(diagnostics);
if (normGen != null) {
- si.normGen = normGen.clone();
+ si.normGen = new HashMap<Integer, Long>();
+ for (Entry<Integer,Long> entry : normGen.entrySet()) {
+ si.normGen.put(entry.getKey(), entry.getValue());
+ }
}
- si.hasVectors = hasVectors;
si.version = version;
return si;
}
@@ -339,7 +376,12 @@ public final class SegmentInfo {
* @param fieldNumber the field index to check
*/
public boolean hasSeparateNorms(int fieldNumber) {
- return normGen != null && normGen[fieldNumber] != NO;
+ if (normGen == null) {
+ return false;
+ }
+
+ Long gen = normGen.get(fieldNumber);
+ return gen != null && gen.longValue() != NO;
}
/**
@@ -349,7 +391,7 @@ public final class SegmentInfo {
if (normGen == null) {
return false;
} else {
- for (long fieldNormGen : normGen) {
+ for (long fieldNormGen : normGen.values()) {
if (fieldNormGen >= YES) {
return true;
}
@@ -359,10 +401,9 @@ public final class SegmentInfo {
return false;
}
- void initNormGen(int numFields) {
+ void initNormGen() {
if (normGen == null) { // normGen is null if this segments file hasn't had any norms set against it yet
- normGen = new long[numFields];
- Arrays.fill(normGen, NO);
+ normGen = new HashMap<Integer, Long>();
}
}
@@ -373,12 +414,13 @@ public final class SegmentInfo {
* @param fieldIndex field whose norm file will be rewritten
*/
void advanceNormGen(int fieldIndex) {
- if (normGen[fieldIndex] == NO) {
- normGen[fieldIndex] = YES;
+ Long gen = normGen.get(fieldIndex);
+ if (gen == null || gen.longValue() == NO) {
+ normGen.put(fieldIndex, new Long(YES));
} else {
- normGen[fieldIndex]++;
+ normGen.put(fieldIndex, gen+1);
}
- clearFiles();
+ clearFilesCache();
}
/**
@@ -388,7 +430,7 @@ public final class SegmentInfo {
*/
public String getNormFileName(int number) {
if (hasSeparateNorms(number)) {
- return IndexFileNames.fileNameFromGeneration(name, "s" + number, normGen[number]);
+ return IndexFileNames.fileNameFromGeneration(name, "s" + number, normGen.get(number));
} else {
// single file for all norms
return IndexFileNames.fileNameFromGeneration(name, IndexFileNames.NORMS_EXTENSION, WITHOUT_GEN);
@@ -403,7 +445,7 @@ public final class SegmentInfo {
*/
void setUseCompoundFile(boolean isCompoundFile) {
this.isCompoundFile = isCompoundFile;
- clearFiles();
+ clearFilesCache();
}
/**
@@ -433,7 +475,7 @@ public final class SegmentInfo {
void setDocStoreIsCompoundFile(boolean v) {
docStoreIsCompoundFile = v;
- clearFiles();
+ clearFilesCache();
}
public String getDocStoreSegment() {
@@ -446,14 +488,14 @@ public final class SegmentInfo {
void setDocStoreOffset(int offset) {
docStoreOffset = offset;
- clearFiles();
+ clearFilesCache();
}
void setDocStore(int offset, String segment, boolean isCompoundFile) {
docStoreOffset = offset;
docStoreSegment = segment;
docStoreIsCompoundFile = isCompoundFile;
- clearFiles();
+ clearFilesCache();
}
/** Save this segment's info. */
@@ -474,9 +516,10 @@ public final class SegmentInfo {
if (normGen == null) {
output.writeInt(NO);
} else {
- output.writeInt(normGen.length);
- for (long fieldNormGen : normGen) {
- output.writeLong(fieldNormGen);
+ output.writeInt(normGen.size());
+ for (Entry<Integer,Long> entry : normGen.entrySet()) {
+ output.writeInt(entry.getKey());
+ output.writeLong(entry.getValue());
}
}
@@ -490,7 +533,7 @@ public final class SegmentInfo {
void setHasProx(boolean hasProx) {
this.hasProx = hasProx;
- clearFiles();
+ clearFilesCache();
}
public boolean getHasProx() {
@@ -572,11 +615,11 @@ public final class SegmentInfo {
}
if (normGen != null) {
- for (int i = 0; i < normGen.length; i++) {
- long gen = normGen[i];
+ for (Entry<Integer,Long> entry : normGen.entrySet()) {
+ long gen = entry.getValue();
if (gen >= YES) {
// Definitely a separate norm file, with generation:
- fileSet.add(IndexFileNames.fileNameFromGeneration(name, IndexFileNames.SEPARATE_NORMS_EXTENSION + i, gen));
+ fileSet.add(IndexFileNames.fileNameFromGeneration(name, IndexFileNames.SEPARATE_NORMS_EXTENSION + entry.getKey(), gen));
}
}
}
@@ -588,7 +631,7 @@ public final class SegmentInfo {
/* Called whenever any change is made that affects which
* files this segment has. */
- private void clearFiles() {
+ private void clearFilesCache() {
files = null;
sizeInBytesNoStore = -1;
sizeInBytesWithStore = -1;
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentInfos.java Mon May 9 13:19:28 2011
@@ -17,25 +17,27 @@ package org.apache.lucene.index;
* limitations under the License.
*/
-import org.apache.lucene.store.Directory;
-import org.apache.lucene.store.IndexInput;
-import org.apache.lucene.store.IndexOutput;
-import org.apache.lucene.store.NoSuchDirectoryException;
-import org.apache.lucene.index.codecs.CodecProvider;
-import org.apache.lucene.index.codecs.SegmentInfosReader;
-import org.apache.lucene.index.codecs.SegmentInfosWriter;
-import org.apache.lucene.util.ThreadInterruptedException;
-
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.PrintStream;
-import java.util.Vector;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
-import java.util.HashSet;
import java.util.HashMap;
+import java.util.HashSet;
import java.util.Map;
+import java.util.Vector;
+
+import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
+import org.apache.lucene.index.codecs.CodecProvider;
+import org.apache.lucene.index.codecs.DefaultSegmentInfosWriter;
+import org.apache.lucene.index.codecs.SegmentInfosReader;
+import org.apache.lucene.index.codecs.SegmentInfosWriter;
+import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IndexInput;
+import org.apache.lucene.store.IndexOutput;
+import org.apache.lucene.store.NoSuchDirectoryException;
+import org.apache.lucene.util.ThreadInterruptedException;
/**
* A collection of segmentInfo objects with methods for operating on
@@ -64,6 +66,11 @@ public final class SegmentInfos extends
* starting with the current time in milliseconds forces to create unique version numbers.
*/
public long version = System.currentTimeMillis();
+
+ private long globalFieldMapVersion = 0; // version of the GFNM for the next commit
+ private long lastGlobalFieldMapVersion = 0; // version of the GFNM file we last successfully read or wrote
+ private long pendingMapVersion = -1; // version of the GFNM itself that we have last successfully written
+ // or -1 if we it was not written. This is set during prepareCommit
private long generation = 0; // generation of the "segments_N" for the next commit
private long lastGeneration = 0; // generation of the "segments_N" file we last successfully read
@@ -75,6 +82,8 @@ public final class SegmentInfos extends
private CodecProvider codecs;
private int format;
+
+ private FieldNumberBiMap globalFieldNumberMap; // this segments global field number map - lazy loaded on demand
/**
* If non-null, information about loading segments_N files
@@ -171,6 +180,15 @@ public final class SegmentInfos extends
"",
lastGeneration);
}
+
+ private String getGlobalFieldNumberName(long version) {
+ /*
+ * This creates a file name ${version}.fnx without a leading underscore
+ * since this file might belong to more than one segment (global map) and
+ * could otherwise easily be confused with a per-segment file.
+ */
+ return IndexFileNames.segmentFileName(""+ version, "", IndexFileNames.GLOBAL_FIELD_NUM_MAP_EXTENSION);
+ }
/**
* Parse the generation off the segments file name and
@@ -261,6 +279,8 @@ public final class SegmentInfos extends
return null;
}
}.run();
+ // either we are on 4.0 or we don't have a lastGlobalFieldMapVersion i.e. its still set to 0
+ assert DefaultSegmentInfosWriter.FORMAT_4_0 <= format || (DefaultSegmentInfosWriter.FORMAT_4_0 > format && lastGlobalFieldMapVersion == 0);
}
// Only non-null after prepareCommit has been called and
@@ -270,15 +290,24 @@ public final class SegmentInfos extends
private void write(Directory directory) throws IOException {
String segmentFileName = getNextSegmentFileName();
-
+ final String globalFieldMapFile;
+ if (globalFieldNumberMap != null && globalFieldNumberMap.isDirty()) {
+ globalFieldMapFile = getGlobalFieldNumberName(++globalFieldMapVersion);
+ pendingMapVersion = writeGlobalFieldMap(globalFieldNumberMap, directory, globalFieldMapFile);
+ } else {
+ globalFieldMapFile = null;
+ }
+
+
// Always advance the generation on write:
if (generation == -1) {
generation = 1;
} else {
generation++;
}
-
+
IndexOutput segnOutput = null;
+
boolean success = false;
@@ -304,6 +333,16 @@ public final class SegmentInfos extends
} catch (Throwable t) {
// Suppress so we keep throwing the original exception
}
+ if (globalFieldMapFile != null) { // delete if written here
+ try {
+ // Try not to leave global field map in
+ // the index:
+ directory.deleteFile(globalFieldMapFile);
+ } catch (Throwable t) {
+ // Suppress so we keep throwing the original exception
+ }
+ }
+ pendingMapVersion = -1;
}
}
}
@@ -719,6 +758,8 @@ public final class SegmentInfos extends
void updateGeneration(SegmentInfos other) {
lastGeneration = other.lastGeneration;
generation = other.generation;
+ lastGlobalFieldMapVersion = other.lastGlobalFieldMapVersion;
+ globalFieldMapVersion = other.globalFieldMapVersion;
}
final void rollbackCommit(Directory dir) throws IOException {
@@ -742,6 +783,16 @@ public final class SegmentInfos extends
// in our caller
}
pendingSegnOutput = null;
+ if (pendingMapVersion != -1) {
+ try {
+ final String fieldMapName = getGlobalFieldNumberName(globalFieldMapVersion--);
+ dir.deleteFile(fieldMapName);
+ } catch (Throwable t) {
+ // Suppress so we keep throwing the original exception
+ // in our caller
+ }
+ pendingMapVersion = -1;
+ }
}
}
@@ -760,6 +811,40 @@ public final class SegmentInfos extends
throw new IllegalStateException("prepareCommit was already called");
write(dir);
}
+
+ private final long writeGlobalFieldMap(FieldNumberBiMap map, Directory dir, String name) throws IOException {
+ final IndexOutput output = dir.createOutput(name);
+ boolean success = false;
+ long version;
+ try {
+ version = map.write(output);
+ success = true;
+ } finally {
+ try {
+ output.close();
+ } catch (Throwable t) {
+ // throw orig excp
+ }
+ if (!success) {
+ try {
+ dir.deleteFile(name);
+ } catch (Throwable t) {
+ // throw orig excp
+ }
+ }
+ }
+ return version;
+ }
+
+ private void readGlobalFieldMap(FieldNumberBiMap map, Directory dir) throws IOException {
+ final String name = getGlobalFieldNumberName(lastGlobalFieldMapVersion);
+ final IndexInput input = dir.openInput(name);
+ try {
+ map.read(input);
+ } finally {
+ input.close();
+ }
+ }
/** Returns all file names referenced by SegmentInfo
* instances matching the provided Directory (ie files
@@ -769,7 +854,17 @@ public final class SegmentInfos extends
public Collection<String> files(Directory dir, boolean includeSegmentsFile) throws IOException {
HashSet<String> files = new HashSet<String>();
if (includeSegmentsFile) {
- files.add(getCurrentSegmentFileName());
+ final String segmentFileName = getCurrentSegmentFileName();
+ if (segmentFileName != null) {
+ /*
+ * TODO: if lastGen == -1 we get might get null here it seems wrong to
+ * add null to the files set
+ */
+ files.add(segmentFileName);
+ }
+ if (lastGlobalFieldMapVersion > 0) {
+ files.add(getGlobalFieldNumberName(lastGlobalFieldMapVersion));
+ }
}
final int size = size();
for(int i=0;i<size;i++) {
@@ -821,6 +916,17 @@ public final class SegmentInfos extends
}
lastGeneration = generation;
+ if (pendingMapVersion != -1) {
+ /*
+ * TODO is it possible that the commit does not succeed here? if another
+ * commit happens at the same time and we lost the race between the
+ * prepareCommit and finishCommit the latest version is already
+ * incremented.
+ */
+ globalFieldNumberMap.commitLastVersion(pendingMapVersion);
+ pendingMapVersion = -1;
+ lastGlobalFieldMapVersion = globalFieldMapVersion;
+ }
try {
IndexOutput genOutput = dir.createOutput(IndexFileNames.SEGMENTS_GEN);
@@ -848,6 +954,7 @@ public final class SegmentInfos extends
prepareCommit(dir);
finishCommit(dir);
}
+
public synchronized String toString(Directory directory) {
StringBuilder buffer = new StringBuilder();
@@ -883,6 +990,8 @@ public final class SegmentInfos extends
clear();
addAll(other);
lastGeneration = other.lastGeneration;
+ lastGlobalFieldMapVersion = other.lastGlobalFieldMapVersion;
+ format = other.format;
}
/** Returns sum of all segment's docCounts. Note that
@@ -900,4 +1009,49 @@ public final class SegmentInfos extends
public void changed() {
version++;
}
+
+ /**
+ * Loads or returns the already loaded the global field number map for this {@link SegmentInfos}.
+ * If this {@link SegmentInfos} has no global field number map the returned instance is empty
+ */
+ synchronized FieldNumberBiMap getOrLoadGlobalFieldNumberMap(Directory dir) throws IOException {
+ if (globalFieldNumberMap != null) {
+ return globalFieldNumberMap;
+ }
+ final FieldNumberBiMap map = new FieldNumberBiMap();
+
+ if (lastGlobalFieldMapVersion > 0) {
+ // if we don't have a global map or this is a SI from a earlier version we just return the empty map;
+ readGlobalFieldMap(map, dir);
+ }
+ if (size() > 0) {
+ if (format > DefaultSegmentInfosWriter.FORMAT_4_0) {
+ assert lastGlobalFieldMapVersion == 0;
+ // build the map up if we open a pre 4.0 index
+ for (SegmentInfo info : this) {
+ final FieldInfos segFieldInfos = info.getFieldInfos();
+ for (FieldInfo fi : segFieldInfos) {
+ map.addOrGet(fi.name, fi.number);
+ }
+ }
+ }
+ }
+ return globalFieldNumberMap = map;
+ }
+
+ /**
+ * Called by {@link SegmentInfosReader} when reading the global field map version
+ */
+ public void setGlobalFieldMapVersion(long version) {
+ lastGlobalFieldMapVersion = globalFieldMapVersion = version;
+ }
+
+ public long getGlobalFieldMapVersion() {
+ return globalFieldMapVersion;
+ }
+
+ // for testing
+ long getLastGlobalFieldMapVersion() {
+ return lastGlobalFieldMapVersion;
+ }
}
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Mon May 9 13:19:28 2011
@@ -26,16 +26,16 @@ import java.util.List;
import org.apache.lucene.document.Document;
import org.apache.lucene.index.IndexReader.FieldOption;
import org.apache.lucene.index.MergePolicy.MergeAbortedException;
-import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.index.codecs.Codec;
-import org.apache.lucene.index.codecs.MergeState;
+import org.apache.lucene.index.codecs.CodecProvider;
import org.apache.lucene.index.codecs.FieldsConsumer;
+import org.apache.lucene.index.codecs.MergeState;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.ReaderUtil;
import org.apache.lucene.util.MultiBits;
+import org.apache.lucene.util.ReaderUtil;
/**
* The SegmentMerger class combines two or more Segments, represented by an IndexReader ({@link #add},
@@ -75,8 +75,8 @@ final class SegmentMerger {
this.payloadProcessorProvider = payloadProcessorProvider;
directory = dir;
this.codecs = codecs;
- this.fieldInfos = fieldInfos;
segment = name;
+ this.fieldInfos = fieldInfos;
if (merge != null) {
checkAbort = new MergeState.CheckAbort(merge, directory);
} else {
@@ -148,7 +148,7 @@ final class SegmentMerger {
boolean storePayloads, boolean omitTFAndPositions)
throws IOException {
for (String field : names) {
- fInfos.add(field, true, storeTermVectors,
+ fInfos.addOrUpdate(field, true, storeTermVectors,
storePositionWithTermVector, storeOffsetWithTermVector, !reader
.hasNorms(field), storePayloads, omitTFAndPositions);
}
@@ -180,9 +180,8 @@ final class SegmentMerger {
SegmentReader segmentReader = (SegmentReader) reader;
boolean same = true;
FieldInfos segmentFieldInfos = segmentReader.fieldInfos();
- int numFieldInfos = segmentFieldInfos.size();
- for (int j = 0; same && j < numFieldInfos; j++) {
- same = fieldInfos.fieldName(j).equals(segmentFieldInfos.fieldName(j));
+ for (FieldInfo fi : segmentFieldInfos) {
+ same = fieldInfos.fieldName(fi.number).equals(fi.name);
}
if (same) {
matchingSegmentReaders[i] = segmentReader;
@@ -208,9 +207,8 @@ final class SegmentMerger {
if (reader instanceof SegmentReader) {
SegmentReader segmentReader = (SegmentReader) reader;
FieldInfos readerFieldInfos = segmentReader.fieldInfos();
- int numReaderFieldInfos = readerFieldInfos.size();
- for (int j = 0; j < numReaderFieldInfos; j++) {
- fieldInfos.add(readerFieldInfos.fieldInfo(j));
+ for (FieldInfo fi : readerFieldInfos) {
+ fieldInfos.add(fi);
}
} else {
addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.TERMVECTOR_WITH_POSITION_OFFSET), true, true, true, false, false);
@@ -220,17 +218,17 @@ final class SegmentMerger {
addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.OMIT_TERM_FREQ_AND_POSITIONS), false, false, false, false, true);
addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.STORES_PAYLOADS), false, false, false, true, false);
addIndexed(reader, fieldInfos, reader.getFieldNames(FieldOption.INDEXED), false, false, false, false, false);
- fieldInfos.add(reader.getFieldNames(FieldOption.UNINDEXED), false);
+ fieldInfos.addOrUpdate(reader.getFieldNames(FieldOption.UNINDEXED), false);
}
}
- final SegmentCodecs codecInfo = SegmentCodecs.build(fieldInfos, this.codecs);
- fieldInfos.write(directory, segment + ".fnm");
+ final SegmentCodecs codecInfo = fieldInfos.buildSegmentCodecs(false);
+ fieldInfos.write(directory, segment + "." + IndexFileNames.FIELD_INFOS_EXTENSION);
int docCount = 0;
setMatchingSegmentReaders();
- final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, fieldInfos);
+ final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment);
try {
int idx = 0;
@@ -312,7 +310,7 @@ final class SegmentMerger {
// NOTE: it's very important to first assign to doc then pass it to
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
Document doc = reader.document(j);
- fieldsWriter.addDocument(doc);
+ fieldsWriter.addDocument(doc, fieldInfos);
docCount++;
checkAbort.work(300);
}
@@ -339,7 +337,7 @@ final class SegmentMerger {
// NOTE: it's very important to first assign to doc then pass it to
// termVectorsWriter.addAllDocVectors; see LUCENE-1282
Document doc = reader.document(docCount);
- fieldsWriter.addDocument(doc);
+ fieldsWriter.addDocument(doc, fieldInfos);
checkAbort.work(300);
}
}
@@ -579,8 +577,7 @@ final class SegmentMerger {
private void mergeNorms() throws IOException {
IndexOutput output = null;
try {
- for (int i = 0, numFieldInfos = fieldInfos.size(); i < numFieldInfos; i++) {
- final FieldInfo fi = fieldInfos.fieldInfo(i);
+ for (FieldInfo fi : fieldInfos) {
if (fi.isIndexed && !fi.omitNorms) {
if (output == null) {
output = directory.createOutput(IndexFileNames.segmentFileName(segment, "", IndexFileNames.NORMS_EXTENSION));
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/SegmentReader.java Mon May 9 13:19:28 2011
@@ -22,23 +22,22 @@ import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.HashSet;
-
import java.util.List;
import java.util.Map;
import java.util.Set;
-
import java.util.concurrent.atomic.AtomicInteger;
+
import org.apache.lucene.document.Document;
import org.apache.lucene.document.FieldSelector;
+import org.apache.lucene.index.codecs.FieldsProducer;
import org.apache.lucene.store.BufferedIndexInput;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
import org.apache.lucene.store.IndexOutput;
import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.Bits;
-import org.apache.lucene.util.CloseableThreadLocal;
-import org.apache.lucene.index.codecs.FieldsProducer;
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CloseableThreadLocal;
/**
* @lucene.experimental
@@ -119,8 +118,8 @@ public class SegmentReader extends Index
dir0 = cfsReader;
}
cfsDir = dir0;
-
- fieldInfos = new FieldInfos(cfsDir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELD_INFOS_EXTENSION));
+ si.loadFieldInfos(cfsDir, false); // prevent opening the CFS to load fieldInfos
+ fieldInfos = si.getFieldInfos();
this.termsIndexDivisor = termsIndexDivisor;
@@ -598,12 +597,12 @@ public class SegmentReader extends Index
&& (!si.hasDeletions() || this.si.getDelFileName().equals(si.getDelFileName()));
boolean normsUpToDate = true;
- boolean[] fieldNormsChanged = new boolean[core.fieldInfos.size()];
- final int fieldCount = core.fieldInfos.size();
- for (int i = 0; i < fieldCount; i++) {
- if (!this.si.getNormFileName(i).equals(si.getNormFileName(i))) {
+ Set<Integer> fieldNormsChanged = new HashSet<Integer>();
+ for (FieldInfo fi : core.fieldInfos) {
+ int fieldNumber = fi.number;
+ if (!this.si.getNormFileName(fieldNumber).equals(si.getNormFileName(fieldNumber))) {
normsUpToDate = false;
- fieldNormsChanged[i] = true;
+ fieldNormsChanged.add(fieldNumber);
}
}
@@ -659,11 +658,10 @@ public class SegmentReader extends Index
clone.norms = new HashMap<String,Norm>();
// Clone norms
- for (int i = 0; i < fieldNormsChanged.length; i++) {
-
+ for (FieldInfo fi : core.fieldInfos) {
// Clone unchanged norms to the cloned reader
- if (doClone || !fieldNormsChanged[i]) {
- final String curField = core.fieldInfos.fieldInfo(i).name;
+ if (doClone || !fieldNormsChanged.contains(fi.number)) {
+ final String curField = fi.name;
Norm norm = this.norms.get(curField);
if (norm != null)
clone.norms.put(curField, (Norm) norm.clone());
@@ -735,7 +733,7 @@ public class SegmentReader extends Index
}
if (normsDirty) { // re-write norms
- si.initNormGen(core.fieldInfos.size());
+ si.initNormGen();
for (final Norm norm : norms.values()) {
if (norm.dirty) {
norm.reWrite(si);
@@ -880,8 +878,7 @@ public class SegmentReader extends Index
ensureOpen();
Set<String> fieldSet = new HashSet<String>();
- for (int i = 0; i < core.fieldInfos.size(); i++) {
- FieldInfo fi = core.fieldInfos.fieldInfo(i);
+ for (FieldInfo fi : core.fieldInfos) {
if (fieldOption == IndexReader.FieldOption.ALL) {
fieldSet.add(fi.name);
}
@@ -954,8 +951,7 @@ public class SegmentReader extends Index
private void openNorms(Directory cfsDir, int readBufferSize) throws IOException {
long nextNormSeek = SegmentMerger.NORMS_HEADER.length; //skip header (header unused for now)
int maxDoc = maxDoc();
- for (int i = 0; i < core.fieldInfos.size(); i++) {
- FieldInfo fi = core.fieldInfos.fieldInfo(i);
+ for (FieldInfo fi : core.fieldInfos) {
if (norms.containsKey(fi.name)) {
// in case this SegmentReader is being re-opened, we might be able to
// reuse some norm instances and skip loading them here
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java Mon May 9 13:19:28 2011
@@ -27,15 +27,13 @@ final class StoredFieldsWriter {
FieldsWriter fieldsWriter;
final DocumentsWriter docWriter;
- final FieldInfos fieldInfos;
int lastDocID;
PerDoc[] docFreeList = new PerDoc[1];
int freeCount;
- public StoredFieldsWriter(DocumentsWriter docWriter, FieldInfos fieldInfos) {
+ public StoredFieldsWriter(DocumentsWriter docWriter) {
this.docWriter = docWriter;
- this.fieldInfos = fieldInfos;
}
public StoredFieldsWriterPerThread addThread(DocumentsWriter.DocState docState) throws IOException {
@@ -62,7 +60,7 @@ final class StoredFieldsWriter {
private synchronized void initFieldsWriter() throws IOException {
if (fieldsWriter == null) {
- fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.getSegment(), fieldInfos);
+ fieldsWriter = new FieldsWriter(docWriter.directory, docWriter.getSegment());
lastDocID = 0;
}
}
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/StoredFieldsWriterPerThread.java Mon May 9 13:19:28 2011
@@ -32,7 +32,7 @@ final class StoredFieldsWriterPerThread
public StoredFieldsWriterPerThread(DocumentsWriter.DocState docState, StoredFieldsWriter storedFieldsWriter) throws IOException {
this.storedFieldsWriter = storedFieldsWriter;
this.docState = docState;
- localFieldsWriter = new FieldsWriter((IndexOutput) null, (IndexOutput) null, storedFieldsWriter.fieldInfos);
+ localFieldsWriter = new FieldsWriter((IndexOutput) null, (IndexOutput) null);
}
public void startDocument() {
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHash.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHash.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHash.java Mon May 9 13:19:28 2011
@@ -57,12 +57,6 @@ final class TermsHash extends InvertedDo
}
@Override
- void setFieldInfos(FieldInfos fieldInfos) {
- this.fieldInfos = fieldInfos;
- consumer.setFieldInfos(fieldInfos);
- }
-
- @Override
public void abort() {
consumer.abort();
if (nextTermsHash != null)
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java Mon May 9 13:19:28 2011
@@ -25,10 +25,4 @@ abstract class TermsHashConsumer {
abstract TermsHashConsumerPerThread addThread(TermsHashPerThread perThread);
abstract void flush(Map<TermsHashConsumerPerThread,Collection<TermsHashConsumerPerField>> threadsAndFields, final SegmentWriteState state) throws IOException;
abstract void abort();
-
- FieldInfos fieldInfos;
-
- void setFieldInfos(FieldInfos fieldInfos) {
- this.fieldInfos = fieldInfos;
}
-}
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerField.java Mon May 9 13:19:28 2011
@@ -39,6 +39,7 @@ final class TermsHashPerField extends In
final DocumentsWriter.DocState docState;
final FieldInvertState fieldState;
TermToBytesRefAttribute termAtt;
+ BytesRef termBytesRef;
// Copied from our perThread
final IntBlockPool intPool;
@@ -53,7 +54,6 @@ final class TermsHashPerField extends In
final BytesRefHash bytesHash;
ParallelPostingsArray postingsArray;
- private final BytesRef termBytesRef;
private final AtomicLong bytesUsed;
public TermsHashPerField(DocInverterPerField docInverterPerField, final TermsHashPerThread perThread, final TermsHashPerThread nextPerThread, final FieldInfo fieldInfo) {
@@ -70,7 +70,6 @@ final class TermsHashPerField extends In
bytesHash = new BytesRefHash(termBytePool, HASH_INIT_SIZE, byteStarts);
streamCount = consumer.getStreamCount();
numPostingInt = 2*streamCount;
- termBytesRef = perThread.termBytesRef;
this.fieldInfo = fieldInfo;
if (nextPerThread != null)
nextPerField = (TermsHashPerField) nextPerThread.addField(docInverterPerField, fieldInfo);
@@ -119,6 +118,7 @@ final class TermsHashPerField extends In
@Override
void start(Fieldable f) {
termAtt = fieldState.attributeSource.getAttribute(TermToBytesRefAttribute.class);
+ termBytesRef = termAtt.getBytesRef();
consumer.start(f);
if (nextPerField != null) {
nextPerField.start(f);
@@ -181,7 +181,7 @@ final class TermsHashPerField extends In
// Get the text & hash of this term.
int termID;
try{
- termID = bytesHash.add(termBytesRef, termAtt.toBytesRef(termBytesRef));
+ termID = bytesHash.add(termBytesRef, termAtt.fillBytesRef());
}catch (MaxBytesLengthExceededException e) {
// Not enough room in current block
// Just skip this term, to remain as robust as
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerThread.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerThread.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/TermsHashPerThread.java Mon May 9 13:19:28 2011
@@ -18,7 +18,6 @@ package org.apache.lucene.index;
*/
import org.apache.lucene.util.ByteBlockPool;
-import org.apache.lucene.util.BytesRef;
import java.io.IOException;
@@ -35,8 +34,6 @@ final class TermsHashPerThread extends I
final boolean primary;
final DocumentsWriter.DocState docState;
- // Used by perField to obtain terms from the analysis chain
- final BytesRef termBytesRef = new BytesRef(10);
public TermsHashPerThread(DocInverterPerThread docInverterPerThread, final TermsHash termsHash, final TermsHash nextTermsHash, final TermsHashPerThread primaryPerThread) {
docState = docInverterPerThread.docState;
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosReader.java Mon May 9 13:19:28 2011
@@ -56,7 +56,9 @@ public class DefaultSegmentInfosReader e
infos.version = input.readLong(); // read version
infos.counter = input.readInt(); // read counter
-
+ if (infos.getFormat() <= DefaultSegmentInfosWriter.FORMAT_4_0) {
+ infos.setGlobalFieldMapVersion(input.readLong());
+ }
for (int i = input.readInt(); i > 0; i--) { // read segmentInfos
SegmentInfo si = new SegmentInfo(directory, format, input, codecs);
if (si.getVersion() == null) {
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/DefaultSegmentInfosWriter.java Mon May 9 13:19:28 2011
@@ -59,6 +59,7 @@ public class DefaultSegmentInfosWriter e
out.writeInt(FORMAT_CURRENT); // write FORMAT
out.writeLong(infos.version);
out.writeInt(infos.counter); // write counter
+ out.writeLong(infos.getGlobalFieldMapVersion());
out.writeInt(infos.size()); // write infos
for (SegmentInfo si : infos) {
si.write(out);
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/index/codecs/preflex/PreFlexFields.java Mon May 9 13:19:28 2011
@@ -19,15 +19,16 @@ package org.apache.lucene.index.codecs.p
import java.io.IOException;
import java.util.Collection;
-import java.util.Iterator;
-import java.util.TreeMap;
+import java.util.Comparator;
import java.util.HashMap;
+import java.util.Iterator;
import java.util.Map;
-import java.util.Comparator;
+import java.util.TreeMap;
-import org.apache.lucene.index.DocsEnum;
+import org.apache.lucene.index.CompoundFileReader;
import org.apache.lucene.index.DocsAndPositionsEnum;
import org.apache.lucene.index.BulkPostingsEnum;
+import org.apache.lucene.index.DocsEnum;
import org.apache.lucene.index.FieldInfo;
import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.FieldsEnum;
@@ -36,7 +37,6 @@ import org.apache.lucene.index.SegmentIn
import org.apache.lucene.index.Term;
import org.apache.lucene.index.Terms;
import org.apache.lucene.index.TermsEnum;
-import org.apache.lucene.index.CompoundFileReader;
import org.apache.lucene.index.codecs.FieldsProducer;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IndexInput;
@@ -96,13 +96,11 @@ public class PreFlexFields extends Field
// so that if an index update removes them we'll still have them
freqStream = dir.openInput(info.name + ".frq", readBufferSize);
boolean anyProx = false;
- final int numFields = fieldInfos.size();
- for(int i=0;i<numFields;i++) {
- final FieldInfo fieldInfo = fieldInfos.fieldInfo(i);
- if (fieldInfo.isIndexed) {
- fields.put(fieldInfo.name, fieldInfo);
- preTerms.put(fieldInfo.name, new PreTerms(fieldInfo));
- if (!fieldInfo.omitTermFreqAndPositions) {
+ for (FieldInfo fi : fieldInfos) {
+ if (fi.isIndexed) {
+ fields.put(fi.name, fi);
+ preTerms.put(fi.name, new PreTerms(fi));
+ if (!fi.omitTermFreqAndPositions) {
anyProx = true;
}
}
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/queryParser/QueryParserBase.java Mon May 9 13:19:28 2011
@@ -532,18 +532,19 @@ public abstract class QueryParserBase {
// ignore
}
+ BytesRef bytes = termAtt == null ? null : termAtt.getBytesRef();
+
if (numTokens == 0)
return null;
else if (numTokens == 1) {
- BytesRef term = new BytesRef();
try {
boolean hasNext = buffer.incrementToken();
assert hasNext == true;
- termAtt.toBytesRef(term);
+ termAtt.fillBytesRef();
} catch (IOException e) {
// safe to ignore, because we know the number of tokens
}
- return newTermQuery(new Term(field, term));
+ return newTermQuery(new Term(field, new BytesRef(bytes)));
} else {
if (severalTokensAtSamePosition || (!quoted && !autoGeneratePhraseQueries)) {
if (positionCount == 1 || (!quoted && !autoGeneratePhraseQueries)) {
@@ -554,17 +555,15 @@ public abstract class QueryParserBase {
BooleanClause.Occur.MUST : BooleanClause.Occur.SHOULD;
for (int i = 0; i < numTokens; i++) {
- BytesRef term = new BytesRef();
try {
boolean hasNext = buffer.incrementToken();
assert hasNext == true;
- termAtt.toBytesRef(term);
+ termAtt.fillBytesRef();
} catch (IOException e) {
// safe to ignore, because we know the number of tokens
}
-
Query currentQuery = newTermQuery(
- new Term(field, term));
+ new Term(field, new BytesRef(bytes)));
q.add(currentQuery, occur);
}
return q;
@@ -576,12 +575,11 @@ public abstract class QueryParserBase {
List<Term> multiTerms = new ArrayList<Term>();
int position = -1;
for (int i = 0; i < numTokens; i++) {
- BytesRef term = new BytesRef();
int positionIncrement = 1;
try {
boolean hasNext = buffer.incrementToken();
assert hasNext == true;
- termAtt.toBytesRef(term);
+ termAtt.fillBytesRef();
if (posIncrAtt != null) {
positionIncrement = posIncrAtt.getPositionIncrement();
}
@@ -598,7 +596,7 @@ public abstract class QueryParserBase {
multiTerms.clear();
}
position += positionIncrement;
- multiTerms.add(new Term(field, term));
+ multiTerms.add(new Term(field, new BytesRef(bytes)));
}
if (enablePositionIncrements) {
mpq.add(multiTerms.toArray(new Term[0]),position);
@@ -613,15 +611,13 @@ public abstract class QueryParserBase {
pq.setSlop(phraseSlop);
int position = -1;
-
for (int i = 0; i < numTokens; i++) {
- BytesRef term = new BytesRef();
int positionIncrement = 1;
try {
boolean hasNext = buffer.incrementToken();
assert hasNext == true;
- termAtt.toBytesRef(term);
+ termAtt.fillBytesRef();
if (posIncrAtt != null) {
positionIncrement = posIncrAtt.getPositionIncrement();
}
@@ -631,9 +627,9 @@ public abstract class QueryParserBase {
if (enablePositionIncrements) {
position += positionIncrement;
- pq.add(new Term(field, term),position);
+ pq.add(new Term(field, new BytesRef(bytes)),position);
} else {
- pq.add(new Term(field, term));
+ pq.add(new Term(field, new BytesRef(bytes)));
}
}
return pq;
@@ -796,13 +792,13 @@ public abstract class QueryParserBase {
source = analyzer.tokenStream(field, new StringReader(part));
}
- BytesRef result = new BytesRef();
TermToBytesRefAttribute termAtt = source.getAttribute(TermToBytesRefAttribute.class);
-
+ BytesRef bytes = termAtt.getBytesRef();
+
try {
if (!source.incrementToken())
throw new IllegalArgumentException("analyzer returned no terms for range part: " + part);
- termAtt.toBytesRef(result);
+ termAtt.fillBytesRef();
if (source.incrementToken())
throw new IllegalArgumentException("analyzer returned too many terms for range part: " + part);
} catch (IOException e) {
@@ -812,8 +808,8 @@ public abstract class QueryParserBase {
try {
source.close();
} catch (IOException ignored) {}
-
- return result;
+
+ return new BytesRef(bytes);
}
/**
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/DefaultSimilarity.java Mon May 9 13:19:28 2011
@@ -20,7 +20,7 @@ import org.apache.lucene.index.FieldInve
*/
/** Expert: Default scoring implementation. */
-public class DefaultSimilarity extends Similarity implements SimilarityProvider {
+public class DefaultSimilarity extends Similarity {
/** Implemented as
* <code>state.getBoost()*lengthNorm(numTerms)</code>, where
@@ -40,11 +40,6 @@ public class DefaultSimilarity extends S
return state.getBoost() * ((float) (1.0 / Math.sqrt(numTerms)));
}
- /** Implemented as <code>1/sqrt(sumOfSquaredWeights)</code>. */
- public float queryNorm(float sumOfSquaredWeights) {
- return (float)(1.0 / Math.sqrt(sumOfSquaredWeights));
- }
-
/** Implemented as <code>sqrt(freq)</code>. */
@Override
public float tf(float freq) {
@@ -63,11 +58,6 @@ public class DefaultSimilarity extends S
return (float)(Math.log(numDocs/(double)(docFreq+1)) + 1.0);
}
- /** Implemented as <code>overlap / maxOverlap</code>. */
- public float coord(int overlap, int maxOverlap) {
- return overlap / (float)maxOverlap;
- }
-
// Default true
protected boolean discountOverlaps = true;
@@ -88,12 +78,4 @@ public class DefaultSimilarity extends S
public boolean getDiscountOverlaps() {
return discountOverlaps;
}
-
- /**
- * Returns this default implementation for all fields.
- * Override this method to customize scoring on a per-field basis.
- */
- public Similarity get(String field) {
- return this;
- }
}
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/IndexSearcher.java Mon May 9 13:19:28 2011
@@ -72,7 +72,7 @@ public class IndexSearcher {
private final ExecutorService executor;
// the default SimilarityProvider
- private static final SimilarityProvider defaultProvider = new DefaultSimilarity();
+ private static final SimilarityProvider defaultProvider = new DefaultSimilarityProvider();
/**
* Expert: returns a default SimilarityProvider instance.
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/QueryTermVector.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/QueryTermVector.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/QueryTermVector.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/QueryTermVector.java Mon May 9 13:19:28 2011
@@ -66,10 +66,10 @@ public class QueryTermVector implements
final TermToBytesRefAttribute termAtt = stream.getAttribute(TermToBytesRefAttribute.class);
hasMoreTokens = stream.incrementToken();
+ BytesRef bytes = termAtt.getBytesRef();
while (hasMoreTokens) {
- BytesRef bytes = new BytesRef();
- termAtt.toBytesRef(bytes);
- terms.add(bytes);
+ termAtt.fillBytesRef();
+ terms.add(new BytesRef(bytes));
hasMoreTokens = stream.incrementToken();
}
processTerms(terms.toArray(new BytesRef[terms.size()]));
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Similarity.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Similarity.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Similarity.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/search/Similarity.java Mon May 9 13:19:28 2011
@@ -362,7 +362,7 @@ import org.apache.lucene.util.SmallFloat
* than another document with fewer query terms.
* This is a search time factor computed in
* {@link SimilarityProvider#coord(int, int) coord(q,d)}
- * by the Similarity in effect at search time.
+ * by the SimilarityProvider in effect at search time.
* <br> <br>
* </li>
*
@@ -373,17 +373,17 @@ import org.apache.lucene.util.SmallFloat
* is a normalizing factor used to make scores between queries comparable.
* This factor does not affect document ranking (since all ranked documents are multiplied by the same factor),
* but rather just attempts to make scores from different queries (or even different indexes) comparable.
- * This is a search time factor computed by the Similarity in effect at search time.
+ * This is a search time factor computed by the SimilarityProvider in effect at search time.
*
* The default computation in
- * {@link org.apache.lucene.search.DefaultSimilarity#queryNorm(float) DefaultSimilarity}
+ * {@link org.apache.lucene.search.DefaultSimilarityProvider#queryNorm(float) DefaultSimilarityProvider}
* produces a <a href="http://en.wikipedia.org/wiki/Euclidean_norm#Euclidean_norm">Euclidean norm</a>:
* <br> <br>
* <table cellpadding="1" cellspacing="0" border="0" align="center">
* <tr>
* <td valign="middle" align="right" rowspan="1">
* queryNorm(q) =
- * {@link org.apache.lucene.search.DefaultSimilarity#queryNorm(float) queryNorm(sumOfSquaredWeights)}
+ * {@link org.apache.lucene.search.DefaultSimilarityProvider#queryNorm(float) queryNorm(sumOfSquaredWeights)}
* =
* </td>
* <td valign="middle" align="center" rowspan="1">
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/store/DataInput.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/store/DataInput.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/store/DataInput.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/store/DataInput.java Mon May 9 13:19:28 2011
@@ -80,6 +80,9 @@ public abstract class DataInput implemen
* @see DataOutput#writeVInt(int)
*/
public int readVInt() throws IOException {
+ /* This is the original code of this method,
+ * but a Hotspot bug (see LUCENE-2975) corrupts the for-loop if
+ * readByte() is inlined. So the loop was unwinded!
byte b = readByte();
int i = b & 0x7F;
for (int shift = 7; (b & 0x80) != 0; shift += 7) {
@@ -87,6 +90,22 @@ public abstract class DataInput implemen
i |= (b & 0x7F) << shift;
}
return i;
+ */
+ byte b = readByte();
+ int i = b & 0x7F;
+ if ((b & 0x80) == 0) return i;
+ b = readByte();
+ i |= (b & 0x7F) << 7;
+ if ((b & 0x80) == 0) return i;
+ b = readByte();
+ i |= (b & 0x7F) << 14;
+ if ((b & 0x80) == 0) return i;
+ b = readByte();
+ i |= (b & 0x7F) << 21;
+ if ((b & 0x80) == 0) return i;
+ b = readByte();
+ assert (b & 0x80) == 0;
+ return i | ((b & 0x7F) << 28);
}
/** Reads eight bytes and returns a long.
@@ -100,6 +119,9 @@ public abstract class DataInput implemen
* nine bytes. Smaller values take fewer bytes. Negative numbers are not
* supported. */
public long readVLong() throws IOException {
+ /* This is the original code of this method,
+ * but a Hotspot bug (see LUCENE-2975) corrupts the for-loop if
+ * readByte() is inlined. So the loop was unwinded!
byte b = readByte();
long i = b & 0x7F;
for (int shift = 7; (b & 0x80) != 0; shift += 7) {
@@ -107,6 +129,34 @@ public abstract class DataInput implemen
i |= (b & 0x7FL) << shift;
}
return i;
+ */
+ byte b = readByte();
+ long i = b & 0x7FL;
+ if ((b & 0x80) == 0) return i;
+ b = readByte();
+ i |= (b & 0x7FL) << 7;
+ if ((b & 0x80) == 0) return i;
+ b = readByte();
+ i |= (b & 0x7FL) << 14;
+ if ((b & 0x80) == 0) return i;
+ b = readByte();
+ i |= (b & 0x7FL) << 21;
+ if ((b & 0x80) == 0) return i;
+ b = readByte();
+ i |= (b & 0x7FL) << 28;
+ if ((b & 0x80) == 0) return i;
+ b = readByte();
+ i |= (b & 0x7FL) << 35;
+ if ((b & 0x80) == 0) return i;
+ b = readByte();
+ i |= (b & 0x7FL) << 42;
+ if ((b & 0x80) == 0) return i;
+ b = readByte();
+ i |= (b & 0x7FL) << 49;
+ if ((b & 0x80) == 0) return i;
+ b = readByte();
+ assert (b & 0x80) == 0;
+ return i | ((b & 0x7FL) << 56);
}
/** Reads a string.
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/ArrayUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/ArrayUtil.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/ArrayUtil.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/ArrayUtil.java Mon May 9 13:19:28 2011
@@ -563,6 +563,7 @@ public final class ArrayUtil {
* @param toIndex end index (exclusive)
*/
public static <T> void quickSort(T[] a, int fromIndex, int toIndex, Comparator<? super T> comp) {
+ if (toIndex-fromIndex <= 1) return;
getSorter(a, comp).quickSort(fromIndex, toIndex-1);
}
@@ -581,6 +582,7 @@ public final class ArrayUtil {
* @param toIndex end index (exclusive)
*/
public static <T extends Comparable<? super T>> void quickSort(T[] a, int fromIndex, int toIndex) {
+ if (toIndex-fromIndex <= 1) return;
getSorter(a).quickSort(fromIndex, toIndex-1);
}
@@ -601,6 +603,7 @@ public final class ArrayUtil {
* @param toIndex end index (exclusive)
*/
public static <T> void mergeSort(T[] a, int fromIndex, int toIndex, Comparator<? super T> comp) {
+ if (toIndex-fromIndex <= 1) return;
getSorter(a, comp).mergeSort(fromIndex, toIndex-1);
}
@@ -619,6 +622,7 @@ public final class ArrayUtil {
* @param toIndex end index (exclusive)
*/
public static <T extends Comparable<? super T>> void mergeSort(T[] a, int fromIndex, int toIndex) {
+ if (toIndex-fromIndex <= 1) return;
getSorter(a).mergeSort(fromIndex, toIndex-1);
}
@@ -639,6 +643,7 @@ public final class ArrayUtil {
* @param toIndex end index (exclusive)
*/
public static <T> void insertionSort(T[] a, int fromIndex, int toIndex, Comparator<? super T> comp) {
+ if (toIndex-fromIndex <= 1) return;
getSorter(a, comp).insertionSort(fromIndex, toIndex-1);
}
@@ -657,6 +662,7 @@ public final class ArrayUtil {
* @param toIndex end index (exclusive)
*/
public static <T extends Comparable<? super T>> void insertionSort(T[] a, int fromIndex, int toIndex) {
+ if (toIndex-fromIndex <= 1) return;
getSorter(a).insertionSort(fromIndex, toIndex-1);
}
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/CollectionUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/CollectionUtil.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/CollectionUtil.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/CollectionUtil.java Mon May 9 13:19:28 2011
@@ -100,7 +100,9 @@ public final class CollectionUtil {
* @throws IllegalArgumentException if list is e.g. a linked list without random access.
*/
public static <T> void quickSort(List<T> list, Comparator<? super T> comp) {
- getSorter(list, comp).quickSort(0, list.size()-1);
+ final int size = list.size();
+ if (size <= 1) return;
+ getSorter(list, comp).quickSort(0, size-1);
}
/**
@@ -110,7 +112,9 @@ public final class CollectionUtil {
* @throws IllegalArgumentException if list is e.g. a linked list without random access.
*/
public static <T extends Comparable<? super T>> void quickSort(List<T> list) {
- getSorter(list).quickSort(0, list.size()-1);
+ final int size = list.size();
+ if (size <= 1) return;
+ getSorter(list).quickSort(0, size-1);
}
// mergeSorts:
@@ -122,7 +126,9 @@ public final class CollectionUtil {
* @throws IllegalArgumentException if list is e.g. a linked list without random access.
*/
public static <T> void mergeSort(List<T> list, Comparator<? super T> comp) {
- getSorter(list, comp).mergeSort(0, list.size()-1);
+ final int size = list.size();
+ if (size <= 1) return;
+ getSorter(list, comp).mergeSort(0, size-1);
}
/**
@@ -132,7 +138,9 @@ public final class CollectionUtil {
* @throws IllegalArgumentException if list is e.g. a linked list without random access.
*/
public static <T extends Comparable<? super T>> void mergeSort(List<T> list) {
- getSorter(list).mergeSort(0, list.size()-1);
+ final int size = list.size();
+ if (size <= 1) return;
+ getSorter(list).mergeSort(0, size-1);
}
// insertionSorts:
@@ -144,7 +152,9 @@ public final class CollectionUtil {
* @throws IllegalArgumentException if list is e.g. a linked list without random access.
*/
public static <T> void insertionSort(List<T> list, Comparator<? super T> comp) {
- getSorter(list, comp).insertionSort(0, list.size()-1);
+ final int size = list.size();
+ if (size <= 1) return;
+ getSorter(list, comp).insertionSort(0, size-1);
}
/**
@@ -154,7 +164,9 @@ public final class CollectionUtil {
* @throws IllegalArgumentException if list is e.g. a linked list without random access.
*/
public static <T extends Comparable<? super T>> void insertionSort(List<T> list) {
- getSorter(list).insertionSort(0, list.size()-1);
+ final int size = list.size();
+ if (size <= 1) return;
+ getSorter(list).insertionSort(0, size-1);
}
}
\ No newline at end of file
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/PriorityQueue.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/PriorityQueue.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/PriorityQueue.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/PriorityQueue.java Mon May 9 13:19:28 2011
@@ -21,8 +21,10 @@ package org.apache.lucene.util;
* least element can always be found in constant time. Put()'s and pop()'s
* require log(size) time.
*
- * <p><b>NOTE</b>: This class pre-allocates a full array of
- * length <code>maxSize+1</code>, in {@link #initialize}.
+ * <p><b>NOTE</b>: This class will pre-allocate a full array of
+ * length <code>maxSize+1</code> if instantiated via the
+ * {@link #PriorityQueue(int,boolean)} constructor with
+ * <code>prepopulate</code> set to <code>true</code>.
*
* @lucene.internal
*/
@@ -83,9 +85,10 @@ public abstract class PriorityQueue<T> {
/**
* This method can be overridden by extending classes to return a sentinel
- * object which will be used by {@link #initialize(int)} to fill the queue, so
- * that the code which uses that queue can always assume it's full and only
- * change the top without attempting to insert any new object.<br>
+ * object which will be used by the {@link PriorityQueue#PriorityQueue(int,boolean)}
+ * constructor to fill the queue, so that the code which uses that queue can always
+ * assume it's full and only change the top without attempting to insert any new
+ * object.<br>
*
* Those sentinel values should always compare worse than any non-sentinel
* value (i.e., {@link #lessThan} should always favor the
@@ -111,11 +114,11 @@ public abstract class PriorityQueue<T> {
* </pre>
*
* <b>NOTE:</b> if this method returns a non-null value, it will be called by
- * {@link #initialize(int)} {@link #size()} times, relying on a new object to
- * be returned and will not check if it's null again. Therefore you should
- * ensure any call to this method creates a new instance and behaves
- * consistently, e.g., it cannot return null if it previously returned
- * non-null.
+ * the {@link PriorityQueue#PriorityQueue(int,boolean)} constructor
+ * {@link #size()} times, relying on a new object to be returned and will not
+ * check if it's null again. Therefore you should ensure any call to this
+ * method creates a new instance and behaves consistently, e.g., it cannot
+ * return null if it previously returned non-null.
*
* @return the sentinel object to use to pre-populate the queue, or null if
* sentinel objects are not supported.
Modified: lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/SpecialOperations.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/SpecialOperations.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/SpecialOperations.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/java/org/apache/lucene/util/automaton/SpecialOperations.java Mon May 9 13:19:28 2011
@@ -29,6 +29,7 @@
package org.apache.lucene.util.automaton;
+import java.util.BitSet;
import java.util.HashMap;
import java.util.HashSet;
import java.util.Set;
@@ -65,7 +66,7 @@ final public class SpecialOperations {
*/
public static boolean isFinite(Automaton a) {
if (a.isSingleton()) return true;
- return isFinite(a.initial, new HashSet<State>());
+ return isFinite(a.initial, new BitSet(a.getNumberOfStates()), new BitSet(a.getNumberOfStates()));
}
/**
@@ -74,11 +75,12 @@ final public class SpecialOperations {
*/
// TODO: not great that this is recursive... in theory a
// large automata could exceed java's stack
- private static boolean isFinite(State s, HashSet<State> path) {
- path.add(s);
+ private static boolean isFinite(State s, BitSet path, BitSet visited) {
+ path.set(s.number);
for (Transition t : s.getTransitions())
- if (path.contains(t.to) || !isFinite(t.to, path)) return false;
- path.remove(s);
+ if (path.get(t.to.number) || (!visited.get(t.to.number) && !isFinite(t.to, path, visited))) return false;
+ path.clear(s.number);
+ visited.set(s.number);
return true;
}
Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/index/RandomIndexWriter.java Mon May 9 13:19:28 2011
@@ -150,10 +150,24 @@ public class RandomIndexWriter implement
return getReader(true);
}
+ private void doRandomOptimize() throws IOException {
+ final int segCount = w.getSegmentCount();
+ if (r.nextBoolean() || segCount == 0) {
+ // full optimize
+ w.optimize();
+ } else {
+ // partial optimize
+ final int limit = _TestUtil.nextInt(r, 1, segCount);
+ w.optimize(limit);
+ assert w.getSegmentCount() <= limit: "limit=" + limit + " actual=" + w.getSegmentCount();
+ }
+ }
+
public IndexReader getReader(boolean applyDeletions) throws IOException {
getReaderCalled = true;
- if (r.nextInt(4) == 2)
- w.optimize();
+ if (r.nextInt(4) == 2) {
+ doRandomOptimize();
+ }
// If we are writing with PreFlexRW, force a full
// IndexReader.open so terms are sorted in codepoint
// order during searching:
@@ -179,7 +193,7 @@ public class RandomIndexWriter implement
// if someone isn't using getReader() API, we want to be sure to
// maybeOptimize since presumably they might open a reader on the dir.
if (getReaderCalled == false && r.nextInt(4) == 2) {
- w.optimize();
+ doRandomOptimize();
}
w.close();
}
Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/LuceneTestCase.java Mon May 9 13:19:28 2011
@@ -142,12 +142,16 @@ public abstract class LuceneTestCase ext
public static final String TEST_DIRECTORY = System.getProperty("tests.directory", "random");
/** Get the number of times to run tests */
public static final int TEST_ITER = Integer.parseInt(System.getProperty("tests.iter", "1"));
+ /** Get the minimum number of times to run tests until a failure happens */
+ public static final int TEST_ITER_MIN = Integer.parseInt(System.getProperty("tests.iter.min", Integer.toString(TEST_ITER)));
/** Get the random seed for tests */
public static final String TEST_SEED = System.getProperty("tests.seed", "random");
/** whether or not nightly tests should run */
public static final boolean TEST_NIGHTLY = Boolean.parseBoolean(System.getProperty("tests.nightly", "false"));
/** the line file used by LineFileDocs */
public static final String TEST_LINE_DOCS_FILE = System.getProperty("tests.linedocsfile", "europarl.lines.txt.gz");
+ /** whether or not to clean threads between test invocations: "false", "perMethod", "perClass" */
+ public static final String TEST_CLEAN_THREADS = System.getProperty("tests.cleanthreads", "perClass");
private static final Pattern codecWithParam = Pattern.compile("(.*)\\(\\s*(\\d+)\\s*\\)");
@@ -342,10 +346,12 @@ public abstract class LuceneTestCase ext
@AfterClass
public static void afterClassLuceneTestCaseJ4() {
- int rogueThreads = threadCleanup("test class");
- if (rogueThreads > 0) {
- // TODO: fail here once the leaks are fixed.
- System.err.println("RESOURCE LEAK: test class left " + rogueThreads + " thread(s) running");
+ if (! "false".equals(TEST_CLEAN_THREADS)) {
+ int rogueThreads = threadCleanup("test class");
+ if (rogueThreads > 0) {
+ // TODO: fail here once the leaks are fixed.
+ System.err.println("RESOURCE LEAK: test class left " + rogueThreads + " thread(s) running");
+ }
}
String codecDescription;
CodecProvider cp = CodecProvider.getDefault();
@@ -490,7 +496,7 @@ public abstract class LuceneTestCase ext
assertTrue("ensure your setUp() calls super.setUp()!!!", setup);
setup = false;
BooleanQuery.setMaxClauseCount(savedBoolMaxClauseCount);
- if (!getClass().getName().startsWith("org.apache.solr")) {
+ if ("perMethod".equals(TEST_CLEAN_THREADS)) {
int rogueThreads = threadCleanup("test method: '" + getName() + "'");
if (rogueThreads > 0) {
System.err.println("RESOURCE LEAK: test method: '" + getName()
@@ -534,7 +540,7 @@ public abstract class LuceneTestCase ext
}
}
- private final static int THREAD_STOP_GRACE_MSEC = 1000;
+ 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>();
@@ -588,9 +594,6 @@ public abstract class LuceneTestCase ext
Thread.setDefaultUncaughtExceptionHandler(null);
if (!t.getName().startsWith("SyncThread")) // avoid zookeeper jre crash
t.interrupt();
- try {
- t.join(THREAD_STOP_GRACE_MSEC);
- } catch (InterruptedException e) { e.printStackTrace(); }
}
}
}
@@ -1207,11 +1210,24 @@ public abstract class LuceneTestCase ext
if (VERBOSE) {
System.out.println("\nNOTE: running test " + arg0.getName());
}
+
+ // only print iteration info if the user requested more than one iterations
+ boolean verbose = VERBOSE && TEST_ITER > 1;
+ int lastIterFailed = -1;
for (int i = 0; i < TEST_ITER; i++) {
- if (VERBOSE && TEST_ITER > 1) {
+ if (verbose) {
System.out.println("\nNOTE: running iter=" + (1+i) + " of " + TEST_ITER);
}
super.runChild(arg0, arg1);
+ if (testsFailed) {
+ lastIterFailed = i;
+ if (i == TEST_ITER_MIN - 1) {
+ if (verbose) {
+ System.out.println("\nNOTE: iteration " + lastIterFailed + " failed !");
+ }
+ break;
+ }
+ }
}
}
Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/_TestUtil.java Mon May 9 13:19:28 2011
@@ -27,6 +27,7 @@ import java.io.OutputStream;
import java.io.PrintStream;
import java.lang.reflect.Method;
import java.util.Enumeration;
+import java.util.List;
import java.util.Random;
import java.util.Map;
import java.util.HashMap;
@@ -35,8 +36,11 @@ import java.util.zip.ZipFile;
import org.junit.Assert;
+import org.apache.lucene.document.Document;
+import org.apache.lucene.document.Fieldable;
import org.apache.lucene.index.CheckIndex;
import org.apache.lucene.index.ConcurrentMergeScheduler;
+import org.apache.lucene.index.FieldInfos;
import org.apache.lucene.index.IndexWriter;
import org.apache.lucene.index.LogMergePolicy;
import org.apache.lucene.index.MergeScheduler;
@@ -325,4 +329,13 @@ public class _TestUtil {
throw new RuntimeException(e);
}
}
+
+ /** Adds field info for a Document. */
+ public static void add(Document doc, FieldInfos fieldInfos) {
+ List<Fieldable> fields = doc.getFields();
+ for (Fieldable field : fields) {
+ fieldInfos.addOrUpdate(field.name(), field.isIndexed(), field.isTermVectorStored(), field.isStorePositionWithTermVector(),
+ field.isStoreOffsetWithTermVector(), field.getOmitNorms(), false, field.getOmitTermFreqAndPositions());
+ }
+ }
}
Modified: lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/automaton/AutomatonTestUtil.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/automaton/AutomatonTestUtil.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/automaton/AutomatonTestUtil.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test-framework/org/apache/lucene/util/automaton/AutomatonTestUtil.java Mon May 9 13:19:28 2011
@@ -373,4 +373,28 @@ public class AutomatonTestUtil {
a.removeDeadTransitions();
}
+ /**
+ * Returns true if the language of this automaton is finite.
+ * <p>
+ * WARNING: this method is slow, it will blow up if the automaton is large.
+ * this is only used to test the correctness of our faster implementation.
+ */
+ public static boolean isFiniteSlow(Automaton a) {
+ if (a.isSingleton()) return true;
+ return isFiniteSlow(a.initial, new HashSet<State>());
+ }
+
+ /**
+ * Checks whether there is a loop containing s. (This is sufficient since
+ * there are never transitions to dead states.)
+ */
+ // TODO: not great that this is recursive... in theory a
+ // large automata could exceed java's stack
+ private static boolean isFiniteSlow(State s, HashSet<State> path) {
+ path.add(s);
+ for (Transition t : s.getTransitions())
+ if (path.contains(t.to) || !isFiniteSlow(t.to, path)) return false;
+ path.remove(s);
+ return true;
+ }
}
Modified: lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/analysis/TestNumericTokenStream.java Mon May 9 13:19:28 2011
@@ -35,13 +35,13 @@ public class TestNumericTokenStream exte
final TermToBytesRefAttribute bytesAtt = stream.getAttribute(TermToBytesRefAttribute.class);
final TypeAttribute typeAtt = stream.getAttribute(TypeAttribute.class);
final NumericTokenStream.NumericTermAttribute numericAtt = stream.getAttribute(NumericTokenStream.NumericTermAttribute.class);
- final BytesRef bytes = new BytesRef();
+ final BytesRef bytes = bytesAtt.getBytesRef();
stream.reset();
assertEquals(64, numericAtt.getValueSize());
for (int shift=0; shift<64; shift+=NumericUtils.PRECISION_STEP_DEFAULT) {
assertTrue("New token is available", stream.incrementToken());
assertEquals("Shift value wrong", shift, numericAtt.getShift());
- final int hash = bytesAtt.toBytesRef(bytes);
+ final int hash = bytesAtt.fillBytesRef();
assertEquals("Hash incorrect", bytes.hashCode(), hash);
assertEquals("Term is incorrectly encoded", lvalue & ~((1L << shift) - 1L), NumericUtils.prefixCodedToLong(bytes));
assertEquals("Term raw value is incorrectly encoded", lvalue & ~((1L << shift) - 1L), numericAtt.getRawValue());
@@ -58,13 +58,13 @@ public class TestNumericTokenStream exte
final TermToBytesRefAttribute bytesAtt = stream.getAttribute(TermToBytesRefAttribute.class);
final TypeAttribute typeAtt = stream.getAttribute(TypeAttribute.class);
final NumericTokenStream.NumericTermAttribute numericAtt = stream.getAttribute(NumericTokenStream.NumericTermAttribute.class);
- final BytesRef bytes = new BytesRef();
+ final BytesRef bytes = bytesAtt.getBytesRef();
stream.reset();
assertEquals(32, numericAtt.getValueSize());
for (int shift=0; shift<32; shift+=NumericUtils.PRECISION_STEP_DEFAULT) {
assertTrue("New token is available", stream.incrementToken());
assertEquals("Shift value wrong", shift, numericAtt.getShift());
- final int hash = bytesAtt.toBytesRef(bytes);
+ final int hash = bytesAtt.fillBytesRef();
assertEquals("Hash incorrect", bytes.hashCode(), hash);
assertEquals("Term is incorrectly encoded", ivalue & ~((1 << shift) - 1), NumericUtils.prefixCodedToInt(bytes));
assertEquals("Term raw value is incorrectly encoded", ((long) ivalue) & ~((1L << shift) - 1L), numericAtt.getRawValue());
Modified: lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/Test2BTerms.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/Test2BTerms.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/Test2BTerms.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/Test2BTerms.java Mon May 9 13:19:28 2011
@@ -74,12 +74,14 @@ public class Test2BTerms extends LuceneT
}
private final static class MyTermAttributeImpl extends AttributeImpl implements TermToBytesRefAttribute {
- public int toBytesRef(BytesRef bs) {
- bs.bytes = bytes.bytes;
- bs.offset = bytes.offset;
- bs.length = bytes.length;
+ public int fillBytesRef() {
return bytes.hashCode();
}
+
+ public BytesRef getBytesRef() {
+ return bytes;
+ }
+
@Override
public void clear() {
}
Modified: lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1101016&r1=1101015&r2=1101016&view=diff
==============================================================================
--- lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/branches/bulkpostings/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java Mon May 9 13:19:28 2011
@@ -938,6 +938,40 @@ public class TestAddIndexes extends Luce
assertTrue(c.failures.size() == 0);
}
+
+ // LUCENE-2996: tests that addIndexes(IndexReader) applies existing deletes correctly.
+ public void testExistingDeletes() throws Exception {
+ Directory[] dirs = new Directory[2];
+ for (int i = 0; i < dirs.length; i++) {
+ dirs[i] = newDirectory();
+ IndexWriterConfig conf = newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer());
+ IndexWriter writer = new IndexWriter(dirs[i], conf);
+ Document doc = new Document();
+ doc.add(new Field("id", "myid", Store.NO, Index.NOT_ANALYZED_NO_NORMS));
+ writer.addDocument(doc);
+ writer.close();
+ }
+
+ IndexWriterConfig conf = new IndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer());
+ IndexWriter writer = new IndexWriter(dirs[0], conf);
+
+ // Now delete the document
+ writer.deleteDocuments(new Term("id", "myid"));
+ IndexReader r = IndexReader.open(dirs[1]);
+ try {
+ writer.addIndexes(r);
+ } finally {
+ r.close();
+ }
+ writer.commit();
+ assertEquals("Documents from the incoming index should not have been deleted", 1, writer.numDocs());
+ writer.close();
+
+ for (Directory dir : dirs) {
+ dir.close();
+ }
+
+ }
private void addDocs3(IndexWriter writer, int numDocs) throws IOException {
for (int i = 0; i < numDocs; i++) {
@@ -1045,8 +1079,8 @@ public class TestAddIndexes extends Luce
IndexWriter w3 = new IndexWriter(dir, conf);
w3.addIndexes(readers);
w3.close();
-
- assertEquals("Only one compound segment should exist", 3, dir.listAll().length);
+ // we should now see segments_X, segments.gen,_Y.cfs, _Z.fnx
+ assertEquals("Only one compound segment should exist", 4, dir.listAll().length);
}
}