mirror of https://github.com/apache/lucene.git
LUCENE-5189: fixed concurrency bug
git-svn-id: https://svn.apache.org/repos/asf/lucene/dev/trunk@1524900 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
parent
872c02c31c
commit
8fcd7a8245
|
@ -433,6 +433,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
|
||||||
final ReadersAndLiveDocs rld = readerMap.get(info);
|
final ReadersAndLiveDocs rld = readerMap.get(info);
|
||||||
if (rld != null) {
|
if (rld != null) {
|
||||||
assert info == rld.info;
|
assert info == rld.info;
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] ReaderPool.drop: " + info);
|
||||||
readerMap.remove(info);
|
readerMap.remove(info);
|
||||||
rld.dropReaders();
|
rld.dropReaders();
|
||||||
}
|
}
|
||||||
|
@ -459,6 +460,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
|
||||||
if (!poolReaders && rld.refCount() == 1) {
|
if (!poolReaders && rld.refCount() == 1) {
|
||||||
// This is the last ref to this RLD, and we're not
|
// This is the last ref to this RLD, and we're not
|
||||||
// pooling, so remove it:
|
// pooling, so remove it:
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] ReaderPool.release: " + rld.info);
|
||||||
if (rld.writeLiveDocs(directory)) {
|
if (rld.writeLiveDocs(directory)) {
|
||||||
// Make sure we only write del docs and field updates for a live segment:
|
// Make sure we only write del docs and field updates for a live segment:
|
||||||
assert infoIsLive(rld.info);
|
assert infoIsLive(rld.info);
|
||||||
|
@ -467,6 +469,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
|
||||||
deleter.checkpoint(segmentInfos, false);
|
deleter.checkpoint(segmentInfos, false);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] ReaderPool.release: drop readers " + rld.info);
|
||||||
rld.dropReaders();
|
rld.dropReaders();
|
||||||
readerMap.remove(rld.info);
|
readerMap.remove(rld.info);
|
||||||
}
|
}
|
||||||
|
@ -3158,6 +3161,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
|
||||||
final Bits currentLiveDocs = rld.getLiveDocs();
|
final Bits currentLiveDocs = rld.getLiveDocs();
|
||||||
final Map<Integer,Map<String,Long>> mergingUpdates = rld.getMergingUpdates();
|
final Map<Integer,Map<String,Long>> mergingUpdates = rld.getMergingUpdates();
|
||||||
|
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] IW.commitMergedDeletes: info=" + info + ", mergingUpdates=" + mergingUpdates);
|
||||||
|
|
||||||
if (prevLiveDocs != null) {
|
if (prevLiveDocs != null) {
|
||||||
|
|
||||||
// If we had deletions on starting the merge we must
|
// If we had deletions on starting the merge we must
|
||||||
|
@ -3284,6 +3289,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
|
||||||
|
|
||||||
// set any updates that came while the segment was merging
|
// set any updates that came while the segment was merging
|
||||||
if (!mergedUpdates.isEmpty()) {
|
if (!mergedUpdates.isEmpty()) {
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] IW.commitMergedDeletes: mergedDeletes.info=" + mergedDeletes.info + ", mergedUpdates=" + mergedUpdates);
|
||||||
assert mergedDeletes != null;
|
assert mergedDeletes != null;
|
||||||
mergedDeletes.setMergingUpdates(mergedUpdates);
|
mergedDeletes.setMergingUpdates(mergedUpdates);
|
||||||
}
|
}
|
||||||
|
@ -3331,6 +3337,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
|
||||||
}
|
}
|
||||||
|
|
||||||
final ReadersAndLiveDocs mergedDeletes = merge.info.info.getDocCount() == 0 ? null : commitMergedDeletes(merge, mergeState);
|
final ReadersAndLiveDocs mergedDeletes = merge.info.info.getDocCount() == 0 ? null : commitMergedDeletes(merge, mergeState);
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] IW.commitMerge: mergedDeletes=" + mergedDeletes);
|
||||||
|
|
||||||
assert mergedDeletes == null || mergedDeletes.getPendingDeleteCount() != 0 || mergedDeletes.hasFieldUpdates();
|
assert mergedDeletes == null || mergedDeletes.getPendingDeleteCount() != 0 || mergedDeletes.hasFieldUpdates();
|
||||||
|
|
||||||
|
@ -3364,6 +3371,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
|
||||||
|
|
||||||
if (mergedDeletes != null) {
|
if (mergedDeletes != null) {
|
||||||
if (dropSegment) {
|
if (dropSegment) {
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] IW.commitMerge: dropChanges " + merge.info);
|
||||||
mergedDeletes.dropChanges();
|
mergedDeletes.dropChanges();
|
||||||
}
|
}
|
||||||
readerPool.release(mergedDeletes);
|
readerPool.release(mergedDeletes);
|
||||||
|
@ -3677,7 +3685,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
|
||||||
setDiagnostics(si, SOURCE_MERGE, details);
|
setDiagnostics(si, SOURCE_MERGE, details);
|
||||||
merge.setInfo(new SegmentInfoPerCommit(si, 0, -1L, -1L));
|
merge.setInfo(new SegmentInfoPerCommit(si, 0, -1L, -1L));
|
||||||
|
|
||||||
// System.out.println("[" + Thread.currentThread().getName() + "] _mergeInit: " + segString(merge.segments) + " into " + si);
|
// System.out.println("[" + Thread.currentThread().getName() + "] IW._mergeInit: " + segString(merge.segments) + " into " + si);
|
||||||
|
|
||||||
// Lock order: IW -> BD
|
// Lock order: IW -> BD
|
||||||
bufferedDeletesStream.prune(segmentInfos);
|
bufferedDeletesStream.prune(segmentInfos);
|
||||||
|
@ -3743,8 +3751,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
|
||||||
assert rld != null;
|
assert rld != null;
|
||||||
if (drop) {
|
if (drop) {
|
||||||
rld.dropChanges();
|
rld.dropChanges();
|
||||||
|
} else {
|
||||||
|
rld.dropMergingUpdates();
|
||||||
}
|
}
|
||||||
rld.setMerging(false);
|
|
||||||
rld.release(sr);
|
rld.release(sr);
|
||||||
readerPool.release(rld);
|
readerPool.release(rld);
|
||||||
if (drop) {
|
if (drop) {
|
||||||
|
@ -3802,13 +3811,9 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
|
||||||
// Hold onto the "live" reader; we will use this to
|
// Hold onto the "live" reader; we will use this to
|
||||||
// commit merged deletes
|
// commit merged deletes
|
||||||
final ReadersAndLiveDocs rld = readerPool.get(info, true);
|
final ReadersAndLiveDocs rld = readerPool.get(info, true);
|
||||||
SegmentReader reader = rld.getReader(true, context);
|
SegmentReader reader = rld.getReaderForMerge(context);
|
||||||
assert reader != null;
|
assert reader != null;
|
||||||
|
|
||||||
// Notify that we are merging, so that we can later copy the updates
|
|
||||||
// that were received while merging to the merged segment.
|
|
||||||
rld.setMerging(true);
|
|
||||||
|
|
||||||
// Carefully pull the most recent live docs:
|
// Carefully pull the most recent live docs:
|
||||||
final Bits liveDocs;
|
final Bits liveDocs;
|
||||||
final int delCount;
|
final int delCount;
|
||||||
|
@ -3860,6 +3865,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit{
|
||||||
segUpto++;
|
segUpto++;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] IW.mergeMiddle: merging " + merge.getMergeReaders());
|
||||||
|
|
||||||
// we pass merge.getMergeReaders() instead of merge.readers to allow the
|
// we pass merge.getMergeReaders() instead of merge.readers to allow the
|
||||||
// OneMerge to return a view over the actual segments to merge
|
// OneMerge to return a view over the actual segments to merge
|
||||||
final SegmentMerger merger = new SegmentMerger(merge.getMergeReaders(),
|
final SegmentMerger merger = new SegmentMerger(merge.getMergeReaders(),
|
||||||
|
|
|
@ -170,29 +170,27 @@ class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates
|
||||||
return reader;
|
return reader;
|
||||||
}
|
}
|
||||||
|
|
||||||
private SegmentReader doGetReaderWithUpdates(IOContext context) throws IOException {
|
private synchronized SegmentReader doGetReaderWithUpdates(IOContext context) throws IOException {
|
||||||
|
assert Thread.holdsLock(writer); // when we get here, we should already have the writer lock
|
||||||
boolean checkpoint = false;
|
boolean checkpoint = false;
|
||||||
try {
|
try {
|
||||||
// don't synchronize the entire method because we cannot call
|
|
||||||
// writer.checkpoint() while holding the RLD lock, otherwise we might hit
|
|
||||||
// a deadlock w/ e.g. a concurrent merging thread.
|
|
||||||
synchronized (this) {
|
|
||||||
checkpoint = writeLiveDocs(info.info.dir);
|
checkpoint = writeLiveDocs(info.info.dir);
|
||||||
if (reader == null) {
|
if (reader == null) {
|
||||||
// We steal returned ref:
|
// We steal returned ref:
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] RLD.doGetReaderWithUpdates: newSR " + info);
|
||||||
reader = new SegmentReader(info, context);
|
reader = new SegmentReader(info, context);
|
||||||
if (liveDocs == null) {
|
if (liveDocs == null) {
|
||||||
liveDocs = reader.getLiveDocs();
|
liveDocs = reader.getLiveDocs();
|
||||||
}
|
}
|
||||||
} else if (checkpoint) {
|
} else if (checkpoint) {
|
||||||
// enroll a new reader with the applied updates
|
// enroll a new reader with the applied updates
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] RLD.doGetReaderWithUpdates: reopenReader " + info);
|
||||||
reopenReader(context);
|
reopenReader(context);
|
||||||
}
|
}
|
||||||
|
|
||||||
// Ref for caller
|
// Ref for caller
|
||||||
reader.incRef();
|
reader.incRef();
|
||||||
return reader;
|
return reader;
|
||||||
}
|
|
||||||
} finally {
|
} finally {
|
||||||
if (checkpoint) {
|
if (checkpoint) {
|
||||||
writer.checkpoint();
|
writer.checkpoint();
|
||||||
|
@ -208,9 +206,11 @@ class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates
|
||||||
// cost of obtaining it.
|
// cost of obtaining it.
|
||||||
if (applyFieldUpdates && hasFieldUpdates()) {
|
if (applyFieldUpdates && hasFieldUpdates()) {
|
||||||
synchronized (writer) {
|
synchronized (writer) {
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] RLD.getReader: getReaderWithUpdates " + info);
|
||||||
return doGetReaderWithUpdates(context);
|
return doGetReaderWithUpdates(context);
|
||||||
}
|
}
|
||||||
} else {
|
} else {
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] RLD.getReader: getReader no updates " + info);
|
||||||
return doGetReader(context);
|
return doGetReader(context);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -367,6 +367,7 @@ class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates
|
||||||
if (hasFieldUpdates) {
|
if (hasFieldUpdates) {
|
||||||
// reader could be null e.g. for a just merged segment (from
|
// reader could be null e.g. for a just merged segment (from
|
||||||
// IndexWriter.commitMergedDeletes).
|
// IndexWriter.commitMergedDeletes).
|
||||||
|
// if (this.reader == null) System.out.println("[" + Thread.currentThread().getName() + "] RLD.writeLiveDocs: newSR " + info);
|
||||||
final SegmentReader reader = this.reader == null ? new SegmentReader(info, IOContext.READONCE) : this.reader;
|
final SegmentReader reader = this.reader == null ? new SegmentReader(info, IOContext.READONCE) : this.reader;
|
||||||
try {
|
try {
|
||||||
// clone FieldInfos so that we can update their numericUpdatesGen
|
// clone FieldInfos so that we can update their numericUpdatesGen
|
||||||
|
@ -396,6 +397,7 @@ class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates
|
||||||
final DocValuesConsumer fieldsConsumer = docValuesFormat.fieldsConsumer(state);
|
final DocValuesConsumer fieldsConsumer = docValuesFormat.fieldsConsumer(state);
|
||||||
boolean fieldsConsumerSuccess = false;
|
boolean fieldsConsumerSuccess = false;
|
||||||
try {
|
try {
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] RLD.writeLiveDocs: applying updates; seg=" + info + " updates=" + numericUpdates);
|
||||||
for (Entry<String,Map<Integer,Long>> e : numericUpdates.entrySet()) {
|
for (Entry<String,Map<Integer,Long>> e : numericUpdates.entrySet()) {
|
||||||
final String field = e.getKey();
|
final String field = e.getKey();
|
||||||
final Map<Integer,Long> updates = e.getValue();
|
final Map<Integer,Long> updates = e.getValue();
|
||||||
|
@ -459,6 +461,7 @@ class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates
|
||||||
}
|
}
|
||||||
} finally {
|
} finally {
|
||||||
if (reader != this.reader) {
|
if (reader != this.reader) {
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] RLD.writeLiveDocs: closeReader " + reader);
|
||||||
reader.close();
|
reader.close();
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -501,7 +504,7 @@ class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates
|
||||||
if (hasFieldUpdates) {
|
if (hasFieldUpdates) {
|
||||||
info.advanceDocValuesGen();
|
info.advanceDocValuesGen();
|
||||||
// copy all the updates to mergingUpdates, so they can later be applied to the merged segment
|
// copy all the updates to mergingUpdates, so they can later be applied to the merged segment
|
||||||
if (isMerging) {
|
if (isMerging || true) {
|
||||||
copyUpdatesToMerging();
|
copyUpdatesToMerging();
|
||||||
}
|
}
|
||||||
numericUpdates.clear();
|
numericUpdates.clear();
|
||||||
|
@ -513,6 +516,9 @@ class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates
|
||||||
}
|
}
|
||||||
|
|
||||||
private void copyUpdatesToMerging() {
|
private void copyUpdatesToMerging() {
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] RLD.copyUpdatesToMerging: " + numericUpdates);
|
||||||
|
// cannot do a simple putAll, even if mergingUpdates is empty, because we
|
||||||
|
// need a shallow copy of the values (maps)
|
||||||
for (Entry<String,Map<Integer,Long>> e : numericUpdates.entrySet()) {
|
for (Entry<String,Map<Integer,Long>> e : numericUpdates.entrySet()) {
|
||||||
String field = e.getKey();
|
String field = e.getKey();
|
||||||
Map<Integer,Long> merging = mergingUpdates.get(field);
|
Map<Integer,Long> merging = mergingUpdates.get(field);
|
||||||
|
@ -525,17 +531,32 @@ class ReadersAndLiveDocs { // TODO (DVU_RENAME) to ReaderAndUpdates
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Indicates whether this segment is currently being merged. Call this just
|
* Returns a reader for merge. This method applies field updates if there are
|
||||||
* before the segment is being merged with {@code true} and when the merge has
|
* any and marks that this segment is currently merging.
|
||||||
* finished and all updates have been applied to the merged segment, call this
|
|
||||||
* with {@code false}.
|
|
||||||
*/
|
*/
|
||||||
public synchronized void setMerging(boolean isMerging) {
|
SegmentReader getReaderForMerge(IOContext context) throws IOException {
|
||||||
this.isMerging = isMerging;
|
// lock ordering must be IW -> RLD, otherwise could cause deadlocks
|
||||||
if (!isMerging) {
|
synchronized (writer) {
|
||||||
mergingUpdates.clear();
|
synchronized (this) {
|
||||||
|
// must execute these two statements as atomic operation, otherwise we
|
||||||
|
// could lose updates if e.g. another thread calls writeLiveDocs in
|
||||||
|
// between, or the updates are applied to the obtained reader, but then
|
||||||
|
// re-applied in IW.commitMergedDeletes (unnecessary work and potential
|
||||||
|
// bugs.
|
||||||
|
isMerging = true;
|
||||||
|
return getReader(true, context);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* Drops all merging updates. Called from IndexWriter after this segment
|
||||||
|
* finished merging (whether successfully or not).
|
||||||
|
*/
|
||||||
|
public synchronized void dropMergingUpdates() {
|
||||||
|
mergingUpdates.clear();
|
||||||
|
isMerging = false;
|
||||||
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Called from IndexWriter after applying deletes to the merged segment, while
|
* Called from IndexWriter after applying deletes to the merged segment, while
|
||||||
|
|
|
@ -102,6 +102,8 @@ public final class SegmentReader extends AtomicReader {
|
||||||
// DVProducer for each field
|
// DVProducer for each field
|
||||||
final Map<Long,List<FieldInfo>> genInfos = getGenInfos(si);
|
final Map<Long,List<FieldInfo>> genInfos = getGenInfos(si);
|
||||||
|
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] SR.init: new reader: " + si + "; gens=" + genInfos.keySet());
|
||||||
|
|
||||||
for (Entry<Long,List<FieldInfo>> e : genInfos.entrySet()) {
|
for (Entry<Long,List<FieldInfo>> e : genInfos.entrySet()) {
|
||||||
Long gen = e.getKey();
|
Long gen = e.getKey();
|
||||||
List<FieldInfo> infos = e.getValue();
|
List<FieldInfo> infos = e.getValue();
|
||||||
|
@ -150,6 +152,8 @@ public final class SegmentReader extends AtomicReader {
|
||||||
this.core = sr.core;
|
this.core = sr.core;
|
||||||
core.incRef();
|
core.incRef();
|
||||||
|
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] SR.init: sharing reader: " + sr + " for gens=" + sr.genDVProducers.keySet());
|
||||||
|
|
||||||
// increment refCount of DocValuesProducers that are used by this reader
|
// increment refCount of DocValuesProducers that are used by this reader
|
||||||
boolean success = false;
|
boolean success = false;
|
||||||
try {
|
try {
|
||||||
|
@ -170,9 +174,11 @@ public final class SegmentReader extends AtomicReader {
|
||||||
if (dvp != null) {
|
if (dvp != null) {
|
||||||
// gen used by given reader, incRef its DVP
|
// gen used by given reader, incRef its DVP
|
||||||
dvp.incRef();
|
dvp.incRef();
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] SR.init: sharing DVP for gen=" + gen + " refCount=" + dvp.getRefCount());
|
||||||
} else {
|
} else {
|
||||||
// this gen is not used by given reader, initialize a new one
|
// this gen is not used by given reader, initialize a new one
|
||||||
dvp = newDocValuesProducer(si, IOContext.READ, dir, dvFormat, gen, infos);
|
dvp = newDocValuesProducer(si, IOContext.READ, dir, dvFormat, gen, infos);
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "] SR.init: new DVP for gen=" + gen + " refCount=" + dvp.getRefCount());
|
||||||
}
|
}
|
||||||
assert dvp != null;
|
assert dvp != null;
|
||||||
genDVProducers.put(gen, dvp);
|
genDVProducers.put(gen, dvp);
|
||||||
|
|
|
@ -707,8 +707,8 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||||
int numRounds = atLeast(15);
|
int numRounds = atLeast(15);
|
||||||
int docID = 0;
|
int docID = 0;
|
||||||
for (int i = 0; i < numRounds; i++) {
|
for (int i = 0; i < numRounds; i++) {
|
||||||
int numDocs = atLeast(2);
|
int numDocs = atLeast(5);
|
||||||
// System.out.println("round=" + i + ", numDocs=" + numDocs);
|
// System.out.println("[" + Thread.currentThread().getName() + "]: round=" + i + ", numDocs=" + numDocs);
|
||||||
for (int j = 0; j < numDocs; j++) {
|
for (int j = 0; j < numDocs; j++) {
|
||||||
Document doc = new Document();
|
Document doc = new Document();
|
||||||
doc.add(new StringField("id", "doc-" + docID, Store.NO));
|
doc.add(new StringField("id", "doc-" + docID, Store.NO));
|
||||||
|
@ -724,12 +724,12 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||||
int fieldIdx = random.nextInt(fieldValues.length);
|
int fieldIdx = random.nextInt(fieldValues.length);
|
||||||
String updateField = "f" + fieldIdx;
|
String updateField = "f" + fieldIdx;
|
||||||
writer.updateNumericDocValue(new Term("key", "all"), updateField, ++fieldValues[fieldIdx]);
|
writer.updateNumericDocValue(new Term("key", "all"), updateField, ++fieldValues[fieldIdx]);
|
||||||
// System.out.println("+++ updated field '" + updateField + "' to value " + fieldValues[fieldIdx]);
|
// System.out.println("[" + Thread.currentThread().getName() + "]: updated field '" + updateField + "' to value " + fieldValues[fieldIdx]);
|
||||||
|
|
||||||
if (random.nextDouble() < 0.2) {
|
if (random.nextDouble() < 0.2) {
|
||||||
int deleteDoc = random.nextInt(docID); // might also delete an already deleted document, ok!
|
int deleteDoc = random.nextInt(docID); // might also delete an already deleted document, ok!
|
||||||
writer.deleteDocuments(new Term("id", "doc-" + deleteDoc));
|
writer.deleteDocuments(new Term("id", "doc-" + deleteDoc));
|
||||||
// System.out.println("--- deleted document: doc-" + deleteDoc);
|
// System.out.println("[" + Thread.currentThread().getName() + "]: deleted document: doc-" + deleteDoc);
|
||||||
}
|
}
|
||||||
|
|
||||||
// verify reader
|
// verify reader
|
||||||
|
@ -737,10 +737,12 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||||
writer.commit();
|
writer.commit();
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "]: reopen reader: " + reader);
|
||||||
DirectoryReader newReader = DirectoryReader.openIfChanged(reader);
|
DirectoryReader newReader = DirectoryReader.openIfChanged(reader);
|
||||||
assertNotNull(newReader);
|
assertNotNull(newReader);
|
||||||
reader.close();
|
reader.close();
|
||||||
reader = newReader;
|
reader = newReader;
|
||||||
|
// System.out.println("[" + Thread.currentThread().getName() + "]: reopened reader: " + reader);
|
||||||
assertTrue(reader.numDocs() > 0); // we delete at most one document per round
|
assertTrue(reader.numDocs() > 0); // we delete at most one document per round
|
||||||
for (AtomicReaderContext context : reader.leaves()) {
|
for (AtomicReaderContext context : reader.leaves()) {
|
||||||
AtomicReader r = context.reader();
|
AtomicReader r = context.reader();
|
||||||
|
@ -754,7 +756,7 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase {
|
||||||
for (int doc = 0; doc < maxDoc; doc++) {
|
for (int doc = 0; doc < maxDoc; doc++) {
|
||||||
if (liveDocs == null || liveDocs.get(doc)) {
|
if (liveDocs == null || liveDocs.get(doc)) {
|
||||||
// System.out.println("doc=" + (doc + context.docBase) + " f='" + f + "' vslue=" + ndv.get(doc));
|
// System.out.println("doc=" + (doc + context.docBase) + " f='" + f + "' vslue=" + ndv.get(doc));
|
||||||
assertEquals("invalid value for doc=" + (doc + context.docBase) + ", field=" + f, fieldValues[field], ndv.get(doc));
|
assertEquals("invalid value for doc=" + doc + ", field=" + f + ", reader=" + r, fieldValues[field], ndv.get(doc));
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue