lucene-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From mikemcc...@apache.org
Subject svn commit: r1631244 - in /lucene/dev/branches/lucene_solr_4_10: ./ lucene/ lucene/core/ lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
Date Sun, 12 Oct 2014 19:26:54 GMT
Author: mikemccand
Date: Sun Oct 12 19:26:53 2014
New Revision: 1631244

URL: http://svn.apache.org/r1631244
Log:
don't allow publishFlushSegment after IW is closed

Modified:
    lucene/dev/branches/lucene_solr_4_10/   (props changed)
    lucene/dev/branches/lucene_solr_4_10/lucene/   (props changed)
    lucene/dev/branches/lucene_solr_4_10/lucene/core/   (props changed)
    lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
    lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java

Modified: lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java?rev=1631244&r1=1631243&r2=1631244&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
(original)
+++ lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
Sun Oct 12 19:26:53 2014
@@ -25,9 +25,11 @@ import java.util.ArrayList;
 import java.util.Collection;
 import java.util.Collections;
 import java.util.HashMap;
+import java.util.HashSet;
 import java.util.List;
 import java.util.Map;
 import java.util.Objects;
+import java.util.Set;
 import java.util.regex.Matcher;
 
 import org.apache.lucene.store.AlreadyClosedException;
@@ -79,7 +81,7 @@ final class IndexFileDeleter implements 
   /* Files that we tried to delete but failed (likely
    * because they are open and we are running on Windows),
    * so we will retry them again later: */
-  private List<String> deletable;
+  private Set<String> deletable;
 
   /* Reference count for all files in the index.
    * Counts how many existing commits reference a file.
@@ -361,7 +363,7 @@ final class IndexFileDeleter implements 
    * Remove the CommitPoints in the commitsToDelete List by
    * DecRef'ing all files from each SegmentInfos.
    */
-  private void deleteCommits() throws IOException {
+  private void deleteCommits() {
 
     int size = commitsToDelete.size();
 
@@ -386,7 +388,7 @@ final class IndexFileDeleter implements 
       commitsToDelete.clear();
 
       // NOTE: does nothing if firstThrowable is null
-      IOUtils.reThrow(firstThrowable);
+      IOUtils.reThrowUnchecked(firstThrowable);
 
       // Now compact commits to remove deleted ones (preserving the sort):
       size = commits.size();
@@ -461,7 +463,7 @@ final class IndexFileDeleter implements 
   }
 
   @Override
-  public void close() throws IOException {
+  public void close() {
     // DecRef old files from the last checkpoint, if any:
     assert locked();
 
@@ -497,14 +499,12 @@ final class IndexFileDeleter implements 
     }
   }
 
-  public void deletePendingFiles() throws IOException {
+  public void deletePendingFiles() {
     assert locked();
     if (deletable != null) {
-      List<String> oldDeletable = deletable;
+      Set<String> oldDeletable = deletable;
       deletable = null;
-      int size = oldDeletable.size();
-      for(int i=0;i<size;i++) {
-        String fileName = oldDeletable.get(i);
+      for(String fileName : oldDeletable) {
         if (infoStream.isEnabled("IFD")) {
           infoStream.message("IFD", "delete pending file " + fileName);
         }
@@ -610,7 +610,7 @@ final class IndexFileDeleter implements 
   }
 
   /** Decrefs all provided files, even on exception; throws first exception hit, if any.
*/
-  void decRef(Collection<String> files) throws IOException {
+  void decRef(Collection<String> files) {
     assert locked();
     Throwable firstThrowable = null;
     for(final String file : files) {
@@ -625,12 +625,12 @@ final class IndexFileDeleter implements 
     }
 
     // NOTE: does nothing if firstThrowable is null
-    IOUtils.reThrow(firstThrowable);
+    IOUtils.reThrowUnchecked(firstThrowable);
   }
 
   /** Decrefs all provided files, ignoring any exceptions hit; call this if
    *  you are already handling an exception. */
-  void decRefWhileHandlingException(Collection<String> files) throws IOException {
+  void decRefWhileHandlingException(Collection<String> files) {
     assert locked();
     for(final String file : files) {
       try {
@@ -640,7 +640,7 @@ final class IndexFileDeleter implements 
     }
   }
 
-  void decRef(String fileName) throws IOException {
+  void decRef(String fileName) {
     assert locked();
     RefCount rc = getRefCount(fileName);
     if (infoStream.isEnabled("IFD")) {
@@ -678,6 +678,8 @@ final class IndexFileDeleter implements 
     RefCount rc;
     if (!refCounts.containsKey(fileName)) {
       rc = new RefCount(fileName);
+      // We should never incRef a file we are already wanting to delete:
+      assert deletable == null || deletable.contains(fileName) == false: "file \"" + fileName
+ "\" cannot be incRef'd: it's already pending delete";
       refCounts.put(fileName, rc);
     } else {
       rc = refCounts.get(fileName);
@@ -685,7 +687,7 @@ final class IndexFileDeleter implements 
     return rc;
   }
 
-  void deleteFiles(List<String> files) throws IOException {
+  void deleteFiles(List<String> files) {
     assert locked();
     for(final String file: files) {
       deleteFile(file);
@@ -694,7 +696,7 @@ final class IndexFileDeleter implements 
 
   /** Deletes the specified files, but only if they are new
    *  (have not yet been incref'd). */
-  void deleteNewFiles(Collection<String> files) throws IOException {
+  void deleteNewFiles(Collection<String> files) {
     assert locked();
     for (final String fileName: files) {
       // NOTE: it's very unusual yet possible for the
@@ -712,8 +714,7 @@ final class IndexFileDeleter implements 
     }
   }
 
-  void deleteFile(String fileName)
-       throws IOException {
+  void deleteFile(String fileName) {
     assert locked();
     ensureOpen();
     try {
@@ -733,7 +734,7 @@ final class IndexFileDeleter implements 
         infoStream.message("IFD", "unable to remove file \"" + fileName + "\": " + e.toString()
+ "; Will re-try later.");
       }
       if (deletable == null) {
-        deletable = new ArrayList<>();
+        deletable = new HashSet<>();
       }
       deletable.add(fileName);                  // add to deletable
     }

Modified: lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java?rev=1631244&r1=1631243&r2=1631244&view=diff
==============================================================================
--- lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
(original)
+++ lucene/dev/branches/lucene_solr_4_10/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
Sun Oct 12 19:26:53 2014
@@ -2138,6 +2138,10 @@ public class IndexWriter implements Clos
         deleter.refresh();
         deleter.close();
 
+        // Must set closed while inside same sync block where we call deleter.refresh, else
concurrent threads may try to sneak a flush in,
+        // after we leave this sync block and before we enter the sync block in the finally
clause below that sets closed:
+        closed = true;
+
         IOUtils.close(writeLock);                     // release write lock
         writeLock = null;
         
@@ -2381,6 +2385,7 @@ public class IndexWriter implements Clos
     try {
       synchronized (this) {
         // Lock order IW -> BDS
+        ensureOpen(false);
         synchronized (bufferedUpdatesStream) {
           if (infoStream.isEnabled("IW")) {
             infoStream.message("IW", "publishFlushedSegment");



Mime
View raw message