Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id D70BF200CAE for ; Wed, 21 Jun 2017 19:47:31 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id D5515160BF7; Wed, 21 Jun 2017 17:47:31 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id BC771160BD0 for ; Wed, 21 Jun 2017 19:47:29 +0200 (CEST) Received: (qmail 28956 invoked by uid 500); 21 Jun 2017 17:47:28 -0000 Mailing-List: contact commits-help@lucene.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@lucene.apache.org Delivered-To: mailing list commits@lucene.apache.org Received: (qmail 28947 invoked by uid 99); 21 Jun 2017 17:47:28 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 21 Jun 2017 17:47:28 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id F3F18E02B4; Wed, 21 Jun 2017 17:47:26 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: mikemccand@apache.org To: commits@lucene.apache.org Date: Wed, 21 Jun 2017 17:47:26 -0000 Message-Id: <77620044324940518f3b26fdf060bd51@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: [1/5] lucene-solr:master: LUCENE-7868: use multiple threads to concurrently resolve deletes and DV udpates archived-at: Wed, 21 Jun 2017 17:47:32 -0000 Repository: lucene-solr Updated Branches: refs/heads/master 1737fce5d -> 58105a203 http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58105a20/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java index c0907a5..ad35f32 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestIndexWriterExceptions.java @@ -1758,259 +1758,6 @@ public class TestIndexWriterExceptions extends LuceneTestCase { dir.close(); } - // Make sure if we hit a transient IOException (e.g., disk - // full), and then the exception stops (e.g., disk frees - // up), so we successfully close IW or open an NRT - // reader, we don't lose any deletes or updates: - public void testNoLostDeletesOrUpdates() throws Throwable { - int deleteCount = 0; - int docBase = 0; - int docCount = 0; - - MockDirectoryWrapper dir = newMockDirectory(); - final AtomicBoolean shouldFail = new AtomicBoolean(); - dir.failOn(new MockDirectoryWrapper.Failure() { - - @Override - public void eval(MockDirectoryWrapper dir) throws IOException { - if (shouldFail.get() == false) { - // Only sometimes throw the exc, so we get - // it sometimes on creating the file, on - // flushing buffer, on closing the file: - return; - } - - if (random().nextInt(3) != 2) { - return; - } - - StackTraceElement[] trace = Thread.currentThread().getStackTrace(); - - boolean sawSeal = false; - boolean sawWrite = false; - for (int i = 0; i < trace.length; i++) { - if ("sealFlushedSegment".equals(trace[i].getMethodName())) { - sawSeal = true; - break; - } - if ("writeLiveDocs".equals(trace[i].getMethodName()) || "writeFieldUpdates".equals(trace[i].getMethodName())) { - sawWrite = true; - } - } - - // Don't throw exc if we are "flushing", else - // the segment is aborted and docs are lost: - if (sawWrite && sawSeal == false) { - if (VERBOSE) { - System.out.println("TEST: now fail; thread=" + Thread.currentThread().getName() + " exc:"); - new Throwable().printStackTrace(System.out); - } - shouldFail.set(false); - throw new FakeIOException(); - } - } - }); - - RandomIndexWriter w = null; - - boolean tragic = false; - - for(int iter=0;iter<10*RANDOM_MULTIPLIER;iter++) { - int numDocs = atLeast(100); - if (VERBOSE) { - System.out.println("\nTEST: iter=" + iter + " numDocs=" + numDocs + " docBase=" + docBase + " delCount=" + deleteCount); - } - if (w == null) { - IndexWriterConfig iwc = newIndexWriterConfig(new MockAnalyzer(random())); - w = new RandomIndexWriter(random(), dir, iwc); - // Since we hit exc during merging, a partial - // forceMerge can easily return when there are still - // too many segments in the index: - w.setDoRandomForceMergeAssert(false); - } - for(int i=0;i docs = new ArrayList<>(); + DirectoryReader r = w.getReader(); + + int numIters = atLeast(1000); + for(int iter=0;iter= lastSortValue); + lastSortValue = sortValue; + liveCount++; + } + } + + assertEquals(docs.size() - deletedCount, liveCount); + } + } + + IOUtils.close(r, w, dir); + } - @Test public void testManyReopensAndFields() throws Exception { Directory dir = newDirectory(); final Random random = random(); @@ -735,6 +921,9 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { IndexWriter writer = new IndexWriter(dir, conf); final boolean isNRT = random.nextBoolean(); + if (VERBOSE) { + System.out.println("TEST: isNRT=" + isNRT); + } DirectoryReader reader; if (isNRT) { reader = DirectoryReader.open(writer); @@ -753,45 +942,58 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { int docID = 0; for (int i = 0; i < numRounds; i++) { int numDocs = atLeast(5); -// System.out.println("[" + Thread.currentThread().getName() + "]: round=" + i + ", numDocs=" + numDocs); + if (VERBOSE) { + System.out.println("TEST: round=" + i + ", numDocs=" + numDocs); + } for (int j = 0; j < numDocs; j++) { Document doc = new Document(); - doc.add(new StringField("id", "doc-" + docID, Store.NO)); + doc.add(new StringField("id", "doc-" + docID, Store.YES)); doc.add(new StringField("key", "all", Store.NO)); // update key // add all fields with their current value for (int f = 0; f < fieldValues.length; f++) { doc.add(new NumericDocValuesField("f" + f, fieldValues[f])); } writer.addDocument(doc); + if (VERBOSE) { + System.out.println("TEST add doc id=" + docID); + } ++docID; } int fieldIdx = random.nextInt(fieldValues.length); + String updateField = "f" + fieldIdx; + if (VERBOSE) { + System.out.println("TEST: update field=" + updateField + " for all docs to value=" + (fieldValues[fieldIdx]+1)); + } writer.updateNumericDocValue(new Term("key", "all"), updateField, ++fieldValues[fieldIdx]); -// System.out.println("[" + Thread.currentThread().getName() + "]: updated field '" + updateField + "' to value " + fieldValues[fieldIdx]); if (random.nextDouble() < 0.2) { - int deleteDoc = random.nextInt(docID); // might also delete an already deleted document, ok! + int deleteDoc = random.nextInt(numDocs); // might also delete an already deleted document, ok! + if (VERBOSE) { + System.out.println("TEST: delete doc id=" + deleteDoc); + } writer.deleteDocuments(new Term("id", "doc-" + deleteDoc)); -// System.out.println("[" + Thread.currentThread().getName() + "]: deleted document: doc-" + deleteDoc); } // verify reader - if (!isNRT) { + if (isNRT == false) { + if (VERBOSE) { + System.out.println("TEST: now commit"); + } writer.commit(); } -// System.out.println("[" + Thread.currentThread().getName() + "]: reopen reader: " + reader); DirectoryReader newReader = DirectoryReader.openIfChanged(reader); assertNotNull(newReader); reader.close(); reader = newReader; -// System.out.println("[" + Thread.currentThread().getName() + "]: reopened reader: " + reader); + if (VERBOSE) { + System.out.println("TEST: got reader maxDoc=" + reader.maxDoc() + " " + reader); + } assertTrue(reader.numDocs() > 0); // we delete at most one document per round for (LeafReaderContext context : reader.leaves()) { LeafReader r = context.reader(); -// System.out.println(((SegmentReader) r).getSegmentName()); Bits liveDocs = r.getLiveDocs(); for (int field = 0; field < fieldValues.length; field++) { String f = "f" + field; @@ -800,21 +1002,18 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { int maxDoc = r.maxDoc(); for (int doc = 0; doc < maxDoc; doc++) { if (liveDocs == null || liveDocs.get(doc)) { -// System.out.println("doc=" + (doc + context.docBase) + " f='" + f + "' vslue=" + ndv.get(doc)); - assertEquals(doc, ndv.advance(doc)); - assertEquals("invalid value for doc=" + doc + ", field=" + f + ", reader=" + r, fieldValues[field], ndv.longValue()); + assertEquals("advanced to wrong doc in seg=" + r, doc, ndv.advance(doc)); + assertEquals("invalid value for docID=" + doc + " id=" + r.document(doc).get("id") + ", field=" + f + ", reader=" + r + " doc=" + r.document(doc), fieldValues[field], ndv.longValue()); } } } } -// System.out.println(); } writer.close(); IOUtils.close(reader, dir); } - @Test public void testUpdateSegmentWithNoDocValues() throws Exception { Directory dir = newDirectory(); IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); @@ -866,7 +1065,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { dir.close(); } - @Test public void testUpdateSegmentWithNoDocValues2() throws Exception { Directory dir = newDirectory(); IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); @@ -949,7 +1147,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { dir.close(); } - @Test public void testUpdateSegmentWithPostingButNoDocValues() throws Exception { Directory dir = newDirectory(); IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); @@ -992,7 +1189,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { dir.close(); } - @Test public void testUpdateNumericDVFieldWithSameNameAsPostingField() throws Exception { // this used to fail because FieldInfos.Builder neglected to update // globalFieldMaps.docValuesTypes map @@ -1017,7 +1213,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { dir.close(); } - @Test public void testStressMultiThreading() throws Exception { final Directory dir = newDirectory(); IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); @@ -1069,28 +1264,23 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { final int field = random().nextInt(numFields); final String f = "f" + field; final String cf = "cf" + field; -// System.out.println("[" + Thread.currentThread().getName() + "] numUpdates=" + numUpdates + " updateTerm=" + t + " field=" + field); long updValue = random.nextInt(); writer.updateDocValues(t, new NumericDocValuesField(f, updValue), new NumericDocValuesField(cf, updValue*2)); if (random.nextDouble() < 0.2) { // delete a random document int doc = random.nextInt(numDocs); -// System.out.println("[" + Thread.currentThread().getName() + "] deleteDoc=doc" + doc); writer.deleteDocuments(new Term("id", "doc" + doc)); } if (random.nextDouble() < 0.05) { // commit every 20 updates on average -// System.out.println("[" + Thread.currentThread().getName() + "] commit"); writer.commit(); } if (random.nextDouble() < 0.1) { // reopen NRT reader (apply updates), on average once every 10 updates if (reader == null) { -// System.out.println("[" + Thread.currentThread().getName() + "] open NRT"); reader = DirectoryReader.open(writer); } else { -// System.out.println("[" + Thread.currentThread().getName() + "] reopen NRT"); DirectoryReader r2 = DirectoryReader.openIfChanged(reader, writer); if (r2 != null) { reader.close(); @@ -1099,7 +1289,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { } } } -// System.out.println("[" + Thread.currentThread().getName() + "] DONE"); success = true; } catch (IOException e) { throw new RuntimeException(e); @@ -1144,7 +1333,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { dir.close(); } - @Test public void testUpdateDifferentDocsInDifferentGens() throws Exception { // update same document multiple times across generations Directory dir = newDirectory(); @@ -1184,7 +1372,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { dir.close(); } - @Test public void testChangeCodec() throws Exception { Directory dir = newDirectory(); IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); @@ -1236,7 +1423,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { dir.close(); } - @Test public void testAddIndexes() throws Exception { Directory dir1 = newDirectory(); IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); @@ -1296,7 +1482,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { IOUtils.close(dir1, dir2); } - @Test public void testDeleteUnusedUpdatesFiles() throws Exception { Directory dir = newDirectory(); IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); @@ -1326,7 +1511,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { dir.close(); } - @Test @Nightly public void testTonsOfUpdates() throws Exception { // LUCENE-5248: make sure that when there are many updates, we don't use too much RAM Directory dir = newDirectory(); @@ -1345,8 +1529,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { updateTerms.add(TestUtil.randomSimpleString(random)); } -// System.out.println("numDocs=" + numDocs + " numNumericFields=" + numNumericFields + " numTerms=" + numTerms); - // build a large index with many NDV fields and update terms for (int i = 0; i < numDocs; i++) { Document doc = new Document(); @@ -1368,7 +1550,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { // many flushes during numeric updates writer.getConfig().setRAMBufferSizeMB(2048.0 / 1024 / 1024); final int numUpdates = atLeast(100); -// System.out.println("numUpdates=" + numUpdates); for (int i = 0; i < numUpdates; i++) { int field = random.nextInt(numNumericFields); Term updateTerm = new Term("upd", RandomPicks.randomFrom(random, updateTerms)); @@ -1396,7 +1577,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { dir.close(); } - @Test public void testUpdatesOrder() throws Exception { Directory dir = newDirectory(); IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); @@ -1413,6 +1593,9 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { writer.updateNumericDocValue(new Term("upd", "t2"), "f1", 3L); // update f1 to 3 writer.updateNumericDocValue(new Term("upd", "t2"), "f2", 3L); // update f2 to 3 writer.updateNumericDocValue(new Term("upd", "t1"), "f1", 4L); // update f1 to 4 (but not f2) + if (VERBOSE) { + System.out.println("TEST: now close"); + } writer.close(); DirectoryReader reader = DirectoryReader.open(dir); @@ -1427,7 +1610,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { dir.close(); } - @Test public void testUpdateAllDeletedSegment() throws Exception { Directory dir = newDirectory(); IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); @@ -1455,7 +1637,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { dir.close(); } - @Test public void testUpdateTwoNonexistingTerms() throws Exception { Directory dir = newDirectory(); IndexWriterConfig conf = newIndexWriterConfig(new MockAnalyzer(random())); @@ -1480,7 +1661,6 @@ public class TestNumericDocValuesUpdates extends LuceneTestCase { dir.close(); } - @Test public void testIOContext() throws Exception { // LUCENE-5591: make sure we pass an IOContext with an approximate // segmentSize in FlushInfo http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58105a20/lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java b/lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java index 112a108..fc56614 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestPerSegmentDeletes.java @@ -68,10 +68,11 @@ public class TestPerSegmentDeletes extends LuceneTestCase { writer.deleteDocuments(new Term("id", "11")); - // flushing without applying deletes means - // there will still be deletes in the segment infos writer.flush(false, false); - assertTrue(writer.bufferedUpdatesStream.any()); + + // deletes are now resolved on flush, so there shouldn't be + // any deletes after flush + assertFalse(writer.bufferedUpdatesStream.any()); // get reader flushes pending deletes // so there should not be anymore http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58105a20/lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java b/lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java index 89d4ad1..3ec0b56 100644 --- a/lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java +++ b/lucene/core/src/test/org/apache/lucene/index/TestPrefixCodedTerms.java @@ -16,14 +16,11 @@ */ package org.apache.lucene.index; - -import java.util.Arrays; import java.util.Iterator; import java.util.Set; import java.util.TreeSet; import org.apache.lucene.index.PrefixCodedTerms.TermIterator; -import org.apache.lucene.util.BytesRef; import org.apache.lucene.util.LuceneTestCase; import org.apache.lucene.util.TestUtil; @@ -73,77 +70,4 @@ public class TestPrefixCodedTerms extends LuceneTestCase { assertFalse(expected.hasNext()); } - - @SuppressWarnings("unchecked") - public void testMergeOne() { - Term t1 = new Term("foo", "a"); - PrefixCodedTerms.Builder b1 = new PrefixCodedTerms.Builder(); - b1.add(t1); - PrefixCodedTerms pb1 = b1.finish(); - - Term t2 = new Term("foo", "b"); - PrefixCodedTerms.Builder b2 = new PrefixCodedTerms.Builder(); - b2.add(t2); - PrefixCodedTerms pb2 = b2.finish(); - - MergedPrefixCodedTermsIterator merged = new MergedPrefixCodedTermsIterator(Arrays.asList(new PrefixCodedTerms[] {pb1, pb2})); - BytesRef term = merged.next(); - assertNotNull(term); - assertEquals("foo", merged.field()); - assertEquals("a", term.utf8ToString()); - term = merged.next(); - assertNotNull(term); - assertEquals("b", term.utf8ToString()); - assertNull(merged.next()); - } - - @SuppressWarnings({"unchecked","rawtypes"}) - public void testMergeRandom() { - PrefixCodedTerms pb[] = new PrefixCodedTerms[TestUtil.nextInt(random(), 2, 10)]; - Set superSet = new TreeSet<>(); - - for (int i = 0; i < pb.length; i++) { - Set terms = new TreeSet<>(); - int nterms = TestUtil.nextInt(random(), 0, 10000); - for (int j = 0; j < nterms; j++) { - String field = TestUtil.randomUnicodeString(random(), 2); - //String field = TestUtil.randomSimpleString(random(), 2); - Term term = new Term(field, TestUtil.randomUnicodeString(random(), 4)); - terms.add(term); - } - superSet.addAll(terms); - - PrefixCodedTerms.Builder b = new PrefixCodedTerms.Builder(); - //System.out.println("TEST: sub " + i + " has " + terms.size() + " terms"); - for (Term ref: terms) { - //System.out.println(" add " + ref.field() + " " + ref.bytes()); - b.add(ref); - } - pb[i] = b.finish(); - } - - Iterator expected = superSet.iterator(); - - MergedPrefixCodedTermsIterator actual = new MergedPrefixCodedTermsIterator(Arrays.asList(pb)); - String field = ""; - - BytesRef lastTerm = null; - BytesRef term; - while ((term = actual.next()) != null) { - if (field != actual.field()) { - field = actual.field(); - lastTerm = null; - } - if (lastTerm != null && lastTerm.equals(term)) { - continue; - } - lastTerm = BytesRef.deepCopyOf(term); - assertTrue(expected.hasNext()); - - Term expectedTerm = expected.next(); - assertEquals(expectedTerm, new Term(field, term)); - } - - assertFalse(expected.hasNext()); - } } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58105a20/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java ---------------------------------------------------------------------- diff --git a/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java b/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java index a1b2a5c..7a7b0ac 100644 --- a/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java +++ b/lucene/core/src/test/org/apache/lucene/search/TestControlledRealTimeReopenThread.java @@ -333,9 +333,11 @@ public class TestControlledRealTimeReopenThread extends ThreadedIndexingAndSearc }; t.start(); writer.waitAfterUpdate = true; // wait in addDocument to let some reopens go through + final long lastGen = writer.updateDocument(new Term("foo", "bar"), doc); // once this returns the doc is already reflected in the last reopen - assertFalse(manager.isSearcherCurrent()); // false since there is a delete in the queue + // We now eagerly resolve deletes so the manager should see it after update: + assertTrue(manager.isSearcherCurrent()); IndexSearcher searcher = manager.acquire(); try { http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58105a20/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java ---------------------------------------------------------------------- diff --git a/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java b/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java index 1503de8..0bd4784 100644 --- a/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java +++ b/lucene/join/src/test/org/apache/lucene/search/join/TestJoinUtil.java @@ -1188,13 +1188,16 @@ public class TestJoinUtil extends LuceneTestCase { private void executeRandomJoin(boolean multipleValuesPerDocument, int maxIndexIter, int maxSearchIter, int numberOfDocumentsToIndex) throws Exception { for (int indexIter = 1; indexIter <= maxIndexIter; indexIter++) { if (VERBOSE) { - System.out.println("indexIter=" + indexIter); + System.out.println("TEST: indexIter=" + indexIter + " numDocs=" + numberOfDocumentsToIndex); } IndexIterationContext context = createContext(numberOfDocumentsToIndex, multipleValuesPerDocument, false); IndexSearcher indexSearcher = context.searcher; + if (VERBOSE) { + System.out.println("TEST: got searcher=" + indexSearcher); + } for (int searchIter = 1; searchIter <= maxSearchIter; searchIter++) { if (VERBOSE) { - System.out.println("searchIter=" + searchIter); + System.out.println("TEST: searchIter=" + searchIter); } int r = random().nextInt(context.randomUniqueValues.length); @@ -1360,9 +1363,9 @@ public class TestJoinUtil extends LuceneTestCase { } final List subValues; { - int start = randomUniqueValuesReplica.size()==numberOfLinkValues? 0 : random.nextInt(randomUniqueValuesReplica.size()-numberOfLinkValues); - subValues = randomUniqueValuesReplica.subList(start, start+numberOfLinkValues); - Collections.shuffle(subValues, random); + int start = randomUniqueValuesReplica.size()==numberOfLinkValues? 0 : random.nextInt(randomUniqueValuesReplica.size()-numberOfLinkValues); + subValues = randomUniqueValuesReplica.subList(start, start+numberOfLinkValues); + Collections.shuffle(subValues, random); } for (String linkValue : subValues) { @@ -1404,6 +1407,9 @@ public class TestJoinUtil extends LuceneTestCase { } if (random.nextBoolean()) { + if (VERBOSE) { + System.out.println("TEST: now force merge"); + } w.forceMerge(1); } w.close(); http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58105a20/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java ---------------------------------------------------------------------- diff --git a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java index 334f784..fc643d2 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java +++ b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/IDVersionPostingsWriter.java @@ -45,6 +45,7 @@ final class IDVersionPostingsWriter extends PushPostingsWriterBase { private long lastVersion; private final Bits liveDocs; + private String segment; public IDVersionPostingsWriter(Bits liveDocs) { this.liveDocs = liveDocs; @@ -58,6 +59,7 @@ final class IDVersionPostingsWriter extends PushPostingsWriterBase { @Override public void init(IndexOutput termsOut, SegmentWriteState state) throws IOException { CodecUtil.writeIndexHeader(termsOut, TERMS_CODEC, VERSION_CURRENT, state.segmentInfo.getId(), state.segmentSuffix); + segment = state.segmentInfo.name; } @Override @@ -87,7 +89,7 @@ final class IDVersionPostingsWriter extends PushPostingsWriterBase { return; } if (lastDocID != -1) { - throw new IllegalArgumentException("term appears in more than one document"); + throw new IllegalArgumentException("term appears in more than one document: " + lastDocID + " and " + docID); } if (termDocFreq != 1) { throw new IllegalArgumentException("term appears more than once in the document"); http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58105a20/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java ---------------------------------------------------------------------- diff --git a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java index d83b915..e9187af 100644 --- a/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java +++ b/lucene/sandbox/src/java/org/apache/lucene/codecs/idversion/VersionBlockTreeTermsWriter.java @@ -161,7 +161,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer { } private final List fields = new ArrayList<>(); - // private final String segment; + private final String segment; /** Create a new writer. The number of items (terms or * sub-blocks) per block will aim to be between @@ -175,6 +175,7 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer { throws IOException { BlockTreeTermsWriter.validateSettings(minItemsInBlock, maxItemsInBlock); + segment = state.segmentInfo.name; maxDoc = state.segmentInfo.maxDoc(); @@ -729,7 +730,6 @@ public final class VersionBlockTreeTermsWriter extends FieldsConsumer { /** Writes one term's worth of postings. */ public void write(BytesRef text, TermsEnum termsEnum) throws IOException { - BlockTermState state = postingsWriter.writeTerm(text, termsEnum, docsSeen); // TODO: LUCENE-5693: we don't need this check if we fix IW to not send deleted docs to us on flush: if (state != null && ((IDVersionPostingsWriter) postingsWriter).lastDocID != -1) { http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58105a20/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java ---------------------------------------------------------------------- diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java index 8cb6665..28ab3b6 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseDocValuesFormatTestCase.java @@ -2106,6 +2106,9 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes String[] uniqueValues = valueSet.toArray(new String[0]); // index some docs + if (VERBOSE) { + System.out.println("\nTEST: now add numDocs=" + numDocs); + } for (int i = 0; i < numDocs; i++) { Document doc = new Document(); Field idField = new StringField("id", Integer.toString(i), Field.Store.NO); @@ -2137,12 +2140,18 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes // delete some docs int numDeletions = random().nextInt(numDocs/10); + if (VERBOSE) { + System.out.println("\nTEST: now delete " + numDeletions + " docs"); + } for (int i = 0; i < numDeletions; i++) { int id = random().nextInt(numDocs); writer.deleteDocuments(new Term("id", Integer.toString(id))); } // compare + if (VERBOSE) { + System.out.println("\nTEST: now get reader"); + } DirectoryReader ir = writer.getReader(); TestUtil.checkReader(ir); for (LeafReaderContext context : ir.leaves()) { @@ -2168,7 +2177,13 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes } } } + if (VERBOSE) { + System.out.println("\nTEST: now close reader"); + } ir.close(); + if (VERBOSE) { + System.out.println("TEST: force merge"); + } writer.forceMerge(1); // compare again @@ -2195,8 +2210,17 @@ public abstract class BaseDocValuesFormatTestCase extends BaseIndexFileFormatTes } } } + if (VERBOSE) { + System.out.println("TEST: close reader"); + } ir.close(); + if (VERBOSE) { + System.out.println("TEST: close writer"); + } writer.close(); + if (VERBOSE) { + System.out.println("TEST: close dir"); + } dir.close(); } http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58105a20/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java ---------------------------------------------------------------------- diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java index 959466a..ab92946 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/index/BaseIndexFileFormatTestCase.java @@ -564,7 +564,7 @@ abstract class BaseIndexFileFormatTestCase extends LuceneTestCase { handleFakeIOException(e, exceptionStream); allowAlreadyClosed = true; } - + if (random().nextInt(10) == 0) { // trigger flush: try { http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/58105a20/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java ---------------------------------------------------------------------- diff --git a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java index 0243a56..3a87c1e 100644 --- a/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java +++ b/lucene/test-framework/src/java/org/apache/lucene/util/LuceneTestCase.java @@ -1183,17 +1183,6 @@ public abstract class LuceneTestCase extends Assert { } if (rarely(r)) { - // change buffered deletes parameters - boolean limitBufferedDeletes = r.nextBoolean(); - if (limitBufferedDeletes) { - c.setMaxBufferedDeleteTerms(TestUtil.nextInt(r, 1, 1000)); - } else { - c.setMaxBufferedDeleteTerms(IndexWriterConfig.DISABLE_AUTO_FLUSH); - } - didChange = true; - } - - if (rarely(r)) { IndexWriter.IndexReaderWarmer curWarmer = c.getMergedSegmentWarmer(); if (curWarmer == null || curWarmer instanceof SimpleMergedSegmentWarmer) { // change warmer parameters