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 2014/01/13 18:34:33 UTC

svn commit: r1557783 - in /lucene/dev/branches/branch_4x: ./ solr/ solr/contrib/ solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/ solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ solr/core/ solr/co...

Author: markrmiller
Date: Mon Jan 13 17:34:31 2014
New Revision: 1557783

URL: http://svn.apache.org/r1557783
Log:
SOLR-4992: Solr eats OutOfMemoryError exceptions in many cases.

Modified:
    lucene/dev/branches/branch_4x/   (props changed)
    lucene/dev/branches/branch_4x/solr/   (props changed)
    lucene/dev/branches/branch_4x/solr/CHANGES.txt   (contents, props changed)
    lucene/dev/branches/branch_4x/solr/contrib/   (props changed)
    lucene/dev/branches/branch_4x/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/LuceneCarrot2StemmerFactory.java
    lucene/dev/branches/branch_4x/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/LuceneCarrot2TokenizerFactory.java
    lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DataImportHandler.java
    lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DataImporter.java
    lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DocBuilder.java
    lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrWriter.java
    lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ZKPropertiesWriter.java
    lucene/dev/branches/branch_4x/solr/core/   (props changed)
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/SolrLogFormatter.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/Overseer.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/SolrZkServer.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ZkController.java
    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/Config.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolr.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CoreContainer.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/Diagnostics.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCore.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCores.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.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/handler/admin/CoreAdminHandler.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/logging/LogWatcher.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/rest/schema/BaseSchemaResource.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/FastLRUCache.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/LFUCache.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/LRUCache.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.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/DirectUpdateHandler2.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/SolrCoreState.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/SolrIndexWriter.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/UpdateLog.java
    lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
    lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyShardSplitTest.java
    lucene/dev/branches/branch_4x/solr/solrj/   (props changed)
    lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrServer.java
    lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrServer.java
    lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
    lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
    lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.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=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/CHANGES.txt (original)
+++ lucene/dev/branches/branch_4x/solr/CHANGES.txt Mon Jan 13 17:34:31 2014
@@ -164,6 +164,9 @@ Bug Fixes
 * SOLR-5567: ZkController getHostAddress duplicates url prefix.
   (Kyle Halliday, Alexey Serba, shalin)
 
+* SOLR-4992: Solr eats OutOfMemoryError exceptions in many cases.
+  (Mark Miller, Daniel Collins)  
+
 Optimizations
 ----------------------
 

Modified: lucene/dev/branches/branch_4x/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/LuceneCarrot2StemmerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/LuceneCarrot2StemmerFactory.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/LuceneCarrot2StemmerFactory.java (original)
+++ lucene/dev/branches/branch_4x/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/LuceneCarrot2StemmerFactory.java Mon Jan 13 17:34:31 2014
@@ -224,7 +224,7 @@ public class LuceneCarrot2StemmerFactory
     public static IStemmer createStemmer() {
       try {
         return new LuceneStemmerAdapter();
-      } catch (Throwable e) {
+      } catch (Exception e) {
         return IdentityStemmer.INSTANCE;
       }
     }

Modified: lucene/dev/branches/branch_4x/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/LuceneCarrot2TokenizerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/LuceneCarrot2TokenizerFactory.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/LuceneCarrot2TokenizerFactory.java (original)
+++ lucene/dev/branches/branch_4x/solr/contrib/clustering/src/java/org/apache/solr/handler/clustering/carrot2/LuceneCarrot2TokenizerFactory.java Mon Jan 13 17:34:31 2014
@@ -82,6 +82,9 @@ public class LuceneCarrot2TokenizerFacto
             .warn("Could not instantiate Smart Chinese Analyzer, clustering quality "
                 + "of Chinese content may be degraded. For best quality clusters, "
                 + "make sure Lucene's Smart Chinese Analyzer JAR is in the classpath");
+        if (e instanceof Error) {
+          throw (Error) e;
+        }
       }
     }
 
@@ -89,6 +92,9 @@ public class LuceneCarrot2TokenizerFacto
       try {
         return new ChineseTokenizer();
       } catch (Throwable e) {
+        if (e instanceof OutOfMemoryError) {
+          throw (OutOfMemoryError) e;
+        }
         return new ExtendedWhitespaceTokenizer();
       }
     }

Modified: lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DataImportHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DataImportHandler.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DataImportHandler.java (original)
+++ lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DataImportHandler.java Mon Jan 13 17:34:31 2014
@@ -101,7 +101,7 @@ public class DataImportHandler extends R
       }
       debugEnabled = StrUtils.parseBool((String)initArgs.get(ENABLE_DEBUG), true);
       importer = new DataImporter(core, myName);         
-    } catch (Throwable e) {
+    } catch (Exception e) {
       LOG.error( DataImporter.MSG.LOAD_EXP, e);
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, DataImporter.MSG.LOAD_EXP, e);
     }

Modified: lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DataImporter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DataImporter.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DataImporter.java (original)
+++ lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DataImporter.java Mon Jan 13 17:34:31 2014
@@ -402,7 +402,7 @@ public class DataImporter {
   public void doFullImport(SolrWriter writer, RequestInfo requestParams) {
     LOG.info("Starting Full Import");
     setStatus(Status.RUNNING_FULL_DUMP);
-
+    boolean success = false;
     try {
       DIHProperties dihPropWriter = createPropertyWriter();
       setIndexStartTime(dihPropWriter.getCurrentTimestamp());
@@ -411,10 +411,14 @@ public class DataImporter {
       docBuilder.execute();
       if (!requestParams.isDebug())
         cumulativeStatistics.add(docBuilder.importStatistics);
-    } catch (Throwable t) {
-      SolrException.log(LOG, "Full Import failed", t);
-      docBuilder.rollback();
+      success = true;
+    } catch (Exception e) {
+      SolrException.log(LOG, "Full Import failed", e);
     } finally {
+      if (!success) {
+        docBuilder.rollback();
+      }
+      
       setStatus(Status.IDLE);
       DocBuilder.INSTANCE.set(null);
     }
@@ -431,7 +435,7 @@ public class DataImporter {
   public void doDeltaImport(SolrWriter writer, RequestInfo requestParams) {
     LOG.info("Starting Delta Import");
     setStatus(Status.RUNNING_DELTA_DUMP);
-
+    boolean success = false;
     try {
       DIHProperties dihPropWriter = createPropertyWriter();
       setIndexStartTime(dihPropWriter.getCurrentTimestamp());
@@ -440,10 +444,13 @@ public class DataImporter {
       docBuilder.execute();
       if (!requestParams.isDebug())
         cumulativeStatistics.add(docBuilder.importStatistics);
-    } catch (Throwable t) {
-      LOG.error("Delta Import Failed", t);
-      docBuilder.rollback();
+      success = true;
+    } catch (Exception e) {
+      LOG.error("Delta Import Failed", e);
     } finally {
+      if (!success) {
+        docBuilder.rollback();
+      }
       setStatus(Status.IDLE);
       DocBuilder.INSTANCE.set(null);
     }

Modified: lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DocBuilder.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DocBuilder.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DocBuilder.java (original)
+++ lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/DocBuilder.java Mon Jan 13 17:34:31 2014
@@ -538,7 +538,7 @@ public class DocBuilder {
               throw e;
           } else
             throw e;
-        } catch (Throwable t) {
+        } catch (Exception t) {
           if (verboseDebug) {
             getDebugLogger().log(DIHLogLevels.ENTITY_EXCEPTION, epw.getEntity().getName(), t);
           }

Modified: lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrWriter.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrWriter.java (original)
+++ lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/SolrWriter.java Mon Jan 13 17:34:31 2014
@@ -105,8 +105,8 @@ public class SolrWriter extends DIHWrite
     try {
       CommitUpdateCommand commit = new CommitUpdateCommand(req,optimize);
       processor.processCommit(commit);
-    } catch (Throwable t) {
-      log.error("Exception while solr commit.", t);
+    } catch (Exception e) {
+      log.error("Exception while solr commit.", e);
     }
   }
 
@@ -115,8 +115,8 @@ public class SolrWriter extends DIHWrite
     try {
       RollbackUpdateCommand rollback = new RollbackUpdateCommand(req);
       processor.processRollback(rollback);
-    } catch (Throwable t) {
-      log.error("Exception while solr rollback.", t);
+    } catch (Exception e) {
+      log.error("Exception while solr rollback.", e);
     }
   }
 

Modified: lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ZKPropertiesWriter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ZKPropertiesWriter.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ZKPropertiesWriter.java (original)
+++ lucene/dev/branches/branch_4x/solr/contrib/dataimporthandler/src/java/org/apache/solr/handler/dataimport/ZKPropertiesWriter.java Mon Jan 13 17:34:31 2014
@@ -92,7 +92,7 @@ public class ZKPropertiesWriter extends 
       if (data != null) {
         props.load(new StringReader(new String(data, "UTF-8")));
       }
-    } catch (Throwable e) {
+    } catch (Exception e) {
       log.warn(
           "Could not read DIH properties from " + path + " :" + e.getClass(), e);
     }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/SolrLogFormatter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/SolrLogFormatter.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/SolrLogFormatter.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/SolrLogFormatter.java Mon Jan 13 17:34:31 2014
@@ -114,9 +114,9 @@ public class SolrLogFormatter extends Fo
   public String format(LogRecord record) {
     try {
       return _format(record);
-    } catch (Throwable th) {
+    } catch (Exception e) {
       // logging swallows exceptions, so if we hit an exception we need to convert it to a string to see it
-      return "ERROR IN SolrLogFormatter! original message:" + record.getMessage() + "\n\tException: " + SolrException.toStr(th);
+      return "ERROR IN SolrLogFormatter! original message:" + record.getMessage() + "\n\tException: " + SolrException.toStr(e);
     }
   }
 
@@ -415,7 +415,7 @@ sb.append("(group_name=").append(tg.getN
       public void run() {
         try {
           go();
-        } catch (Throwable e) {
+        } catch (Exception e) {
           e.printStackTrace();
         }
       }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ElectionContext.java Mon Jan 13 17:34:31 2014
@@ -209,8 +209,8 @@ final class ShardLeaderElectionContext e
       boolean success = false;
       try {
         success = syncStrategy.sync(zkController, core, leaderProps);
-      } catch (Throwable t) {
-        SolrException.log(log, "Exception while trying to sync", t);
+      } catch (Exception e) {
+        SolrException.log(log, "Exception while trying to sync", e);
         success = false;
       }
       
@@ -268,12 +268,13 @@ final class ShardLeaderElectionContext e
         core.close();
       }
     }
-    
+    boolean success = false;
     try {
       super.runLeaderProcess(weAreReplacement);
-    } catch (Throwable t) {
-      SolrException.log(log, "There was a problem trying to register as the leader", t);
-      cancelElection();
+      success = true;
+    } catch (Exception e) {
+      SolrException.log(log, "There was a problem trying to register as the leader", e);
+  
       try {
         core = cc.getCore(coreName);
         if (core == null) {
@@ -287,9 +288,16 @@ final class ShardLeaderElectionContext e
         // we could not publish ourselves as leader - rejoin election
         rejoinLeaderElection(leaderSeqPath, core);
       } finally {
-        if (core != null) {
-          core.close();
+        try {
+          if (!success) {
+            cancelElection();
+          }
+        } finally {
+          if (core != null) {
+            core.close();
+          }
         }
+        
       }
     }
     
@@ -386,11 +394,7 @@ final class ShardLeaderElectionContext e
     
     cancelElection();
     
-    try {
-      core.getUpdateHandler().getSolrCoreState().doRecovery(cc, core.getCoreDescriptor());
-    } catch (Throwable t) {
-      SolrException.log(log, "Error trying to start recovery", t);
-    }
+    core.getUpdateHandler().getSolrCoreState().doRecovery(cc, core.getCoreDescriptor());
     
     leaderElector.joinElection(this, true);
   }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/Overseer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/Overseer.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/Overseer.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/Overseer.java Mon Jan 13 17:34:31 2014
@@ -963,6 +963,7 @@ public class Overseer {
 
   private String adminPath;
   
+  // overseer not responsible for closing reader
   public Overseer(ShardHandler shardHandler, String adminPath, final ZkStateReader reader) throws KeeperException, InterruptedException {
     this.reader = reader;
     this.shardHandler = shardHandler;
@@ -992,20 +993,24 @@ public class Overseer {
   }
   
   public void close() {
-    if (updaterThread != null) {
-      try {
-        updaterThread.close();
-        updaterThread.interrupt();
-      } catch (Throwable t) {
-        log.error("Error closing updaterThread", t);
+    try {
+      if (updaterThread != null) {
+        try {
+          updaterThread.close();
+          updaterThread.interrupt();
+        } catch (Exception e) {
+          log.error("Error closing updaterThread", e);
+        }
       }
-    }
-    if (ccThread != null) {
-      try {
-        ccThread.close();
-        ccThread.interrupt();
-      } catch (Throwable t) {
-        log.error("Error closing ccThread", t);
+    } finally {
+      
+      if (ccThread != null) {
+        try {
+          ccThread.close();
+          ccThread.interrupt();
+        } catch (Exception e) {
+          log.error("Error closing ccThread", e);
+        }
       }
     }
     updaterThread = null;

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/OverseerCollectionProcessor.java Mon Jan 13 17:34:31 2014
@@ -189,7 +189,7 @@ public class OverseerCollectionProcessor
          } catch (InterruptedException e) {
            Thread.currentThread().interrupt();
            return;
-         } catch (Throwable e) {
+         } catch (Exception e) {
            SolrException.log(log, "", e);
          }
        }
@@ -255,13 +255,13 @@ public class OverseerCollectionProcessor
             + operation);
       }
 
-    } catch (Throwable t) {
+    } catch (Exception e) {
       SolrException.log(log, "Collection " + operation + " of " + operation
-          + " failed", t);
-      results.add("Operation " + operation + " caused exception:", t);
+          + " failed", e);
+      results.add("Operation " + operation + " caused exception:", e);
       SimpleOrderedMap nl = new SimpleOrderedMap();
-      nl.add("msg", t.getMessage());
-      nl.add("rspCode", t instanceof SolrException ? ((SolrException)t).code() : -1);
+      nl.add("msg", e.getMessage());
+      nl.add("rspCode", e instanceof SolrException ? ((SolrException)e).code() : -1);
       results.add("exception", nl);
     } 
     

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/RecoveryStrategy.java Mon Jan 13 17:34:31 2014
@@ -245,15 +245,15 @@ public class RecoveryStrategy extends Th
 
       try {
         doRecovery(core);
-      }  catch (InterruptedException e) {
+      } catch (InterruptedException e) {
         Thread.currentThread().interrupt();
         SolrException.log(log, "", e);
         throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR, "",
             e);
-      } catch (Throwable t) {
-        log.error("", t);
+      } catch (Exception e) {
+        log.error("", e);
         throw new ZooKeeperException(SolrException.ErrorCode.SERVER_ERROR,
-            "", t);
+            "", e);
       }
     } finally {
       if (core != null) core.close();
@@ -282,8 +282,8 @@ public class RecoveryStrategy extends Th
     try {
       recentUpdates = ulog.getRecentUpdates();
       recentVersions = recentUpdates.getVersions(ulog.numRecordsToKeep);
-    } catch (Throwable t) {
-      SolrException.log(log, "Corrupt tlog - ignoring. core=" + coreName, t);
+    } catch (Exception e) {
+      SolrException.log(log, "Corrupt tlog - ignoring. core=" + coreName, e);
       recentVersions = new ArrayList<Long>(0);
     } finally {
       if (recentUpdates != null) {
@@ -311,8 +311,8 @@ public class RecoveryStrategy extends Th
         }
         
         log.info("###### startupVersions=" + startingVersions);
-      } catch (Throwable t) {
-        SolrException.log(log, "Error getting recent versions. core=" + coreName, t);
+      } catch (Exception e) {
+        SolrException.log(log, "Error getting recent versions. core=" + coreName, e);
         recentVersions = new ArrayList<Long>(0);
       }
     }
@@ -331,9 +331,9 @@ public class RecoveryStrategy extends Th
               + coreName);
           firstTime = false; // skip peersync
         }
-      } catch (Throwable t) {
+      } catch (Exception e) {
         SolrException.log(log, "Error trying to get ulog starting operation. core="
-            + coreName, t);
+            + coreName, e);
         firstTime = false; // skip peersync
       }
     }
@@ -449,21 +449,21 @@ public class RecoveryStrategy extends Th
           Thread.currentThread().interrupt();
           log.warn("Recovery was interrupted", e);
           retries = INTERRUPTED;
-        } catch (Throwable t) {
-          SolrException.log(log, "Error while trying to recover", t);
+        } catch (Exception e) {
+          SolrException.log(log, "Error while trying to recover", e);
         } finally {
           if (!replayed) {
             try {
               ulog.dropBufferedUpdates();
-            } catch (Throwable t) {
-              SolrException.log(log, "", t);
+            } catch (Exception e) {
+              SolrException.log(log, "", e);
             }
           }
 
         }
 
-      } catch (Throwable t) {
-        SolrException.log(log, "Error while trying to recover. core=" + coreName, t);
+      } catch (Exception e) {
+        SolrException.log(log, "Error while trying to recover. core=" + coreName, e);
       }
 
       if (!successfulRecovery) {
@@ -486,9 +486,9 @@ public class RecoveryStrategy extends Th
               try {
                 recoveryFailed(core, zkController, baseUrl, coreZkNodeName,
                     core.getCoreDescriptor());
-              } catch (Throwable t) {
+              } catch (Exception e) {
                 SolrException.log(log,
-                    "Could not publish that recovery failed", t);
+                    "Could not publish that recovery failed", e);
               }
             } else {
               SolrException.log(log,
@@ -496,15 +496,15 @@ public class RecoveryStrategy extends Th
               try {
                 recoveryFailed(core, zkController, baseUrl, coreZkNodeName,
                     core.getCoreDescriptor());
-              } catch (Throwable t) {
+              } catch (Exception e) {
                 SolrException.log(log,
-                    "Could not publish that recovery failed", t);
+                    "Could not publish that recovery failed", e);
               }
             }
             break;
           }
 
-        } catch (Throwable e) {
+        } catch (Exception e) {
           SolrException.log(log, "core=" + coreName, e);
         }
 

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/SolrZkServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/SolrZkServer.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/SolrZkServer.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/SolrZkServer.java Mon Jan 13 17:34:31 2014
@@ -121,7 +121,7 @@ public class SolrZkServer {
             zkServer.runFromConfig(sc);
           }
           log.info("ZooKeeper Server exited.");
-        } catch (Throwable e) {
+        } catch (Exception e) {
           log.error("ZooKeeper Server ERROR", e);
           throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, e);
         }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/SyncStrategy.java Mon Jan 13 17:34:31 2014
@@ -211,8 +211,8 @@ public class SyncStrategy {
            } else {
              requestRecovery(leaderProps, ((ShardCoreRequest)srsp.getShardRequest()).baseUrl, ((ShardCoreRequest)srsp.getShardRequest()).coreName);
            }
-         } catch (Throwable t) {
-           SolrException.log(log, ZkCoreNodeProps.getCoreUrl(leaderProps) + ": Could not tell a replica to recover", t);
+         } catch (Exception e) {
+           SolrException.log(log, ZkCoreNodeProps.getCoreUrl(leaderProps) + ": Could not tell a replica to recover", e);
          }
       } else {
         log.info(ZkCoreNodeProps.getCoreUrl(leaderProps) + ": " + " sync completed with " + srsp.getShardAddress());
@@ -278,6 +278,9 @@ public class SyncStrategy {
           server.request(recoverRequestCmd);
         } catch (Throwable t) {
           SolrException.log(log, ZkCoreNodeProps.getCoreUrl(leaderProps) + ": Could not tell a replica to recover", t);
+          if (t instanceof Error) {
+            throw (Error) t;
+          }
         } finally {
           server.shutdown();
         }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ZkController.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ZkController.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ZkController.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/cloud/ZkController.java Mon Jan 13 17:34:31 2014
@@ -249,8 +249,8 @@ public final class ZkController {
                   // with connection loss
                   try {
                     register(descriptor.getName(), descriptor, true, true);
-                  } catch (Throwable t) {
-                    SolrException.log(log, "Error registering SolrCore", t);
+                  } catch (Exception e) {
+                    SolrException.log(log, "Error registering SolrCore", e);
                   }
                 }
               }
@@ -385,33 +385,38 @@ public final class ZkController {
    */
   public void close() {
     this.isClosed = true;
-    
-    for (ElectionContext context : electionContexts.values()) {
+    try {
+      for (ElectionContext context : electionContexts.values()) {
+        try {
+          context.close();
+        } catch (Exception e) {
+          log.error("Error closing overseer", e);
+        }
+      }
+    } finally {
       try {
-        context.close();
-      } catch (Throwable t) {
-        log.error("Error closing overseer", t);
+        try {
+          overseer.close();
+        } catch (Exception e) {
+          log.error("Error closing overseer", e);
+        }
+      } finally {
+        try {
+          try {
+            zkStateReader.close();
+          } catch (Exception e) {
+            log.error("Error closing zkStateReader", e);
+          }
+        } finally {
+          try {
+            zkClient.close();
+          } catch (Exception e) {
+            log.error("Error closing zkClient", e);
+          }
+        }
       }
     }
     
-    try {
-      overseer.close();
-    } catch(Throwable t) {
-      log.error("Error closing overseer", t);
-    }
-    
-    try {
-      zkStateReader.close();
-    } catch(Throwable t) {
-      log.error("Error closing zkStateReader", t);
-    } 
-    
-    try {
-      zkClient.close();;
-    } catch(Throwable t) {
-      log.error("Error closing zkClient", t);
-    } 
-    
   }
 
   /**
@@ -473,7 +478,7 @@ public final class ZkController {
               }
             }
           }
-        } catch (Throwable e) {
+        } catch (Exception e) {
           SolrException.log(log,
               "Error while looking for a better host name than 127.0.0.1", e);
         }

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=1557783&r1=1557782&r2=1557783&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 Mon Jan 13 17:34:31 2014
@@ -186,8 +186,8 @@ public abstract class CachingDirectoryFa
             }
           }
           assert val.refCnt == 0 : val.refCnt;
-        } catch (Throwable t) {
-          SolrException.log(log, "Error closing directory", t);
+        } catch (Exception e) {
+          SolrException.log(log, "Error closing directory", e);
         }
       }
       
@@ -203,8 +203,8 @@ public abstract class CachingDirectoryFa
               closedDirs.add(v);
             }
           }
-        } catch (Throwable t) {
-          SolrException.log(log, "Error closing directory", t);
+        } catch (Exception e) {
+          SolrException.log(log, "Error closing directory", e);
         }
       }
 
@@ -212,8 +212,8 @@ public abstract class CachingDirectoryFa
         log.info("Removing directory after core close: " + val.path);
         try {
           removeDirectory(val);
-        } catch (Throwable t) {
-          SolrException.log(log, "Error removing directory", t);
+        } catch (Exception e) {
+          SolrException.log(log, "Error removing directory", e);
         }
       }
       
@@ -238,8 +238,8 @@ public abstract class CachingDirectoryFa
       for (CloseListener listener : listeners) {
         try {
           listener.preClose();
-        } catch (Throwable t) {
-          SolrException.log(log, "Error executing preClose for directory", t);
+        } catch (Exception e) {
+          SolrException.log(log, "Error executing preClose for directory", e);
         }
       }
     }
@@ -279,8 +279,8 @@ public abstract class CachingDirectoryFa
         log.info("Removing directory before core close: " + val.path);
         try {
           removeDirectory(val);
-        } catch (Throwable t) {
-          SolrException.log(log, "Error removing directory", t);
+        } catch (Exception e) {
+          SolrException.log(log, "Error removing directory", e);
         }
       } else {
         removeEntries.add(val);
@@ -291,8 +291,8 @@ public abstract class CachingDirectoryFa
       for (CloseListener listener : listeners) {
         try {
           listener.postClose();
-        } catch (Throwable t) {
-          SolrException.log(log, "Error executing postClose for directory", t);
+        } catch (Exception e) {
+          SolrException.log(log, "Error executing postClose for directory", e);
         }
       }
     }
@@ -303,8 +303,8 @@ public abstract class CachingDirectoryFa
     try {
       log.info("Closing directory: " + val.path);
       val.directory.close();
-    } catch (Throwable t) {
-      SolrException.log(log, "Error closing directory", t);
+    } catch (Exception e) {
+      SolrException.log(log, "Error closing directory", e);
     }
   }
 

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/Config.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/Config.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/Config.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/Config.java Mon Jan 13 17:34:31 2014
@@ -260,7 +260,7 @@ public class Config {
       throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,"Error in xpath:" + xstr + " for " + name,e);
     } catch (SolrException e) {
       throw(e);
-    } catch (Throwable e) {
+    } catch (Exception e) {
       SolrException.log(log,"Error in xpath",e);
       throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,"Error in xpath:" + xstr+ " for " + name,e);
     }
@@ -290,7 +290,7 @@ public class Config {
       throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,"Error in xpath:" + xstr + " for " + name,e);
     } catch (SolrException e) {
       throw(e);
-    } catch (Throwable e) {
+    } catch (Exception e) {
       SolrException.log(log,"Error in xpath",e);
       throw new SolrException( SolrException.ErrorCode.SERVER_ERROR,"Error in xpath:" + xstr+ " for " + name,e);
     }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolr.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolr.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolr.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/ConfigSolr.java Mon Jan 13 17:34:31 2014
@@ -19,6 +19,7 @@ package org.apache.solr.core;
 
 import com.google.common.base.Charsets;
 import com.google.common.io.ByteStreams;
+
 import org.apache.commons.io.IOUtils;
 import org.apache.solr.common.SolrException;
 import org.apache.solr.logging.LogWatcherConfig;
@@ -33,6 +34,7 @@ import org.xml.sax.InputSource;
 import javax.xml.xpath.XPath;
 import javax.xml.xpath.XPathConstants;
 import javax.xml.xpath.XPathExpressionException;
+
 import java.io.ByteArrayInputStream;
 import java.io.ByteArrayOutputStream;
 import java.io.File;
@@ -292,7 +294,7 @@ public abstract class ConfigSolr {
     try {
       return readProperties(((NodeList) config.evaluate(
           path, XPathConstants.NODESET)).item(0));
-    } catch (Throwable e) {
+    } catch (Exception e) {
       SolrException.log(log, null, e);
     }
     return null;

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CoreContainer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CoreContainer.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CoreContainer.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/CoreContainer.java Mon Jan 13 17:34:31 2014
@@ -257,21 +257,24 @@ public class CoreContainer {
                   }
                   c = create(cd);
                   registerCore(cd.isTransient(), name, c, false, false);
-                } catch (Throwable t) {
+                } catch (Exception e) {
+                  SolrException.log(log, null, e);
+                  try {
               /*    if (isZooKeeperAware()) {
                     try {
                       zkSys.zkController.unregister(name, cd);
-                    } catch (InterruptedException e) {
+                    } catch (InterruptedException e2) {
                       Thread.currentThread().interrupt();
-                      SolrException.log(log, null, e);
-                    } catch (KeeperException e) {
-                      SolrException.log(log, null, e);
+                      SolrException.log(log, null, e2);
+                    } catch (KeeperException e3) {
+                      SolrException.log(log, null, e3);
                     }
                   }*/
-                  SolrException.log(log, null, t);
-                  if (c != null) {
-                    c.close();
-                  }
+                  } finally {
+                    if (c != null) {
+                      c.close();
+                    }
+                  }            
                 }
                 return c;
               }
@@ -279,8 +282,8 @@ public class CoreContainer {
             pending.add(completionService.submit(task));
 
           }
-        } catch (Throwable ex) {
-          SolrException.log(log, null, ex);
+        } catch (Exception e) {
+          SolrException.log(log, null, e);
         }
       }
 
@@ -416,8 +419,8 @@ public class CoreContainer {
     for (SolrCore core : cores) {
       try {
         core.getSolrCoreState().cancelRecovery();
-      } catch (Throwable t) {
-        SolrException.log(log, "Error canceling recovery for core", t);
+      } catch (Exception e) {
+        SolrException.log(log, "Error canceling recovery for core", e);
       }
     }
   }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/Diagnostics.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/Diagnostics.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/Diagnostics.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/Diagnostics.java Mon Jan 13 17:34:31 2014
@@ -33,8 +33,8 @@ public class Diagnostics {
   public static void call(Callable callable, Object... data) {
     try {
       callable.call(data);
-    } catch (Throwable th) {
-      log.error("TEST HOOK EXCEPTION", th);
+    } catch (Exception e) {
+      log.error("TEST HOOK EXCEPTION", e);
     }
   }
 

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCore.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCore.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCore.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCore.java Mon Jan 13 17:34:31 2014
@@ -833,7 +833,11 @@ public final class SolrCore implements S
     } catch (Throwable e) {
       latch.countDown();//release the latch, otherwise we block trying to do the close.  This should be fine, since counting down on a latch of 0 is still fine
       //close down the searcher and any other resources, if it exists, as this is not recoverable
+      if (e instanceof OutOfMemoryError) {
+        throw (OutOfMemoryError)e;
+      }
       close();
+      
       throw new SolrException(SolrException.ErrorCode.SERVER_ERROR, 
                               e.getMessage(), e);
     } finally {
@@ -1001,7 +1005,10 @@ public final class SolrCore implements S
          try {
            hook.preClose( this );
          } catch (Throwable e) {
-           SolrException.log(log, e);           
+           SolrException.log(log, e);       
+           if (e instanceof Error) {
+             throw (Error) e;
+           }
          }
       }
     }
@@ -1011,6 +1018,9 @@ public final class SolrCore implements S
       infoRegistry.clear();
     } catch (Throwable e) {
       SolrException.log(log, e);
+      if (e instanceof Error) {
+        throw (Error) e;
+      }
     }
 
     try {
@@ -1019,6 +1029,9 @@ public final class SolrCore implements S
       }
     } catch (Throwable e) {
       SolrException.log(log,e);
+      if (e instanceof Error) {
+        throw (Error) e;
+      }
     }
     
     boolean coreStateClosed = false;
@@ -1032,12 +1045,18 @@ public final class SolrCore implements S
       }
     } catch (Throwable e) {
       SolrException.log(log, e);
+      if (e instanceof Error) {
+        throw (Error) e;
+      }
     }
     
     try {
       ExecutorUtil.shutdownAndAwaitTermination(searcherExecutor);
     } catch (Throwable e) {
       SolrException.log(log, e);
+      if (e instanceof Error) {
+        throw (Error) e;
+      }
     }
 
     try {
@@ -1051,14 +1070,20 @@ public final class SolrCore implements S
       closeSearcher();
     } catch (Throwable e) {
       SolrException.log(log,e);
+      if (e instanceof Error) {
+        throw (Error) e;
+      }
     }
     
     if (coreStateClosed) {
       
       try {
         directoryFactory.close();
-      } catch (Throwable t) {
-        SolrException.log(log, t);
+      } catch (Throwable e) {
+        SolrException.log(log, e);
+        if (e instanceof Error) {
+          throw (Error) e;
+        }
       }
       
     }
@@ -1070,6 +1095,9 @@ public final class SolrCore implements S
            hook.postClose( this );
          } catch (Throwable e) {
            SolrException.log(log, e);
+           if (e instanceof Error) {
+             throw (Error) e;
+           }
          }
       }
     }
@@ -1644,6 +1672,9 @@ public final class SolrCore implements S
                   newSearcher.warm(currSearcher);
                 } catch (Throwable e) {
                   SolrException.log(log,e);
+                  if (e instanceof Error) {
+                    throw (Error) e;
+                  }
                 }
                 return null;
               }
@@ -1662,6 +1693,9 @@ public final class SolrCore implements S
                   }
                 } catch (Throwable e) {
                   SolrException.log(log,null,e);
+                  if (e instanceof Error) {
+                    throw (Error) e;
+                  }
                 }
                 return null;
               }
@@ -1680,6 +1714,9 @@ public final class SolrCore implements S
                   }
                 } catch (Throwable e) {
                   SolrException.log(log,null,e);
+                  if (e instanceof Error) {
+                    throw (Error) e;
+                  }
                 }
                 return null;
               }
@@ -1701,6 +1738,9 @@ public final class SolrCore implements S
                   registerSearcher(newSearchHolder);
                 } catch (Throwable e) {
                   SolrException.log(log, e);
+                  if (e instanceof Error) {
+                    throw (Error) e;
+                  }
                 } finally {
                   // we are all done with the old searcher we used
                   // for warming...
@@ -1776,7 +1816,7 @@ public final class SolrCore implements S
             searcherList.remove(this);
           }
           resource.close();
-        } catch (Throwable e) {
+        } catch (Exception e) {
           // do not allow decref() operations to fail since they are typically called in finally blocks
           // and throwing another exception would be very unexpected.
           SolrException.log(log, "Error closing searcher:" + this, e);
@@ -1824,7 +1864,7 @@ public final class SolrCore implements S
         newSearcher.register(); // register subitems (caches)
         log.info(logid+"Registered new searcher " + newSearcher);
 
-      } catch (Throwable e) {
+      } catch (Exception e) {
         // an exception in register() shouldn't be fatal.
         log(e);
       } finally {
@@ -1869,8 +1909,8 @@ public final class SolrCore implements S
     // if (req.getParams().getBool(ShardParams.IS_SHARD,false) && !(handler instanceof SearchHandler))
     //   throw new SolrException(SolrException.ErrorCode.BAD_REQUEST,"isShard is only acceptable with search handlers");
 
-    handler.handleRequest(req,rsp);
 
+    handler.handleRequest(req,rsp);
     postDecorateResponse(handler, req, rsp);
 
     if (log.isInfoEnabled() && rsp.getToLog().size() > 0) {

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCores.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCores.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCores.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrCores.java Mon Jan 13 17:34:31 2014
@@ -118,8 +118,11 @@ class SolrCores {
       for (SolrCore core : coreList) {
         try {
           core.close();
-        } catch (Throwable t) {
-          SolrException.log(CoreContainer.log, "Error shutting down core", t);
+        } catch (Throwable e) {
+          SolrException.log(CoreContainer.log, "Error shutting down core", e);
+          if (e instanceof Error) {
+            throw (Error) e;
+          }
         }
       }
     } while (coreList.size() > 0);

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/core/SolrResourceLoader.java Mon Jan 13 17:34:31 2014
@@ -675,8 +675,8 @@ public class SolrResourceLoader implemen
     for (SolrInfoMBean bean : arr) {
       try {
         infoRegistry.put(bean.getName(), bean);
-      } catch (Throwable t) {
-        log.warn("could not register MBean '" + bean.getName() + "'.", t);
+      } catch (Exception e) {
+        log.warn("could not register MBean '" + bean.getName() + "'.", e);
       }
     }
   }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/PingRequestHandler.java Mon Jan 13 17:34:31 2014
@@ -253,8 +253,8 @@ public class PingRequestHandler extends 
       core.execute(handler, req, pingrsp );
       ex = pingrsp.getException();
     }
-    catch( Throwable th ) {
-      ex = th;
+    catch( Exception e ) {
+      ex = e;
     }
     
     // Send an error or an 'OK' message (response code will be 200)

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=1557783&r1=1557782&r2=1557783&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 Mon Jan 13 17:34:31 2014
@@ -1356,10 +1356,10 @@ public class SnapPuller {
           is = new InflaterInputStream(is);
         }
         return new FastInputStream(is);
-      } catch (Throwable t) {
+      } catch (Exception e) {
         //close stream on error
         IOUtils.closeQuietly(is);
-        throw new IOException("Could not download file '" + fileName + "'", t);
+        throw new IOException("Could not download file '" + fileName + "'", e);
       } finally {
         s.shutdown();
       }
@@ -1620,10 +1620,10 @@ public class SnapPuller {
           is = new InflaterInputStream(is);
         }
         return new FastInputStream(is);
-      } catch (Throwable t) {
+      } catch (Exception e) {
         //close stream on error
         IOUtils.closeQuietly(is);
-        throw new IOException("Could not download file '" + fileName + "'", t);
+        throw new IOException("Could not download file '" + fileName + "'", e);
       } finally {
         s.shutdown();
       }
@@ -1683,19 +1683,13 @@ public class SnapPuller {
   public void destroy() {
     try {
       if (executorService != null) executorService.shutdown();
-    } catch (Throwable e) {
-      SolrException.log(LOG, e);
-    }
-    try {
-      abortPull();
-    } catch (Throwable e) {
-      SolrException.log(LOG, e);
-    }
-    try {
-      if (executorService != null) ExecutorUtil
-          .shutdownNowAndAwaitTermination(executorService);
-    } catch (Throwable e) {
-      SolrException.log(LOG, e);
+    } finally {
+      try {
+        abortPull();
+      } finally {
+        if (executorService != null) ExecutorUtil
+            .shutdownNowAndAwaitTermination(executorService);
+      }
     }
   }
 

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/admin/CoreAdminHandler.java Mon Jan 13 17:34:31 2014
@@ -766,8 +766,11 @@ public class CoreAdminHandler extends Re
             }  catch (InterruptedException e) {
               Thread.currentThread().interrupt();
               SolrException.log(log, "", e);
-            } catch (Throwable t) {
-              SolrException.log(log, "", t);
+            } catch (Throwable e) {
+              SolrException.log(log, "", e);
+              if (e instanceof Error) {
+                throw (Error) e;
+              }
             }
             
             core.getUpdateHandler().getSolrCoreState().doRecovery(coreContainer, core.getCoreDescriptor());
@@ -1005,7 +1008,7 @@ public class CoreAdminHandler extends Re
     } catch (InterruptedException e) {
       Thread.currentThread().interrupt();
       log.warn("Recovery was interrupted", e);
-    } catch (Throwable e) {
+    } catch (Exception e) {
       if (e instanceof SolrException)
         throw (SolrException)e;
       else

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/admin/SystemInfoHandler.java Mon Jan 13 17:34:31 2014
@@ -177,7 +177,7 @@ public class SystemInfoHandler extends R
         info.add( "uptime", execute( "uptime" ) );
       }
     }
-    catch( Throwable ex ) {
+    catch( Exception ex ) {
       ex.printStackTrace();
     } 
     return info;

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandler.java Mon Jan 13 17:34:31 2014
@@ -165,7 +165,7 @@ public class HttpShardHandler extends Sh
         }
         catch( ConnectException cex ) {
           srsp.setException(cex); //????
-        } catch (Throwable th) {
+        } catch (Exception th) {
           srsp.setException(th);
           if (th instanceof SolrException) {
             srsp.setResponseCode(((SolrException)th).code());

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/handler/component/HttpShardHandlerFactory.java Mon Jan 13 17:34:31 2014
@@ -175,25 +175,18 @@ public class HttpShardHandlerFactory ext
   public void close() {
     try {
       ExecutorUtil.shutdownNowAndAwaitTermination(commExecutor);
-    } catch (Throwable e) {
-      SolrException.log(log, e);
-    }
-    
-    try {
-      if(defaultClient != null) {
-        defaultClient.getConnectionManager().shutdown();
-      }
-    } catch (Throwable e) {
-      SolrException.log(log, e);
-    }
-    try {
-      if(loadbalancer != null) {
-        loadbalancer.shutdown();
+    } finally {
+      try {
+        if (defaultClient != null) {
+          defaultClient.getConnectionManager().shutdown();
+        }
+      } finally {
+        
+        if (loadbalancer != null) {
+          loadbalancer.shutdown();
+        }
       }
-    } catch (Throwable e) {
-      SolrException.log(log, e);
     }
-
   }
 
   /**

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/logging/LogWatcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/logging/LogWatcher.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/logging/LogWatcher.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/logging/LogWatcher.java Mon Jan 13 17:34:31 2014
@@ -161,6 +161,9 @@ public abstract class LogWatcher<E> {
     }
     catch (Throwable e) {
       log.warn("Unable to read SLF4J version.  LogWatcher will be disabled: " + e);
+      if (e instanceof OutOfMemoryError) {
+        throw (OutOfMemoryError) e;
+      }
       return null;
     }
 
@@ -179,6 +182,9 @@ public abstract class LogWatcher<E> {
     }
     catch (Throwable e) {
       log.warn("Unable to load LogWatcher {}: {}", fname, e);
+      if (e instanceof OutOfMemoryError) {
+        throw (OutOfMemoryError) e;
+      }
     }
 
     return null;

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/request/SolrRequestInfo.java Mon Jan 13 17:34:31 2014
@@ -64,8 +64,8 @@ public class SolrRequestInfo {
         for (Closeable hook : info.closeHooks) {
           try {
             hook.close();
-          } catch (Throwable throwable) {
-            SolrException.log(SolrCore.log, "Exception during close hook", throwable);
+          } catch (Exception e) {
+            SolrException.log(SolrCore.log, "Exception during close hook", e);
           }
         }
       }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/rest/schema/BaseSchemaResource.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/rest/schema/BaseSchemaResource.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/rest/schema/BaseSchemaResource.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/rest/schema/BaseSchemaResource.java Mon Jan 13 17:34:31 2014
@@ -134,6 +134,9 @@ abstract class BaseSchemaResource extend
           }
         }
       } catch (Throwable t) {
+        if (t instanceof OutOfMemoryError) {
+          throw (OutOfMemoryError) t;
+        }
         setExisting(false);
         throw new ResourceException(t);
       }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/FastLRUCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/FastLRUCache.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/FastLRUCache.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/FastLRUCache.java Mon Jan 13 17:34:31 2014
@@ -160,7 +160,7 @@ public class FastLRUCache<K,V> extends S
                   this, old, itemsArr[i].getKey(), itemsArr[i].getValue());
           if (!continueRegen) break;
         }
-        catch (Throwable e) {
+        catch (Exception e) {
           SolrException.log(log, "Error during auto-warming of key:" + itemsArr[i].getKey(), e);
         }
       }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/LFUCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/LFUCache.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/LFUCache.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/LFUCache.java Mon Jan 13 17:34:31 2014
@@ -182,7 +182,7 @@ public class LFUCache<K, V> implements S
           boolean continueRegen = regenerator.regenerateItem(searcher,
               this, old, itemsArr[i].getKey(), itemsArr[i].getValue());
           if (!continueRegen) break;
-        } catch (Throwable e) {
+        } catch (Exception e) {
           SolrException.log(log, "Error during auto-warming of key:" + itemsArr[i].getKey(), e);
         }
       }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/LRUCache.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/LRUCache.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/LRUCache.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/LRUCache.java Mon Jan 13 17:34:31 2014
@@ -188,7 +188,7 @@ public class LRUCache<K,V> extends SolrC
           boolean continueRegen = regenerator.regenerateItem(searcher, this, old, keys[i], vals[i]);
           if (!continueRegen) break;
         }
-        catch (Throwable e) {
+        catch (Exception e) {
           SolrException.log(log,"Error during auto-warming of key:" + keys[i], e);
         }
       }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/search/SolrIndexSearcher.java Mon Jan 13 17:34:31 2014
@@ -174,9 +174,9 @@ public class SolrIndexSearcher extends I
     Directory dir = directoryFactory.get(path, DirContext.DEFAULT, config.lockType);
     try {
       reader = core.getIndexReaderFactory().newReader(dir, core);
-    } catch (Throwable t) {
+    } catch (Exception e) {
       directoryFactory.release(dir);
-      throw new SolrException(ErrorCode.SERVER_ERROR, "Error opening Reader", t);
+      throw new SolrException(ErrorCode.SERVER_ERROR, "Error opening Reader", e);
     }
     return reader;
   }
@@ -346,8 +346,8 @@ public class SolrIndexSearcher extends I
     long cpg = reader.getIndexCommit().getGeneration();
     try {
       if (closeReader) reader.decRef();
-    } catch (Throwable t) {
-      SolrException.log(log, "Problem dec ref'ing reader", t);
+    } catch (Exception e) {
+      SolrException.log(log, "Problem dec ref'ing reader", e);
     }
 
     if (directoryFactory.searchersReserveCommitPoints()) {

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/servlet/SolrDispatchFilter.java Mon Jan 13 17:34:31 2014
@@ -136,6 +136,9 @@ public class SolrDispatchFilter implemen
       // catch this so our filter still works
       log.error( "Could not start Solr. Check solr/home property and the logs");
       SolrCore.log( t );
+      if (t instanceof Error) {
+        throw (Error) t;
+      }
     }
 
     log.info("SolrDispatchFilter.init() done");
@@ -434,17 +437,25 @@ public class SolrDispatchFilter implemen
       } 
       catch (Throwable ex) {
         sendError( core, solrReq, request, (HttpServletResponse)response, ex );
-        return;
-      } 
-      finally {
-        if( solrReq != null ) {
-          log.debug("Closing out SolrRequest: {}", solrReq);
-          solrReq.close();
+        if (ex instanceof Error) {
+          throw (Error) ex;
         }
-        if (core != null) {
-          core.close();
+        return;
+      } finally {
+        try {
+          if (solrReq != null) {
+            log.debug("Closing out SolrRequest: {}", solrReq);
+            solrReq.close();
+          }
+        } finally {
+          try {
+            if (core != null) {
+              core.close();
+            }
+          } finally {
+            SolrRequestInfo.clearRequestInfo();
+          }
         }
-        SolrRequestInfo.clearRequestInfo();        
       }
     }
 
@@ -758,6 +769,7 @@ public class SolrDispatchFilter implemen
       ServletRequest request, 
       HttpServletResponse response, 
       Throwable ex) throws IOException {
+    Exception exp = null;
     SolrCore localCore = null;
     try {
       SolrQueryResponse solrResp = new SolrQueryResponse();
@@ -786,15 +798,21 @@ public class SolrDispatchFilter implemen
       QueryResponseWriter writer = core.getQueryResponseWriter(req);
       writeResponse(solrResp, response, writer, req, Method.GET);
     }
-    catch( Throwable t ) { // This error really does not matter
-      SimpleOrderedMap info = new SimpleOrderedMap();
-      int code = ResponseUtils.getErrorInfo(ex, info, log);
-      response.sendError( code, info.toString() );
+    catch (Exception e) { // This error really does not matter
+         exp = e;
     } finally {
-      if (core == null && localCore != null) {
-        localCore.close();
+      try {
+        if (exp != null) {
+          SimpleOrderedMap info = new SimpleOrderedMap();
+          int code = ResponseUtils.getErrorInfo(ex, info, log);
+          response.sendError(code, info.toString());
+        }
+      } finally {
+        if (core == null && localCore != null) {
+          localCore.close();
+        }
       }
-    }
+   }
   }
 
   //---------------------------------------------------------------------

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=1557783&r1=1557782&r2=1557783&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 Mon Jan 13 17:34:31 2014
@@ -71,8 +71,8 @@ public final class DefaultSolrCoreState 
         indexWriter.close();
       }
       indexWriter = null;
-    } catch (Throwable t) {
-      log.error("Error during shutdown of writer.", t);
+    } catch (Exception e) {
+      log.error("Error during shutdown of writer.", e);
     } 
   }
   
@@ -162,17 +162,17 @@ public final class DefaultSolrCoreState 
             try {
               log.info("Closing old IndexWriter... core=" + coreName);
               indexWriter.close();
-            } catch (Throwable t) {
+            } catch (Exception e) {
               SolrException.log(log, "Error closing old IndexWriter. core="
-                  + coreName, t);
+                  + coreName, e);
             }
           } else {
             try {
               log.info("Rollback old IndexWriter... core=" + coreName);
               indexWriter.rollback();
-            } catch (Throwable t) {
+            } catch (Exception e) {
               SolrException.log(log, "Error rolling back old IndexWriter. core="
-                  + coreName, t);
+                  + coreName, e);
             }
           }
         }
@@ -220,17 +220,17 @@ public final class DefaultSolrCoreState 
           try {
             log.info("Closing old IndexWriter... core=" + coreName);
             indexWriter.close();
-          } catch (Throwable t) {
+          } catch (Exception e) {
             SolrException.log(log, "Error closing old IndexWriter. core="
-                + coreName, t);
+                + coreName, e);
           }
         } else {
           try {
             log.info("Rollback old IndexWriter... core=" + coreName);
             indexWriter.rollback();
-          } catch (Throwable t) {
+          } catch (Exception e) {
             SolrException.log(log, "Error rolling back old IndexWriter. core="
-                + coreName, t);
+                + coreName, e);
           }
         }
       }

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/DirectUpdateHandler2.java Mon Jan 13 17:34:31 2014
@@ -751,6 +751,9 @@ public class DirectUpdateHandler2 extend
         }
       } catch (Throwable th) {
         log.error("Error in final commit", th);
+        if (th instanceof OutOfMemoryError) {
+          throw (OutOfMemoryError) th;
+        }
       }
 
       // we went through the normal process to commit, so we don't have to artificially
@@ -759,6 +762,9 @@ public class DirectUpdateHandler2 extend
         if (ulog != null) ulog.close(false);
       }  catch (Throwable th) {
         log.error("Error closing log files", th);
+        if (th instanceof OutOfMemoryError) {
+          throw (OutOfMemoryError) th;
+        }
       }
 
       if (writer != null) writer.close();

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/SolrCoreState.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/SolrCoreState.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/SolrCoreState.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/SolrCoreState.java Mon Jan 13 17:34:31 2014
@@ -70,8 +70,8 @@ public abstract class SolrCoreState {
       try {
         log.info("Closing SolrCoreState");
         close(closer);
-      } catch (Throwable t) {
-        log.error("Error closing SolrCoreState", t);
+      } catch (Exception e) {
+        log.error("Error closing SolrCoreState", e);
       }
     }
     return close;

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=1557783&r1=1557782&r2=1557783&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 Mon Jan 13 17:34:31 2014
@@ -134,14 +134,17 @@ public class SolrIndexWriter extends Ind
           // don't allow interruption
           continue;
         } catch (Throwable t) {
+          if (t instanceof OutOfMemoryError) {
+            throw (OutOfMemoryError) t;
+          }
           log.error("Error closing IndexWriter, trying rollback", t);
           super.rollback();
         }
         if (IndexWriter.isLocked(directory)) {
           try {
             IndexWriter.unlock(directory);
-          } catch (Throwable t) {
-            log.error("Coud not unlock directory after seemingly failed IndexWriter#close()", t);
+          } catch (Exception e) {
+            log.error("Coud not unlock directory after seemingly failed IndexWriter#close()", e);
           }
         }
         break;

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/UpdateLog.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/UpdateLog.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/UpdateLog.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/UpdateLog.java Mon Jan 13 17:34:31 2014
@@ -421,7 +421,7 @@ public class UpdateLog implements Plugin
         try {
           RefCounted<SolrIndexSearcher> holder = uhandler.core.openNewSearcher(true, true);
           holder.decref();
-        } catch (Throwable e) {
+        } catch (Exception e) {
           SolrException.log(log, "Error opening realtime searcher for deleteByQuery", e);
         }
 
@@ -491,7 +491,7 @@ public class UpdateLog implements Plugin
         try {
           RefCounted<SolrIndexSearcher> holder = uhandler.core.openNewSearcher(true, true);
           holder.decref();
-        } catch (Throwable e) {
+        } catch (Exception e) {
           SolrException.log(log, "Error opening realtime searcher for deleteByQuery", e);
         }
 
@@ -512,7 +512,7 @@ public class UpdateLog implements Plugin
       try {
         RefCounted<SolrIndexSearcher> holder = uhandler.core.openNewSearcher(true, true);
         holder.decref();
-      } catch (Throwable e) {
+      } catch (Exception e) {
         SolrException.log(log, "Error opening realtime searcher for deleteByQuery", e);
       }
 
@@ -859,7 +859,7 @@ public class UpdateLog implements Plugin
     synchronized (this) {
       try {
         ExecutorUtil.shutdownNowAndAwaitTermination(recoveryExecutor);
-      } catch (Throwable e) {
+      } catch (Exception e) {
         SolrException.log(log, e);
       }
 
@@ -1222,7 +1222,7 @@ public class UpdateLog implements Plugin
           recoveryInfo.errors++;
           SolrException.log(log, e);
         }
-      } catch (Throwable e) {
+      } catch (Exception e) {
         recoveryInfo.errors++;
         SolrException.log(log, e);
       } finally {
@@ -1295,7 +1295,7 @@ public class UpdateLog implements Plugin
             SolrException.log(log,e);
           } catch (IOException e) {
             SolrException.log(log,e);
-          } catch (Throwable e) {
+          } catch (Exception e) {
             SolrException.log(log,e);
           }
 
@@ -1381,7 +1381,7 @@ public class UpdateLog implements Plugin
             recoveryInfo.errors++;
             loglog.warn("REYPLAY_ERR: IOException reading log", ex);
             // could be caused by an incomplete flush if recovering from log
-          } catch (Throwable ex) {
+          } catch (Exception ex) {
             recoveryInfo.errors++;
             loglog.warn("REPLAY_ERR: Exception replaying log", ex);
             // something wrong with the request?

Modified: lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/java/org/apache/solr/update/UpdateShardHandler.java Mon Jan 13 17:34:31 2014
@@ -73,7 +73,7 @@ public class UpdateShardHandler {
   public void close() {
     try {
       ExecutorUtil.shutdownAndAwaitTermination(updateExecutor);
-    } catch (Throwable e) {
+    } catch (Exception e) {
       SolrException.log(log, e);
     } finally {
       clientConnectionManager.shutdown();

Modified: lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyShardSplitTest.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyShardSplitTest.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyShardSplitTest.java (original)
+++ lucene/dev/branches/branch_4x/solr/core/src/test/org/apache/solr/cloud/ChaosMonkeyShardSplitTest.java Mon Jan 13 17:34:31 2014
@@ -184,23 +184,23 @@ public class ChaosMonkeyShardSplitTest e
               Thread.sleep(800);
               overseerClient.close();
               overseerClient = electNewOverseer(zkAddress);
-            } catch (Throwable e) {
+            } catch (Exception e) {
               // e.printStackTrace();
             }
           }
           try {
             Thread.sleep(100);
-          } catch (Throwable e) {
+          } catch (Exception e) {
             // e.printStackTrace();
           }
         }
-      } catch (Throwable t) {
+      } catch (Exception t) {
         // ignore
       } finally {
         if (overseerClient != null) {
           try {
             overseerClient.close();
-          } catch (Throwable t) {
+          } catch (Exception t) {
             // ignore
           }
         }

Modified: lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrServer.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrServer.java (original)
+++ lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/ConcurrentUpdateSolrServer.java Mon Jan 13 17:34:31 2014
@@ -250,6 +250,9 @@ public class ConcurrentUpdateSolrServer 
           }
         }
       } catch (Throwable e) {
+        if (e instanceof OutOfMemoryError) {
+          throw (OutOfMemoryError) e;
+        }
         handleError(e);
       } finally {
 

Modified: lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrServer.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrServer.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrServer.java (original)
+++ lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/client/solrj/impl/HttpSolrServer.java Mon Jan 13 17:34:31 2014
@@ -510,9 +510,12 @@ public class HttpSolrServer extends Solr
       if (respBody != null && shouldClose) {
         try {
           respBody.close();
-        } catch (Throwable t) {} // ignore
-        if (!success) {
-          method.abort();
+        } catch (IOException e) {
+          log.error("", e);
+        } finally {
+          if (!success) {
+            method.abort();
+          }
         }
       }
     }

Modified: lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java (original)
+++ lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ConnectionManager.java Mon Jan 13 17:34:31 2014
@@ -146,7 +146,7 @@ public class ConnectionManager implement
                   Thread.currentThread().interrupt();
                   // we must have been asked to stop
                   throw new RuntimeException(e);
-                } catch (Throwable t) {
+                } catch (Exception t) {
                   closeKeeper(keeper);
                   throw new RuntimeException(t);
                 }

Modified: lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java (original)
+++ lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/SolrZkClient.java Mon Jan 13 17:34:31 2014
@@ -123,7 +123,7 @@ public class SolrZkClient {
               }
             }
           });
-    } catch (Throwable e) {
+    } catch (Exception e) {
       connManager.close();
       if (keeper != null) {
         try {
@@ -137,7 +137,7 @@ public class SolrZkClient {
     
     try {
       connManager.waitForConnected(clientConnectTimeout);
-    } catch (Throwable e) {
+    } catch (Exception e) {
       connManager.close();
       try {
         keeper.close();

Modified: lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java
URL: http://svn.apache.org/viewvc/lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java?rev=1557783&r1=1557782&r2=1557783&view=diff
==============================================================================
--- lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java (original)
+++ lucene/dev/branches/branch_4x/solr/solrj/src/java/org/apache/solr/common/cloud/ZkClientConnectionStrategy.java Mon Jan 13 17:34:31 2014
@@ -43,8 +43,8 @@ public abstract class ZkClientConnection
     for (DisconnectedListener listener : disconnectedListeners) {
       try {
         listener.disconnected();
-      } catch (Throwable t) {
-        SolrException.log(log, "", t);
+      } catch (Exception e) {
+        SolrException.log(log, "", e);
       }
     }
   }
@@ -53,8 +53,8 @@ public abstract class ZkClientConnection
     for (ConnectedListener listener : connectedListeners) {
       try {
         listener.connected();
-      } catch (Throwable t) {
-        SolrException.log(log, "", t);
+      } catch (Exception e) {
+        SolrException.log(log, "", e);
       }
     }
   }