mirror of https://github.com/apache/lucene.git
LUCENE-3631: make SegmentReader read-only (IndexWriter now privately tracks the pending deletes against a segment)
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1221404 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
d1ea4f76f1
commit
4d51b9012b
|
@ -26,7 +26,7 @@ import org.apache.lucene.util.Bits;
|
|||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.ReaderUtil;
|
||||
|
||||
abstract class BaseMultiReader<R extends IndexReader> extends IndexReader implements Cloneable {
|
||||
abstract class BaseMultiReader<R extends IndexReader> extends IndexReader {
|
||||
protected final R[] subReaders;
|
||||
protected final int[] starts; // 1st docno for each segment
|
||||
private final ReaderContext topLevelContext;
|
||||
|
@ -63,9 +63,6 @@ abstract class BaseMultiReader<R extends IndexReader> extends IndexReader implem
|
|||
@Override
|
||||
protected abstract IndexReader doOpenIfChanged() throws CorruptIndexException, IOException;
|
||||
|
||||
@Override
|
||||
public abstract Object clone();
|
||||
|
||||
@Override
|
||||
public Bits getLiveDocs() {
|
||||
throw new UnsupportedOperationException("please use MultiFields.getLiveDocs, or wrap your IndexReader with SlowMultiReaderWrapper, if you really need a top level Bits liveDocs");
|
||||
|
|
|
@ -211,22 +211,25 @@ class BufferedDeletesStream {
|
|||
|
||||
// Lock order: IW -> BD -> RP
|
||||
assert readerPool.infoIsLive(info);
|
||||
final SegmentReader reader = readerPool.get(info, false, IOContext.READ);
|
||||
final IndexWriter.ReadersAndLiveDocs rld = readerPool.get(info, true);
|
||||
final SegmentReader reader = rld.getReader(IOContext.READ);
|
||||
int delCount = 0;
|
||||
final boolean segAllDeletes;
|
||||
try {
|
||||
if (coalescedDeletes != null) {
|
||||
//System.out.println(" del coalesced");
|
||||
delCount += applyTermDeletes(coalescedDeletes.termsIterable(), reader);
|
||||
delCount += applyQueryDeletes(coalescedDeletes.queriesIterable(), reader);
|
||||
delCount += applyTermDeletes(coalescedDeletes.termsIterable(), rld, reader);
|
||||
delCount += applyQueryDeletes(coalescedDeletes.queriesIterable(), rld, reader);
|
||||
}
|
||||
//System.out.println(" del exact");
|
||||
// Don't delete by Term here; DocumentsWriterPerThread
|
||||
// already did that on flush:
|
||||
delCount += applyQueryDeletes(packet.queriesIterable(), reader);
|
||||
segAllDeletes = reader.numDocs() == 0;
|
||||
delCount += applyQueryDeletes(packet.queriesIterable(), rld, reader);
|
||||
final int fullDelCount = rld.info.getDelCount() + rld.pendingDeleteCount;
|
||||
assert fullDelCount <= rld.info.docCount;
|
||||
segAllDeletes = fullDelCount == rld.info.docCount;
|
||||
} finally {
|
||||
readerPool.release(reader, IOContext.Context.READ);
|
||||
readerPool.release(reader, false);
|
||||
}
|
||||
anyNewDeletes |= delCount > 0;
|
||||
|
||||
|
@ -238,7 +241,7 @@ class BufferedDeletesStream {
|
|||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "seg=" + info + " segGen=" + segGen + " segDeletes=[" + packet + "]; coalesced deletes=[" + (coalescedDeletes == null ? "null" : coalescedDeletes) + "] delCount=" + delCount + (segAllDeletes ? " 100% deleted" : ""));
|
||||
infoStream.message("BD", "seg=" + info + " segGen=" + segGen + " segDeletes=[" + packet + "]; coalesced deletes=[" + (coalescedDeletes == null ? "null" : coalescedDeletes) + "] newDelCount=" + delCount + (segAllDeletes ? " 100% deleted" : ""));
|
||||
}
|
||||
|
||||
if (coalescedDeletes == null) {
|
||||
|
@ -260,15 +263,18 @@ class BufferedDeletesStream {
|
|||
if (coalescedDeletes != null) {
|
||||
// Lock order: IW -> BD -> RP
|
||||
assert readerPool.infoIsLive(info);
|
||||
SegmentReader reader = readerPool.get(info, false, IOContext.READ);
|
||||
final IndexWriter.ReadersAndLiveDocs rld = readerPool.get(info, true);
|
||||
final SegmentReader reader = rld.getReader(IOContext.READ);
|
||||
int delCount = 0;
|
||||
final boolean segAllDeletes;
|
||||
try {
|
||||
delCount += applyTermDeletes(coalescedDeletes.termsIterable(), reader);
|
||||
delCount += applyQueryDeletes(coalescedDeletes.queriesIterable(), reader);
|
||||
segAllDeletes = reader.numDocs() == 0;
|
||||
delCount += applyTermDeletes(coalescedDeletes.termsIterable(), rld, reader);
|
||||
delCount += applyQueryDeletes(coalescedDeletes.queriesIterable(), rld, reader);
|
||||
final int fullDelCount = rld.info.getDelCount() + rld.pendingDeleteCount;
|
||||
assert fullDelCount <= rld.info.docCount;
|
||||
segAllDeletes = fullDelCount == rld.info.docCount;
|
||||
} finally {
|
||||
readerPool.release(reader, IOContext.Context.READ);
|
||||
readerPool.release(reader, false);
|
||||
}
|
||||
anyNewDeletes |= delCount > 0;
|
||||
|
||||
|
@ -280,7 +286,7 @@ class BufferedDeletesStream {
|
|||
}
|
||||
|
||||
if (infoStream.isEnabled("BD")) {
|
||||
infoStream.message("BD", "seg=" + info + " segGen=" + segGen + " coalesced deletes=[" + (coalescedDeletes == null ? "null" : coalescedDeletes) + "] delCount=" + delCount + (segAllDeletes ? " 100% deleted" : ""));
|
||||
infoStream.message("BD", "seg=" + info + " segGen=" + segGen + " coalesced deletes=[" + (coalescedDeletes == null ? "null" : coalescedDeletes) + "] newDelCount=" + delCount + (segAllDeletes ? " 100% deleted" : ""));
|
||||
}
|
||||
}
|
||||
info.setBufferedDeletesGen(nextGen);
|
||||
|
@ -348,7 +354,7 @@ class BufferedDeletesStream {
|
|||
}
|
||||
|
||||
// Delete by Term
|
||||
private synchronized long applyTermDeletes(Iterable<Term> termsIter, SegmentReader reader) throws IOException {
|
||||
private synchronized long applyTermDeletes(Iterable<Term> termsIter, IndexWriter.ReadersAndLiveDocs rld, SegmentReader reader) throws IOException {
|
||||
long delCount = 0;
|
||||
Fields fields = reader.fields();
|
||||
if (fields == null) {
|
||||
|
@ -362,7 +368,9 @@ class BufferedDeletesStream {
|
|||
DocsEnum docs = null;
|
||||
|
||||
assert checkDeleteTerm(null);
|
||||
|
||||
|
||||
boolean any = false;
|
||||
|
||||
//System.out.println(Thread.currentThread().getName() + " del terms reader=" + reader);
|
||||
for (Term term : termsIter) {
|
||||
// Since we visit terms sorted, we gain performance
|
||||
|
@ -387,7 +395,7 @@ class BufferedDeletesStream {
|
|||
// System.out.println(" term=" + term);
|
||||
|
||||
if (termsEnum.seekExact(term.bytes(), false)) {
|
||||
DocsEnum docsEnum = termsEnum.docs(reader.getLiveDocs(), docs, false);
|
||||
DocsEnum docsEnum = termsEnum.docs(rld.liveDocs, docs, false);
|
||||
//System.out.println("BDS: got docsEnum=" + docsEnum);
|
||||
|
||||
if (docsEnum != null) {
|
||||
|
@ -396,14 +404,19 @@ class BufferedDeletesStream {
|
|||
//System.out.println(Thread.currentThread().getName() + " del term=" + term + " doc=" + docID);
|
||||
if (docID == DocsEnum.NO_MORE_DOCS) {
|
||||
break;
|
||||
}
|
||||
// NOTE: there is no limit check on the docID
|
||||
// when deleting by Term (unlike by Query)
|
||||
// because on flush we apply all Term deletes to
|
||||
// each segment. So all Term deleting here is
|
||||
// against prior segments:
|
||||
if (!any) {
|
||||
rld.initWritableLiveDocs();
|
||||
any = true;
|
||||
}
|
||||
if (rld.delete(docID)) {
|
||||
delCount++;
|
||||
}
|
||||
reader.deleteDocument(docID);
|
||||
// TODO: we could/should change
|
||||
// reader.deleteDocument to return boolean
|
||||
// true if it did in fact delete, because here
|
||||
// we could be deleting an already-deleted doc
|
||||
// which makes this an upper bound:
|
||||
delCount++;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -422,9 +435,10 @@ class BufferedDeletesStream {
|
|||
}
|
||||
|
||||
// Delete by query
|
||||
private static long applyQueryDeletes(Iterable<QueryAndLimit> queriesIter, SegmentReader reader) throws IOException {
|
||||
private static long applyQueryDeletes(Iterable<QueryAndLimit> queriesIter, IndexWriter.ReadersAndLiveDocs rld, SegmentReader reader) throws IOException {
|
||||
long delCount = 0;
|
||||
final AtomicReaderContext readerContext = (AtomicReaderContext) reader.getTopReaderContext();
|
||||
boolean any = false;
|
||||
for (QueryAndLimit ent : queriesIter) {
|
||||
Query query = ent.query;
|
||||
int limit = ent.limit;
|
||||
|
@ -434,13 +448,18 @@ class BufferedDeletesStream {
|
|||
if (it != null) {
|
||||
while(true) {
|
||||
int doc = it.nextDoc();
|
||||
if (doc >= limit)
|
||||
if (doc >= limit) {
|
||||
break;
|
||||
}
|
||||
|
||||
reader.deleteDocument(doc);
|
||||
// as we use getLiveDocs() to filter out already deleted documents,
|
||||
// we only delete live documents, so the counting is right:
|
||||
delCount++;
|
||||
if (!any) {
|
||||
rld.initWritableLiveDocs();
|
||||
any = true;
|
||||
}
|
||||
|
||||
if (rld.delete(doc)) {
|
||||
delCount++;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -17,19 +17,6 @@ package org.apache.lucene.index;
|
|||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.codecs.BlockTreeTermsReader;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.document.FieldType; // for javadocs
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.index.DocValues.SortedSource;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.PrintStream;
|
||||
|
@ -41,16 +28,19 @@ import java.util.HashMap;
|
|||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
import org.apache.lucene.codecs.BlockTreeTermsReader;
|
||||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.FieldType; // for javadocs
|
||||
import org.apache.lucene.index.DocValues.SortedSource;
|
||||
import org.apache.lucene.index.DocValues.Source;
|
||||
import org.apache.lucene.search.DocIdSetIterator;
|
||||
import org.apache.lucene.search.IndexSearcher;
|
||||
import org.apache.lucene.search.TermQuery;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.store.FSDirectory;
|
||||
import org.apache.lucene.store.IOContext;
|
||||
import org.apache.lucene.store.IndexInput;
|
||||
import org.apache.lucene.document.Document;
|
||||
|
||||
import org.apache.lucene.store.FSDirectory;
|
||||
import org.apache.lucene.util.Bits;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.CommandLineUtil;
|
||||
|
@ -534,18 +524,18 @@ public class CheckIndex {
|
|||
}
|
||||
if (infoStream != null)
|
||||
infoStream.print(" test: open reader.........");
|
||||
reader = SegmentReader.get(info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, IOContext.DEFAULT);
|
||||
reader = new SegmentReader(info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, IOContext.DEFAULT);
|
||||
|
||||
segInfoStat.openReaderPassed = true;
|
||||
|
||||
final int numDocs = reader.numDocs();
|
||||
toLoseDocCount = numDocs;
|
||||
if (reader.hasDeletions()) {
|
||||
if (reader.liveDocs.count() != info.docCount - info.getDelCount()) {
|
||||
throw new RuntimeException("delete count mismatch: info=" + (info.docCount - info.getDelCount()) + " vs reader=" + reader.liveDocs.count());
|
||||
if (reader.numDocs() != info.docCount - info.getDelCount()) {
|
||||
throw new RuntimeException("delete count mismatch: info=" + (info.docCount - info.getDelCount()) + " vs reader=" + reader.numDocs());
|
||||
}
|
||||
if ((info.docCount-reader.liveDocs.count()) > reader.maxDoc()) {
|
||||
throw new RuntimeException("too many deleted docs: maxDoc()=" + reader.maxDoc() + " vs del count=" + (info.docCount-reader.liveDocs.count()));
|
||||
if ((info.docCount-reader.numDocs()) > reader.maxDoc()) {
|
||||
throw new RuntimeException("too many deleted docs: maxDoc()=" + reader.maxDoc() + " vs del count=" + (info.docCount-reader.numDocs()));
|
||||
}
|
||||
if (info.docCount - numDocs != info.getDelCount()) {
|
||||
throw new RuntimeException("delete count mismatch: info=" + info.getDelCount() + " vs reader=" + (info.docCount - numDocs));
|
||||
|
|
|
@ -349,7 +349,7 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
|
|||
try {
|
||||
synchronized(this) {
|
||||
if (verbose()) {
|
||||
message(" consider merge " + merge.segString(dir));
|
||||
message(" consider merge " + writer.segString(merge.segments));
|
||||
}
|
||||
|
||||
// OK to spawn a new merge thread to handle this
|
||||
|
@ -457,7 +457,7 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
|
|||
tWriter.mergeInit(merge);
|
||||
updateMergeThreads();
|
||||
if (verbose()) {
|
||||
message(" merge thread: do another merge " + merge.segString(dir));
|
||||
message(" merge thread: do another merge " + tWriter.segString(merge.segments));
|
||||
}
|
||||
} else {
|
||||
break;
|
||||
|
@ -492,9 +492,14 @@ public class ConcurrentMergeScheduler extends MergeScheduler {
|
|||
@Override
|
||||
public String toString() {
|
||||
MergePolicy.OneMerge merge = getRunningMerge();
|
||||
if (merge == null)
|
||||
if (merge == null) {
|
||||
merge = startMerge;
|
||||
return "merge thread: " + merge.segString(dir);
|
||||
}
|
||||
try {
|
||||
return "merge thread: " + tWriter.segString(merge.segments);
|
||||
} catch (IOException ioe) {
|
||||
throw new RuntimeException(ioe);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -62,7 +62,7 @@ final class DirectoryReader extends BaseMultiReader<SegmentReader> {
|
|||
IOException prior = null;
|
||||
boolean success = false;
|
||||
try {
|
||||
readers[i] = SegmentReader.get(sis.info(i), termInfosIndexDivisor, IOContext.READ);
|
||||
readers[i] = new SegmentReader(sis.info(i), termInfosIndexDivisor, IOContext.READ);
|
||||
success = true;
|
||||
} catch(IOException ex) {
|
||||
prior = ex;
|
||||
|
@ -94,7 +94,8 @@ final class DirectoryReader extends BaseMultiReader<SegmentReader> {
|
|||
try {
|
||||
final SegmentInfo info = infos.info(i);
|
||||
assert info.dir == dir;
|
||||
final SegmentReader reader = writer.readerPool.getReadOnlyClone(info, IOContext.READ);
|
||||
final IndexWriter.ReadersAndLiveDocs rld = writer.readerPool.get(info, true);
|
||||
final SegmentReader reader = rld.getReadOnlyClone(IOContext.READ);
|
||||
if (reader.numDocs() > 0 || writer.getKeepFullyDeletedSegments()) {
|
||||
readers.add(reader);
|
||||
infosUpto++;
|
||||
|
@ -116,7 +117,7 @@ final class DirectoryReader extends BaseMultiReader<SegmentReader> {
|
|||
|
||||
/** This constructor is only used for {@link #doOpenIfChanged()} */
|
||||
static DirectoryReader open(Directory directory, IndexWriter writer, SegmentInfos infos, SegmentReader[] oldReaders,
|
||||
boolean doClone, int termInfosIndexDivisor) throws IOException {
|
||||
int termInfosIndexDivisor) throws IOException {
|
||||
// we put the old SegmentReaders in a map, that allows us
|
||||
// to lookup a reader using its segment name
|
||||
final Map<String,Integer> segmentReaders = new HashMap<String,Integer>();
|
||||
|
@ -151,24 +152,21 @@ final class DirectoryReader extends BaseMultiReader<SegmentReader> {
|
|||
SegmentReader newReader;
|
||||
if (newReaders[i] == null || infos.info(i).getUseCompoundFile() != newReaders[i].getSegmentInfo().getUseCompoundFile()) {
|
||||
|
||||
// We should never see a totally new segment during cloning
|
||||
assert !doClone;
|
||||
|
||||
// this is a new reader; in case we hit an exception we can close it safely
|
||||
newReader = SegmentReader.get(infos.info(i), termInfosIndexDivisor, IOContext.READ);
|
||||
newReader = new SegmentReader(infos.info(i), termInfosIndexDivisor, IOContext.READ);
|
||||
readerShared[i] = false;
|
||||
newReaders[i] = newReader;
|
||||
} else {
|
||||
newReader = newReaders[i].reopenSegment(infos.info(i), doClone);
|
||||
if (newReader == null) {
|
||||
// this reader will be shared between the old and the new one,
|
||||
// so we must incRef it
|
||||
if (newReaders[i].getSegmentInfo().getDelGen() == infos.info(i).getDelGen()) {
|
||||
// No change; this reader will be shared between
|
||||
// the old and the new one, so we must incRef
|
||||
// it:
|
||||
readerShared[i] = true;
|
||||
newReaders[i].incRef();
|
||||
} else {
|
||||
readerShared[i] = false;
|
||||
// Steal ref returned to us by reopenSegment:
|
||||
newReaders[i] = newReader;
|
||||
// Steal the ref returned by SegmentReader ctor:
|
||||
newReaders[i] = new SegmentReader(infos.info(i), newReaders[i], IOContext.READ);
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
|
@ -223,16 +221,6 @@ final class DirectoryReader extends BaseMultiReader<SegmentReader> {
|
|||
return buffer.toString();
|
||||
}
|
||||
|
||||
@Override
|
||||
public final synchronized Object clone() {
|
||||
try {
|
||||
DirectoryReader newReader = doOpenIfChanged((SegmentInfos) segmentInfos.clone(), true, writer);
|
||||
return newReader;
|
||||
} catch (Exception ex) {
|
||||
throw new RuntimeException(ex);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected final IndexReader doOpenIfChanged() throws CorruptIndexException, IOException {
|
||||
return doOpenIfChanged(null);
|
||||
|
@ -302,13 +290,13 @@ final class DirectoryReader extends BaseMultiReader<SegmentReader> {
|
|||
protected Object doBody(String segmentFileName) throws CorruptIndexException, IOException {
|
||||
final SegmentInfos infos = new SegmentInfos();
|
||||
infos.read(directory, segmentFileName);
|
||||
return doOpenIfChanged(infos, false, null);
|
||||
return doOpenIfChanged(infos, null);
|
||||
}
|
||||
}.run(commit);
|
||||
}
|
||||
|
||||
private synchronized DirectoryReader doOpenIfChanged(SegmentInfos infos, boolean doClone, IndexWriter writer) throws CorruptIndexException, IOException {
|
||||
return DirectoryReader.open(directory, writer, infos, subReaders, doClone, termInfosIndexDivisor);
|
||||
private synchronized DirectoryReader doOpenIfChanged(SegmentInfos infos, IndexWriter writer) throws CorruptIndexException, IOException {
|
||||
return DirectoryReader.open(directory, writer, infos, subReaders, termInfosIndexDivisor);
|
||||
}
|
||||
|
||||
/** Version number when this IndexReader was opened. */
|
||||
|
|
|
@ -450,8 +450,10 @@ final class IndexFileDeleter {
|
|||
public void checkpoint(SegmentInfos segmentInfos, boolean isCommit) throws IOException {
|
||||
assert locked();
|
||||
|
||||
assert Thread.holdsLock(writer);
|
||||
|
||||
if (infoStream.isEnabled("IFD")) {
|
||||
infoStream.message("IFD", "now checkpoint \"" + segmentInfos.toString(directory) + "\" [" + segmentInfos.size() + " segments " + "; isCommit = " + isCommit + "]");
|
||||
infoStream.message("IFD", "now checkpoint \"" + writer.segString(segmentInfos) + "\" [" + segmentInfos.size() + " segments " + "; isCommit = " + isCommit + "]");
|
||||
}
|
||||
|
||||
// Try again now to delete any previously un-deletable
|
||||
|
|
|
@ -71,7 +71,7 @@ import org.apache.lucene.util.ReaderUtil; // for javadocs
|
|||
<code>IndexReader</code> instance; use your own
|
||||
(non-Lucene) objects instead.
|
||||
*/
|
||||
public abstract class IndexReader implements Cloneable,Closeable {
|
||||
public abstract class IndexReader implements Closeable {
|
||||
|
||||
/**
|
||||
* A custom listener that's invoked when the IndexReader
|
||||
|
@ -152,6 +152,8 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
|
||||
/** Expert: returns the current refCount for this reader */
|
||||
public final int getRefCount() {
|
||||
// NOTE: don't ensureOpen, so that callers can see
|
||||
// refCount is 0 (reader is closed)
|
||||
return refCount.get();
|
||||
}
|
||||
|
||||
|
@ -492,15 +494,6 @@ public abstract class IndexReader implements Cloneable,Closeable {
|
|||
return writer.getReader(applyAllDeletes);
|
||||
}
|
||||
|
||||
/**
|
||||
* Efficiently clones the IndexReader (sharing most
|
||||
* internal state).
|
||||
*/
|
||||
@Override
|
||||
public synchronized Object clone() {
|
||||
throw new UnsupportedOperationException("This reader does not implement clone()");
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the directory associated with this index. The Default
|
||||
* implementation returns the directory specified by subclasses when
|
||||
|
|
File diff suppressed because it is too large
Load Diff
|
@ -61,19 +61,6 @@ public class MultiReader extends BaseMultiReader<IndexReader> {
|
|||
|
||||
@Override
|
||||
protected synchronized IndexReader doOpenIfChanged() throws CorruptIndexException, IOException {
|
||||
return doReopen(false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized Object clone() {
|
||||
try {
|
||||
return doReopen(true);
|
||||
} catch (Exception ex) {
|
||||
throw new RuntimeException(ex);
|
||||
}
|
||||
}
|
||||
|
||||
private IndexReader doReopen(boolean doClone) throws CorruptIndexException, IOException {
|
||||
ensureOpen();
|
||||
|
||||
boolean changed = false;
|
||||
|
@ -82,17 +69,12 @@ public class MultiReader extends BaseMultiReader<IndexReader> {
|
|||
boolean success = false;
|
||||
try {
|
||||
for (int i = 0; i < subReaders.length; i++) {
|
||||
if (doClone) {
|
||||
newSubReaders[i] = (IndexReader) subReaders[i].clone();
|
||||
final IndexReader newSubReader = IndexReader.openIfChanged(subReaders[i]);
|
||||
if (newSubReader != null) {
|
||||
newSubReaders[i] = newSubReader;
|
||||
changed = true;
|
||||
} else {
|
||||
final IndexReader newSubReader = IndexReader.openIfChanged(subReaders[i]);
|
||||
if (newSubReader != null) {
|
||||
newSubReaders[i] = newSubReader;
|
||||
changed = true;
|
||||
} else {
|
||||
newSubReaders[i] = subReaders[i];
|
||||
}
|
||||
newSubReaders[i] = subReaders[i];
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
|
|
|
@ -205,16 +205,6 @@ public class ParallelReader extends IndexReader {
|
|||
return fields;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized Object clone() {
|
||||
// doReopen calls ensureOpen
|
||||
try {
|
||||
return doReopen(true);
|
||||
} catch (Exception ex) {
|
||||
throw new RuntimeException(ex);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Tries to reopen the subreaders.
|
||||
* <br>
|
||||
|
@ -236,11 +226,6 @@ public class ParallelReader extends IndexReader {
|
|||
*/
|
||||
@Override
|
||||
protected synchronized IndexReader doOpenIfChanged() throws CorruptIndexException, IOException {
|
||||
// doReopen calls ensureOpen
|
||||
return doReopen(false);
|
||||
}
|
||||
|
||||
private IndexReader doReopen(boolean doClone) throws CorruptIndexException, IOException {
|
||||
ensureOpen();
|
||||
|
||||
boolean reopened = false;
|
||||
|
@ -251,16 +236,11 @@ public class ParallelReader extends IndexReader {
|
|||
try {
|
||||
for (final IndexReader oldReader : readers) {
|
||||
IndexReader newReader = null;
|
||||
if (doClone) {
|
||||
newReader = (IndexReader) oldReader.clone();
|
||||
newReader = IndexReader.openIfChanged(oldReader);
|
||||
if (newReader != null) {
|
||||
reopened = true;
|
||||
} else {
|
||||
newReader = IndexReader.openIfChanged(oldReader);
|
||||
if (newReader != null) {
|
||||
reopened = true;
|
||||
} else {
|
||||
newReader = oldReader;
|
||||
}
|
||||
newReader = oldReader;
|
||||
}
|
||||
newReaders.add(newReader);
|
||||
}
|
||||
|
|
|
@ -62,10 +62,10 @@ final class SegmentCoreReaders {
|
|||
|
||||
private final SegmentReader owner;
|
||||
|
||||
StoredFieldsReader fieldsReaderOrig;
|
||||
TermVectorsReader termVectorsReaderOrig;
|
||||
CompoundFileDirectory cfsReader;
|
||||
CompoundFileDirectory storeCFSReader;
|
||||
final StoredFieldsReader fieldsReaderOrig;
|
||||
final TermVectorsReader termVectorsReaderOrig;
|
||||
final CompoundFileDirectory cfsReader;
|
||||
final CompoundFileDirectory storeCFSReader;
|
||||
|
||||
private final Set<CoreClosedListener> coreClosedListeners =
|
||||
Collections.synchronizedSet(new LinkedHashSet<CoreClosedListener>());
|
||||
|
@ -88,6 +88,8 @@ final class SegmentCoreReaders {
|
|||
if (si.getUseCompoundFile()) {
|
||||
cfsReader = new CompoundFileDirectory(dir, IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
|
||||
dir0 = cfsReader;
|
||||
} else {
|
||||
cfsReader = null;
|
||||
}
|
||||
cfsDir = dir0;
|
||||
si.loadFieldInfos(cfsDir, false); // prevent opening the CFS to load fieldInfos
|
||||
|
@ -104,6 +106,38 @@ final class SegmentCoreReaders {
|
|||
// kinda jaky to assume the codec handles the case of no norms file at all gracefully?!
|
||||
norms = codec.normsFormat().normsReader(cfsDir, si, fieldInfos, context, dir);
|
||||
perDocProducer = codec.docValuesFormat().docsProducer(segmentReadState);
|
||||
|
||||
final Directory storeDir;
|
||||
if (si.getDocStoreOffset() != -1) {
|
||||
if (si.getDocStoreIsCompoundFile()) {
|
||||
storeCFSReader = new CompoundFileDirectory(dir,
|
||||
IndexFileNames.segmentFileName(si.getDocStoreSegment(), "", IndexFileNames.COMPOUND_FILE_STORE_EXTENSION),
|
||||
context, false);
|
||||
storeDir = storeCFSReader;
|
||||
assert storeDir != null;
|
||||
} else {
|
||||
storeCFSReader = null;
|
||||
storeDir = dir;
|
||||
assert storeDir != null;
|
||||
}
|
||||
} else if (si.getUseCompoundFile()) {
|
||||
storeDir = cfsReader;
|
||||
storeCFSReader = null;
|
||||
assert storeDir != null;
|
||||
} else {
|
||||
storeDir = dir;
|
||||
storeCFSReader = null;
|
||||
assert storeDir != null;
|
||||
}
|
||||
|
||||
fieldsReaderOrig = si.getCodec().storedFieldsFormat().fieldsReader(storeDir, si, fieldInfos, context);
|
||||
|
||||
if (si.getHasVectors()) { // open term vector files only as needed
|
||||
termVectorsReaderOrig = si.getCodec().termVectorsFormat().vectorsReader(storeDir, si, fieldInfos, context);
|
||||
} else {
|
||||
termVectorsReaderOrig = null;
|
||||
}
|
||||
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
|
@ -118,23 +152,24 @@ final class SegmentCoreReaders {
|
|||
this.owner = owner;
|
||||
}
|
||||
|
||||
synchronized TermVectorsReader getTermVectorsReaderOrig() {
|
||||
TermVectorsReader getTermVectorsReaderOrig() {
|
||||
return termVectorsReaderOrig;
|
||||
}
|
||||
|
||||
synchronized StoredFieldsReader getFieldsReaderOrig() {
|
||||
StoredFieldsReader getFieldsReaderOrig() {
|
||||
return fieldsReaderOrig;
|
||||
}
|
||||
|
||||
synchronized void incRef() {
|
||||
void incRef() {
|
||||
ref.incrementAndGet();
|
||||
}
|
||||
|
||||
synchronized Directory getCFSReader() {
|
||||
Directory getCFSReader() {
|
||||
return cfsReader;
|
||||
}
|
||||
|
||||
synchronized void decRef() throws IOException {
|
||||
void decRef() throws IOException {
|
||||
//System.out.println("core.decRef seg=" + owner.getSegmentInfo() + " rc=" + ref);
|
||||
if (ref.decrementAndGet() == 0) {
|
||||
IOUtils.close(fields, perDocProducer, termVectorsReaderOrig,
|
||||
fieldsReaderOrig, cfsReader, storeCFSReader, norms);
|
||||
|
@ -158,46 +193,6 @@ final class SegmentCoreReaders {
|
|||
coreClosedListeners.remove(listener);
|
||||
}
|
||||
|
||||
synchronized void openDocStores(SegmentInfo si) throws IOException {
|
||||
|
||||
assert si.name.equals(segment);
|
||||
|
||||
if (fieldsReaderOrig == null) {
|
||||
final Directory storeDir;
|
||||
if (si.getDocStoreOffset() != -1) {
|
||||
if (si.getDocStoreIsCompoundFile()) {
|
||||
assert storeCFSReader == null;
|
||||
storeCFSReader = new CompoundFileDirectory(dir,
|
||||
IndexFileNames.segmentFileName(si.getDocStoreSegment(), "", IndexFileNames.COMPOUND_FILE_STORE_EXTENSION),
|
||||
context, false);
|
||||
storeDir = storeCFSReader;
|
||||
assert storeDir != null;
|
||||
} else {
|
||||
storeDir = dir;
|
||||
assert storeDir != null;
|
||||
}
|
||||
} else if (si.getUseCompoundFile()) {
|
||||
// In some cases, we were originally opened when CFS
|
||||
// was not used, but then we are asked to open doc
|
||||
// stores after the segment has switched to CFS
|
||||
if (cfsReader == null) {
|
||||
cfsReader = new CompoundFileDirectory(dir,IndexFileNames.segmentFileName(segment, "", IndexFileNames.COMPOUND_FILE_EXTENSION), context, false);
|
||||
}
|
||||
storeDir = cfsReader;
|
||||
assert storeDir != null;
|
||||
} else {
|
||||
storeDir = dir;
|
||||
assert storeDir != null;
|
||||
}
|
||||
|
||||
fieldsReaderOrig = si.getCodec().storedFieldsFormat().fieldsReader(storeDir, si, fieldInfos, context);
|
||||
|
||||
if (si.getHasVectors()) { // open term vector files only as needed
|
||||
termVectorsReaderOrig = si.getCodec().termVectorsFormat().vectorsReader(storeDir, si, fieldInfos, context);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "SegmentCoreReader(owner=" + owner + ")";
|
||||
|
|
|
@ -362,7 +362,8 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
final SegmentInfo info = it.next();
|
||||
if (info.getDelCount() == info.docCount) {
|
||||
it.remove();
|
||||
segmentSet.remove(info);
|
||||
final boolean didRemove = segmentSet.remove(info);
|
||||
assert didRemove;
|
||||
}
|
||||
}
|
||||
assert segmentSet.size() == segments.size();
|
||||
|
@ -995,8 +996,11 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
}
|
||||
}
|
||||
|
||||
// the rest of the segments in list are duplicates, so don't remove from map, only list!
|
||||
segments.subList(newSegIdx, segments.size()).clear();
|
||||
|
||||
// Either we found place to insert segment, or, we did
|
||||
// not, but only because all segments we merged became
|
||||
// not, but only because all segments we merged becamee
|
||||
// deleted while we are merging, in which case it should
|
||||
// be the case that the new segment is also all deleted,
|
||||
// we insert it at the beginning if it should not be dropped:
|
||||
|
@ -1004,9 +1008,6 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
segments.add(0, merge.info);
|
||||
}
|
||||
|
||||
// the rest of the segments in list are duplicates, so don't remove from map, only list!
|
||||
segments.subList(newSegIdx, segments.size()).clear();
|
||||
|
||||
// update the Set
|
||||
if (!dropSegment) {
|
||||
segmentSet.add(merge.info);
|
||||
|
@ -1112,5 +1113,4 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentInfo> {
|
|||
return -1;
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -23,7 +23,6 @@ import java.util.Collection;
|
|||
import java.util.HashSet;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.codecs.PerDocProducer;
|
||||
|
@ -39,27 +38,21 @@ import org.apache.lucene.util.CloseableThreadLocal;
|
|||
/**
|
||||
* @lucene.experimental
|
||||
*/
|
||||
public final class SegmentReader extends IndexReader implements Cloneable {
|
||||
private final boolean readOnly;
|
||||
public final class SegmentReader extends IndexReader {
|
||||
|
||||
private SegmentInfo si;
|
||||
private final SegmentInfo si;
|
||||
private final ReaderContext readerContext = new AtomicReaderContext(this);
|
||||
final CloseableThreadLocal<StoredFieldsReader> fieldsReaderLocal = new FieldsReaderLocal();
|
||||
final CloseableThreadLocal<TermVectorsReader> termVectorsLocal = new CloseableThreadLocal<TermVectorsReader>();
|
||||
private final CloseableThreadLocal<StoredFieldsReader> fieldsReaderLocal = new FieldsReaderLocal();
|
||||
private final CloseableThreadLocal<TermVectorsReader> termVectorsLocal = new CloseableThreadLocal<TermVectorsReader>();
|
||||
|
||||
volatile BitVector liveDocs = null;
|
||||
volatile Object combinedCoreAndDeletesKey;
|
||||
AtomicInteger liveDocsRef = null;
|
||||
boolean hasChanges = false;
|
||||
private final BitVector liveDocs;
|
||||
|
||||
// TODO: remove deletions from SR
|
||||
private int pendingDeleteCount;
|
||||
private boolean rollbackHasChanges = false;
|
||||
private SegmentInfo rollbackSegmentInfo;
|
||||
private int rollbackPendingDeleteCount;
|
||||
// end TODO
|
||||
// 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;
|
||||
|
||||
SegmentCoreReaders core;
|
||||
private final SegmentCoreReaders core;
|
||||
|
||||
/**
|
||||
* Sets the initial value
|
||||
|
@ -75,56 +68,73 @@ public final class SegmentReader extends IndexReader implements Cloneable {
|
|||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
public static SegmentReader get(SegmentInfo si, int termInfosIndexDivisor, IOContext context) throws CorruptIndexException, IOException {
|
||||
return get(true, si, true, termInfosIndexDivisor, context);
|
||||
}
|
||||
|
||||
// TODO: remove deletions from SR
|
||||
static SegmentReader getRW(SegmentInfo si, boolean doOpenStores, int termInfosIndexDivisor, IOContext context) throws CorruptIndexException, IOException {
|
||||
return get(false, si, doOpenStores, termInfosIndexDivisor, context);
|
||||
}
|
||||
|
||||
/**
|
||||
* @throws CorruptIndexException if the index is corrupt
|
||||
* @throws IOException if there is a low-level IO error
|
||||
*/
|
||||
private static SegmentReader get(boolean readOnly,
|
||||
SegmentInfo si,
|
||||
boolean doOpenStores,
|
||||
int termInfosIndexDivisor,
|
||||
IOContext context)
|
||||
throws CorruptIndexException, IOException {
|
||||
|
||||
SegmentReader instance = new SegmentReader(readOnly, si);
|
||||
public SegmentReader(SegmentInfo si, int termInfosIndexDivisor, IOContext context) throws IOException {
|
||||
this.si = si;
|
||||
boolean success = false;
|
||||
try {
|
||||
instance.core = new SegmentCoreReaders(instance, si.dir, si, context, termInfosIndexDivisor);
|
||||
if (doOpenStores) {
|
||||
instance.core.openDocStores(si);
|
||||
core = new SegmentCoreReaders(this, si.dir, si, context, termInfosIndexDivisor);
|
||||
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));
|
||||
} else {
|
||||
assert si.getDelCount() == 0;
|
||||
liveDocs = null;
|
||||
}
|
||||
instance.loadLiveDocs(context);
|
||||
numDocs = si.docCount - si.getDelCount();
|
||||
assert checkLiveCounts(false);
|
||||
success = true;
|
||||
} finally {
|
||||
|
||||
// With lock-less commits, it's entirely possible (and
|
||||
// fine) to hit a FileNotFound exception above. In
|
||||
// this case, we want to explicitly close any subset
|
||||
// of things that were opened so that we don't have to
|
||||
// wait for a GC to do so.
|
||||
if (!success) {
|
||||
instance.doClose();
|
||||
doClose();
|
||||
}
|
||||
}
|
||||
return instance;
|
||||
}
|
||||
|
||||
private SegmentReader(boolean readOnly, SegmentInfo si) {
|
||||
this.readOnly = readOnly;
|
||||
// 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(), new IOContext(IOContext.READ, true));
|
||||
numDocs = si.docCount - si.getDelCount();
|
||||
assert checkLiveCounts(false);
|
||||
|
||||
// We share core w/ parent:
|
||||
parent.core.incRef();
|
||||
core = parent.core;
|
||||
}
|
||||
|
||||
void openDocStores() throws IOException {
|
||||
core.openDocStores(si);
|
||||
// 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;
|
||||
|
||||
assert liveDocs != null;
|
||||
this.liveDocs = liveDocs;
|
||||
|
||||
this.numDocs = numDocs;
|
||||
|
||||
assert checkLiveCounts(true);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -133,123 +143,37 @@ public final class SegmentReader extends IndexReader implements Cloneable {
|
|||
return liveDocs;
|
||||
}
|
||||
|
||||
private boolean checkLiveCounts() throws IOException {
|
||||
final int recomputedCount = liveDocs.getRecomputedCount();
|
||||
// First verify BitVector is self consistent:
|
||||
assert liveDocs.count() == recomputedCount : "live count=" + liveDocs.count() + " vs recomputed count=" + recomputedCount;
|
||||
|
||||
assert si.getDelCount() == si.docCount - recomputedCount :
|
||||
"delete count mismatch: info=" + si.getDelCount() + " vs BitVector=" + (si.docCount-recomputedCount);
|
||||
|
||||
// Verify # deletes does not exceed maxDoc for this
|
||||
// segment:
|
||||
assert si.getDelCount() <= maxDoc() :
|
||||
"delete count mismatch: " + recomputedCount + ") exceeds max doc (" + maxDoc() + ") for segment " + si.name;
|
||||
|
||||
return true;
|
||||
}
|
||||
|
||||
private void loadLiveDocs(IOContext context) throws IOException {
|
||||
// NOTE: the bitvector is stored using the regular directory, not cfs
|
||||
if (si.hasDeletions()) {
|
||||
liveDocs = new BitVector(directory(), si.getDelFileName(), new IOContext(context, true));
|
||||
liveDocsRef = new AtomicInteger(1);
|
||||
assert checkLiveCounts();
|
||||
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);
|
||||
}
|
||||
} else {
|
||||
assert si.getDelCount() == 0;
|
||||
}
|
||||
// we need a key reflecting actual deletes (if existent or not):
|
||||
combinedCoreAndDeletesKey = new Object();
|
||||
}
|
||||
|
||||
/** Clones are always in readOnly mode */
|
||||
@Override
|
||||
public final synchronized Object clone() {
|
||||
try {
|
||||
return reopenSegment(si, true);
|
||||
} catch (Exception ex) {
|
||||
throw new RuntimeException(ex);
|
||||
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;
|
||||
}
|
||||
}
|
||||
|
||||
// used by DirectoryReader:
|
||||
synchronized SegmentReader reopenSegment(SegmentInfo si, boolean doClone) throws CorruptIndexException, IOException {
|
||||
ensureOpen();
|
||||
boolean deletionsUpToDate = (this.si.hasDeletions() == si.hasDeletions())
|
||||
&& (!si.hasDeletions() || this.si.getDelFileName().equals(si.getDelFileName()));
|
||||
|
||||
// if we're cloning we need to run through the reopenSegment logic
|
||||
// also if both old and new readers aren't readonly, we clone to avoid sharing modifications
|
||||
if (deletionsUpToDate && !doClone && readOnly) {
|
||||
return null;
|
||||
}
|
||||
|
||||
// When cloning, the incoming SegmentInfos should not
|
||||
// have any changes in it:
|
||||
assert !doClone || (deletionsUpToDate);
|
||||
|
||||
// clone reader
|
||||
SegmentReader clone = new SegmentReader(true, si);
|
||||
|
||||
boolean success = false;
|
||||
try {
|
||||
core.incRef();
|
||||
clone.core = core;
|
||||
clone.pendingDeleteCount = pendingDeleteCount;
|
||||
clone.combinedCoreAndDeletesKey = combinedCoreAndDeletesKey;
|
||||
|
||||
if (doClone) {
|
||||
if (liveDocs != null) {
|
||||
liveDocsRef.incrementAndGet();
|
||||
clone.liveDocs = liveDocs;
|
||||
clone.liveDocsRef = liveDocsRef;
|
||||
}
|
||||
} else {
|
||||
if (!deletionsUpToDate) {
|
||||
// load deleted docs
|
||||
assert clone.liveDocs == null;
|
||||
clone.loadLiveDocs(IOContext.READ);
|
||||
} else if (liveDocs != null) {
|
||||
liveDocsRef.incrementAndGet();
|
||||
clone.liveDocs = liveDocs;
|
||||
clone.liveDocsRef = liveDocsRef;
|
||||
}
|
||||
}
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
// An exception occurred during reopen, we have to decRef the norms
|
||||
// that we incRef'ed already and close singleNormsStream and FieldsReader
|
||||
clone.decRef();
|
||||
}
|
||||
}
|
||||
|
||||
return clone;
|
||||
return true;
|
||||
}
|
||||
|
||||
/** @lucene.internal */
|
||||
public StoredFieldsReader getFieldsReader() {
|
||||
return fieldsReaderLocal.get();
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
protected void doClose() throws IOException {
|
||||
if (hasChanges) {
|
||||
doCommit();
|
||||
}
|
||||
|
||||
//System.out.println("SR.close seg=" + si);
|
||||
termVectorsLocal.close();
|
||||
fieldsReaderLocal.close();
|
||||
|
||||
if (liveDocs != null) {
|
||||
liveDocsRef.decrementAndGet();
|
||||
// null so if an app hangs on to us we still free most ram
|
||||
liveDocs = null;
|
||||
}
|
||||
|
||||
if (core != null) {
|
||||
core.decRef();
|
||||
}
|
||||
|
@ -409,12 +333,9 @@ public final class SegmentReader extends IndexReader implements Cloneable {
|
|||
|
||||
@Override
|
||||
public String toString() {
|
||||
final StringBuilder buffer = new StringBuilder();
|
||||
if (hasChanges) {
|
||||
buffer.append('*');
|
||||
}
|
||||
buffer.append(si.toString(core.dir, pendingDeleteCount));
|
||||
return buffer.toString();
|
||||
// SegmentInfo.toString takes dir and number of
|
||||
// *pending* deletions; so we reverse compute that here:
|
||||
return si.toString(core.dir, si.docCount - numDocs - si.getDelCount());
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -437,10 +358,6 @@ public final class SegmentReader extends IndexReader implements Cloneable {
|
|||
return si;
|
||||
}
|
||||
|
||||
void setSegmentInfo(SegmentInfo info) {
|
||||
si = info;
|
||||
}
|
||||
|
||||
/** Returns the directory this index resides in. */
|
||||
@Override
|
||||
public Directory directory() {
|
||||
|
@ -460,7 +377,7 @@ public final class SegmentReader extends IndexReader implements Cloneable {
|
|||
|
||||
@Override
|
||||
public Object getCombinedCoreAndDeletesKey() {
|
||||
return combinedCoreAndDeletesKey;
|
||||
return this;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -478,103 +395,6 @@ public final class SegmentReader extends IndexReader implements Cloneable {
|
|||
return perDoc.docValues(field);
|
||||
}
|
||||
|
||||
/**
|
||||
* Clones the deleteDocs BitVector. May be overridden by subclasses. New and experimental.
|
||||
* @param bv BitVector to clone
|
||||
* @return New BitVector
|
||||
*/
|
||||
// TODO: remove deletions from SR
|
||||
BitVector cloneDeletedDocs(BitVector bv) {
|
||||
ensureOpen();
|
||||
return (BitVector)bv.clone();
|
||||
}
|
||||
|
||||
// TODO: remove deletions from SR
|
||||
void doCommit() throws IOException {
|
||||
assert hasChanges;
|
||||
startCommit();
|
||||
boolean success = false;
|
||||
try {
|
||||
commitChanges();
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
rollbackCommit();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// TODO: remove deletions from SR
|
||||
private void startCommit() {
|
||||
rollbackSegmentInfo = (SegmentInfo) si.clone();
|
||||
rollbackHasChanges = hasChanges;
|
||||
rollbackPendingDeleteCount = pendingDeleteCount;
|
||||
}
|
||||
|
||||
// TODO: remove deletions from SR
|
||||
private void rollbackCommit() {
|
||||
si.reset(rollbackSegmentInfo);
|
||||
hasChanges = rollbackHasChanges;
|
||||
pendingDeleteCount = rollbackPendingDeleteCount;
|
||||
}
|
||||
|
||||
// TODO: remove deletions from SR
|
||||
private synchronized void commitChanges() throws IOException {
|
||||
si.advanceDelGen();
|
||||
|
||||
assert liveDocs.length() == si.docCount;
|
||||
|
||||
// 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 = si.getDelFileName();
|
||||
boolean success = false;
|
||||
try {
|
||||
liveDocs.write(directory(), delFileName, IOContext.DEFAULT);
|
||||
success = true;
|
||||
} finally {
|
||||
if (!success) {
|
||||
try {
|
||||
directory().deleteFile(delFileName);
|
||||
} catch (Throwable t) {
|
||||
// suppress this so we keep throwing the
|
||||
// original exception
|
||||
}
|
||||
}
|
||||
}
|
||||
si.setDelCount(si.getDelCount()+pendingDeleteCount);
|
||||
pendingDeleteCount = 0;
|
||||
assert (maxDoc()-liveDocs.count()) == si.getDelCount(): "delete count mismatch during commit: info=" + si.getDelCount() + " vs BitVector=" + (maxDoc()-liveDocs.count());
|
||||
hasChanges = false;
|
||||
}
|
||||
|
||||
// TODO: remove deletions from SR
|
||||
synchronized void deleteDocument(int docNum) throws IOException {
|
||||
if (readOnly)
|
||||
throw new UnsupportedOperationException("this SegmentReader is read only");
|
||||
hasChanges = true;
|
||||
if (liveDocs == null) {
|
||||
liveDocs = new BitVector(maxDoc());
|
||||
liveDocs.setAll();
|
||||
liveDocsRef = new AtomicInteger(1);
|
||||
}
|
||||
// there is more than 1 SegmentReader with a reference to this
|
||||
// liveDocs BitVector so decRef the current liveDocsRef,
|
||||
// clone the BitVector, create a new liveDocsRef
|
||||
if (liveDocsRef.get() > 1) {
|
||||
AtomicInteger oldRef = liveDocsRef;
|
||||
liveDocs = cloneDeletedDocs(liveDocs);
|
||||
liveDocsRef = new AtomicInteger(1);
|
||||
oldRef.decrementAndGet();
|
||||
}
|
||||
// we need a key reflecting actual deletes (if existent or not):
|
||||
combinedCoreAndDeletesKey = new Object();
|
||||
// liveDocs are now dirty:
|
||||
if (liveDocs.getAndClear(docNum)) {
|
||||
pendingDeleteCount++;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Called when the shared core for this SegmentReader
|
||||
* is closed.
|
||||
|
|
|
@ -58,6 +58,10 @@ public class TestExternalCodecs extends LuceneTestCase {
|
|||
public void testPerFieldCodec() throws Exception {
|
||||
|
||||
final int NUM_DOCS = atLeast(173);
|
||||
if (VERBOSE) {
|
||||
System.out.println("TEST: NUM_DOCS=" + NUM_DOCS);
|
||||
}
|
||||
|
||||
MockDirectoryWrapper dir = newDirectory();
|
||||
dir.setCheckIndexOnClose(false); // we use a custom codec provider
|
||||
IndexWriter w = new IndexWriter(
|
||||
|
@ -101,7 +105,14 @@ public class TestExternalCodecs extends LuceneTestCase {
|
|||
System.out.println("\nTEST: now delete 2nd doc");
|
||||
}
|
||||
w.deleteDocuments(new Term("id", "44"));
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: now force merge");
|
||||
}
|
||||
w.forceMerge(1);
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: now open reader");
|
||||
}
|
||||
r = IndexReader.open(w, true);
|
||||
assertEquals(NUM_DOCS-2, r.maxDoc());
|
||||
assertEquals(NUM_DOCS-2, r.numDocs());
|
||||
|
@ -112,6 +123,9 @@ public class TestExternalCodecs extends LuceneTestCase {
|
|||
assertEquals(0, s.search(new TermQuery(new Term("id", "77")), 1).totalHits);
|
||||
assertEquals(0, s.search(new TermQuery(new Term("id", "44")), 1).totalHits);
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: now close NRT reader");
|
||||
}
|
||||
r.close();
|
||||
|
||||
w.close();
|
||||
|
|
|
@ -443,6 +443,9 @@ public class TestAddIndexes extends LuceneTestCase {
|
|||
setMergePolicy(newLogMergePolicy(4))
|
||||
);
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: now addIndexes");
|
||||
}
|
||||
writer.addIndexes(aux, new MockDirectoryWrapper(random, new RAMDirectory(aux, newIOContext(random))));
|
||||
assertEquals(1020, writer.maxDoc());
|
||||
assertEquals(1000, writer.getDocCount(0));
|
||||
|
|
|
@ -197,8 +197,8 @@ public class TestDoc extends LuceneTestCase {
|
|||
private SegmentInfo merge(Directory dir, SegmentInfo si1, SegmentInfo si2, String merged, boolean useCompoundFile)
|
||||
throws Exception {
|
||||
IOContext context = newIOContext(random);
|
||||
SegmentReader r1 = SegmentReader.get(si1, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, context);
|
||||
SegmentReader r2 = SegmentReader.get(si2, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, context);
|
||||
SegmentReader r1 = new SegmentReader(si1, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, context);
|
||||
SegmentReader r2 = new SegmentReader(si2, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, context);
|
||||
|
||||
final Codec codec = Codec.getDefault();
|
||||
SegmentMerger merger = new SegmentMerger(InfoStream.getDefault(), si1.dir, IndexWriterConfig.DEFAULT_TERM_INDEX_INTERVAL, merged, MergeState.CheckAbort.NONE, null, new FieldInfos(new FieldInfos.FieldNumberBiMap()), codec, context);
|
||||
|
@ -225,7 +225,7 @@ public class TestDoc extends LuceneTestCase {
|
|||
|
||||
private void printSegment(PrintWriter out, SegmentInfo si)
|
||||
throws Exception {
|
||||
SegmentReader reader = SegmentReader.get(si, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
SegmentReader reader = new SegmentReader(si, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
|
||||
for (int i = 0; i < reader.numDocs(); i++)
|
||||
out.println(reader.document(i));
|
||||
|
|
|
@ -64,7 +64,7 @@ public class TestDocumentWriter extends LuceneTestCase {
|
|||
SegmentInfo info = writer.newestSegment();
|
||||
writer.close();
|
||||
//After adding the document, we should be able to read it back in
|
||||
SegmentReader reader = SegmentReader.get(info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
SegmentReader reader = new SegmentReader(info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
assertTrue(reader != null);
|
||||
Document doc = reader.document(0);
|
||||
assertTrue(doc != null);
|
||||
|
@ -94,7 +94,7 @@ public class TestDocumentWriter extends LuceneTestCase {
|
|||
|
||||
// test that the norms are not present in the segment if
|
||||
// omitNorms is true
|
||||
for (FieldInfo fi : reader.core.fieldInfos) {
|
||||
for (FieldInfo fi : reader.fieldInfos()) {
|
||||
if (fi.isIndexed) {
|
||||
assertTrue(fi.omitNorms == !reader.hasNorms(fi.name));
|
||||
}
|
||||
|
@ -125,7 +125,7 @@ public class TestDocumentWriter extends LuceneTestCase {
|
|||
writer.commit();
|
||||
SegmentInfo info = writer.newestSegment();
|
||||
writer.close();
|
||||
SegmentReader reader = SegmentReader.get(info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
SegmentReader reader = new SegmentReader(info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
|
||||
DocsAndPositionsEnum termPositions = MultiFields.getTermPositionsEnum(reader, MultiFields.getLiveDocs(reader),
|
||||
"repeated", new BytesRef("repeated"));
|
||||
|
@ -197,7 +197,7 @@ public class TestDocumentWriter extends LuceneTestCase {
|
|||
writer.commit();
|
||||
SegmentInfo info = writer.newestSegment();
|
||||
writer.close();
|
||||
SegmentReader reader = SegmentReader.get(info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
SegmentReader reader = new SegmentReader(info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
|
||||
DocsAndPositionsEnum termPositions = MultiFields.getTermPositionsEnum(reader, reader.getLiveDocs(), "f1", new BytesRef("a"));
|
||||
assertTrue(termPositions.nextDoc() != termPositions.NO_MORE_DOCS);
|
||||
|
@ -241,7 +241,7 @@ public class TestDocumentWriter extends LuceneTestCase {
|
|||
writer.commit();
|
||||
SegmentInfo info = writer.newestSegment();
|
||||
writer.close();
|
||||
SegmentReader reader = SegmentReader.get(info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
SegmentReader reader = new SegmentReader(info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
|
||||
DocsAndPositionsEnum termPositions = reader.termPositionsEnum(reader.getLiveDocs(), "preanalyzed", new BytesRef("term1"));
|
||||
assertTrue(termPositions.nextDoc() != termPositions.NO_MORE_DOCS);
|
||||
|
|
|
@ -717,36 +717,6 @@ public class TestIndexReader extends LuceneTestCase {
|
|||
dir.close();
|
||||
}
|
||||
|
||||
// LUCENE-1579: Ensure that on a cloned reader, segments
|
||||
// reuse the doc values arrays in FieldCache
|
||||
public void testFieldCacheReuseAfterClone() throws Exception {
|
||||
Directory dir = newDirectory();
|
||||
IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)));
|
||||
Document doc = new Document();
|
||||
doc.add(newField("number", "17", StringField.TYPE_UNSTORED));
|
||||
writer.addDocument(doc);
|
||||
writer.close();
|
||||
|
||||
// Open reader
|
||||
IndexReader r = getOnlySegmentReader(IndexReader.open(dir));
|
||||
final int[] ints = FieldCache.DEFAULT.getInts(r, "number", false);
|
||||
assertEquals(1, ints.length);
|
||||
assertEquals(17, ints[0]);
|
||||
|
||||
// Clone reader
|
||||
IndexReader r2 = (IndexReader) r.clone();
|
||||
r.close();
|
||||
assertTrue(r2 != r);
|
||||
final int[] ints2 = FieldCache.DEFAULT.getInts(r2, "number", false);
|
||||
r2.close();
|
||||
|
||||
assertEquals(1, ints2.length);
|
||||
assertEquals(17, ints2[0]);
|
||||
assertTrue(ints == ints2);
|
||||
|
||||
dir.close();
|
||||
}
|
||||
|
||||
// LUCENE-1579: Ensure that on a reopened reader, that any
|
||||
// shared segments reuse the doc values arrays in
|
||||
// FieldCache
|
||||
|
|
|
@ -1,86 +0,0 @@
|
|||
package org.apache.lucene.index;
|
||||
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
* contributor license agreements. See the NOTICE file distributed with
|
||||
* this work for additional information regarding copyright ownership.
|
||||
* The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
* (the "License"); you may not use this file except in compliance with
|
||||
* the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing, software
|
||||
* distributed under the License is distributed on an "AS IS" BASIS,
|
||||
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
* See the License for the specific language governing permissions and
|
||||
* limitations under the License.
|
||||
*/
|
||||
|
||||
import org.apache.lucene.analysis.MockAnalyzer;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.LuceneTestCase;
|
||||
|
||||
/**
|
||||
* Tests cloning multiple types of readers, modifying the liveDocs and norms
|
||||
* and verifies copy on write semantics of the liveDocs and norms is
|
||||
* implemented properly
|
||||
*/
|
||||
public class TestIndexReaderClone extends LuceneTestCase {
|
||||
|
||||
public void testDirectoryReader() throws Exception {
|
||||
final Directory dir = createIndex(0);
|
||||
performDefaultTests(IndexReader.open(dir));
|
||||
dir.close();
|
||||
}
|
||||
|
||||
public void testMultiReader() throws Exception {
|
||||
final Directory dir1 = createIndex(0);
|
||||
final IndexReader r1 = IndexReader.open(dir1);
|
||||
final Directory dir2 = createIndex(0);
|
||||
final IndexReader r2 = IndexReader.open(dir2);
|
||||
final MultiReader mr = new MultiReader(r1, r2);
|
||||
performDefaultTests(mr);
|
||||
dir1.close();
|
||||
dir2.close();
|
||||
}
|
||||
|
||||
public void testParallelReader() throws Exception {
|
||||
final Directory dir1 = createIndex(0);
|
||||
final IndexReader r1 = IndexReader.open(dir1);
|
||||
final Directory dir2 = createIndex(1);
|
||||
final IndexReader r2 = IndexReader.open(dir2);
|
||||
final ParallelReader pr = new ParallelReader();
|
||||
pr.add(r1);
|
||||
pr.add(r2);
|
||||
performDefaultTests(pr);
|
||||
dir1.close();
|
||||
dir2.close();
|
||||
}
|
||||
|
||||
private Directory createIndex(int no) throws Exception {
|
||||
final Directory dir = newDirectory();
|
||||
IndexWriter w = new IndexWriter(
|
||||
dir,
|
||||
newIndexWriterConfig(TEST_VERSION_CURRENT, new MockAnalyzer(random)).
|
||||
setMergePolicy(newLogMergePolicy(false))
|
||||
);
|
||||
Document doc = new Document();
|
||||
doc.add(newField("field"+no, "yes it's stored", TextField.TYPE_STORED));
|
||||
w.addDocument(doc);
|
||||
w.close();
|
||||
return dir;
|
||||
}
|
||||
|
||||
private void performDefaultTests(IndexReader r1) throws Exception {
|
||||
IndexReader r2 = (IndexReader) r1.clone();
|
||||
assertTrue(r1 != r2);
|
||||
TestIndexReader.assertIndexEquals(r1, r2);
|
||||
r1.close();
|
||||
r2.close();
|
||||
TestIndexReaderReopen.assertReaderClosed(r1, true, true);
|
||||
TestIndexReaderReopen.assertReaderClosed(r2, true, true);
|
||||
}
|
||||
}
|
|
@ -1128,8 +1128,7 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
writer2.close();
|
||||
|
||||
IndexReader r1 = IndexReader.open(dir2);
|
||||
IndexReader r2 = (IndexReader) r1.clone();
|
||||
writer.addIndexes(r1, r2);
|
||||
writer.addIndexes(r1, r1);
|
||||
writer.close();
|
||||
|
||||
IndexReader r3 = IndexReader.open(dir);
|
||||
|
@ -1137,7 +1136,6 @@ public class TestIndexWriter extends LuceneTestCase {
|
|||
r3.close();
|
||||
|
||||
r1.close();
|
||||
r2.close();
|
||||
|
||||
dir2.close();
|
||||
dir.close();
|
||||
|
|
|
@ -38,8 +38,8 @@ public class TestMultiReader extends TestDirectoryReader {
|
|||
IndexReader reader;
|
||||
|
||||
sis.read(dir);
|
||||
SegmentReader reader1 = SegmentReader.get(sis.info(0), IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
SegmentReader reader2 = SegmentReader.get(sis.info(1), IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
SegmentReader reader1 = new SegmentReader(sis.info(0), IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
SegmentReader reader2 = new SegmentReader(sis.info(1), IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
readers[0] = reader1;
|
||||
readers[1] = reader2;
|
||||
assertTrue(reader1 != null);
|
||||
|
|
|
@ -24,7 +24,6 @@ import org.apache.lucene.analysis.MockAnalyzer;
|
|||
import org.apache.lucene.codecs.Codec;
|
||||
import org.apache.lucene.document.Document;
|
||||
import org.apache.lucene.document.TextField;
|
||||
import org.apache.lucene.index.IndexWriterConfig.OpenMode;
|
||||
import org.apache.lucene.store.Directory;
|
||||
import org.apache.lucene.util.BytesRef;
|
||||
import org.apache.lucene.util.InfoStream;
|
||||
|
@ -54,8 +53,8 @@ public class TestSegmentMerger extends LuceneTestCase {
|
|||
SegmentInfo info1 = DocHelper.writeDoc(random, merge1Dir, doc1);
|
||||
DocHelper.setupDoc(doc2);
|
||||
SegmentInfo info2 = DocHelper.writeDoc(random, merge2Dir, doc2);
|
||||
reader1 = SegmentReader.get(info1, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
reader2 = SegmentReader.get(info2, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
reader1 = new SegmentReader(info1, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
reader2 = new SegmentReader(info2, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -86,9 +85,9 @@ public class TestSegmentMerger extends LuceneTestCase {
|
|||
assertTrue(docsMerged == 2);
|
||||
final FieldInfos fieldInfos = mergeState.fieldInfos;
|
||||
//Should be able to open a new SegmentReader against the new directory
|
||||
SegmentReader mergedReader = SegmentReader.getRW(new SegmentInfo(mergedSegment, docsMerged, mergedDir, false,
|
||||
SegmentReader mergedReader = new SegmentReader(new SegmentInfo(mergedSegment, docsMerged, mergedDir, false,
|
||||
codec, fieldInfos),
|
||||
true, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
assertTrue(mergedReader != null);
|
||||
assertTrue(mergedReader.numDocs() == 2);
|
||||
Document newDoc1 = mergedReader.document(0);
|
||||
|
|
|
@ -41,7 +41,7 @@ public class TestSegmentReader extends LuceneTestCase {
|
|||
dir = newDirectory();
|
||||
DocHelper.setupDoc(testDoc);
|
||||
SegmentInfo info = DocHelper.writeDoc(random, dir, testDoc);
|
||||
reader = SegmentReader.get(info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, IOContext.READ);
|
||||
reader = new SegmentReader(info, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, IOContext.READ);
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -73,50 +73,6 @@ public class TestSegmentReader extends LuceneTestCase {
|
|||
}
|
||||
}
|
||||
|
||||
public void testDelete() throws IOException {
|
||||
Document docToDelete = new Document();
|
||||
DocHelper.setupDoc(docToDelete);
|
||||
SegmentInfo info = DocHelper.writeDoc(random, dir, docToDelete);
|
||||
SegmentReader deleteReader = SegmentReader.getRW(info, true, IndexReader.DEFAULT_TERMS_INDEX_DIVISOR, newIOContext(random));
|
||||
assertNotNull(deleteReader);
|
||||
assertEquals(1, deleteReader.numDocs());
|
||||
final Object combKey = deleteReader.getCombinedCoreAndDeletesKey();
|
||||
final Object coreKey = deleteReader.getCoreCacheKey();
|
||||
assertNotNull(combKey);
|
||||
assertNotNull(coreKey);
|
||||
assertNotSame(combKey, coreKey);
|
||||
|
||||
SegmentReader clone1 = (SegmentReader) deleteReader.clone();
|
||||
assertSame(coreKey, clone1.getCoreCacheKey());
|
||||
assertSame(combKey, clone1.getCombinedCoreAndDeletesKey());
|
||||
|
||||
deleteReader.deleteDocument(0);
|
||||
final Object newCombKey = deleteReader.getCombinedCoreAndDeletesKey();
|
||||
assertNotNull(newCombKey);
|
||||
assertNotSame(combKey, newCombKey);
|
||||
assertSame(coreKey, deleteReader.getCoreCacheKey());
|
||||
assertFalse(deleteReader.getLiveDocs().get(0));
|
||||
assertTrue(deleteReader.hasDeletions());
|
||||
assertTrue(deleteReader.numDocs() == 0);
|
||||
|
||||
SegmentReader clone2 = (SegmentReader) deleteReader.clone();
|
||||
assertSame(coreKey, clone2.getCoreCacheKey());
|
||||
assertSame(newCombKey, clone2.getCombinedCoreAndDeletesKey());
|
||||
assertFalse(clone2.getLiveDocs().get(0));
|
||||
assertTrue(clone2.hasDeletions());
|
||||
assertEquals(0, clone2.numDocs());
|
||||
clone2.close();
|
||||
|
||||
assertSame(coreKey, clone1.getCoreCacheKey());
|
||||
assertSame(combKey, clone1.getCombinedCoreAndDeletesKey());
|
||||
assertNull(clone1.getLiveDocs());
|
||||
assertFalse(clone1.hasDeletions());
|
||||
assertEquals(1, clone2.numDocs());
|
||||
clone1.close();
|
||||
|
||||
deleteReader.close();
|
||||
}
|
||||
|
||||
public void testGetFieldNameVariations() {
|
||||
Collection<String> result = reader.getFieldNames(IndexReader.FieldOption.ALL);
|
||||
assertTrue(result != null);
|
||||
|
|
|
@ -56,7 +56,7 @@ public class TestSegmentTermDocs extends LuceneTestCase {
|
|||
|
||||
public void testTermDocs(int indexDivisor) throws IOException {
|
||||
//After adding the document, we should be able to read it back in
|
||||
SegmentReader reader = SegmentReader.get(info, indexDivisor, newIOContext(random));
|
||||
SegmentReader reader = new SegmentReader(info, indexDivisor, newIOContext(random));
|
||||
assertTrue(reader != null);
|
||||
assertEquals(indexDivisor, reader.getTermInfosIndexDivisor());
|
||||
|
||||
|
@ -79,7 +79,7 @@ public class TestSegmentTermDocs extends LuceneTestCase {
|
|||
public void testBadSeek(int indexDivisor) throws IOException {
|
||||
{
|
||||
//After adding the document, we should be able to read it back in
|
||||
SegmentReader reader = SegmentReader.get(info, indexDivisor, newIOContext(random));
|
||||
SegmentReader reader = new SegmentReader(info, indexDivisor, newIOContext(random));
|
||||
assertTrue(reader != null);
|
||||
DocsEnum termDocs = _TestUtil.docs(random, reader,
|
||||
"textField2",
|
||||
|
@ -93,7 +93,7 @@ public class TestSegmentTermDocs extends LuceneTestCase {
|
|||
}
|
||||
{
|
||||
//After adding the document, we should be able to read it back in
|
||||
SegmentReader reader = SegmentReader.get(info, indexDivisor, newIOContext(random));
|
||||
SegmentReader reader = new SegmentReader(info, indexDivisor, newIOContext(random));
|
||||
assertTrue(reader != null);
|
||||
DocsEnum termDocs = _TestUtil.docs(random, reader,
|
||||
"junk",
|
||||
|
|
|
@ -132,6 +132,10 @@ public class TestTieredMergePolicy extends LuceneTestCase {
|
|||
assertEquals(numDocs, r.numDocs());
|
||||
r.close();
|
||||
|
||||
if (VERBOSE) {
|
||||
System.out.println("\nTEST: delete doc");
|
||||
}
|
||||
|
||||
w.deleteDocuments(new Term("id", ""+(42+17)));
|
||||
|
||||
r = w.getReader();
|
||||
|
|
Loading…
Reference in New Issue