You are viewing a plain text version of this content. The canonical link for it is here.
Posted to solr-commits@lucene.apache.org by ho...@apache.org on 2009/08/26 06:18:23 UTC

svn commit: r807872 - in /lucene/solr/trunk: ./ contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/ src/common/org/apache/solr/common/util/ src/java/org/apache/solr/core/ src/java/org/apache/solr/update/ src/test/org/apache/solr...

Author: hossman
Date: Wed Aug 26 04:18:22 2009
New Revision: 807872

URL: http://svn.apache.org/viewvc?rev=807872&view=rev
Log:
SOLR-914: General finalize() improvements

Modified:
    lucene/solr/trunk/CHANGES.txt
    lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/JdbcDataSource.java
    lucene/solr/trunk/src/common/org/apache/solr/common/util/ConcurrentLRUCache.java
    lucene/solr/trunk/src/java/org/apache/solr/core/CoreContainer.java
    lucene/solr/trunk/src/java/org/apache/solr/update/SolrIndexWriter.java
    lucene/solr/trunk/src/test/org/apache/solr/search/TestFastLRUCache.java

Modified: lucene/solr/trunk/CHANGES.txt
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/CHANGES.txt?rev=807872&r1=807871&r2=807872&view=diff
==============================================================================
--- lucene/solr/trunk/CHANGES.txt (original)
+++ lucene/solr/trunk/CHANGES.txt Wed Aug 26 04:18:22 2009
@@ -502,6 +502,11 @@
     effect QueryElevationComponentTest was refactored, and a bug in
     that test was found. (hossman)
 
+59. SOLR-914: General finalize() improvements.  No finalizer delegates
+    to the respective close/destroy method w/o first checking if it's
+    already been closed/destroyed; if it hasn't a, SEVERE error is
+    logged first.  (noble, hossman)
+
 Other Changes
 ----------------------
  1. Upgraded to Lucene 2.4.0 (yonik)

Modified: lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/JdbcDataSource.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/JdbcDataSource.java?rev=807872&r1=807871&r2=807872&view=diff
==============================================================================
--- lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/JdbcDataSource.java (original)
+++ lucene/solr/trunk/contrib/dataimporthandler/src/main/java/org/apache/solr/handler/dataimport/JdbcDataSource.java Wed Aug 26 04:18:22 2009
@@ -373,16 +373,23 @@
 
   protected void finalize() throws Throwable {
     try {
-      conn.close();
+      if(!isClosed){
+        LOG.error("JdbcDataSource was not closed prior to finalize(), indicates a bug -- POSSIBLE RESOURCE LEAK!!!");
+        close();
+      }
     } finally {
       super.finalize();
     }
   }
 
+  private boolean isClosed = false;
   public void close() {
     try {
       conn.close();
     } catch (Exception e) {
+      LOG.error("Ignoring Error when closing connection", e);
+    } finally{
+      isClosed = true;
     }
 
   }

Modified: lucene/solr/trunk/src/common/org/apache/solr/common/util/ConcurrentLRUCache.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/common/org/apache/solr/common/util/ConcurrentLRUCache.java?rev=807872&r1=807871&r2=807872&view=diff
==============================================================================
--- lucene/solr/trunk/src/common/org/apache/solr/common/util/ConcurrentLRUCache.java (original)
+++ lucene/solr/trunk/src/common/org/apache/solr/common/util/ConcurrentLRUCache.java Wed Aug 26 04:18:22 2009
@@ -17,6 +17,8 @@
  */
 
 import org.apache.lucene.util.PriorityQueue;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 
 import java.util.LinkedHashMap;
 import java.util.Map;
@@ -40,6 +42,7 @@
  * @since solr 1.4
  */
 public class ConcurrentLRUCache<K,V> {
+  private static Logger log = LoggerFactory.getLogger(ConcurrentLRUCache.class);
 
   private final ConcurrentHashMap<Object, CacheEntry> map;
   private final int upperWaterMark, lowerWaterMark;
@@ -490,10 +493,14 @@
     }
   }
 
-
+ private boolean isDestroyed =  false;
   public void destroy() {
-    if(cleanupThread != null){
-      cleanupThread.stopThread();
+    try {
+      if(cleanupThread != null){
+        cleanupThread.stopThread();
+      }
+    } finally {
+      isDestroyed = true;
     }
   }
 
@@ -583,8 +590,11 @@
 
   protected void finalize() throws Throwable {
     try {
-      destroy();
-    } finally { 
+      if(!isDestroyed){
+        log.error("ConcurrentLRUCache was not destroyed prior to finalize(), indicates a bug -- POSSIBLE RESOURCE LEAK!!!");
+        destroy();
+      }
+    } finally {
       super.finalize();
     }
   }

Modified: lucene/solr/trunk/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/core/CoreContainer.java?rev=807872&r1=807871&r2=807872&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/core/CoreContainer.java Wed Aug 26 04:18:22 2009
@@ -271,24 +271,31 @@
     }
     return properties;
   }
-
+  private boolean isShutDown = false;
   /**
    * Stops all cores.
    */
   public void shutdown() {
     synchronized(cores) {
-      for(SolrCore core : cores.values()) {
-        core.close();
+      try {
+        for(SolrCore core : cores.values()) {
+          core.close();
+        }
+        cores.clear();
+      } finally {
+        isShutDown = true;
       }
-      cores.clear();
     }
   }
   
   @Override
   protected void finalize() throws Throwable {
-    try { 
-      shutdown();
-    } finally { 
+    try {
+      if(!isShutDown){
+        log.error("CoreContainer was not shutdown prior to finalize(), indicates a bug -- POSSIBLE RESOURCE LEAK!!!");
+        shutdown();
+      }
+    } finally {
       super.finalize();
     }
   }

Modified: lucene/solr/trunk/src/java/org/apache/solr/update/SolrIndexWriter.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/java/org/apache/solr/update/SolrIndexWriter.java?rev=807872&r1=807871&r2=807872&view=diff
==============================================================================
--- lucene/solr/trunk/src/java/org/apache/solr/update/SolrIndexWriter.java (original)
+++ lucene/solr/trunk/src/java/org/apache/solr/update/SolrIndexWriter.java Wed Aug 26 04:18:22 2009
@@ -213,19 +213,26 @@
    * }
    * ****
    */
-
+  private boolean isClosed = false;
   public void close() throws IOException {
     log.debug("Closing Writer " + name);
-    super.close();
-    if(infoStream != null) {
-      infoStream.close();
+    try {
+      super.close();
+      if(infoStream != null) {
+        infoStream.close();
+      }
+    } finally {
+      isClosed = true;
     }
   }
 
   @Override
   protected void finalize() throws Throwable {
     try {
-      super.close();
+      if(!isClosed){
+        log.error("SolrIndexWriter was not closed prior to finalize(), indicates a bug -- POSSIBLE RESOURCE LEAK!!!");
+        close();
+      }
     } finally { 
       super.finalize();
     }

Modified: lucene/solr/trunk/src/test/org/apache/solr/search/TestFastLRUCache.java
URL: http://svn.apache.org/viewvc/lucene/solr/trunk/src/test/org/apache/solr/search/TestFastLRUCache.java?rev=807872&r1=807871&r2=807872&view=diff
==============================================================================
--- lucene/solr/trunk/src/test/org/apache/solr/search/TestFastLRUCache.java (original)
+++ lucene/solr/trunk/src/test/org/apache/solr/search/TestFastLRUCache.java Wed Aug 26 04:18:22 2009
@@ -51,6 +51,7 @@
     scNew.init(l, o, cr);
     scNew.warm(null, sc);
     scNew.setState(SolrCache.State.LIVE);
+    sc.close();
     scNew.put(103, "103");
     assertEquals("90", scNew.get(90));
     assertEquals(null, scNew.get(50));
@@ -63,6 +64,7 @@
     assertEquals(5L, nl.get("cumulative_lookups"));
     assertEquals(2L, nl.get("cumulative_hits"));
     assertEquals(102L, nl.get("cumulative_inserts"));
+    scNew.close();
   }
 
   public void testOldestItems() {
@@ -79,6 +81,7 @@
     assertNotNull(m.get(5));
     assertNotNull(m.get(4));
     assertNotNull(m.get(2));
+    cache.destroy();
   }
 
   void doPerfTest(int iter, int cacheSize, int maxKey) {
@@ -102,6 +105,7 @@
         else if (sz > maxSize) maxSize=sz;
       }
     }
+    cache.destroy();
 
     long end = System.currentTimeMillis();
     System.out.println("time=" + (end-start) + ", minSize="+minSize+",maxSize="+maxSize);