You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@lucene.apache.org by dw...@apache.org on 2017/04/28 10:17:35 UTC

lucene-solr:master: LUCENE-7796: Make IOUtils.reThrow idiom declare Error return type so callers may use it in a way that compiler knows subsequent code is unreachable. reThrow is now deprecated in favor of IOUtils.rethrowAlways.

Repository: lucene-solr
Updated Branches:
  refs/heads/master 816b806d8 -> e52d86099


LUCENE-7796: Make IOUtils.reThrow idiom declare Error return type so
callers may use it in a way that compiler knows subsequent code is
unreachable. reThrow is now deprecated in favor of IOUtils.rethrowAlways.


Project: http://git-wip-us.apache.org/repos/asf/lucene-solr/repo
Commit: http://git-wip-us.apache.org/repos/asf/lucene-solr/commit/e52d8609
Tree: http://git-wip-us.apache.org/repos/asf/lucene-solr/tree/e52d8609
Diff: http://git-wip-us.apache.org/repos/asf/lucene-solr/diff/e52d8609

Branch: refs/heads/master
Commit: e52d8609949369aebae0297c855936138141c557
Parents: 816b806
Author: Dawid Weiss <dw...@apache.org>
Authored: Fri Apr 28 10:58:55 2017 +0200
Committer: Dawid Weiss <dw...@apache.org>
Committed: Fri Apr 28 12:17:21 2017 +0200

----------------------------------------------------------------------
 lucene/CHANGES.txt                              |  6 ++
 .../codecs/simpletext/SimpleTextBKDWriter.java  | 18 ++--
 .../org/apache/lucene/codecs/CodecUtil.java     |  2 +-
 .../lucene/index/BufferedUpdatesStream.java     |  5 +-
 .../org/apache/lucene/index/CheckIndex.java     | 32 +++----
 .../apache/lucene/index/IndexFileDeleter.java   | 12 +--
 .../org/apache/lucene/index/IndexReader.java    |  4 +-
 .../org/apache/lucene/index/IndexWriter.java    | 21 +++--
 .../apache/lucene/index/SegmentCoreReaders.java |  5 +-
 .../apache/lucene/index/SegmentDocValues.java   |  3 +-
 .../org/apache/lucene/index/SegmentReader.java  |  5 +-
 .../lucene/index/StandardDirectoryReader.java   |  9 +-
 .../java/org/apache/lucene/util/IOUtils.java    | 92 +++++++++++++++-----
 .../org/apache/lucene/util/OfflineSorter.java   |  6 +-
 .../org/apache/lucene/util/bkd/BKDWriter.java   | 19 ++--
 .../index/TestDemoParallelLeafReader.java       | 12 ++-
 .../lucene/replicator/http/HttpClientBase.java  |  7 +-
 .../apache/lucene/replicator/nrt/CopyJob.java   |  2 +-
 .../lucene/replicator/nrt/ReplicaNode.java      |  2 +-
 .../lucene/index/OwnCacheKeyMultiReader.java    |  5 +-
 .../org/apache/lucene/mockfile/VerboseFS.java   | 14 ++-
 .../lucene/store/MockDirectoryWrapper.java      |  2 +-
 22 files changed, 186 insertions(+), 97 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/CHANGES.txt
----------------------------------------------------------------------
diff --git a/lucene/CHANGES.txt b/lucene/CHANGES.txt
index c5022f1..3c361e3 100644
--- a/lucene/CHANGES.txt
+++ b/lucene/CHANGES.txt
@@ -128,6 +128,12 @@ Optimizations
 
 Other
 
+* LUCENE-7796: Make IOUtils.reThrow idiom declare Error return type so 
+  callers may use it in a way that compiler knows subsequent code is 
+  unreachable. reThrow is now deprecated in favor of IOUtils.rethrowAlways
+  with a slightly different semantics (see javadoc). (Hossman, Robert Muir, 
+  Dawid Weiss)
+
 * LUCENE-7754: Inner classes should be static whenever possible.
   (Daniel Jelinski via Adrien Grand)
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java
----------------------------------------------------------------------
diff --git a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java
index dd89537..fdb1df6 100644
--- a/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java
+++ b/lucene/codecs/src/java/org/apache/lucene/codecs/simpletext/SimpleTextBKDWriter.java
@@ -1170,7 +1170,8 @@ final class SimpleTextBKDWriter implements Closeable {
 
   /** Called on exception, to check whether the checksum is also corrupt in this source, and add that
    *  information (checksum matched or didn't) as a suppressed exception. */
-  private void verifyChecksum(Throwable priorException, PointWriter writer) throws IOException {
+  private Error verifyChecksum(Throwable priorException, PointWriter writer) throws IOException {
+    assert priorException != null;
     // TODO: we could improve this, to always validate checksum as we recurse, if we shared left and
     // right reader after recursing to children, and possibly within recursed children,
     // since all together they make a single pass through the file.  But this is a sizable re-org,
@@ -1181,10 +1182,10 @@ final class SimpleTextBKDWriter implements Closeable {
       try (ChecksumIndexInput in = tempDir.openChecksumInput(tempFileName, IOContext.READONCE)) {
         CodecUtil.checkFooter(in, priorException);
       }
-    } else {
-      // We are reading from heap; nothing to add:
-      IOUtils.reThrow(priorException);
     }
+
+    // We are reading from heap; nothing to add:
+    throw IOUtils.rethrowAlways(priorException);
   }
 
   /** Marks bits for the ords (points) that belong in the right sub tree (those docs that have values >= the splitValue). */
@@ -1206,7 +1207,7 @@ final class SimpleTextBKDWriter implements Closeable {
         reader.markOrds(rightCount-1, ordBitSet);
       }
     } catch (Throwable t) {
-      verifyChecksum(t, source.writer);
+      throw verifyChecksum(t, source.writer);
     }
 
     return scratch1;
@@ -1255,10 +1256,7 @@ final class SimpleTextBKDWriter implements Closeable {
       }
       return new PathSlice(writer, 0, count);
     } catch (Throwable t) {
-      verifyChecksum(t, source.writer);
-
-      // Dead code but javac disagrees:
-      return null;
+      throw verifyChecksum(t, source.writer);
     }
   }
 
@@ -1564,7 +1562,7 @@ final class SimpleTextBKDWriter implements Closeable {
           leftSlices[dim] = new PathSlice(leftPointWriter, 0, leftCount);
           rightSlices[dim] = new PathSlice(rightPointWriter, 0, rightCount);
         } catch (Throwable t) {
-          verifyChecksum(t, slices[dim].writer);
+          throw verifyChecksum(t, slices[dim].writer);
         }
       }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java b/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
index da487d0..a625b47 100644
--- a/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
+++ b/lucene/core/src/java/org/apache/lucene/codecs/CodecUtil.java
@@ -467,7 +467,7 @@ public final class CodecUtil {
         // catch-all for things that shouldn't go wrong (e.g. OOM during readInt) but could...
         priorException.addSuppressed(new CorruptIndexException("checksum status indeterminate: unexpected exception", in, t));
       }
-      IOUtils.reThrow(priorException);
+      throw IOUtils.rethrowAlways(priorException);
     }
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java b/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java
index e5aae4f..51933ac 100644
--- a/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java
+++ b/lucene/core/src/java/org/apache/lucene/index/BufferedUpdatesStream.java
@@ -463,8 +463,9 @@ class BufferedUpdatesStream implements Accountable {
     }
 
     if (success) {
-      // Does nothing if firstExc is null:
-      IOUtils.reThrow(firstExc);
+      if (firstExc != null) {
+        throw IOUtils.rethrowAlways(firstExc);
+      }
     }
 
     if (infoStream.isEnabled("BD")) {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
index c7ad0f4..e448d81 100644
--- a/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
+++ b/lucene/core/src/java/org/apache/lucene/index/CheckIndex.java
@@ -529,7 +529,7 @@ public final class CheckIndex implements Closeable {
       sis = SegmentInfos.readCommit(dir, lastSegmentsFile);
     } catch (Throwable t) {
       if (failFast) {
-        IOUtils.reThrow(t);
+        throw IOUtils.rethrowAlways(t);
       }
       msg(infoStream, "ERROR: could not read any segments file in directory");
       result.missingSegments = true;
@@ -565,11 +565,12 @@ public final class CheckIndex implements Closeable {
       input = dir.openInput(segmentsFileName, IOContext.READONCE);
     } catch (Throwable t) {
       if (failFast) {
-        IOUtils.reThrow(t);
+        throw IOUtils.rethrowAlways(t);
       }
       msg(infoStream, "ERROR: could not open segments file in directory");
-      if (infoStream != null)
+      if (infoStream != null) {
         t.printStackTrace(infoStream);
+      }
       result.cantOpenSegments = true;
       return result;
     }
@@ -577,11 +578,12 @@ public final class CheckIndex implements Closeable {
       /*int format =*/ input.readInt();
     } catch (Throwable t) {
       if (failFast) {
-        IOUtils.reThrow(t);
+        throw IOUtils.rethrowAlways(t);
       }
       msg(infoStream, "ERROR: could not read segment file version in directory");
-      if (infoStream != null)
+      if (infoStream != null) {
         t.printStackTrace(infoStream);
+      }
       result.missingSegmentVersion = true;
       return result;
     } finally {
@@ -789,7 +791,7 @@ public final class CheckIndex implements Closeable {
 
       } catch (Throwable t) {
         if (failFast) {
-          IOUtils.reThrow(t);
+          throw IOUtils.rethrowAlways(t);
         }
         msg(infoStream, "FAILED");
         String comment;
@@ -883,7 +885,7 @@ public final class CheckIndex implements Closeable {
         msg(infoStream, String.format(Locale.ROOT, "OK [took %.3f sec]", nsToSec(System.nanoTime()-startNS)));
       } catch (Throwable e) {
         if (failFast) {
-          IOUtils.reThrow(e);
+          throw IOUtils.rethrowAlways(e);
         }
         msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
         status.error = e;
@@ -941,7 +943,7 @@ public final class CheckIndex implements Closeable {
       
     } catch (Throwable e) {
       if (failFast) {
-        IOUtils.reThrow(e);
+        throw IOUtils.rethrowAlways(e);
       }
       msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
       status.error = e;
@@ -974,7 +976,7 @@ public final class CheckIndex implements Closeable {
       status.totFields = fieldInfos.size();
     } catch (Throwable e) {
       if (failFast) {
-        IOUtils.reThrow(e);
+        throw IOUtils.rethrowAlways(e);
       }
       msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
       status.error = e;
@@ -1013,7 +1015,7 @@ public final class CheckIndex implements Closeable {
       msg(infoStream, String.format(Locale.ROOT, "OK [%d fields] [took %.3f sec]", status.totFields, nsToSec(System.nanoTime()-startNS)));
     } catch (Throwable e) {
       if (failFast) {
-        IOUtils.reThrow(e);
+        throw IOUtils.rethrowAlways(e);
       }
       msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
       status.error = e;
@@ -1769,7 +1771,7 @@ public final class CheckIndex implements Closeable {
       status = checkFields(fields, reader.getLiveDocs(), maxDoc, fieldInfos, true, false, infoStream, verbose, version);
     } catch (Throwable e) {
       if (failFast) {
-        IOUtils.reThrow(e);
+        throw IOUtils.rethrowAlways(e);
       }
       msg(infoStream, "ERROR: " + e);
       status = new Status.TermIndexStatus();
@@ -1845,7 +1847,7 @@ public final class CheckIndex implements Closeable {
 
     } catch (Throwable e) {
       if (failFast) {
-        IOUtils.reThrow(e);
+        throw IOUtils.rethrowAlways(e);
       }
       msg(infoStream, "ERROR: " + e);
       status.error = e;
@@ -2079,7 +2081,7 @@ public final class CheckIndex implements Closeable {
                                     nsToSec(System.nanoTime() - startNS)));
     } catch (Throwable e) {
       if (failFast) {
-        IOUtils.reThrow(e);
+        throw IOUtils.rethrowAlways(e);
       }
       msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
       status.error = e;
@@ -2126,7 +2128,7 @@ public final class CheckIndex implements Closeable {
                                     nsToSec(System.nanoTime()-startNS)));
     } catch (Throwable e) {
       if (failFast) {
-        IOUtils.reThrow(e);
+        throw IOUtils.rethrowAlways(e);
       }
       msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
       status.error = e;
@@ -2567,7 +2569,7 @@ public final class CheckIndex implements Closeable {
                                     status.totVectors, vectorAvg, nsToSec(System.nanoTime() - startNS)));
     } catch (Throwable e) {
       if (failFast) {
-        IOUtils.reThrow(e);
+        throw IOUtils.rethrowAlways(e);
       }
       msg(infoStream, "ERROR [" + String.valueOf(e.getMessage()) + "]");
       status.error = e;

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java b/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
index 38d1688..f7f196d 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexFileDeleter.java
@@ -364,7 +364,7 @@ final class IndexFileDeleter implements Closeable {
    * Remove the CommitPoints in the commitsToDelete List by
    * DecRef'ing all files from each SegmentInfos.
    */
-  private void deleteCommits() {
+  private void deleteCommits() throws IOException {
 
     int size = commitsToDelete.size();
 
@@ -388,8 +388,9 @@ final class IndexFileDeleter implements Closeable {
       }
       commitsToDelete.clear();
 
-      // NOTE: does nothing if firstThrowable is null
-      IOUtils.reThrowUnchecked(firstThrowable);
+      if (firstThrowable != null) {
+        throw IOUtils.rethrowAlways(firstThrowable);
+      }
 
       // Now compact commits to remove deleted ones (preserving the sort):
       size = commits.size();
@@ -599,8 +600,9 @@ final class IndexFileDeleter implements Closeable {
       }
     }
 
-    // NOTE: does nothing if firstThrowable is null
-    IOUtils.reThrow(firstThrowable);
+    if (firstThrowable != null) {
+      throw IOUtils.rethrowAlways(firstThrowable);
+    }
   }
 
   /** Decrefs all provided files, ignoring any exceptions hit; call this if

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/core/src/java/org/apache/lucene/index/IndexReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexReader.java b/lucene/core/src/java/org/apache/lucene/index/IndexReader.java
index eb3a6db..3efd587 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexReader.java
@@ -144,7 +144,9 @@ public abstract class IndexReader implements Closeable {
   // overridden by StandardDirectoryReader and SegmentReader
   void notifyReaderClosedListeners(Throwable th) throws IOException {
     // nothing to notify in the base impl, just rethrow
-    IOUtils.reThrow(th);
+    if (th != null) {
+      throw IOUtils.rethrowAlways(th);
+    }
   }
 
   private void reportCloseToParentReaders() {

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
index 9a29150..14fbbae 100644
--- a/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/index/IndexWriter.java
@@ -611,7 +611,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
           }
         } catch (Throwable t) {
           if (doSave) {
-            IOUtils.reThrow(t);
+            throw IOUtils.rethrowAlways(t);
           } else if (priorE == null) {
             priorE = t;
           }
@@ -631,14 +631,16 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
           rld.dropReaders();
         } catch (Throwable t) {
           if (doSave) {
-            IOUtils.reThrow(t);
+            throw IOUtils.rethrowAlways(t);
           } else if (priorE == null) {
             priorE = t;
           }
         }
       }
       assert readerMap.size() == 0;
-      IOUtils.reThrow(priorE);
+      if (priorE != null) {
+        throw IOUtils.rethrowAlways(priorE);
+      }
     }
 
     /**
@@ -3330,7 +3332,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
       if (commitCompleted) {
         tragicEvent(t, "finishCommit");
       } else {
-        IOUtils.reThrow(t);
+        throw IOUtils.rethrowAlways(t);
       }
     }
 
@@ -3898,7 +3900,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
         throw (MergePolicy.MergeAbortedException) t;
       }
     } else {
-      IOUtils.reThrow(t);
+      assert t != null;
+      throw IOUtils.rethrowAlways(t);
     }
   }
 
@@ -4238,8 +4241,8 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
     }
     
     // If any error occurred, throw it.
-    if (!suppressExceptions) {
-      IOUtils.reThrow(th);
+    if (!suppressExceptions && th != null) {
+      throw IOUtils.rethrowAlways(th);
     }
   }
 
@@ -4815,7 +4818,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
       // It's possible you could have a really bad day
       if (this.tragedy != null) {
         // Another thread is already dealing / has dealt with the tragedy:
-        IOUtils.reThrow(tragedy);
+        throw IOUtils.rethrowAlways(tragedy);
       }
 
       this.tragedy = tragedy;
@@ -4826,7 +4829,7 @@ public class IndexWriter implements Closeable, TwoPhaseCommit, Accountable {
       rollbackInternal();
     }
 
-    IOUtils.reThrow(tragedy);
+    throw IOUtils.rethrowAlways(tragedy);
   }
 
   /** If this {@code IndexWriter} was closed as a side-effect of a tragic exception,

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
index 99e503b..ce2d448 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentCoreReaders.java
@@ -210,7 +210,10 @@ final class SegmentCoreReaders {
           }
         }
       }
-      IOUtils.reThrow(th);
+      
+      if (th != null) {
+        throw IOUtils.rethrowAlways(th);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java b/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java
index 4838799..568baa1 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentDocValues.java
@@ -90,8 +90,9 @@ final class SegmentDocValues {
         }
       }
     }
+
     if (t != null) {
-      IOUtils.reThrow(t);
+      throw IOUtils.rethrowAlways(t);
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java b/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
index 930340c..c8235d5 100644
--- a/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/SegmentReader.java
@@ -303,7 +303,10 @@ public final class SegmentReader extends CodecReader {
           }
         }
       }
-      IOUtils.reThrow(th);
+      
+      if (th != null) {
+        IOUtils.rethrowAlways(th);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java b/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
index bedf17e..a605c55 100644
--- a/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
+++ b/lucene/core/src/java/org/apache/lucene/index/StandardDirectoryReader.java
@@ -391,7 +391,9 @@ public final class StandardDirectoryReader extends DirectoryReader {
     }
 
     // throw the first exception
-    IOUtils.reThrow(firstExc);
+    if (firstExc != null) {
+      throw IOUtils.rethrowAlways(firstExc);
+    }
   }
 
   @Override
@@ -504,7 +506,10 @@ public final class StandardDirectoryReader extends DirectoryReader {
           }
         }
       }
-      IOUtils.reThrow(th);
+      
+      if (th != null) {
+        throw IOUtils.rethrowAlways(th);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/IOUtils.java b/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
index ce8884c..d48474e 100644
--- a/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
+++ b/lucene/core/src/java/org/apache/lucene/util/IOUtils.java
@@ -96,7 +96,9 @@ public final class IOUtils {
       }
     }
 
-    reThrow(th);
+    if (th != null) {
+      throw rethrowAlways(th);
+    }
   }
 
   /**
@@ -229,7 +231,9 @@ public final class IOUtils {
       }
     }
 
-    reThrow(th);
+    if (th != null) {
+      throw rethrowAlways(th);
+    }
   }
 
   public static void deleteFiles(Directory dir, String... files) throws IOException {
@@ -300,7 +304,9 @@ public final class IOUtils {
       }
     }
 
-    reThrow(th);
+    if (th != null) {
+      throw rethrowAlways(th);
+    }
   }
   
   /**
@@ -376,37 +382,83 @@ public final class IOUtils {
   }
 
   /**
-   * Simple utility method that takes a previously caught
-   * {@code Throwable} and rethrows either {@code
-   * IOException} or an unchecked exception.  If the
-   * argument is null then this method does nothing.
+   * This utility method takes a previously caught (non-null)
+   * {@code Throwable} and rethrows either the original argument
+   * if it was a subclass of the {@code IOException} or an 
+   * {@code RuntimeException} with the cause set to the argument.
+   * 
+   * <p>This method <strong>never returns any value</strong>, even though it declares
+   * a return value of type {@link Error}. The return value declaration
+   * is very useful to let the compiler know that the code path following
+   * the invocation of this method is unreachable. So in most cases the
+   * invocation of this method will be guarded by an {@code if} and
+   * used together with a {@code throw} statement, as in:
+   * </p>
+   * <pre>{@code
+   *   if (t != null) throw IOUtils.rethrowAlways(t)
+   * }
+   * </pre>
+   * 
+   * @param th The throwable to rethrow, <strong>must not be null</strong>.
+   * @return This method always results in an exception, it never returns any value. 
+   *         See method documentation for detailsa and usage example.
+   * @throws IOException if the argument was an instance of IOException
+   * @throws RuntimeException with the {@link RuntimeException#getCause()} set
+   *         to the argument, if it was not an instance of IOException. 
    */
+  public static Error rethrowAlways(Throwable th) throws IOException, RuntimeException {
+    if (th == null) {
+      throw new AssertionError("rethrow argument must not be null.");
+    }
+
+    if (th instanceof IOException) {
+      throw (IOException) th;
+    }
+
+    if (th instanceof RuntimeException) {
+      throw (RuntimeException) th;
+    }
+
+    if (th instanceof Error) {
+      throw (Error) th;
+    }
+
+    throw new RuntimeException(th);
+  }
+
+  /**
+   * Rethrows the argument as {@code IOException} or {@code RuntimeException} 
+   * if it's not null.
+   * 
+   * @deprecated This method is deprecated in favor of {@link #rethrowAlways}. Code should
+   * be updated to {@link #rethrowAlways} and guarded with an additional null-argument check
+   * (because {@link #rethrowAlways} is not accepting null arguments). 
+   */
+  @Deprecated
   public static void reThrow(Throwable th) throws IOException {
     if (th != null) {
-      if (th instanceof IOException) {
-        throw (IOException) th;
-      }
-      reThrowUnchecked(th);
+      throw rethrowAlways(th);
     }
   }
-
+  
   /**
-   * Simple utility method that takes a previously caught
-   * {@code Throwable} and rethrows it as an unchecked exception.
-   * If the argument is null then this method does nothing.
+   * @deprecated This method is deprecated in favor of {@link #rethrowAlways}. Code should
+   * be updated to {@link #rethrowAlways} and guarded with an additional null-argument check
+   * (because {@link #rethrowAlways} is not accepting null arguments). 
    */
+  @Deprecated
   public static void reThrowUnchecked(Throwable th) {
     if (th != null) {
-      if (th instanceof RuntimeException) {
-        throw (RuntimeException) th;
-      }
       if (th instanceof Error) {
         throw (Error) th;
       }
+      if (th instanceof RuntimeException) {
+        throw (RuntimeException) th;
+      }
       throw new RuntimeException(th);
-    }
+    }    
   }
-
+  
   /**
    * Ensure that any writes to the given file is written to the storage device that contains it.
    * @param fileToSync the file to fsync

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java b/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
index 0bd3066..7c6ed8d 100644
--- a/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/OfflineSorter.java
@@ -633,9 +633,9 @@ public class OfflineSorter {
     } catch (InterruptedException ie) {
       throw new ThreadInterruptedException(ie);
     } catch (ExecutionException ee) {
-      IOUtils.reThrow(ee.getCause());
-      // oh so soon to go away:
-      return null;
+      // Theoretically cause can be null; guard against that.
+      Throwable cause = ee.getCause();
+      throw IOUtils.rethrowAlways(cause != null ? cause : ee);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
index 1575a5b..5b2f43e 100644
--- a/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
+++ b/lucene/core/src/java/org/apache/lucene/util/bkd/BKDWriter.java
@@ -1362,7 +1362,9 @@ public class BKDWriter implements Closeable {
 
   /** Called on exception, to check whether the checksum is also corrupt in this source, and add that
    *  information (checksum matched or didn't) as a suppressed exception. */
-  private void verifyChecksum(Throwable priorException, PointWriter writer) throws IOException {
+  private Error verifyChecksum(Throwable priorException, PointWriter writer) throws IOException {
+    assert priorException != null;
+
     // TODO: we could improve this, to always validate checksum as we recurse, if we shared left and
     // right reader after recursing to children, and possibly within recursed children,
     // since all together they make a single pass through the file.  But this is a sizable re-org,
@@ -1373,10 +1375,10 @@ public class BKDWriter implements Closeable {
       try (ChecksumIndexInput in = tempDir.openChecksumInput(tempFileName, IOContext.READONCE)) {
         CodecUtil.checkFooter(in, priorException);
       }
-    } else {
-      // We are reading from heap; nothing to add:
-      IOUtils.reThrow(priorException);
     }
+    
+    // We are reading from heap; nothing to add:
+    throw IOUtils.rethrowAlways(priorException);
   }
 
   /** Marks bits for the ords (points) that belong in the right sub tree (those docs that have values >= the splitValue). */
@@ -1398,7 +1400,7 @@ public class BKDWriter implements Closeable {
         reader.markOrds(rightCount-1, ordBitSet);
       }
     } catch (Throwable t) {
-      verifyChecksum(t, source.writer);
+      throw verifyChecksum(t, source.writer);
     }
 
     return scratch1;
@@ -1469,10 +1471,7 @@ public class BKDWriter implements Closeable {
       }
       return new PathSlice(writer, 0, count);
     } catch (Throwable t) {
-      verifyChecksum(t, source.writer);
-
-      // Dead code but javac disagrees:
-      return null;
+      throw verifyChecksum(t, source.writer);
     }
   }
 
@@ -1797,7 +1796,7 @@ public class BKDWriter implements Closeable {
           leftSlices[dim] = new PathSlice(leftPointWriter, 0, leftCount);
           rightSlices[dim] = new PathSlice(rightPointWriter, 0, rightCount);
         } catch (Throwable t) {
-          verifyChecksum(t, slices[dim].writer);
+          throw verifyChecksum(t, slices[dim].writer);
         }
       }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
----------------------------------------------------------------------
diff --git a/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java b/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
index 7b7ec99..2f0121a 100644
--- a/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
+++ b/lucene/core/src/test/org/apache/lucene/index/TestDemoParallelLeafReader.java
@@ -237,8 +237,11 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
             firstExc = t;
           }
         }
+        
         // throw the first exception
-        IOUtils.reThrow(firstExc);
+        if (firstExc != null) {
+          throw IOUtils.rethrowAlways(firstExc);
+        }
       }
 
       @Override
@@ -549,10 +552,11 @@ public class TestDemoParallelLeafReader extends LuceneTestCase {
             }
           }
 
-          // If any error occured, throw it.
-          IOUtils.reThrow(th);
+          if (th != null) {
+            throw IOUtils.rethrowAlways(th);
+          }
         }
-    
+
         @Override
         public void setMergeInfo(SegmentCommitInfo info) {
           // Record that this merged segment is current as of this schemaGen:

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/replicator/src/java/org/apache/lucene/replicator/http/HttpClientBase.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/http/HttpClientBase.java b/lucene/replicator/src/java/org/apache/lucene/replicator/http/HttpClientBase.java
index 920f1de..cbce590 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/http/HttpClientBase.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/http/HttpClientBase.java
@@ -127,12 +127,13 @@ public abstract class HttpClientBase implements Closeable {
     Throwable t;
     try {
       t = (Throwable) in.readObject();
+      assert t != null;
     } catch (Throwable th) { 
       throw new RuntimeException("Failed to read exception object: " + statusLine, th);
     } finally {
       in.close();
     }
-    IOUtils.reThrow(t);
+    throw IOUtils.rethrowAlways(t);
   }
   
   /**
@@ -260,9 +261,7 @@ public abstract class HttpClientBase implements Closeable {
         }
       }
     }
-    assert th != null; // extra safety - if we get here, it means the callable failed
-    IOUtils.reThrow(th);
-    return null; // silly, if we're here, IOUtils.reThrow always throws an exception 
+    throw IOUtils.rethrowAlways(th); 
   }
   
   @Override

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java
index 2bdcf1d..74e7c88 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/CopyJob.java
@@ -96,7 +96,7 @@ public abstract class CopyJob implements Comparable<CopyJob> {
       } catch (Throwable t) {
         dest.message("xfer: exc during transferAndCancel");
         cancel("exc during transferAndCancel", t);
-        IOUtils.reThrow(t);
+        throw IOUtils.rethrowAlways(t);
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
----------------------------------------------------------------------
diff --git a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
index 853446b..2567fd9 100644
--- a/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
+++ b/lucene/replicator/src/java/org/apache/lucene/replicator/nrt/ReplicaNode.java
@@ -312,7 +312,7 @@ public abstract class ReplicaNode extends Node {
       } else {
         dir.close();
       }
-      IOUtils.reThrow(t);
+      throw IOUtils.rethrowAlways(t);
     }
   }
   

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/test-framework/src/java/org/apache/lucene/index/OwnCacheKeyMultiReader.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/index/OwnCacheKeyMultiReader.java b/lucene/test-framework/src/java/org/apache/lucene/index/OwnCacheKeyMultiReader.java
index a412ed8..1840472 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/index/OwnCacheKeyMultiReader.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/index/OwnCacheKeyMultiReader.java
@@ -70,7 +70,10 @@ public final class OwnCacheKeyMultiReader extends MultiReader {
           }
         }
       }
-      IOUtils.reThrow(th);
+      
+      if (th != null) {
+        throw IOUtils.rethrowAlways(th);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/test-framework/src/java/org/apache/lucene/mockfile/VerboseFS.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/mockfile/VerboseFS.java b/lucene/test-framework/src/java/org/apache/lucene/mockfile/VerboseFS.java
index 5ca3493..7056277 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/mockfile/VerboseFS.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/mockfile/VerboseFS.java
@@ -65,7 +65,7 @@ public class VerboseFS extends FilterFileSystemProvider {
       if (infoStream.isEnabled("FS")) {
         infoStream.message("FS", text + " (FAILED: " + exception + ")");
       }
-      IOUtils.reThrow(exception);
+      throw IOUtils.rethrowAlways(exception);
     }
   }
   
@@ -164,7 +164,9 @@ public class VerboseFS extends FilterFileSystemProvider {
       if (containsDestructive(options)) {
         sop("newFileChannel" + options + ": " + path(path), exception);
       } else {
-        IOUtils.reThrow(exception);
+        if (exception != null) {
+          throw IOUtils.rethrowAlways(exception);
+        }
       }
     }
     throw new AssertionError();
@@ -181,7 +183,9 @@ public class VerboseFS extends FilterFileSystemProvider {
       if (containsDestructive(options)) {
         sop("newAsynchronousFileChannel" + options + ": " + path(path), exception);
       } else {
-        IOUtils.reThrow(exception);
+        if (exception != null) {
+          throw IOUtils.rethrowAlways(exception);
+        }
       }
     }
     throw new AssertionError();
@@ -198,7 +202,9 @@ public class VerboseFS extends FilterFileSystemProvider {
       if (containsDestructive(options)) {
         sop("newByteChannel" + options + ": " + path(path), exception);
       } else {
-        IOUtils.reThrow(exception);
+        if (exception != null) {
+          throw IOUtils.rethrowAlways(exception);
+        }
       }
     }
     throw new AssertionError();

http://git-wip-us.apache.org/repos/asf/lucene-solr/blob/e52d8609/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
----------------------------------------------------------------------
diff --git a/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java b/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
index d15476a..60f671c 100644
--- a/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
+++ b/lucene/test-framework/src/java/org/apache/lucene/store/MockDirectoryWrapper.java
@@ -1025,7 +1025,7 @@ public class MockDirectoryWrapper extends BaseDirectoryWrapper {
             System.out.println("MockDirectoryWrapper: throw exc");
             t.printStackTrace(System.out);
           }
-          IOUtils.reThrow(t);
+          throw IOUtils.rethrowAlways(t);
         }
       }
     }