You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by ma...@apache.org on 2012/07/18 18:19:29 UTC

svn commit: r1363002 - in /lucene/dev/branches/branch_4x: ./ solr/ solr/core/src/java/org/apache/solr/core/ solr/core/src/java/org/apache/solr/handler/ solr/core/src/java/org/apache/solr/update/

Author: markrmiller
Date: Wed Jul 18 16:19:29 2012
New Revision: 1363002

URL: http://svn.apache.org/viewvc?rev=1363002&view=rev
Log:
SOLR-1781: Replication index directories not always cleaned up.

Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/solr/   (props changed)
    lucene/dev/branches/branch_4x/solr/CHANGES.txt
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/SnapPuller.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java

Modified: lucene/dev/branches/branch_4x/solr/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/CHANGES.txt?rev=1363002&r1=1363001&r2=1363002&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/solr/CHANGES.txt Wed Jul 18 16:19:29 2012
@@ -84,9 +84,10 @@ Bug Fixes
 * SOLR-3377: edismax fails to correctly parse a fielded query wrapped by parens.
   This regression was introduced in 3.6.  (Bernd Fehling, Jan Høydahl, yonik)
   
-* SOLR-3621: fix rare concurrency issue when opening a new IndexWriter for replication or rollback.
+* SOLR-3621: Fix rare concurrency issue when opening a new IndexWriter for replication or rollback.
   (Mark Miller)
 
+* SOLR-1781: Replication index directories not always cleaned up. (Terje Sten Bjerkseth, Mark Miller)
 
 Other Changes
 

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java?rev=1363002&r1=1363001&r2=1363002&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CachingDirectoryFactory.java Wed Jul 18 16:19:29 2012
@@ -19,7 +19,9 @@ package org.apache.solr.core;
 
 import java.io.File;
 import java.io.IOException;
+import java.util.ArrayList;
 import java.util.HashMap;
+import java.util.List;
 import java.util.Locale;
 import java.util.Map;
 
@@ -54,6 +56,46 @@ public abstract class CachingDirectoryFa
   
   protected Map<Directory,CacheValue> byDirectoryCache = new HashMap<Directory,CacheValue>();
   
+  protected Map<Directory,List<CloseListener>> closeListeners = new HashMap<Directory,List<CloseListener>>();
+  
+  public interface CloseListener {
+    public void onClose();
+  }
+  
+  @Override
+  public void addCloseListener(Directory dir, CloseListener closeListener) {
+    synchronized (this) {
+      if (!byDirectoryCache.containsKey(dir)) {
+        throw new IllegalArgumentException("Unknown directory: " + dir
+            + " " + byDirectoryCache);
+      }
+      List<CloseListener> listeners = closeListeners.get(dir);
+      if (listeners == null) {
+        listeners = new ArrayList<CloseListener>();
+        closeListeners.put(dir, listeners);
+      }
+      listeners.add(closeListener);
+      
+      closeListeners.put(dir, listeners);
+    }
+  }
+  
+  @Override
+  public void doneWithDirectory(Directory directory) throws IOException {
+    synchronized (this) {
+      CacheValue cacheValue = byDirectoryCache.get(directory);
+      if (cacheValue == null) {
+        throw new IllegalArgumentException("Unknown directory: " + directory
+            + " " + byDirectoryCache);
+      }
+      cacheValue.doneWithDir = true;
+      if (cacheValue.refCnt == 0) {
+        cacheValue.refCnt++; // this will go back to 0 in close
+        close(directory);
+      }
+    }
+  }
+  
   /*
    * (non-Javadoc)
    * 
@@ -82,6 +124,13 @@ public abstract class CachingDirectoryFa
         directory.close();
         byDirectoryCache.remove(directory);
         byPathCache.remove(cacheValue.path);
+        List<CloseListener> listeners = closeListeners.remove(directory);
+        if (listeners != null) {
+          for (CloseListener listener : listeners) {
+            listener.onClose();
+          }
+          closeListeners.remove(directory);
+        }
       }
     }
   }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java?rev=1363002&r1=1363001&r2=1363002&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/DirectoryFactory.java Wed Jul 18 16:19:29 2012
@@ -21,6 +21,7 @@ import java.io.Closeable;
 import java.io.IOException;
 
 import org.apache.lucene.store.Directory;
+import org.apache.solr.core.CachingDirectoryFactory.CloseListener;
 import org.apache.solr.util.plugin.NamedListInitializedPlugin;
 
 /**
@@ -31,6 +32,24 @@ public abstract class DirectoryFactory i
     Closeable {
   
   /**
+   * Indicates a Directory will no longer be used, and when it's ref count
+   * hits 0, it can be closed. On shutdown all directories will be closed
+   * with this has been called or not. This is simply to allow early cleanup.
+   * 
+   * @param directory
+   * @throws IOException 
+   */
+  public abstract void doneWithDirectory(Directory directory) throws IOException;
+  
+  /**
+   * Adds a close listener for a Directory.
+   * 
+   * @param dir
+   * @param closeListener
+   */
+  public abstract void addCloseListener(Directory dir, CloseListener closeListener);
+  
+  /**
    * Close the this and all of the Directories it contains.
    * 
    * @throws IOException
@@ -62,7 +81,9 @@ public abstract class DirectoryFactory i
   /**
    * Returns the Directory for a given path, using the specified rawLockType.
    * Will return the same Directory instance for the same path unless forceNew,
-   * in which case a new Directory is returned.
+   * in which case a new Directory is returned. There is no need to call
+   * {@link #doneWithDirectory(Directory)} in this case - the old Directory
+   * will be closed when it's ref count hits 0.
    * 
    * @throws IOException
    */

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/SnapPuller.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/SnapPuller.java?rev=1363002&r1=1363001&r2=1363002&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/SnapPuller.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/SnapPuller.java Wed Jul 18 16:19:29 2012
@@ -20,6 +20,7 @@ import org.apache.commons.io.IOUtils;
 import org.apache.http.client.HttpClient;
 import org.apache.lucene.index.IndexCommit;
 import org.apache.lucene.index.IndexWriter;
+import org.apache.lucene.store.Directory;
 import org.apache.solr.client.solrj.SolrServer;
 import org.apache.solr.client.solrj.SolrServerException;
 import org.apache.solr.client.solrj.impl.HttpClientUtil;
@@ -32,6 +33,7 @@ import org.apache.solr.common.params.Sol
 import org.apache.solr.common.util.FastInputStream;
 import org.apache.solr.util.FileUtils;
 import org.apache.solr.common.util.NamedList;
+import org.apache.solr.core.CachingDirectoryFactory.CloseListener;
 import org.apache.solr.core.SolrCore;
 import org.apache.solr.core.IndexDeletionPolicyWrapper;
 import static org.apache.solr.handler.ReplicationHandler.*;
@@ -321,9 +323,10 @@ public class SnapPuller {
       LOG.info("Starting download to " + tmpIndexDir + " fullCopy=" + isFullCopyNeeded);
       successfulInstall = false;
       boolean deleteTmpIdxDir = true;
-      File indexDir = null ;
+
+      final File indexDir = new File(core.getIndexDir());
+      Directory oldDirectory = null;
       try {
-        indexDir = new File(core.getIndexDir());
         downloadIndexFiles(isFullCopyNeeded, tmpIndexDir, latestGeneration);
         LOG.info("Total time taken for download : " + ((System.currentTimeMillis() - replicationStartTime) / 1000) + " secs");
         Collection<Map<String, Object>> modifiedConfFiles = getModifiedConfFiles(confFilesToDownload);
@@ -345,14 +348,42 @@ public class SnapPuller {
           if (isFullCopyNeeded) {
             successfulInstall = modifyIndexProps(tmpIndexDir.getName());
             deleteTmpIdxDir =  false;
+            RefCounted<IndexWriter> iw = core.getUpdateHandler().getSolrCoreState().getIndexWriter(core);
+            try {
+               oldDirectory = iw.get().getDirectory();
+            } finally {
+              iw.decref();
+            }
           } else {
             successfulInstall = copyIndexFiles(tmpIndexDir, indexDir);
           }
           if (successfulInstall) {
             logReplicationTimeAndConfFiles(modifiedConfFiles, successfulInstall);
-            doCommit();
           }
         }
+        
+        if (isFullCopyNeeded) {
+          // we have to do this before commit
+          core.getDirectoryFactory().addCloseListener(oldDirectory, new CloseListener(){
+
+            @Override
+            public void onClose() {
+              LOG.info("removing old index directory " + indexDir);
+              delTree(indexDir);
+            }
+            
+          });
+        }
+        
+        if (successfulInstall) {
+          if (isFullCopyNeeded) {
+            // let the system know we are changing dir's and the old one
+            // may be closed
+            core.getDirectoryFactory().doneWithDirectory(oldDirectory);
+          }
+          doCommit();
+        }
+        
         replicationStartTime = 0;
         return successfulInstall;
       } catch (ReplicationHandlerException e) {
@@ -368,10 +399,7 @@ public class SnapPuller {
         if (deleteTmpIdxDir) {
           LOG.info("removing temporary index download directory " + tmpIndexDir);
           delTree(tmpIndexDir);
-        } else {
-          LOG.info("removing old index directory " + indexDir);
-          delTree(indexDir);
-        }
+        } 
       }
     } finally {
       if (!successfulInstall) {

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java?rev=1363002&r1=1363001&r2=1363002&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/DefaultSolrCoreState.java Wed Jul 18 16:19:29 2012
@@ -21,6 +21,7 @@ import java.io.IOException;
 
 import org.apache.lucene.index.IndexWriter;
 import org.apache.solr.cloud.RecoveryStrategy;
+import org.apache.solr.common.SolrException;
 import org.apache.solr.core.CoreContainer;
 import org.apache.solr.core.DirectoryFactory;
 import org.apache.solr.core.SolrCore;
@@ -92,10 +93,16 @@ public final class DefaultSolrCoreState 
         wait();
       } catch (InterruptedException e) {}
     }
-    try {
-      if (indexWriter != null) {
+    
+    if (indexWriter != null) {
+      try {
         indexWriter.close();
+      } catch (Exception e) {
+        SolrException.log(log, "Error closing old IndexWriter", e);
       }
+    }
+    
+    try {
       indexWriter = createMainIndexWriter(core, "DirectUpdateHandler2", false,
           true);
       // we need to null this so it picks up the new writer next get call

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java?rev=1363002&r1=1363001&r2=1363002&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java Wed Jul 18 16:19:29 2012
@@ -122,10 +122,11 @@ public class SolrIndexWriter extends Ind
     final InfoStream infoStream = isClosed ? null : getConfig().getInfoStream();    
     try {
       super.close();
+    } finally {
       if(infoStream != null) {
         infoStream.close();
       }
-    } finally {
+      
       isClosed = true;
 
       directoryFactory.release(directory);