mirror of https://github.com/apache/lucene.git
LUCENE-6299: IndexWriter was failing to enforce the 2.1 billion doc limit in one index
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1662571 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
commit
62130ae70c
|
@ -717,6 +717,9 @@ Bug fixes
|
||||||
sorted (set) doc values instance at the same time.
|
sorted (set) doc values instance at the same time.
|
||||||
(Tom Shally, Robert Muir, Adrien Grand)
|
(Tom Shally, Robert Muir, Adrien Grand)
|
||||||
|
|
||||||
|
* LUCENE-6299: IndexWriter was failing to enforce the 2.1 billion doc
|
||||||
|
limit. (Robert Muir, Mike McCandless)
|
||||||
|
|
||||||
API Changes
|
API Changes
|
||||||
|
|
||||||
* LUCENE-6212: Deprecate IndexWriter APIs that accept per-document Analyzer.
|
* LUCENE-6212: Deprecate IndexWriter APIs that accept per-document Analyzer.
|
||||||
|
|
|
@ -64,24 +64,32 @@ public abstract class BaseCompositeReader<R extends IndexReader> extends Composi
|
||||||
* cloned and not protected for modification, the subclass is responsible
|
* cloned and not protected for modification, the subclass is responsible
|
||||||
* to do this.
|
* to do this.
|
||||||
*/
|
*/
|
||||||
protected BaseCompositeReader(R[] subReaders) {
|
protected BaseCompositeReader(R[] subReaders) throws IOException {
|
||||||
this.subReaders = subReaders;
|
this.subReaders = subReaders;
|
||||||
this.subReadersList = Collections.unmodifiableList(Arrays.asList(subReaders));
|
this.subReadersList = Collections.unmodifiableList(Arrays.asList(subReaders));
|
||||||
starts = new int[subReaders.length + 1]; // build starts array
|
starts = new int[subReaders.length + 1]; // build starts array
|
||||||
int maxDoc = 0, numDocs = 0;
|
long maxDoc = 0, numDocs = 0;
|
||||||
for (int i = 0; i < subReaders.length; i++) {
|
for (int i = 0; i < subReaders.length; i++) {
|
||||||
starts[i] = maxDoc;
|
starts[i] = (int) maxDoc;
|
||||||
final IndexReader r = subReaders[i];
|
final IndexReader r = subReaders[i];
|
||||||
maxDoc += r.maxDoc(); // compute maxDocs
|
maxDoc += r.maxDoc(); // compute maxDocs
|
||||||
if (maxDoc < 0 /* overflow */ || maxDoc > IndexWriter.getActualMaxDocs()) {
|
|
||||||
throw new IllegalArgumentException("Too many documents, composite IndexReaders cannot exceed " + IndexWriter.getActualMaxDocs());
|
|
||||||
}
|
|
||||||
numDocs += r.numDocs(); // compute numDocs
|
numDocs += r.numDocs(); // compute numDocs
|
||||||
r.registerParentReader(this);
|
r.registerParentReader(this);
|
||||||
}
|
}
|
||||||
starts[subReaders.length] = maxDoc;
|
|
||||||
this.maxDoc = maxDoc;
|
if (maxDoc > IndexWriter.getActualMaxDocs()) {
|
||||||
this.numDocs = numDocs;
|
if (this instanceof DirectoryReader) {
|
||||||
|
// A single index has too many documents and it is corrupt (IndexWriter prevents this as of LUCENE-6299)
|
||||||
|
throw new CorruptIndexException("Too many documents: an index cannot exceed " + IndexWriter.getActualMaxDocs() + " but readers have total maxDoc=" + maxDoc, Arrays.toString(subReaders));
|
||||||
|
} else {
|
||||||
|
// Caller is building a MultiReader and it has too many documents; this case is just illegal arguments:
|
||||||
|
throw new IllegalArgumentException("Too many documents: composite IndexReaders cannot exceed " + IndexWriter.getActualMaxDocs() + " but readers have total maxDoc=" + maxDoc);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
this.maxDoc = Math.toIntExact(maxDoc);
|
||||||
|
starts[subReaders.length] = this.maxDoc;
|
||||||
|
this.numDocs = Math.toIntExact(numDocs);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -305,7 +305,7 @@ public abstract class DirectoryReader extends BaseCompositeReader<LeafReader> {
|
||||||
* Subclasses of {@code DirectoryReader} should take care to not allow
|
* Subclasses of {@code DirectoryReader} should take care to not allow
|
||||||
* modification of this internal array, e.g. {@link #doOpenIfChanged()}.
|
* modification of this internal array, e.g. {@link #doOpenIfChanged()}.
|
||||||
*/
|
*/
|
||||||
protected DirectoryReader(Directory directory, LeafReader[] segmentReaders) {
|
protected DirectoryReader(Directory directory, LeafReader[] segmentReaders) throws IOException {
|
||||||
super(segmentReaders);
|
super(segmentReaders);
|
||||||
this.directory = directory;
|
this.directory = directory;
|
||||||
}
|
}
|
||||||
|
|
|
@ -227,11 +227,13 @@ final class DocumentsWriter implements Closeable, Accountable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
synchronized void lockAndAbortAll(IndexWriter indexWriter) {
|
/** Returns how many documents were aborted. */
|
||||||
|
synchronized long lockAndAbortAll(IndexWriter indexWriter) {
|
||||||
assert indexWriter.holdsFullFlushLock();
|
assert indexWriter.holdsFullFlushLock();
|
||||||
if (infoStream.isEnabled("DW")) {
|
if (infoStream.isEnabled("DW")) {
|
||||||
infoStream.message("DW", "lockAndAbortAll");
|
infoStream.message("DW", "lockAndAbortAll");
|
||||||
}
|
}
|
||||||
|
long abortedDocCount = 0;
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
try {
|
try {
|
||||||
deleteQueue.clear();
|
deleteQueue.clear();
|
||||||
|
@ -239,12 +241,13 @@ final class DocumentsWriter implements Closeable, Accountable {
|
||||||
for (int i = 0; i < limit; i++) {
|
for (int i = 0; i < limit; i++) {
|
||||||
final ThreadState perThread = perThreadPool.getThreadState(i);
|
final ThreadState perThread = perThreadPool.getThreadState(i);
|
||||||
perThread.lock();
|
perThread.lock();
|
||||||
abortThreadState(perThread);
|
abortedDocCount += abortThreadState(perThread);
|
||||||
}
|
}
|
||||||
deleteQueue.clear();
|
deleteQueue.clear();
|
||||||
flushControl.abortPendingFlushes();
|
flushControl.abortPendingFlushes();
|
||||||
flushControl.waitForFlush();
|
flushControl.waitForFlush();
|
||||||
success = true;
|
success = true;
|
||||||
|
return abortedDocCount;
|
||||||
} finally {
|
} finally {
|
||||||
if (infoStream.isEnabled("DW")) {
|
if (infoStream.isEnabled("DW")) {
|
||||||
infoStream.message("DW", "finished lockAndAbortAll success=" + success);
|
infoStream.message("DW", "finished lockAndAbortAll success=" + success);
|
||||||
|
@ -256,21 +259,27 @@ final class DocumentsWriter implements Closeable, Accountable {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private final void abortThreadState(final ThreadState perThread) {
|
/** Returns how many documents were aborted. */
|
||||||
|
private final int abortThreadState(final ThreadState perThread) {
|
||||||
assert perThread.isHeldByCurrentThread();
|
assert perThread.isHeldByCurrentThread();
|
||||||
if (perThread.isActive()) { // we might be closed
|
if (perThread.isActive()) { // we might be closed
|
||||||
if (perThread.isInitialized()) {
|
if (perThread.isInitialized()) {
|
||||||
try {
|
try {
|
||||||
subtractFlushedNumDocs(perThread.dwpt.getNumDocsInRAM());
|
int abortedDocCount = perThread.dwpt.getNumDocsInRAM();
|
||||||
|
subtractFlushedNumDocs(abortedDocCount);
|
||||||
perThread.dwpt.abort();
|
perThread.dwpt.abort();
|
||||||
|
return abortedDocCount;
|
||||||
} finally {
|
} finally {
|
||||||
flushControl.doOnAbort(perThread);
|
flushControl.doOnAbort(perThread);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
flushControl.doOnAbort(perThread);
|
flushControl.doOnAbort(perThread);
|
||||||
|
// This DWPT was never initialized so it has no indexed documents:
|
||||||
|
return 0;
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
assert closed;
|
assert closed;
|
||||||
|
return 0;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -201,17 +201,18 @@ class DocumentsWriterPerThread {
|
||||||
|
|
||||||
/** Anything that will add N docs to the index should reserve first to
|
/** Anything that will add N docs to the index should reserve first to
|
||||||
* make sure it's allowed. */
|
* make sure it's allowed. */
|
||||||
private void reserveDoc() {
|
private void reserveOneDoc() {
|
||||||
if (pendingNumDocs.incrementAndGet() > IndexWriter.getActualMaxDocs()) {
|
if (pendingNumDocs.incrementAndGet() > IndexWriter.getActualMaxDocs()) {
|
||||||
// Reserve failed
|
// Reserve failed: put the one doc back and throw exc:
|
||||||
pendingNumDocs.decrementAndGet();
|
pendingNumDocs.decrementAndGet();
|
||||||
throw new IllegalStateException("number of documents in the index cannot exceed " + IndexWriter.getActualMaxDocs());
|
throw new IllegalArgumentException("number of documents in the index cannot exceed " + IndexWriter.getActualMaxDocs());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public void updateDocument(IndexDocument doc, Analyzer analyzer, Term delTerm) throws IOException, AbortingException {
|
public void updateDocument(IndexDocument doc, Analyzer analyzer, Term delTerm) throws IOException, AbortingException {
|
||||||
testPoint("DocumentsWriterPerThread addDocument start");
|
testPoint("DocumentsWriterPerThread addDocument start");
|
||||||
assert deleteQueue != null;
|
assert deleteQueue != null;
|
||||||
|
reserveOneDoc();
|
||||||
docState.doc = doc;
|
docState.doc = doc;
|
||||||
docState.analyzer = analyzer;
|
docState.analyzer = analyzer;
|
||||||
docState.docID = numDocsInRAM;
|
docState.docID = numDocsInRAM;
|
||||||
|
@ -224,7 +225,6 @@ class DocumentsWriterPerThread {
|
||||||
// document, so the counter will be "wrong" in that case, but
|
// document, so the counter will be "wrong" in that case, but
|
||||||
// it's very hard to fix (we can't easily distinguish aborting
|
// it's very hard to fix (we can't easily distinguish aborting
|
||||||
// vs non-aborting exceptions):
|
// vs non-aborting exceptions):
|
||||||
reserveDoc();
|
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
try {
|
try {
|
||||||
try {
|
try {
|
||||||
|
@ -261,7 +261,7 @@ class DocumentsWriterPerThread {
|
||||||
// document, so the counter will be "wrong" in that case, but
|
// document, so the counter will be "wrong" in that case, but
|
||||||
// it's very hard to fix (we can't easily distinguish aborting
|
// it's very hard to fix (we can't easily distinguish aborting
|
||||||
// vs non-aborting exceptions):
|
// vs non-aborting exceptions):
|
||||||
reserveDoc();
|
reserveOneDoc();
|
||||||
docState.doc = doc;
|
docState.doc = doc;
|
||||||
docState.docID = numDocsInRAM;
|
docState.docID = numDocsInRAM;
|
||||||
docCount++;
|
docCount++;
|
||||||
|
|
|
@ -185,13 +185,13 @@ public class ExitableDirectoryReader extends FilterDirectoryReader {
|
||||||
* @param in DirectoryReader that this ExitableDirectoryReader wraps around to make it Exitable.
|
* @param in DirectoryReader that this ExitableDirectoryReader wraps around to make it Exitable.
|
||||||
* @param queryTimeout The object to periodically check if the query should time out.
|
* @param queryTimeout The object to periodically check if the query should time out.
|
||||||
*/
|
*/
|
||||||
public ExitableDirectoryReader(DirectoryReader in, QueryTimeout queryTimeout) {
|
public ExitableDirectoryReader(DirectoryReader in, QueryTimeout queryTimeout) throws IOException {
|
||||||
super(in, new ExitableSubReaderWrapper(queryTimeout));
|
super(in, new ExitableSubReaderWrapper(queryTimeout));
|
||||||
this.queryTimeout = queryTimeout;
|
this.queryTimeout = queryTimeout;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
|
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
|
||||||
return new ExitableDirectoryReader(in, queryTimeout);
|
return new ExitableDirectoryReader(in, queryTimeout);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -200,7 +200,7 @@ public class ExitableDirectoryReader extends FilterDirectoryReader {
|
||||||
* can be used normally (e.g. passed to {@link DirectoryReader#openIfChanged(DirectoryReader)})
|
* can be used normally (e.g. passed to {@link DirectoryReader#openIfChanged(DirectoryReader)})
|
||||||
* and so on.
|
* and so on.
|
||||||
*/
|
*/
|
||||||
public static DirectoryReader wrap(DirectoryReader in, QueryTimeout queryTimeout) {
|
public static DirectoryReader wrap(DirectoryReader in, QueryTimeout queryTimeout) throws IOException {
|
||||||
return new ExitableDirectoryReader(in, queryTimeout);
|
return new ExitableDirectoryReader(in, queryTimeout);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -79,7 +79,7 @@ public abstract class FilterDirectoryReader extends DirectoryReader {
|
||||||
* @param in the DirectoryReader to filter
|
* @param in the DirectoryReader to filter
|
||||||
* @param wrapper the SubReaderWrapper to use to wrap subreaders
|
* @param wrapper the SubReaderWrapper to use to wrap subreaders
|
||||||
*/
|
*/
|
||||||
public FilterDirectoryReader(DirectoryReader in, SubReaderWrapper wrapper) {
|
public FilterDirectoryReader(DirectoryReader in, SubReaderWrapper wrapper) throws IOException {
|
||||||
super(in.directory(), wrapper.wrap(in.getSequentialSubReaders()));
|
super(in.directory(), wrapper.wrap(in.getSequentialSubReaders()));
|
||||||
this.in = in;
|
this.in = in;
|
||||||
}
|
}
|
||||||
|
@ -93,9 +93,9 @@ public abstract class FilterDirectoryReader extends DirectoryReader {
|
||||||
* @param in the DirectoryReader to wrap
|
* @param in the DirectoryReader to wrap
|
||||||
* @return the wrapped DirectoryReader
|
* @return the wrapped DirectoryReader
|
||||||
*/
|
*/
|
||||||
protected abstract DirectoryReader doWrapDirectoryReader(DirectoryReader in);
|
protected abstract DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException;
|
||||||
|
|
||||||
private final DirectoryReader wrapDirectoryReader(DirectoryReader in) {
|
private final DirectoryReader wrapDirectoryReader(DirectoryReader in) throws IOException {
|
||||||
return in == null ? null : doWrapDirectoryReader(in);
|
return in == null ? null : doWrapDirectoryReader(in);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -196,7 +196,7 @@ import org.apache.lucene.util.Version;
|
||||||
public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
|
|
||||||
/** Hard limit on maximum number of documents that may be added to the
|
/** Hard limit on maximum number of documents that may be added to the
|
||||||
* index. If you try to add more than this you'll hit {@code IllegalStateException}. */
|
* index. If you try to add more than this you'll hit {@code IllegalArgumentException}. */
|
||||||
// We defensively subtract 128 to be well below the lowest
|
// We defensively subtract 128 to be well below the lowest
|
||||||
// ArrayUtil.MAX_ARRAY_LENGTH on "typical" JVMs. We don't just use
|
// ArrayUtil.MAX_ARRAY_LENGTH on "typical" JVMs. We don't just use
|
||||||
// ArrayUtil.MAX_ARRAY_LENGTH here because this can vary across JVMs:
|
// ArrayUtil.MAX_ARRAY_LENGTH here because this can vary across JVMs:
|
||||||
|
@ -839,6 +839,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
}
|
}
|
||||||
|
|
||||||
rollbackSegments = segmentInfos.createBackupSegmentInfos();
|
rollbackSegments = segmentInfos.createBackupSegmentInfos();
|
||||||
|
pendingNumDocs.set(segmentInfos.totalDocCount());
|
||||||
|
|
||||||
// start with previous field numbers, but new FieldInfos
|
// start with previous field numbers, but new FieldInfos
|
||||||
globalFieldNumberMap = getFieldNumberMap();
|
globalFieldNumberMap = getFieldNumberMap();
|
||||||
|
@ -2055,13 +2056,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
*/
|
*/
|
||||||
try {
|
try {
|
||||||
synchronized (fullFlushLock) {
|
synchronized (fullFlushLock) {
|
||||||
docWriter.lockAndAbortAll(this);
|
long abortedDocCount = docWriter.lockAndAbortAll(this);
|
||||||
|
pendingNumDocs.addAndGet(-abortedDocCount);
|
||||||
|
|
||||||
processEvents(false, true);
|
processEvents(false, true);
|
||||||
synchronized (this) {
|
synchronized (this) {
|
||||||
try {
|
try {
|
||||||
// Abort any running merges
|
// Abort any running merges
|
||||||
abortMerges();
|
abortMerges();
|
||||||
// Remove all segments
|
// Remove all segments
|
||||||
|
pendingNumDocs.addAndGet(-segmentInfos.totalDocCount());
|
||||||
segmentInfos.clear();
|
segmentInfos.clear();
|
||||||
// Ask deleter to locate unreferenced files & remove them:
|
// Ask deleter to locate unreferenced files & remove them:
|
||||||
deleter.checkpoint(segmentInfos, false);
|
deleter.checkpoint(segmentInfos, false);
|
||||||
|
@ -2077,6 +2081,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
++changeCount;
|
++changeCount;
|
||||||
segmentInfos.changed();
|
segmentInfos.changed();
|
||||||
globalFieldNumberMap.clear();
|
globalFieldNumberMap.clear();
|
||||||
|
|
||||||
success = true;
|
success = true;
|
||||||
} finally {
|
} finally {
|
||||||
docWriter.unlockAllAfterAbortAll(this);
|
docWriter.unlockAllAfterAbortAll(this);
|
||||||
|
@ -2317,6 +2322,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
* @throws IOException if there is a low-level IO error
|
* @throws IOException if there is a low-level IO error
|
||||||
* @throws LockObtainFailedException if we were unable to
|
* @throws LockObtainFailedException if we were unable to
|
||||||
* acquire the write lock in at least one directory
|
* acquire the write lock in at least one directory
|
||||||
|
* @throws IllegalArgumentException if addIndexes would cause
|
||||||
|
* the index to exceed {@link #MAX_DOCS}
|
||||||
*/
|
*/
|
||||||
public void addIndexes(Directory... dirs) throws IOException {
|
public void addIndexes(Directory... dirs) throws IOException {
|
||||||
ensureOpen();
|
ensureOpen();
|
||||||
|
@ -2335,16 +2342,25 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
flush(false, true);
|
flush(false, true);
|
||||||
|
|
||||||
List<SegmentCommitInfo> infos = new ArrayList<>();
|
List<SegmentCommitInfo> infos = new ArrayList<>();
|
||||||
int totalDocCount = 0;
|
|
||||||
boolean success = false;
|
// long so we can detect int overflow:
|
||||||
try {
|
long totalDocCount = 0;
|
||||||
|
List<SegmentInfos> commits = new ArrayList<>(dirs.length);
|
||||||
for (Directory dir : dirs) {
|
for (Directory dir : dirs) {
|
||||||
if (infoStream.isEnabled("IW")) {
|
if (infoStream.isEnabled("IW")) {
|
||||||
infoStream.message("IW", "addIndexes: process directory " + dir);
|
infoStream.message("IW", "addIndexes: process directory " + dir);
|
||||||
}
|
}
|
||||||
SegmentInfos sis = SegmentInfos.readLatestCommit(dir); // read infos from dir
|
SegmentInfos sis = SegmentInfos.readLatestCommit(dir); // read infos from dir
|
||||||
totalDocCount += sis.totalDocCount();
|
totalDocCount += sis.totalDocCount();
|
||||||
|
commits.add(sis);
|
||||||
|
}
|
||||||
|
|
||||||
|
// Best-effort up front check:
|
||||||
|
testReserveDocs(totalDocCount);
|
||||||
|
|
||||||
|
boolean success = false;
|
||||||
|
try {
|
||||||
|
for (SegmentInfos sis : commits) {
|
||||||
for (SegmentCommitInfo info : sis) {
|
for (SegmentCommitInfo info : sis) {
|
||||||
assert !infos.contains(info): "dup info dir=" + info.info.dir + " name=" + info.info.name;
|
assert !infos.contains(info): "dup info dir=" + info.info.dir + " name=" + info.info.name;
|
||||||
|
|
||||||
|
@ -2367,12 +2383,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
} finally {
|
} finally {
|
||||||
if (!success) {
|
if (!success) {
|
||||||
for(SegmentCommitInfo sipc : infos) {
|
for(SegmentCommitInfo sipc : infos) {
|
||||||
for(String file : sipc.files()) {
|
IOUtils.deleteFilesIgnoringExceptions(directory, sipc.files().toArray(new String[0]));
|
||||||
try {
|
|
||||||
directory.deleteFile(file);
|
|
||||||
} catch (Throwable t) {
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -2381,9 +2392,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
success = false;
|
success = false;
|
||||||
try {
|
try {
|
||||||
ensureOpen();
|
ensureOpen();
|
||||||
// Make sure adding the new documents to this index won't
|
|
||||||
// exceed the limit:
|
// Now reserve the docs, just before we update SIS:
|
||||||
reserveDocs(totalDocCount);
|
reserveDocs(totalDocCount);
|
||||||
|
|
||||||
success = true;
|
success = true;
|
||||||
} finally {
|
} finally {
|
||||||
if (!success) {
|
if (!success) {
|
||||||
|
@ -2441,10 +2453,14 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
* if the index is corrupt
|
* if the index is corrupt
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
* if there is a low-level IO error
|
* if there is a low-level IO error
|
||||||
|
* @throws IllegalArgumentException
|
||||||
|
* if addIndexes would cause the index to exceed {@link #MAX_DOCS}
|
||||||
*/
|
*/
|
||||||
public void addIndexes(CodecReader... readers) throws IOException {
|
public void addIndexes(CodecReader... readers) throws IOException {
|
||||||
ensureOpen();
|
ensureOpen();
|
||||||
int numDocs = 0;
|
|
||||||
|
// long so we can detect int overflow:
|
||||||
|
long numDocs = 0;
|
||||||
|
|
||||||
try {
|
try {
|
||||||
if (infoStream.isEnabled("IW")) {
|
if (infoStream.isEnabled("IW")) {
|
||||||
|
@ -2457,11 +2473,10 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
numDocs += leaf.numDocs();
|
numDocs += leaf.numDocs();
|
||||||
}
|
}
|
||||||
|
|
||||||
// Make sure adding the new documents to this index won't
|
// Best-effort up front check:
|
||||||
// exceed the limit:
|
testReserveDocs(numDocs);
|
||||||
reserveDocs(numDocs);
|
|
||||||
|
|
||||||
final IOContext context = new IOContext(new MergeInfo(numDocs, -1, false, -1));
|
final IOContext context = new IOContext(new MergeInfo(Math.toIntExact(numDocs), -1, false, -1));
|
||||||
|
|
||||||
// TODO: somehow we should fix this merge so it's
|
// TODO: somehow we should fix this merge so it's
|
||||||
// abortable so that IW.close(false) is able to stop it
|
// abortable so that IW.close(false) is able to stop it
|
||||||
|
@ -2553,11 +2568,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
return;
|
return;
|
||||||
}
|
}
|
||||||
ensureOpen();
|
ensureOpen();
|
||||||
|
|
||||||
|
// Now reserve the docs, just before we update SIS:
|
||||||
|
reserveDocs(numDocs);
|
||||||
|
|
||||||
segmentInfos.add(infoPerCommit);
|
segmentInfos.add(infoPerCommit);
|
||||||
checkpoint();
|
checkpoint();
|
||||||
}
|
}
|
||||||
} catch (OutOfMemoryError oom) {
|
} catch (OutOfMemoryError oom) {
|
||||||
tragicEvent(oom, "addIndexes(IndexReader...)");
|
tragicEvent(oom, "addIndexes(CodecReader...)");
|
||||||
}
|
}
|
||||||
maybeMerge();
|
maybeMerge();
|
||||||
}
|
}
|
||||||
|
@ -4616,15 +4635,31 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
|
||||||
|
|
||||||
/** Anything that will add N docs to the index should reserve first to
|
/** Anything that will add N docs to the index should reserve first to
|
||||||
* make sure it's allowed. This will throw {@code
|
* make sure it's allowed. This will throw {@code
|
||||||
* IllegalStateException} if it's not allowed. */
|
* IllegalArgumentException} if it's not allowed. */
|
||||||
private void reserveDocs(int numDocs) {
|
private void reserveDocs(long addedNumDocs) {
|
||||||
if (pendingNumDocs.addAndGet(numDocs) > actualMaxDocs) {
|
assert addedNumDocs >= 0;
|
||||||
// Reserve failed
|
if (pendingNumDocs.addAndGet(addedNumDocs) > actualMaxDocs) {
|
||||||
pendingNumDocs.addAndGet(-numDocs);
|
// Reserve failed: put the docs back and throw exc:
|
||||||
throw new IllegalStateException("number of documents in the index cannot exceed " + actualMaxDocs);
|
pendingNumDocs.addAndGet(-addedNumDocs);
|
||||||
|
tooManyDocs(addedNumDocs);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/** Does a best-effort check, that the current index would accept this many additional docs, but does not actually reserve them.
|
||||||
|
*
|
||||||
|
* @throws IllegalArgumentException if there would be too many docs */
|
||||||
|
private void testReserveDocs(long addedNumDocs) {
|
||||||
|
assert addedNumDocs >= 0;
|
||||||
|
if (pendingNumDocs.get() + addedNumDocs > actualMaxDocs) {
|
||||||
|
tooManyDocs(addedNumDocs);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
private void tooManyDocs(long addedNumDocs) {
|
||||||
|
assert addedNumDocs >= 0;
|
||||||
|
throw new IllegalArgumentException("number of documents in the index cannot exceed " + actualMaxDocs + " (current document count is " + pendingNumDocs.get() + "; added numDocs is " + addedNumDocs + ")");
|
||||||
|
}
|
||||||
|
|
||||||
/** Wraps the incoming {@link Directory} so that we assign a per-thread
|
/** Wraps the incoming {@link Directory} so that we assign a per-thread
|
||||||
* {@link MergeRateLimiter} to all created {@link IndexOutput}s. */
|
* {@link MergeRateLimiter} to all created {@link IndexOutput}s. */
|
||||||
private Directory addMergeRateLimiters(Directory in) {
|
private Directory addMergeRateLimiters(Directory in) {
|
||||||
|
|
|
@ -45,7 +45,7 @@ public class MultiReader extends BaseCompositeReader<IndexReader> {
|
||||||
* <p>Note that all subreaders are closed if this Multireader is closed.</p>
|
* <p>Note that all subreaders are closed if this Multireader is closed.</p>
|
||||||
* @param subReaders set of (sub)readers
|
* @param subReaders set of (sub)readers
|
||||||
*/
|
*/
|
||||||
public MultiReader(IndexReader... subReaders) {
|
public MultiReader(IndexReader... subReaders) throws IOException {
|
||||||
this(subReaders, true);
|
this(subReaders, true);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -55,7 +55,7 @@ public class MultiReader extends BaseCompositeReader<IndexReader> {
|
||||||
* @param closeSubReaders indicates whether the subreaders should be closed
|
* @param closeSubReaders indicates whether the subreaders should be closed
|
||||||
* when this MultiReader is closed
|
* when this MultiReader is closed
|
||||||
*/
|
*/
|
||||||
public MultiReader(IndexReader[] subReaders, boolean closeSubReaders) {
|
public MultiReader(IndexReader[] subReaders, boolean closeSubReaders) throws IOException {
|
||||||
super(subReaders.clone());
|
super(subReaders.clone());
|
||||||
this.closeSubReaders = closeSubReaders;
|
this.closeSubReaders = closeSubReaders;
|
||||||
if (!closeSubReaders) {
|
if (!closeSubReaders) {
|
||||||
|
|
|
@ -284,6 +284,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
||||||
if (numSegments < 0) {
|
if (numSegments < 0) {
|
||||||
throw new CorruptIndexException("invalid segment count: " + numSegments, input);
|
throw new CorruptIndexException("invalid segment count: " + numSegments, input);
|
||||||
}
|
}
|
||||||
|
long totalDocs = 0;
|
||||||
for (int seg = 0; seg < numSegments; seg++) {
|
for (int seg = 0; seg < numSegments; seg++) {
|
||||||
String segName = input.readString();
|
String segName = input.readString();
|
||||||
final byte segmentID[];
|
final byte segmentID[];
|
||||||
|
@ -297,6 +298,7 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
||||||
Codec codec = Codec.forName(input.readString());
|
Codec codec = Codec.forName(input.readString());
|
||||||
SegmentInfo info = codec.segmentInfoFormat().read(directory, segName, segmentID, IOContext.READ);
|
SegmentInfo info = codec.segmentInfoFormat().read(directory, segName, segmentID, IOContext.READ);
|
||||||
info.setCodec(codec);
|
info.setCodec(codec);
|
||||||
|
totalDocs += info.getDocCount();
|
||||||
long delGen = input.readLong();
|
long delGen = input.readLong();
|
||||||
int delCount = input.readInt();
|
int delCount = input.readInt();
|
||||||
if (delCount < 0 || delCount > info.getDocCount()) {
|
if (delCount < 0 || delCount > info.getDocCount()) {
|
||||||
|
@ -323,6 +325,11 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
||||||
|
|
||||||
CodecUtil.checkFooter(input);
|
CodecUtil.checkFooter(input);
|
||||||
|
|
||||||
|
// LUCENE-6299: check we are in bounds
|
||||||
|
if (totalDocs > IndexWriter.getActualMaxDocs()) {
|
||||||
|
throw new CorruptIndexException("Too many documents: an index cannot exceed " + IndexWriter.getActualMaxDocs() + " but readers have total maxDoc=" + totalDocs, input);
|
||||||
|
}
|
||||||
|
|
||||||
return infos;
|
return infos;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -722,11 +729,13 @@ public final class SegmentInfos implements Cloneable, Iterable<SegmentCommitInfo
|
||||||
/** Returns sum of all segment's docCounts. Note that
|
/** Returns sum of all segment's docCounts. Note that
|
||||||
* this does not include deletions */
|
* this does not include deletions */
|
||||||
public int totalDocCount() {
|
public int totalDocCount() {
|
||||||
int count = 0;
|
long count = 0;
|
||||||
for(SegmentCommitInfo info : this) {
|
for(SegmentCommitInfo info : this) {
|
||||||
count += info.info.getDocCount();
|
count += info.info.getDocCount();
|
||||||
}
|
}
|
||||||
return count;
|
// we should never hit this, checks should happen elsewhere...
|
||||||
|
assert count <= IndexWriter.getActualMaxDocs();
|
||||||
|
return Math.toIntExact(count);
|
||||||
}
|
}
|
||||||
|
|
||||||
/** Call this before committing if changes have been made to the
|
/** Call this before committing if changes have been made to the
|
||||||
|
|
|
@ -38,7 +38,7 @@ final class StandardDirectoryReader extends DirectoryReader {
|
||||||
|
|
||||||
/** called only from static open() methods */
|
/** called only from static open() methods */
|
||||||
StandardDirectoryReader(Directory directory, LeafReader[] readers, IndexWriter writer,
|
StandardDirectoryReader(Directory directory, LeafReader[] readers, IndexWriter writer,
|
||||||
SegmentInfos sis, boolean applyAllDeletes) {
|
SegmentInfos sis, boolean applyAllDeletes) throws IOException {
|
||||||
super(directory, readers);
|
super(directory, readers);
|
||||||
this.writer = writer;
|
this.writer = writer;
|
||||||
this.segmentInfos = sis;
|
this.segmentInfos = sis;
|
||||||
|
@ -52,19 +52,24 @@ final class StandardDirectoryReader extends DirectoryReader {
|
||||||
protected DirectoryReader doBody(String segmentFileName) throws IOException {
|
protected DirectoryReader doBody(String segmentFileName) throws IOException {
|
||||||
SegmentInfos sis = SegmentInfos.readCommit(directory, segmentFileName);
|
SegmentInfos sis = SegmentInfos.readCommit(directory, segmentFileName);
|
||||||
final SegmentReader[] readers = new SegmentReader[sis.size()];
|
final SegmentReader[] readers = new SegmentReader[sis.size()];
|
||||||
for (int i = sis.size()-1; i >= 0; i--) {
|
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
try {
|
try {
|
||||||
|
for (int i = sis.size()-1; i >= 0; i--) {
|
||||||
readers[i] = new SegmentReader(sis.info(i), IOContext.READ);
|
readers[i] = new SegmentReader(sis.info(i), IOContext.READ);
|
||||||
|
}
|
||||||
|
|
||||||
|
// This may throw CorruptIndexException if there are too many docs, so
|
||||||
|
// it must be inside try clause so we close readers in that case:
|
||||||
|
DirectoryReader reader = new StandardDirectoryReader(directory, readers, null, sis, false);
|
||||||
success = true;
|
success = true;
|
||||||
|
|
||||||
|
return reader;
|
||||||
} finally {
|
} finally {
|
||||||
if (!success) {
|
if (success == false) {
|
||||||
IOUtils.closeWhileHandlingException(readers);
|
IOUtils.closeWhileHandlingException(readers);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
return new StandardDirectoryReader(directory, readers, null, sis, false);
|
|
||||||
}
|
|
||||||
}.run(commit);
|
}.run(commit);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -191,7 +191,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
private class ParallelLeafDirectoryReader extends FilterDirectoryReader {
|
private class ParallelLeafDirectoryReader extends FilterDirectoryReader {
|
||||||
public ParallelLeafDirectoryReader(DirectoryReader in) {
|
public ParallelLeafDirectoryReader(DirectoryReader in) throws IOException {
|
||||||
super(in, new FilterDirectoryReader.SubReaderWrapper() {
|
super(in, new FilterDirectoryReader.SubReaderWrapper() {
|
||||||
final long currentSchemaGen = getCurrentSchemaGen();
|
final long currentSchemaGen = getCurrentSchemaGen();
|
||||||
@Override
|
@Override
|
||||||
|
@ -207,7 +207,7 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
|
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
|
||||||
return new ParallelLeafDirectoryReader(in);
|
return new ParallelLeafDirectoryReader(in);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -17,8 +17,10 @@ package org.apache.lucene.index;
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
import java.util.concurrent.CountDownLatch;
|
||||||
|
|
||||||
import org.apache.lucene.document.Document;
|
import org.apache.lucene.document.Document;
|
||||||
import org.apache.lucene.document.Field;
|
import org.apache.lucene.document.Field;
|
||||||
|
@ -28,11 +30,13 @@ import org.apache.lucene.search.SortField;
|
||||||
import org.apache.lucene.search.TermQuery;
|
import org.apache.lucene.search.TermQuery;
|
||||||
import org.apache.lucene.search.TopDocs;
|
import org.apache.lucene.search.TopDocs;
|
||||||
import org.apache.lucene.store.Directory;
|
import org.apache.lucene.store.Directory;
|
||||||
|
import org.apache.lucene.store.FilterDirectory;
|
||||||
|
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||||
|
import org.apache.lucene.store.NoLockFactory;
|
||||||
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
import org.apache.lucene.util.LuceneTestCase.SuppressCodecs;
|
||||||
import org.apache.lucene.util.LuceneTestCase;
|
import org.apache.lucene.util.LuceneTestCase;
|
||||||
import org.apache.lucene.util.TestUtil;
|
import org.apache.lucene.util.TestUtil;
|
||||||
import org.apache.lucene.util.TimeUnits;
|
import org.apache.lucene.util.TimeUnits;
|
||||||
|
|
||||||
import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
|
import com.carrotsearch.randomizedtesting.annotations.TimeoutSuite;
|
||||||
|
|
||||||
@SuppressCodecs({ "SimpleText", "Memory", "Direct" })
|
@SuppressCodecs({ "SimpleText", "Memory", "Direct" })
|
||||||
|
@ -93,7 +97,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
|
||||||
try {
|
try {
|
||||||
w.addDocument(new Document());
|
w.addDocument(new Document());
|
||||||
fail("didn't hit exception");
|
fail("didn't hit exception");
|
||||||
} catch (IllegalStateException ise) {
|
} catch (IllegalArgumentException iae) {
|
||||||
// expected
|
// expected
|
||||||
}
|
}
|
||||||
w.close();
|
w.close();
|
||||||
|
@ -116,7 +120,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
|
||||||
try {
|
try {
|
||||||
w.addDocuments(Collections.singletonList(new Document()));
|
w.addDocuments(Collections.singletonList(new Document()));
|
||||||
fail("didn't hit exception");
|
fail("didn't hit exception");
|
||||||
} catch (IllegalStateException ise) {
|
} catch (IllegalArgumentException iae) {
|
||||||
// expected
|
// expected
|
||||||
}
|
}
|
||||||
w.close();
|
w.close();
|
||||||
|
@ -139,7 +143,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
|
||||||
try {
|
try {
|
||||||
w.updateDocument(new Term("field", "foo"), new Document());
|
w.updateDocument(new Term("field", "foo"), new Document());
|
||||||
fail("didn't hit exception");
|
fail("didn't hit exception");
|
||||||
} catch (IllegalStateException ise) {
|
} catch (IllegalArgumentException iae) {
|
||||||
// expected
|
// expected
|
||||||
}
|
}
|
||||||
w.close();
|
w.close();
|
||||||
|
@ -162,7 +166,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
|
||||||
try {
|
try {
|
||||||
w.updateDocuments(new Term("field", "foo"), Collections.singletonList(new Document()));
|
w.updateDocuments(new Term("field", "foo"), Collections.singletonList(new Document()));
|
||||||
fail("didn't hit exception");
|
fail("didn't hit exception");
|
||||||
} catch (IllegalStateException ise) {
|
} catch (IllegalArgumentException iae) {
|
||||||
// expected
|
// expected
|
||||||
}
|
}
|
||||||
w.close();
|
w.close();
|
||||||
|
@ -201,7 +205,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
|
||||||
try {
|
try {
|
||||||
w.addDocument(new Document());
|
w.addDocument(new Document());
|
||||||
fail("didn't hit exception");
|
fail("didn't hit exception");
|
||||||
} catch (IllegalStateException ise) {
|
} catch (IllegalArgumentException iae) {
|
||||||
// expected
|
// expected
|
||||||
}
|
}
|
||||||
w.close();
|
w.close();
|
||||||
|
@ -247,7 +251,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
|
||||||
try {
|
try {
|
||||||
w.addDocument(new Document());
|
w.addDocument(new Document());
|
||||||
fail("didn't hit exception");
|
fail("didn't hit exception");
|
||||||
} catch (IllegalStateException ise) {
|
} catch (IllegalArgumentException iae) {
|
||||||
// expected
|
// expected
|
||||||
}
|
}
|
||||||
w.close();
|
w.close();
|
||||||
|
@ -273,7 +277,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
|
||||||
try {
|
try {
|
||||||
w2.addIndexes(new Directory[] {dir});
|
w2.addIndexes(new Directory[] {dir});
|
||||||
fail("didn't hit exception");
|
fail("didn't hit exception");
|
||||||
} catch (IllegalStateException ise) {
|
} catch (IllegalArgumentException iae) {
|
||||||
// expected
|
// expected
|
||||||
}
|
}
|
||||||
assertEquals(1, w2.maxDoc());
|
assertEquals(1, w2.maxDoc());
|
||||||
|
@ -281,7 +285,7 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
|
||||||
try {
|
try {
|
||||||
TestUtil.addIndexesSlowly(w2, ir);
|
TestUtil.addIndexesSlowly(w2, ir);
|
||||||
fail("didn't hit exception");
|
fail("didn't hit exception");
|
||||||
} catch (IllegalStateException ise) {
|
} catch (IllegalArgumentException iae) {
|
||||||
// expected
|
// expected
|
||||||
}
|
}
|
||||||
w2.close();
|
w2.close();
|
||||||
|
@ -370,6 +374,103 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
|
||||||
dir2.close();
|
dir2.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* LUCENE-6299: Test if addindexes(Dir[]) prevents exceeding max docs.
|
||||||
|
*/
|
||||||
|
public void testAddTooManyIndexesDir() throws Exception {
|
||||||
|
// we cheat and add the same one over again... IW wants a write lock on each
|
||||||
|
Directory dir = newDirectory(random(), NoLockFactory.INSTANCE);
|
||||||
|
Document doc = new Document();
|
||||||
|
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
|
||||||
|
for (int i = 0; i < 100000; i++) {
|
||||||
|
w.addDocument(doc);
|
||||||
|
}
|
||||||
|
w.forceMerge(1);
|
||||||
|
w.commit();
|
||||||
|
w.close();
|
||||||
|
|
||||||
|
// wrap this with disk full, so test fails faster and doesn't fill up real disks.
|
||||||
|
MockDirectoryWrapper dir2 = newMockDirectory();
|
||||||
|
w = new IndexWriter(dir2, new IndexWriterConfig(null));
|
||||||
|
w.commit(); // don't confuse checkindex
|
||||||
|
dir2.setMaxSizeInBytes(dir2.sizeInBytes() + 65536); // 64KB
|
||||||
|
Directory dirs[] = new Directory[1 + (IndexWriter.MAX_DOCS / 100000)];
|
||||||
|
for (int i = 0; i < dirs.length; i++) {
|
||||||
|
// bypass iw check for duplicate dirs
|
||||||
|
dirs[i] = new FilterDirectory(dir) {};
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
w.addIndexes(dirs);
|
||||||
|
fail("didn't get expected exception");
|
||||||
|
} catch (IllegalArgumentException expected) {
|
||||||
|
// pass
|
||||||
|
} catch (IOException fakeDiskFull) {
|
||||||
|
final Exception e;
|
||||||
|
if (fakeDiskFull.getMessage() != null && fakeDiskFull.getMessage().startsWith("fake disk full")) {
|
||||||
|
e = new RuntimeException("test failed: IW checks aren't working and we are executing addIndexes");
|
||||||
|
e.addSuppressed(fakeDiskFull);
|
||||||
|
} else {
|
||||||
|
e = fakeDiskFull;
|
||||||
|
}
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
|
||||||
|
w.close();
|
||||||
|
dir.close();
|
||||||
|
dir2.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* LUCENE-6299: Test if addindexes(CodecReader[]) prevents exceeding max docs.
|
||||||
|
*/
|
||||||
|
public void testAddTooManyIndexesCodecReader() throws Exception {
|
||||||
|
// we cheat and add the same one over again... IW wants a write lock on each
|
||||||
|
Directory dir = newDirectory(random(), NoLockFactory.INSTANCE);
|
||||||
|
Document doc = new Document();
|
||||||
|
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
|
||||||
|
for (int i = 0; i < 100000; i++) {
|
||||||
|
w.addDocument(doc);
|
||||||
|
}
|
||||||
|
w.forceMerge(1);
|
||||||
|
w.commit();
|
||||||
|
w.close();
|
||||||
|
|
||||||
|
// wrap this with disk full, so test fails faster and doesn't fill up real disks.
|
||||||
|
MockDirectoryWrapper dir2 = newMockDirectory();
|
||||||
|
w = new IndexWriter(dir2, new IndexWriterConfig(null));
|
||||||
|
w.commit(); // don't confuse checkindex
|
||||||
|
dir2.setMaxSizeInBytes(dir2.sizeInBytes() + 65536); // 64KB
|
||||||
|
IndexReader r = DirectoryReader.open(dir);
|
||||||
|
CodecReader segReader = (CodecReader) r.leaves().get(0).reader();
|
||||||
|
|
||||||
|
CodecReader readers[] = new CodecReader[1 + (IndexWriter.MAX_DOCS / 100000)];
|
||||||
|
for (int i = 0; i < readers.length; i++) {
|
||||||
|
readers[i] = segReader;
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
w.addIndexes(readers);
|
||||||
|
fail("didn't get expected exception");
|
||||||
|
} catch (IllegalArgumentException expected) {
|
||||||
|
// pass
|
||||||
|
} catch (IOException fakeDiskFull) {
|
||||||
|
final Exception e;
|
||||||
|
if (fakeDiskFull.getMessage() != null && fakeDiskFull.getMessage().startsWith("fake disk full")) {
|
||||||
|
e = new RuntimeException("test failed: IW checks aren't working and we are executing addIndexes");
|
||||||
|
e.addSuppressed(fakeDiskFull);
|
||||||
|
} else {
|
||||||
|
e = fakeDiskFull;
|
||||||
|
}
|
||||||
|
throw e;
|
||||||
|
}
|
||||||
|
|
||||||
|
r.close();
|
||||||
|
w.close();
|
||||||
|
dir.close();
|
||||||
|
dir2.close();
|
||||||
|
}
|
||||||
|
|
||||||
public void testTooLargeMaxDocs() throws Exception {
|
public void testTooLargeMaxDocs() throws Exception {
|
||||||
try {
|
try {
|
||||||
IndexWriter.setMaxDocs(Integer.MAX_VALUE);
|
IndexWriter.setMaxDocs(Integer.MAX_VALUE);
|
||||||
|
@ -378,4 +479,244 @@ public class TestIndexWriterMaxDocs extends LuceneTestCase {
|
||||||
// expected
|
// expected
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// LUCENE-6299
|
||||||
|
public void testDeleteAll() throws Exception {
|
||||||
|
setIndexWriterMaxDocs(1);
|
||||||
|
try {
|
||||||
|
Directory dir = newDirectory();
|
||||||
|
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
|
||||||
|
w.addDocument(new Document());
|
||||||
|
try {
|
||||||
|
w.addDocument(new Document());
|
||||||
|
fail("didn't hit exception");
|
||||||
|
} catch (IllegalArgumentException iae) {
|
||||||
|
// expected
|
||||||
|
}
|
||||||
|
w.deleteAll();
|
||||||
|
w.addDocument(new Document());
|
||||||
|
try {
|
||||||
|
w.addDocument(new Document());
|
||||||
|
fail("didn't hit exception");
|
||||||
|
} catch (IllegalArgumentException iae) {
|
||||||
|
// expected
|
||||||
|
}
|
||||||
|
w.close();
|
||||||
|
dir.close();
|
||||||
|
} finally {
|
||||||
|
restoreIndexWriterMaxDocs();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// LUCENE-6299
|
||||||
|
public void testDeleteAllAfterFlush() throws Exception {
|
||||||
|
setIndexWriterMaxDocs(2);
|
||||||
|
try {
|
||||||
|
Directory dir = newDirectory();
|
||||||
|
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
|
||||||
|
w.addDocument(new Document());
|
||||||
|
w.getReader().close();
|
||||||
|
w.addDocument(new Document());
|
||||||
|
try {
|
||||||
|
w.addDocument(new Document());
|
||||||
|
fail("didn't hit exception");
|
||||||
|
} catch (IllegalArgumentException iae) {
|
||||||
|
// expected
|
||||||
|
}
|
||||||
|
w.deleteAll();
|
||||||
|
w.addDocument(new Document());
|
||||||
|
w.addDocument(new Document());
|
||||||
|
try {
|
||||||
|
w.addDocument(new Document());
|
||||||
|
fail("didn't hit exception");
|
||||||
|
} catch (IllegalArgumentException iae) {
|
||||||
|
// expected
|
||||||
|
}
|
||||||
|
w.close();
|
||||||
|
dir.close();
|
||||||
|
} finally {
|
||||||
|
restoreIndexWriterMaxDocs();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// LUCENE-6299
|
||||||
|
public void testDeleteAllAfterCommit() throws Exception {
|
||||||
|
setIndexWriterMaxDocs(2);
|
||||||
|
try {
|
||||||
|
Directory dir = newDirectory();
|
||||||
|
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
|
||||||
|
w.addDocument(new Document());
|
||||||
|
w.commit();
|
||||||
|
w.addDocument(new Document());
|
||||||
|
try {
|
||||||
|
w.addDocument(new Document());
|
||||||
|
fail("didn't hit exception");
|
||||||
|
} catch (IllegalArgumentException iae) {
|
||||||
|
// expected
|
||||||
|
}
|
||||||
|
w.deleteAll();
|
||||||
|
w.addDocument(new Document());
|
||||||
|
w.addDocument(new Document());
|
||||||
|
try {
|
||||||
|
w.addDocument(new Document());
|
||||||
|
fail("didn't hit exception");
|
||||||
|
} catch (IllegalArgumentException iae) {
|
||||||
|
// expected
|
||||||
|
}
|
||||||
|
w.close();
|
||||||
|
dir.close();
|
||||||
|
} finally {
|
||||||
|
restoreIndexWriterMaxDocs();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// LUCENE-6299
|
||||||
|
public void testDeleteAllMultipleThreads() throws Exception {
|
||||||
|
int limit = TestUtil.nextInt(random(), 2, 10);
|
||||||
|
setIndexWriterMaxDocs(limit);
|
||||||
|
try {
|
||||||
|
Directory dir = newDirectory();
|
||||||
|
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
|
||||||
|
|
||||||
|
CountDownLatch startingGun = new CountDownLatch(1);
|
||||||
|
Thread[] threads = new Thread[limit];
|
||||||
|
for(int i=0;i<limit;i++) {
|
||||||
|
threads[i] = new Thread() {
|
||||||
|
@Override
|
||||||
|
public void run() {
|
||||||
|
try {
|
||||||
|
startingGun.await();
|
||||||
|
w.addDocument(new Document());
|
||||||
|
} catch (Exception e) {
|
||||||
|
throw new RuntimeException(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
};
|
||||||
|
threads[i].start();
|
||||||
|
}
|
||||||
|
|
||||||
|
startingGun.countDown();
|
||||||
|
|
||||||
|
for(Thread thread : threads) {
|
||||||
|
thread.join();
|
||||||
|
}
|
||||||
|
|
||||||
|
try {
|
||||||
|
w.addDocument(new Document());
|
||||||
|
fail("didn't hit exception");
|
||||||
|
} catch (IllegalArgumentException iae) {
|
||||||
|
// expected
|
||||||
|
}
|
||||||
|
w.deleteAll();
|
||||||
|
for(int i=0;i<limit;i++) {
|
||||||
|
w.addDocument(new Document());
|
||||||
|
}
|
||||||
|
try {
|
||||||
|
w.addDocument(new Document());
|
||||||
|
fail("didn't hit exception");
|
||||||
|
} catch (IllegalArgumentException iae) {
|
||||||
|
// expected
|
||||||
|
}
|
||||||
|
w.close();
|
||||||
|
dir.close();
|
||||||
|
} finally {
|
||||||
|
restoreIndexWriterMaxDocs();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// LUCENE-6299
|
||||||
|
public void testDeleteAllAfterClose() throws Exception {
|
||||||
|
setIndexWriterMaxDocs(2);
|
||||||
|
try {
|
||||||
|
Directory dir = newDirectory();
|
||||||
|
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
|
||||||
|
w.addDocument(new Document());
|
||||||
|
w.close();
|
||||||
|
w = new IndexWriter(dir, new IndexWriterConfig(null));
|
||||||
|
w.addDocument(new Document());
|
||||||
|
try {
|
||||||
|
w.addDocument(new Document());
|
||||||
|
fail("didn't hit exception");
|
||||||
|
} catch (IllegalArgumentException iae) {
|
||||||
|
// expected
|
||||||
|
}
|
||||||
|
w.deleteAll();
|
||||||
|
w.addDocument(new Document());
|
||||||
|
w.addDocument(new Document());
|
||||||
|
try {
|
||||||
|
w.addDocument(new Document());
|
||||||
|
fail("didn't hit exception");
|
||||||
|
} catch (IllegalArgumentException iae) {
|
||||||
|
// expected
|
||||||
|
}
|
||||||
|
w.close();
|
||||||
|
dir.close();
|
||||||
|
} finally {
|
||||||
|
restoreIndexWriterMaxDocs();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// LUCENE-6299
|
||||||
|
public void testAcrossTwoIndexWriters() throws Exception {
|
||||||
|
setIndexWriterMaxDocs(1);
|
||||||
|
try {
|
||||||
|
Directory dir = newDirectory();
|
||||||
|
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
|
||||||
|
w.addDocument(new Document());
|
||||||
|
w.close();
|
||||||
|
w = new IndexWriter(dir, new IndexWriterConfig(null));
|
||||||
|
try {
|
||||||
|
w.addDocument(new Document());
|
||||||
|
fail("didn't hit exception");
|
||||||
|
} catch (IllegalArgumentException iae) {
|
||||||
|
// expected
|
||||||
|
}
|
||||||
|
w.close();
|
||||||
|
dir.close();
|
||||||
|
} finally {
|
||||||
|
restoreIndexWriterMaxDocs();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
// LUCENE-6299
|
||||||
|
public void testCorruptIndexExceptionTooLarge() throws Exception {
|
||||||
|
Directory dir = newDirectory();
|
||||||
|
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
|
||||||
|
w.addDocument(new Document());
|
||||||
|
w.addDocument(new Document());
|
||||||
|
w.close();
|
||||||
|
|
||||||
|
setIndexWriterMaxDocs(1);
|
||||||
|
try {
|
||||||
|
DirectoryReader.open(dir);
|
||||||
|
fail("didn't hit exception");
|
||||||
|
} catch (CorruptIndexException cie) {
|
||||||
|
// expected
|
||||||
|
} finally {
|
||||||
|
restoreIndexWriterMaxDocs();
|
||||||
|
}
|
||||||
|
|
||||||
|
dir.close();
|
||||||
|
}
|
||||||
|
|
||||||
|
// LUCENE-6299
|
||||||
|
public void testCorruptIndexExceptionTooLargeWriter() throws Exception {
|
||||||
|
Directory dir = newDirectory();
|
||||||
|
IndexWriter w = new IndexWriter(dir, new IndexWriterConfig(null));
|
||||||
|
w.addDocument(new Document());
|
||||||
|
w.addDocument(new Document());
|
||||||
|
w.close();
|
||||||
|
|
||||||
|
setIndexWriterMaxDocs(1);
|
||||||
|
try {
|
||||||
|
new IndexWriter(dir, new IndexWriterConfig(null));
|
||||||
|
fail("didn't hit exception");
|
||||||
|
} catch (CorruptIndexException cie) {
|
||||||
|
// expected
|
||||||
|
} finally {
|
||||||
|
restoreIndexWriterMaxDocs();
|
||||||
|
}
|
||||||
|
|
||||||
|
dir.close();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -457,7 +457,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class MyFilterDirectoryReader extends FilterDirectoryReader {
|
private static class MyFilterDirectoryReader extends FilterDirectoryReader {
|
||||||
public MyFilterDirectoryReader(DirectoryReader in) {
|
public MyFilterDirectoryReader(DirectoryReader in) throws IOException {
|
||||||
super(in,
|
super(in,
|
||||||
new FilterDirectoryReader.SubReaderWrapper() {
|
new FilterDirectoryReader.SubReaderWrapper() {
|
||||||
@Override
|
@Override
|
||||||
|
@ -470,7 +470,7 @@ public class TestSearcherManager extends ThreadedIndexingAndSearchingTestCase {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
|
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
|
||||||
return new MyFilterDirectoryReader(in);
|
return new MyFilterDirectoryReader(in);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -19,8 +19,10 @@ package org.apache.lucene.store;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
|
import org.apache.lucene.document.Document;
|
||||||
import org.apache.lucene.index.IndexWriter;
|
import org.apache.lucene.index.IndexWriter;
|
||||||
import org.apache.lucene.index.IndexWriterConfig;
|
import org.apache.lucene.index.IndexWriterConfig;
|
||||||
|
import org.apache.lucene.index.RandomIndexWriter;
|
||||||
import org.apache.lucene.util.IOUtils;
|
import org.apache.lucene.util.IOUtils;
|
||||||
import org.apache.lucene.util.LuceneTestCase;
|
import org.apache.lucene.util.LuceneTestCase;
|
||||||
|
|
||||||
|
@ -91,4 +93,16 @@ public class TestMockDirectoryWrapper extends LuceneTestCase {
|
||||||
dir.close();
|
dir.close();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
public void testMDWinsideOfMDW() throws Exception {
|
||||||
|
// add MDW inside another MDW
|
||||||
|
Directory dir = new MockDirectoryWrapper(random(), newMockDirectory());
|
||||||
|
RandomIndexWriter iw = new RandomIndexWriter(random(), dir);
|
||||||
|
for (int i = 0; i < 20; i++) {
|
||||||
|
iw.addDocument(new Document());
|
||||||
|
}
|
||||||
|
iw.commit();
|
||||||
|
iw.close();
|
||||||
|
dir.close();
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -31,7 +31,6 @@ import java.util.SortedSet;
|
||||||
import java.util.TreeSet;
|
import java.util.TreeSet;
|
||||||
|
|
||||||
import org.apache.lucene.analysis.Analyzer;
|
import org.apache.lucene.analysis.Analyzer;
|
||||||
import org.apache.lucene.index.PostingsEnum;
|
|
||||||
import org.apache.lucene.index.FieldInfo;
|
import org.apache.lucene.index.FieldInfo;
|
||||||
import org.apache.lucene.index.IndexOptions;
|
import org.apache.lucene.index.IndexOptions;
|
||||||
import org.apache.lucene.index.IndexReader;
|
import org.apache.lucene.index.IndexReader;
|
||||||
|
@ -39,6 +38,7 @@ import org.apache.lucene.index.IndexReaderContext;
|
||||||
import org.apache.lucene.index.LeafReader;
|
import org.apache.lucene.index.LeafReader;
|
||||||
import org.apache.lucene.index.LeafReaderContext;
|
import org.apache.lucene.index.LeafReaderContext;
|
||||||
import org.apache.lucene.index.MultiReader;
|
import org.apache.lucene.index.MultiReader;
|
||||||
|
import org.apache.lucene.index.PostingsEnum;
|
||||||
import org.apache.lucene.index.ReaderUtil;
|
import org.apache.lucene.index.ReaderUtil;
|
||||||
import org.apache.lucene.index.StoredFieldVisitor;
|
import org.apache.lucene.index.StoredFieldVisitor;
|
||||||
import org.apache.lucene.index.Term;
|
import org.apache.lucene.index.Term;
|
||||||
|
@ -100,7 +100,14 @@ public class PostingsHighlighter {
|
||||||
// unnecessary.
|
// unnecessary.
|
||||||
|
|
||||||
/** for rewriting: we don't want slow processing from MTQs */
|
/** for rewriting: we don't want slow processing from MTQs */
|
||||||
private static final IndexReader EMPTY_INDEXREADER = new MultiReader();
|
private static final IndexReader EMPTY_INDEXREADER;
|
||||||
|
static {
|
||||||
|
try {
|
||||||
|
EMPTY_INDEXREADER = new MultiReader();
|
||||||
|
} catch (IOException bogus) {
|
||||||
|
throw new RuntimeException(bogus);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
/** Default maximum content size to process. Typically snippets
|
/** Default maximum content size to process. Typically snippets
|
||||||
* closer to the beginning of the document better summarize its content */
|
* closer to the beginning of the document better summarize its content */
|
||||||
|
|
|
@ -146,14 +146,14 @@ public class UninvertingReader extends FilterLeafReader {
|
||||||
* can be used normally (e.g. passed to {@link DirectoryReader#openIfChanged(DirectoryReader)})
|
* can be used normally (e.g. passed to {@link DirectoryReader#openIfChanged(DirectoryReader)})
|
||||||
* and so on.
|
* and so on.
|
||||||
*/
|
*/
|
||||||
public static DirectoryReader wrap(DirectoryReader in, final Map<String,Type> mapping) {
|
public static DirectoryReader wrap(DirectoryReader in, final Map<String,Type> mapping) throws IOException {
|
||||||
return new UninvertingDirectoryReader(in, mapping);
|
return new UninvertingDirectoryReader(in, mapping);
|
||||||
}
|
}
|
||||||
|
|
||||||
static class UninvertingDirectoryReader extends FilterDirectoryReader {
|
static class UninvertingDirectoryReader extends FilterDirectoryReader {
|
||||||
final Map<String,Type> mapping;
|
final Map<String,Type> mapping;
|
||||||
|
|
||||||
public UninvertingDirectoryReader(DirectoryReader in, final Map<String,Type> mapping) {
|
public UninvertingDirectoryReader(DirectoryReader in, final Map<String,Type> mapping) throws IOException {
|
||||||
super(in, new FilterDirectoryReader.SubReaderWrapper() {
|
super(in, new FilterDirectoryReader.SubReaderWrapper() {
|
||||||
@Override
|
@Override
|
||||||
public LeafReader wrap(LeafReader reader) {
|
public LeafReader wrap(LeafReader reader) {
|
||||||
|
@ -164,7 +164,7 @@ public class UninvertingReader extends FilterLeafReader {
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
|
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
|
||||||
return new UninvertingDirectoryReader(in, mapping);
|
return new UninvertingDirectoryReader(in, mapping);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -17,6 +17,8 @@ package org.apache.lucene.index;
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* A {@link DirectoryReader} that wraps all its subreaders with
|
* A {@link DirectoryReader} that wraps all its subreaders with
|
||||||
* {@link AssertingLeafReader}
|
* {@link AssertingLeafReader}
|
||||||
|
@ -30,12 +32,12 @@ public class AssertingDirectoryReader extends FilterDirectoryReader {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public AssertingDirectoryReader(DirectoryReader in) {
|
public AssertingDirectoryReader(DirectoryReader in) throws IOException {
|
||||||
super(in, new AssertingSubReaderWrapper());
|
super(in, new AssertingSubReaderWrapper());
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
|
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
|
||||||
return new AssertingDirectoryReader(in);
|
return new AssertingDirectoryReader(in);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -599,7 +599,7 @@ public abstract class BaseStoredFieldsFormatTestCase extends BaseIndexFileFormat
|
||||||
|
|
||||||
private static class DummyFilterDirectoryReader extends FilterDirectoryReader {
|
private static class DummyFilterDirectoryReader extends FilterDirectoryReader {
|
||||||
|
|
||||||
public DummyFilterDirectoryReader(DirectoryReader in) {
|
public DummyFilterDirectoryReader(DirectoryReader in) throws IOException {
|
||||||
super(in, new SubReaderWrapper() {
|
super(in, new SubReaderWrapper() {
|
||||||
@Override
|
@Override
|
||||||
public LeafReader wrap(LeafReader reader) {
|
public LeafReader wrap(LeafReader reader) {
|
||||||
|
@ -609,7 +609,7 @@ public abstract class BaseStoredFieldsFormatTestCase extends BaseIndexFileFormat
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
|
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
|
||||||
return new DummyFilterDirectoryReader(in);
|
return new DummyFilterDirectoryReader(in);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -17,6 +17,7 @@ package org.apache.lucene.index;
|
||||||
* limitations under the License.
|
* limitations under the License.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -38,12 +39,12 @@ public class MismatchedDirectoryReader extends FilterDirectoryReader {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public MismatchedDirectoryReader(DirectoryReader in, Random random) {
|
public MismatchedDirectoryReader(DirectoryReader in, Random random) throws IOException {
|
||||||
super(in, new MismatchedSubReaderWrapper(random));
|
super(in, new MismatchedSubReaderWrapper(random));
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) {
|
protected DirectoryReader doWrapDirectoryReader(DirectoryReader in) throws IOException {
|
||||||
return new AssertingDirectoryReader(in);
|
return new AssertingDirectoryReader(in);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,6 +22,7 @@ import java.util.List;
|
||||||
import java.util.Random;
|
import java.util.Random;
|
||||||
|
|
||||||
import junit.framework.Assert;
|
import junit.framework.Assert;
|
||||||
|
|
||||||
import org.apache.lucene.analysis.MockAnalyzer;
|
import org.apache.lucene.analysis.MockAnalyzer;
|
||||||
import org.apache.lucene.document.Document;
|
import org.apache.lucene.document.Document;
|
||||||
import org.apache.lucene.index.AllDeletedFilterReader;
|
import org.apache.lucene.index.AllDeletedFilterReader;
|
||||||
|
@ -133,7 +134,7 @@ public class QueryUtils {
|
||||||
public static class FCInvisibleMultiReader extends MultiReader {
|
public static class FCInvisibleMultiReader extends MultiReader {
|
||||||
private final Object cacheKey = new Object();
|
private final Object cacheKey = new Object();
|
||||||
|
|
||||||
public FCInvisibleMultiReader(IndexReader... readers) {
|
public FCInvisibleMultiReader(IndexReader... readers) throws IOException {
|
||||||
super(readers);
|
super(readers);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
|
@ -28,13 +28,13 @@ import org.apache.lucene.util.TestUtil;
|
||||||
// do NOT make any methods in this class synchronized, volatile
|
// do NOT make any methods in this class synchronized, volatile
|
||||||
// do NOT import anything from the concurrency package.
|
// do NOT import anything from the concurrency package.
|
||||||
// no randoms, no nothing.
|
// no randoms, no nothing.
|
||||||
public class BaseDirectoryWrapper extends FilterDirectory {
|
public abstract class BaseDirectoryWrapper extends FilterDirectory {
|
||||||
|
|
||||||
private boolean checkIndexOnClose = true;
|
private boolean checkIndexOnClose = true;
|
||||||
private boolean crossCheckTermVectorsOnClose = true;
|
private boolean crossCheckTermVectorsOnClose = true;
|
||||||
protected volatile boolean isOpen = true;
|
protected volatile boolean isOpen = true;
|
||||||
|
|
||||||
public BaseDirectoryWrapper(Directory delegate) {
|
protected BaseDirectoryWrapper(Directory delegate) {
|
||||||
super(delegate);
|
super(delegate);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -72,10 +72,4 @@ public class BaseDirectoryWrapper extends FilterDirectory {
|
||||||
public boolean getCrossCheckTermVectorsOnClose() {
|
public boolean getCrossCheckTermVectorsOnClose() {
|
||||||
return crossCheckTermVectorsOnClose;
|
return crossCheckTermVectorsOnClose;
|
||||||
}
|
}
|
||||||
|
|
||||||
// why does this class override this method?
|
|
||||||
@Override
|
|
||||||
public void copyFrom(Directory from, String src, String dest, IOContext context) throws IOException {
|
|
||||||
in.copyFrom(from, src, dest, context);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -209,23 +209,6 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
|
||||||
useSlowOpenClosers = v;
|
useSlowOpenClosers = v;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns true if {@link #in} must sync its files.
|
|
||||||
* Currently, only {@link NRTCachingDirectory} requires sync'ing its files
|
|
||||||
* because otherwise they are cached in an internal {@link RAMDirectory}. If
|
|
||||||
* other directories require that too, they should be added to this method.
|
|
||||||
*/
|
|
||||||
private boolean mustSync() {
|
|
||||||
Directory delegate = in;
|
|
||||||
while (delegate instanceof FilterDirectory) {
|
|
||||||
if (delegate instanceof NRTCachingDirectory) {
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
delegate = ((FilterDirectory) delegate).getDelegate();
|
|
||||||
}
|
|
||||||
return delegate instanceof NRTCachingDirectory;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public synchronized void sync(Collection<String> names) throws IOException {
|
public synchronized void sync(Collection<String> names) throws IOException {
|
||||||
maybeYield();
|
maybeYield();
|
||||||
|
@ -233,17 +216,14 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
|
||||||
if (crashed) {
|
if (crashed) {
|
||||||
throw new IOException("cannot sync after crash");
|
throw new IOException("cannot sync after crash");
|
||||||
}
|
}
|
||||||
// don't wear out our hardware so much in tests.
|
// always pass thru fsync, directories rely on this.
|
||||||
if (LuceneTestCase.rarely(randomState) || mustSync()) {
|
// 90% of time, we use DisableFsyncFS which omits the real calls.
|
||||||
for (String name : names) {
|
for (String name : names) {
|
||||||
// randomly fail with IOE on any file
|
// randomly fail with IOE on any file
|
||||||
maybeThrowIOException(name);
|
maybeThrowIOException(name);
|
||||||
in.sync(Collections.singleton(name));
|
in.sync(Collections.singleton(name));
|
||||||
unSyncedFiles.remove(name);
|
unSyncedFiles.remove(name);
|
||||||
}
|
}
|
||||||
} else {
|
|
||||||
unSyncedFiles.removeAll(names);
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -1053,18 +1033,36 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
// TODO: why does this class override this method?
|
|
||||||
// we should use the default implementation so all of our checks work?
|
|
||||||
@Override
|
|
||||||
public synchronized void copyFrom(Directory from, String src, String dest, IOContext context) throws IOException {
|
|
||||||
maybeYield();
|
|
||||||
// randomize the IOContext here?
|
|
||||||
in.copyFrom(from, src, dest, context);
|
|
||||||
}
|
|
||||||
|
|
||||||
/** Use this when throwing fake {@code IOException},
|
/** Use this when throwing fake {@code IOException},
|
||||||
* e.g. from {@link MockDirectoryWrapper.Failure}. */
|
* e.g. from {@link MockDirectoryWrapper.Failure}. */
|
||||||
public static class FakeIOException extends IOException {
|
public static class FakeIOException extends IOException {
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public String toString() {
|
||||||
|
if (maxSize != 0) {
|
||||||
|
return "MockDirectoryWrapper(" + in + ", current=" + maxUsedSize + ",max=" + maxSize + ")";
|
||||||
|
} else {
|
||||||
|
return super.toString();
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
|
// don't override optional methods like copyFrom: we need the default impl for things like disk
|
||||||
|
// full checks. we randomly exercise "raw" directories anyway. We ensure default impls are used:
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public final ChecksumIndexInput openChecksumInput(String name, IOContext context) throws IOException {
|
||||||
|
return super.openChecksumInput(name, context);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public final void copyFrom(Directory from, String src, String dest, IOContext context) throws IOException {
|
||||||
|
super.copyFrom(from, src, dest, context);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected final void ensureOpen() throws AlreadyClosedException {
|
||||||
|
super.ensureOpen();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,49 @@
|
||||||
|
package org.apache.lucene.store;
|
||||||
|
|
||||||
|
/*
|
||||||
|
* 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 java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Delegates all operations, even optional ones, to the wrapped directory.
|
||||||
|
* <p>
|
||||||
|
* This class is used if you want the most realistic testing, but still
|
||||||
|
* with a checkindex on close. If you want asserts and evil things,
|
||||||
|
* use MockDirectoryWrapper instead.
|
||||||
|
*/
|
||||||
|
public final class RawDirectoryWrapper extends BaseDirectoryWrapper {
|
||||||
|
|
||||||
|
public RawDirectoryWrapper(Directory delegate) {
|
||||||
|
super(delegate);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void copyFrom(Directory from, String src, String dest, IOContext context) throws IOException {
|
||||||
|
in.copyFrom(from, src, dest, context);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ChecksumIndexInput openChecksumInput(String name, IOContext context) throws IOException {
|
||||||
|
return in.openChecksumInput(name, context);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected void ensureOpen() throws AlreadyClosedException {
|
||||||
|
in.ensureOpen();
|
||||||
|
}
|
||||||
|
}
|
|
@ -82,6 +82,7 @@ import com.carrotsearch.randomizedtesting.rules.NoClassHooksShadowingRule;
|
||||||
import com.carrotsearch.randomizedtesting.rules.NoInstanceHooksOverridesRule;
|
import com.carrotsearch.randomizedtesting.rules.NoInstanceHooksOverridesRule;
|
||||||
import com.carrotsearch.randomizedtesting.rules.StaticFieldsInvariantRule;
|
import com.carrotsearch.randomizedtesting.rules.StaticFieldsInvariantRule;
|
||||||
import com.carrotsearch.randomizedtesting.rules.SystemPropertiesInvariantRule;
|
import com.carrotsearch.randomizedtesting.rules.SystemPropertiesInvariantRule;
|
||||||
|
|
||||||
import org.apache.lucene.analysis.Analyzer;
|
import org.apache.lucene.analysis.Analyzer;
|
||||||
import org.apache.lucene.analysis.MockAnalyzer;
|
import org.apache.lucene.analysis.MockAnalyzer;
|
||||||
import org.apache.lucene.document.Field;
|
import org.apache.lucene.document.Field;
|
||||||
|
@ -110,6 +111,7 @@ import org.apache.lucene.store.MergeInfo;
|
||||||
import org.apache.lucene.store.MockDirectoryWrapper;
|
import org.apache.lucene.store.MockDirectoryWrapper;
|
||||||
import org.apache.lucene.store.MockDirectoryWrapper.Throttling;
|
import org.apache.lucene.store.MockDirectoryWrapper.Throttling;
|
||||||
import org.apache.lucene.store.NRTCachingDirectory;
|
import org.apache.lucene.store.NRTCachingDirectory;
|
||||||
|
import org.apache.lucene.store.RawDirectoryWrapper;
|
||||||
import org.apache.lucene.util.automaton.AutomatonTestUtil;
|
import org.apache.lucene.util.automaton.AutomatonTestUtil;
|
||||||
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
import org.apache.lucene.util.automaton.CompiledAutomaton;
|
||||||
import org.apache.lucene.util.automaton.RegExp;
|
import org.apache.lucene.util.automaton.RegExp;
|
||||||
|
@ -1317,7 +1319,7 @@ public abstract class LuceneTestCase extends Assert {
|
||||||
}
|
}
|
||||||
|
|
||||||
if (bare) {
|
if (bare) {
|
||||||
BaseDirectoryWrapper base = new BaseDirectoryWrapper(directory);
|
BaseDirectoryWrapper base = new RawDirectoryWrapper(directory);
|
||||||
closeAfterSuite(new CloseableDirectory(base, suiteFailureMarker));
|
closeAfterSuite(new CloseableDirectory(base, suiteFailureMarker));
|
||||||
return base;
|
return base;
|
||||||
} else {
|
} else {
|
||||||
|
|
|
@ -67,13 +67,13 @@ import org.apache.lucene.uninverting.UninvertingReader;
|
||||||
import org.apache.lucene.util.Bits;
|
import org.apache.lucene.util.Bits;
|
||||||
import org.apache.lucene.util.BytesRef;
|
import org.apache.lucene.util.BytesRef;
|
||||||
import org.apache.lucene.util.FixedBitSet;
|
import org.apache.lucene.util.FixedBitSet;
|
||||||
import org.apache.solr.common.SolrException;
|
|
||||||
import org.apache.solr.common.SolrException.ErrorCode;
|
import org.apache.solr.common.SolrException.ErrorCode;
|
||||||
|
import org.apache.solr.common.SolrException;
|
||||||
import org.apache.solr.common.params.ModifiableSolrParams;
|
import org.apache.solr.common.params.ModifiableSolrParams;
|
||||||
import org.apache.solr.common.util.NamedList;
|
import org.apache.solr.common.util.NamedList;
|
||||||
import org.apache.solr.common.util.SimpleOrderedMap;
|
import org.apache.solr.common.util.SimpleOrderedMap;
|
||||||
import org.apache.solr.core.DirectoryFactory;
|
|
||||||
import org.apache.solr.core.DirectoryFactory.DirContext;
|
import org.apache.solr.core.DirectoryFactory.DirContext;
|
||||||
|
import org.apache.solr.core.DirectoryFactory;
|
||||||
import org.apache.solr.core.SolrConfig;
|
import org.apache.solr.core.SolrConfig;
|
||||||
import org.apache.solr.core.SolrCore;
|
import org.apache.solr.core.SolrCore;
|
||||||
import org.apache.solr.core.SolrInfoMBean;
|
import org.apache.solr.core.SolrInfoMBean;
|
||||||
|
@ -167,7 +167,7 @@ public class SolrIndexSearcher extends IndexSearcher implements Closeable,SolrIn
|
||||||
|
|
||||||
// TODO: wrap elsewhere and return a "map" from the schema that overrides get() ?
|
// TODO: wrap elsewhere and return a "map" from the schema that overrides get() ?
|
||||||
// this reader supports reopen
|
// this reader supports reopen
|
||||||
private static DirectoryReader wrapReader(SolrCore core, DirectoryReader reader) {
|
private static DirectoryReader wrapReader(SolrCore core, DirectoryReader reader) throws IOException {
|
||||||
assert reader != null;
|
assert reader != null;
|
||||||
return ExitableDirectoryReader.wrap
|
return ExitableDirectoryReader.wrap
|
||||||
(UninvertingReader.wrap(reader, core.getLatestSchema().getUninversionMap(reader)),
|
(UninvertingReader.wrap(reader, core.getLatestSchema().getUninversionMap(reader)),
|
||||||
|
|
Loading…
Reference in New Issue