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 2012/01/29 12:33:48 UTC
svn commit: r1237245 - in /lucene/dev/trunk: ./
dev-tools/idea/lucene/contrib/ lucene/ lucene/contrib/
lucene/contrib/sandbox/src/test/org/apache/lucene/sandbox/queries/regex/
lucene/src/java/org/apache/lucene/codecs/
lucene/src/java/org/apache/lucene/...
Author: rmuir
Date: Sun Jan 29 11:33:46 2012
New Revision: 1237245
URL: http://svn.apache.org/viewvc?rev=1237245&view=rev
Log:
LUCENE-3661: move deletes under codec
Added:
lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/LiveDocsFormat.java
- copied unchanged from r1237242, lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/codecs/LiveDocsFormat.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/BitVector.java
- copied unchanged from r1237242, lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/codecs/lucene40/BitVector.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40LiveDocsFormat.java
- copied unchanged from r1237242, lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40LiveDocsFormat.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java
- copied unchanged from r1237242, lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextLiveDocsFormat.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/MutableBits.java
- copied unchanged from r1237242, lucene/dev/branches/lucene3661/lucene/src/java/org/apache/lucene/util/MutableBits.java
lucene/dev/trunk/lucene/src/test/org/apache/lucene/codecs/lucene40/TestBitVector.java
- copied unchanged from r1237242, lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/codecs/lucene40/TestBitVector.java
lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestMixedCodecs.java
- copied unchanged from r1237242, lucene/dev/branches/lucene3661/lucene/src/test/org/apache/lucene/index/TestMixedCodecs.java
Removed:
lucene/dev/trunk/lucene/src/java/org/apache/lucene/util/BitVector.java
lucene/dev/trunk/lucene/src/test/org/apache/lucene/util/TestBitVector.java
Modified:
lucene/dev/trunk/ (props changed)
lucene/dev/trunk/dev-tools/idea/lucene/contrib/ (props changed)
lucene/dev/trunk/lucene/ (props changed)
lucene/dev/trunk/lucene/contrib/CHANGES.txt (props changed)
lucene/dev/trunk/lucene/contrib/sandbox/src/test/org/apache/lucene/sandbox/queries/regex/TestSpanRegexQuery.java (props changed)
lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/Codec.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/MappingMultiDocsEnum.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/appending/AppendingCodec.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xCodec.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/TermInfosReaderIndex.java (props changed)
lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/CheckIndex.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DirectoryReader.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileNames.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MergePolicy.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MergeState.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java
lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java
lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWCodec.java
lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexReaderReopen.java
lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
lucene/dev/trunk/modules/analysis/kuromoji/ (props changed)
lucene/dev/trunk/modules/benchmark/ (props changed)
lucene/dev/trunk/modules/facet/ (props changed)
lucene/dev/trunk/modules/queryparser/src/test/org/apache/lucene/queryparser/xml/builders/TestNumericRangeFilterBuilder.java (props changed)
lucene/dev/trunk/solr/ (props changed)
lucene/dev/trunk/solr/CHANGES.txt (props changed)
lucene/dev/trunk/solr/LICENSE.txt (props changed)
lucene/dev/trunk/solr/NOTICE.txt (props changed)
lucene/dev/trunk/solr/README.txt (props changed)
lucene/dev/trunk/solr/build.xml (props changed)
lucene/dev/trunk/solr/client/ (props changed)
lucene/dev/trunk/solr/common-build.xml (props changed)
lucene/dev/trunk/solr/contrib/ (props changed)
lucene/dev/trunk/solr/contrib/clustering/src/test-files/ (props changed)
lucene/dev/trunk/solr/contrib/dataimporthandler-extras/src/java/ (props changed)
lucene/dev/trunk/solr/contrib/dataimporthandler/src/java/ (props changed)
lucene/dev/trunk/solr/contrib/dataimporthandler/src/test-files/ (props changed)
lucene/dev/trunk/solr/contrib/dataimporthandler/src/test/org/ (props changed)
lucene/dev/trunk/solr/contrib/uima/src/java/ (props changed)
lucene/dev/trunk/solr/contrib/uima/src/test-files/ (props changed)
lucene/dev/trunk/solr/core/ (props changed)
lucene/dev/trunk/solr/core/src/java/ (props changed)
lucene/dev/trunk/solr/core/src/test/ (props changed)
lucene/dev/trunk/solr/dev-tools/ (props changed)
lucene/dev/trunk/solr/example/ (props changed)
lucene/dev/trunk/solr/lib/ (props changed)
lucene/dev/trunk/solr/scripts/ (props changed)
lucene/dev/trunk/solr/site/ (props changed)
lucene/dev/trunk/solr/site-src/ (props changed)
lucene/dev/trunk/solr/solrj/ (props changed)
lucene/dev/trunk/solr/solrj/src/java/ (props changed)
lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/ (props changed)
lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/client/solrj/ (props changed)
lucene/dev/trunk/solr/solrj/src/test/org/apache/solr/common/ (props changed)
lucene/dev/trunk/solr/test-framework/ (props changed)
lucene/dev/trunk/solr/testlogging.properties (props changed)
lucene/dev/trunk/solr/webapp/ (props changed)
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/Codec.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/Codec.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/Codec.java Sun Jan 29 11:33:46 2012
@@ -22,6 +22,7 @@ import java.util.Set;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.util.NamedSPILoader;
+import org.apache.lucene.store.CompoundFileDirectory;
import org.apache.lucene.store.Directory;
/**
@@ -43,7 +44,11 @@ public abstract class Codec implements N
return name;
}
+ /** Populates <code>files</code> with all filenames needed for
+ * the <code>info</code> segment.
+ */
public void files(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
+ assert (dir instanceof CompoundFileDirectory) == false;
postingsFormat().files(dir, info, "", files);
storedFieldsFormat().files(dir, info, files);
termVectorsFormat().files(dir, info, files);
@@ -54,6 +59,14 @@ public abstract class Codec implements N
normsFormat().files(dir, info, files);
}
+ /** Populates <code>files</code> with any filenames that are
+ * stored outside of CFS for the <code>info</code> segment.
+ */
+ public void separateFiles(Directory dir, SegmentInfo info, Set<String> files) throws IOException {
+ liveDocsFormat().separateFiles(dir, info, files);
+ normsFormat().separateFiles(dir, info, files);
+ }
+
/** Encodes/decodes postings */
public abstract PostingsFormat postingsFormat();
@@ -75,6 +88,9 @@ public abstract class Codec implements N
/** Encodes/decodes document normalization values */
public abstract NormsFormat normsFormat();
+ /** Encodes/decodes live docs */
+ public abstract LiveDocsFormat liveDocsFormat();
+
/** looks up a codec by name */
public static Codec forName(String name) {
return loader.lookup(name);
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/MappingMultiDocsEnum.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/MappingMultiDocsEnum.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/MappingMultiDocsEnum.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/MappingMultiDocsEnum.java Sun Jan 29 11:33:46 2012
@@ -78,6 +78,7 @@ public final class MappingMultiDocsEnum
current = subs[upto].docsEnum;
currentBase = mergeState.docBase[reader];
currentMap = mergeState.docMaps[reader];
+ assert currentMap == null || currentMap.length == subs[upto].slice.length: "readerIndex=" + reader + " subs.len=" + subs.length + " len1=" + currentMap.length + " vs " + subs[upto].slice.length;
}
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/appending/AppendingCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/appending/AppendingCodec.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/appending/AppendingCodec.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/appending/AppendingCodec.java Sun Jan 29 11:33:46 2012
@@ -20,6 +20,7 @@ package org.apache.lucene.codecs.appendi
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfosFormat;
@@ -28,6 +29,7 @@ import org.apache.lucene.codecs.TermVect
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40TermVectorsFormat;
@@ -50,6 +52,7 @@ public class AppendingCodec extends Code
private final TermVectorsFormat vectors = new Lucene40TermVectorsFormat();
private final DocValuesFormat docValues = new Lucene40DocValuesFormat();
private final NormsFormat norms = new Lucene40NormsFormat();
+ private final LiveDocsFormat liveDocs = new Lucene40LiveDocsFormat();
@Override
public PostingsFormat postingsFormat() {
@@ -85,4 +88,9 @@ public class AppendingCodec extends Code
public NormsFormat normsFormat() {
return norms;
}
+
+ @Override
+ public LiveDocsFormat liveDocsFormat() {
+ return liveDocs;
+ }
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xCodec.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xCodec.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene3x/Lucene3xCodec.java Sun Jan 29 11:33:46 2012
@@ -23,18 +23,23 @@ import java.util.Set;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PerDocConsumer;
import org.apache.lucene.codecs.PerDocProducer;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfosFormat;
import org.apache.lucene.codecs.StoredFieldsFormat;
+import org.apache.lucene.codecs.StoredFieldsWriter;
import org.apache.lucene.codecs.TermVectorsFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
import org.apache.lucene.index.PerDocWriteState;
import org.apache.lucene.index.SegmentInfo;
import org.apache.lucene.index.SegmentReadState;
import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.IOContext;
+import org.apache.lucene.util.MutableBits;
/**
* Supports the Lucene 3.x index format (readonly)
@@ -47,7 +52,12 @@ public class Lucene3xCodec extends Codec
private final PostingsFormat postingsFormat = new Lucene3xPostingsFormat();
// TODO: this should really be a different impl
- private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat();
+ private final StoredFieldsFormat fieldsFormat = new Lucene40StoredFieldsFormat() {
+ @Override
+ public StoredFieldsWriter fieldsWriter(Directory directory, String segment, IOContext context) throws IOException {
+ throw new UnsupportedOperationException("this codec can only be used for reading");
+ }
+ };
private final TermVectorsFormat vectorsFormat = new Lucene3xTermVectorsFormat();
@@ -57,6 +67,14 @@ public class Lucene3xCodec extends Codec
private final NormsFormat normsFormat = new Lucene3xNormsFormat();
+ // TODO: this should really be a different impl
+ private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat() {
+ @Override
+ public void writeLiveDocs(MutableBits bits, Directory dir, SegmentInfo info, IOContext context) throws IOException {
+ throw new UnsupportedOperationException("this codec can only be used for reading");
+ }
+ };
+
// 3.x doesn't support docvalues
private final DocValuesFormat docValuesFormat = new DocValuesFormat() {
@Override
@@ -107,4 +125,9 @@ public class Lucene3xCodec extends Codec
public NormsFormat normsFormat() {
return normsFormat;
}
+
+ @Override
+ public LiveDocsFormat liveDocsFormat() {
+ return liveDocsFormat;
+ }
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/Lucene40Codec.java Sun Jan 29 11:33:46 2012
@@ -20,6 +20,7 @@ package org.apache.lucene.codecs.lucene4
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfosFormat;
@@ -42,6 +43,8 @@ public class Lucene40Codec extends Codec
private final DocValuesFormat docValuesFormat = new Lucene40DocValuesFormat();
private final SegmentInfosFormat infosFormat = new Lucene40SegmentInfosFormat();
private final NormsFormat normsFormat = new Lucene40NormsFormat();
+ private final LiveDocsFormat liveDocsFormat = new Lucene40LiveDocsFormat();
+
private final PostingsFormat postingsFormat = new PerFieldPostingsFormat() {
@Override
public PostingsFormat getPostingsFormatForField(String field) {
@@ -87,6 +90,11 @@ public class Lucene40Codec extends Codec
public NormsFormat normsFormat() {
return normsFormat;
}
+
+ @Override
+ public LiveDocsFormat liveDocsFormat() {
+ return liveDocsFormat;
+ }
/** Returns the postings format that should be used for writing
* new segments of <code>field</code>.
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/lucene40/values/Floats.java Sun Jan 29 11:33:46 2012
@@ -109,7 +109,7 @@ public class Floats {
throws IOException {
super(dir, id, CODEC_NAME, VERSION_CURRENT, maxDoc, context, type);
arrayTemplate = DocValuesArray.TEMPLATES.get(type);
- assert size == 4 || size == 8;
+ assert size == 4 || size == 8: "wrong size=" + size + " type=" + type + " id=" + id;
}
@Override
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/codecs/simpletext/SimpleTextCodec.java Sun Jan 29 11:33:46 2012
@@ -20,6 +20,7 @@ package org.apache.lucene.codecs.simplet
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfosFormat;
@@ -43,6 +44,7 @@ public final class SimpleTextCodec exten
private final DocValuesFormat docValues = new Lucene40DocValuesFormat();
// TODO: need a plain-text impl (using the above)
private final NormsFormat normsFormat = new SimpleTextNormsFormat();
+ private final LiveDocsFormat liveDocs = new SimpleTextLiveDocsFormat();
public SimpleTextCodec() {
super("SimpleText");
@@ -82,4 +84,9 @@ public final class SimpleTextCodec exten
public NormsFormat normsFormat() {
return normsFormat;
}
+
+ @Override
+ public LiveDocsFormat liveDocsFormat() {
+ return liveDocs;
+ }
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/BufferedDeletesStream.java Sun Jan 29 11:33:46 2012
@@ -147,7 +147,7 @@ class BufferedDeletesStream {
};
/** Resolves the buffered deleted Term/Query/docIDs, into
- * actual deleted docIDs in the liveDocs BitVector for
+ * actual deleted docIDs in the liveDocs MutableBits for
* each SegmentReader. */
public synchronized ApplyDeletesResult applyDeletes(IndexWriter.ReaderPool readerPool, List<SegmentInfo> infos) throws IOException {
final long t0 = System.currentTimeMillis();
@@ -206,7 +206,7 @@ class BufferedDeletesStream {
delIDX--;
} else if (packet != null && segGen == packet.delGen()) {
- assert packet.isSegmentPrivate : "Packet and Segments deletegen can only match on a segment private del packet";
+ assert packet.isSegmentPrivate : "Packet and Segments deletegen can only match on a segment private del packet gen=" + segGen;
//System.out.println(" eq");
// Lock order: IW -> BD -> RP
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/CheckIndex.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/CheckIndex.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/CheckIndex.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/CheckIndex.java Sun Jan 29 11:33:46 2012
@@ -174,8 +174,8 @@ public class CheckIndex {
/** True if this segment has pending deletions. */
public boolean hasDeletions;
- /** Name of the current deletions file name. */
- public String deletionsFileName;
+ /** Current deletions generation. */
+ public long deletionsGen;
/** Number of deleted documents. */
public int numDeleted;
@@ -526,15 +526,14 @@ public class CheckIndex {
segInfoStat.docStoreCompoundFile = info.getDocStoreIsCompoundFile();
}
- final String delFileName = info.getDelFileName();
- if (delFileName == null){
+ if (info.hasDeletions()) {
msg(" no deletions");
segInfoStat.hasDeletions = false;
}
else{
- msg(" has deletions [delFileName=" + delFileName + "]");
+ msg(" has deletions [delGen=" + info.getDelGen() + "]");
segInfoStat.hasDeletions = true;
- segInfoStat.deletionsFileName = delFileName;
+ segInfoStat.deletionsGen = info.getDelGen();
}
if (infoStream != null)
infoStream.print(" test: open reader.........");
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DirectoryReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DirectoryReader.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DirectoryReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DirectoryReader.java Sun Jan 29 11:33:46 2012
@@ -166,7 +166,9 @@ final class DirectoryReader extends Base
} else {
readerShared[i] = false;
// Steal the ref returned by SegmentReader ctor:
- newReaders[i] = new SegmentReader(infos.info(i), newReaders[i], IOContext.READ);
+ assert infos.info(i).dir == newReaders[i].getSegmentInfo().dir;
+ assert infos.info(i).hasDeletions();
+ newReaders[i] = new SegmentReader(infos.info(i), newReaders[i].core, IOContext.READ);
}
}
success = true;
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/DocumentsWriterPerThread.java Sun Jan 29 11:33:46 2012
@@ -30,12 +30,12 @@ import org.apache.lucene.search.similari
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.FlushInfo;
import org.apache.lucene.store.IOContext;
-import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.Counter;
import org.apache.lucene.util.ByteBlockPool.Allocator;
import org.apache.lucene.util.ByteBlockPool.DirectTrackingAllocator;
import org.apache.lucene.util.InfoStream;
import org.apache.lucene.util.RamUsageEstimator;
+import org.apache.lucene.util.MutableBits;
public class DocumentsWriterPerThread {
@@ -114,13 +114,15 @@ public class DocumentsWriterPerThread {
static class FlushedSegment {
final SegmentInfo segmentInfo;
final BufferedDeletes segmentDeletes;
- final BitVector liveDocs;
+ final MutableBits liveDocs;
+ final int delCount;
private FlushedSegment(SegmentInfo segmentInfo,
- BufferedDeletes segmentDeletes, BitVector liveDocs) {
+ BufferedDeletes segmentDeletes, MutableBits liveDocs, int delCount) {
this.segmentInfo = segmentInfo;
this.segmentDeletes = segmentDeletes;
this.liveDocs = liveDocs;
+ this.delCount = delCount;
}
}
@@ -448,11 +450,11 @@ public class DocumentsWriterPerThread {
// happens when an exception is hit processing that
// doc, eg if analyzer has some problem w/ the text):
if (pendingDeletes.docIDs.size() > 0) {
- flushState.liveDocs = new BitVector(numDocsInRAM);
- flushState.liveDocs.invertAll();
+ flushState.liveDocs = codec.liveDocsFormat().newLiveDocs(numDocsInRAM);
for(int delDocID : pendingDeletes.docIDs) {
flushState.liveDocs.clear(delDocID);
}
+ flushState.delCountOnFlush = pendingDeletes.docIDs.size();
pendingDeletes.bytesUsed.addAndGet(-pendingDeletes.docIDs.size() * BufferedDeletes.BYTES_PER_DEL_DOCID);
pendingDeletes.docIDs.clear();
}
@@ -475,7 +477,7 @@ public class DocumentsWriterPerThread {
pendingDeletes.terms.clear();
final SegmentInfo newSegment = new SegmentInfo(segment, flushState.numDocs, directory, false, flushState.codec, fieldInfos.asReadOnly());
if (infoStream.isEnabled("DWPT")) {
- infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : (flushState.numDocs - flushState.liveDocs.count())) + " deleted docs");
+ infoStream.message("DWPT", "new segment has " + (flushState.liveDocs == null ? 0 : (flushState.numDocs - flushState.delCountOnFlush)) + " deleted docs");
infoStream.message("DWPT", "new segment has " + (newSegment.getHasVectors() ? "vectors" : "no vectors"));
infoStream.message("DWPT", "flushedFiles=" + newSegment.files());
infoStream.message("DWPT", "flushed codec=" + newSegment.getCodec());
@@ -504,7 +506,7 @@ public class DocumentsWriterPerThread {
doAfterFlush();
success = true;
- return new FlushedSegment(newSegment, segmentDeletes, flushState.liveDocs);
+ return new FlushedSegment(newSegment, segmentDeletes, flushState.liveDocs, flushState.delCountOnFlush);
} finally {
if (!success) {
if (segment != null) {
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/FreqProxTermsWriterPerField.java Sun Jan 29 11:33:46 2012
@@ -28,7 +28,6 @@ import org.apache.lucene.codecs.Postings
import org.apache.lucene.codecs.TermStats;
import org.apache.lucene.codecs.TermsConsumer;
import org.apache.lucene.index.FieldInfo.IndexOptions;
-import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.BytesRef;
import org.apache.lucene.util.FixedBitSet;
import org.apache.lucene.util.RamUsageEstimator;
@@ -461,11 +460,15 @@ final class FreqProxTermsWriterPerField
// Mark it deleted. TODO: we could also skip
// writing its postings; this would be
// deterministic (just for this Term's docs).
+
+ // TODO: can we do this reach-around in a cleaner way????
if (state.liveDocs == null) {
- state.liveDocs = new BitVector(state.numDocs);
- state.liveDocs.invertAll();
+ state.liveDocs = docState.docWriter.codec.liveDocsFormat().newLiveDocs(state.numDocs);
+ }
+ if (state.liveDocs.get(docID)) {
+ state.delCountOnFlush++;
+ state.liveDocs.clear(docID);
}
- state.liveDocs.clear(docID);
}
totTF += termDocFreq;
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileNames.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileNames.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileNames.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexFileNames.java Sun Jan 29 11:33:46 2012
@@ -57,9 +57,6 @@ public final class IndexFileNames {
/** Extension of compound file for doc store files*/
public static final String COMPOUND_FILE_STORE_EXTENSION = "cfx";
- /** Extension of deletes */
- public static final String DELETES_EXTENSION = "del";
-
/**
* This array contains all filename extensions used by
* Lucene's index files, with one exception, namely the
@@ -70,7 +67,6 @@ public final class IndexFileNames {
public static final String INDEX_EXTENSIONS[] = new String[] {
COMPOUND_FILE_EXTENSION,
COMPOUND_FILE_ENTRIES_EXTENSION,
- DELETES_EXTENSION,
GEN_EXTENSION,
COMPOUND_FILE_STORE_EXTENSION,
};
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/IndexWriter.java Sun Jan 29 11:33:46 2012
@@ -30,10 +30,10 @@ import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.atomic.AtomicInteger;
-import java.util.regex.Pattern;
import org.apache.lucene.analysis.Analyzer;
import org.apache.lucene.codecs.Codec;
+import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.index.DocumentsWriterPerThread.FlushedSegment;
import org.apache.lucene.index.FieldInfos.FieldNumberBiMap;
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
@@ -48,10 +48,11 @@ import org.apache.lucene.store.IOContext
import org.apache.lucene.store.Lock;
import org.apache.lucene.store.LockObtainFailedException;
import org.apache.lucene.store.MergeInfo;
-import org.apache.lucene.util.BitVector;
+import org.apache.lucene.util.Bits;
import org.apache.lucene.util.Constants;
import org.apache.lucene.util.IOUtils;
import org.apache.lucene.util.InfoStream;
+import org.apache.lucene.util.MutableBits;
import org.apache.lucene.util.ThreadInterruptedException;
import org.apache.lucene.util.TwoPhaseCommit;
@@ -416,7 +417,7 @@ public class IndexWriter implements Clos
// docs, and it's copy-on-write (cloned whenever we need
// to change it but it's been shared to an external NRT
// reader).
- public BitVector liveDocs;
+ public Bits liveDocs;
// How many further deletions we've done against
// liveDocs vs when we loaded it or last wrote it:
@@ -446,6 +447,24 @@ public class IndexWriter implements Clos
return rc > myRefCounts;
}
+ // Call only from assert!
+ public synchronized boolean verifyDocCounts() {
+ int count;
+ if (liveDocs != null) {
+ count = 0;
+ for(int docID=0;docID<info.docCount;docID++) {
+ if (liveDocs.get(docID)) {
+ count++;
+ }
+ }
+ } else {
+ count = info.docCount;
+ }
+
+ assert info.docCount - info.getDelCount() - pendingDeleteCount == count: "info.docCount=" + info.docCount + " info.getDelCount()=" + info.getDelCount() + " pendingDeleteCount=" + pendingDeleteCount + " count=" + count;;
+ return true;
+ }
+
// Returns true if any reader remains
public synchronized boolean removeReader(SegmentReader sr, boolean drop) throws IOException {
if (sr == reader) {
@@ -468,17 +487,6 @@ public class IndexWriter implements Clos
return reader != null || mergeReader != null;
}
- // Called only from assert
- private boolean countsMatch() {
- if (liveDocs == null) {
- assert pendingDeleteCount == 0;
- } else {
- assert liveDocs.count() == info.docCount - info.getDelCount() - pendingDeleteCount :
- "liveDocs.count()=" + liveDocs.count() + " info.docCount=" + info.docCount + " info.delCount=" + info.getDelCount() + " pendingDelCount=" + pendingDeleteCount;
- }
- return true;
- }
-
// Get reader for searching/deleting
public synchronized SegmentReader getReader(IOContext context) throws IOException {
//System.out.println(" livedocs=" + rld.liveDocs);
@@ -486,7 +494,7 @@ public class IndexWriter implements Clos
if (reader == null) {
reader = new SegmentReader(info, config.getReaderTermsIndexDivisor(), context);
if (liveDocs == null) {
- liveDocs = (BitVector) reader.getLiveDocs();
+ liveDocs = reader.getLiveDocs();
}
//System.out.println("ADD seg=" + rld.info + " isMerge=" + isMerge + " " + readerMap.size() + " in pool");
}
@@ -513,7 +521,7 @@ public class IndexWriter implements Clos
} else {
mergeReader = new SegmentReader(info, -1, context);
if (liveDocs == null) {
- liveDocs = (BitVector) mergeReader.getLiveDocs();
+ liveDocs = mergeReader.getLiveDocs();
}
}
}
@@ -526,8 +534,10 @@ public class IndexWriter implements Clos
public synchronized boolean delete(int docID) {
assert liveDocs != null;
assert docID >= 0 && docID < liveDocs.length();
- final boolean didDelete = liveDocs.getAndClear(docID);
+ assert !shared;
+ final boolean didDelete = liveDocs.get(docID);
if (didDelete) {
+ ((MutableBits) liveDocs).clear(docID);
pendingDeleteCount++;
//System.out.println(" new del seg=" + info + " docID=" + docID + " pendingDelCount=" + pendingDeleteCount + " totDelCount=" + (info.docCount-liveDocs.count()));
}
@@ -557,17 +567,16 @@ public class IndexWriter implements Clos
getReader(context).decRef();
assert reader != null;
}
- assert countsMatch();
shared = true;
if (liveDocs != null) {
- return new SegmentReader(reader, liveDocs, info.docCount - info.getDelCount() - pendingDeleteCount);
+ return new SegmentReader(reader.getSegmentInfo(), reader.core, liveDocs, info.docCount - info.getDelCount() - pendingDeleteCount);
} else {
reader.incRef();
return reader;
}
}
- public synchronized void initWritableLiveDocs() {
+ public synchronized void initWritableLiveDocs() throws IOException {
assert Thread.holdsLock(IndexWriter.this);
//System.out.println("initWritableLivedocs seg=" + info + " liveDocs=" + liveDocs + " shared=" + shared);
if (shared) {
@@ -575,12 +584,12 @@ public class IndexWriter implements Clos
// SegmentReader sharing the current liveDocs
// instance; must now make a private clone so we can
// change it:
+ LiveDocsFormat liveDocsFormat = info.getCodec().liveDocsFormat();
if (liveDocs == null) {
//System.out.println("create BV seg=" + info);
- liveDocs = new BitVector(info.docCount);
- liveDocs.setAll();
+ liveDocs = liveDocsFormat.newLiveDocs(info.docCount);
} else {
- liveDocs = (BitVector) liveDocs.clone();
+ liveDocs = liveDocsFormat.newLiveDocs(liveDocs);
}
shared = false;
} else {
@@ -588,11 +597,10 @@ public class IndexWriter implements Clos
}
}
- public synchronized BitVector getReadOnlyLiveDocs() {
+ public synchronized Bits getReadOnlyLiveDocs() {
//System.out.println("getROLiveDocs seg=" + info);
assert Thread.holdsLock(IndexWriter.this);
shared = true;
- assert countsMatch();
//if (liveDocs != null) {
//System.out.println(" liveCount=" + liveDocs.count());
//}
@@ -611,29 +619,20 @@ public class IndexWriter implements Clos
// Save in case we need to rollback on failure:
final SegmentInfo sav = (SegmentInfo) info.clone();
info.advanceDelGen();
+ info.setDelCount(info.getDelCount() + pendingDeleteCount);
// We can write directly to the actual name (vs to a
// .tmp & renaming it) because the file is not live
// until segments file is written:
- final String delFileName = info.getDelFileName();
boolean success = false;
try {
- liveDocs.write(dir, delFileName, IOContext.DEFAULT);
+ info.getCodec().liveDocsFormat().writeLiveDocs((MutableBits)liveDocs, dir, info, IOContext.DEFAULT);
success = true;
} finally {
if (!success) {
info.reset(sav);
- try {
- dir.deleteFile(delFileName);
- } catch (Throwable t) {
- // Suppress this so we keep throwing the
- // original exception
- }
}
}
- assert (info.docCount - liveDocs.count()) == info.getDelCount() + pendingDeleteCount:
- "delete count mismatch during commit: seg=" + info + " info.delCount=" + info.getDelCount() + " vs BitVector=" + (info.docCount-liveDocs.count() + " pendingDelCount=" + pendingDeleteCount);
- info.setDelCount(info.getDelCount() + pendingDeleteCount);
pendingDeleteCount = 0;
return true;
} else {
@@ -2205,7 +2204,7 @@ public class IndexWriter implements Clos
/**
* Prepares the {@link SegmentInfo} for the new flushed segment and persists
- * the deleted documents {@link BitVector}. Use
+ * the deleted documents {@link MutableBits}. Use
* {@link #publishFlushedSegment(SegmentInfo, FrozenBufferedDeletes)} to
* publish the returned {@link SegmentInfo} together with its segment private
* delete packet.
@@ -2252,33 +2251,23 @@ public class IndexWriter implements Clos
// Must write deleted docs after the CFS so we don't
// slurp the del file into CFS:
if (flushedSegment.liveDocs != null) {
- final int delCount = flushedSegment.segmentInfo.docCount - flushedSegment.liveDocs.count();
+ final int delCount = flushedSegment.delCount;
assert delCount > 0;
newSegment.setDelCount(delCount);
newSegment.advanceDelGen();
- final String delFileName = newSegment.getDelFileName();
if (infoStream.isEnabled("IW")) {
- infoStream.message("IW", "flush: write " + delCount + " deletes to " + delFileName);
- }
- boolean success2 = false;
- try {
- // TODO: in the NRT case it'd be better to hand
- // this del vector over to the
- // shortly-to-be-opened SegmentReader and let it
- // carry the changes; there's no reason to use
- // filesystem as intermediary here.
- flushedSegment.liveDocs.write(directory, delFileName, context);
- success2 = true;
- } finally {
- if (!success2) {
- try {
- directory.deleteFile(delFileName);
- } catch (Throwable t) {
- // suppress this so we keep throwing the
- // original exception
- }
- }
+ infoStream.message("IW", "flush: write " + delCount + " deletes gen=" + flushedSegment.segmentInfo.getDelGen());
}
+
+ // TODO: in the NRT case it'd be better to hand
+ // this del vector over to the
+ // shortly-to-be-opened SegmentReader and let it
+ // carry the changes; there's no reason to use
+ // filesystem as intermediary here.
+
+ SegmentInfo info = flushedSegment.segmentInfo;
+ Codec codec = info.getCodec();
+ codec.liveDocsFormat().writeLiveDocs(flushedSegment.liveDocs, directory, info, context);
}
success = true;
@@ -3032,8 +3021,8 @@ public class IndexWriter implements Clos
SegmentInfo info = sourceSegments.get(i);
minGen = Math.min(info.getBufferedDeletesGen(), minGen);
final int docCount = info.docCount;
- final BitVector prevLiveDocs = merge.readerLiveDocs.get(i);
- final BitVector currentLiveDocs;
+ final Bits prevLiveDocs = merge.readerLiveDocs.get(i);
+ final Bits currentLiveDocs;
ReadersAndLiveDocs rld = readerPool.get(info, false);
// We enrolled in mergeInit:
assert rld != null;
@@ -3052,7 +3041,13 @@ public class IndexWriter implements Clos
// newly flushed deletes but mapping them to the new
// docIDs.
- if (currentLiveDocs.count() < prevLiveDocs.count()) {
+ // Since we copy-on-write, if any new deletes were
+ // applied after merging has started, we can just
+ // check if the before/after liveDocs have changed.
+ // If so, we must carefully merge the liveDocs one
+ // doc at a time:
+ if (currentLiveDocs != prevLiveDocs) {
+
// This means this segment received new deletes
// since we started the merge, so we
// must merge them:
@@ -3071,8 +3066,7 @@ public class IndexWriter implements Clos
}
}
} else {
- assert currentLiveDocs.count() == prevLiveDocs.count(): "currentLiveDocs.count()==" + currentLiveDocs.count() + " vs prevLiveDocs.count()=" + prevLiveDocs.count() + " info=" + info;
- docUpto += currentLiveDocs.count();
+ docUpto += info.docCount - info.getDelCount() - rld.pendingDeleteCount;
}
} else if (currentLiveDocs != null) {
// This segment had no deletes before but now it
@@ -3576,13 +3570,12 @@ public class IndexWriter implements Clos
}
merge.readers = new ArrayList<SegmentReader>();
- merge.readerLiveDocs = new ArrayList<BitVector>();
+ merge.readerLiveDocs = new ArrayList<Bits>();
// This is try/finally to make sure merger's readers are
// closed:
boolean success = false;
try {
- int totDocCount = 0;
int segUpto = 0;
while(segUpto < sourceSegments.size()) {
@@ -3595,13 +3588,15 @@ public class IndexWriter implements Clos
assert reader != null;
// Carefully pull the most recent live docs:
- final BitVector liveDocs;
+ final Bits liveDocs;
synchronized(this) {
// Must sync to ensure BufferedDeletesStream
// cannot change liveDocs/pendingDeleteCount while
// we pull a copy:
liveDocs = rld.getReadOnlyLiveDocs();
+ assert rld.verifyDocCounts();
+
if (infoStream.isEnabled("IW")) {
if (rld.pendingDeleteCount != 0) {
infoStream.message("IW", "seg=" + info + " delCount=" + info.getDelCount() + " pendingDelCount=" + rld.pendingDeleteCount);
@@ -3612,23 +3607,16 @@ public class IndexWriter implements Clos
}
}
}
-
merge.readerLiveDocs.add(liveDocs);
merge.readers.add(reader);
-
- if (liveDocs == null || liveDocs.count() > 0) {
+ final int delCount = rld.pendingDeleteCount + info.getDelCount();
+ assert delCount <= info.docCount;
+ if (delCount < info.docCount) {
merger.add(reader, liveDocs);
- totDocCount += liveDocs == null ? reader.maxDoc() : liveDocs.count();
- } else {
- //System.out.println(" skip seg: fully deleted");
}
segUpto++;
}
- if (infoStream.isEnabled("IW")) {
- infoStream.message("IW", "merge: total " + totDocCount + " docs");
- }
-
merge.checkAborted(directory);
// This is where all the work happens:
@@ -3639,11 +3627,9 @@ public class IndexWriter implements Clos
merge.info.setCodec(codec);
if (infoStream.isEnabled("IW")) {
- infoStream.message("IW", "merge codec=" + codec);
+ infoStream.message("IW", "merge codec=" + codec + " docCount=" + mergedDocCount);
}
- assert mergedDocCount == totDocCount: "mergedDocCount=" + mergedDocCount + " vs " + totDocCount;
-
// Very important to do this before opening the reader
// because codec must know if prox was written for
// this segment:
@@ -4089,11 +4075,8 @@ public class IndexWriter implements Clos
Collection<String> files = info.files();
CompoundFileDirectory cfsDir = new CompoundFileDirectory(directory, fileName, context, true);
try {
+ assert assertNoSeparateFiles(files, directory, info);
for (String file : files) {
- assert !IndexFileNames.matchesExtension(file, IndexFileNames.DELETES_EXTENSION)
- : ".del file is not allowed in .cfs: " + file;
- assert !isSeparateNormsFile(file)
- : "separate norms file (.s[0-9]+) is not allowed in .cfs: " + file;
directory.copy(cfsDir, file, file, context);
checkAbort.work(directory.fileLength(file));
}
@@ -4106,15 +4089,17 @@ public class IndexWriter implements Clos
/**
- * Returns true if the given filename ends with the separate norms file
- * pattern: {@code SEPARATE_NORMS_EXTENSION + "[0-9]+"}.
- * @deprecated only for asserting
- */
- @Deprecated
- private static boolean isSeparateNormsFile(String filename) {
- int idx = filename.lastIndexOf('.');
- if (idx == -1) return false;
- String ext = filename.substring(idx + 1);
- return Pattern.matches("s[0-9]+", ext);
+ * used only by assert: checks that filenames about to be put in cfs belong.
+ */
+ private static boolean assertNoSeparateFiles(Collection<String> files,
+ Directory dir, SegmentInfo info) throws IOException {
+ // maybe this is overkill, but codec naming clashes would be bad.
+ Set<String> separateFiles = new HashSet<String>();
+ info.getCodec().separateFiles(dir, info, separateFiles);
+
+ for (String file : files) {
+ assert !separateFiles.contains(file) : file + " should not go in CFS!";
+ }
+ return true;
}
}
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MergePolicy.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MergePolicy.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MergePolicy.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MergePolicy.java Sun Jan 29 11:33:46 2012
@@ -24,7 +24,7 @@ import java.util.Map;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.MergeInfo;
-import org.apache.lucene.util.BitVector;
+import org.apache.lucene.util.Bits;
import org.apache.lucene.util.SetOnce.AlreadySetException;
import org.apache.lucene.util.SetOnce;
@@ -74,7 +74,7 @@ public abstract class MergePolicy implem
int maxNumSegments = -1; // used by IndexWriter
public long estimatedMergeBytes; // used by IndexWriter
List<SegmentReader> readers; // used by IndexWriter
- List<BitVector> readerLiveDocs; // used by IndexWriter
+ List<Bits> readerLiveDocs; // used by IndexWriter
public final List<SegmentInfo> segments;
public final int totalDocCount;
boolean aborted;
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MergeState.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MergeState.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MergeState.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/MergeState.java Sun Jan 29 11:33:46 2012
@@ -41,10 +41,10 @@ public class MergeState {
}
public FieldInfos fieldInfos;
- public List<IndexReaderAndLiveDocs> readers; // Readers & liveDocs being merged
- public int[][] docMaps; // Maps docIDs around deletions
- public int[] docBase; // New docID base per reader
- public int mergedDocCount; // Total # merged docs
+ public List<IndexReaderAndLiveDocs> readers; // Readers & liveDocs being merged
+ public int[][] docMaps; // Maps docIDs around deletions
+ public int[] docBase; // New docID base per reader
+ public int mergedDocCount; // Total # merged docs
public CheckAbort checkAbort;
public InfoStream infoStream;
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentInfo.java Sun Jan 29 11:33:46 2012
@@ -326,16 +326,6 @@ public final class SegmentInfo implement
return si;
}
- public String getDelFileName() {
- if (delGen == NO) {
- // In this case we know there is no deletion filename
- // against this segment
- return null;
- } else {
- return IndexFileNames.fileNameFromGeneration(name, IndexFileNames.DELETES_EXTENSION, delGen);
- }
- }
-
/**
* @deprecated separate norms are not supported in >= 4.0
*/
@@ -494,6 +484,9 @@ public final class SegmentInfo implement
} else {
codec.files(dir, this, fileSet);
}
+
+ // regardless of compound file setting: these files are always in the directory
+ codec.separateFiles(dir, this, fileSet);
if (docStoreOffset != -1) {
// We are sharing doc stores (stored fields, term
@@ -505,18 +498,6 @@ public final class SegmentInfo implement
}
}
- String delFileName = IndexFileNames.fileNameFromGeneration(name, IndexFileNames.DELETES_EXTENSION, delGen);
- if (delFileName != null && (delGen >= YES || dir.fileExists(delFileName))) {
- fileSet.add(delFileName);
- }
-
- // because separate norm files are unconditionally stored outside cfs,
- // we must explicitly ask for their filenames if we might have separate norms:
- // remove this when 3.x indexes are no longer supported
- if (normGen != null) {
- codec.normsFormat().separateFiles(dir, this, fileSet);
- }
-
files = new ArrayList<String>(fileSet);
return files;
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentMerger.java Sun Jan 29 11:33:46 2012
@@ -104,16 +104,12 @@ final class SegmentMerger {
// IndexWriter.close(false) takes to actually stop the
// threads.
- final int numReaders = mergeState.readers.size();
- // Remap docIDs
- mergeState.docMaps = new int[numReaders][];
- mergeState.docBase = new int[numReaders];
- mergeState.dirPayloadProcessor = new PayloadProcessorProvider.DirPayloadProcessor[numReaders];
- mergeState.currentPayloadProcessor = new PayloadProcessorProvider.PayloadProcessor[numReaders];
+ mergeState.mergedDocCount = setDocMaps();
mergeFieldInfos();
setMatchingSegmentReaders();
- mergeState.mergedDocCount = mergeFields();
+ int numMerged = mergeFields();
+ assert numMerged == mergeState.mergedDocCount;
final SegmentWriteState segmentWriteState = new SegmentWriteState(mergeState.infoStream, directory, segment, mergeState.fieldInfos, mergeState.mergedDocCount, termIndexInterval, codec, null, context);
mergeTerms(segmentWriteState);
@@ -124,7 +120,7 @@ final class SegmentMerger {
}
if (mergeState.fieldInfos.hasVectors()) {
- int numMerged = mergeVectors();
+ numMerged = mergeVectors();
assert numMerged == mergeState.mergedDocCount;
}
@@ -283,37 +279,31 @@ final class SegmentMerger {
}
}
- private final void mergeTerms(SegmentWriteState segmentWriteState) throws CorruptIndexException, IOException {
- int docBase = 0;
-
- final List<Fields> fields = new ArrayList<Fields>();
- final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
-
- for(MergeState.IndexReaderAndLiveDocs r : mergeState.readers) {
- final Fields f = r.reader.fields();
- final int maxDoc = r.reader.maxDoc();
- if (f != null) {
- slices.add(new ReaderUtil.Slice(docBase, maxDoc, fields.size()));
- fields.add(f);
- }
- docBase += maxDoc;
- }
-
+ // NOTE: removes any "all deleted" readers from mergeState.readers
+ private int setDocMaps() throws IOException {
final int numReaders = mergeState.readers.size();
- docBase = 0;
+ // Remap docIDs
+ mergeState.docMaps = new int[numReaders][];
+ mergeState.docBase = new int[numReaders];
+ mergeState.dirPayloadProcessor = new PayloadProcessorProvider.DirPayloadProcessor[numReaders];
+ mergeState.currentPayloadProcessor = new PayloadProcessorProvider.PayloadProcessor[numReaders];
+
+ int docBase = 0;
- for(int i=0;i<numReaders;i++) {
+ int i = 0;
+ while(i < mergeState.readers.size()) {
final MergeState.IndexReaderAndLiveDocs reader = mergeState.readers.get(i);
mergeState.docBase[i] = docBase;
final int maxDoc = reader.reader.maxDoc();
- if (reader.liveDocs != null) {
+ final int docCount;
+ final Bits liveDocs = reader.liveDocs;
+ final int[] docMap;
+ if (liveDocs != null) {
int delCount = 0;
- final Bits liveDocs = reader.liveDocs;
- assert liveDocs != null;
- final int[] docMap = mergeState.docMaps[i] = new int[maxDoc];
+ docMap = new int[maxDoc];
int newDocID = 0;
for(int j=0;j<maxDoc;j++) {
if (!liveDocs.get(j)) {
@@ -323,14 +313,41 @@ final class SegmentMerger {
docMap[j] = newDocID++;
}
}
- docBase += maxDoc - delCount;
+ docCount = maxDoc - delCount;
} else {
- docBase += maxDoc;
+ docCount = maxDoc;
+ docMap = null;
}
+ mergeState.docMaps[i] = docMap;
+ docBase += docCount;
+
if (mergeState.payloadProcessorProvider != null) {
mergeState.dirPayloadProcessor[i] = mergeState.payloadProcessorProvider.getDirProcessor(reader.reader.directory());
}
+
+ i++;
+ }
+
+ return docBase;
+ }
+
+ private final void mergeTerms(SegmentWriteState segmentWriteState) throws CorruptIndexException, IOException {
+
+ final List<Fields> fields = new ArrayList<Fields>();
+ final List<ReaderUtil.Slice> slices = new ArrayList<ReaderUtil.Slice>();
+
+ int docBase = 0;
+
+ for(int readerIndex=0;readerIndex<mergeState.readers.size();readerIndex++) {
+ final MergeState.IndexReaderAndLiveDocs r = mergeState.readers.get(readerIndex);
+ final Fields f = r.reader.fields();
+ final int maxDoc = r.reader.maxDoc();
+ if (f != null) {
+ slices.add(new ReaderUtil.Slice(docBase, maxDoc, readerIndex));
+ fields.add(f);
+ }
+ docBase += maxDoc;
}
final FieldsConsumer consumer = codec.postingsFormat().fieldsConsumer(segmentWriteState);
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentReader.java Sun Jan 29 11:33:46 2012
@@ -25,7 +25,6 @@ import org.apache.lucene.codecs.StoredFi
import org.apache.lucene.codecs.TermVectorsReader;
import org.apache.lucene.search.FieldCache; // javadocs
import org.apache.lucene.store.IOContext;
-import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.Bits;
/**
@@ -36,14 +35,14 @@ public final class SegmentReader extends
private final SegmentInfo si;
private final ReaderContext readerContext = new AtomicReaderContext(this);
- private final BitVector liveDocs;
+ private final Bits liveDocs;
// Normally set to si.docCount - si.delDocCount, unless we
// were created as an NRT reader from IW, in which case IW
// tells us the docCount:
private final int numDocs;
- private final SegmentCoreReaders core;
+ final SegmentCoreReaders core;
/**
* @throws CorruptIndexException if the index is corrupt
@@ -56,13 +55,12 @@ public final class SegmentReader extends
try {
if (si.hasDeletions()) {
// NOTE: the bitvector is stored using the regular directory, not cfs
- liveDocs = new BitVector(directory(), si.getDelFileName(), new IOContext(IOContext.READ, true));
+ liveDocs = si.getCodec().liveDocsFormat().readLiveDocs(directory(), si, new IOContext(IOContext.READ, true));
} else {
assert si.getDelCount() == 0;
liveDocs = null;
}
numDocs = si.docCount - si.getDelCount();
- assert checkLiveCounts(false);
success = true;
} finally {
// With lock-less commits, it's entirely possible (and
@@ -76,46 +74,26 @@ public final class SegmentReader extends
}
}
- // TODO: really these next 2 ctors could take
- // SegmentCoreReaders... that's all we do w/ the parent
- // SR:
-
// Create new SegmentReader sharing core from a previous
// SegmentReader and loading new live docs from a new
// deletes file. Used by openIfChanged.
- SegmentReader(SegmentInfo si, SegmentReader parent, IOContext context) throws IOException {
- assert si.dir == parent.getSegmentInfo().dir;
- this.si = si;
-
- // It's no longer possible to unDeleteAll, so, we can
- // only be created if we have deletions:
- assert si.hasDeletions();
-
- // ... but load our own deleted docs:
- liveDocs = new BitVector(si.dir, si.getDelFileName(), context);
- numDocs = si.docCount - si.getDelCount();
- assert checkLiveCounts(false);
-
- // We share core w/ parent:
- parent.core.incRef();
- core = parent.core;
+ SegmentReader(SegmentInfo si, SegmentCoreReaders core, IOContext context) throws IOException {
+ this(si, core, si.getCodec().liveDocsFormat().readLiveDocs(si.dir, si, context), si.docCount - si.getDelCount());
}
// Create new SegmentReader sharing core from a previous
// SegmentReader and using the provided in-memory
// liveDocs. Used by IndexWriter to provide a new NRT
// reader:
- SegmentReader(SegmentReader parent, BitVector liveDocs, int numDocs) throws IOException {
- this.si = parent.si;
- parent.core.incRef();
- this.core = parent.core;
+ SegmentReader(SegmentInfo si, SegmentCoreReaders core, Bits liveDocs, int numDocs) throws IOException {
+ this.si = si;
+ this.core = core;
+ core.incRef();
assert liveDocs != null;
this.liveDocs = liveDocs;
this.numDocs = numDocs;
-
- assert checkLiveCounts(true);
}
@Override
@@ -124,27 +102,6 @@ public final class SegmentReader extends
return liveDocs;
}
- private boolean checkLiveCounts(boolean isNRT) throws IOException {
- if (liveDocs != null) {
- if (liveDocs.size() != si.docCount) {
- throw new CorruptIndexException("document count mismatch: deleted docs count " + liveDocs.size() + " vs segment doc count " + si.docCount + " segment=" + si.name);
- }
-
- final int recomputedCount = liveDocs.getRecomputedCount();
- // Verify BitVector is self consistent:
- assert liveDocs.count() == recomputedCount : "live count=" + liveDocs.count() + " vs recomputed count=" + recomputedCount;
-
- // Verify our docCount matches:
- assert numDocs == recomputedCount :
- "delete count mismatch: numDocs=" + numDocs + " vs BitVector=" + (si.docCount-recomputedCount);
-
- assert isNRT || si.docCount - si.getDelCount() == recomputedCount :
- "si.docCount=" + si.docCount + "si.getDelCount()=" + si.getDelCount() + " recomputedCount=" + recomputedCount;
- }
-
- return true;
- }
-
@Override
protected void doClose() throws IOException {
//System.out.println("SR.close seg=" + si);
Modified: lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java (original)
+++ lucene/dev/trunk/lucene/src/java/org/apache/lucene/index/SegmentWriteState.java Sun Jan 29 11:33:46 2012
@@ -20,8 +20,8 @@ package org.apache.lucene.index;
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.store.Directory;
import org.apache.lucene.store.IOContext;
-import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.InfoStream;
+import org.apache.lucene.util.MutableBits;
/**
* @lucene.experimental
@@ -32,6 +32,7 @@ public class SegmentWriteState {
public final String segmentName;
public final FieldInfos fieldInfos;
public final int numDocs;
+ public int delCountOnFlush;
// Deletes to apply while we are flushing the segment. A
// Term is enrolled in here if it was deleted at one
@@ -41,7 +42,7 @@ public class SegmentWriteState {
public final BufferedDeletes segDeletes;
// Lazily created:
- public BitVector liveDocs;
+ public MutableBits liveDocs;
public final Codec codec;
public final String segmentSuffix;
@@ -83,5 +84,6 @@ public class SegmentWriteState {
codec = state.codec;
this.segmentSuffix = segmentSuffix;
segDeletes = state.segDeletes;
+ delCountOnFlush = state.delCountOnFlush;
}
}
Modified: lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWCodec.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWCodec.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWCodec.java (original)
+++ lucene/dev/trunk/lucene/src/test-framework/java/org/apache/lucene/codecs/preflexrw/PreFlexRWCodec.java Sun Jan 29 11:33:46 2012
@@ -18,11 +18,15 @@ package org.apache.lucene.codecs.preflex
*/
import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfosFormat;
+import org.apache.lucene.codecs.StoredFieldsFormat;
import org.apache.lucene.codecs.TermVectorsFormat;
import org.apache.lucene.codecs.lucene3x.Lucene3xCodec;
+import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
import org.apache.lucene.util.LuceneTestCase;
/**
@@ -35,6 +39,10 @@ public class PreFlexRWCodec extends Luce
private final FieldInfosFormat fieldInfos = new PreFlexRWFieldInfosFormat();
private final TermVectorsFormat termVectors = new PreFlexRWTermVectorsFormat();
private final SegmentInfosFormat segmentInfos = new PreFlexRWSegmentInfosFormat();
+ // TODO: this should really be a different impl
+ private final LiveDocsFormat liveDocs = new Lucene40LiveDocsFormat();
+ // TODO: this should really be a different impl
+ private final StoredFieldsFormat storedFields = new Lucene40StoredFieldsFormat();
@Override
public PostingsFormat postingsFormat() {
@@ -80,4 +88,22 @@ public class PreFlexRWCodec extends Luce
return super.termVectorsFormat();
}
}
+
+ @Override
+ public LiveDocsFormat liveDocsFormat() {
+ if (LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE) {
+ return liveDocs;
+ } else {
+ return super.liveDocsFormat();
+ }
+ }
+
+ @Override
+ public StoredFieldsFormat storedFieldsFormat() {
+ if (LuceneTestCase.PREFLEX_IMPERSONATION_IS_ACTIVE) {
+ return storedFields;
+ } else {
+ return super.storedFieldsFormat();
+ }
+ }
}
Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestAddIndexes.java Sun Jan 29 11:33:46 2012
@@ -27,6 +27,7 @@ import org.apache.lucene.analysis.MockAn
import org.apache.lucene.codecs.Codec;
import org.apache.lucene.codecs.DocValuesFormat;
import org.apache.lucene.codecs.FieldInfosFormat;
+import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.codecs.NormsFormat;
import org.apache.lucene.codecs.PostingsFormat;
import org.apache.lucene.codecs.SegmentInfosFormat;
@@ -35,6 +36,7 @@ import org.apache.lucene.codecs.TermVect
import org.apache.lucene.codecs.lucene40.Lucene40Codec;
import org.apache.lucene.codecs.lucene40.Lucene40DocValuesFormat;
import org.apache.lucene.codecs.lucene40.Lucene40FieldInfosFormat;
+import org.apache.lucene.codecs.lucene40.Lucene40LiveDocsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40NormsFormat;
import org.apache.lucene.codecs.lucene40.Lucene40SegmentInfosFormat;
import org.apache.lucene.codecs.lucene40.Lucene40StoredFieldsFormat;
@@ -1156,6 +1158,11 @@ public class TestAddIndexes extends Luce
public NormsFormat normsFormat() {
return new Lucene40NormsFormat();
}
+
+ @Override
+ public LiveDocsFormat liveDocsFormat() {
+ return new Lucene40LiveDocsFormat();
+ }
}
/*
Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestBackwardsCompatibility.java Sun Jan 29 11:33:46 2012
@@ -497,17 +497,25 @@ public class TestBackwardsCompatibility
writer.deleteDocuments(searchTerm);
writer.close();
- // Now verify file names:
+ // Now verify file names... TODO: fix this test better, we could populate from
+ // separateFiles() or something.
String[] expected = new String[] {"_0.cfs", "_0.cfe",
"_0_1.del",
"segments_2",
"segments.gen"};
+
+ String[] expectedSimpleText = new String[] {"_0.cfs", "_0.cfe",
+ "_0_1.liv",
+ "segments_2",
+ "segments.gen"};
String[] actual = dir.listAll();
Arrays.sort(expected);
+ Arrays.sort(expectedSimpleText);
Arrays.sort(actual);
- if (!Arrays.equals(expected, actual)) {
- fail("incorrect filenames in index: expected:\n " + asString(expected) + "\n actual:\n " + asString(actual));
+ if (!Arrays.equals(expected, actual) && !Arrays.equals(expectedSimpleText, actual)) {
+ fail("incorrect filenames in index: expected:\n " + asString(expected)
+ + "\n or " + asString(expectedSimpleText) + "\n actual:\n " + asString(actual));
}
dir.close();
} finally {
Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexFileDeleter.java Sun Jan 29 11:33:46 2012
@@ -87,17 +87,20 @@ public class TestIndexFileDeleter extend
}
*/
+ // TODO: fix this test better
+ String ext = Codec.getDefault().getName().equals("SimpleText") ? ".liv" : ".del";
+
// Create a bogus separate del file for a
// segment that already has a separate del file:
- copyFile(dir, "_0_1.del", "_0_2.del");
+ copyFile(dir, "_0_1" + ext, "_0_2" + ext);
// Create a bogus separate del file for a
// segment that does not yet have a separate del file:
- copyFile(dir, "_0_1.del", "_1_1.del");
+ copyFile(dir, "_0_1" + ext, "_1_1" + ext);
// Create a bogus separate del file for a
// non-existent segment:
- copyFile(dir, "_0_1.del", "_188_1.del");
+ copyFile(dir, "_0_1" + ext, "_188_1" + ext);
// Create a bogus segment file:
copyFile(dir, "_0.cfs", "_188.cfs");
Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexReaderReopen.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexReaderReopen.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexReaderReopen.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexReaderReopen.java Sun Jan 29 11:33:46 2012
@@ -42,7 +42,6 @@ import org.apache.lucene.search.TermQuer
import org.apache.lucene.search.similarities.DefaultSimilarity;
import org.apache.lucene.store.AlreadyClosedException;
import org.apache.lucene.store.Directory;
-import org.apache.lucene.util.BitVector;
import org.apache.lucene.util.Bits;
import org.apache.lucene.util.LuceneTestCase;
import org.apache.lucene.util._TestUtil;
Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterDelete.java Sun Jan 29 11:33:46 2012
@@ -961,7 +961,8 @@ public class TestIndexWriterDelete exten
}
w.updateDocument(delTerm, doc);
// Eventually segment 0 should get a del docs:
- if (dir.fileExists("_0_1.del")) {
+ // TODO: fix this test
+ if (dir.fileExists("_0_1.del") || dir.fileExists("_0_1.liv") ) {
if (VERBOSE) {
System.out.println("TEST: deletes created @ count=" + count);
}
@@ -1006,7 +1007,8 @@ public class TestIndexWriterDelete exten
}
w.updateDocument(delTerm, doc);
// Eventually segment 0 should get a del docs:
- if (dir.fileExists("_0_1.del")) {
+ // TODO: fix this test
+ if (dir.fileExists("_0_1.del") || dir.fileExists("_0_1.liv")) {
break;
}
count++;
@@ -1052,7 +1054,8 @@ public class TestIndexWriterDelete exten
doc.add(newField("body", sb.toString(), TextField.TYPE_UNSTORED));
w.updateDocument(new Term("id", ""+id), doc);
docsInSegment.incrementAndGet();
- if (dir.fileExists("_0_1.del")) {
+ // TODO: fix this test
+ if (dir.fileExists("_0_1.del") || dir.fileExists("_0_1.liv")) {
if (VERBOSE) {
System.out.println("TEST: deletes created @ id=" + id);
}
Modified: lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java
URL: http://svn.apache.org/viewvc/lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java?rev=1237245&r1=1237244&r2=1237245&view=diff
==============================================================================
--- lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java (original)
+++ lucene/dev/trunk/lucene/src/test/org/apache/lucene/index/TestIndexWriterOnDiskFull.java Sun Jan 29 11:33:46 2012
@@ -20,6 +20,7 @@ package org.apache.lucene.index;
import java.io.IOException;
import org.apache.lucene.analysis.MockAnalyzer;
+import org.apache.lucene.codecs.LiveDocsFormat;
import org.apache.lucene.document.Document;
import org.apache.lucene.document.FieldType;
import org.apache.lucene.document.TextField;
@@ -459,13 +460,13 @@ public class TestIndexWriterOnDiskFull e
}
StackTraceElement[] trace = new Exception().getStackTrace();
for (int i = 0; i < trace.length; i++) {
- if ("org.apache.lucene.index.SegmentMerger".equals(trace[i].getClassName()) && "mergeTerms".equals(trace[i].getMethodName()) && !didFail1) {
+ if (SegmentMerger.class.getName().equals(trace[i].getClassName()) && "mergeTerms".equals(trace[i].getMethodName()) && !didFail1) {
didFail1 = true;
throw new IOException("fake disk full during mergeTerms");
}
- if ("org.apache.lucene.util.BitVector".equals(trace[i].getClassName()) && "write".equals(trace[i].getMethodName()) && !didFail2) {
+ if (LiveDocsFormat.class.getName().equals(trace[i].getClassName()) && "writeLiveDocs".equals(trace[i].getMethodName()) && !didFail2) {
didFail2 = true;
- throw new IOException("fake disk full while writing BitVector");
+ throw new IOException("fake disk full while writing LiveDocs");
}
}
}