You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2021/03/10 09:48:18 UTC

[lucene] branch jira/LUCENE-8692 created (now e93313d)

This is an automated email from the ASF dual-hosted git repository.

dweiss pushed a change to branch jira/LUCENE-8692
in repository https://gitbox.apache.org/repos/asf/lucene.git.


      at e93313d  LUCENE-8692: fix file leak in MockRandomPostingsFormat that surfaces when corrupting files

This branch includes the following new commits:

     new af9da98  LUCENE-8692: updates to TestStressIndexing2 demonstrating bug
     new 2b28d9d  LUCENE-8692: add additional logging to Solr test that first surfaced bug
     new 4bd7d68  LUCENE-8692: initial attempt at fixing bug
     new e93313d  LUCENE-8692: fix file leak in MockRandomPostingsFormat that surfaces when corrupting files

The 4 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.



[lucene] 03/04: LUCENE-8692: initial attempt at fixing bug

Posted by dw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

dweiss pushed a commit to branch jira/LUCENE-8692
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 4bd7d6810dbf50ae8f07c1051378f482ff953073
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Tue Mar 5 16:03:31 2019 -0700

    LUCENE-8692: initial attempt at fixing bug
---
 .../src/java/org/apache/lucene/index/IndexWriter.java | 19 ++++++++++++++++---
 1 file changed, 16 insertions(+), 3 deletions(-)

diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index f9aaf34..eb3d790 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -2151,8 +2151,15 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
 
   final void maybeMerge(MergePolicy mergePolicy, MergeTrigger trigger, int maxNumSegments) throws IOException {
     ensureOpen(false);
-    boolean newMergesFound = updatePendingMerges(mergePolicy, trigger, maxNumSegments);
-    mergeScheduler.merge(this, trigger, newMergesFound);
+    try {
+      boolean newMergesFound = updatePendingMerges(mergePolicy, trigger, maxNumSegments);
+      mergeScheduler.merge(this, trigger, newMergesFound);
+    } catch (MergePolicy.MergeAbortedException mae) {
+      throw mae;
+    } catch (Throwable t) {
+      tragicEvent(t, "maybeMerge");
+      throw t;
+    }
   }
 
   private synchronized boolean updatePendingMerges(MergePolicy mergePolicy, MergeTrigger trigger, int maxNumSegments)
@@ -3261,6 +3268,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
           }
         }
       } catch (VirtualMachineError tragedy) {
+        // nocommit: should we be catching Throwable & treating as traggic here?
+        // nocommit: if we get an IOException here does it come direct from dir, or has docWriter already tracked it as traggic?
         tragicEvent(tragedy, "prepareCommit");
         throw tragedy;
       } finally {
@@ -4839,7 +4848,11 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable,
           segmentInfos.updateGeneration(toSync);
         }
       }
-    } catch (VirtualMachineError tragedy) {
+    } catch (MergePolicy.MergeAbortedException mae) {
+      throw mae;
+    } catch (Throwable tragedy) {
+      // nocommit: simonw has questions/concerns about catching Throwable here
+      // nocommit: can/should we catch something narrower?
       tragicEvent(tragedy, "startCommit");
       throw tragedy;
     }


[lucene] 04/04: LUCENE-8692: fix file leak in MockRandomPostingsFormat that surfaces when corrupting files

Posted by dw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

dweiss pushed a commit to branch jira/LUCENE-8692
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit e93313d42c4e5642e58707b74ed119776086b590
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Tue Mar 5 16:04:00 2019 -0700

    LUCENE-8692: fix file leak in MockRandomPostingsFormat that surfaces when corrupting files
---
 .../mockrandom/MockRandomPostingsFormat.java       | 24 +++++++++++++---------
 1 file changed, 14 insertions(+), 10 deletions(-)

diff --git a/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java b/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java
index 6f57a2e..46f448e 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/codecs/mockrandom/MockRandomPostingsFormat.java
@@ -263,19 +263,23 @@ public final class MockRandomPostingsFormat extends PostingsFormat {
     return fields;
   }
 
-  @Override
-  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
-
+  private static long parseSeedFromSegment(final SegmentReadState state) throws IOException {
     final String seedFileName = IndexFileNames.segmentFileName(state.segmentInfo.name, state.segmentSuffix, SEED_EXT);
-    final ChecksumIndexInput in = state.directory.openChecksumInput(seedFileName, state.context);
-    CodecUtil.checkIndexHeader(in, "MockRandomSeed", 0, 0, state.segmentInfo.getId(), state.segmentSuffix);
-    final long seed = in.readLong();
-    CodecUtil.checkFooter(in);
-    if (LuceneTestCase.VERBOSE) {
-      System.out.println("MockRandomCodec: reading from seg=" + state.segmentInfo.name + " formatID=" + state.segmentSuffix + " seed=" + seed);
+    try (final ChecksumIndexInput in = state.directory.openChecksumInput(seedFileName, state.context)) {
+      CodecUtil.checkIndexHeader(in, "MockRandomSeed", 0, 0, state.segmentInfo.getId(), state.segmentSuffix);
+      final long seed = in.readLong();
+      CodecUtil.checkFooter(in);
+      if (LuceneTestCase.VERBOSE) {
+        System.out.println("MockRandomCodec: reading from seg=" + state.segmentInfo.name + " formatID=" + state.segmentSuffix + " seed=" + seed);
+      }
+      return seed;
     }
-    in.close();
+  }
+  
+  @Override
+  public FieldsProducer fieldsProducer(SegmentReadState state) throws IOException {
 
+    final long seed = parseSeedFromSegment(state);
     final Random random = new Random(seed);
     
     int readBufferSize = TestUtil.nextInt(random, 1, 4096);


[lucene] 01/04: LUCENE-8692: updates to TestStressIndexing2 demonstrating bug

Posted by dw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

dweiss pushed a commit to branch jira/LUCENE-8692
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit af9da98e87ee2a44532cd9baf2a1bc6e64865b0d
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Tue Mar 5 16:02:26 2019 -0700

    LUCENE-8692: updates to TestStressIndexing2 demonstrating bug
---
 .../apache/lucene/index/TestStressIndexing2.java   | 162 +++++++++++++++++++--
 1 file changed, 151 insertions(+), 11 deletions(-)

diff --git a/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java b/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java
index 05d8797..ad1c3ee 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestStressIndexing2.java
@@ -16,7 +16,9 @@
  */
 package org.apache.lucene.index;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
+import java.nio.file.NoSuchFileException;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.Collections;
@@ -31,13 +33,16 @@ import org.apache.lucene.analysis.MockAnalyzer;
 import org.apache.lucene.document.Document;
 import org.apache.lucene.document.Field;
 import org.apache.lucene.document.FieldType;
+import org.apache.lucene.document.NumericDocValuesField;
 import org.apache.lucene.document.TextField;
 import org.apache.lucene.index.IndexWriterConfig.OpenMode;
 import org.apache.lucene.search.DocIdSetIterator;
 import org.apache.lucene.search.TermQuery;
 import org.apache.lucene.store.Directory;
+import org.apache.lucene.store.MockDirectoryWrapper;
 import org.apache.lucene.util.Bits;
 import org.apache.lucene.util.BytesRef;
+import org.apache.lucene.util.IOUtils;
 import org.apache.lucene.util.LuceneTestCase;
 import org.apache.lucene.util.TestUtil;
 
@@ -115,7 +120,40 @@ public class TestStressIndexing2 extends LuceneTestCase {
     }
   }
 
+  public void testRandomCorruptionIsTragic() throws Exception {
+    
+    final MockDirectoryWrapper dir = newMockDirectory();
+    dir.setCheckIndexOnClose(false); // we are corrupting it!
+    
+    final IndexWriterConfig iwc = newIndexWriterConfig();
+    final MergeScheduler ms = iwc.getMergeScheduler();
+    if (ms instanceof ConcurrentMergeScheduler) {
+      ((ConcurrentMergeScheduler) ms).setSuppressExceptions();
+    }
+    final IndexWriter w = new IndexWriter(dir, iwc);
 
+    try {
+      // NOTE: we never need to 'start' this thread, we're just going to use it's logic as a Runnable
+      CorruptibleIndexingThread th = new CorruptibleIndexingThread();
+      th.w = w;
+      th.dir = dir;
+      th.base = 0;
+      th.range = atLeast(10);
+      th.iterations = atLeast(5);
+      
+      th.run();
+      
+    } finally {
+      if (ms instanceof ConcurrentMergeScheduler) {
+        // Sneaky: CMS's merge thread will be concurrently rolling back IW due
+        // to the tragedy, with this main thread, so we have to wait here
+        // to ensure the rollback has finished, else MDW still sees open files:
+        ((ConcurrentMergeScheduler) ms).sync();
+      }
+      IOUtils.closeWhileHandlingException(w, dir);
+    }
+  }
+  
   static Term idTerm = new Term("id","");
   IndexingThread[] threads;
   static Comparator<IndexableField> fieldNameComparator = new Comparator<IndexableField>() {
@@ -765,8 +803,9 @@ public class TestStressIndexing2 extends LuceneTestCase {
       
       ArrayList<Field> fields = new ArrayList<>();
       String idString = getIdString();
-      Field idField =  newField("id", idString, customType1);
-      fields.add(idField);
+
+      fields.add(newField("id", idString, customType1));
+      fields.add(new NumericDocValuesField("docValues", nextInt(5000)));
 
       Map<String,FieldType> tvTypes = new HashMap<>();
 
@@ -853,6 +892,26 @@ public class TestStressIndexing2 extends LuceneTestCase {
       docs.put(idString, d);
     }
 
+    public void updateDocVal() throws IOException {
+      if (docs.isEmpty()) {
+        indexDoc();
+        return;
+      }
+      
+      final String idString = (new ArrayList<>(docs.keySet())).get(nextInt(docs.size()));
+      
+      if (VERBOSE) {
+        System.out.println(Thread.currentThread().getName() + ": dv update id:" + idString);
+      }
+      
+      final NumericDocValuesField val = new NumericDocValuesField("docValues", nextInt(5000));
+      w.updateDocValues(new Term("id",idString), val);
+      
+      final Document doc = docs.get(idString);
+      doc.removeFields("docValues");
+      doc.add(val);
+    }
+    
     public void deleteDoc() throws IOException {
       String idString = getIdString();
       if (VERBOSE) {
@@ -871,27 +930,108 @@ public class TestStressIndexing2 extends LuceneTestCase {
       docs.remove(idString);
     }
 
+    public void doOneRandomOp() throws IOException {
+      final int what = nextInt(100);
+      if (what < 5) {
+        deleteDoc();
+      } else if (what < 10) {
+        deleteByQuery();
+      } else if (what < 20) {
+        updateDocVal();
+      } else {
+        indexDoc();
+      }
+    }
+    
     @Override
     public void run() {
       try {
         r = new Random(base+range+seed);
         for (int i=0; i<iterations; i++) {
-          int what = nextInt(100);
-          if (what < 5) {
-            deleteDoc();
-          } else if (what < 10) {
-            deleteByQuery();
-          } else {
-            indexDoc();
-          }
+          doOneRandomOp();
         }
       } catch (Throwable e) {
         throw new RuntimeException(e);
       }
 
-      synchronized (this) {
+      synchronized (this) { // nocommit: what/why is this here?
         docs.size();
       }
     }
   }
+
+  public class CorruptibleIndexingThread extends IndexingThread {
+    MockDirectoryWrapper dir;
+    
+    @Override
+    public void run() {
+      assert null != dir;
+      assert null != w;
+      r = new Random(base+range+seed);
+      
+      // we're going to loop (effectively) forever
+      // introducing a small amount of corruption then attempting an index update
+      // once we encounter an exception from the index update, we will assert that the IW
+      // has recorded a traggic exception
+      
+      int totalCorruption = 0;
+      int numIndexOpsSucceded = 0;
+      boolean gotExpectedFailure = false;
+      
+      // "while (true)" safety valve, prevent infinite loop if IW is so broken it never throws exceptions
+      // NOTE: test could get very 'lucky' (need MockDirectoryWrapper.alwaysCorrupt to be publc)
+      while (totalCorruption < 999999) {
+        
+        // do some random corruption of a *few* files
+        try {
+          final List<String> allFiles = Arrays.asList(dir.listAll());
+          Collections.sort(allFiles);
+          Collections.shuffle(allFiles, r);
+          try {
+            dir.corruptFiles(allFiles.subList(0, Math.min(allFiles.size(), RANDOM_MULTIPLIER)));
+            totalCorruption++;
+          } catch (RuntimeException | FileNotFoundException | NoSuchFileException e) {
+            // merges can lead to this exception
+          }
+        } catch (IOException e) {
+          assertNull("IOException trying to corrupt", e);
+        }
+        
+        // do some index updates
+        try {
+          doOneRandomOp();
+          numIndexOpsSucceded++;
+
+          // nocommit: do a LOT of commits for now since that's where the (known) problem seems to be
+          // if (r.nextInt(100) < 10) { 
+          if (r.nextInt(100) < 50) {
+            w.commit();
+            numIndexOpsSucceded++;
+          }
+          
+        } catch (Throwable t) {
+          // NOTE: we don't use expectThrows because there's no garuntee that
+          // the update we attempt will cause an exception, but if it *does* cause an exception,
+          // then it must have been tragic.
+          //
+          // (nothing we're doing that might cause an exception should ever be "non-tragic")
+          try {
+            assertNotNull("index update encountered throwable, but no tragic event recorded: "
+                          + t.toString(),
+                          w.getTragicException());
+            assertFalse(w.isOpen());
+          } catch (AssertionError a) {
+            a.addSuppressed(t);
+            throw a;
+          }
+          gotExpectedFailure = true;
+          break;
+        }
+      }
+      assertTrue("Safety Valve: " + totalCorruption + " calls to corruptFiles() and " + numIndexOpsSucceded +
+                 " index ops succeded w/o any IndexWriter exceptions?",
+                 gotExpectedFailure);
+      
+    }
+  }
 }


[lucene] 02/04: LUCENE-8692: add additional logging to Solr test that first surfaced bug

Posted by dw...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

dweiss pushed a commit to branch jira/LUCENE-8692
in repository https://gitbox.apache.org/repos/asf/lucene.git

commit 2b28d9d9d0cdc35dfdc202f23a9ef1d27b2a6083
Author: Chris Hostetter <ho...@apache.org>
AuthorDate: Tue Mar 5 16:02:48 2019 -0700

    LUCENE-8692: add additional logging to Solr test that first surfaced bug
---
 solr/core/src/test/org/apache/solr/cloud/LeaderTragicEventTest.java | 4 +++-
 1 file changed, 3 insertions(+), 1 deletion(-)

diff --git a/solr/core/src/test/org/apache/solr/cloud/LeaderTragicEventTest.java b/solr/core/src/test/org/apache/solr/cloud/LeaderTragicEventTest.java
index 9bb44f4..4a6c223 100644
--- a/solr/core/src/test/org/apache/solr/cloud/LeaderTragicEventTest.java
+++ b/solr/core/src/test/org/apache/solr/cloud/LeaderTragicEventTest.java
@@ -84,11 +84,13 @@ public class LeaderTragicEventTest extends SolrCloudTestCase {
 
       waitForState("Timeout waiting for new replica become leader", collection, (liveNodes, collectionState) -> {
         Slice slice = collectionState.getSlice("shard1");
-
+        log.info("Waiting for leader: numReps={} oldLeaderName={} currentLeader={}",
+                 slice.getReplicas().size(), oldLeader.getName(), slice.getLeader());
         if (slice.getReplicas().size() != 2) return false;
         if (slice.getLeader() == null) return false;
         if (slice.getLeader().getName().equals(oldLeader.getName())) return false;
 
+        log.info("New leader found: {}", slice.getLeader().getName());
         return true;
       });
       ClusterStateUtil.waitForAllActiveAndLiveReplicas(cluster.getSolrClient().getZkStateReader(), collection, 120000);