You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by si...@apache.org on 2018/04/12 10:38:07 UTC

[1/3] lucene-solr:branch_7x: LUCENE-8233: Harden PendingDeletes modification assertions

Repository: lucene-solr
Updated Branches:
  refs/heads/branch_7x df736af82 -> 8785035db


LUCENE-8233: Harden PendingDeletes modification assertions

This change adds a missing call to PendingDeletes#onNewReader and
hardens the assertion when a PendingDelete can actually be modified ie.
receive deletes and updates. Now PendingDeltes are also initialized
when no reader is provided but the SegmentCommitInfo has evidence that there
is are no deletes.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/4ea6924d
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/4ea6924d
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/4ea6924d

Branch: refs/heads/branch_7x
Commit: 4ea6924dbe73c6682f6677d88552908452459ebe
Parents: df736af
Author: Simon Willnauer <si...@apache.org>
Authored: Thu Apr 12 11:35:15 2018 +0200
Committer: Simon Willnauer <si...@apache.org>
Committed: Thu Apr 12 11:38:07 2018 +0200

----------------------------------------------------------------------
 .../org/apache/lucene/index/PendingDeletes.java |  8 ++++++-
 .../apache/lucene/index/PendingSoftDeletes.java |  3 ++-
 .../apache/lucene/index/ReadersAndUpdates.java  |  1 +
 .../apache/lucene/index/TestPendingDeletes.java |  8 +++----
 .../lucene/index/TestPendingSoftDeletes.java    | 23 +++++++++++++++-----
 5 files changed, 31 insertions(+), 12 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4ea6924d/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java b/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
index bce704c..fca42b4 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PendingDeletes.java
@@ -52,7 +52,10 @@ class PendingDeletes {
   }
 
   PendingDeletes(SegmentCommitInfo info) {
-    this(info, null, false);
+    this(info, null, info.hasDeletions() == false);
+    // if we don't have deletions we can mark it as initialized since we might receive deletes on a segment
+    // without having a reader opened on it ie. after a merge when we apply the deletes that IW received while merging.
+    // For segments that were published we enforce a reader in the BufferedUpdatesStream.SegmentState ctor
   }
 
   private PendingDeletes(SegmentCommitInfo info, Bits liveDocs, boolean liveDocsInitialized) {
@@ -65,6 +68,9 @@ class PendingDeletes {
 
 
   protected MutableBits getMutableBits() throws IOException {
+    // if we pull mutable bits but we haven't been initialized something is completely off.
+    // this means we receive deletes without having the bitset that is on-disk ready to be cloned
+    assert liveDocsInitialized : "can't delete if liveDocs are not initialized";
     if (liveDocsShared) {
       // Copy on write: this means we've cloned a
       // SegmentReader sharing the current liveDocs

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4ea6924d/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java b/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
index b73ac83..0d7852b 100644
--- a/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
+++ b/lucene/core/src/java/org/apache/lucene/index/PendingSoftDeletes.java
@@ -67,7 +67,7 @@ final class PendingSoftDeletes extends PendingDeletes {
   void onNewReader(SegmentReader reader, SegmentCommitInfo info) throws IOException {
     super.onNewReader(reader, info);
     hardDeletes.onNewReader(reader, info);
-    if (dvGeneration != info.getDocValuesGen()) { // only re-calculate this if we haven't seen this generation
+    if (dvGeneration < info.getDocValuesGen()) { // only re-calculate this if we haven't seen this generation
       final DocIdSetIterator iterator = DocValuesFieldExistsQuery.getDocValuesDocIdSetIterator(field, reader);
       if (iterator == null) { // nothing is deleted we don't have a soft deletes field in this segment
         this.pendingDeleteCount = 0;
@@ -120,6 +120,7 @@ final class PendingSoftDeletes extends PendingDeletes {
   void onDocValuesUpdate(FieldInfo info, List<DocValuesFieldUpdates> updatesToApply) throws IOException {
     if (field.equals(info.name)) {
       assert dvGeneration < info.getDocValuesGen() : "we have seen this generation update already: " + dvGeneration + " vs. " + info.getDocValuesGen();
+      assert dvGeneration != -2 : "docValues generation is still uninitialized";
       DocValuesFieldUpdates.Iterator[] subs = new DocValuesFieldUpdates.Iterator[updatesToApply.size()];
       for(int i=0; i<subs.length; i++) {
         subs[i] = updatesToApply.get(i).iterator();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4ea6924d/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
index 4851d45..6dc8864 100644
--- a/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
+++ b/lucene/core/src/java/org/apache/lucene/index/ReadersAndUpdates.java
@@ -619,6 +619,7 @@ final class ReadersAndUpdates {
       final SegmentReader reader;
       if (this.reader == null) {
         reader = new SegmentReader(info, indexCreatedVersionMajor, IOContext.READONCE);
+        pendingDeletes.onNewReader(reader, info);
       } else {
         reader = this.reader;
       }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4ea6924d/lucene/core/src/test/org/apache/lucene/index/TestPendingDeletes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPendingDeletes.java b/lucene/core/src/test/org/apache/lucene/index/TestPendingDeletes.java
index e150e06..bbe309a 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPendingDeletes.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPendingDeletes.java
@@ -40,7 +40,7 @@ public class TestPendingDeletes extends LuceneTestCase {
     RAMDirectory dir = new RAMDirectory();
     SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "test", 10, false, Codec.getDefault(),
         Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
-    SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, 0, 0, 0);
+    SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, -1, -1, -1);
     PendingDeletes deletes = newPendingDeletes(commitInfo);
     assertNull(deletes.getLiveDocs());
     int docToDelete = TestUtil.nextInt(random(), 0, 7);
@@ -76,7 +76,7 @@ public class TestPendingDeletes extends LuceneTestCase {
     RAMDirectory dir = new RAMDirectory();
     SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "test", 6, false, Codec.getDefault(),
         Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
-    SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, 0, 0, 0);
+    SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, -1, -1, -1);
     PendingDeletes deletes = newPendingDeletes(commitInfo);
     assertFalse(deletes.writeLiveDocs(dir));
     assertEquals(0, dir.listAll().length);
@@ -86,7 +86,7 @@ public class TestPendingDeletes extends LuceneTestCase {
       deletes.liveDocsShared();
       deletes.delete(2);
     }
-    assertEquals(0, commitInfo.getDelGen());
+    assertEquals(-1, commitInfo.getDelGen());
     assertEquals(0, commitInfo.getDelCount());
 
     assertEquals(secondDocDeletes ? 2 : 1, deletes.numPendingDeletes());
@@ -133,7 +133,7 @@ public class TestPendingDeletes extends LuceneTestCase {
     RAMDirectory dir = new RAMDirectory();
     SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "test", 3, false, Codec.getDefault(),
         Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
-    SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, 0, 0, 0);
+    SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, -1, -1, -1);
     PendingDeletes deletes = newPendingDeletes(commitInfo);
     for (int i = 0; i < 3; i++) {
       assertTrue(deletes.delete(i));

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/4ea6924d/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java b/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
index c428a4b..255ff9e 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestPendingSoftDeletes.java
@@ -45,7 +45,7 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
 
   public void testDeleteSoft() throws IOException {
     Directory dir = newDirectory();
-    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); // no soft delete field hier
+    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig()); // no soft delete field here
     Document doc = new Document();
     doc.add(new StringField("id", "1", Field.Store.YES));
     writer.softUpdateDocument(new Term("id", "1"), doc,
@@ -97,8 +97,20 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
     RAMDirectory dir = new RAMDirectory();
     SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "test", 10, false, Codec.getDefault(),
         Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
-    SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, 0, 0, 0);
+    SegmentCommitInfo commitInfo = new SegmentCommitInfo(si, 0, -1, -1, -1);
+    IndexWriter writer = new IndexWriter(dir, newIndexWriterConfig());
+    for (int i = 0; i < si.maxDoc(); i++) {
+      writer.addDocument(new Document());
+    }
+    writer.forceMerge(1);
+    writer.commit();
+    DirectoryReader reader = writer.getReader();
+    assertEquals(1, reader.leaves().size());
+    SegmentReader segmentReader = (SegmentReader) reader.leaves().get(0).reader();
     PendingSoftDeletes deletes = newPendingDeletes(commitInfo);
+    deletes.onNewReader(segmentReader, commitInfo);
+    reader.close();
+    writer.close();
     FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, 0, Collections.emptyMap(), 0, 0);
     List<Integer> docsDeleted = Arrays.asList(1, 3, 7, 8, DocIdSetIterator.NO_MORE_DOCS);
     List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 10));
@@ -152,10 +164,9 @@ public class TestPendingSoftDeletes extends TestPendingDeletes {
     assertEquals(1, reader.leaves().size());
     SegmentReader segmentReader = (SegmentReader) reader.leaves().get(0).reader();
     SegmentCommitInfo segmentInfo = segmentReader.getSegmentInfo();
-    SegmentInfo si = new SegmentInfo(dir, Version.LATEST, Version.LATEST, "test", 3, false, Codec.getDefault(),
-        Collections.emptyMap(), StringHelper.randomId(), new HashMap<>(), null);
-    PendingSoftDeletes deletes = newPendingDeletes(segmentInfo);
-    FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getDocValuesGen(), Collections.emptyMap(), 0, 0);
+    PendingDeletes deletes = newPendingDeletes(segmentInfo);
+    deletes.onNewReader(segmentReader, segmentInfo);
+    FieldInfo fieldInfo = new FieldInfo("_soft_deletes", 1, false, false, false, IndexOptions.NONE, DocValuesType.NUMERIC, segmentInfo.getNextDocValuesGen(), Collections.emptyMap(), 0, 0);
     List<Integer> docsDeleted = Arrays.asList(1, DocIdSetIterator.NO_MORE_DOCS);
     List<DocValuesFieldUpdates> updates = Arrays.asList(singleUpdate(docsDeleted, 3));
     deletes.onDocValuesUpdate(fieldInfo, updates);


[3/3] lucene-solr:branch_7x: Fix lint error of unused imports in LeaderVoteWaitTimeoutTest

Posted by si...@apache.org.
Fix lint error of unused imports in LeaderVoteWaitTimeoutTest


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/8785035d
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/8785035d
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/8785035d

Branch: refs/heads/branch_7x
Commit: 8785035db4d037b21dd59dfcd562224e58bbccdd
Parents: 19847b9
Author: Simon Willnauer <si...@apache.org>
Authored: Thu Apr 12 12:28:28 2018 +0200
Committer: Simon Willnauer <si...@apache.org>
Committed: Thu Apr 12 12:28:28 2018 +0200

----------------------------------------------------------------------
 .../src/test/org/apache/solr/cloud/LeaderVoteWaitTimeoutTest.java | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/8785035d/solr/core/src/test/org/apache/solr/cloud/LeaderVoteWaitTimeoutTest.java
----------------------------------------------------------------------
diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderVoteWaitTimeoutTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderVoteWaitTimeoutTest.java
index 6d32c3a..34b9350 100644
--- a/solr/core/src/test/org/apache/solr/cloud/LeaderVoteWaitTimeoutTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/LeaderVoteWaitTimeoutTest.java
@@ -25,7 +25,6 @@ import java.util.Arrays;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
-import java.util.Set;
 
 import org.apache.solr.JSONTestUtil;
 import org.apache.solr.client.solrj.SolrServerException;
@@ -34,8 +33,6 @@ import org.apache.solr.client.solrj.impl.HttpSolrClient;
 import org.apache.solr.client.solrj.request.CollectionAdminRequest;
 import org.apache.solr.client.solrj.request.QueryRequest;
 import org.apache.solr.common.SolrInputDocument;
-import org.apache.solr.common.cloud.CollectionStatePredicate;
-import org.apache.solr.common.cloud.DocCollection;
 import org.apache.solr.common.cloud.Replica;
 import org.apache.solr.common.cloud.ZkCoreNodeProps;
 import org.apache.solr.common.util.NamedList;


[2/3] lucene-solr:branch_7x: Fix TestLRUQueryCache#testDocValuesUpdatesDontBreakCache

Posted by si...@apache.org.
Fix TestLRUQueryCache#testDocValuesUpdatesDontBreakCache

this test was flagged as BadApple and referred to SOLR-12028
The test stated clearly that the usage of newSearch(reader) is
dangerous since it might add concurrency to the test. This commit
respects this comment and removes all subsequent useage of
newSearcher(...)


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/19847b9b
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/19847b9b
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/19847b9b

Branch: refs/heads/branch_7x
Commit: 19847b9bb87fc9d03f7b6f8dcba7a02f943b8930
Parents: 4ea6924
Author: Simon Willnauer <si...@apache.org>
Authored: Thu Apr 12 12:12:21 2018 +0200
Committer: Simon Willnauer <si...@apache.org>
Committed: Thu Apr 12 12:12:21 2018 +0200

----------------------------------------------------------------------
 .../test/org/apache/lucene/search/TestLRUQueryCache.java    | 9 ++++-----
 1 file changed, 4 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/19847b9b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
index 361c1d7..25e5953 100644
--- a/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
+++ b/lucene/core/src/test/org/apache/lucene/search/TestLRUQueryCache.java
@@ -1480,16 +1480,15 @@ public class TestLRUQueryCache extends LuceneTestCase {
   }
 
   @Test
-  @BadApple(bugUrl="https://issues.apache.org/jira/browse/SOLR-12028")
   public void testDocValuesUpdatesDontBreakCache() throws IOException {
     Directory dir = newDirectory();
     IndexWriterConfig iwc = newIndexWriterConfig().setMergePolicy(NoMergePolicy.INSTANCE);
-    //RandomIndexWriter w = new RandomIndexWriter(random(), dir, iwc);
     IndexWriter w = new IndexWriter(dir, iwc);
     w.addDocument(new Document());
     w.commit();
     DirectoryReader reader = DirectoryReader.open(w);
 
+    // IMPORTANT:
     // Don't use newSearcher(), because that will sometimes use an ExecutorService, and
     // we need to be single threaded to ensure that LRUQueryCache doesn't skip the cache
     // due to thread contention
@@ -1511,7 +1510,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
     w.addDocument(doc);
     reader.close();
     reader = DirectoryReader.open(w);
-    searcher = newSearcher(reader);
+    searcher = new AssertingIndexSearcher(random(), reader); // no newSearcher(reader) - see comment above
     searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
     searcher.setQueryCache(cache);
 
@@ -1520,7 +1519,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
 
     reader.close();
     reader = DirectoryReader.open(w);
-    searcher = newSearcher(reader);
+    searcher = new AssertingIndexSearcher(random(), reader); // no newSearcher(reader) - see comment above
     searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
     searcher.setQueryCache(cache);
 
@@ -1531,7 +1530,7 @@ public class TestLRUQueryCache extends LuceneTestCase {
     w.updateNumericDocValue(new Term("text", "text"), "field", 2l);
     reader.close();
     reader = DirectoryReader.open(w);
-    searcher = newSearcher(reader);
+    searcher = new AssertingIndexSearcher(random(), reader); // no newSearcher(reader) - see comment above
     searcher.setQueryCachingPolicy(QueryCachingPolicy.ALWAYS_CACHE);
     searcher.setQueryCache(cache);