You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by bu...@apache.org on 2011/02/22 02:01:11 UTC
svn commit: r1073192 [9/32] - in /lucene/dev/branches/realtime_search: ./
dev-tools/eclipse/ dev-tools/idea/.idea/ dev-tools/idea/lucene/contrib/ant/
dev-tools/idea/lucene/contrib/demo/
dev-tools/idea/lucene/contrib/highlighter/ dev-tools/idea/lucene/c...
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/LogMergePolicy.java Tue Feb 22 01:00:39 2011
@@ -18,6 +18,11 @@ package org.apache.lucene.index;
*/
import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
import java.util.Set;
/** <p>This class implements a {@link MergePolicy} that tries
@@ -67,6 +72,7 @@ public abstract class LogMergePolicy ext
// out there wrote his own LMP ...
protected long maxMergeSizeForOptimize = Long.MAX_VALUE;
protected int maxMergeDocs = DEFAULT_MAX_MERGE_DOCS;
+ protected boolean requireContiguousMerge = false;
protected double noCFSRatio = DEFAULT_NO_CFS_RATIO;
@@ -105,6 +111,21 @@ public abstract class LogMergePolicy ext
writer.get().message("LMP: " + message);
}
+ /** If true, merges must be in-order slice of the
+ * segments. If false, then the merge policy is free to
+ * pick any segments. The default is false, which is
+ * in general more efficient than true since it gives the
+ * merge policy more freedom to pick closely sized
+ * segments. */
+ public void setRequireContiguousMerge(boolean v) {
+ requireContiguousMerge = v;
+ }
+
+ /** See {@link #setRequireContiguousMerge}. */
+ public boolean getRequireContiguousMerge() {
+ return requireContiguousMerge;
+ }
+
/** <p>Returns the number of segments that are merged at
* once and also controls the total number of segments
* allowed to accumulate in the index.</p> */
@@ -357,6 +378,8 @@ public abstract class LogMergePolicy ext
return null;
}
+ // TODO: handle non-contiguous merge case differently?
+
// Find the newest (rightmost) segment that needs to
// be optimized (other segments may have been flushed
// since optimize started):
@@ -454,6 +477,36 @@ public abstract class LogMergePolicy ext
return spec;
}
+ private static class SegmentInfoAndLevel implements Comparable<SegmentInfoAndLevel> {
+ SegmentInfo info;
+ float level;
+ int index;
+
+ public SegmentInfoAndLevel(SegmentInfo info, float level, int index) {
+ this.info = info;
+ this.level = level;
+ this.index = index;
+ }
+
+ // Sorts largest to smallest
+ public int compareTo(SegmentInfoAndLevel other) {
+ if (level < other.level)
+ return 1;
+ else if (level > other.level)
+ return -1;
+ else
+ return 0;
+ }
+ }
+
+ private static class SortByIndex implements Comparator<SegmentInfoAndLevel> {
+ public int compare(SegmentInfoAndLevel o1, SegmentInfoAndLevel o2) {
+ return o1.index - o2.index;
+ }
+ }
+
+ private static final SortByIndex sortByIndex = new SortByIndex();
+
/** Checks if any merges are now necessary and returns a
* {@link MergePolicy.MergeSpecification} if so. A merge
* is necessary when there are more than {@link
@@ -470,18 +523,38 @@ public abstract class LogMergePolicy ext
// Compute levels, which is just log (base mergeFactor)
// of the size of each segment
- float[] levels = new float[numSegments];
+ final List<SegmentInfoAndLevel> levels = new ArrayList<SegmentInfoAndLevel>();
final float norm = (float) Math.log(mergeFactor);
+ final Collection<SegmentInfo> mergingSegments = writer.get().getMergingSegments();
+
for(int i=0;i<numSegments;i++) {
final SegmentInfo info = infos.info(i);
long size = size(info);
+ // When we require contiguous merge, we still add the
+ // segment to levels to avoid merging "across" a set
+ // of segment being merged:
+ if (!requireContiguousMerge && mergingSegments.contains(info)) {
+ if (verbose()) {
+ message("seg " + info.name + " already being merged; skip");
+ }
+ continue;
+ }
+
// Floor tiny segments
- if (size < 1)
+ if (size < 1) {
size = 1;
- levels[i] = (float) Math.log(size)/norm;
- message("seg " + info.name + " level=" + levels[i]);
+ }
+ final SegmentInfoAndLevel infoLevel = new SegmentInfoAndLevel(info, (float) Math.log(size)/norm, i);
+ levels.add(infoLevel);
+ if (verbose()) {
+ message("seg " + info.name + " level=" + infoLevel.level + " size=" + size);
+ }
+ }
+
+ if (!requireContiguousMerge) {
+ Collections.sort(levels);
}
final float levelFloor;
@@ -499,14 +572,16 @@ public abstract class LogMergePolicy ext
MergeSpecification spec = null;
+ final int numMergeableSegments = levels.size();
+
int start = 0;
- while(start < numSegments) {
+ while(start < numMergeableSegments) {
// Find max level of all segments not already
// quantized.
- float maxLevel = levels[start];
- for(int i=1+start;i<numSegments;i++) {
- final float level = levels[i];
+ float maxLevel = levels.get(start).level;
+ for(int i=1+start;i<numMergeableSegments;i++) {
+ final float level = levels.get(i).level;
if (level > maxLevel)
maxLevel = level;
}
@@ -525,9 +600,9 @@ public abstract class LogMergePolicy ext
levelBottom = levelFloor;
}
- int upto = numSegments-1;
+ int upto = numMergeableSegments-1;
while(upto >= start) {
- if (levels[upto] >= levelBottom) {
+ if (levels.get(upto).level >= levelBottom) {
break;
}
upto--;
@@ -540,18 +615,26 @@ public abstract class LogMergePolicy ext
while(end <= 1+upto) {
boolean anyTooLarge = false;
for(int i=start;i<end;i++) {
- final SegmentInfo info = infos.info(i);
+ final SegmentInfo info = levels.get(i).info;
anyTooLarge |= (size(info) >= maxMergeSize || sizeDocs(info) >= maxMergeDocs);
}
if (!anyTooLarge) {
if (spec == null)
spec = new MergeSpecification();
- if (verbose())
+ if (verbose()) {
message(" " + start + " to " + end + ": add this merge");
- spec.add(new OneMerge(infos.range(start, end)));
- } else if (verbose())
+ }
+ Collections.sort(levels.subList(start, end), sortByIndex);
+ final SegmentInfos mergeInfos = new SegmentInfos();
+ for(int i=start;i<end;i++) {
+ mergeInfos.add(levels.get(i).info);
+ assert infos.contains(levels.get(i).info);
+ }
+ spec.add(new OneMerge(mergeInfos));
+ } else if (verbose()) {
message(" " + start + " to " + end + ": contains segment over maxMergeSize or maxMergeDocs; skipping");
+ }
start = end;
end = start + mergeFactor;
@@ -598,7 +681,8 @@ public abstract class LogMergePolicy ext
sb.append("maxMergeSizeForOptimize=").append(maxMergeSizeForOptimize).append(", ");
sb.append("calibrateSizeByDeletes=").append(calibrateSizeByDeletes).append(", ");
sb.append("maxMergeDocs=").append(maxMergeDocs).append(", ");
- sb.append("useCompoundFile=").append(useCompoundFile);
+ sb.append("useCompoundFile=").append(useCompoundFile).append(", ");
+ sb.append("requireContiguousMerge=").append(requireContiguousMerge);
sb.append("]");
return sb.toString();
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/MergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/MergePolicy.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/MergePolicy.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/MergePolicy.java Tue Feb 22 01:00:39 2011
@@ -72,8 +72,8 @@ public abstract class MergePolicy implem
long mergeGen; // used by IndexWriter
boolean isExternal; // used by IndexWriter
int maxNumSegmentsOptimize; // used by IndexWriter
- SegmentReader[] readers; // used by IndexWriter
- SegmentReader[] readersClone; // used by IndexWriter
+ List<SegmentReader> readers; // used by IndexWriter
+ List<SegmentReader> readerClones; // used by IndexWriter
public final SegmentInfos segments;
boolean aborted;
Throwable error;
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/MultiReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/MultiReader.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/MultiReader.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/MultiReader.java Tue Feb 22 01:00:39 2011
@@ -20,13 +20,14 @@ package org.apache.lucene.index;
import java.io.IOException;
import java.util.Collection;
import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.FieldSelector;
-import org.apache.lucene.search.FieldCache; // not great (circular); used only to purge FieldCache entry on close
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.ReaderUtil;
+import org.apache.lucene.util.MapBackedSet;
/** An IndexReader which reads multiple indexes, appending
* their content. */
@@ -82,6 +83,7 @@ public class MultiReader extends IndexRe
}
}
starts[subReaders.length] = maxDoc;
+ readerFinishedListeners = new MapBackedSet<ReaderFinishedListener>(new ConcurrentHashMap<ReaderFinishedListener,Boolean>());
return ReaderUtil.buildReaderContext(this);
}
@@ -345,11 +347,6 @@ public class MultiReader extends IndexRe
subReaders[i].close();
}
}
-
- // NOTE: only needed in case someone had asked for
- // FieldCache for top-level reader (which is generally
- // not a good idea):
- FieldCache.DEFAULT.purge(this);
}
@Override
@@ -386,7 +383,24 @@ public class MultiReader extends IndexRe
return subReaders;
}
+ @Override
public ReaderContext getTopReaderContext() {
return topLevelContext;
}
+
+ @Override
+ public void addReaderFinishedListener(ReaderFinishedListener listener) {
+ super.addReaderFinishedListener(listener);
+ for(IndexReader sub : subReaders) {
+ sub.addReaderFinishedListener(listener);
+ }
+ }
+
+ @Override
+ public void removeReaderFinishedListener(ReaderFinishedListener listener) {
+ super.removeReaderFinishedListener(listener);
+ for(IndexReader sub : subReaders) {
+ sub.removeReaderFinishedListener(listener);
+ }
+ }
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/NoMergeScheduler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/NoMergeScheduler.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/NoMergeScheduler.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/NoMergeScheduler.java Tue Feb 22 01:00:39 2011
@@ -23,7 +23,7 @@ import java.io.IOException;
* A {@link MergeScheduler} which never executes any merges. It is also a
* singleton and can be accessed through {@link NoMergeScheduler#INSTANCE}. Use
* it if you want to prevent an {@link IndexWriter} from ever executing merges,
- * irregardles of the {@link MergePolicy} used. Note that you can achieve the
+ * irregardless of the {@link MergePolicy} used. Note that you can achieve the
* same thing by using {@link NoMergePolicy}, however with
* {@link NoMergeScheduler} you also ensure that no unnecessary code of any
* {@link MergeScheduler} implementation is ever executed. Hence it is
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/NormsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/NormsWriter.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/NormsWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/NormsWriter.java Tue Feb 22 01:00:39 2011
@@ -32,7 +32,6 @@ import org.apache.lucene.store.IndexOutp
final class NormsWriter extends InvertedDocEndConsumer {
- private FieldInfos fieldInfos;
@Override
public void abort() {}
@@ -40,16 +39,11 @@ final class NormsWriter extends Inverted
// We only write the _X.nrm file at flush
void files(Collection<String> files) {}
- @Override
- void setFieldInfos(FieldInfos fieldInfos) {
- this.fieldInfos = fieldInfos;
- }
-
/** Produce _X.nrm if any document had a field with norms
* not disabled */
@Override
public void flush(Map<FieldInfo,InvertedDocEndConsumerPerField> fieldsToFlush, SegmentWriteState state) throws IOException {
- if (!fieldInfos.hasNorms()) {
+ if (!state.fieldInfos.hasNorms()) {
return;
}
@@ -59,15 +53,10 @@ final class NormsWriter extends Inverted
try {
normsOut.writeBytes(SegmentMerger.NORMS_HEADER, 0, SegmentMerger.NORMS_HEADER.length);
- final int numField = fieldInfos.size();
-
int normCount = 0;
- for(int fieldNumber=0;fieldNumber<numField;fieldNumber++) {
-
- final FieldInfo fieldInfo = fieldInfos.fieldInfo(fieldNumber);
-
- NormsWriterPerField toWrite = (NormsWriterPerField) fieldsToFlush.get(fieldInfo);
+ for (FieldInfo fi : state.fieldInfos) {
+ NormsWriterPerField toWrite = (NormsWriterPerField) fieldsToFlush.get(fi);
int upto = 0;
if (toWrite != null && toWrite.upto > 0) {
@@ -87,7 +76,7 @@ final class NormsWriter extends Inverted
assert upto == toWrite.upto;
toWrite.reset();
- } else if (fieldInfo.isIndexed && !fieldInfo.omitNorms) {
+ } else if (fi.isIndexed && !fi.omitNorms) {
normCount++;
// Fill entire field with default norm:
for(;upto<state.numDocs;upto++)
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/NormsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/NormsWriterPerField.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/NormsWriterPerField.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/NormsWriterPerField.java Tue Feb 22 01:00:39 2011
@@ -72,7 +72,7 @@ final class NormsWriterPerField extends
assert norms.length == upto;
norms = ArrayUtil.grow(norms, 1+upto);
}
- final float norm = similarity.computeNorm(fieldInfo.name, fieldState);
+ final float norm = similarity.computeNorm(fieldState);
norms[upto] = similarity.encodeNormValue(norm);
docIDs[upto] = docState.docID;
upto++;
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/OrdTermState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/OrdTermState.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/OrdTermState.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/OrdTermState.java Tue Feb 22 01:00:39 2011
@@ -30,4 +30,9 @@ public class OrdTermState extends TermSt
assert other instanceof OrdTermState : "can not copy from " + other.getClass().getName();
this.ord = ((OrdTermState) other).ord;
}
+
+ @Override
+ public String toString() {
+ return "OrdTermState ord=" + ord;
+ }
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/ParallelReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/ParallelReader.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/ParallelReader.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/ParallelReader.java Tue Feb 22 01:00:39 2011
@@ -22,11 +22,12 @@ import org.apache.lucene.document.FieldS
import org.apache.lucene.document.FieldSelectorResult;
import org.apache.lucene.document.Fieldable;
import org.apache.lucene.util.Bits;
-import org.apache.lucene.search.FieldCache; // not great (circular); used only to purge FieldCache entry on close
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.MapBackedSet;
import java.io.IOException;
import java.util.*;
+import java.util.concurrent.ConcurrentHashMap;
/** An IndexReader which reads multiple, parallel indexes. Each index added
@@ -73,6 +74,7 @@ public class ParallelReader extends Inde
public ParallelReader(boolean closeSubReaders) throws IOException {
super();
this.incRefReaders = !closeSubReaders;
+ readerFinishedListeners = new MapBackedSet<ReaderFinishedListener>(new ConcurrentHashMap<ReaderFinishedListener,Boolean>());
}
/** {@inheritDoc} */
@@ -529,8 +531,6 @@ public class ParallelReader extends Inde
readers.get(i).close();
}
}
-
- FieldCache.DEFAULT.purge(this);
}
@Override
@@ -548,6 +548,21 @@ public class ParallelReader extends Inde
return topLevelReaderContext;
}
+ @Override
+ public void addReaderFinishedListener(ReaderFinishedListener listener) {
+ super.addReaderFinishedListener(listener);
+ for (IndexReader reader : readers) {
+ reader.addReaderFinishedListener(listener);
+ }
+ }
+
+ @Override
+ public void removeReaderFinishedListener(ReaderFinishedListener listener) {
+ super.removeReaderFinishedListener(listener);
+ for (IndexReader reader : readers) {
+ reader.removeReaderFinishedListener(listener);
+ }
+ }
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/Payload.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/Payload.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/Payload.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/Payload.java Tue Feb 22 01:00:39 2011
@@ -17,8 +17,6 @@ package org.apache.lucene.index;
* limitations under the License.
*/
-import java.io.Serializable;
-
import org.apache.lucene.analysis.TokenStream;
import org.apache.lucene.util.ArrayUtil;
@@ -34,7 +32,7 @@ import org.apache.lucene.util.ArrayUtil;
* to retrieve the payloads from the index.<br>
*
*/
-public class Payload implements Serializable, Cloneable {
+public class Payload implements Cloneable {
/** the byte array containing the payload data */
protected byte[] data;
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/PayloadProcessorProvider.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/PayloadProcessorProvider.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/PayloadProcessorProvider.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/PayloadProcessorProvider.java Tue Feb 22 01:00:39 2011
@@ -24,7 +24,7 @@ import org.apache.lucene.util.BytesRef;
/**
* Provides a {@link DirPayloadProcessor} to be used for a {@link Directory}.
- * This allows using differnt {@link DirPayloadProcessor}s for different
+ * This allows using different {@link DirPayloadProcessor}s for different
* directories, for e.g. to perform different processing of payloads of
* different directories.
* <p>
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/PerFieldCodecWrapper.java Tue Feb 22 01:00:39 2011
@@ -67,7 +67,7 @@ final class PerFieldCodecWrapper extends
@Override
public TermsConsumer addField(FieldInfo field) throws IOException {
- final FieldsConsumer fields = consumers.get(field.codecId);
+ final FieldsConsumer fields = consumers.get(field.getCodecId());
return fields.addField(field);
}
@@ -100,18 +100,16 @@ final class PerFieldCodecWrapper extends
public FieldsReader(Directory dir, FieldInfos fieldInfos, SegmentInfo si,
int readBufferSize, int indexDivisor) throws IOException {
- final int fieldCount = fieldInfos.size();
final Map<Codec, FieldsProducer> producers = new HashMap<Codec, FieldsProducer>();
boolean success = false;
try {
- for (int i = 0; i < fieldCount; i++) {
- FieldInfo fi = fieldInfos.fieldInfo(i);
+ for (FieldInfo fi : fieldInfos) {
if (fi.isIndexed) { // TODO this does not work for non-indexed fields
fields.add(fi.name);
- Codec codec = segmentCodecs.codecs[fi.codecId];
+ Codec codec = segmentCodecs.codecs[fi.getCodecId()];
if (!producers.containsKey(codec)) {
producers.put(codec, codec.fieldsProducer(new SegmentReadState(dir,
- si, fieldInfos, readBufferSize, indexDivisor, ""+fi.codecId)));
+ si, fieldInfos, readBufferSize, indexDivisor, ""+fi.getCodecId())));
}
codecs.put(fi.name, producers.get(codec));
}
@@ -204,6 +202,7 @@ final class PerFieldCodecWrapper extends
}
}
+ @Override
public FieldsProducer fieldsProducer(SegmentReadState state)
throws IOException {
return new FieldsReader(state.dir, state.fieldInfos, state.segmentInfo,
@@ -213,7 +212,7 @@ final class PerFieldCodecWrapper extends
@Override
public void files(Directory dir, SegmentInfo info, String codecId, Set<String> files)
throws IOException {
- // ignore codecid sicne segmentCodec will assign it per codec
+ // ignore codecid since segmentCodec will assign it per codec
segmentCodecs.files(dir, info, files);
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/PersistentSnapshotDeletionPolicy.java Tue Feb 22 01:00:39 2011
@@ -103,7 +103,7 @@ public class PersistentSnapshotDeletionP
* @param mode
* specifies whether a new index should be created, deleting all
* existing snapshots information (immediately), or open an existing
- * index, initializing the class with the snapsthots information.
+ * index, initializing the class with the snapshots information.
* @param matchVersion
* specifies the {@link Version} that should be used when opening the
* IndexWriter.
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentCodecs.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentCodecs.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentCodecs.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentCodecs.java Tue Feb 22 01:00:39 2011
@@ -74,22 +74,20 @@ final class SegmentCodecs implements Clo
}
static SegmentCodecs build(FieldInfos infos, CodecProvider provider) {
- final int size = infos.size();
final Map<Codec, Integer> codecRegistry = new IdentityHashMap<Codec, Integer>();
final ArrayList<Codec> codecs = new ArrayList<Codec>();
- for (int i = 0; i < size; i++) {
- final FieldInfo info = infos.fieldInfo(i);
- if (info.isIndexed) {
+ for (FieldInfo fi : infos) {
+ if (fi.isIndexed) {
final Codec fieldCodec = provider.lookup(provider
- .getFieldCodec(info.name));
+ .getFieldCodec(fi.name));
Integer ord = codecRegistry.get(fieldCodec);
if (ord == null) {
ord = Integer.valueOf(codecs.size());
codecRegistry.put(fieldCodec, ord);
codecs.add(fieldCodec);
}
- info.codecId = ord.intValue();
+ fi.setCodecId(ord.intValue());
}
}
return new SegmentCodecs(provider, codecs.toArray(Codec.EMPTY));
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentInfo.java Tue Feb 22 01:00:39 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
@@ -41,6 +42,9 @@ import java.util.ArrayList;
*/
public final class SegmentInfo {
+ @Deprecated
+ // remove with hasVector and hasProx
+ static final int CHECK_FIELDINFOS = -2; // hasVector and hasProx use this for bw compatibility
static final int NO = -1; // e.g. no norms; no deletes;
static final int YES = 1; // e.g. have norms; have deletes;
static final int WITHOUT_GEN = 0; // a file name that has no GEN in it.
@@ -62,15 +66,15 @@ 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;
- private List<String> files; // cached list of files that this segment uses
+ private volatile List<String> files; // cached list of files that this segment uses
// in the Directory
- private long sizeInBytesNoStore = -1; // total byte size of all but the store files (computed on demand)
- private long sizeInBytesWithStore = -1; // total byte size of all of our files (computed on demand)
+ private volatile long sizeInBytesNoStore = -1; // total byte size of all but the store files (computed on demand)
+ private volatile long sizeInBytesWithStore = -1; // total byte size of all of our files (computed on demand)
@Deprecated private int docStoreOffset; // if this segment shares stored fields & vectors, this
// offset is where in that file this segment's docs begin
@@ -80,23 +84,33 @@ public final class SegmentInfo {
private int delCount; // How many deleted docs in this segment
- private boolean hasProx; // True if this segment has any fields with omitTermFreqAndPositions==false
+ @Deprecated
+ // remove when we don't have to support old indexes anymore that had this field
+ private int hasProx = CHECK_FIELDINFOS; // True if this segment has any fields with omitTermFreqAndPositions==false
+
+ @Deprecated
+ // remove when we don't have to support old indexes anymore that had this field
+ private int hasVectors = CHECK_FIELDINFOS; // True if this segment wrote term vectors
- private boolean hasVectors; // True if this segment wrote term vectors
+ private FieldInfos fieldInfos;
private SegmentCodecs segmentCodecs;
private Map<String,String> diagnostics;
- // Tracks the Lucene version this segment was created with, since 3.1. Null
+ // Tracks the Lucene version this segment was created with, since 3.1. Null
// indicates an older than 3.0 index, and it's used to detect a too old index.
- // The format expected is "x.y" - "2.x" for pre-3.0 indexes (or null), and
+ // The format expected is "x.y" - "2.x" for pre-3.0 indexes (or null), and
// specific versions afterwards ("3.0", "3.1" etc.).
// see Constants.LUCENE_MAIN_VERSION.
private String version;
-
+
+ // NOTE: only used in-RAM by IW to track buffered deletes;
+ // this is never written to/read from the Directory
+ private long bufferedDeletesGen;
+
public SegmentInfo(String name, int docCount, Directory dir, boolean isCompoundFile,
- boolean hasProx, SegmentCodecs segmentCodecs, boolean hasVectors) {
+ SegmentCodecs segmentCodecs, FieldInfos fieldInfos) {
this.name = name;
this.docCount = docCount;
this.dir = dir;
@@ -104,18 +118,17 @@ public final class SegmentInfo {
this.isCompoundFile = isCompoundFile;
this.docStoreOffset = -1;
this.docStoreSegment = name;
- this.hasProx = hasProx;
this.segmentCodecs = segmentCodecs;
- 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;
@@ -126,11 +139,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;
@@ -182,17 +198,35 @@ 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;
+ Directory dir0 = dir;
+ if (isCompoundFile) {
+ 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();
+ }
+ }
+
delCount = input.readInt();
assert delCount <= docCount;
- hasProx = input.readByte() == YES;
+ hasProx = input.readByte();
// System.out.println(Thread.currentThread().getName() + ": si.read hasProx=" + hasProx + " seg=" + name);
segmentCodecs = new SegmentCodecs(codecs);
@@ -206,7 +240,7 @@ public final class SegmentInfo {
diagnostics = input.readStringStringMap();
if (format <= DefaultSegmentInfosWriter.FORMAT_HAS_VECTORS) {
- hasVectors = input.readByte() == 1;
+ hasVectors = input.readByte();
} else {
final String storesSegment;
final String ext;
@@ -227,7 +261,11 @@ public final class SegmentInfo {
dirToTest = dir;
}
try {
- hasVectors = dirToTest.fileExists(IndexFileNames.segmentFileName(storesSegment, "", IndexFileNames.VECTORS_INDEX_EXTENSION));
+ if (dirToTest.fileExists(IndexFileNames.segmentFileName(storesSegment, "", IndexFileNames.VECTORS_INDEX_EXTENSION))) {
+ hasVectors = YES;
+ } else {
+ hasVectors = NO;
+ }
} finally {
if (isCompoundFile) {
dirToTest.close();
@@ -243,35 +281,42 @@ public final class SegmentInfo {
*/
public long sizeInBytes(boolean includeDocStores) throws IOException {
if (includeDocStores) {
- if (sizeInBytesWithStore != -1) return sizeInBytesWithStore;
- sizeInBytesWithStore = 0;
+ if (sizeInBytesWithStore != -1) {
+ return sizeInBytesWithStore;
+ }
+ long sum = 0;
for (final String fileName : files()) {
- // We don't count bytes used by a shared doc store against this segment
+ // We don't count bytes used by a shared doc store
+ // against this segment
if (docStoreOffset == -1 || !IndexFileNames.isDocStoreFile(fileName)) {
- sizeInBytesWithStore += dir.fileLength(fileName);
+ sum += dir.fileLength(fileName);
}
}
+ sizeInBytesWithStore = sum;
return sizeInBytesWithStore;
} else {
- if (sizeInBytesNoStore != -1) return sizeInBytesNoStore;
- sizeInBytesNoStore = 0;
+ if (sizeInBytesNoStore != -1) {
+ return sizeInBytesNoStore;
+ }
+ long sum = 0;
for (final String fileName : files()) {
if (IndexFileNames.isDocStoreFile(fileName)) {
continue;
}
- sizeInBytesNoStore += dir.fileLength(fileName);
+ sum += dir.fileLength(fileName);
}
+ sizeInBytesNoStore = sum;
return sizeInBytesNoStore;
}
}
- public boolean getHasVectors() throws IOException {
- return hasVectors;
+ public boolean getHasVectors() {
+ return hasVectors == CHECK_FIELDINFOS ?
+ (fieldInfos == null ? true : fieldInfos.hasVectors()) : hasVectors == YES;
}
- public void setHasVectors(boolean v) {
- hasVectors = v;
- clearFiles();
+ public FieldInfos getFieldInfos() {
+ return fieldInfos;
}
public boolean hasDeletions() {
@@ -289,17 +334,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);
+ SegmentInfo si = new SegmentInfo(name, docCount, dir, isCompoundFile, segmentCodecs,
+ fieldInfos == null ? null : (FieldInfos) fieldInfos.clone());
si.docStoreOffset = docStoreOffset;
si.docStoreSegment = docStoreSegment;
si.docStoreIsCompoundFile = docStoreIsCompoundFile;
@@ -307,8 +353,12 @@ 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.hasProx = hasProx;
si.hasVectors = hasVectors;
si.version = version;
return si;
@@ -330,7 +380,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;
}
/**
@@ -340,7 +395,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;
}
@@ -350,10 +405,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>();
}
}
@@ -364,12 +418,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();
}
/**
@@ -379,7 +434,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);
@@ -394,7 +449,7 @@ public final class SegmentInfo {
*/
void setUseCompoundFile(boolean isCompoundFile) {
this.isCompoundFile = isCompoundFile;
- clearFiles();
+ clearFilesCache();
}
/**
@@ -427,7 +482,7 @@ public final class SegmentInfo {
@Deprecated
public void setDocStoreIsCompoundFile(boolean docStoreIsCompoundFile) {
this.docStoreIsCompoundFile = docStoreIsCompoundFile;
- clearFiles();
+ clearFilesCache();
}
@Deprecated
@@ -435,7 +490,7 @@ public final class SegmentInfo {
docStoreOffset = offset;
docStoreSegment = segment;
docStoreIsCompoundFile = isCompoundFile;
- clearFiles();
+ clearFilesCache();
}
@Deprecated
@@ -446,7 +501,7 @@ public final class SegmentInfo {
@Deprecated
void setDocStoreOffset(int offset) {
docStoreOffset = offset;
- clearFiles();
+ clearFilesCache();
}
@Deprecated
@@ -474,27 +529,24 @@ 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());
}
}
output.writeByte((byte) (isCompoundFile ? YES : NO));
output.writeInt(delCount);
- output.writeByte((byte) (hasProx ? 1:0));
+ output.writeByte((byte) hasProx);
segmentCodecs.write(output);
output.writeStringStringMap(diagnostics);
- output.writeByte((byte) (hasVectors ? 1 : 0));
- }
-
- void setHasProx(boolean hasProx) {
- this.hasProx = hasProx;
- clearFiles();
+ output.writeByte((byte) hasVectors);
}
public boolean getHasProx() {
- return hasProx;
+ return hasProx == CHECK_FIELDINFOS ?
+ (fieldInfos == null ? true : fieldInfos.hasProx()) : hasProx == YES;
}
/** Can only be called once. */
@@ -550,7 +602,7 @@ public final class SegmentInfo {
} else {
fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.FIELDS_EXTENSION));
- if (hasVectors) {
+ if (getHasVectors()) {
fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_INDEX_EXTENSION));
fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
fileSet.add(IndexFileNames.segmentFileName(docStoreSegment, "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
@@ -559,7 +611,7 @@ public final class SegmentInfo {
} else if (!useCompoundFile) {
fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.FIELDS_INDEX_EXTENSION));
fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.FIELDS_EXTENSION));
- if (hasVectors) {
+ if (getHasVectors()) {
fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.VECTORS_INDEX_EXTENSION));
fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.VECTORS_DOCUMENTS_EXTENSION));
fileSet.add(IndexFileNames.segmentFileName(name, "", IndexFileNames.VECTORS_FIELDS_EXTENSION));
@@ -572,11 +624,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 +640,7 @@ public final class SegmentInfo {
/* Called whenever any change is made that affects which
* files this segment has. */
- private void clearFiles() {
+ void clearFilesCache() {
files = null;
sizeInBytesNoStore = -1;
sizeInBytesWithStore = -1;
@@ -623,7 +675,7 @@ public final class SegmentInfo {
if (this.dir != dir) {
s.append('x');
}
- if (hasVectors) {
+ if (getHasVectors()) {
s.append('v');
}
s.append(docCount);
@@ -672,16 +724,23 @@ public final class SegmentInfo {
* <b>NOTE:</b> this method is used for internal purposes only - you should
* not modify the version of a SegmentInfo, or it may result in unexpected
* exceptions thrown when you attempt to open the index.
- *
+ *
* @lucene.internal
*/
public void setVersion(String version) {
this.version = version;
}
-
+
/** Returns the version of the code which wrote the segment. */
public String getVersion() {
return version;
}
-
+
+ long getBufferedDeletesGen() {
+ return bufferedDeletesGen;
+ }
+
+ void setBufferedDeletesGen(long v) {
+ bufferedDeletesGen = v;
+ }
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentInfos.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentInfos.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentInfos.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentInfos.java Tue Feb 22 01:00:39 2011
@@ -17,25 +17,25 @@ 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.codecs.CodecProvider;
+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
@@ -308,6 +308,19 @@ public final class SegmentInfos extends
}
}
+ /** Prunes any segment whose docs are all deleted. */
+ public void pruneDeletedSegments() {
+ int segIdx = 0;
+ while(segIdx < size()) {
+ final SegmentInfo info = info(segIdx);
+ if (info.getDelCount() == info.docCount) {
+ remove(segIdx);
+ } else {
+ segIdx++;
+ }
+ }
+ }
+
/**
* Returns a copy of this instance, also copying each
* SegmentInfo.
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Tue Feb 22 01:00:39 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 {
@@ -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;
@@ -207,9 +206,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);
@@ -223,14 +221,13 @@ final class SegmentMerger {
}
}
final SegmentCodecs codecInfo = SegmentCodecs.build(fieldInfos, this.codecs);
- fieldInfos.write(directory, segment + ".fnm");
+ fieldInfos.write(directory, segment + "." + IndexFileNames.FIELD_INFOS_EXTENSION);
int docCount = 0;
setMatchingSegmentReaders();
- // merge field values
- final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment, fieldInfos);
+ final FieldsWriter fieldsWriter = new FieldsWriter(directory, segment);
try {
int idx = 0;
@@ -266,7 +263,7 @@ final class SegmentMerger {
// details.
throw new RuntimeException("mergeFields produced an invalid result: docCount is " + docCount + " but fdx file size is " + fdxFileLength + " file=" + fileName + " file exists?=" + directory.fileExists(fileName) + "; now aborting this merge to prevent index corruption");
- segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codecInfo);
+ segmentWriteState = new SegmentWriteState(null, directory, segment, fieldInfos, docCount, termIndexInterval, codecInfo, null);
return docCount;
}
@@ -312,7 +309,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 +336,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);
}
}
@@ -574,8 +571,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/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentReader.java Tue Feb 22 01:00:39 2011
@@ -22,24 +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.search.FieldCache; // not great (circular); used only to purge FieldCache entry on close
import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.CloseableThreadLocal;
/**
* @lucene.experimental
@@ -121,7 +119,7 @@ public class SegmentReader extends Index
}
cfsDir = dir0;
- fieldInfos = new FieldInfos(cfsDir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.FIELD_INFOS_EXTENSION));
+ fieldInfos = si.getFieldInfos();
this.termsIndexDivisor = termsIndexDivisor;
@@ -183,13 +181,9 @@ public class SegmentReader extends Index
storeCFSReader.close();
}
- // Force FieldCache to evict our entries at this
- // point. If the exception occurred while
- // initializing the core readers, then
- // origInstance will be null, and we don't want
- // to call FieldCache.purge (it leads to NPE):
+ // Now, notify any ReaderFinished listeners:
if (origInstance != null) {
- FieldCache.DEFAULT.purge(origInstance);
+ origInstance.notifyReaderFinishedListeners();
}
}
}
@@ -603,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);
}
}
@@ -633,6 +627,7 @@ public class SegmentReader extends Index
clone.si = si;
clone.readBufferSize = readBufferSize;
clone.pendingDeleteCount = pendingDeleteCount;
+ clone.readerFinishedListeners = readerFinishedListeners;
if (!openReadOnly && hasChanges) {
// My pending changes transfer to the new reader
@@ -663,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());
@@ -739,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);
@@ -884,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);
}
@@ -963,8 +956,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
@@ -1203,4 +1195,14 @@ public class SegmentReader extends Index
public int getTermInfosIndexDivisor() {
return core.termsIndexDivisor;
}
+
+ @Override
+ protected void readerFinished() {
+ // Do nothing here -- we have more careful control on
+ // when to notify that a SegmentReader has finished,
+ // because a given core is shared across many cloned
+ // SegmentReaders. We only notify once that core is no
+ // longer used (all SegmentReaders sharing it have been
+ // closed).
+ }
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java Tue Feb 22 01:00:39 2011
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
import java.io.PrintStream;
import org.apache.lucene.store.Directory;
+import org.apache.lucene.util.BitVector;
/**
* @lucene.experimental
@@ -32,6 +33,16 @@ public class SegmentWriteState {
public final int numDocs;
public boolean hasVectors;
+ // Deletes to apply while we are flushing the segment. A
+ // Term is enrolled in here if it was deleted at one
+ // point, and it's mapped to the docIDUpto, meaning any
+ // docID < docIDUpto containing this term should be
+ // deleted.
+ public final BufferedDeletes segDeletes;
+
+ // Lazily created:
+ public BitVector deletedDocs;
+
final SegmentCodecs segmentCodecs;
public final String codecId;
@@ -42,23 +53,10 @@ public class SegmentWriteState {
* tweaking this is rarely useful.*/
public int termIndexInterval; // TODO: this should be private to the codec, not settable here or in IWC
- /** Expert: The fraction of TermDocs entries stored in skip tables,
- * used to accelerate {@link DocsEnum#advance(int)}. Larger values result in
- * smaller indexes, greater acceleration, but fewer accelerable cases, while
- * smaller values result in bigger indexes, less acceleration and more
- * accelerable cases. More detailed experiments would be useful here. */
- public final int skipInterval = 16;
-
- /** Expert: The maximum number of skip levels. Smaller values result in
- * slightly smaller indexes, but slower skipping in big posting lists.
- */
- public final int maxSkipLevels = 10;
-
-
-
public SegmentWriteState(PrintStream infoStream, Directory directory, String segmentName, FieldInfos fieldInfos,
- int numDocs, int termIndexInterval, SegmentCodecs segmentCodecs) {
+ int numDocs, int termIndexInterval, SegmentCodecs segmentCodecs, BufferedDeletes segDeletes) {
this.infoStream = infoStream;
+ this.segDeletes = segDeletes;
this.directory = directory;
this.segmentName = segmentName;
this.fieldInfos = fieldInfos;
@@ -80,5 +78,6 @@ public class SegmentWriteState {
termIndexInterval = state.termIndexInterval;
segmentCodecs = state.segmentCodecs;
this.codecId = codecId;
+ segDeletes = state.segDeletes;
}
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/StoredFieldsWriter.java Tue Feb 22 01:00:39 2011
@@ -28,16 +28,14 @@ final class StoredFieldsWriter {
FieldsWriter fieldsWriter;
final DocumentsWriterPerThread docWriter;
- final FieldInfos fieldInfos;
int lastDocID;
int freeCount;
final DocumentsWriterPerThread.DocState docState;
- public StoredFieldsWriter(DocumentsWriterPerThread docWriter, FieldInfos fieldInfos) {
+ public StoredFieldsWriter(DocumentsWriterPerThread docWriter) {
this.docWriter = docWriter;
- this.fieldInfos = fieldInfos;
this.docState = docWriter.docState;
}
@@ -79,7 +77,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/realtime_search/lucene/src/java/org/apache/lucene/index/Term.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/Term.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/Term.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/Term.java Tue Feb 22 01:00:39 2011
@@ -30,7 +30,7 @@ import org.apache.lucene.util.StringHelp
Note that terms may represent more than words from text fields, but also
things like dates, email addresses, urls, etc. */
-public final class Term implements Comparable<Term>, java.io.Serializable {
+public final class Term implements Comparable<Term> {
String field;
BytesRef bytes;
@@ -199,11 +199,4 @@ public final class Term implements Compa
@Override
public final String toString() { return field + ":" + bytes.utf8ToString(); }
-
- private void readObject(java.io.ObjectInputStream in)
- throws java.io.IOException, ClassNotFoundException
- {
- in.defaultReadObject();
- field = StringHelper.intern(field);
- }
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermState.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermState.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermState.java Tue Feb 22 01:00:39 2011
@@ -44,4 +44,9 @@ public abstract class TermState implemen
throw new RuntimeException(cnse);
}
}
-}
\ No newline at end of file
+
+ @Override
+ public String toString() {
+ return "TermState";
+ }
+}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermVectorOffsetInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermVectorOffsetInfo.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermVectorOffsetInfo.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermVectorOffsetInfo.java Tue Feb 22 01:00:39 2011
@@ -1,7 +1,5 @@
package org.apache.lucene.index;
-import java.io.Serializable;
-
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
@@ -24,7 +22,7 @@ import java.io.Serializable;
* offset information. This offset information is the character offset as set during the Analysis phase (and thus may not be the actual offset in the
* original content).
*/
-public class TermVectorOffsetInfo implements Serializable {
+public class TermVectorOffsetInfo {
/**
* Convenience declaration when creating a {@link org.apache.lucene.index.TermPositionVector} that stores only position information.
*/
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermVectorsTermsWriterPerField.java Tue Feb 22 01:00:39 2011
@@ -263,6 +263,7 @@ final class TermVectorsTermsWriterPerFie
int[] lastOffsets; // Last offset we saw
int[] lastPositions; // Last position where this term occurred
+ @Override
ParallelPostingsArray newInstance(int size) {
return new TermVectorsPostingsArray(size);
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermsHash.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermsHash.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermsHash.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermsHash.java Tue Feb 22 01:00:39 2011
@@ -74,12 +74,6 @@ final class TermsHash extends InvertedDo
}
@Override
- void setFieldInfos(FieldInfos fieldInfos) {
- this.fieldInfos = fieldInfos;
- consumer.setFieldInfos(fieldInfos);
- }
-
- @Override
public void abort() {
reset();
try {
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/TermsHashConsumer.java Tue Feb 22 01:00:39 2011
@@ -26,10 +26,4 @@ abstract class TermsHashConsumer {
abstract void startDocument() throws IOException;
abstract void finishDocument(TermsHash termsHash) throws IOException;
abstract public TermsHashConsumerPerField addField(TermsHashPerField termsHashPerField, FieldInfo fieldInfo);
-
- FieldInfos fieldInfos;
-
- void setFieldInfos(FieldInfos fieldInfos) {
- this.fieldInfos = fieldInfos;
- }
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/BlockTermState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/BlockTermState.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/BlockTermState.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/BlockTermState.java Tue Feb 22 01:00:39 2011
@@ -16,6 +16,7 @@ package org.apache.lucene.index.codecs;
* limitations under the License.
*/
+import org.apache.lucene.index.DocsEnum; // javadocs
import org.apache.lucene.index.OrdTermState;
import org.apache.lucene.index.TermState;
@@ -50,6 +51,6 @@ public class BlockTermState extends OrdT
@Override
public String toString() {
- return super.toString() + "ord=" + ord + " docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termCount=" + termCount + " blockFP=" + blockFilePointer;
+ return "ord=" + ord + " docFreq=" + docFreq + " totalTermFreq=" + totalTermFreq + " termCount=" + termCount + " blockFP=" + blockFilePointer;
}
}
Modified: lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java?rev=1073192&r1=1073191&r2=1073192&view=diff
==============================================================================
--- lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java (original)
+++ lucene/dev/branches/realtime_search/lucene/src/java/org/apache/lucene/index/codecs/BlockTermsReader.java Tue Feb 22 01:00:39 2011
@@ -66,9 +66,6 @@ public class BlockTermsReader extends Fi
private final TreeMap<String,FieldReader> fields = new TreeMap<String,FieldReader>();
- // Comparator that orders our terms
- private final Comparator<BytesRef> termComp;
-
// Caches the most recently looked-up field + terms:
private final DoubleBarrelLRUCache<FieldAndTerm,BlockTermState> termsCache;
@@ -108,17 +105,16 @@ public class BlockTermsReader extends Fi
}
}
- private String segment;
+ //private String segment;
public BlockTermsReader(TermsIndexReaderBase indexReader, Directory dir, FieldInfos fieldInfos, String segment, PostingsReaderBase postingsReader, int readBufferSize,
- Comparator<BytesRef> termComp, int termsCacheSize, String codecId)
+ int termsCacheSize, String codecId)
throws IOException {
this.postingsReader = postingsReader;
termsCache = new DoubleBarrelLRUCache<FieldAndTerm,BlockTermState>(termsCacheSize);
- this.termComp = termComp;
- this.segment = segment;
+ //this.segment = segment;
in = dir.openInput(IndexFileNames.segmentFileName(segment, codecId, BlockTermsWriter.TERMS_EXTENSION),
readBufferSize);
@@ -260,7 +256,7 @@ public class BlockTermsReader extends Fi
@Override
public Comparator<BytesRef> getComparator() {
- return termComp;
+ return BytesRef.getUTF8SortedAsUnicodeComparator();
}
@Override
@@ -342,23 +338,29 @@ public class BlockTermsReader extends Fi
@Override
public Comparator<BytesRef> getComparator() {
- return termComp;
+ return BytesRef.getUTF8SortedAsUnicodeComparator();
}
+ // TODO: we may want an alternate mode here which is
+ // "if you are about to return NOT_FOUND I won't use
+ // the terms data from that"; eg FuzzyTermsEnum will
+ // (usually) just immediately call seek again if we
+ // return NOT_FOUND so it's a waste for us to fill in
+ // the term that was actually NOT_FOUND
@Override
public SeekStatus seek(final BytesRef target, final boolean useCache) throws IOException {
if (indexEnum == null) {
throw new IllegalStateException("terms index was not loaded");
}
-
- //System.out.println("BTR.seek seg=" + segment + " target=" + fieldInfo.name + ":" + target.utf8ToString() + " " + target + " current=" + term().utf8ToString() + " " + term() + " useCache=" + useCache + " indexIsCurrent=" + indexIsCurrent + " didIndexNext=" + didIndexNext + " seekPending=" + seekPending + " divisor=" + indexReader.getDivisor() + " this=" + this);
+
/*
+ System.out.println("BTR.seek seg=" + segment + " target=" + fieldInfo.name + ":" + target.utf8ToString() + " " + target + " current=" + term().utf8ToString() + " " + term() + " useCache=" + useCache + " indexIsCurrent=" + indexIsCurrent + " didIndexNext=" + didIndexNext + " seekPending=" + seekPending + " divisor=" + indexReader.getDivisor() + " this=" + this);
if (didIndexNext) {
if (nextIndexTerm == null) {
- //System.out.println(" nextIndexTerm=null");
+ System.out.println(" nextIndexTerm=null");
} else {
- //System.out.println(" nextIndexTerm=" + nextIndexTerm.utf8ToString());
+ System.out.println(" nextIndexTerm=" + nextIndexTerm.utf8ToString());
}
}
*/
@@ -386,7 +388,7 @@ public class BlockTermsReader extends Fi
// is after current term but before next index term:
if (indexIsCurrent) {
- final int cmp = termComp.compare(term, target);
+ final int cmp = BytesRef.getUTF8SortedAsUnicodeComparator().compare(term, target);
if (cmp == 0) {
// Already at the requested term
@@ -404,7 +406,7 @@ public class BlockTermsReader extends Fi
didIndexNext = true;
}
- if (nextIndexTerm == null || termComp.compare(target, nextIndexTerm) < 0) {
+ if (nextIndexTerm == null || BytesRef.getUTF8SortedAsUnicodeComparator().compare(target, nextIndexTerm) < 0) {
// Optimization: requested term is within the
// same term block we are now in; skip seeking
// (but do scanning):
@@ -434,48 +436,175 @@ public class BlockTermsReader extends Fi
state.ord = indexEnum.ord()-1;
}
- // NOTE: the first _next() after an index seek is
- // a bit wasteful, since it redundantly reads some
- // suffix bytes into the buffer. We could avoid storing
- // those bytes in the primary file, but then when
- // next()ing over an index term we'd have to
- // special case it:
term.copy(indexEnum.term());
//System.out.println(" seek: term=" + term.utf8ToString());
} else {
- ////System.out.println(" skip seek");
+ //System.out.println(" skip seek");
+ if (state.termCount == state.blockTermCount && !nextBlock()) {
+ indexIsCurrent = false;
+ return SeekStatus.END;
+ }
}
seekPending = false;
- // Now scan:
- while (_next() != null) {
- final int cmp = termComp.compare(term, target);
- if (cmp == 0) {
- // Match!
- if (useCache) {
- // Store in cache
- decodeMetaData();
- termsCache.put(new FieldAndTerm(fieldTerm), (BlockTermState) state.clone());
+ int common = 0;
+
+ // Scan within block. We could do this by calling
+ // _next() and testing the resulting term, but this
+ // is wasteful. Instead, we first confirm the
+ // target matches the common prefix of this block,
+ // and then we scan the term bytes directly from the
+ // termSuffixesreader's byte[], saving a copy into
+ // the BytesRef term per term. Only when we return
+ // do we then copy the bytes into the term.
+
+ while(true) {
+
+ // First, see if target term matches common prefix
+ // in this block:
+ if (common < termBlockPrefix) {
+ final int cmp = (term.bytes[common]&0xFF) - (target.bytes[target.offset + common]&0xFF);
+ if (cmp < 0) {
+
+ // TODO: maybe we should store common prefix
+ // in block header? (instead of relying on
+ // last term of previous block)
+
+ // Target's prefix is after the common block
+ // prefix, so term cannot be in this block
+ // but it could be in next block. We
+ // must scan to end-of-block to set common
+ // prefix for next block:
+ if (state.termCount < state.blockTermCount) {
+ while(state.termCount < state.blockTermCount-1) {
+ state.termCount++;
+ state.ord++;
+ termSuffixesReader.skipBytes(termSuffixesReader.readVInt());
+ }
+ final int suffix = termSuffixesReader.readVInt();
+ term.length = termBlockPrefix + suffix;
+ if (term.bytes.length < term.length) {
+ term.grow(term.length);
+ }
+ termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
+ }
+ state.ord++;
+
+ if (!nextBlock()) {
+ indexIsCurrent = false;
+ return SeekStatus.END;
+ }
+ common = 0;
+
+ } else if (cmp > 0) {
+ // Target's prefix is before the common prefix
+ // of this block, so we position to start of
+ // block and return NOT_FOUND:
+ assert state.termCount == 0;
+
+ final int suffix = termSuffixesReader.readVInt();
+ term.length = termBlockPrefix + suffix;
+ if (term.bytes.length < term.length) {
+ term.grow(term.length);
+ }
+ termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
+ return SeekStatus.NOT_FOUND;
+ } else {
+ common++;
}
- //System.out.println(" FOUND");
- return SeekStatus.FOUND;
- } else if (cmp > 0) {
- //System.out.println(" NOT_FOUND term=" + term.utf8ToString());
- return SeekStatus.NOT_FOUND;
+
+ continue;
}
-
+
+ // Test every term in this block
+ while (true) {
+ state.termCount++;
+ state.ord++;
+
+ final int suffix = termSuffixesReader.readVInt();
+
+ // We know the prefix matches, so just compare the new suffix:
+ final int termLen = termBlockPrefix + suffix;
+ int bytePos = termSuffixesReader.getPosition();
+
+ boolean next = false;
+ final int limit = target.offset + (termLen < target.length ? termLen : target.length);
+ int targetPos = target.offset + termBlockPrefix;
+ while(targetPos < limit) {
+ final int cmp = (termSuffixes[bytePos++]&0xFF) - (target.bytes[targetPos++]&0xFF);
+ if (cmp < 0) {
+ // Current term is still before the target;
+ // keep scanning
+ next = true;
+ break;
+ } else if (cmp > 0) {
+ // Done! Current term is after target. Stop
+ // here, fill in real term, return NOT_FOUND.
+ term.length = termBlockPrefix + suffix;
+ if (term.bytes.length < term.length) {
+ term.grow(term.length);
+ }
+ termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
+ //System.out.println(" NOT_FOUND");
+ return SeekStatus.NOT_FOUND;
+ }
+ }
+
+ if (!next && target.length <= termLen) {
+ term.length = termBlockPrefix + suffix;
+ if (term.bytes.length < term.length) {
+ term.grow(term.length);
+ }
+ termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
+
+ if (target.length == termLen) {
+ // Done! Exact match. Stop here, fill in
+ // real term, return FOUND.
+ //System.out.println(" FOUND");
+
+ if (useCache) {
+ // Store in cache
+ decodeMetaData();
+ //System.out.println(" cache! state=" + state);
+ termsCache.put(new FieldAndTerm(fieldTerm), (BlockTermState) state.clone());
+ }
+
+ return SeekStatus.FOUND;
+ } else {
+ //System.out.println(" NOT_FOUND");
+ return SeekStatus.NOT_FOUND;
+ }
+ }
+
+ if (state.termCount == state.blockTermCount) {
+ // Must pre-fill term for next block's common prefix
+ term.length = termBlockPrefix + suffix;
+ if (term.bytes.length < term.length) {
+ term.grow(term.length);
+ }
+ termSuffixesReader.readBytes(term.bytes, termBlockPrefix, suffix);
+ break;
+ } else {
+ termSuffixesReader.skipBytes(suffix);
+ }
+ }
+
// The purpose of the terms dict index is to seek
// the enum to the closest index term before the
// term we are looking for. So, we should never
// cross another index term (besides the first
// one) while we are scanning:
+
assert indexIsCurrent;
- }
- indexIsCurrent = false;
- //System.out.println(" END");
- return SeekStatus.END;
+ if (!nextBlock()) {
+ //System.out.println(" END");
+ indexIsCurrent = false;
+ return SeekStatus.END;
+ }
+ common = 0;
+ }
}
@Override
@@ -515,12 +644,10 @@ public class BlockTermsReader extends Fi
decode all metadata up to the current term. */
private BytesRef _next() throws IOException {
//System.out.println("BTR._next seg=" + segment + " this=" + this + " termCount=" + state.termCount + " (vs " + state.blockTermCount + ")");
- if (state.termCount == state.blockTermCount) {
- if (!nextBlock()) {
- //System.out.println(" eof");
- indexIsCurrent = false;
- return null;
- }
+ if (state.termCount == state.blockTermCount && !nextBlock()) {
+ //System.out.println(" eof");
+ indexIsCurrent = false;
+ return null;
}
// TODO: cutover to something better for these ints! simple64?
@@ -646,6 +773,7 @@ public class BlockTermsReader extends Fi
return SeekStatus.FOUND;
}
+ @Override
public long ord() {
if (!doOrd) {
throw new UnsupportedOperationException();
@@ -688,7 +816,7 @@ public class BlockTermsReader extends Fi
}
//System.out.println(" termSuffixes len=" + len);
in.readBytes(termSuffixes, 0, len);
- termSuffixesReader.reset(termSuffixes);
+ termSuffixesReader.reset(termSuffixes, 0, len);
// docFreq, totalTermFreq
len = in.readVInt();
@@ -697,7 +825,7 @@ public class BlockTermsReader extends Fi
}
//System.out.println(" freq bytes len=" + len);
in.readBytes(docFreqBytes, 0, len);
- freqReader.reset(docFreqBytes);
+ freqReader.reset(docFreqBytes, 0, len);
metaDataUpto = 0;
state.termCount = 0;
@@ -716,23 +844,32 @@ public class BlockTermsReader extends Fi
if (!seekPending) {
// lazily catch up on metadata decode:
final int limit = state.termCount;
+ // We must set/incr state.termCount because
+ // postings impl can look at this
state.termCount = metaDataUpto;
+ // TODO: better API would be "jump straight to term=N"???
while (metaDataUpto < limit) {
- //System.out.println(" decode");
+ //System.out.println(" decode mdUpto=" + metaDataUpto);
// TODO: we could make "tiers" of metadata, ie,
// decode docFreq/totalTF but don't decode postings
// metadata; this way caller could get
// docFreq/totalTF w/o paying decode cost for
// postings
+
+ // TODO: if docFreq were bulk decoded we could
+ // just skipN here:
state.docFreq = freqReader.readVInt();
+ //System.out.println(" dF=" + state.docFreq);
if (!fieldInfo.omitTermFreqAndPositions) {
state.totalTermFreq = state.docFreq + freqReader.readVLong();
+ //System.out.println(" totTF=" + state.totalTermFreq);
}
+
postingsReader.nextTerm(fieldInfo, state);
metaDataUpto++;
state.termCount++;
}
- } else {
+ //} else {
//System.out.println(" skip! seekPending");
}
}